From e7873550d0dfa229b5171f776826948d0b87e9c7 Mon Sep 17 00:00:00 2001 From: Joshua Harlow Date: Fri, 16 Oct 2015 10:44:16 -0700 Subject: [PATCH 01/74] Some executors are not async so update docstring to reflect that Change-Id: I84db5adf5af0372d521e05f7c4277e1fb570f881 --- oslo_messaging/_executors/impl_pooledexecutor.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/oslo_messaging/_executors/impl_pooledexecutor.py b/oslo_messaging/_executors/impl_pooledexecutor.py index c0837701c..9962370ec 100644 --- a/oslo_messaging/_executors/impl_pooledexecutor.py +++ b/oslo_messaging/_executors/impl_pooledexecutor.py @@ -33,7 +33,7 @@ _pool_opts = [ class PooledExecutor(base.ExecutorBase): - """A message executor which integrates with some async executor. + """A message executor which integrates with some executor. This will create a message thread that polls for messages from a dispatching thread and on reception of an incoming message places the From a3fa8ffec975bc56c306e92b763cdf902dd45613 Mon Sep 17 00:00:00 2001 From: OpenStack Proposal Bot Date: Mon, 19 Oct 2015 23:32:00 +0000 Subject: [PATCH 02/74] Updated from global requirements Change-Id: I882a029c98255087bee77ac54c0bf0538f53db33 --- requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/requirements.txt b/requirements.txt index f00a9dc37..9d82a670c 100644 --- a/requirements.txt +++ b/requirements.txt @@ -8,7 +8,7 @@ futurist>=0.1.2 # Apache-2.0 oslo.config>=2.3.0 # Apache-2.0 oslo.context>=0.2.0 # Apache-2.0 oslo.log>=1.8.0 # Apache-2.0 -oslo.utils>=2.4.0 # Apache-2.0 +oslo.utils!=2.6.0,>=2.4.0 # Apache-2.0 oslo.serialization>=1.4.0 # Apache-2.0 oslo.service>=0.10.0 # Apache-2.0 oslo.i18n>=1.5.0 # Apache-2.0 From aec50602d557a8b1b4130f9886991f18db596e54 Mon Sep 17 00:00:00 2001 From: Matthew Booth Date: Mon, 19 Oct 2015 10:31:50 +0100 Subject: [PATCH 03/74] Rename MessageHandlingServer._executor for readability MessageHandlingServer has both MessageHandlingServer.executor, which is the name of an executor type, and MessageHandlingServer._executor, which is an instance of that type. Ideally we would rename MessageHandlingServer.executor, but as this is referenced from outside the class we change _executor instead to _executor_obj. Change-Id: Id69ba7a0729cc66d266327dac2fd4eab50f2814c --- oslo_messaging/server.py | 20 ++++++++++---------- oslo_messaging/tests/rpc/test_server.py | 6 +++--- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/oslo_messaging/server.py b/oslo_messaging/server.py index 476e899cd..02bae191a 100644 --- a/oslo_messaging/server.py +++ b/oslo_messaging/server.py @@ -112,7 +112,7 @@ class MessageHandlingServer(service.ServiceBase): raise ExecutorLoadFailure(self.executor, ex) else: self._executor_cls = mgr.driver - self._executor = None + self._executor_obj = None self._running = False super(MessageHandlingServer, self).__init__() @@ -131,19 +131,19 @@ class MessageHandlingServer(service.ServiceBase): choose to dispatch messages in a new thread, coroutine or simply the current thread. """ - if self._executor is not None: + if self._executor_obj is not None: return with self._state_cond: - if self._executor is not None: + if self._executor_obj is not None: return try: listener = self.dispatcher._listen(self.transport) except driver_base.TransportDriverError as ex: raise ServerListenError(self.target, ex) self._running = True - self._executor = self._executor_cls(self.conf, listener, - self.dispatcher) - self._executor.start() + self._executor_obj = self._executor_cls(self.conf, listener, + self.dispatcher) + self._executor_obj.start() self._state_cond.notify_all() def stop(self): @@ -155,9 +155,9 @@ class MessageHandlingServer(service.ServiceBase): server are still in use. See 'wait' for more details. """ with self._state_cond: - if self._executor is not None: + if self._executor_obj is not None: self._running = False - self._executor.stop() + self._executor_obj.stop() self._state_cond.notify_all() def wait(self): @@ -190,8 +190,8 @@ class MessageHandlingServer(service.ServiceBase): " messages to finish processing, it has" " been %0.2f seconds and stop() still has" " not been called"), w.elapsed()) - executor = self._executor - self._executor = None + executor = self._executor_obj + self._executor_obj = None if executor is not None: # We are the lucky calling thread to wait on the executor to # actually finish. diff --git a/oslo_messaging/tests/rpc/test_server.py b/oslo_messaging/tests/rpc/test_server.py index b1f8961c5..93b5e9d69 100644 --- a/oslo_messaging/tests/rpc/test_server.py +++ b/oslo_messaging/tests/rpc/test_server.py @@ -120,14 +120,14 @@ class TestRPCServer(test_utils.BaseTestCase, ServerSetupMixin): server = oslo_messaging.get_rpc_server(transport, target, endpoints, serializer=serializer) # Mocking executor - server._executor = mock.Mock() + server._executor_obj = mock.Mock() # Here assigning executor's listener object to listener variable # before calling wait method, because in wait method we are # setting executor to None. - listener = server._executor.listener + listener = server._executor_obj.listener # call server wait method server.wait() - self.assertIsNone(server._executor) + self.assertIsNone(server._executor_obj) self.assertEqual(1, listener.cleanup.call_count) def test_no_target_server(self): From 9d74ee40c6080a8ee656e0783fc72287e0d55e4d Mon Sep 17 00:00:00 2001 From: Matthew Booth Date: Mon, 19 Oct 2015 10:40:15 +0100 Subject: [PATCH 04/74] Fix assumptions in test_server_wait_method test_server_wait_method was calling server.wait without having previously called server.start and server.stop. This happened to work because it also injected server._executor_obj. This is problematic, though, as it assumes internal details of the server and does not represent the calling contract of server.wait, which is that it must follow server.stop (which must itself also follow server.start). This change makes the necessary changes to call server.wait in the correct sequence. Change-Id: I205683ac6e0f2d64606bb06d08d3d1419f7645f4 --- oslo_messaging/tests/rpc/test_server.py | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/oslo_messaging/tests/rpc/test_server.py b/oslo_messaging/tests/rpc/test_server.py index 93b5e9d69..9a2b53b24 100644 --- a/oslo_messaging/tests/rpc/test_server.py +++ b/oslo_messaging/tests/rpc/test_server.py @@ -117,14 +117,23 @@ class TestRPCServer(test_utils.BaseTestCase, ServerSetupMixin): endpoints = [object()] serializer = object() + class MagicMockIgnoreArgs(mock.MagicMock): + '''A MagicMock which can never misinterpret the arguments passed to + it during construction.''' + + def __init__(self, *args, **kwargs): + super(MagicMockIgnoreArgs, self).__init__() + server = oslo_messaging.get_rpc_server(transport, target, endpoints, serializer=serializer) # Mocking executor - server._executor_obj = mock.Mock() + server._executor_cls = MagicMockIgnoreArgs # Here assigning executor's listener object to listener variable # before calling wait method, because in wait method we are # setting executor to None. + server.start() listener = server._executor_obj.listener + server.stop() # call server wait method server.wait() self.assertIsNone(server._executor_obj) From 3f3c489aafc1461835b6c266c8c1e742d88d725b Mon Sep 17 00:00:00 2001 From: Matthew Booth Date: Mon, 19 Oct 2015 13:04:37 +0100 Subject: [PATCH 05/74] Fix a race calling blocking MessageHandlingServer.start() This fixes a race due to the quirkiness of the blocking executor. The blocking executor does not create a separate thread, but is instead explicitly executed in the calling thread. Other threads will, however, continue to interact with it. In the non-blocking case, the executor will have done certain initialisation in start() before starting a worker thread and returning control to the caller. That is, the caller can be sure that this initialisation has occurred when control is returned. However, in the blocking case, control is never returned. We currently work round this by setting self._running to True before executing executor.start, and by not doing any locking whatsoever in MessageHandlingServer. However, this current means there is a race whereby executor.stop() can run before executor.start(). This is fragile and extremely difficult to reason about robustly, if not currently broken. The solution is to split the initialisation from the execution in the blocking case. executor.start() is no longer a blocking operation for the blocking executor. As for the non-blocking case, executor.start() returns as soon as initialisation is complete, indicating that it is safe to subsequently call stop(). Actual execution is done explicitly via the new execute() method, which blocks. In doing this, we also make FakeBlockingThread a more complete implementation of threading.Thread. This fixes a related issue in that, previously, calling server.wait() on a blocking executor from another thread would not wait for the completion of the executor. This has a knock-on effect in test_server's ServerSetupMixin. This mixin created an endpoint with a stop method which called server.stop(). However, as this is executed by the executor, and also joins the executor thread, which is now blocking, this results in a deadlock. I am satisfied that, in general, this is not a sane thing to do. However, it is useful for these tests. We fix the tests by making the stop method non-blocking, and do the actual stop and wait calls from the main thread. Change-Id: I0d332f74c06c22b44179319432153e15b69f2f45 --- oslo_messaging/_executors/impl_blocking.py | 68 ++++++++++++++++--- oslo_messaging/server.py | 5 +- .../tests/executors/test_executor.py | 15 +++- oslo_messaging/tests/rpc/test_server.py | 47 +++++++++---- 4 files changed, 108 insertions(+), 27 deletions(-) diff --git a/oslo_messaging/_executors/impl_blocking.py b/oslo_messaging/_executors/impl_blocking.py index b59818f5c..b788c47f4 100644 --- a/oslo_messaging/_executors/impl_blocking.py +++ b/oslo_messaging/_executors/impl_blocking.py @@ -14,28 +14,57 @@ # under the License. import futurist +import threading from oslo_messaging._executors import impl_pooledexecutor +from oslo_utils import timeutils class FakeBlockingThread(object): + '''A minimal implementation of threading.Thread which does not create a + thread or start executing the target when start() is called. Instead, the + caller must explicitly execute the non-blocking thread.execute() method + after start() has been called. + ''' + def __init__(self, target): self._target = target + self._running = False + self._running_cond = threading.Condition() def start(self): - self._target() + if self._running: + # Not a user error. No need to translate. + raise RuntimeError('FakeBlockingThread already started') - @staticmethod - def join(timeout=None): - pass + with self._running_cond: + self._running = True + self._running_cond.notify_all() - @staticmethod - def stop(): - pass + def join(self, timeout=None): + with timeutils.StopWatch(duration=timeout) as w, self._running_cond: + while self._running: + self._running_cond.wait(w.leftover(return_none=True)) - @staticmethod - def is_alive(): - return False + # Thread.join() does not raise an exception on timeout. It is + # the caller's responsibility to check is_alive(). + if w.expired(): + return + + def is_alive(self): + return self._running + + def execute(self): + if not self._running: + # Not a user error. No need to translate. + raise RuntimeError('FakeBlockingThread not started') + + try: + self._target() + finally: + with self._running_cond: + self._running = False + self._running_cond.notify_all() class BlockingExecutor(impl_pooledexecutor.PooledExecutor): @@ -52,3 +81,22 @@ class BlockingExecutor(impl_pooledexecutor.PooledExecutor): _executor_cls = lambda __, ___: futurist.SynchronousExecutor() _thread_cls = FakeBlockingThread + + def __init__(self, *args, **kwargs): + super(BlockingExecutor, self).__init__(*args, **kwargs) + + def execute(self): + '''Explicitly run the executor in the current context.''' + # NOTE(mdbooth): Splitting start into start and execute for the + # blocking executor closes a potential race. On a non-blocking + # executor, calling start performs some initialisation synchronously + # before starting the executor and returning control to the caller. In + # the non-blocking caller there was no externally visible boundary + # between the completion of initialisation and the start of execution, + # meaning the caller cannot indicate to another thread that + # initialisation is complete. With the split, the start call for the + # blocking executor becomes analogous to the non-blocking case, + # indicating that initialisation is complete. The caller can then + # synchronously call execute. + if self._poller is not None: + self._poller.execute() diff --git a/oslo_messaging/server.py b/oslo_messaging/server.py index 02bae191a..491ccbf52 100644 --- a/oslo_messaging/server.py +++ b/oslo_messaging/server.py @@ -140,12 +140,15 @@ class MessageHandlingServer(service.ServiceBase): listener = self.dispatcher._listen(self.transport) except driver_base.TransportDriverError as ex: raise ServerListenError(self.target, ex) - self._running = True self._executor_obj = self._executor_cls(self.conf, listener, self.dispatcher) self._executor_obj.start() + self._running = True self._state_cond.notify_all() + if self.executor == 'blocking': + self._executor_obj.execute() + def stop(self): """Stop handling incoming messages. diff --git a/oslo_messaging/tests/executors/test_executor.py b/oslo_messaging/tests/executors/test_executor.py index 007d3ac6a..1e175fdf8 100644 --- a/oslo_messaging/tests/executors/test_executor.py +++ b/oslo_messaging/tests/executors/test_executor.py @@ -81,6 +81,12 @@ class TestExecutor(test_utils.BaseTestCase): aioeventlet_class = None is_aioeventlet = (self.executor == aioeventlet_class) + if impl_blocking is not None: + blocking_class = impl_blocking.BlockingExecutor + else: + blocking_class = None + is_blocking = (self.executor == blocking_class) + if is_aioeventlet: policy = aioeventlet.EventLoopPolicy() trollius.set_event_loop_policy(policy) @@ -110,8 +116,15 @@ class TestExecutor(test_utils.BaseTestCase): endpoint = mock.MagicMock(return_value=simple_coroutine('result')) event = eventlet.event.Event() - else: + elif is_blocking: + def run_executor(executor): + executor.start() + executor.execute() + executor.wait() + endpoint = mock.MagicMock(return_value='result') + event = None + else: def run_executor(executor): executor.start() executor.wait() diff --git a/oslo_messaging/tests/rpc/test_server.py b/oslo_messaging/tests/rpc/test_server.py index 9a2b53b24..258dacb24 100644 --- a/oslo_messaging/tests/rpc/test_server.py +++ b/oslo_messaging/tests/rpc/test_server.py @@ -27,22 +27,38 @@ load_tests = testscenarios.load_tests_apply_scenarios class ServerSetupMixin(object): - class Server(object): + class Server(threading.Thread): def __init__(self, transport, topic, server, endpoint, serializer): + self.controller = ServerSetupMixin.ServerController() target = oslo_messaging.Target(topic=topic, server=server) - self._server = oslo_messaging.get_rpc_server(transport, - target, - [endpoint, self], - serializer=serializer) + self.server = oslo_messaging.get_rpc_server(transport, + target, + [endpoint, + self.controller], + serializer=serializer) + + super(ServerSetupMixin.Server, self).__init__() + self.daemon = True + + def wait(self): + # Wait for the executor to process the stop message, indicating all + # test messages have been processed + self.controller.stopped.wait() + + # Check start() does nothing with a running server + self.server.start() + self.server.stop() + self.server.wait() + + def run(self): + self.server.start() + + class ServerController(object): + def __init__(self): + self.stopped = threading.Event() def stop(self, ctxt): - # Check start() does nothing with a running server - self._server.start() - self._server.stop() - self._server.wait() - - def start(self): - self._server.start() + self.stopped.set() class TestSerializer(object): @@ -72,13 +88,14 @@ class ServerSetupMixin(object): thread.daemon = True thread.start() - return thread + return server - def _stop_server(self, client, server_thread, topic=None): + def _stop_server(self, client, server, topic=None): if topic is not None: client = client.prepare(topic=topic) client.cast({}, 'stop') - server_thread.join(timeout=30) + server.wait() + def _setup_client(self, transport, topic='testtopic'): return oslo_messaging.RPCClient(transport, From f9b14c03377cde4abbd6280b60478eeb91634c29 Mon Sep 17 00:00:00 2001 From: Matthew Booth Date: Wed, 21 Oct 2015 11:36:28 +0100 Subject: [PATCH 06/74] Remove unused event in ServerThreadHelper Change-Id: Ib9ebe363f29cf9a0034550ad852882c2cde8bb49 --- oslo_messaging/tests/utils.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/oslo_messaging/tests/utils.py b/oslo_messaging/tests/utils.py index 8ea89c5ed..eacfaedc8 100644 --- a/oslo_messaging/tests/utils.py +++ b/oslo_messaging/tests/utils.py @@ -66,7 +66,6 @@ class ServerThreadHelper(threading.Thread): self.daemon = True self._server = server self._stop_event = threading.Event() - self._wait_event = threading.Event() def run(self): self._server.start() @@ -75,7 +74,6 @@ class ServerThreadHelper(threading.Thread): self._server.start() self._server.stop() self._server.wait() - self._wait_event.set() def stop(self): self._stop_event.set() From a6da2a98c367de2c1c023acf914c3f4ad5a41783 Mon Sep 17 00:00:00 2001 From: Matthew Booth Date: Fri, 16 Oct 2015 16:16:23 +0100 Subject: [PATCH 07/74] Trivial locking cleanup in test_listener ListenerSetupMixin.ThreadTracker was reading self._received_msgs unlocked and sleep/looping until the desired value was reached. Replaced this pattern with a threading.Condition. Change-Id: Id4731caee2104bdb231e78e7b460905a0aaf84bf --- oslo_messaging/tests/notify/test_listener.py | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/oslo_messaging/tests/notify/test_listener.py b/oslo_messaging/tests/notify/test_listener.py index 1644936ad..fb4a5e011 100644 --- a/oslo_messaging/tests/notify/test_listener.py +++ b/oslo_messaging/tests/notify/test_listener.py @@ -53,16 +53,18 @@ class ListenerSetupMixin(object): def __init__(self): self._received_msgs = 0 self.threads = [] - self.lock = threading.Lock() + self.lock = threading.Condition() def info(self, ctxt, publisher_id, event_type, payload, metadata): # NOTE(sileht): this run into an other thread with self.lock: self._received_msgs += 1 + self.lock.notify_all() def wait_for_messages(self, expect_messages): - while self._received_msgs < expect_messages: - time.sleep(0.01) + with self.lock: + while self._received_msgs < expect_messages: + self.lock.wait() def stop(self): for thread in self.threads: From c4a7ac0b653543e8a3ba10060cabdb114fb6672b Mon Sep 17 00:00:00 2001 From: Cyril Roelandt Date: Wed, 21 Oct 2015 17:08:12 +0200 Subject: [PATCH 08/74] Use yaml.safe_load instead of yaml.load We currently use yaml.load to read a user-written config file. This can lead to malicious code execution, so we should use yaml.safe_load instead. Found using bandit. Change-Id: I27792f0435bc3cb9b9d31846d07a8d47a1e7679d --- oslo_messaging/notify/_impl_routing.py | 2 +- oslo_messaging/tests/notify/test_notifier.py | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/oslo_messaging/notify/_impl_routing.py b/oslo_messaging/notify/_impl_routing.py index bf07e673e..6c5fd18d2 100644 --- a/oslo_messaging/notify/_impl_routing.py +++ b/oslo_messaging/notify/_impl_routing.py @@ -61,7 +61,7 @@ class RoutingDriver(notifier.Driver): return # Infer which drivers are used from the config file. - self.routing_groups = yaml.load( + self.routing_groups = yaml.safe_load( self._get_notifier_config_file(filename)) if not self.routing_groups: self.routing_groups = {} # In case we got None from load() diff --git a/oslo_messaging/tests/notify/test_notifier.py b/oslo_messaging/tests/notify/test_notifier.py index 557b9bb25..e55913d51 100644 --- a/oslo_messaging/tests/notify/test_notifier.py +++ b/oslo_messaging/tests/notify/test_notifier.py @@ -412,7 +412,7 @@ group_1: - blah.zoo.* - zip """ - groups = yaml.load(config) + groups = yaml.safe_load(config) group = groups['group_1'] # No matching event ... @@ -443,7 +443,7 @@ group_1: - info - error """ - groups = yaml.load(config) + groups = yaml.safe_load(config) group = groups['group_1'] # No matching priority @@ -476,7 +476,7 @@ group_1: accepted_events: - foo.* """ - groups = yaml.load(config) + groups = yaml.safe_load(config) group = groups['group_1'] # Valid event, but no matching priority From ea106e9a090aee65879ef6c676c248f54d08a46f Mon Sep 17 00:00:00 2001 From: Oleksii Zamiatin Date: Thu, 8 Oct 2015 22:26:01 +0300 Subject: [PATCH 09/74] Fix target resolution mismatch in neutron, nova, heat Some tempest tests were failing because of NoSuchMethod, UnsupportedVersion and other missed endpoint errors. This fix provides new listener per each target and more straight-forward matchmaker target resolution logic. Change-Id: I4bfb42048630a0eab075e462ad1e22ebe9a45820 Closes-Bug: #1501682 --- oslo_messaging/_drivers/impl_zmq.py | 2 +- .../_drivers/zmq_driver/broker/zmq_broker.py | 8 +- .../zmq_driver/broker/zmq_queue_proxy.py | 74 +++++++++-------- .../client/publishers/zmq_dealer_publisher.py | 79 +++++++++++++++++-- .../client/publishers/zmq_publisher_base.py | 7 ++ .../client/publishers/zmq_req_publisher.py | 45 ++++++++++- .../_drivers/zmq_driver/client/zmq_client.py | 4 +- .../_drivers/zmq_driver/client/zmq_request.py | 11 +++ .../zmq_driver/matchmaker/matchmaker_redis.py | 37 ++++----- .../zmq_driver/poller/threading_poller.py | 7 +- .../server/consumers/zmq_consumer_base.py | 3 +- .../server/consumers/zmq_router_consumer.py | 51 ++++++++---- .../zmq_driver/server/zmq_incoming_message.py | 18 +++-- .../_drivers/zmq_driver/server/zmq_server.py | 8 +- .../_drivers/zmq_driver/zmq_address.py | 11 +++ .../_drivers/zmq_driver/zmq_names.py | 16 ++++ .../_drivers/zmq_driver/zmq_socket.py | 6 ++ 17 files changed, 286 insertions(+), 101 deletions(-) diff --git a/oslo_messaging/_drivers/impl_zmq.py b/oslo_messaging/_drivers/impl_zmq.py index f91c34166..aed6563a9 100644 --- a/oslo_messaging/_drivers/impl_zmq.py +++ b/oslo_messaging/_drivers/impl_zmq.py @@ -229,7 +229,7 @@ class ZmqDriver(base.BaseDriver): :param target: Message destination target :type target: oslo_messaging.Target """ - server = self.server.get() + server = zmq_server.ZmqServer(self, self.conf, self.matchmaker) server.listen(target) return server diff --git a/oslo_messaging/_drivers/zmq_driver/broker/zmq_broker.py b/oslo_messaging/_drivers/zmq_driver/broker/zmq_broker.py index 5f20b807d..8351e2ef9 100644 --- a/oslo_messaging/_drivers/zmq_driver/broker/zmq_broker.py +++ b/oslo_messaging/_drivers/zmq_driver/broker/zmq_broker.py @@ -16,7 +16,6 @@ import logging import os from oslo_utils import excutils -import six from stevedore import driver from oslo_messaging._drivers.zmq_driver.broker import zmq_queue_proxy @@ -51,11 +50,8 @@ class ZmqBroker(object): ).driver(self.conf) self.context = zmq.Context() - self.queue = six.moves.queue.Queue() - self.proxies = [zmq_queue_proxy.OutgoingQueueProxy( - conf, self.context, self.queue, self.matchmaker), - zmq_queue_proxy.IncomingQueueProxy( - conf, self.context, self.queue) + self.proxies = [zmq_queue_proxy.UniversalQueueProxy( + conf, self.context, self.matchmaker) ] def _create_ipc_dirs(self): diff --git a/oslo_messaging/_drivers/zmq_driver/broker/zmq_queue_proxy.py b/oslo_messaging/_drivers/zmq_driver/broker/zmq_queue_proxy.py index 11114d008..dd6665a03 100644 --- a/oslo_messaging/_drivers/zmq_driver/broker/zmq_queue_proxy.py +++ b/oslo_messaging/_drivers/zmq_driver/broker/zmq_queue_proxy.py @@ -14,65 +14,69 @@ import logging -import six - from oslo_messaging._drivers.zmq_driver.broker import zmq_base_proxy from oslo_messaging._drivers.zmq_driver.client.publishers\ import zmq_dealer_publisher from oslo_messaging._drivers.zmq_driver import zmq_address from oslo_messaging._drivers.zmq_driver import zmq_async +from oslo_messaging._drivers.zmq_driver import zmq_names from oslo_messaging._i18n import _LI zmq = zmq_async.import_zmq(zmq_concurrency='native') LOG = logging.getLogger(__name__) -class OutgoingQueueProxy(zmq_base_proxy.BaseProxy): +class UniversalQueueProxy(zmq_base_proxy.BaseProxy): + + def __init__(self, conf, context, matchmaker): + super(UniversalQueueProxy, self).__init__(conf, context) + self.poller = zmq_async.get_poller(zmq_concurrency='native') + + self.router_socket = context.socket(zmq.ROUTER) + self.router_socket.bind(zmq_address.get_broker_address(conf)) + + self.poller.register(self.router_socket, self._receive_in_request) + LOG.info(_LI("Polling at universal proxy")) - def __init__(self, conf, context, queue, matchmaker): - super(OutgoingQueueProxy, self).__init__(conf, context) - self.queue = queue self.matchmaker = matchmaker - self.publisher = zmq_dealer_publisher.DealerPublisher( - conf, matchmaker) - LOG.info(_LI("Polling at outgoing proxy ...")) + reply_receiver = zmq_dealer_publisher.ReplyReceiver(self.poller) + self.publisher = zmq_dealer_publisher.DealerPublisherProxy( + conf, matchmaker, reply_receiver) def run(self): - try: - request = self.queue.get(timeout=self.conf.rpc_poll_timeout) - LOG.info(_LI("Redirecting request %s to TCP publisher ...") - % request) - self.publisher.send_request(request) - except six.moves.queue.Empty: + message, socket = self.poller.poll(self.conf.rpc_poll_timeout) + if message is None: return + if socket == self.router_socket: + self._redirect_in_request(message) + else: + self._redirect_reply(message) -class IncomingQueueProxy(zmq_base_proxy.BaseProxy): + def _redirect_in_request(self, request): + LOG.info(_LI("-> Redirecting request %s to TCP publisher") + % request) + self.publisher.send_request(request) - def __init__(self, conf, context, queue): - super(IncomingQueueProxy, self).__init__(conf, context) - self.poller = zmq_async.get_poller( - zmq_concurrency='native') - - self.queue = queue - - self.socket = context.socket(zmq.ROUTER) - self.socket.bind(zmq_address.get_broker_address(conf)) - self.poller.register(self.socket, self.receive_request) - LOG.info(_LI("Polling at incoming proxy ...")) - - def run(self): - request, socket = self.poller.poll(self.conf.rpc_poll_timeout) - if request is None: + def _redirect_reply(self, reply): + LOG.info(_LI("Reply proxy %s") % reply) + if reply[zmq_names.IDX_REPLY_TYPE] == zmq_names.ACK_TYPE: + LOG.info(_LI("Acknowledge dropped %s") % reply) return - LOG.info(_LI("Received request and queue it: %s") % str(request)) + LOG.info(_LI("<- Redirecting reply to ROUTER: reply: %s") + % reply[zmq_names.IDX_REPLY_BODY:]) - self.queue.put(request) + self.router_socket.send_multipart(reply[zmq_names.IDX_REPLY_BODY:]) - def receive_request(self, socket): + def _receive_in_request(self, socket): reply_id = socket.recv() assert reply_id is not None, "Valid id expected" empty = socket.recv() assert empty == b'', "Empty delimiter expected" - return socket.recv_pyobj() + envelope = socket.recv_pyobj() + if envelope[zmq_names.FIELD_MSG_TYPE] == zmq_names.CALL_TYPE: + envelope[zmq_names.FIELD_REPLY_ID] = reply_id + payload = socket.recv_multipart() + payload.insert(0, envelope) + return payload diff --git a/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_dealer_publisher.py b/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_dealer_publisher.py index 2c8fc5ec5..602e5a99d 100644 --- a/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_dealer_publisher.py +++ b/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_dealer_publisher.py @@ -29,12 +29,10 @@ class DealerPublisher(zmq_publisher_base.PublisherMultisend): def __init__(self, conf, matchmaker): super(DealerPublisher, self).__init__(conf, matchmaker, zmq.DEALER) - self.ack_receiver = AcknowledgementReceiver() def send_request(self, request): - if request.msg_type == zmq_names.CALL_TYPE: - raise zmq_publisher_base.UnsupportedSendPattern(request.msg_type) + self._check_request_pattern(request) dealer_socket, hosts = self._check_hosts_connections(request.target) @@ -47,25 +45,26 @@ class DealerPublisher(zmq_publisher_base.PublisherMultisend): % request.msg_type) return - self.ack_receiver.track_socket(dealer_socket.handle) - if request.msg_type in zmq_names.MULTISEND_TYPES: for _ in range(dealer_socket.connections_count()): self._send_request(dealer_socket, request) else: self._send_request(dealer_socket, request) + def _check_request_pattern(self, request): + if request.msg_type == zmq_names.CALL_TYPE: + raise zmq_publisher_base.UnsupportedSendPattern(request.msg_type) + def _send_request(self, socket, request): socket.send(b'', zmq.SNDMORE) socket.send_pyobj(request) - LOG.info(_LI("Sending message %(message)s to a target %(target)s") - % {"message": request.message, + LOG.info(_LI("Sending message_id %(message)s to a target %(target)s") + % {"message": request.message_id, "target": request.target}) def cleanup(self): - self.ack_receiver.cleanup() super(DealerPublisher, self).cleanup() @@ -81,7 +80,10 @@ class DealerPublisherLight(zmq_publisher_base.PublisherBase): if request.msg_type == zmq_names.CALL_TYPE: raise zmq_publisher_base.UnsupportedSendPattern(request.msg_type) + envelope = request.create_envelope() + self.socket.send(b'', zmq.SNDMORE) + self.socket.send_pyobj(envelope, zmq.SNDMORE) self.socket.send_pyobj(request) def cleanup(self): @@ -89,6 +91,67 @@ class DealerPublisherLight(zmq_publisher_base.PublisherBase): self.socket.close() +class DealerPublisherProxy(DealerPublisher): + + def __init__(self, conf, matchmaker, reply_receiver): + super(DealerPublisherProxy, self).__init__(conf, matchmaker) + self.reply_receiver = reply_receiver + + def send_request(self, multipart_message): + + envelope = multipart_message[zmq_names.MULTIPART_IDX_ENVELOPE] + + LOG.info(_LI("Envelope: %s") % envelope) + + target = envelope[zmq_names.FIELD_TARGET] + dealer_socket, hosts = self._check_hosts_connections(target) + + if not dealer_socket.connections: + # NOTE(ozamiatin): Here we can provide + # a queue for keeping messages to send them later + # when some listener appears. However such approach + # being more reliable will consume additional memory. + LOG.warning(_LW("Request %s was dropped because no connection") + % envelope[zmq_names.FIELD_MSG_TYPE]) + return + + self.reply_receiver.track_socket(dealer_socket.handle) + + LOG.info(_LI("Sending message %(message)s to a target %(target)s") + % {"message": envelope[zmq_names.FIELD_MSG_ID], + "target": envelope[zmq_names.FIELD_TARGET]}) + + if envelope[zmq_names.FIELD_MSG_TYPE] in zmq_names.MULTISEND_TYPES: + for _ in range(dealer_socket.connections_count()): + self._send_request(dealer_socket, multipart_message) + else: + self._send_request(dealer_socket, multipart_message) + + def _send_request(self, socket, multipart_message): + + socket.send(b'', zmq.SNDMORE) + socket.send_pyobj( + multipart_message[zmq_names.MULTIPART_IDX_ENVELOPE], + zmq.SNDMORE) + socket.send(multipart_message[zmq_names.MULTIPART_IDX_BODY]) + + +class ReplyReceiver(object): + + def __init__(self, poller): + self.poller = poller + LOG.info(_LI("Reply waiter created in broker")) + + def _receive_reply(self, socket): + return socket.recv_multipart() + + def track_socket(self, socket): + self.poller.register(socket, self._receive_reply) + + def cleanup(self): + self.poller.close() + + class AcknowledgementReceiver(object): def __init__(self): diff --git a/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_publisher_base.py b/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_publisher_base.py index faee64d25..0a8098af3 100644 --- a/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_publisher_base.py +++ b/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_publisher_base.py @@ -89,6 +89,13 @@ class PublisherBase(object): :param request: Message data and destination container object :type request: zmq_request.Request """ + LOG.info(_LI("Sending %(type)s message_id %(message)s to a target" + "%(target)s key: %(key)s, host:%(host)s") + % {"type": request.msg_type, + "message": request.message_id, + "target": request.target, + "key": zmq_address.target_to_key(request.target), + "host": request.host}) socket.send_pyobj(request) def cleanup(self): diff --git a/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_req_publisher.py b/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_req_publisher.py index d4dbaa9ab..c6063b8ab 100644 --- a/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_req_publisher.py +++ b/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_req_publisher.py @@ -14,6 +14,9 @@ import contextlib import logging +import uuid + +import six import oslo_messaging from oslo_messaging._drivers import common as rpc_common @@ -40,24 +43,34 @@ class ReqPublisher(zmq_publisher_base.PublisherBase): if request.msg_type != zmq_names.CALL_TYPE: raise zmq_publisher_base.UnsupportedSendPattern(request.msg_type) - socket = self._connect_to_host(request.target, request.timeout) + socket, connect_address = self._connect_to_host(request.target, + request.timeout) + request.host = connect_address self._send_request(socket, request) return self._receive_reply(socket, request) + def _resolve_host_address(self, target, timeout=0): + host = self.matchmaker.get_single_host(target, timeout) + return zmq_address.get_tcp_direct_address(host) + def _connect_to_host(self, target, timeout=0): try: self.zmq_context = zmq.Context() socket = self.zmq_context.socket(zmq.REQ) - host = self.matchmaker.get_single_host(target, timeout) - connect_address = zmq_address.get_tcp_direct_address(host) + if six.PY3: + socket.setsockopt_string(zmq.IDENTITY, str(uuid.uuid1())) + else: + socket.identity = str(uuid.uuid1()) + + connect_address = self._resolve_host_address(target, timeout) LOG.info(_LI("Connecting REQ to %s") % connect_address) socket.connect(connect_address) self.outbound_sockets[str(target)] = socket - return socket + return socket, connect_address except zmq.ZMQError as e: errmsg = _LE("Error connecting to socket: %s") % str(e) @@ -77,6 +90,7 @@ class ReqPublisher(zmq_publisher_base.PublisherBase): if reply is None: raise oslo_messaging.MessagingTimeout( "Timeout %s seconds was reached" % request.timeout) + LOG.info(_LI("Received reply %s") % reply) if reply[zmq_names.FIELD_FAILURE]: raise rpc_common.deserialize_remote_exception( reply[zmq_names.FIELD_FAILURE], @@ -87,3 +101,26 @@ class ReqPublisher(zmq_publisher_base.PublisherBase): def close(self): # For contextlib compatibility self.cleanup() + + +class ReqPublisherLight(ReqPublisher): + + def __init__(self, conf, matchmaker): + super(ReqPublisherLight, self).__init__(conf, matchmaker) + + def _resolve_host_address(self, target, timeout=0): + return zmq_address.get_broker_address(self.conf) + + def _send_request(self, socket, request): + + LOG.info(_LI("Sending %(type)s message_id %(message)s" + " to a target %(target)s, host:%(host)s") + % {"type": request.msg_type, + "message": request.message_id, + "target": request.target, + "host": request.host}) + + envelope = request.create_envelope() + + socket.send_pyobj(envelope, zmq.SNDMORE) + socket.send_pyobj(request) diff --git a/oslo_messaging/_drivers/zmq_driver/client/zmq_client.py b/oslo_messaging/_drivers/zmq_driver/client/zmq_client.py index 3e7888d5f..9fb38224d 100644 --- a/oslo_messaging/_drivers/zmq_driver/client/zmq_client.py +++ b/oslo_messaging/_drivers/zmq_driver/client/zmq_client.py @@ -37,16 +37,18 @@ class ZmqClient(object): if self.conf.zmq_use_broker: self.dealer_publisher = zmq_dealer_publisher.DealerPublisherLight( conf, zmq_address.get_broker_address(self.conf)) + self.req_publisher_cls = zmq_req_publisher.ReqPublisherLight else: self.dealer_publisher = zmq_dealer_publisher.DealerPublisher( conf, matchmaker) + self.req_publisher_cls = zmq_req_publisher.ReqPublisher def send_call(self, target, context, message, timeout=None, retry=None): with contextlib.closing(zmq_request.CallRequest( target, context=context, message=message, timeout=timeout, retry=retry, allowed_remote_exmods=self.allowed_remote_exmods)) as request: - with contextlib.closing(zmq_req_publisher.ReqPublisher( + with contextlib.closing(self.req_publisher_cls( self.conf, self.matchmaker)) as req_publisher: return req_publisher.send_request(request) diff --git a/oslo_messaging/_drivers/zmq_driver/client/zmq_request.py b/oslo_messaging/_drivers/zmq_driver/client/zmq_request.py index 92d444a33..455b7ba5a 100644 --- a/oslo_messaging/_drivers/zmq_driver/client/zmq_request.py +++ b/oslo_messaging/_drivers/zmq_driver/client/zmq_request.py @@ -63,6 +63,12 @@ class Request(object): self.message = message self.retry = retry self.message_id = str(uuid.uuid1()) + self.proxy_reply_id = None + + def create_envelope(self): + return {'msg_type': self.msg_type, + 'message_id': self.message_id, + 'target': self.target} @abc.abstractproperty def msg_type(self): @@ -86,6 +92,11 @@ class RpcRequest(Request): super(RpcRequest, self).__init__(*args, **kwargs) + def create_envelope(self): + envelope = super(RpcRequest, self).create_envelope() + envelope['timeout'] = self.timeout + return envelope + class CallRequest(RpcRequest): diff --git a/oslo_messaging/_drivers/zmq_driver/matchmaker/matchmaker_redis.py b/oslo_messaging/_drivers/zmq_driver/matchmaker/matchmaker_redis.py index c8402c6c8..cbf4e1066 100644 --- a/oslo_messaging/_drivers/zmq_driver/matchmaker/matchmaker_redis.py +++ b/oslo_messaging/_drivers/zmq_driver/matchmaker/matchmaker_redis.py @@ -17,6 +17,7 @@ from oslo_config import cfg from oslo_utils import importutils from oslo_messaging._drivers.zmq_driver.matchmaker import base +from oslo_messaging._drivers.zmq_driver import zmq_address redis = importutils.try_import('redis') LOG = logging.getLogger(__name__) @@ -48,34 +49,30 @@ class RedisMatchMaker(base.MatchMakerBase): password=self.conf.matchmaker_redis.password, ) - def _target_to_key(self, target): - attributes = ['topic', 'exchange', 'server'] - prefix = "ZMQ-target" - key = ":".join((getattr(target, attr) or "*") for attr in attributes) - return "%s-%s" % (prefix, key) - - def _get_keys_by_pattern(self, pattern): - return self._redis.keys(pattern) - def _get_hosts_by_key(self, key): return self._redis.lrange(key, 0, -1) def register(self, target, hostname): - key = self._target_to_key(target) - if hostname not in self._get_hosts_by_key(key): - self._redis.lpush(key, hostname) + + if target.topic and target.server: + key = zmq_address.target_to_key(target) + if hostname not in self._get_hosts_by_key(key): + self._redis.lpush(key, hostname) + + if target.topic: + if hostname not in self._get_hosts_by_key(target.topic): + self._redis.lpush(target.topic, hostname) + + if target.server: + if hostname not in self._get_hosts_by_key(target.server): + self._redis.lpush(target.server, hostname) def unregister(self, target, hostname): - key = self._target_to_key(target) + key = zmq_address.target_to_key(target) self._redis.lrem(key, 0, hostname) def get_hosts(self, target): - pattern = self._target_to_key(target) - if "*" not in pattern: - # pattern have no placeholders, so this is valid key - return self._get_hosts_by_key(pattern) - hosts = [] - for key in self._get_keys_by_pattern(pattern): - hosts.extend(self._get_hosts_by_key(key)) + key = zmq_address.target_to_key(target) + hosts.extend(self._get_hosts_by_key(key)) return hosts diff --git a/oslo_messaging/_drivers/zmq_driver/poller/threading_poller.py b/oslo_messaging/_drivers/zmq_driver/poller/threading_poller.py index c0a46d981..518d32e49 100644 --- a/oslo_messaging/_drivers/zmq_driver/poller/threading_poller.py +++ b/oslo_messaging/_drivers/zmq_driver/poller/threading_poller.py @@ -38,12 +38,17 @@ class ThreadingPoller(zmq_poller.ZmqPoller): self.recv_methods = {} def register(self, socket, recv_method=None): + if socket in self.recv_methods: + return if recv_method is not None: self.recv_methods[socket] = recv_method self.poller.register(socket, zmq.POLLIN) def poll(self, timeout=None): - timeout *= 1000 # zmq poller waits milliseconds + + if timeout: + timeout *= 1000 # zmq poller waits milliseconds + sockets = None try: diff --git a/oslo_messaging/_drivers/zmq_driver/server/consumers/zmq_consumer_base.py b/oslo_messaging/_drivers/zmq_driver/server/consumers/zmq_consumer_base.py index 8bb2461e7..f002c45af 100644 --- a/oslo_messaging/_drivers/zmq_driver/server/consumers/zmq_consumer_base.py +++ b/oslo_messaging/_drivers/zmq_driver/server/consumers/zmq_consumer_base.py @@ -19,6 +19,7 @@ import six from oslo_messaging._drivers import common as rpc_common from oslo_messaging._drivers.zmq_driver import zmq_async +from oslo_messaging._drivers.zmq_driver import zmq_names from oslo_messaging._drivers.zmq_driver import zmq_socket from oslo_messaging._i18n import _LE, _LI @@ -44,7 +45,7 @@ class ConsumerBase(object): self.sockets.append(socket) self.poller.register(socket, self.receive_message) LOG.info(_LI("Run %(stype)s consumer on %(addr)s:%(port)d"), - {"stype": socket_type, + {"stype": zmq_names.socket_type_str(socket_type), "addr": socket.bind_address, "port": socket.port}) return socket diff --git a/oslo_messaging/_drivers/zmq_driver/server/consumers/zmq_router_consumer.py b/oslo_messaging/_drivers/zmq_driver/server/consumers/zmq_router_consumer.py index f6016607e..9c529dac0 100644 --- a/oslo_messaging/_drivers/zmq_driver/server/consumers/zmq_router_consumer.py +++ b/oslo_messaging/_drivers/zmq_driver/server/consumers/zmq_router_consumer.py @@ -43,11 +43,7 @@ class RouterIncomingMessage(base.IncomingMessage): """Reply is not needed for non-call messages""" def acknowledge(self): - LOG.info("Sending acknowledge for %s", self.msg_id) - ack_message = {zmq_names.FIELD_ID: self.msg_id} - self.socket.send(self.reply_id, zmq.SNDMORE) - self.socket.send(b'', zmq.SNDMORE) - self.socket.send_pyobj(ack_message) + LOG.info("Not sending acknowledge for %s", self.msg_id) def requeue(self): """Requeue is not supported""" @@ -61,11 +57,11 @@ class RouterConsumer(zmq_consumer_base.SingleSocketConsumer): self.targets = [] self.host = zmq_address.combine_address(self.conf.rpc_zmq_host, self.port) + LOG.info("[%s] Run ROUTER consumer" % self.host) def listen(self, target): - LOG.info("Listen to target %s on %s:%d" % - (target, self.address, self.port)) + LOG.info("[%s] Listen to target %s" % (self.host, target)) self.targets.append(target) self.matchmaker.register(target=target, @@ -76,21 +72,25 @@ class RouterConsumer(zmq_consumer_base.SingleSocketConsumer): for target in self.targets: self.matchmaker.unregister(target, self.host) + def _receive_request(self, socket): + reply_id = socket.recv() + empty = socket.recv() + assert empty == b'', 'Bad format: empty delimiter expected' + request = socket.recv_pyobj() + return request, reply_id + def receive_message(self, socket): try: - reply_id = socket.recv() - empty = socket.recv() - assert empty == b'', 'Bad format: empty delimiter expected' - request = socket.recv_pyobj() - - LOG.info(_LI("Received %(msg_type)s message %(msg)s") - % {"msg_type": request.msg_type, - "msg": str(request.message)}) + request, reply_id = self._receive_request(socket) + LOG.info(_LI("[%(host)s] Received %(type)s, %(id)s, %(target)s") + % {"host": self.host, + "type": request.msg_type, + "id": request.message_id, + "target": request.target}) if request.msg_type == zmq_names.CALL_TYPE: return zmq_incoming_message.ZmqIncomingRequest( - self.server, request.context, request.message, socket, - reply_id, self.poller) + self.server, socket, reply_id, request, self.poller) elif request.msg_type in zmq_names.NON_BLOCKING_TYPES: return RouterIncomingMessage( self.server, request.context, request.message, socket, @@ -100,3 +100,20 @@ class RouterConsumer(zmq_consumer_base.SingleSocketConsumer): except zmq.ZMQError as e: LOG.error(_LE("Receiving message failed: %s") % str(e)) + + +class RouterConsumerBroker(RouterConsumer): + + def __init__(self, conf, poller, server): + super(RouterConsumerBroker, self).__init__(conf, poller, server) + + def _receive_request(self, socket): + reply_id = socket.recv() + empty = socket.recv() + assert empty == b'', 'Bad format: empty delimiter expected' + envelope = socket.recv_pyobj() + request = socket.recv_pyobj() + + if zmq_names.FIELD_REPLY_ID in envelope: + request.proxy_reply_id = envelope[zmq_names.FIELD_REPLY_ID] + return request, reply_id diff --git a/oslo_messaging/_drivers/zmq_driver/server/zmq_incoming_message.py b/oslo_messaging/_drivers/zmq_driver/server/zmq_incoming_message.py index f43ec2325..5e932493f 100644 --- a/oslo_messaging/_drivers/zmq_driver/server/zmq_incoming_message.py +++ b/oslo_messaging/_drivers/zmq_driver/server/zmq_incoming_message.py @@ -28,10 +28,12 @@ zmq = zmq_async.import_zmq() class ZmqIncomingRequest(base.IncomingMessage): - def __init__(self, listener, context, message, socket, rep_id, poller): - super(ZmqIncomingRequest, self).__init__(listener, context, message) + def __init__(self, listener, socket, rep_id, request, poller): + super(ZmqIncomingRequest, self).__init__(listener, request.context, + request.message) self.reply_socket = socket self.reply_id = rep_id + self.request = request self.received = None self.poller = poller @@ -39,15 +41,21 @@ class ZmqIncomingRequest(base.IncomingMessage): if failure is not None: failure = rpc_common.serialize_remote_exception(failure, log_failure) - message_reply = {zmq_names.FIELD_REPLY: reply, + message_reply = {zmq_names.FIELD_TYPE: zmq_names.REPLY_TYPE, + zmq_names.FIELD_REPLY: reply, zmq_names.FIELD_FAILURE: failure, - zmq_names.FIELD_LOG_FAILURE: log_failure} + zmq_names.FIELD_LOG_FAILURE: log_failure, + zmq_names.FIELD_ID: self.request.proxy_reply_id} - LOG.info("Replying %s REP", (str(message_reply))) + LOG.info("Replying %s REP", (str(self.request.message_id))) self.received = True self.reply_socket.send(self.reply_id, zmq.SNDMORE) self.reply_socket.send(b'', zmq.SNDMORE) + if self.request.proxy_reply_id: + self.reply_socket.send_string(zmq_names.REPLY_TYPE, zmq.SNDMORE) + self.reply_socket.send(self.request.proxy_reply_id, zmq.SNDMORE) + self.reply_socket.send(b'', zmq.SNDMORE) self.reply_socket.send_pyobj(message_reply) self.poller.resume_polling(self.reply_socket) diff --git a/oslo_messaging/_drivers/zmq_driver/server/zmq_server.py b/oslo_messaging/_drivers/zmq_driver/server/zmq_server.py index afe03b81b..0680e7fa5 100644 --- a/oslo_messaging/_drivers/zmq_driver/server/zmq_server.py +++ b/oslo_messaging/_drivers/zmq_driver/server/zmq_server.py @@ -31,8 +31,12 @@ class ZmqServer(base.Listener): super(ZmqServer, self).__init__(driver) self.matchmaker = matchmaker self.poller = zmq_async.get_poller() - self.rpc_consumer = zmq_router_consumer.RouterConsumer( - conf, self.poller, self) + if conf.zmq_use_broker: + self.rpc_consumer = zmq_router_consumer.RouterConsumerBroker( + conf, self.poller, self) + else: + self.rpc_consumer = zmq_router_consumer.RouterConsumer( + conf, self.poller, self) self.notify_consumer = self.rpc_consumer self.consumers = [self.rpc_consumer] diff --git a/oslo_messaging/_drivers/zmq_driver/zmq_address.py b/oslo_messaging/_drivers/zmq_driver/zmq_address.py index e8c48291b..afc92490f 100644 --- a/oslo_messaging/_drivers/zmq_driver/zmq_address.py +++ b/oslo_messaging/_drivers/zmq_driver/zmq_address.py @@ -27,3 +27,14 @@ def get_tcp_random_address(conf): def get_broker_address(conf): return "ipc://%s/zmq-broker" % conf.rpc_zmq_ipc_dir + + +def target_to_key(target): + if target.topic and target.server: + attributes = ['topic', 'server'] + key = ".".join(getattr(target, attr) for attr in attributes) + return key + if target.topic: + return target.topic + if target.server: + return target.server diff --git a/oslo_messaging/_drivers/zmq_driver/zmq_names.py b/oslo_messaging/_drivers/zmq_driver/zmq_names.py index a317456e7..f7401ab21 100644 --- a/oslo_messaging/_drivers/zmq_driver/zmq_names.py +++ b/oslo_messaging/_drivers/zmq_driver/zmq_names.py @@ -17,10 +17,23 @@ from oslo_messaging._drivers.zmq_driver import zmq_async zmq = zmq_async.import_zmq() +FIELD_TYPE = 'type' FIELD_FAILURE = 'failure' FIELD_REPLY = 'reply' FIELD_LOG_FAILURE = 'log_failure' FIELD_ID = 'id' +FIELD_MSG_ID = 'message_id' +FIELD_MSG_TYPE = 'msg_type' +FIELD_REPLY_ID = 'reply_id' +FIELD_TARGET = 'target' + + +IDX_REPLY_TYPE = 1 +IDX_REPLY_BODY = 2 + +MULTIPART_IDX_ENVELOPE = 0 +MULTIPART_IDX_BODY = 1 + CALL_TYPE = 'call' CAST_TYPE = 'cast' @@ -28,6 +41,9 @@ CAST_FANOUT_TYPE = 'cast-f' NOTIFY_TYPE = 'notify' NOTIFY_FANOUT_TYPE = 'notify-f' +REPLY_TYPE = 'reply' +ACK_TYPE = 'ack' + MESSAGE_TYPES = (CALL_TYPE, CAST_TYPE, CAST_FANOUT_TYPE, diff --git a/oslo_messaging/_drivers/zmq_driver/zmq_socket.py b/oslo_messaging/_drivers/zmq_driver/zmq_socket.py index 2a4144c5a..8e51e30f1 100644 --- a/oslo_messaging/_drivers/zmq_driver/zmq_socket.py +++ b/oslo_messaging/_drivers/zmq_driver/zmq_socket.py @@ -57,6 +57,9 @@ class ZmqSocket(object): def send_pyobj(self, *args, **kwargs): self.handle.send_pyobj(*args, **kwargs) + def send_multipart(self, *args, **kwargs): + self.handle.send_multipart(*args, **kwargs) + def recv(self, *args, **kwargs): return self.handle.recv(*args, **kwargs) @@ -69,6 +72,9 @@ class ZmqSocket(object): def recv_pyobj(self, *args, **kwargs): return self.handle.recv_pyobj(*args, **kwargs) + def recv_multipart(self, *args, **kwargs): + return self.handle.recv_multipart(*args, **kwargs) + def close(self, *args, **kwargs): self.handle.close(*args, **kwargs) From d05278f762dd44d8e328ba11888e524e42a42c3f Mon Sep 17 00:00:00 2001 From: Dina Belova Date: Fri, 16 Oct 2015 17:32:55 +0300 Subject: [PATCH 10/74] Modify simulator.py tool Introduce mechanism of generating real life messages to the tool using the information gathered during Rally testing. This change allows to generate messages of the specfic length due to the distribution observed on real environment. messages_length.txt file contains lengths of string JSON objects that were later sent through the MQ layer during deployment and deletion of 50 VMs. simulator.py was modified to use this data as a baseline to generate random string messages of the required length with the needed probability. Change-Id: Iae21f90b5ca202bf0e83f1149baef8b42c64eb55 --- tools/messages_length.yaml | 13 +++++++ tools/simulator.py | 77 ++++++++++++++++++++++++++++++++++---- 2 files changed, 82 insertions(+), 8 deletions(-) create mode 100644 tools/messages_length.yaml diff --git a/tools/messages_length.yaml b/tools/messages_length.yaml new file mode 100644 index 000000000..092ed513c --- /dev/null +++ b/tools/messages_length.yaml @@ -0,0 +1,13 @@ +# The numbers below present the length of the messages (in string equivalent) +# that were sent through the MQ backend (RabbitMQ) during the +# boot_and_delete_server Rally scenario run (50 times, concurrency equal to 3). +# The information was gathered via adding log to the _send method of +# AMQPDriverBase class after all lines related to the msg object modifications. + +# Message length was gathered to introduce real-like message generator for +# simulator.py oslo.messaging tool, that could introduce traffic closer to the +# real control plane load and estimate both message length and size (in bytes) +# going through the MQ layer. + +test_data: + string_lengths: 806, 992, 992, 1116, 1116, 1191, 1595, 1199, 1043, 1210, 1220, 1191, 1123, 1624, 2583, 1153, 4412, 1642, 1210, 1590, 1500, 1500, 1500, 1500, 1500, 1500, 6386, 6368, 6386, 6368, 6386, 11292, 2136, 5407, 6368, 11292, 2136, 5407, 2116, 2116, 11292, 2136, 5398, 5407, 4357, 5431, 2116, 2116, 5398, 4407, 5431, 2116, 2116, 5398, 4457, 5431, 4387, 2627, 4387, 2094, 2038, 2627, 2094, 2038, 5438, 4387, 5438, 2310, 2310, 2627, 2094, 2496, 2038, 5451, 2310, 5438, 2496, 2496, 2240, 2099, 2240, 1500, 2099, 2626, 5451, 2240, 2626, 1555, 1555, 1702, 1500, 5451, 1702, 2450, 2450, 1570, 1155, 4539, 1570, 4539, 1641, 2099, 1641, 2626, 1555, 1702, 2450, 1570, 3518, 5710, 1641, 2226, 2643, 3382, 6671, 3518, 2531, 2226, 2643, 2124, 3382, 5500, 3518, 2531, 2226, 2643, 965, 2124, 3382, 5500, 6858, 2531, 1177, 965, 2124, 5687, 1177, 965, 1575, 1500, 1500, 2549, 7745, 1575, 5687, 7688, 2183, 1177, 2549, 965, 6574, 7688, 2183, 7270, 2128, 7270, 2128, 1575, 6535, 2549, 6574, 6480, 2643, 2584, 6535, 1220, 2644, 7688, 2183, 1500, 1676, 2611, 1500, 6480, 2611, 2643, 1624, 2241, 1153, 4696, 7270, 2128, 2584, 2644, 1590, 2611, 2611, 1555, 2241, 1555, 6535, 6480, 2643, 2584, 2644, 2611, 2611, 2241, 1555, 1702, 2450, 1570, 3308, 2043, 3518, 2582, 2398, 2226, 2093, 3420, 6576, 2142, 4480, 6536, 2298, 2608, 1855, 1880, 2175, 6386, 6368, 11292, 2136, 5407, 2116, 2116, 5398, 4504, 5431, 4434, 2627, 2094, 2038, 5438, 2310, 2496, 2240, 5451, 2099, 2626, 1555, 1702, 2450, 1570, 4539, 1641, 3518, 2226, 2643, 3382, 5500, 2531, 2124, 5687, 1177, 965, 1575, 2549, 6574, 7688, 2183, 1500, 7270, 2128, 1500, 6535, 6480, 2643, 2584, 2644, 2611, 2611, 2241, 1555, 1702, 2450, 1570, 3308, 2043, 3518, 1575, 2582, 2398, 2226, 2093, 3420, 6576, 2142, 4532, 6536, 2298, 2608, 1855, 1880, 2175, 1575, 6386, 6368, 11292, 2136, 5407, 2116, 2116, 5398, 4532, 5431, 4434, 2627, 2094, 2038, 5438, 2310, 2496, 2240, 5451, 2099, 2626, 1555, 1702, 2450, 1570, 4539, 1641, 3518, 2226, 2643, 3382, 5500, 2531, 2124, 5687, 1177, 965, 1575, 1575, 2549, 6574, 7688, 2183, 1500, 7270, 2128, 1500, 6535, 6480, 2643, 2584, 2644, 2611, 2611, 2241, 1555, 1702, 2450, 1570, 3308, 2043, 3518, 2582, 2398, 2226, 2093, 3420, 6576, 2142, 4532, 6536, 2298, 2608, 1855, 1880, 2175, 6386, 6368, 11292, 2136, 5407, 2116, 2116, 5398, 4582, 5431, 4484, 2627, 2094, 2038, 5438, 2310, 2496, 2240, 2099, 2626, 5451, 1555, 1702, 2450, 1570, 4539, 1641, 3518, 2226, 2643, 3382, 5500, 2531, 2124, 1575, 5687, 1177, 965, 1575, 2549, 6574, 7688, 2183, 7270, 1500, 2128, 1500, 6535, 6480, 2643, 2584, 2644, 2611, 2611, 2241, 1555, 1702, 2450, 1570, 3308, 2043, 3518, 2582, 2398, 2226, 2093, 3420, 6576, 2142, 4582, 6536, 2298, 2608, 1855, 1880, 2175, 6386, 6368, 11292, 2136, 5407, 2116, 2116, 5398, 4582, 5431, 4484, 2627, 2094, 2038, 5438, 2310, 2496, 2240, 5451, 2099, 2626, 1555, 1702, 2450, 1570, 4539, 1641, 3518, 2226, 2643, 3382, 5500, 2531, 2124, 5687, 1177, 965, 1575, 1575, 2549, 6574, 7688, 2183, 7270, 2128, 1500, 6535, 6480, 2643, 2584, 2644, 2611, 2611, 2241, 1500, 1555, 1702, 2450, 1570, 3308, 2043, 3518, 2582, 2398, 2226, 2093, 3420, 6576, 2142, 4582, 6536, 2298, 2608, 1855, 1880, 2175, 6386, 6368, 11292, 2136, 5407, 2116, 2116, 5398, 4582, 5431, 4484, 2627, 2094, 2038, 5438, 2310, 2496, 2240, 5451, 2099, 2626, 1555, 1702, 2450, 1570, 4539, 1641, 3518, 2226, 2643, 3382, 5500, 2531, 2124, 5687, 1177, 965, 1575, 1575, 2549, 6574, 7688, 2183, 7270, 2128, 6535, 6480, 2643, 2584, 2644, 2611, 2611, 2241, 1555, 1702, 2450, 1570, 3308, 2043, 3518, 1500, 2582, 2398, 2226, 2093, 3420, 6576, 2142, 4582, 6536, 2298, 2608, 1855, 1880, 1500, 2175, 6386, 6368, 11292, 2136, 5407, 2116, 2116, 5398, 4582, 5431, 4484, 2627, 2094, 2038, 5438, 2310, 2496, 2240, 5451, 2099, 2626, 1555, 1702, 2450, 1570, 4539, 1641, 3518, 2226, 2643, 3382, 5500, 2531, 2124, 5687, 1177, 965, 1575, 1575, 2549, 6574, 7688, 2183, 7270, 2128, 6516, 2300, 6516, 5839, 6156, 6512, 1597, 1500, 1026, 1676, 1500, 6516, 4505, 1220, 2300, 6516, 1624, 6535, 1153, 4668, 5839, 2228, 6156, 1590, 6480, 2643, 6512, 2228, 2584, 1611, 2644, 1102, 1701, 2611, 4354, 2449, 2611, 2241, 1555, 1702, 2450, 1570, 3308, 2043, 3518, 1575, 2582, 2398, 6386, 2226, 6368, 2093, 3420, 6576, 2142, 4452, 11292, 2136, 6536, 5407, 6386, 6368, 2298, 2116, 2116, 2608, 5398, 1855, 1880, 2175, 4526, 5431, 11292, 2136, 5407, 4456, 2627, 2094, 2038, 2116, 2310, 2496, 5438, 2116, 2240, 5398, 5451, 4604, 5431, 2099, 2626, 1555, 4506, 2627, 1702, 2094, 2038, 5438, 2310, 2450, 2496, 4539, 2240, 1641, 2099, 1500, 1570, 6386, 2626, 5451, 1555, 6368, 1500, 1702, 2450, 11292, 2136, 1570, 5407, 3518, 2116, 2116, 5398, 4539, 2226, 1641, 4604, 2643, 5431, 3382, 3518, 5500, 4506, 2531, 2627, 2094, 2038, 5438, 2226, 2310, 2124, 2643, 3382, 5451, 2496, 5500, 2240, 2531, 2099, 2626, 1555, 5687, 2124, 1177, 1702, 965, 2450, 1570, 4539, 1641, 1575, 3518, 2226, 2643, 3382, 5500, 1575, 5687, 2531, 1177, 965, 6574, 2549, 2124, 1500, 1500, 7688, 2183, 7270, 2128, 1575, 5687, 1177, 2549, 6574, 965, 6535, 7688, 2183, 6480, 2643, 2584, 2644, 2611, 2611, 2241, 1555, 1500, 1702, 1500, 2450, 1570, 3308, 2043, 3518, 7270, 2128, 2582, 2398, 2226, 2093, 3420, 6576, 2142, 1575, 2549, 6574, 4604, 6535, 6536, 7688, 2183, 2298, 6480, 2643, 2608, 1855, 1880, 2175, 2584, 2644, 2611, 2611, 2241, 1555, 1702, 2450, 1570, 3308, 2043, 3518, 1500, 1500, 7270, 2128, 2582, 2398, 2226, 2093, 3420, 6576, 1575, 6386, 6368, 11292, 2136, 5407, 2116, 2116, 5398, 4604, 5431, 2142, 4604, 6535, 6536, 4506, 2627, 2094, 2038, 2298, 6480, 2643, 2310, 5438, 2608, 2496, 1855, 1880, 2175, 2584, 2240, 2644, 2099, 2626, 5451, 2611, 1555, 2611, 2241, 1702, 2450, 1555, 1570, 1702, 2450, 1570, 3308, 2043, 3518, 4539, 1641, 3518, 2582, 2398, 6386, 2226, 6368, 2093, 3420, 6576, 2226, 2643, 3382, 5500, 2142, 4604, 11292, 2136, 6536, 5407, 2531, 2116, 2116, 2124, 5398, 2298, 2608, 1855, 1880, 2175, 4604, 5431, 5687, 1177, 4506, 965, 2627, 2094, 2038, 5438, 2310, 2496, 2240, 5451, 2099, 2626, 1555, 1702, 2450, 1570, 1575, 1500, 4539, 1641, 1500, 1575, 2549, 6574, 3518, 7688, 2183, 2226, 2643, 3382, 5500, 2531, 2124, 7270, 2128, 6386, 6368, 11292, 2136, 5407, 5687, 1177, 2116, 2116, 5398, 965, 4604, 6535, 5431, 6480, 2643, 4506, 2584, 2627, 2094, 2644, 2038, 5438, 2611, 2310, 2611, 5451, 2496, 2241, 2240, 1575, 1555, 1702, 2450, 2099, 1570, 2626, 3308, 1555, 2043, 3518, 1702, 4539, 1575, 2450, 1641, 1570, 2549, 1500, 6574, 1500, 1220, 2582, 2398, 2226, 2093, 7688, 2183, 3420, 1624, 6576, 1676, 3518, 1153, 4717, 2142, 1590, 4501, 2226, 6536, 1611, 2643, 7270, 2128, 1102, 1701, 3382, 5500, 2449, 2298, 2608, 1855, 2531, 1880, 2175, 2124, 6535, 6480, 2643, 2584, 5687, 2644, 1177, 2611, 965, 2611, 2241, 1555, 1702, 2450, 6386, 6368, 1570, 3308, 2043, 3518, 11292, 2136, 5407, 2116, 2582, 2116, 2398, 5398, 2226, 2093, 4551, 3420, 6576, 5431, 1575, 1500, 6574, 1500, 4481, 2549, 1575, 2627, 2142, 2094, 2038, 5438, 2310, 2496, 4579, 6536, 2240, 2099, 7688, 2183, 2626, 5451, 1555, 2298, 1702, 2450, 1570, 2608, 1855, 1880, 2175, 7270, 2128, 4539, 1641, 3518, 2226, 2643, 3382, 5500, 2531, 2124, 6535, 6480, 2643, 2584, 2644, 2611, 2611, 5687, 2241, 1177, 965, 1555, 6386, 6368, 1702, 2450, 1570, 11292, 2136, 3308, 5407, 2043, 3518, 2116, 2116, 5398, 2582, 4579, 2398, 5431, 2226, 2093, 3420, 4481, 1500, 6576, 2627, 2094, 2038, 5438, 1500, 2142, 2310, 1575, 1575, 2496, 2240, 6574, 2099, 4579, 2626, 1555, 2549, 5451, 1702, 6536, 2450, 1570, 7688, 2183, 2298, 2608, 1855, 1880, 2175, 3518, 5710, 2226, 1641, 2643, 3382, 6671, 7270, 2128, 2531, 6386, 6368, 11292, 2136, 5407, 2116, 2116, 5398, 2124, 4629, 5431, 6535, 4531, 2627, 2094, 2038, 2310, 6480, 2643, 2496, 5438, 6858, 2584, 1177, 2240, 965, 2644, 1500, 2611, 5451, 2611, 2241, 2099, 1500, 2626, 1555, 1555, 1702, 2450, 1702, 1575, 1570, 2450, 4539, 1570, 1641, 3308, 2043, 3518, 1575, 3518, 2549, 7745, 2582, 2398, 2226, 2643, 2226, 7688, 2093, 2183, 3382, 3420, 5500, 6576, 2531, 2124, 2142, 4629, 6536, 2298, 2608, 7270, 2128, 1855, 1880, 2175, 5687, 1177, 965, 6535, 6480, 2643, 2584, 2644, 6386, 6368, 2611, 2611, 2241, 11292, 2136, 5407, 1555, 1500, 1702, 2116, 2116, 1500, 5398, 2450, 1570, 3308, 4629, 2043, 5431, 3518, 1575, 4531, 2549, 2627, 2094, 2038, 5438, 6574, 2582, 2310, 2496, 2398, 5451, 2240, 7688, 2183, 2226, 1575, 2093, 3420, 2099, 2626, 1555, 6576, 1702, 2450, 2142, 1570, 4629, 6536, 4539, 1641, 2298, 2608, 1855, 1880, 2175, 7270, 2128, 3518, 2226, 2643, 3382, 5500, 1500, 2531, 1500, 2124, 6535, 6480, 2643, 2584, 2644, 2611, 2611, 2241, 5687, 6386, 1177, 1555, 6368, 965, 1702, 2450, 11292, 1570, 2136, 3308, 5407, 2043, 3518, 2116, 2116, 5398, 1575, 2582, 4679, 2398, 2226, 5431, 2093, 3420, 6576, 4581, 2627, 2094, 2038, 2310, 1575, 2496, 2549, 2142, 5438, 6574, 2240, 4679, 6536, 7688, 2183, 5451, 2099, 2626, 2298, 1555, 2608, 1855, 1880, 2175, 1702, 2450, 1570, 7270, 4539, 1500, 2128, 1641, 1500, 1597, 1066, 3518, 2226, 2643, 3382, 5500, 1220, 2531, 1624, 2124, 1153, 1676, 4818, 6386, 6535, 6368, 1624, 6480, 2643, 2584, 1611, 2644, 5687, 2611, 11292, 2136, 2611, 2241, 1177, 965, 1102, 1701, 5407, 2449, 1555, 1575, 1702, 2116, 2450, 2116, 1570, 5398, 3308, 2043, 3518, 4602, 5431, 2582, 2398, 4532, 2226, 2627, 2094, 2038, 2093, 5438, 2310, 3420, 2496, 6576, 1575, 2240, 5451, 2549, 2142, 6574, 4630, 6536, 2099, 2626, 1500, 7688, 2183, 1500, 4539, 1555, 2298, 1641, 2608, 1702, 1855, 1880, 2175, 2450, 1570, 7270, 2128, 3518, 2226, 2643, 3382, 5500, 2531, 6386, 6368, 6535, 2124, 6480, 2643, 11292, 2136, 2584, 5407, 2644, 2611, 2611, 2241, 2116, 2116, 5687, 5398, 1177, 1555, 965, 1575, 1702, 2450, 4630, 1570, 3308, 5431, 2043, 3518, 4532, 2627, 2094, 2038, 5438, 2310, 2496, 2582, 2398, 2240, 5451, 2226, 2093, 1500, 2099, 3420, 6576, 2626, 1500, 1555, 1575, 6574, 2549, 2142, 1702, 4630, 4539, 2450, 1641, 6536, 1570, 7688, 2183, 2298, 2608, 1855, 1880, 2175, 7270, 2128, 3518, 2226, 2643, 3382, 5500, 2531, 2124, 6535, 6480, 2643, 2584, 2644, 2611, 2611, 5687, 2241, 1177, 965, 1555, 1702, 6386, 2450, 6368, 1570, 3308, 2043, 1575, 1500, 3518, 11292, 2136, 5407, 1500, 2582, 2116, 2398, 2116, 2226, 5398, 2093, 3420, 6576, 4680, 5431, 2142, 4680, 6536, 4582, 1575, 2627, 2094, 2038, 5438, 6574, 2549, 2310, 5451, 2496, 2298, 2240, 2608, 1855, 1880, 2175, 7688, 2183, 2099, 2626, 1555, 1702, 2450, 1570, 4539, 1641, 3518, 2226, 2643, 3382, 5500, 7270, 2128, 2531, 6386, 6368, 11292, 2136, 5407, 2116, 2116, 5398, 4680, 5431, 4582, 1500, 2627, 2094, 2038, 2310, 2124, 2496, 5438, 1500, 2240, 5451, 6535, 2099, 2626, 1555, 5687, 1177, 1702, 965, 6480, 2643, 2450, 2584, 1570, 2644, 2611, 1575, 4539, 2611, 1641, 2241, 1555, 1702, 3518, 2450, 1570, 3308, 1575, 2043, 3518, 2226, 2549, 2643, 6574, 3382, 5500, 2531, 7688, 2183, 2582, 2398, 2124, 2226, 2093, 3420, 6576, 2142, 4680, 6536, 5687, 1177, 2298, 965, 2608, 1855, 1880, 2175, 7270, 2128, 1500, 1500, 6386, 6368, 11292, 2136, 5407, 2116, 2116, 5398, 4680, 5431, 4582, 1575, 2627, 2094, 2038, 5438, 2549, 6574, 2310, 2496, 5451, 6535, 1575, 2240, 6480, 2643, 2099, 2626, 7688, 2183, 2584, 1555, 2644, 1702, 2611, 2611, 2450, 1570, 2241, 4539, 1641, 1555, 7270, 2128, 1712, 1702, 1154, 2450, 1570, 3308, 2043, 1500, 3518, 3518, 1500, 2582, 2398, 1220, 2226, 2226, 2643, 2093, 1624, 3420, 6576, 3382, 1153, 5500, 6535, 2531, 2124, 4768, 1624, 2142, 1676, 4552, 6480, 6536, 2643, 2584, 2644, 2611, 2298, 2611, 2608, 1855, 1880, 2241, 2175, 5687, 1177, 965, 1555, 1702, 2450, 1570, 3308, 2043, 3518, 2582, 2398, 2226, 2093, 3420, 6576, 2142, 4552, 1575, 1575, 6536, 6386, 2549, 6368, 6574, 1500, 2298, 1500, 7688, 2183, 2608, 11292, 1855, 1880, 2175, 2136, 5407, 2116, 2116, 5398, 4552, 5431, 7270, 4482, 2128, 2627, 2094, 2038, 2310, 5438, 2496, 2240, 5451, 2099, 2626, 1555, 1702, 2450, 1570, 6386, 6368, 6535, 4539, 1641, 11292, 2136, 5407, 6480, 2643, 1575, 2584, 3518, 2644, 2611, 2611, 2116, 2116, 2241, 5398, 2226, 2643, 1555, 1702, 3382, 5500, 4580, 2450, 1570, 5431, 3308, 2043, 2531, 3518, 4482, 2124, 2627, 2094, 2038, 2310, 2496, 5438, 2582, 5451, 2240, 2398, 2226, 5687, 2093, 2099, 3420, 2626, 1177, 1555, 6576, 965, 1702, 2450, 1570, 2142, 4580, 4539, 6536, 1641, 1500, 2298, 1500, 2608, 1855, 1880, 2175, 3518, 2226, 2643, 3382, 5500, 2531, 2124, 1575, 2549, 6574, 5687, 7688, 2183, 1177, 965, 7270, 2128, 6386, 6368, 11292, 2136, 5407, 2116, 2116, 5398, 4630, 1575, 5431, 1500, 1575, 4532, 1500, 2627, 2094, 2038, 5438, 2310, 2496, 2549, 6574, 6535, 2240, 7688, 2183, 2099, 2626, 5451, 6480, 2643, 1555, 2584, 2644, 1702, 2611, 2450, 1570, 2611, 7270, 2241, 2128, 1555, 1702, 4539, 1641, 2450, 1570, 3308, 2043, 3518, 3518, 6535, 6480, 2643, 2582, 2226, 2398, 2226, 2584, 2644, 2643, 2611, 2093, 2611, 3382, 3420, 2241, 5500, 6576, 1500, 1500, 2531, 1555, 2142, 4630, 6536, 2124, 1702, 2450, 1570, 2298, 5687, 2608, 1855, 1880, 2175, 3308, 2043, 1177, 965, 3518, 2582, 2398, 2226, 2093, 3420, 6576, 2142, 4630, 6536, 2298, 2608, 1855, 1880, 2175, 1575, 1575, 6386, 6368, 2549, 6574, 11292, 2136, 7688, 2183, 5407, 2116, 2116, 5398, 4630, 5431, 4532, 2627, 2094, 2038, 2310, 5438, 7270, 2496, 2128, 1500, 1500, 2240, 2099, 5451, 2626, 1555, 6386, 6368, 1702, 2450, 1570, 11292, 1575, 2136, 5407, 4539, 2116, 1641, 2116, 5398, 6535, 3518, 6480, 2643, 4630, 5431, 2226, 2643, 2584, 2644, 2611, 3382, 2611, 2241, 5500, 1555, 4532, 2627, 2094, 2038, 2531, 1702, 2310, 2450, 1570, 2496, 2124, 3308, 5438, 2240, 2043, 3518, 2099, 5451, 2626, 1555, 1702, 2582, 2398, 5687, 2450, 2226, 1570, 1177, 965, 2093, 3420, 6576, 2142, 4630, 4539, 6536, 1641, 1500, 3518, 1500, 2298, 2608, 1855, 1880, 2175, 2226, 2643, 1220, 3382, 5500, 1575, 1676, 2531, 2549, 6574, 1624, 2124, 7688, 2183, 1153, 4741, 1590, 1611, 5687, 1102, 1701, 1177, 965, 2449, 1597, 1066, 7270, 2128, 1575, 6386, 6368, 11292, 2136, 5407, 2116, 2116, 5398, 4525, 5431, 4455, 2627, 2094, 2038, 5438, 2310, 2496, 1500, 2240, 5451, 1500, 2099, 2626, 1555, 1702, 2450, 1570, 1575, 4539, 1641, 2549, 6574, 6535, 3518, 7688, 2183, 6480, 2643, 2584, 2644, 2226, 2611, 2643, 2611, 3382, 2241, 5500, 1555, 2531, 7270, 2124, 2128, 1702, 2450, 1570, 3308, 2043, 3518, 2582, 2398, 2226, 2093, 3420, 6576, 2142, 4553, 6536, 1500, 1500, 2298, 2608, 1855, 1880, 2175, 6535, 5687, 1177, 965, 6480, 2643, 2584, 2644, 2611, 2611, 2241, 1555, 1702, 2450, 1570, 3308, 2043, 3518, 2582, 2398, 2226, 2093, 3420, 6576, 1575, 1575, 6574, 6386, 2549, 2142, 6368, 4553, 11292, 2136, 6536, 5407, 7688, 2183, 2116, 2298, 2116, 5398, 2608, 1855, 1880, 2175, 1500, 1500, 7270, 2128, 4553, 5431, 4455, 6386, 6368, 2627, 2094, 2038, 5438, 2310, 2496, 2240, 6535, 5451, 11292, 2136, 6480, 2643, 5407, 2584, 2099, 2116, 2626, 2644, 1555, 2116, 2611, 5398, 1702, 2611, 1575, 2450, 2241, 4539, 4553, 1570, 1555, 1641, 5431, 1702, 2450, 4455, 1570, 2627, 2094, 2038, 3308, 5438, 2310, 2043, 2496, 3518, 2240, 3518, 5451, 2099, 2626, 2226, 2643, 2582, 2398, 3382, 1555, 5500, 2226, 1702, 2093, 2531, 2450, 3420, 1570, 6576, 2124, 4539, 1641, 2142, 4553, 6536, 2298, 3518, 1500, 2608, 1855, 1880, 2175, 1500, 2226, 2643, 3382, 5500, 5687, 2531, 1177, 965, 2124, 6386, 6368, 11292, 2136, 5407, 1575, 5687, 2549, 6574, 1177, 2116, 965, 2116, 7688, 2183, 5398, 4553, 5431, 1575, 4455, 2627, 2094, 2038, 5438, 2310, 2496, 1500, 7270, 1500, 2128, 2240, 5451, 2099, 2626, 1555, 1702, 2450, 1570, 1575, 2549, 4539, 6574, 1641, 6535, 3518, 7688, 2183, 6480, 2643, 2584, 2226, 2644, 2643, 2611, 3382, 2611, 5500, 2241, 1555, 2531, 1702, 2450, 2124, 1570, 7270, 2128, 3308, 2043, 3518, 2582, 1500, 2398, 2226, 1500, 2093, 5687, 3420, 1177, 6576, 2142, 4553, 965, 6536, 6535, 2298, 2608, 6480, 1855, 2643, 1880, 2175, 2584, 2644, 2611, 1220, 2611, 2241, 1555, 1702, 2450, 1570, 1676, 3308, 2043, 3518, 1575, 2582, 2398, 1624, 2226, 2549, 6574, 2093, 3420, 1153, 6386, 6576, 7688, 6368, 2183, 1575, 4767, 1624, 11292, 2136, 5407, 2142, 4551, 1611, 7270, 2128, 1102, 1701, 1500, 2449, 1500, 6536, 2116, 2116, 5398, 2298, 2608, 1855, 1880, 2175, 4551, 5431, 4481, 2627, 2094, 2038, 5438, 2310, 2496, 5451, 6535, 2240, 2099, 6480, 2643, 2626, 1555, 2584, 2644, 1702, 4539, 2611, 6386, 1641, 2450, 2611, 6368, 1570, 2241, 1555, 1575, 1702, 11292, 2450, 1570, 2136, 5407, 3308, 2043, 3518, 2116, 3518, 2116, 5398, 4579, 2582, 2226, 5431, 2398, 2643, 2226, 2093, 3382, 3420, 5500, 4481, 6576, 2627, 2094, 2038, 5438, 2531, 2310, 2496, 5451, 2142, 2124, 4579, 2240, 6536, 2099, 2626, 1555, 2298, 2608, 1702, 1855, 1880, 2175, 2450, 1570, 4539, 1641, 5687, 1500, 1177, 965, 1500, 3518, 2226, 2643, 3382, 5500, 2531, 2124, 1575, 2549, 6574, 7688, 2183, 5687, 1177, 965, 6386, 6368, 11292, 2136, 1575, 5407, 2116, 2116, 5398, 1500, 1500, 4579, 7270, 2128, 5431, 4481, 1575, 2627, 2094, 2038, 5438, 2549, 2310, 6574, 2496, 6535, 5451, 2240, 7688, 2183, 2099, 6457, 2643, 2626, 1555, 2584, 4539, 2644, 2611, 1641, 1702, 7270, 2128, 2611, 2450, 2241, 1570, 1555, 1500, 1500, 1702, 2450, 1570, 3308, 2043, 3518, 3518, 6535, 2582, 2398, 2226, 2643, 6480, 2643, 3382, 2226, 5500, 2584, 2644, 2093, 3420, 2611, 6553, 2531, 2611, 2124, 2241, 2142, 4579, 1555, 6513, 1702, 2298, 2450, 1570, 2608, 1855, 1880, 2175, 3308, 2043, 3518, 5687, 1177, 965, 2582, 2398, 2226, 2093, 3420, 6576, 2142, 4579, 6536, 2298, 2608, 1855, 1880, 2175, 1575, 1575, 2549, 6574, 6386, 6368, 7688, 2183, 11292, 2136, 5407, 1500, 2116, 1500, 2116, 5398, 4579, 5431, 4481, 2627, 2094, 2038, 5438, 2310, 2496, 2240, 5451, 2099, 2626, 7270, 2128, 1555, 1575, 1702, 2450, 4539, 1570, 6386, 1641, 6368, 11292, 2136, 6535, 5407, 6480, 2643, 2116, 2116, 3518, 2584, 5398, 2644, 2611, 2226, 2643, 4629, 2611, 5431, 3382, 2241, 5500, 4531, 1555, 2531, 2627, 2094, 2038, 1702, 2310, 5438, 2450, 2496, 2124, 1570, 3308, 2240, 2043, 3518, 5451, 2099, 1500, 2626, 1500, 1555, 5687, 1702, 1177, 2450, 2582, 965, 1570, 2398, 2226, 2093, 3420, 6576, 4539, 1641, 2142, 4629, 6536, 3518, 2298, 2608, 1855, 1880, 2175, 2226, 2643, 3382, 5500, 1575, 1220, 2531, 1676, 2549, 6574, 2124, 1624, 7688, 2183, 1153, 4769, 1624, 1611, 1102, 1701, 5687, 2449, 1177, 1597, 965, 1066, 7270, 2128, 1500, 6386, 1500, 6368, 11292, 2136, 5407, 2116, 2116, 5398, 1575, 4553, 5431, 4483, 2627, 2094, 2038, 5438, 2310, 2496, 2240, 2099, 2626, 1555, 1702, 2450, 1570, 1575, 5451, 6535, 6574, 2549, 6480, 2643, 3518, 2584, 2644, 7688, 2183, 2226, 2611, 2643, 2611, 5710, 2241, 3382, 1641, 1555, 6671, 1702, 2450, 1570, 3308, 2531, 2043, 3518, 2124, 1500, 2582, 1500, 2398, 2226, 2093, 3420, 7270, 2128, 6576, 2142, 6858, 4581, 1177, 6536, 2298, 965, 2608, 6535, 1855, 1880, 2175, 6480, 2643, 2584, 2644, 2611, 2611, 2241, 1555, 1702, 2450, 1570, 3308, 2043, 3518, 2582, 2398, 2226, 2093, 3420, 6576, 1575, 2142, 4581, 2549, 7745, 6536, 1575, 2298, 2608, 7688, 2183, 1855, 1880, 2175, 6386, 6368, 1500, 1500, 11292, 2136, 5407, 7270, 2128, 2116, 2116, 5398, 4631, 6386, 6368, 5431, 11292, 2136, 5407, 4533, 2627, 2094, 2038, 2310, 2496, 2116, 5438, 2116, 5398, 2240, 2099, 6535, 2626, 6480, 2643, 5451, 2584, 2644, 4631, 1555, 5431, 2611, 4533, 2627, 2094, 2038, 1702, 2310, 2496, 2611, 2241, 2450, 1570, 2240, 5438, 2099, 2626, 1555, 5451, 1555, 1702, 4539, 1641, 1702, 2450, 2450, 1570, 1570, 3518, 3308, 2043, 3518, 2226, 1575, 2643, 4539, 3382, 5500, 2582, 2398, 3518, 2226, 1641, 2226, 2093, 3420, 2643, 6576, 2531, 3382, 2124, 5500, 2142, 4631, 6536, 2531, 2298, 2608, 1855, 1880, 2175, 2124, 5687, 1177, 965, 1500, 1500, 1575, 5687, 1177, 2549, 965, 6574, 7688, 2183, 7270, 2128, 1575, 6386, 6368, 11292, 2136, 5407, 2116, 2116, 5398, 1575, 4631, 1500, 2549, 1500, 5431, 6574, 6535, 4533, 2627, 2094, 2038, 7688, 2183, 2310, 6480, 2643, 2496, 5438, 2240, 2584, 2099, 2626, 2644, 2611, 5451, 1555, 2611, 1702, 2241, 2450, 1570, 1555, 1702, 2450, 1570, 7270, 3308, 2128, 4539, 2043, 3518, 1641, 3518, 2582, 2226, 2398, 2643, 2226, 2093, 3382, 3420, 5500, 6576, 2531, 2142, 4631, 2124, 6536, 6535, 2298, 2608, 6480, 1855, 2643, 1880, 2175, 2584, 2644, 2611, 2611, 2241, 5687, 1177, 1555, 965, 1702, 2450, 1570, 3308, 2043, 3518, 1500, 1500, 2582, 2398, 2226, 2093, 3420, 6576, 2142, 4631, 6536, 2298, 1575, 2608, 1855, 1880, 2175, 6574, 1575, 1676, 7688, 2183, 1220, 2549, 1624, 1153, 4691, 6386, 6368, 1590, 1611, 7270, 2128, 1102, 1701, 11292, 2136, 2449, 5407, 1500, 1500, 2116, 2116, 5398, 4549, 5431, 6535, 6386, 6480, 6368, 2643, 4479, 2627, 2094, 2038, 2584, 2644, 5438, 1575, 2310, 5451, 2496, 2611, 2240, 2099, 2611, 2241, 2626, 11292, 2136, 1555, 5407, 1702, 2450, 1555, 1702, 2116, 1570, 2116, 2450, 5398, 4539, 1570, 1641, 4577, 3308, 5431, 2043, 3518, 3518, 4479, 2226, 2627, 2094, 2038, 5438, 2643, 2310, 3382, 5500, 2496, 2582, 5451, 2240, 2398, 2099, 2531, 2626, 1555, 2226, 2093, 1702, 2124, 3420, 2450, 1570, 6576, 2142, 4577, 6536, 4539, 1641, 2298, 5687, 2608, 1855, 1880, 2175, 1177, 965, 3518, 2226, 2643, 3382, 5500, 2531, 2124, 1500, 1500, 1575, 5687, 2549, 1177, 6574, 965, 7688, 2183, 6386, 6368, 1575, 7270, 2128, 11292, 2136, 5407, 2116, 2116, 5398, 4577, 5431, 4479, 1575, 2627, 2094, 2038, 5438, 2549, 2310, 6574, 2496, 6535, 1500, 5451, 1500, 2240, 6480, 2643, 7688, 2183, 2584, 2099, 2644, 2626, 1555, 2611, 2611, 1702, 4539, 2450, 2241, 1570, 1641, 1555, 1702, 2450, 1570, 3308, 7270, 2043, 2128, 3518, 3518, 2582, 2398, 2226, 2226, 2643, 2093, 3382, 3420, 5500, 6576, 2142, 2531, 4577, 6536, 6535, 6480, 2643, 2124, 2584, 2644, 2298, 2608, 2611, 1855, 1880, 2175, 2611, 2241, 1555, 1702, 2450, 1570, 5687, 3308, 1177, 2043, 965, 3518, 2582, 2398, 2226, 2093, 3420, 6576, 1500, 2142, 1500, 4577, 6536, 2298, 2608, 1855, 1880, 2175, 1575, 2549, 6574, 7688, 2183, 1575, 6386, 6368, 11292, 2136, 5407, 2116, 2116, 5398, 7270, 2128, 4627, 5431, 4529, 2627, 2094, 2038, 5438, 2310, 2496, 6386, 6368, 6535, 11292, 2136, 5407, 2240, 2099, 5451, 2626, 6480, 2643, 1555, 2584, 2116, 2644, 1702, 2611, 2116, 2450, 5398, 2611, 1570, 2241, 4539, 4627, 1641, 1555, 1500, 5431, 1500, 1702, 2450, 4529, 1570, 2627, 2094, 3518, 2038, 5438, 3308, 2310, 2043, 3518, 2226, 2496, 2643, 3382, 5451, 1575, 2240, 5500, 2582, 2398, 2226, 2099, 2626, 2093, 3420, 1555, 2531, 6576, 2124, 1702, 4539, 2450, 2142, 1570, 1641, 4627, 6536, 2298, 2608, 1855, 1880, 2175, 5687, 1177, 965, 3518, 2226, 2643, 3382, 5500, 2531, 1575, 2124, 2549, 6574, 6386, 7688, 2183, 6368, 1568, 5687, 1177, 11292, 965, 2136, 5407, 1500, 1500, 2116, 2116, 5398, 7270, 2128, 1712, 1575, 4627, 1154, 5431, 4529, 2627, 2094, 2038, 2310, 5438, 2496, 2240, 5451, 1676, 2099, 2626, 1555, 1220, 1702, 2450, 1575, 1570, 2549, 6574, 6535, 1624, 4539, 7688, 2183, 1641, 1500, 1500, 6480, 2643, 3518, 1153, 2584, 2644, 2226, 4817, 2611, 2643, 2611, 1590, 3382, 2241, 5500, 1624, 1555, 2559, 2561, 2559, 2531, 1702, 2124, 7270, 2579, 2579, 2450, 1611, 1570, 2128, 3308, 1102, 1701, 2449, 2043, 3518, 1597, 1106, 2582, 5687, 2398, 2226, 1177, 2093, 3420, 6576, 965, 6535, 2142, 4601, 6536, 6480, 2643, 2584, 2644, 2298, 1500, 2608, 1500, 2611, 1855, 1880, 2175, 2611, 2241, 1555, 1702, 2450, 1570, 1575, 3308, 2043, 3518, 1575, 2549, 6574, 2582, 2398, 2226, 7688, 2093, 2183, 3420, 6576, 2142, 4601, 6536, 2298, 6386, 2608, 6368, 1855, 1880, 2175, 7270, 2128, 11292, 2136, 5407, 2116, 2116, 5398, 4601, 5431, 4531, 2627, 2094, 2038, 2310, 5438, 2496, 2240, 1500, 5451, 1500, 6535, 2099, 2626, 1555, 6480, 2643, 2584, 1702, 2644, 2450, 2611, 1570, 2611, 2241, 1555, 4539, 1641, 1702, 2450, 1570, 3308, 2043, 3518, 3518, 2582, 2226, 2398, 2643, 2226, 2093, 3382, 3420, 5500, 6576, 2531, 2142, 4629, 2124, 6536, 2298, 2608, 1855, 1880, 2175, 5687, 1177, 965, 1575, 1575, 2549, 6574, 7688, 2183, 7270, 2128, 6535, 6480, 2643, 2584, 2644, 2611, 2611, 2241, 1555, 1702, 2450, 1570, 3308, 2043, 3518, 1500, 1500, 2582, 2398, 2226, 2093, 3420, 6576, 2142, 4629, 6536, 2298, 2608, 1855, 1880, 2175, 1575, 7291, 2128, 6534, 6479, 2643, 2584, 2644, 2611, 2611, 2241, 1555, 1702, 2450, 1570, 3308, 2043, 3518, 2582, 2398, 2226, 2093, 3420, 6576, 2142, 4629, 7291, 2128, 6536, 2298, 6534, 2608, 1855, 1880, 2175, 6479, 2643, 2584, 2644, 2611, 2611, 2241, 1555, 1702, 2450, 1570, 3308, 2043, 3518, 2582, 2398, 2226, 1500, 2093, 3420, 1500, 6576, 2142, 4629, 6536, 2298, 2608, 1855, 1880, 2175, 1575, 1500, 1500, 1220, 1624, 1153, 4412, 1676, 1590, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1676, 1220, 1624, 1153, 4412, 1597, 908, 1590, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1676, 1220, 1624, 1153, 1500, 4412, 1500, 1590, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1597, 908, 1500, 1500, 1676, 1220, 1624, 1153, 4412, 1590, 1500, 1500, 1500, 1500, 1500, 1500 \ No newline at end of file diff --git a/tools/simulator.py b/tools/simulator.py index 8b8fc0bbc..d3dda3eec 100755 --- a/tools/simulator.py +++ b/tools/simulator.py @@ -13,14 +13,19 @@ import eventlet eventlet.monkey_patch() -import os - import argparse +import collections import datetime import logging +import os +import random +import string import sys import threading import time +import yaml + +from scipy.stats import rv_discrete from oslo_config import cfg import oslo_messaging as messaging @@ -28,6 +33,8 @@ from oslo_messaging import notify # noqa from oslo_messaging import rpc # noqa LOG = logging.getLogger() +RANDOM_VARIABLE = None +CURRENT_PID = None USAGE = """ Usage: ./simulator.py [-h] [--url URL] [-d DEBUG]\ {notify-server,notify-client,rpc-server,rpc-client} ... @@ -40,6 +47,29 @@ Usage example: --exit-wait 15000 -p 64 -m 64""" +def init_random_generator(): + data = [] + with open('./messages_length.yaml') as m_file: + content = yaml.load(m_file) + data += [int(n) for n in content[ + 'test_data']['string_lengths'].split(', ')] + + ranges = collections.defaultdict(int) + for msg_length in data: + range_start = (msg_length / 500) * 500 + 1 + ranges[range_start] += 1 + + ranges_start = sorted(ranges.keys()) + total_count = len(data) + ranges_dist = [] + for r in ranges_start: + r_dist = float(ranges[r]) / total_count + ranges_dist.append(r_dist) + + random_var = rv_discrete(values=(ranges_start, ranges_dist)) + return random_var + + class LoggingNoParsingFilter(logging.Filter): def filter(self, record): msg = record.getMessage() @@ -136,9 +166,19 @@ def send_msg(_id, transport, target, messages, wait_after_msg, timeout, client = client.prepare(timeout=timeout) rpc_method = _rpc_cast if is_cast else _rpc_call - for i in range(0, messages): - msg = "test message %d" % i - LOG.info("SEND: %s" % msg) + ranges = RANDOM_VARIABLE.rvs(size=messages) + i = 0 + for range_start in ranges: + length = random.randint(range_start, range_start + 497) + msg = ''.join(random.choice(string.lowercase) for x in range(length)) \ + + ' ' + str(i) + i += 1 + # temporary file to log approximate bytes size of messages + with open('./oslo_%s_%s.log' % (target.topic, CURRENT_PID), 'a+') as f: + # 37 additional bytes for Python String object size canculation. + # In fact we may ignore these bytes, and estimate the data flow + # via number of symbols + f.write(str(length + 37) + '\n') rpc_method(client, msg) if wait_after_msg > 0: time.sleep(wait_after_msg) @@ -197,6 +237,9 @@ def main(): parser.add_argument('-d', '--debug', dest='debug', type=bool, default=False, help="Turn on DEBUG logging level instead of WARN") + parser.add_argument('-tp', '--topic', dest='topic', + default="profiler_topic", + help="Topic to publish/receive messages to/from.") subparsers = parser.add_subparsers(dest='mode', help='notify/rpc server/client mode') @@ -246,7 +289,7 @@ def main(): cfg.CONF.project = 'oslo.messaging' transport = messaging.get_transport(cfg.CONF, url=args.url) - target = messaging.Target(topic='profiler_topic', server='profiler_server') + target = messaging.Target(topic=args.topic, server='profiler_server') if args.mode == 'rpc-server': if args.url.startswith('zmq'): @@ -266,11 +309,29 @@ def main(): args.is_cast) time_ellapsed = (datetime.datetime.now() - start).total_seconds() msg_count = args.messages * args.threads - print ('%d messages was sent for %s seconds. Bandwight is %s msg/sec' - % (msg_count, time_ellapsed, (msg_count / time_ellapsed))) + log_msg = '%d messages was sent for %s seconds. ' \ + 'Bandwidth is %s msg/sec' % (msg_count, time_ellapsed, + (msg_count / time_ellapsed)) + print (log_msg) + with open('./oslo_res_%s.txt' % args.topic, 'a+') as f: + f.write(log_msg + '\n') + + with open('./oslo_%s_%s.log' % (args.topic, CURRENT_PID), 'a+') as f: + data = f.read() + data = [int(i) for i in data.split()] + data_sum = sum(data) + log_msg = '%s bytes were sent for %s seconds. Bandwidth is %s b/s' % ( + data_sum, time_ellapsed, (data_sum / time_ellapsed)) + print(log_msg) + with open('./oslo_res_%s.txt' % args.topic, 'a+') as f: + f.write(log_msg + '\n') + os.remove('./oslo_%s_%s.log' % (args.topic, CURRENT_PID)) + LOG.info("calls finished, wait %d seconds" % args.exit_wait) time.sleep(args.exit_wait) if __name__ == '__main__': + RANDOM_VARIABLE = init_random_generator() + CURRENT_PID = os.getpid() main() From a162d65a23814c5dc8d7bbc27a5db65dab3e7a33 Mon Sep 17 00:00:00 2001 From: OpenStack Proposal Bot Date: Fri, 23 Oct 2015 06:27:02 +0000 Subject: [PATCH 11/74] Imported Translations from Zanata For more information about this automatic import see: https://wiki.openstack.org/wiki/Translations/Infrastructure Change-Id: I260a5e3ae5d394c8c01dc80751f979296650af18 --- .../LC_MESSAGES/oslo.messaging-log-error.po | 9 +++---- .../LC_MESSAGES/oslo.messaging-log-info.po | 9 +++---- .../LC_MESSAGES/oslo.messaging-log-warning.po | 12 +++------ .../LC_MESSAGES/oslo.messaging-log-error.po | 13 +++++----- .../LC_MESSAGES/oslo.messaging-log-warning.po | 26 ------------------- .../LC_MESSAGES/oslo.messaging-log-error.po | 11 ++++---- .../LC_MESSAGES/oslo.messaging-log-warning.po | 26 ------------------- .../locale/oslo.messaging-log-critical.pot | 20 -------------- .../locale/oslo.messaging-log-warning.pot | 18 ++++++++----- .../LC_MESSAGES/oslo.messaging-log-error.po | 11 ++++---- 10 files changed, 42 insertions(+), 113 deletions(-) delete mode 100644 oslo.messaging/locale/es/LC_MESSAGES/oslo.messaging-log-warning.po delete mode 100644 oslo.messaging/locale/fr/LC_MESSAGES/oslo.messaging-log-warning.po delete mode 100644 oslo.messaging/locale/oslo.messaging-log-critical.pot diff --git a/oslo.messaging/locale/en_GB/LC_MESSAGES/oslo.messaging-log-error.po b/oslo.messaging/locale/en_GB/LC_MESSAGES/oslo.messaging-log-error.po index 1fce8b65e..2a8f94dba 100644 --- a/oslo.messaging/locale/en_GB/LC_MESSAGES/oslo.messaging-log-error.po +++ b/oslo.messaging/locale/en_GB/LC_MESSAGES/oslo.messaging-log-error.po @@ -8,19 +8,18 @@ # OpenStack Infra , 2015. #zanata msgid "" msgstr "" -"Project-Id-Version: oslo.messaging 2.5.1.dev7\n" +"Project-Id-Version: oslo.messaging 2.7.1.dev15\n" "Report-Msgid-Bugs-To: EMAIL@ADDRESS\n" -"POT-Creation-Date: 2015-09-16 18:44+0000\n" +"POT-Creation-Date: 2015-10-23 06:27+0000\n" "PO-Revision-Date: 2015-08-27 12:47+0000\n" "Last-Translator: Andi Chandler \n" -"Language-Team: English (United Kingdom) (http://www.transifex.com/openstack/" -"oslomessaging/language/en_GB/)\n" +"Language-Team: English (United Kingdom)\n" "Language: en-GB\n" "MIME-Version: 1.0\n" "Content-Type: text/plain; charset=UTF-8\n" "Content-Transfer-Encoding: 8bit\n" -"Generated-By: Babel 2.0\n" "Plural-Forms: nplurals=2; plural=(n != 1);\n" +"Generated-By: Babel 2.0\n" "X-Generator: Zanata 3.7.1\n" #, python-format diff --git a/oslo.messaging/locale/en_GB/LC_MESSAGES/oslo.messaging-log-info.po b/oslo.messaging/locale/en_GB/LC_MESSAGES/oslo.messaging-log-info.po index 9280d3220..6a9211d95 100644 --- a/oslo.messaging/locale/en_GB/LC_MESSAGES/oslo.messaging-log-info.po +++ b/oslo.messaging/locale/en_GB/LC_MESSAGES/oslo.messaging-log-info.po @@ -8,19 +8,18 @@ # OpenStack Infra , 2015. #zanata msgid "" msgstr "" -"Project-Id-Version: oslo.messaging 2.5.1.dev7\n" +"Project-Id-Version: oslo.messaging 2.7.1.dev15\n" "Report-Msgid-Bugs-To: EMAIL@ADDRESS\n" -"POT-Creation-Date: 2015-09-16 18:44+0000\n" +"POT-Creation-Date: 2015-10-23 06:26+0000\n" "PO-Revision-Date: 2015-08-27 12:47+0000\n" "Last-Translator: Andi Chandler \n" -"Language-Team: English (United Kingdom) (http://www.transifex.com/openstack/" -"oslomessaging/language/en_GB/)\n" +"Language-Team: English (United Kingdom)\n" "Language: en-GB\n" "MIME-Version: 1.0\n" "Content-Type: text/plain; charset=UTF-8\n" "Content-Transfer-Encoding: 8bit\n" -"Generated-By: Babel 2.0\n" "Plural-Forms: nplurals=2; plural=(n != 1);\n" +"Generated-By: Babel 2.0\n" "X-Generator: Zanata 3.7.1\n" #, python-format diff --git a/oslo.messaging/locale/en_GB/LC_MESSAGES/oslo.messaging-log-warning.po b/oslo.messaging/locale/en_GB/LC_MESSAGES/oslo.messaging-log-warning.po index 61e720610..ca64daf3a 100644 --- a/oslo.messaging/locale/en_GB/LC_MESSAGES/oslo.messaging-log-warning.po +++ b/oslo.messaging/locale/en_GB/LC_MESSAGES/oslo.messaging-log-warning.po @@ -8,28 +8,24 @@ # OpenStack Infra , 2015. #zanata msgid "" msgstr "" -"Project-Id-Version: oslo.messaging 2.5.1.dev7\n" +"Project-Id-Version: oslo.messaging 2.7.1.dev15\n" "Report-Msgid-Bugs-To: EMAIL@ADDRESS\n" -"POT-Creation-Date: 2015-09-16 18:44+0000\n" +"POT-Creation-Date: 2015-10-23 06:27+0000\n" "PO-Revision-Date: 2015-08-27 12:55+0000\n" "Last-Translator: Andi Chandler \n" -"Language-Team: English (United Kingdom) (http://www.transifex.com/openstack/" -"oslomessaging/language/en_GB/)\n" +"Language-Team: English (United Kingdom)\n" "Language: en-GB\n" "MIME-Version: 1.0\n" "Content-Type: text/plain; charset=UTF-8\n" "Content-Transfer-Encoding: 8bit\n" -"Generated-By: Babel 2.0\n" "Plural-Forms: nplurals=2; plural=(n != 1);\n" +"Generated-By: Babel 2.0\n" "X-Generator: Zanata 3.7.1\n" #, python-format msgid "Failed to load any notifiers for %s" msgstr "Failed to load any notifiers for %s" -msgid "start/stop/wait must be called in the same thread" -msgstr "start/stop/wait must be called in the same thread" - msgid "" "wait() should be called after stop() as it waits for existing messages to " "finish processing" diff --git a/oslo.messaging/locale/es/LC_MESSAGES/oslo.messaging-log-error.po b/oslo.messaging/locale/es/LC_MESSAGES/oslo.messaging-log-error.po index a0ff82857..18978068a 100644 --- a/oslo.messaging/locale/es/LC_MESSAGES/oslo.messaging-log-error.po +++ b/oslo.messaging/locale/es/LC_MESSAGES/oslo.messaging-log-error.po @@ -6,21 +6,22 @@ # Translators: # Adriana Chisco Landazábal , 2015 # Miriam Godinez , 2015 +# OpenStack Infra , 2015. #zanata msgid "" msgstr "" -"Project-Id-Version: oslo.messaging\n" +"Project-Id-Version: oslo.messaging 2.7.1.dev15\n" "Report-Msgid-Bugs-To: EMAIL@ADDRESS\n" -"POT-Creation-Date: 2015-09-08 06:18+0000\n" -"PO-Revision-Date: 2015-09-07 22:46+0000\n" +"POT-Creation-Date: 2015-10-23 06:27+0000\n" +"PO-Revision-Date: 2015-09-07 10:46+0000\n" "Last-Translator: Miriam Godinez \n" -"Language-Team: Spanish (http://www.transifex.com/openstack/oslomessaging/" -"language/es/)\n" +"Language-Team: Spanish\n" "Language: es\n" "MIME-Version: 1.0\n" "Content-Type: text/plain; charset=UTF-8\n" "Content-Transfer-Encoding: 8bit\n" -"Generated-By: Babel 2.0\n" "Plural-Forms: nplurals=2; plural=(n != 1);\n" +"Generated-By: Babel 2.0\n" +"X-Generator: Zanata 3.7.1\n" #, python-format msgid "An exception occurred processing the API call: %s " diff --git a/oslo.messaging/locale/es/LC_MESSAGES/oslo.messaging-log-warning.po b/oslo.messaging/locale/es/LC_MESSAGES/oslo.messaging-log-warning.po deleted file mode 100644 index f7970147c..000000000 --- a/oslo.messaging/locale/es/LC_MESSAGES/oslo.messaging-log-warning.po +++ /dev/null @@ -1,26 +0,0 @@ -# Translations template for oslo.messaging. -# Copyright (C) 2015 ORGANIZATION -# This file is distributed under the same license as the oslo.messaging -# project. -# -# Translators: -# Adriana Chisco Landazábal , 2015 -# Lucía Pradillos , 2015 -msgid "" -msgstr "" -"Project-Id-Version: oslo.messaging\n" -"Report-Msgid-Bugs-To: EMAIL@ADDRESS\n" -"POT-Creation-Date: 2015-08-26 06:33+0000\n" -"PO-Revision-Date: 2015-08-26 03:46+0000\n" -"Last-Translator: Lucía Pradillos \n" -"Language-Team: Spanish (http://www.transifex.com/openstack/oslomessaging/" -"language/es/)\n" -"Language: es\n" -"MIME-Version: 1.0\n" -"Content-Type: text/plain; charset=UTF-8\n" -"Content-Transfer-Encoding: 8bit\n" -"Generated-By: Babel 2.0\n" -"Plural-Forms: nplurals=2; plural=(n != 1);\n" - -msgid "start/stop/wait must be called in the same thread" -msgstr "empezar/parar/esperar debe ser llamado en el mismo hilo" diff --git a/oslo.messaging/locale/fr/LC_MESSAGES/oslo.messaging-log-error.po b/oslo.messaging/locale/fr/LC_MESSAGES/oslo.messaging-log-error.po index d6931b3e2..585e8b228 100644 --- a/oslo.messaging/locale/fr/LC_MESSAGES/oslo.messaging-log-error.po +++ b/oslo.messaging/locale/fr/LC_MESSAGES/oslo.messaging-log-error.po @@ -5,21 +5,22 @@ # # Translators: # Maxime COQUEREL , 2014 +# OpenStack Infra , 2015. #zanata msgid "" msgstr "" -"Project-Id-Version: oslo.messaging\n" +"Project-Id-Version: oslo.messaging 2.7.1.dev15\n" "Report-Msgid-Bugs-To: EMAIL@ADDRESS\n" -"POT-Creation-Date: 2015-08-04 06:29+0000\n" +"POT-Creation-Date: 2015-10-23 06:27+0000\n" "PO-Revision-Date: 2014-09-25 08:57+0000\n" "Last-Translator: Maxime COQUEREL \n" -"Language-Team: French (http://www.transifex.com/openstack/oslomessaging/" -"language/fr/)\n" +"Language-Team: French\n" "Language: fr\n" "MIME-Version: 1.0\n" "Content-Type: text/plain; charset=UTF-8\n" "Content-Transfer-Encoding: 8bit\n" -"Generated-By: Babel 2.0\n" "Plural-Forms: nplurals=2; plural=(n > 1);\n" +"Generated-By: Babel 2.0\n" +"X-Generator: Zanata 3.7.1\n" #, python-format msgid "An exception occurred processing the API call: %s " diff --git a/oslo.messaging/locale/fr/LC_MESSAGES/oslo.messaging-log-warning.po b/oslo.messaging/locale/fr/LC_MESSAGES/oslo.messaging-log-warning.po deleted file mode 100644 index c7a5cc23d..000000000 --- a/oslo.messaging/locale/fr/LC_MESSAGES/oslo.messaging-log-warning.po +++ /dev/null @@ -1,26 +0,0 @@ -# Translations template for oslo.messaging. -# Copyright (C) 2015 ORGANIZATION -# This file is distributed under the same license as the oslo.messaging -# project. -# -# Translators: -# Lucas Mascaro , 2015 -# Maxime COQUEREL , 2014 -msgid "" -msgstr "" -"Project-Id-Version: oslo.messaging\n" -"Report-Msgid-Bugs-To: EMAIL@ADDRESS\n" -"POT-Creation-Date: 2015-08-18 06:33+0000\n" -"PO-Revision-Date: 2015-08-17 22:45+0000\n" -"Last-Translator: Lucas Mascaro \n" -"Language-Team: French (http://www.transifex.com/openstack/oslomessaging/" -"language/fr/)\n" -"Language: fr\n" -"MIME-Version: 1.0\n" -"Content-Type: text/plain; charset=UTF-8\n" -"Content-Transfer-Encoding: 8bit\n" -"Generated-By: Babel 2.0\n" -"Plural-Forms: nplurals=2; plural=(n > 1);\n" - -msgid "start/stop/wait must be called in the same thread" -msgstr "start/stop/wait doivent être appellés dans le même thread " diff --git a/oslo.messaging/locale/oslo.messaging-log-critical.pot b/oslo.messaging/locale/oslo.messaging-log-critical.pot deleted file mode 100644 index d921c5a9c..000000000 --- a/oslo.messaging/locale/oslo.messaging-log-critical.pot +++ /dev/null @@ -1,20 +0,0 @@ -# Translations template for oslo.messaging. -# Copyright (C) 2015 ORGANIZATION -# This file is distributed under the same license as the oslo.messaging -# project. -# FIRST AUTHOR , 2015. -# -#, fuzzy -msgid "" -msgstr "" -"Project-Id-Version: oslo.messaging 2.1.0\n" -"Report-Msgid-Bugs-To: EMAIL@ADDRESS\n" -"POT-Creation-Date: 2015-07-29 06:39+0000\n" -"PO-Revision-Date: YEAR-MO-DA HO:MI+ZONE\n" -"Last-Translator: FULL NAME \n" -"Language-Team: LANGUAGE \n" -"MIME-Version: 1.0\n" -"Content-Type: text/plain; charset=utf-8\n" -"Content-Transfer-Encoding: 8bit\n" -"Generated-By: Babel 2.0\n" - diff --git a/oslo.messaging/locale/oslo.messaging-log-warning.pot b/oslo.messaging/locale/oslo.messaging-log-warning.pot index de9167049..2dbd51efd 100644 --- a/oslo.messaging/locale/oslo.messaging-log-warning.pot +++ b/oslo.messaging/locale/oslo.messaging-log-warning.pot @@ -7,20 +7,16 @@ #, fuzzy msgid "" msgstr "" -"Project-Id-Version: oslo.messaging 2.4.1.dev1\n" +"Project-Id-Version: oslo.messaging 2.7.1.dev15\n" "Report-Msgid-Bugs-To: EMAIL@ADDRESS\n" -"POT-Creation-Date: 2015-08-26 06:33+0000\n" +"POT-Creation-Date: 2015-10-23 06:27+0000\n" "PO-Revision-Date: YEAR-MO-DA HO:MI+ZONE\n" "Last-Translator: FULL NAME \n" "Language-Team: LANGUAGE \n" "MIME-Version: 1.0\n" "Content-Type: text/plain; charset=utf-8\n" "Content-Transfer-Encoding: 8bit\n" -"Generated-By: Babel 2.0\n" - -#: oslo_messaging/server.py:145 -msgid "start/stop/wait must be called in the same thread" -msgstr "" +"Generated-By: Babel 2.1.1\n" #: oslo_messaging/server.py:178 msgid "" @@ -28,6 +24,14 @@ msgid "" " finish processing" msgstr "" +#: oslo_messaging/server.py:191 +#, python-format +msgid "" +"wait() should have been called after stop() as wait() waits for existing " +"messages to finish processing, it has been %0.2f seconds and stop() still" +" has not been called" +msgstr "" + #: oslo_messaging/notify/_impl_routing.py:80 #, python-format msgid "Failed to load any notifiers for %s" diff --git a/oslo.messaging/locale/ru/LC_MESSAGES/oslo.messaging-log-error.po b/oslo.messaging/locale/ru/LC_MESSAGES/oslo.messaging-log-error.po index 14a6dc7b6..cf613ada1 100644 --- a/oslo.messaging/locale/ru/LC_MESSAGES/oslo.messaging-log-error.po +++ b/oslo.messaging/locale/ru/LC_MESSAGES/oslo.messaging-log-error.po @@ -5,23 +5,24 @@ # # Translators: # kogamatranslator49 , 2015 +# OpenStack Infra , 2015. #zanata msgid "" msgstr "" -"Project-Id-Version: oslo.messaging\n" +"Project-Id-Version: oslo.messaging 2.7.1.dev15\n" "Report-Msgid-Bugs-To: EMAIL@ADDRESS\n" -"POT-Creation-Date: 2015-08-04 06:29+0000\n" +"POT-Creation-Date: 2015-10-23 06:27+0000\n" "PO-Revision-Date: 2015-07-05 11:39+0000\n" "Last-Translator: kogamatranslator49 \n" -"Language-Team: Russian (http://www.transifex.com/openstack/oslomessaging/" -"language/ru/)\n" +"Language-Team: Russian\n" "Language: ru\n" "MIME-Version: 1.0\n" "Content-Type: text/plain; charset=UTF-8\n" "Content-Transfer-Encoding: 8bit\n" -"Generated-By: Babel 2.0\n" "Plural-Forms: nplurals=4; plural=(n%10==1 && n%100!=11 ? 0 : n%10>=2 && n" "%10<=4 && (n%100<12 || n%100>14) ? 1 : n%10==0 || (n%10>=5 && n%10<=9) || (n" "%100>=11 && n%100<=14)? 2 : 3);\n" +"Generated-By: Babel 2.0\n" +"X-Generator: Zanata 3.7.1\n" #, python-format msgid "An exception occurred processing the API call: %s " From 52e624891fc500c8ab9f3f10ef45258ce740916a Mon Sep 17 00:00:00 2001 From: Cyril Roelandt Date: Fri, 23 Oct 2015 15:01:33 +0200 Subject: [PATCH 12/74] Use "secret=True" for password-related options This makes sure the value of the option is not leaked in the logs. Found using bandit. Change-Id: I6db2eea1d3f1ad3cacb749dbb9766c5d32cf047f --- oslo_messaging/_drivers/protocols/amqp/opts.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/oslo_messaging/_drivers/protocols/amqp/opts.py b/oslo_messaging/_drivers/protocols/amqp/opts.py index cba1fd339..5c69c966c 100644 --- a/oslo_messaging/_drivers/protocols/amqp/opts.py +++ b/oslo_messaging/_drivers/protocols/amqp/opts.py @@ -64,6 +64,7 @@ amqp1_opts = [ cfg.StrOpt('ssl_key_password', default=None, deprecated_group='amqp1', + secret=True, help='Password for decrypting ssl_key_file (if encrypted)'), cfg.BoolOpt('allow_insecure_clients', @@ -94,5 +95,6 @@ amqp1_opts = [ cfg.StrOpt('password', default='', deprecated_group='amqp1', + secret=True, help='Password for message broker authentication') ] From d700c382791b6352bb80a0dc455589085881669f Mon Sep 17 00:00:00 2001 From: Matthew Booth Date: Mon, 19 Oct 2015 14:11:23 +0100 Subject: [PATCH 13/74] Robustify locking in MessageHandlingServer This change formalises locking in MessageHandlingServer. It allows the user to make calls in any order and it will ensure, with locking, that these will be reordered appropriately. It also adds locking for internal state when using the blocking executor, which closes a number of races. It fixes a regression introduced in change gI3cfbe1bf02d451e379b1dcc23dacb0139c03be76. If multiple threads called wait() simultaneously, only 1 of them would wait and the others would return immediately, despite message handling not having completed. With this change only 1 will call the underlying wait, but all will wait on its completion. We add a common logging mechanism when waiting too long. Specifically, we now log a single message when waiting on any lock for longer than 30 seconds. We remove DummyCondition as it no longer has any users. Change-Id: I9d516b208446963dcd80b75e2d5a2cecb1187efa --- oslo_messaging/_utils.py | 23 --- oslo_messaging/server.py | 260 +++++++++++++++++------- oslo_messaging/tests/rpc/test_server.py | 210 +++++++++++++++++++ 3 files changed, 402 insertions(+), 91 deletions(-) diff --git a/oslo_messaging/_utils.py b/oslo_messaging/_utils.py index 1bb20b089..cec94bb48 100644 --- a/oslo_messaging/_utils.py +++ b/oslo_messaging/_utils.py @@ -116,29 +116,6 @@ def fetch_current_thread_functor(): return lambda: threading.current_thread() -class DummyCondition(object): - def acquire(self): - pass - - def notify(self): - pass - - def notify_all(self): - pass - - def wait(self, timeout=None): - pass - - def release(self): - pass - - def __enter__(self): - self.acquire() - - def __exit__(self, type, value, traceback): - self.release() - - class DummyLock(object): def acquire(self): pass diff --git a/oslo_messaging/server.py b/oslo_messaging/server.py index 491ccbf52..f1739ad90 100644 --- a/oslo_messaging/server.py +++ b/oslo_messaging/server.py @@ -23,16 +23,17 @@ __all__ = [ 'ServerListenError', ] +import functools +import inspect import logging import threading +import traceback from oslo_service import service from oslo_utils import timeutils from stevedore import driver from oslo_messaging._drivers import base as driver_base -from oslo_messaging._i18n import _LW -from oslo_messaging import _utils from oslo_messaging import exceptions LOG = logging.getLogger(__name__) @@ -62,7 +63,170 @@ class ServerListenError(MessagingServerError): self.ex = ex -class MessageHandlingServer(service.ServiceBase): +class _OrderedTask(object): + """A task which must be executed in a particular order. + + A caller may wait for this task to complete by calling + `wait_for_completion`. + + A caller may run this task with `run_once`, which will ensure that however + many times the task is called it only runs once. Simultaneous callers will + block until the running task completes, which means that any caller can be + sure that the task has completed after run_once returns. + """ + + INIT = 0 # The task has not yet started + RUNNING = 1 # The task is running somewhere + COMPLETE = 2 # The task has run somewhere + + # We generate a log message if we wait for a lock longer than + # LOG_AFTER_WAIT_SECS seconds + LOG_AFTER_WAIT_SECS = 30 + + def __init__(self, name): + """Create a new _OrderedTask. + + :param name: The name of this task. Used in log messages. + """ + + super(_OrderedTask, self).__init__() + + self._name = name + self._cond = threading.Condition() + self._state = self.INIT + + def _wait(self, condition, warn_msg): + """Wait while condition() is true. Write a log message if condition() + has not become false within LOG_AFTER_WAIT_SECS. + """ + with timeutils.StopWatch(duration=self.LOG_AFTER_WAIT_SECS) as w: + logged = False + while condition(): + wait = None if logged else w.leftover() + self._cond.wait(wait) + + if not logged and w.expired(): + LOG.warn(warn_msg) + LOG.debug(''.join(traceback.format_stack())) + # Only log once. After than we wait indefinitely without + # logging. + logged = True + + def wait_for_completion(self, caller): + """Wait until this task has completed. + + :param caller: The name of the task which is waiting. + """ + with self._cond: + self._wait(lambda: self._state != self.COMPLETE, + '%s has been waiting for %s to complete for longer ' + 'than %i seconds' + % (caller, self._name, self.LOG_AFTER_WAIT_SECS)) + + def run_once(self, fn): + """Run a task exactly once. If it is currently running in another + thread, wait for it to complete. If it has already run, return + immediately without running it again. + + :param fn: The task to run. It must be a callable taking no arguments. + It may optionally return another callable, which also takes + no arguments, which will be executed after completion has + been signaled to other threads. + """ + with self._cond: + if self._state == self.INIT: + self._state = self.RUNNING + # Note that nothing waits on RUNNING, so no need to notify + + # We need to release the condition lock before calling out to + # prevent deadlocks. Reacquire it immediately afterwards. + self._cond.release() + try: + post_fn = fn() + finally: + self._cond.acquire() + self._state = self.COMPLETE + self._cond.notify_all() + + if post_fn is not None: + # Release the condition lock before calling out to prevent + # deadlocks. Reacquire it immediately afterwards. + self._cond.release() + try: + post_fn() + finally: + self._cond.acquire() + elif self._state == self.RUNNING: + self._wait(lambda: self._state == self.RUNNING, + '%s has been waiting on another thread to complete ' + 'for longer than %i seconds' + % (self._name, self.LOG_AFTER_WAIT_SECS)) + + +class _OrderedTaskRunner(object): + """Mixin for a class which executes ordered tasks.""" + + def __init__(self, *args, **kwargs): + super(_OrderedTaskRunner, self).__init__(*args, **kwargs) + + # Get a list of methods on this object which have the _ordered + # attribute + self._tasks = [name + for (name, member) in inspect.getmembers(self) + if inspect.ismethod(member) and + getattr(member, '_ordered', False)] + self.init_task_states() + + def init_task_states(self): + # Note that we don't need to lock this. Once created, the _states dict + # is immutable. Get and set are (individually) atomic operations in + # Python, and we only set after the dict is fully created. + self._states = {task: _OrderedTask(task) for task in self._tasks} + + @staticmethod + def decorate_ordered(fn, state, after): + + @functools.wraps(fn) + def wrapper(self, *args, **kwargs): + # Store the states we started with in case the state wraps on us + # while we're sleeping. We must wait and run_once in the same + # epoch. If the epoch ended while we were sleeping, run_once will + # safely do nothing. + states = self._states + + # Wait for the given preceding state to complete + if after is not None: + states[after].wait_for_completion(state) + + # Run this state + states[state].run_once(lambda: fn(self, *args, **kwargs)) + return wrapper + + +def ordered(after=None): + """A method which will be executed as an ordered task. The method will be + called exactly once, however many times it is called. If it is called + multiple times simultaneously it will only be called once, but all callers + will wait until execution is complete. + + If `after` is given, this method will not run until `after` has completed. + + :param after: Optionally, another method decorated with `ordered`. Wait for + the completion of `after` before executing this method. + """ + if after is not None: + after = after.__name__ + + def _ordered(fn): + # Set an attribute on the method so we can find it later + setattr(fn, '_ordered', True) + state = fn.__name__ + + return _OrderedTaskRunner.decorate_ordered(fn, state, after) + return _ordered + + +class MessageHandlingServer(service.ServiceBase, _OrderedTaskRunner): """Server for handling messages. Connect a transport to a dispatcher that knows how to process the @@ -94,29 +258,18 @@ class MessageHandlingServer(service.ServiceBase): self.dispatcher = dispatcher self.executor = executor - # NOTE(sileht): we use a lock to protect the state change of the - # server, we don't want to call stop until the transport driver - # is fully started. Except for the blocking executor that have - # start() that doesn't return - if self.executor != "blocking": - self._state_cond = threading.Condition() - self._dummy_cond = False - else: - self._state_cond = _utils.DummyCondition() - self._dummy_cond = True - try: mgr = driver.DriverManager('oslo.messaging.executors', self.executor) except RuntimeError as ex: raise ExecutorLoadFailure(self.executor, ex) - else: - self._executor_cls = mgr.driver - self._executor_obj = None - self._running = False + + self._executor_cls = mgr.driver + self._executor_obj = None super(MessageHandlingServer, self).__init__() + @ordered() def start(self): """Start handling incoming messages. @@ -131,24 +284,21 @@ class MessageHandlingServer(service.ServiceBase): choose to dispatch messages in a new thread, coroutine or simply the current thread. """ - if self._executor_obj is not None: - return - with self._state_cond: - if self._executor_obj is not None: - return - try: - listener = self.dispatcher._listen(self.transport) - except driver_base.TransportDriverError as ex: - raise ServerListenError(self.target, ex) - self._executor_obj = self._executor_cls(self.conf, listener, - self.dispatcher) - self._executor_obj.start() - self._running = True - self._state_cond.notify_all() + try: + listener = self.dispatcher._listen(self.transport) + except driver_base.TransportDriverError as ex: + raise ServerListenError(self.target, ex) + executor = self._executor_cls(self.conf, listener, self.dispatcher) + executor.start() + self._executor_obj = executor if self.executor == 'blocking': - self._executor_obj.execute() + # N.B. This will be executed unlocked and unordered, so + # we can't rely on the value of self._executor_obj when this runs. + # We explicitly pass the local variable. + return lambda: executor.execute() + @ordered(after=start) def stop(self): """Stop handling incoming messages. @@ -157,12 +307,9 @@ class MessageHandlingServer(service.ServiceBase): some messages, and underlying driver resources associated to this server are still in use. See 'wait' for more details. """ - with self._state_cond: - if self._executor_obj is not None: - self._running = False - self._executor_obj.stop() - self._state_cond.notify_all() + self._executor_obj.stop() + @ordered(after=stop) def wait(self): """Wait for message processing to complete. @@ -173,37 +320,14 @@ class MessageHandlingServer(service.ServiceBase): Once it's finished, the underlying driver resources associated to this server are released (like closing useless network connections). """ - with self._state_cond: - if self._running: - LOG.warn(_LW("wait() should be called after stop() as it " - "waits for existing messages to finish " - "processing")) - w = timeutils.StopWatch() - w.start() - while self._running: - # NOTE(harlowja): 1.0 seconds was mostly chosen at - # random, but it seems like a reasonable value to - # use to avoid spamming the logs with to much - # information. - self._state_cond.wait(1.0) - if self._running and not self._dummy_cond: - LOG.warn( - _LW("wait() should have been called" - " after stop() as wait() waits for existing" - " messages to finish processing, it has" - " been %0.2f seconds and stop() still has" - " not been called"), w.elapsed()) - executor = self._executor_obj + try: + self._executor_obj.wait() + finally: + # Close listener connection after processing all messages + self._executor_obj.listener.cleanup() self._executor_obj = None - if executor is not None: - # We are the lucky calling thread to wait on the executor to - # actually finish. - try: - executor.wait() - finally: - # Close listener connection after processing all messages - executor.listener.cleanup() - executor = None + + self.init_task_states() def reset(self): """Reset service. diff --git a/oslo_messaging/tests/rpc/test_server.py b/oslo_messaging/tests/rpc/test_server.py index 258dacb24..1a2d2aa63 100644 --- a/oslo_messaging/tests/rpc/test_server.py +++ b/oslo_messaging/tests/rpc/test_server.py @@ -13,6 +13,8 @@ # License for the specific language governing permissions and limitations # under the License. +import eventlet +import time import threading from oslo_config import cfg @@ -20,6 +22,7 @@ import testscenarios import mock import oslo_messaging +from oslo_messaging import server as server_module from oslo_messaging.tests import utils as test_utils load_tests = testscenarios.load_tests_apply_scenarios @@ -528,3 +531,210 @@ class TestMultipleServers(test_utils.BaseTestCase, ServerSetupMixin): TestMultipleServers.generate_scenarios() + +class TestServerLocking(test_utils.BaseTestCase): + def setUp(self): + super(TestServerLocking, self).setUp(conf=cfg.ConfigOpts()) + + def _logmethod(name): + def method(self): + with self._lock: + self._calls.append(name) + return method + + executors = [] + class FakeExecutor(object): + def __init__(self, *args, **kwargs): + self._lock = threading.Lock() + self._calls = [] + self.listener = mock.MagicMock() + executors.append(self) + + start = _logmethod('start') + stop = _logmethod('stop') + wait = _logmethod('wait') + execute = _logmethod('execute') + self.executors = executors + + self.server = oslo_messaging.MessageHandlingServer(mock.Mock(), + mock.Mock()) + self.server._executor_cls = FakeExecutor + + def test_start_stop_wait(self): + # Test a simple execution of start, stop, wait in order + + thread = eventlet.spawn(self.server.start) + self.server.stop() + self.server.wait() + + self.assertEqual(len(self.executors), 1) + executor = self.executors[0] + self.assertEqual(executor._calls, + ['start', 'execute', 'stop', 'wait']) + self.assertTrue(executor.listener.cleanup.called) + + def test_reversed_order(self): + # Test that if we call wait, stop, start, these will be correctly + # reordered + + wait = eventlet.spawn(self.server.wait) + # This is non-deterministic, but there's not a great deal we can do + # about that + eventlet.sleep(0) + + stop = eventlet.spawn(self.server.stop) + eventlet.sleep(0) + + start = eventlet.spawn(self.server.start) + + self.server.wait() + + self.assertEqual(len(self.executors), 1) + executor = self.executors[0] + self.assertEqual(executor._calls, + ['start', 'execute', 'stop', 'wait']) + + def test_wait_for_running_task(self): + # Test that if 2 threads call a method simultaneously, both will wait, + # but only 1 will call the underlying executor method. + + start_event = threading.Event() + finish_event = threading.Event() + + running_event = threading.Event() + done_event = threading.Event() + + runner = [None] + class SteppingFakeExecutor(self.server._executor_cls): + def start(self): + # Tell the test which thread won the race + runner[0] = eventlet.getcurrent() + running_event.set() + + start_event.wait() + super(SteppingFakeExecutor, self).start() + done_event.set() + + finish_event.wait() + self.server._executor_cls = SteppingFakeExecutor + + start1 = eventlet.spawn(self.server.start) + start2 = eventlet.spawn(self.server.start) + + # Wait until one of the threads starts running + running_event.wait() + runner = runner[0] + waiter = start2 if runner == start1 else start2 + + waiter_finished = threading.Event() + waiter.link(lambda _: waiter_finished.set()) + + # At this point, runner is running start(), and waiter() is waiting for + # it to complete. runner has not yet logged anything. + self.assertEqual(1, len(self.executors)) + executor = self.executors[0] + + self.assertEqual(executor._calls, []) + self.assertFalse(waiter_finished.is_set()) + + # Let the runner log the call + start_event.set() + done_event.wait() + + # We haven't signalled completion yet, so execute shouldn't have run + self.assertEqual(executor._calls, ['start']) + self.assertFalse(waiter_finished.is_set()) + + # Let the runner complete + finish_event.set() + waiter.wait() + runner.wait() + + # Check that both threads have finished, start was only called once, + # and execute ran + self.assertTrue(waiter_finished.is_set()) + self.assertEqual(executor._calls, ['start', 'execute']) + + def test_state_wrapping(self): + # Test that we behave correctly if a thread waits, and the server state + # has wrapped when it it next scheduled + + # Ensure that if 2 threads wait for the completion of 'start', the + # first will wait until complete_event is signalled, but the second + # will continue + complete_event = threading.Event() + complete_waiting_callback = threading.Event() + + start_state = self.server._states['start'] + old_wait_for_completion = start_state.wait_for_completion + waited = [False] + def new_wait_for_completion(*args, **kwargs): + if not waited[0]: + waited[0] = True + complete_waiting_callback.set() + complete_event.wait() + old_wait_for_completion(*args, **kwargs) + start_state.wait_for_completion = new_wait_for_completion + + # thread1 will wait for start to complete until we signal it + thread1 = eventlet.spawn(self.server.stop) + thread1_finished = threading.Event() + thread1.link(lambda _: thread1_finished.set()) + + self.server.start() + complete_waiting_callback.wait() + + # The server should have started, but stop should not have been called + self.assertEqual(1, len(self.executors)) + self.assertEqual(self.executors[0]._calls, ['start', 'execute']) + self.assertFalse(thread1_finished.is_set()) + + self.server.stop() + self.server.wait() + + # We should have gone through all the states, and thread1 should still + # be waiting + self.assertEqual(1, len(self.executors)) + self.assertEqual(self.executors[0]._calls, ['start', 'execute', + 'stop', 'wait']) + self.assertFalse(thread1_finished.is_set()) + + # Start again + self.server.start() + + # We should now record 2 executors + self.assertEqual(2, len(self.executors)) + self.assertEqual(self.executors[0]._calls, ['start', 'execute', + 'stop', 'wait']) + self.assertEqual(self.executors[1]._calls, ['start', 'execute']) + self.assertFalse(thread1_finished.is_set()) + + # Allow thread1 to complete + complete_event.set() + thread1_finished.wait() + + # thread1 should now have finished, and stop should not have been + # called again on either the first or second executor + self.assertEqual(2, len(self.executors)) + self.assertEqual(self.executors[0]._calls, ['start', 'execute', + 'stop', 'wait']) + self.assertEqual(self.executors[1]._calls, ['start', 'execute']) + self.assertTrue(thread1_finished.is_set()) + + @mock.patch.object(server_module._OrderedTask, + 'LOG_AFTER_WAIT_SECS', 1) + @mock.patch.object(server_module, 'LOG') + def test_timeout_logging(self, mock_log): + # Test that we generate a log message if we wait longer than + # LOG_AFTER_WAIT_SECS + + log_event = threading.Event() + mock_log.warn.side_effect = lambda _: log_event.set() + + # Call stop without calling start. We should log a wait after 1 second + thread = eventlet.spawn(self.server.stop) + log_event.wait() + + # Redundant given that we already waited, but it's nice to assert + self.assertTrue(mock_log.warn.called) + thread.kill() From 9e5fb5697d3f7259f01e3416af0582090d20859a Mon Sep 17 00:00:00 2001 From: Cyril Roelandt Date: Fri, 23 Oct 2015 20:07:39 +0200 Subject: [PATCH 14/74] Remove a useless statement This statement is useless since both 'username' and 'password' are set to None in the for loop, and that they are not used outside of the loop. Removing this line also help us getting rid of a false positive thrown by bandit. Change-Id: I2aa1a16f30928b77aa40c5a900e35b7bf752658a --- oslo_messaging/transport.py | 1 - 1 file changed, 1 deletion(-) diff --git a/oslo_messaging/transport.py b/oslo_messaging/transport.py index 5a8c3891a..6fb3c8e42 100644 --- a/oslo_messaging/transport.py +++ b/oslo_messaging/transport.py @@ -381,7 +381,6 @@ class TransportURL(object): hosts = [] - username = password = '' for host in url.netloc.split(','): if not host: continue From f5d189723eac0215758766a8750233d8a26b38f3 Mon Sep 17 00:00:00 2001 From: OpenStack Proposal Bot Date: Sat, 24 Oct 2015 00:26:49 +0000 Subject: [PATCH 15/74] Updated from global requirements Change-Id: If7fdb576d25c0742ce47824209ee70cba5d78d33 --- requirements.txt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/requirements.txt b/requirements.txt index 9d82a670c..d703e20b2 100644 --- a/requirements.txt +++ b/requirements.txt @@ -5,11 +5,11 @@ pbr>=1.6 futurist>=0.1.2 # Apache-2.0 -oslo.config>=2.3.0 # Apache-2.0 +oslo.config>=2.6.0 # Apache-2.0 oslo.context>=0.2.0 # Apache-2.0 oslo.log>=1.8.0 # Apache-2.0 oslo.utils!=2.6.0,>=2.4.0 # Apache-2.0 -oslo.serialization>=1.4.0 # Apache-2.0 +oslo.serialization>=1.10.0 # Apache-2.0 oslo.service>=0.10.0 # Apache-2.0 oslo.i18n>=1.5.0 # Apache-2.0 stevedore>=1.5.0 # Apache-2.0 From 2786a9ded35fdd427e22133abd050fc7bbebdcab Mon Sep 17 00:00:00 2001 From: Cyril Roelandt Date: Mon, 26 Oct 2015 11:01:11 +0100 Subject: [PATCH 16/74] Add a "bandit" target to tox.ini This will allow us to find potential security issues, such as those fixed by 52e624891fc500c8ab9f3f10ef45258ce740916a and c4a7ac0b653543e8a3ba10060cabdb114fb6672b . Change-Id: I21aa0ca79232784069e55da46920eb43250d8939 --- bandit.yaml | 362 ++++++++++++++++++++++++++++++++++++++++++ test-requirements.txt | 3 + tox.ini | 6 +- 3 files changed, 370 insertions(+), 1 deletion(-) create mode 100644 bandit.yaml diff --git a/bandit.yaml b/bandit.yaml new file mode 100644 index 000000000..d41da69aa --- /dev/null +++ b/bandit.yaml @@ -0,0 +1,362 @@ +# optional: after how many files to update progress +#show_progress_every: 100 + +# optional: plugins directory name +#plugins_dir: plugins + +# optional: plugins discovery name pattern +plugin_name_pattern: '*.py' + +# optional: terminal escape sequences to display colors +#output_colors: +# DEFAULT: \033[0m +# HEADER: \033[95m +# LOW: \033[94m +# MEDIUM: \033[93m +# HIGH: \033[91m + +# optional: log format string +#log_format: "[%(module)s]\t%(levelname)s\t%(message)s" + +# globs of files which should be analyzed +include: + - '*.py' + - '*.pyw' + +# a list of strings, which if found in the path will cause files to be excluded +# for example /tests/ - to remove all files in tests directory +exclude_dirs: + - '/tests/' + +profiles: + oslo.messaging: + include: + - any_other_function_with_shell_equals_true + # Some occurrences in the olso.messaging code, but not much to do + # to get rid of these warnings, so just skip this. + # - assert_used + - blacklist_calls + - blacklist_import_func + - blacklist_imports + - exec_used + - execute_with_run_as_root_equals_true + - hardcoded_bind_all_interfaces + - hardcoded_password_string + - hardcoded_password_funcarg + - hardcoded_password_default + - hardcoded_sql_expressions + - hardcoded_tmp_directory + - jinja2_autoescape_false + - linux_commands_wildcard_injection + - paramiko_calls + - password_config_option_not_marked_secret + - request_with_no_cert_validation + - set_bad_file_permissions + - subprocess_popen_with_shell_equals_true + - subprocess_without_shell_equals_true + - start_process_with_a_shell + - start_process_with_no_shell + - start_process_with_partial_path + - ssl_with_bad_defaults + - ssl_with_bad_version + - ssl_with_no_version + # This might be nice to have, but we currently ignore a lot of + # exceptions during the cleanup phases, so this throws a lot + # false positives. + # - try_except_pass + - use_of_mako_templates + - weak_cryptographic_key + + XSS: + include: + - jinja2_autoescape_false + - use_of_mako_templates + + ShellInjection: + include: + - subprocess_popen_with_shell_equals_true + - subprocess_without_shell_equals_true + - any_other_function_with_shell_equals_true + - start_process_with_a_shell + - start_process_with_no_shell + - start_process_with_partial_path + exclude: + + SqlInjection: + include: + - hardcoded_sql_expressions + +blacklist_calls: + bad_name_sets: + - pickle: + qualnames: + - pickle.loads + - pickle.load + - pickle.Unpickler + - cPickle.loads + - cPickle.load + - cPickle.Unpickler + message: > + Pickle library appears to be in use, possible security issue. + - marshal: + qualnames: [marshal.load, marshal.loads] + message: > + Deserialization with the marshal module is possibly dangerous. + - md5: + qualnames: + - hashlib.md5 + - Crypto.Hash.MD2.new + - Crypto.Hash.MD4.new + - Crypto.Hash.MD5.new + - cryptography.hazmat.primitives.hashes.MD5 + message: Use of insecure MD2, MD4, or MD5 hash function. + - ciphers: + qualnames: + - Crypto.Cipher.ARC2.new + - Crypto.Cipher.ARC4.new + - Crypto.Cipher.Blowfish.new + - Crypto.Cipher.DES.new + - Crypto.Cipher.XOR.new + - cryptography.hazmat.primitives.ciphers.algorithms.ARC4 + - cryptography.hazmat.primitives.ciphers.algorithms.Blowfish + - cryptography.hazmat.primitives.ciphers.algorithms.IDEA + message: > + Use of insecure cipher {func}. Replace with a known secure + cipher such as AES. + level: HIGH + - cipher_modes: + qualnames: + - cryptography.hazmat.primitives.ciphers.modes.ECB + message: Use of insecure cipher mode {func}. + - mktemp_q: + qualnames: [tempfile.mktemp] + message: Use of insecure and deprecated function (mktemp). + - eval: + qualnames: [eval] + message: > + Use of possibly insecure function - consider using safer + ast.literal_eval. + - mark_safe: + names: [mark_safe] + message: > + Use of mark_safe() may expose cross-site scripting + vulnerabilities and should be reviewed. + - httpsconnection: + qualnames: [httplib.HTTPSConnection] + message: > + Use of HTTPSConnection does not provide security, see + https://wiki.openstack.org/wiki/OSSN/OSSN-0033 + - yaml_load: + qualnames: [yaml.load] + message: > + Use of unsafe yaml load. Allows instantiation of arbitrary + objects. Consider yaml.safe_load(). + - urllib_urlopen: + qualnames: + - urllib.urlopen + - urllib.urlretrieve + - urllib.URLopener + - urllib.FancyURLopener + - urllib2.urlopen + - urllib2.Request + message: > + Audit url open for permitted schemes. Allowing use of file:/ or + custom schemes is often unexpected. + - telnetlib: + qualnames: + - telnetlib.* + message: > + Telnet-related funtions are being called. Telnet is considered + insecure. Use SSH or some other encrypted protocol. + level: HIGH + # Most of this is based off of Christian Heimes' work on defusedxml: + # https://pypi.python.org/pypi/defusedxml/#defusedxml-sax + - xml_bad_cElementTree: + qualnames: + - xml.etree.cElementTree.parse + - xml.etree.cElementTree.iterparse + - xml.etree.cElementTree.fromstring + - xml.etree.cElementTree.XMLParser + message: > + Using {func} to parse untrusted XML data is known to be + vulnerable to XML attacks. Replace {func} with its defusedxml + equivalent function. + - xml_bad_ElementTree: + qualnames: + - xml.etree.ElementTree.parse + - xml.etree.ElementTree.iterparse + - xml.etree.ElementTree.fromstring + - xml.etree.ElementTree.XMLParser + message: > + Using {func} to parse untrusted XML data is known to be + vulnerable to XML attacks. Replace {func} with its defusedxml + equivalent function. + - xml_bad_expatreader: + qualnames: [xml.sax.expatreader.create_parser] + message: > + Using {func} to parse untrusted XML data is known to be + vulnerable to XML attacks. Replace {func} with its defusedxml + equivalent function. + - xml_bad_expatbuilder: + qualnames: + - xml.dom.expatbuilder.parse + - xml.dom.expatbuilder.parseString + message: > + Using {func} to parse untrusted XML data is known to be + vulnerable to XML attacks. Replace {func} with its defusedxml + equivalent function. + - xml_bad_sax: + qualnames: + - xml.sax.parse + - xml.sax.parseString + - xml.sax.make_parser + message: > + Using {func} to parse untrusted XML data is known to be + vulnerable to XML attacks. Replace {func} with its defusedxml + equivalent function. + - xml_bad_minidom: + qualnames: + - xml.dom.minidom.parse + - xml.dom.minidom.parseString + message: > + Using {func} to parse untrusted XML data is known to be + vulnerable to XML attacks. Replace {func} with its defusedxml + equivalent function. + - xml_bad_pulldom: + qualnames: + - xml.dom.pulldom.parse + - xml.dom.pulldom.parseString + message: > + Using {func} to parse untrusted XML data is known to be + vulnerable to XML attacks. Replace {func} with its defusedxml + equivalent function. + - xml_bad_etree: + qualnames: + - lxml.etree.parse + - lxml.etree.fromstring + - lxml.etree.RestrictedElement + - lxml.etree.GlobalParserTLS + - lxml.etree.getDefaultParser + - lxml.etree.check_docinfo + message: > + Using {func} to parse untrusted XML data is known to be + vulnerable to XML attacks. Replace {func} with its defusedxml + equivalent function. + + +shell_injection: + # Start a process using the subprocess module, or one of its wrappers. + subprocess: + - subprocess.Popen + - subprocess.call + - subprocess.check_call + - subprocess.check_output + - utils.execute + - utils.execute_with_timeout + # Start a process with a function vulnerable to shell injection. + shell: + - os.system + - os.popen + - os.popen2 + - os.popen3 + - os.popen4 + - popen2.popen2 + - popen2.popen3 + - popen2.popen4 + - popen2.Popen3 + - popen2.Popen4 + - commands.getoutput + - commands.getstatusoutput + # Start a process with a function that is not vulnerable to shell injection. + no_shell: + - os.execl + - os.execle + - os.execlp + - os.execlpe + - os.execv + - os.execve + - os.execvp + - os.execvpe + - os.spawnl + - os.spawnle + - os.spawnlp + - os.spawnlpe + - os.spawnv + - os.spawnve + - os.spawnvp + - os.spawnvpe + - os.startfile + +blacklist_imports: + bad_import_sets: + - telnet: + imports: [telnetlib] + level: HIGH + message: > + A telnet-related module is being imported. Telnet is + considered insecure. Use SSH or some other encrypted protocol. + - info_libs: + imports: [pickle, cPickle, subprocess, Crypto] + level: LOW + message: > + Consider possible security implications associated with + {module} module. + + # Most of this is based off of Christian Heimes' work on defusedxml: + # https://pypi.python.org/pypi/defusedxml/#defusedxml-sax + + - xml_libs: + imports: + - xml.etree.cElementTree + - xml.etree.ElementTree + - xml.sax.expatreader + - xml.sax + - xml.dom.expatbuilder + - xml.dom.minidom + - xml.dom.pulldom + - lxml.etree + - lxml + message: > + Using {module} to parse untrusted XML data is known to be + vulnerable to XML attacks. Replace {module} with the equivalent + defusedxml package. + level: LOW + - xml_libs_high: + imports: [xmlrpclib] + message: > + Using {module} to parse untrusted XML data is known to be + vulnerable to XML attacks. Use defused.xmlrpc.monkey_patch() + function to monkey-patch xmlrpclib and mitigate XML + vulnerabilities. + level: HIGH + +hardcoded_tmp_directory: + tmp_dirs: [/tmp, /var/tmp, /dev/shm] + +hardcoded_password: + # Support for full path, relative path and special "%(site_data_dir)s" + # substitution (/usr/{local}/share) + word_list: "%(site_data_dir)s/wordlist/default-passwords" + +ssl_with_bad_version: + bad_protocol_versions: + - PROTOCOL_SSLv2 + - SSLv2_METHOD + - SSLv23_METHOD + - PROTOCOL_SSLv3 # strict option + - PROTOCOL_TLSv1 # strict option + - SSLv3_METHOD # strict option + - TLSv1_METHOD # strict option + +password_config_option_not_marked_secret: + function_names: + - oslo.config.cfg.StrOpt + - oslo_config.cfg.StrOpt + +execute_with_run_as_root_equals_true: + function_names: + - ceilometer.utils.execute + - cinder.utils.execute + - neutron.agent.linux.utils.execute + - nova.utils.execute + - nova.utils.trycmd diff --git a/test-requirements.txt b/test-requirements.txt index 693f069b1..53d199ceb 100644 --- a/test-requirements.txt +++ b/test-requirements.txt @@ -36,3 +36,6 @@ oslosphinx>=2.5.0 # Apache-2.0 # AMQP 1.0 support depends on the Qpid Proton AMQP 1.0 # development libraries. pyngus>=2.0.0 # Apache-2.0 + +# Bandit security code scanner +bandit>=0.13.2 diff --git a/tox.ini b/tox.ini index c576bed72..1689b87a9 100644 --- a/tox.ini +++ b/tox.ini @@ -1,5 +1,5 @@ [tox] -envlist = py34,py27,pep8 +envlist = py34,py27,pep8,bandit [testenv] setenv = @@ -38,6 +38,10 @@ commands = {toxinidir}/setup-test-env-qpid.sh 1.0 python setup.py testr --slowes [testenv:py27-func-zeromq] commands = {toxinidir}/setup-test-env-zmq.sh python setup.py testr --slowest --testr-args='oslo_messaging.tests.functional' +[testenv:bandit] +deps = -r{toxinidir}/test-requirements.txt +commands = bandit -c bandit.yaml -r oslo_messaging -n5 -p oslo.messaging + [flake8] show-source = True ignore = H237,H402,H405,H904 From 64518fa170f3c36529453782a82d855c3e98f8af Mon Sep 17 00:00:00 2001 From: Clint Byrum Date: Fri, 30 Oct 2015 14:09:37 -0700 Subject: [PATCH 17/74] Move supported messaging drivers in-tree Up until now it has only been available in the OpenStack spec, but it is a living document and I believe we can maintain it in oslo.messaging's tree. Change-Id: I7bb9e5f02004f857d8f75909fcc0d05f2882a77d --- doc/source/index.rst | 1 + doc/source/supported-messaging-drivers.rst | 60 ++++++++++++++++++++++ 2 files changed, 61 insertions(+) create mode 100644 doc/source/supported-messaging-drivers.rst diff --git a/doc/source/index.rst b/doc/source/index.rst index 5f45af872..9a6873cbd 100644 --- a/doc/source/index.rst +++ b/doc/source/index.rst @@ -23,6 +23,7 @@ Contents opts conffixture drivers + supported-messaging-drivers AMQP1.0 zmq_driver FAQ diff --git a/doc/source/supported-messaging-drivers.rst b/doc/source/supported-messaging-drivers.rst new file mode 100644 index 000000000..75c7a844f --- /dev/null +++ b/doc/source/supported-messaging-drivers.rst @@ -0,0 +1,60 @@ +============================= + Supported Messaging Drivers +============================= + +RabbitMQ may not be sufficient for the entire community as the community +grows. Pluggability is still something we should maintain, but we should +have a very high standard for drivers that are shipped and documented +as being supported. + +This document defines a very clear policy as to the requirements +for drivers to be carried in oslo.messaging and thus supported by the +OpenStack community as a whole. We will deprecate any drivers that do not +meet the requirements, and announce said deprecations in any appropriate +channels to give users time to signal their needs. Deprecation will last +for two release cycles before removing the code. We will also review and +update documentation to annotate which drivers are supported and which +are deprecated given these policies + +Policy +------ + +Testing +~~~~~~~ + +* Must have unit and/or functional test coverage of at least 60% as + reported by coverage report. Unit tests must be run for all versions + of python oslo.messaging currently gates on. + +* Must have integration testing including at least 3 popular oslo.messaging + dependents, preferrably at the minimum a devstack-gate job with Nova, + Cinder, and Neutron. + +* All testing above must be voting in the gate of oslo.messaging. + +Documentation +~~~~~~~~~~~~~ + +* Must have a reasonable amount of documentation including documentation + in the official OpenStack deployment guide. + +Support +~~~~~~~ + +* Must have at least two individuals from the community commited to + triaging and fixing bugs, and responding to test failures in a timely + manner. + +Prospective Drivers +~~~~~~~~~~~~~~~~~~~ + +* Drivers that intend to meet the requirements above, but that do not yet + meet them will be given one full release cycle, or 6 months, whichever + is longer, to comply before being marked for deprecation. Their use, + however, will not be supported by the community. This will prevent a + chicken and egg problem for new drivers. + +.. note:: + + This work is licensed under a Creative Commons Attribution 3.0 Unported License. + http://creativecommons.org/licenses/by/3.0/legalcode From e5b15ce642775c00bc163d89f6961b505911715f Mon Sep 17 00:00:00 2001 From: Sean Dague Date: Tue, 10 Nov 2015 15:33:14 +0000 Subject: [PATCH 18/74] Revert "Robustify locking in MessageHandlingServer" This reverts commit d700c382791b6352bb80a0dc455589085881669f. This commit is causing a timeout/lock wait condition when using the in memory rpc bus. It exposed in the Nova unit / functional tests which use this extensively. Change-Id: I9610a5533383955f926dbbb78ab679f45cd7bcdb Closes-Bug: #1514876 --- oslo_messaging/_utils.py | 23 +++ oslo_messaging/server.py | 260 +++++++----------------- oslo_messaging/tests/rpc/test_server.py | 210 ------------------- 3 files changed, 91 insertions(+), 402 deletions(-) diff --git a/oslo_messaging/_utils.py b/oslo_messaging/_utils.py index cec94bb48..1bb20b089 100644 --- a/oslo_messaging/_utils.py +++ b/oslo_messaging/_utils.py @@ -116,6 +116,29 @@ def fetch_current_thread_functor(): return lambda: threading.current_thread() +class DummyCondition(object): + def acquire(self): + pass + + def notify(self): + pass + + def notify_all(self): + pass + + def wait(self, timeout=None): + pass + + def release(self): + pass + + def __enter__(self): + self.acquire() + + def __exit__(self, type, value, traceback): + self.release() + + class DummyLock(object): def acquire(self): pass diff --git a/oslo_messaging/server.py b/oslo_messaging/server.py index f1739ad90..491ccbf52 100644 --- a/oslo_messaging/server.py +++ b/oslo_messaging/server.py @@ -23,17 +23,16 @@ __all__ = [ 'ServerListenError', ] -import functools -import inspect import logging import threading -import traceback from oslo_service import service from oslo_utils import timeutils from stevedore import driver from oslo_messaging._drivers import base as driver_base +from oslo_messaging._i18n import _LW +from oslo_messaging import _utils from oslo_messaging import exceptions LOG = logging.getLogger(__name__) @@ -63,170 +62,7 @@ class ServerListenError(MessagingServerError): self.ex = ex -class _OrderedTask(object): - """A task which must be executed in a particular order. - - A caller may wait for this task to complete by calling - `wait_for_completion`. - - A caller may run this task with `run_once`, which will ensure that however - many times the task is called it only runs once. Simultaneous callers will - block until the running task completes, which means that any caller can be - sure that the task has completed after run_once returns. - """ - - INIT = 0 # The task has not yet started - RUNNING = 1 # The task is running somewhere - COMPLETE = 2 # The task has run somewhere - - # We generate a log message if we wait for a lock longer than - # LOG_AFTER_WAIT_SECS seconds - LOG_AFTER_WAIT_SECS = 30 - - def __init__(self, name): - """Create a new _OrderedTask. - - :param name: The name of this task. Used in log messages. - """ - - super(_OrderedTask, self).__init__() - - self._name = name - self._cond = threading.Condition() - self._state = self.INIT - - def _wait(self, condition, warn_msg): - """Wait while condition() is true. Write a log message if condition() - has not become false within LOG_AFTER_WAIT_SECS. - """ - with timeutils.StopWatch(duration=self.LOG_AFTER_WAIT_SECS) as w: - logged = False - while condition(): - wait = None if logged else w.leftover() - self._cond.wait(wait) - - if not logged and w.expired(): - LOG.warn(warn_msg) - LOG.debug(''.join(traceback.format_stack())) - # Only log once. After than we wait indefinitely without - # logging. - logged = True - - def wait_for_completion(self, caller): - """Wait until this task has completed. - - :param caller: The name of the task which is waiting. - """ - with self._cond: - self._wait(lambda: self._state != self.COMPLETE, - '%s has been waiting for %s to complete for longer ' - 'than %i seconds' - % (caller, self._name, self.LOG_AFTER_WAIT_SECS)) - - def run_once(self, fn): - """Run a task exactly once. If it is currently running in another - thread, wait for it to complete. If it has already run, return - immediately without running it again. - - :param fn: The task to run. It must be a callable taking no arguments. - It may optionally return another callable, which also takes - no arguments, which will be executed after completion has - been signaled to other threads. - """ - with self._cond: - if self._state == self.INIT: - self._state = self.RUNNING - # Note that nothing waits on RUNNING, so no need to notify - - # We need to release the condition lock before calling out to - # prevent deadlocks. Reacquire it immediately afterwards. - self._cond.release() - try: - post_fn = fn() - finally: - self._cond.acquire() - self._state = self.COMPLETE - self._cond.notify_all() - - if post_fn is not None: - # Release the condition lock before calling out to prevent - # deadlocks. Reacquire it immediately afterwards. - self._cond.release() - try: - post_fn() - finally: - self._cond.acquire() - elif self._state == self.RUNNING: - self._wait(lambda: self._state == self.RUNNING, - '%s has been waiting on another thread to complete ' - 'for longer than %i seconds' - % (self._name, self.LOG_AFTER_WAIT_SECS)) - - -class _OrderedTaskRunner(object): - """Mixin for a class which executes ordered tasks.""" - - def __init__(self, *args, **kwargs): - super(_OrderedTaskRunner, self).__init__(*args, **kwargs) - - # Get a list of methods on this object which have the _ordered - # attribute - self._tasks = [name - for (name, member) in inspect.getmembers(self) - if inspect.ismethod(member) and - getattr(member, '_ordered', False)] - self.init_task_states() - - def init_task_states(self): - # Note that we don't need to lock this. Once created, the _states dict - # is immutable. Get and set are (individually) atomic operations in - # Python, and we only set after the dict is fully created. - self._states = {task: _OrderedTask(task) for task in self._tasks} - - @staticmethod - def decorate_ordered(fn, state, after): - - @functools.wraps(fn) - def wrapper(self, *args, **kwargs): - # Store the states we started with in case the state wraps on us - # while we're sleeping. We must wait and run_once in the same - # epoch. If the epoch ended while we were sleeping, run_once will - # safely do nothing. - states = self._states - - # Wait for the given preceding state to complete - if after is not None: - states[after].wait_for_completion(state) - - # Run this state - states[state].run_once(lambda: fn(self, *args, **kwargs)) - return wrapper - - -def ordered(after=None): - """A method which will be executed as an ordered task. The method will be - called exactly once, however many times it is called. If it is called - multiple times simultaneously it will only be called once, but all callers - will wait until execution is complete. - - If `after` is given, this method will not run until `after` has completed. - - :param after: Optionally, another method decorated with `ordered`. Wait for - the completion of `after` before executing this method. - """ - if after is not None: - after = after.__name__ - - def _ordered(fn): - # Set an attribute on the method so we can find it later - setattr(fn, '_ordered', True) - state = fn.__name__ - - return _OrderedTaskRunner.decorate_ordered(fn, state, after) - return _ordered - - -class MessageHandlingServer(service.ServiceBase, _OrderedTaskRunner): +class MessageHandlingServer(service.ServiceBase): """Server for handling messages. Connect a transport to a dispatcher that knows how to process the @@ -258,18 +94,29 @@ class MessageHandlingServer(service.ServiceBase, _OrderedTaskRunner): self.dispatcher = dispatcher self.executor = executor + # NOTE(sileht): we use a lock to protect the state change of the + # server, we don't want to call stop until the transport driver + # is fully started. Except for the blocking executor that have + # start() that doesn't return + if self.executor != "blocking": + self._state_cond = threading.Condition() + self._dummy_cond = False + else: + self._state_cond = _utils.DummyCondition() + self._dummy_cond = True + try: mgr = driver.DriverManager('oslo.messaging.executors', self.executor) except RuntimeError as ex: raise ExecutorLoadFailure(self.executor, ex) - - self._executor_cls = mgr.driver - self._executor_obj = None + else: + self._executor_cls = mgr.driver + self._executor_obj = None + self._running = False super(MessageHandlingServer, self).__init__() - @ordered() def start(self): """Start handling incoming messages. @@ -284,21 +131,24 @@ class MessageHandlingServer(service.ServiceBase, _OrderedTaskRunner): choose to dispatch messages in a new thread, coroutine or simply the current thread. """ - try: - listener = self.dispatcher._listen(self.transport) - except driver_base.TransportDriverError as ex: - raise ServerListenError(self.target, ex) - executor = self._executor_cls(self.conf, listener, self.dispatcher) - executor.start() - self._executor_obj = executor + if self._executor_obj is not None: + return + with self._state_cond: + if self._executor_obj is not None: + return + try: + listener = self.dispatcher._listen(self.transport) + except driver_base.TransportDriverError as ex: + raise ServerListenError(self.target, ex) + self._executor_obj = self._executor_cls(self.conf, listener, + self.dispatcher) + self._executor_obj.start() + self._running = True + self._state_cond.notify_all() if self.executor == 'blocking': - # N.B. This will be executed unlocked and unordered, so - # we can't rely on the value of self._executor_obj when this runs. - # We explicitly pass the local variable. - return lambda: executor.execute() + self._executor_obj.execute() - @ordered(after=start) def stop(self): """Stop handling incoming messages. @@ -307,9 +157,12 @@ class MessageHandlingServer(service.ServiceBase, _OrderedTaskRunner): some messages, and underlying driver resources associated to this server are still in use. See 'wait' for more details. """ - self._executor_obj.stop() + with self._state_cond: + if self._executor_obj is not None: + self._running = False + self._executor_obj.stop() + self._state_cond.notify_all() - @ordered(after=stop) def wait(self): """Wait for message processing to complete. @@ -320,14 +173,37 @@ class MessageHandlingServer(service.ServiceBase, _OrderedTaskRunner): Once it's finished, the underlying driver resources associated to this server are released (like closing useless network connections). """ - try: - self._executor_obj.wait() - finally: - # Close listener connection after processing all messages - self._executor_obj.listener.cleanup() + with self._state_cond: + if self._running: + LOG.warn(_LW("wait() should be called after stop() as it " + "waits for existing messages to finish " + "processing")) + w = timeutils.StopWatch() + w.start() + while self._running: + # NOTE(harlowja): 1.0 seconds was mostly chosen at + # random, but it seems like a reasonable value to + # use to avoid spamming the logs with to much + # information. + self._state_cond.wait(1.0) + if self._running and not self._dummy_cond: + LOG.warn( + _LW("wait() should have been called" + " after stop() as wait() waits for existing" + " messages to finish processing, it has" + " been %0.2f seconds and stop() still has" + " not been called"), w.elapsed()) + executor = self._executor_obj self._executor_obj = None - - self.init_task_states() + if executor is not None: + # We are the lucky calling thread to wait on the executor to + # actually finish. + try: + executor.wait() + finally: + # Close listener connection after processing all messages + executor.listener.cleanup() + executor = None def reset(self): """Reset service. diff --git a/oslo_messaging/tests/rpc/test_server.py b/oslo_messaging/tests/rpc/test_server.py index 1a2d2aa63..258dacb24 100644 --- a/oslo_messaging/tests/rpc/test_server.py +++ b/oslo_messaging/tests/rpc/test_server.py @@ -13,8 +13,6 @@ # License for the specific language governing permissions and limitations # under the License. -import eventlet -import time import threading from oslo_config import cfg @@ -22,7 +20,6 @@ import testscenarios import mock import oslo_messaging -from oslo_messaging import server as server_module from oslo_messaging.tests import utils as test_utils load_tests = testscenarios.load_tests_apply_scenarios @@ -531,210 +528,3 @@ class TestMultipleServers(test_utils.BaseTestCase, ServerSetupMixin): TestMultipleServers.generate_scenarios() - -class TestServerLocking(test_utils.BaseTestCase): - def setUp(self): - super(TestServerLocking, self).setUp(conf=cfg.ConfigOpts()) - - def _logmethod(name): - def method(self): - with self._lock: - self._calls.append(name) - return method - - executors = [] - class FakeExecutor(object): - def __init__(self, *args, **kwargs): - self._lock = threading.Lock() - self._calls = [] - self.listener = mock.MagicMock() - executors.append(self) - - start = _logmethod('start') - stop = _logmethod('stop') - wait = _logmethod('wait') - execute = _logmethod('execute') - self.executors = executors - - self.server = oslo_messaging.MessageHandlingServer(mock.Mock(), - mock.Mock()) - self.server._executor_cls = FakeExecutor - - def test_start_stop_wait(self): - # Test a simple execution of start, stop, wait in order - - thread = eventlet.spawn(self.server.start) - self.server.stop() - self.server.wait() - - self.assertEqual(len(self.executors), 1) - executor = self.executors[0] - self.assertEqual(executor._calls, - ['start', 'execute', 'stop', 'wait']) - self.assertTrue(executor.listener.cleanup.called) - - def test_reversed_order(self): - # Test that if we call wait, stop, start, these will be correctly - # reordered - - wait = eventlet.spawn(self.server.wait) - # This is non-deterministic, but there's not a great deal we can do - # about that - eventlet.sleep(0) - - stop = eventlet.spawn(self.server.stop) - eventlet.sleep(0) - - start = eventlet.spawn(self.server.start) - - self.server.wait() - - self.assertEqual(len(self.executors), 1) - executor = self.executors[0] - self.assertEqual(executor._calls, - ['start', 'execute', 'stop', 'wait']) - - def test_wait_for_running_task(self): - # Test that if 2 threads call a method simultaneously, both will wait, - # but only 1 will call the underlying executor method. - - start_event = threading.Event() - finish_event = threading.Event() - - running_event = threading.Event() - done_event = threading.Event() - - runner = [None] - class SteppingFakeExecutor(self.server._executor_cls): - def start(self): - # Tell the test which thread won the race - runner[0] = eventlet.getcurrent() - running_event.set() - - start_event.wait() - super(SteppingFakeExecutor, self).start() - done_event.set() - - finish_event.wait() - self.server._executor_cls = SteppingFakeExecutor - - start1 = eventlet.spawn(self.server.start) - start2 = eventlet.spawn(self.server.start) - - # Wait until one of the threads starts running - running_event.wait() - runner = runner[0] - waiter = start2 if runner == start1 else start2 - - waiter_finished = threading.Event() - waiter.link(lambda _: waiter_finished.set()) - - # At this point, runner is running start(), and waiter() is waiting for - # it to complete. runner has not yet logged anything. - self.assertEqual(1, len(self.executors)) - executor = self.executors[0] - - self.assertEqual(executor._calls, []) - self.assertFalse(waiter_finished.is_set()) - - # Let the runner log the call - start_event.set() - done_event.wait() - - # We haven't signalled completion yet, so execute shouldn't have run - self.assertEqual(executor._calls, ['start']) - self.assertFalse(waiter_finished.is_set()) - - # Let the runner complete - finish_event.set() - waiter.wait() - runner.wait() - - # Check that both threads have finished, start was only called once, - # and execute ran - self.assertTrue(waiter_finished.is_set()) - self.assertEqual(executor._calls, ['start', 'execute']) - - def test_state_wrapping(self): - # Test that we behave correctly if a thread waits, and the server state - # has wrapped when it it next scheduled - - # Ensure that if 2 threads wait for the completion of 'start', the - # first will wait until complete_event is signalled, but the second - # will continue - complete_event = threading.Event() - complete_waiting_callback = threading.Event() - - start_state = self.server._states['start'] - old_wait_for_completion = start_state.wait_for_completion - waited = [False] - def new_wait_for_completion(*args, **kwargs): - if not waited[0]: - waited[0] = True - complete_waiting_callback.set() - complete_event.wait() - old_wait_for_completion(*args, **kwargs) - start_state.wait_for_completion = new_wait_for_completion - - # thread1 will wait for start to complete until we signal it - thread1 = eventlet.spawn(self.server.stop) - thread1_finished = threading.Event() - thread1.link(lambda _: thread1_finished.set()) - - self.server.start() - complete_waiting_callback.wait() - - # The server should have started, but stop should not have been called - self.assertEqual(1, len(self.executors)) - self.assertEqual(self.executors[0]._calls, ['start', 'execute']) - self.assertFalse(thread1_finished.is_set()) - - self.server.stop() - self.server.wait() - - # We should have gone through all the states, and thread1 should still - # be waiting - self.assertEqual(1, len(self.executors)) - self.assertEqual(self.executors[0]._calls, ['start', 'execute', - 'stop', 'wait']) - self.assertFalse(thread1_finished.is_set()) - - # Start again - self.server.start() - - # We should now record 2 executors - self.assertEqual(2, len(self.executors)) - self.assertEqual(self.executors[0]._calls, ['start', 'execute', - 'stop', 'wait']) - self.assertEqual(self.executors[1]._calls, ['start', 'execute']) - self.assertFalse(thread1_finished.is_set()) - - # Allow thread1 to complete - complete_event.set() - thread1_finished.wait() - - # thread1 should now have finished, and stop should not have been - # called again on either the first or second executor - self.assertEqual(2, len(self.executors)) - self.assertEqual(self.executors[0]._calls, ['start', 'execute', - 'stop', 'wait']) - self.assertEqual(self.executors[1]._calls, ['start', 'execute']) - self.assertTrue(thread1_finished.is_set()) - - @mock.patch.object(server_module._OrderedTask, - 'LOG_AFTER_WAIT_SECS', 1) - @mock.patch.object(server_module, 'LOG') - def test_timeout_logging(self, mock_log): - # Test that we generate a log message if we wait longer than - # LOG_AFTER_WAIT_SECS - - log_event = threading.Event() - mock_log.warn.side_effect = lambda _: log_event.set() - - # Call stop without calling start. We should log a wait after 1 second - thread = eventlet.spawn(self.server.stop) - log_event.wait() - - # Redundant given that we already waited, but it's nice to assert - self.assertTrue(mock_log.warn.called) - thread.kill() From 33243c26aca82474a719c74a0e8f29fab644a3f5 Mon Sep 17 00:00:00 2001 From: BANASHANKAR KALEBELAGUNDI VEERA Date: Mon, 9 Nov 2015 12:15:53 -0800 Subject: [PATCH 19/74] Fixing the server example code Added server.stop() before server.wait() Change-Id: I9764c77e0aa076b6a7b9bb9715e2ead89b12126f --- oslo_messaging/rpc/client.py | 4 ++-- oslo_messaging/rpc/server.py | 12 ++++++++++-- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/oslo_messaging/rpc/client.py b/oslo_messaging/rpc/client.py index 04992288f..f67f673dd 100644 --- a/oslo_messaging/rpc/client.py +++ b/oslo_messaging/rpc/client.py @@ -229,7 +229,7 @@ class RPCClient(object): class TestClient(object): def __init__(self, transport): - target = messaging.Target(topic='testtopic', version='2.0') + target = messaging.Target(topic='test', version='2.0') self._client = messaging.RPCClient(transport, target) def test(self, ctxt, arg): @@ -254,7 +254,7 @@ class RPCClient(object): For example:: transport = messaging.get_transport(cfg.CONF) - target = messaging.Target(topic='testtopic', version='2.0') + target = messaging.Target(topic='test', version='2.0') client = messaging.RPCClient(transport, target) client.call(ctxt, 'test', arg=arg) diff --git a/oslo_messaging/rpc/server.py b/oslo_messaging/rpc/server.py index 855e3d9a6..74dbede44 100644 --- a/oslo_messaging/rpc/server.py +++ b/oslo_messaging/rpc/server.py @@ -44,6 +44,7 @@ A simple example of an RPC server with multiple endpoints might be:: from oslo_config import cfg import oslo_messaging + import time class ServerControlEndpoint(object): @@ -54,7 +55,7 @@ A simple example of an RPC server with multiple endpoints might be:: self.server = server def stop(self, ctx): - if server: + if self.server: self.server.stop() class TestEndpoint(object): @@ -70,7 +71,14 @@ A simple example of an RPC server with multiple endpoints might be:: ] server = oslo_messaging.get_rpc_server(transport, target, endpoints, executor='blocking') - server.start() + try: + server.start() + while True: + time.sleep(1) + except KeyboardInterrupt: + print("Stopping server") + + server.stop() server.wait() Clients can invoke methods on the server by sending the request to a topic and From 6621b9010e6c46cfff72be4ce0f7f5c2cad81b45 Mon Sep 17 00:00:00 2001 From: Davanum Srinivas Date: Fri, 9 Oct 2015 15:44:35 -0700 Subject: [PATCH 20/74] Decouple transport for RPC and Notification Add a new configuration option for setting up an alternate notification_transport_url that can be used for notifications. This allows operators to separate the transport mechanisms used for RPC and Notifications. DocImpact Closes-Bug: #1504622 Change-Id: Ief6f95ea906bfd95b3218a930c9db5d8a764beb9 --- oslo_messaging/notify/__init__.py | 1 + oslo_messaging/notify/listener.py | 9 +++--- oslo_messaging/notify/log_handler.py | 2 +- oslo_messaging/notify/logger.py | 3 +- oslo_messaging/notify/middleware.py | 3 +- oslo_messaging/notify/notifier.py | 20 +++++++++++-- oslo_messaging/tests/notify/test_listener.py | 31 +++++++++++++------- 7 files changed, 49 insertions(+), 20 deletions(-) diff --git a/oslo_messaging/notify/__init__.py b/oslo_messaging/notify/__init__.py index dd5304d46..c946e34f8 100644 --- a/oslo_messaging/notify/__init__.py +++ b/oslo_messaging/notify/__init__.py @@ -15,6 +15,7 @@ __all__ = ['Notifier', 'LoggingNotificationHandler', + 'get_notification_transport', 'get_notification_listener', 'NotificationResult', 'NotificationFilter', diff --git a/oslo_messaging/notify/listener.py b/oslo_messaging/notify/listener.py index 3460ef962..8786e9d4a 100644 --- a/oslo_messaging/notify/listener.py +++ b/oslo_messaging/notify/listener.py @@ -19,12 +19,13 @@ 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:: +A transport can be obtained simply by calling the get_notification_transport() +method:: - transport = messaging.get_transport(conf) + transport = messaging.get_notification_transport(conf) which will load the appropriate transport driver according to the user's -messaging configuration. See get_transport() for more details. +messaging configuration. See get_notification_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 @@ -56,7 +57,7 @@ A simple example of a notification listener with multiple endpoints might be:: def error(self, ctxt, publisher_id, event_type, payload, metadata): do_something(payload) - transport = oslo_messaging.get_transport(cfg.CONF) + transport = oslo_messaging.get_notification_transport(cfg.CONF) targets = [ oslo_messaging.Target(topic='notifications') oslo_messaging.Target(topic='notifications_bis') diff --git a/oslo_messaging/notify/log_handler.py b/oslo_messaging/notify/log_handler.py index 2137a8c17..7fefb9b29 100644 --- a/oslo_messaging/notify/log_handler.py +++ b/oslo_messaging/notify/log_handler.py @@ -21,7 +21,7 @@ class LoggingErrorNotificationHandler(logging.Handler): # at runtime. import oslo_messaging logging.Handler.__init__(self, *args, **kwargs) - self._transport = oslo_messaging.get_transport(cfg.CONF) + self._transport = oslo_messaging.get_notification_transport(cfg.CONF) self._notifier = oslo_messaging.Notifier( self._transport, publisher_id='error.publisher') diff --git a/oslo_messaging/notify/logger.py b/oslo_messaging/notify/logger.py index 3748533b8..6b96b58d2 100644 --- a/oslo_messaging/notify/logger.py +++ b/oslo_messaging/notify/logger.py @@ -19,7 +19,6 @@ import logging from oslo_config import cfg from oslo_messaging.notify import notifier -from oslo_messaging import transport class LoggingNotificationHandler(logging.Handler): @@ -47,7 +46,7 @@ class LoggingNotificationHandler(logging.Handler): def __init__(self, url, publisher_id=None, driver=None, topic=None, serializer=None): self.notifier = notifier.Notifier( - transport.get_transport(self.CONF, url), + notifier.get_notification_transport(self.CONF, url), publisher_id, driver, topic, serializer() if serializer else None) diff --git a/oslo_messaging/notify/middleware.py b/oslo_messaging/notify/middleware.py index 9c6c34294..b5d3092b5 100644 --- a/oslo_messaging/notify/middleware.py +++ b/oslo_messaging/notify/middleware.py @@ -59,7 +59,8 @@ class RequestNotifier(base.Middleware): def __init__(self, app, **conf): self.notifier = notify.Notifier( - oslo_messaging.get_transport(cfg.CONF, conf.get('url')), + oslo_messaging.get_notification_transport(cfg.CONF, + conf.get('url')), publisher_id=conf.get('publisher_id', os.path.basename(sys.argv[0]))) self.service_name = conf.get('service_name') diff --git a/oslo_messaging/notify/notifier.py b/oslo_messaging/notify/notifier.py index cc8a33868..dec25b16f 100644 --- a/oslo_messaging/notify/notifier.py +++ b/oslo_messaging/notify/notifier.py @@ -25,6 +25,7 @@ import six from stevedore import named from oslo_messaging import serializer as msg_serializer +from oslo_messaging import transport as msg_transport _notifier_opts = [ cfg.MultiStrOpt('notification_driver', @@ -32,6 +33,10 @@ _notifier_opts = [ help='The Drivers(s) to handle sending notifications. ' 'Possible values are messaging, messagingv2, ' 'routing, log, test, noop'), + cfg.StrOpt('notification_transport_url', + help='A URL representing the messaging driver to use for ' + 'notifications. If not set, we fall back to the same ' + 'configuration used for RPC.'), cfg.ListOpt('notification_topics', default=['notifications', ], deprecated_name='topics', @@ -75,6 +80,15 @@ class Driver(object): pass +def get_notification_transport(conf, url=None, + allowed_remote_exmods=None, aliases=None): + if url is None: + conf.register_opts(_notifier_opts) + url = conf.notification_transport_url + return msg_transport.get_transport(conf, url, + allowed_remote_exmods, aliases) + + class Notifier(object): """Send notification messages. @@ -94,7 +108,8 @@ class Notifier(object): A Notifier object can be instantiated with a transport object and a publisher ID: - notifier = messaging.Notifier(get_transport(CONF), 'compute') + notifier = messaging.Notifier(get_notification_transport(CONF), + 'compute') and notifications are sent via drivers chosen with the notification_driver config option and on the topics chosen with the notification_topics config @@ -103,7 +118,8 @@ class Notifier(object): Alternatively, a Notifier object can be instantiated with a specific driver or topic:: - notifier = notifier.Notifier(RPC_TRANSPORT, + transport = notifier.get_notification_transport(CONF) + notifier = notifier.Notifier(transport, 'compute.host', driver='messaging', topic='notifications') diff --git a/oslo_messaging/tests/notify/test_listener.py b/oslo_messaging/tests/notify/test_listener.py index fb4a5e011..810933e31 100644 --- a/oslo_messaging/tests/notify/test_listener.py +++ b/oslo_messaging/tests/notify/test_listener.py @@ -21,6 +21,7 @@ import testscenarios import oslo_messaging from oslo_messaging.notify import dispatcher +from oslo_messaging.notify import notifier as msg_notifier from oslo_messaging.tests import utils as test_utils from six.moves import mock @@ -126,7 +127,8 @@ class TestNotifyListener(test_utils.BaseTestCase, ListenerSetupMixin): ListenerSetupMixin.setUp(self) def test_constructor(self): - transport = oslo_messaging.get_transport(self.conf, url='fake:') + transport = msg_notifier.get_notification_transport( + self.conf, url='fake:') target = oslo_messaging.Target(topic='foo') endpoints = [object()] @@ -141,7 +143,8 @@ class TestNotifyListener(test_utils.BaseTestCase, ListenerSetupMixin): self.assertEqual('blocking', listener.executor) def test_no_target_topic(self): - transport = oslo_messaging.get_transport(self.conf, url='fake:') + transport = msg_notifier.get_notification_transport( + self.conf, url='fake:') listener = oslo_messaging.get_notification_listener( transport, @@ -155,7 +158,8 @@ class TestNotifyListener(test_utils.BaseTestCase, ListenerSetupMixin): self.assertTrue(False) def test_unknown_executor(self): - transport = oslo_messaging.get_transport(self.conf, url='fake:') + transport = msg_notifier.get_notification_transport( + self.conf, url='fake:') try: oslo_messaging.get_notification_listener(transport, [], [], @@ -167,7 +171,8 @@ class TestNotifyListener(test_utils.BaseTestCase, ListenerSetupMixin): self.assertTrue(False) def test_one_topic(self): - transport = oslo_messaging.get_transport(self.conf, url='fake:') + transport = msg_notifier.get_notification_transport( + self.conf, url='fake:') endpoint = mock.Mock() endpoint.info.return_value = None @@ -184,7 +189,8 @@ class TestNotifyListener(test_utils.BaseTestCase, ListenerSetupMixin): {'message_id': mock.ANY, 'timestamp': mock.ANY}) def test_two_topics(self): - transport = oslo_messaging.get_transport(self.conf, url='fake:') + transport = msg_notifier.get_notification_transport( + self.conf, url='fake:') endpoint = mock.Mock() endpoint.info.return_value = None @@ -210,7 +216,8 @@ class TestNotifyListener(test_utils.BaseTestCase, ListenerSetupMixin): any_order=True) def test_two_exchanges(self): - transport = oslo_messaging.get_transport(self.conf, url='fake:') + transport = msg_notifier.get_notification_transport( + self.conf, url='fake:') endpoint = mock.Mock() endpoint.info.return_value = None @@ -254,7 +261,8 @@ class TestNotifyListener(test_utils.BaseTestCase, ListenerSetupMixin): any_order=True) def test_two_endpoints(self): - transport = oslo_messaging.get_transport(self.conf, url='fake:') + transport = msg_notifier.get_notification_transport( + self.conf, url='fake:') endpoint1 = mock.Mock() endpoint1.info.return_value = None @@ -279,7 +287,8 @@ class TestNotifyListener(test_utils.BaseTestCase, ListenerSetupMixin): 'message_id': mock.ANY}) def test_requeue(self): - transport = oslo_messaging.get_transport(self.conf, url='fake:') + transport = msg_notifier.get_notification_transport( + self.conf, url='fake:') endpoint = mock.Mock() endpoint.info = mock.Mock() @@ -303,7 +312,8 @@ class TestNotifyListener(test_utils.BaseTestCase, ListenerSetupMixin): {'timestamp': mock.ANY, 'message_id': mock.ANY})]) def test_two_pools(self): - transport = oslo_messaging.get_transport(self.conf, url='fake:') + transport = msg_notifier.get_notification_transport( + self.conf, url='fake:') endpoint1 = mock.Mock() endpoint1.info.return_value = None @@ -336,7 +346,8 @@ class TestNotifyListener(test_utils.BaseTestCase, ListenerSetupMixin): mocked_endpoint_call(1)]) def test_two_pools_three_listener(self): - transport = oslo_messaging.get_transport(self.conf, url='fake:') + transport = msg_notifier.get_notification_transport( + self.conf, url='fake:') endpoint1 = mock.Mock() endpoint1.info.return_value = None From 8130e833a4863d3945dffcb0818ae1cbab42ba45 Mon Sep 17 00:00:00 2001 From: OpenStack Proposal Bot Date: Wed, 11 Nov 2015 17:23:28 +0000 Subject: [PATCH 21/74] Updated from global requirements Change-Id: I3fe04f3751e79f87517d4078faea03344fe8f68e --- requirements.txt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/requirements.txt b/requirements.txt index d703e20b2..7dc7a69b5 100644 --- a/requirements.txt +++ b/requirements.txt @@ -7,7 +7,7 @@ pbr>=1.6 futurist>=0.1.2 # Apache-2.0 oslo.config>=2.6.0 # Apache-2.0 oslo.context>=0.2.0 # Apache-2.0 -oslo.log>=1.8.0 # Apache-2.0 +oslo.log>=1.12.0 # Apache-2.0 oslo.utils!=2.6.0,>=2.4.0 # Apache-2.0 oslo.serialization>=1.10.0 # Apache-2.0 oslo.service>=0.10.0 # Apache-2.0 @@ -35,7 +35,7 @@ amqp>=1.4.0 kombu>=3.0.7 # middleware -oslo.middleware>=2.8.0 # Apache-2.0 +oslo.middleware>=2.9.0 # Apache-2.0 # needed by the aioeventlet executor aioeventlet>=0.4 From 26d9362e5d8075af8feebb0f108b2c18a2c6a08a Mon Sep 17 00:00:00 2001 From: Dmitry Tantsur Date: Wed, 11 Nov 2015 10:39:06 +0100 Subject: [PATCH 22/74] Make "Connect(ing|ed) to AMQP server" log messages DEBUG level According to our guidelines, INFO level is for unit-of-work messages valueable for an operator. This rules out "connecting" message. As to "connected", while it might fall under guidelines, it seems to flood logs without too much value, see for example: http://logs.openstack.org/98/219298/9/check/gate-tempest-dsvm-ironic-pxe_ipa/53784fb/logs/screen-ir-api.txt.gz?level=INFO Change-Id: I65e0f19590c42d25e5551d45af37416a01a7d638 --- oslo_messaging/_drivers/impl_rabbit.py | 8 ++++---- oslo_messaging/tests/drivers/test_impl_rabbit.py | 6 ++++-- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/oslo_messaging/_drivers/impl_rabbit.py b/oslo_messaging/_drivers/impl_rabbit.py index eaa866882..be41cc0f5 100644 --- a/oslo_messaging/_drivers/impl_rabbit.py +++ b/oslo_messaging/_drivers/impl_rabbit.py @@ -465,8 +465,8 @@ class Connection(object): }, ) - LOG.info(_LI('Connecting to AMQP server on %(hostname)s:%(port)s'), - self.connection.info()) + LOG.debug('Connecting to AMQP server on %(hostname)s:%(port)s', + self.connection.info()) # NOTE(sileht): kombu recommend to run heartbeat_check every # seconds, but we use a lock around the kombu connection @@ -491,8 +491,8 @@ class Connection(object): if purpose == rpc_amqp.PURPOSE_SEND: self._heartbeat_start() - LOG.info(_LI('Connected to AMQP server on %(hostname)s:%(port)s'), - self.connection.info()) + LOG.debug('Connected to AMQP server on %(hostname)s:%(port)s', + self.connection.info()) # NOTE(sileht): value chosen according the best practice from kombu # http://kombu.readthedocs.org/en/latest/reference/kombu.common.html#kombu.common.eventloop diff --git a/oslo_messaging/tests/drivers/test_impl_rabbit.py b/oslo_messaging/tests/drivers/test_impl_rabbit.py index 915715450..06c78982a 100644 --- a/oslo_messaging/tests/drivers/test_impl_rabbit.py +++ b/oslo_messaging/tests/drivers/test_impl_rabbit.py @@ -91,10 +91,12 @@ class TestHeartbeat(test_utils.BaseTestCase): if not heartbeat_side_effect: self.assertEqual(1, fake_ensure_connection.call_count) - self.assertEqual(2, fake_logger.info.call_count) + self.assertEqual(2, fake_logger.debug.call_count) + self.assertEqual(0, fake_logger.info.call_count) else: self.assertEqual(2, fake_ensure_connection.call_count) - self.assertEqual(3, fake_logger.info.call_count) + self.assertEqual(2, fake_logger.debug.call_count) + self.assertEqual(1, fake_logger.info.call_count) self.assertIn(mock.call(info, mock.ANY), fake_logger.info.mock_calls) From d571b6642513974be84be115663f0bdaf441b3fe Mon Sep 17 00:00:00 2001 From: Oleksii Zamiatin Date: Wed, 11 Nov 2015 16:25:24 +0200 Subject: [PATCH 23/74] [zmq] Update zmq-deployment guide according to the new driver New driver introduced some new options and changed its architecture. The first update of the deployment guide after the driver being reimplemented. Following driver updates should be reflected in the guide as well. Change-Id: Id8629907560e335dfcff688082fe943b3657568c Closes-Bug: #1497278 --- doc/source/zmq_driver.rst | 126 +++++++++++++++++--------------------- 1 file changed, 55 insertions(+), 71 deletions(-) diff --git a/doc/source/zmq_driver.rst b/doc/source/zmq_driver.rst index eff2bf3ce..23c6d4d23 100644 --- a/doc/source/zmq_driver.rst +++ b/doc/source/zmq_driver.rst @@ -8,9 +8,9 @@ ZeroMQ Driver Deployment Guide Introduction ============ -0MQ (also known as ZeroMQ or zmq) looks like an embeddable -networking library but acts like a concurrency framework. It gives -you sockets that carry atomic messages across various transports +0MQ (also known as ZeroMQ or zmq) is embeddable networking library +but acts like a concurrency framework. It gives you sockets +that carry atomic messages across various transports like in-process, inter-process, TCP, and multicast. You can connect sockets N-to-N with patterns like fan-out, pub-sub, task distribution, and request-reply. It's fast enough to be the fabric for clustered @@ -96,8 +96,9 @@ must be set to the hostname of the current node. rpc_backend = zmq rpc_zmq_host = {hostname} + Match Making (mandatory) -------------------------- +------------------------ The ZeroMQ driver implements a matching capability to discover hosts available for communication when sending to a bare topic. This allows broker-less @@ -105,35 +106,20 @@ communications. The MatchMaker is pluggable and it provides two different MatchMaker classes. -MatchMakerLocalhost: default matchmaker driver for all-in-one scenario (messages +DummyMatchMaker: default matchmaker driver for all-in-one scenario (messages are sent to itself). -MatchMakerRing: loads a static hash table from a JSON file, sends messages to -a certain host via directed topics or cycles hosts per bare topic and supports -broker-less fanout messaging. On fanout messages returns an array of directed -topics (messages are sent to all destinations). - -MatchMakerRedis: loads the hash table from a remote Redis server, supports +RedisMatchMaker: loads the hash table from a remote Redis server, supports dynamic host/topic registrations, host expiration, and hooks for consuming applications to acknowledge or neg-acknowledge topic.host service availability. To set the MatchMaker class, use option 'rpc_zmq_matchmaker' in [DEFAULT]. - rpc_zmq_matchmaker = local - or - rpc_zmq_matchmaker = ring + rpc_zmq_matchmaker = dummy or rpc_zmq_matchmaker = redis -To specify the ring file for MatchMakerRing, use option 'ringfile' in -[matchmaker_ring]. - -For example:: - - [matchmaker_ring] - ringfile = /etc/oslo/oslo_matchmaker_ring.json - -To specify the Redis server for MatchMakerRedis, use options in +To specify the Redis server for RedisMatchMaker, use options in [matchmaker_redis] of each project. [matchmaker_redis] @@ -141,47 +127,36 @@ To specify the Redis server for MatchMakerRedis, use options in port = 6379 password = None + MatchMaker Data Source (mandatory) ------------------------------------ +---------------------------------- MatchMaker data source is stored in files or Redis server discussed in the previous section. How to make up the database is the key issue for making ZeroMQ driver work. -If deploying the MatchMakerRing, a ring file is required. The format of the ring -file should contain a hash where each key is a base topic and the values are -hostname arrays to be sent to. - -For example:: - - /etc/oslo/oslo_matchmaker_ring.json - { - "scheduler": ["host1", "host2"], - "conductor": ["host1", "host2"], - } - -The AMQP-based methods like RabbitMQ and Qpid don't require any knowledge -about the source and destination of any topic. However, ZeroMQ driver -with MatchMakerRing does. The challenging task is that you should learn -and get all the (K, V) pairs from each OpenStack project to make up the -matchmaker ring file. - -If deploying the MatchMakerRedis, a Redis server is required. Each (K, V) pair +If deploying the RedisMatchMaker, a Redis server is required. Each (K, V) pair stored in Redis is that the key is a base topic and the corresponding values are hostname arrays to be sent to. -Message Receivers (mandatory) -------------------------------- -Each machine running OpenStack services, or sending RPC messages, must run the -'oslo-messaging-zmq-receiver' daemon. This receives replies to call requests and -routes responses via IPC to blocked callers. +Proxy to avoid blocking (optional) +---------------------------------- -The way that deploy the receiver process is to run it under a new user 'oslo' -and give all openstack daemons access via group membership of 'oslo' - this -supports using /var/run/openstack as a shared IPC directory for all openstack -processes, allowing different services to be hosted on the same server, served -by a single oslo-messaging-zmq-receiver process. +Each machine running OpenStack services, or sending RPC messages, may run the +'oslo-messaging-zmq-broker' daemon. This is needed to avoid blocking +if a listener (server) appears after the sender (client). + +Running the local broker (proxy) or not is defined by the option 'zmq_use_broker' +(True by default). This option can be set in [DEFAULT] section. + +For example:: + + zmq_use_broker = False + + +In case of using the broker all publishers (clients) talk to servers over +the local broker connecting to it via IPC transport. The IPC runtime directory, 'rpc_zmq_ipc_dir', can be set in [DEFAULT] section. @@ -191,28 +166,15 @@ For example:: The parameters for the script oslo-messaging-zmq-receiver should be:: - oslo-messaging-zmq-receiver + oslo-messaging-zmq-broker --config-file /etc/oslo/zeromq.conf - --log-file /var/log/oslo/zmq-receiver.log + --log-file /var/log/oslo/zmq-broker.log You can specify ZeroMQ options in /etc/oslo/zeromq.conf if necessary. -Thread Pool (optional) ------------------------ - -Each service will launch threads for incoming requests. These threads are -maintained via a pool, the maximum number of threads is limited by -rpc_thread_pool_size. The default value is 1024. (This is a common RPC -configuration variable, also applicable to Kombu and Qpid) - -This configuration can be set in [DEFAULT] section. - -For example:: - - rpc_thread_pool_size = 1024 Listening Address (optional) ------------------------------- +---------------------------- All services bind to an IP address or Ethernet adapter. By default, all services bind to '*', effectively binding to 0.0.0.0. This may be changed with the option @@ -224,6 +186,19 @@ For example:: rpc_zmq_bind_address = * +Currently zmq driver uses dynamic port binding mechanism, which means that +each listener will allocate port of a random number. Ports range is controlled +by two options 'rpc_zmq_min_port' and 'rpc_zmq_max_port'. Change them to +restrict current service's port binding range. 'rpc_zmq_bind_port_retries' +controls number of retries before 'ports range exceeded' failure. + +For example:: + + rpc_zmq_min_port = 9050 + rpc_zmq_max_port = 10050 + rpc_zmq_bind_port_retries = 100 + + DevStack Support ---------------- @@ -232,10 +207,19 @@ ZeroMQ driver has been supported by DevStack. The configuration is as follows:: ENABLED_SERVICES+=,-rabbit,-qpid,zeromq ZEROMQ_MATCHMAKER=redis +In local.conf [localrc] section need to enable zmq plugin which lives in +`devstack-plugin-zmq`_ repository. + +For example:: + + enable_plugin zmq https://github.com/openstack/devstack-plugin-zmq.git + +.. _devstack-plugin-zmq: https://github.com/openstack/devstack-plugin-zmq.git + + Current Status ---------------- +-------------- The current development status of ZeroMQ driver is shown in `wiki`_. .. _wiki: https://wiki.openstack.org/ZeroMQ - From 517ae12b17afd6fdfa4358bf17cc4f4e109b4036 Mon Sep 17 00:00:00 2001 From: Oleksii Zamiatin Date: Wed, 11 Nov 2015 12:13:44 +0200 Subject: [PATCH 24/74] [zmq][matchmaker] Distinguish targets by listener types In order to have a possibility to pass messages via different pipelines (not over DEALER/ROUTER only) we need an information in the name service about socket type assigned to the target. Change-Id: I7cdba6c2c91af7f63ecca30c94faecef2c2eff8b Closes-Bug: #1497326 --- .../client/publishers/zmq_dealer_publisher.py | 6 ++-- .../client/publishers/zmq_pub_publisher.py | 3 +- .../client/publishers/zmq_publisher_base.py | 8 ++--- .../client/publishers/zmq_push_publisher.py | 3 +- .../client/publishers/zmq_req_publisher.py | 3 +- .../_drivers/zmq_driver/matchmaker/base.py | 27 ++++++++------ .../zmq_driver/matchmaker/matchmaker_redis.py | 22 ++++++------ .../server/consumers/zmq_router_consumer.py | 7 ++-- .../_drivers/zmq_driver/zmq_address.py | 16 ++++++--- .../zmq/matchmaker/test_impl_matchmaker.py | 36 +++++++++---------- 10 files changed, 76 insertions(+), 55 deletions(-) diff --git a/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_dealer_publisher.py b/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_dealer_publisher.py index 602e5a99d..922607c7e 100644 --- a/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_dealer_publisher.py +++ b/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_dealer_publisher.py @@ -34,7 +34,8 @@ class DealerPublisher(zmq_publisher_base.PublisherMultisend): self._check_request_pattern(request) - dealer_socket, hosts = self._check_hosts_connections(request.target) + dealer_socket, hosts = self._check_hosts_connections( + request.target, zmq_names.socket_type_str(zmq.ROUTER)) if not dealer_socket.connections: # NOTE(ozamiatin): Here we can provide @@ -104,7 +105,8 @@ class DealerPublisherProxy(DealerPublisher): LOG.info(_LI("Envelope: %s") % envelope) target = envelope[zmq_names.FIELD_TARGET] - dealer_socket, hosts = self._check_hosts_connections(target) + dealer_socket, hosts = self._check_hosts_connections( + target, zmq_names.socket_type_str(zmq.ROUTER)) if not dealer_socket.connections: # NOTE(ozamiatin): Here we can provide diff --git a/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_pub_publisher.py b/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_pub_publisher.py index 228724b6c..1cd3360eb 100644 --- a/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_pub_publisher.py +++ b/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_pub_publisher.py @@ -35,7 +35,8 @@ class PubPublisher(zmq_publisher_base.PublisherMultisend): if request.msg_type not in zmq_names.NOTIFY_TYPES: raise zmq_publisher_base.UnsupportedSendPattern(request.msg_type) - pub_socket, hosts = self._check_hosts_connections(request.target) + pub_socket, hosts = self._check_hosts_connections( + request.target, zmq_names.socket_type_str(zmq.SUB)) self._send_request(pub_socket, request) def _send_request(self, socket, request): diff --git a/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_publisher_base.py b/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_publisher_base.py index 0a8098af3..46e8ef535 100644 --- a/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_publisher_base.py +++ b/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_publisher_base.py @@ -90,11 +90,10 @@ class PublisherBase(object): :type request: zmq_request.Request """ LOG.info(_LI("Sending %(type)s message_id %(message)s to a target" - "%(target)s key: %(key)s, host:%(host)s") + "%(target)s host:%(host)s") % {"type": request.msg_type, "message": request.message_id, "target": request.target, - "key": zmq_address.target_to_key(request.target), "host": request.host}) socket.send_pyobj(request) @@ -122,10 +121,11 @@ class PublisherMultisend(PublisherBase): self.socket_type = socket_type self.matchmaker = matchmaker - def _check_hosts_connections(self, target): + def _check_hosts_connections(self, target, listener_type): # TODO(ozamiatin): Place for significant optimization # Matchmaker cache should be implemented - hosts = self.matchmaker.get_hosts(target) + hosts = self.matchmaker.get_hosts( + target, listener_type) if str(target) in self.outbound_sockets: socket = self.outbound_sockets[str(target)] else: diff --git a/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_push_publisher.py b/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_push_publisher.py index b8fc4fe51..7fcb46961 100644 --- a/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_push_publisher.py +++ b/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_push_publisher.py @@ -35,7 +35,8 @@ class PushPublisher(zmq_publisher_base.PublisherMultisend): if request.msg_type == zmq_names.CALL_TYPE: raise zmq_publisher_base.UnsupportedSendPattern(request.msg_type) - push_socket, hosts = self._check_hosts_connections(request.target) + push_socket, hosts = self._check_hosts_connections( + request.target, zmq_names.socket_type_str(zmq.PULL)) if not push_socket.connections: LOG.warning(_LW("Request %s was dropped because no connection") diff --git a/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_req_publisher.py b/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_req_publisher.py index c6063b8ab..ace229ba5 100644 --- a/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_req_publisher.py +++ b/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_req_publisher.py @@ -50,7 +50,8 @@ class ReqPublisher(zmq_publisher_base.PublisherBase): return self._receive_reply(socket, request) def _resolve_host_address(self, target, timeout=0): - host = self.matchmaker.get_single_host(target, timeout) + host = self.matchmaker.get_single_host( + target, zmq_names.socket_type_str(zmq.ROUTER), timeout) return zmq_address.get_tcp_direct_address(host) def _connect_to_host(self, target, timeout=0): diff --git a/oslo_messaging/_drivers/zmq_driver/matchmaker/base.py b/oslo_messaging/_drivers/zmq_driver/matchmaker/base.py index 8b2365b41..7b9b69d79 100644 --- a/oslo_messaging/_drivers/zmq_driver/matchmaker/base.py +++ b/oslo_messaging/_drivers/zmq_driver/matchmaker/base.py @@ -20,6 +20,7 @@ import retrying import six import oslo_messaging +from oslo_messaging._drivers.zmq_driver import zmq_address from oslo_messaging._i18n import _LI, _LW @@ -35,27 +36,31 @@ class MatchMakerBase(object): self.conf = conf @abc.abstractmethod - def register(self, target, hostname): + def register(self, target, hostname, listener_type): """Register target on nameserver. :param target: the target for host :type target: Target :param hostname: host for the topic in "host:port" format :type hostname: String + :param listener_type: Listener socket type ROUTER, SUB etc. + :type listener_type: String """ @abc.abstractmethod - def unregister(self, target, hostname): + def unregister(self, target, hostname, listener_type): """Unregister target from nameserver. :param target: the target for host :type target: Target :param hostname: host for the topic in "host:port" format :type hostname: String + :param listener_type: Listener socket type ROUTER, SUB etc. + :type listener_type: String """ @abc.abstractmethod - def get_hosts(self, target): + def get_hosts(self, target, listener_type): """Get all hosts from nameserver by target. :param target: the default target for invocations @@ -63,7 +68,7 @@ class MatchMakerBase(object): :returns: a list of "hostname:port" hosts """ - def get_single_host(self, target, timeout=None, retry=0): + def get_single_host(self, target, listener_type, timeout=None, retry=0): """Get a single host by target. :param target: the target for messages @@ -101,7 +106,7 @@ class MatchMakerBase(object): @_retry def _get_single_host(): - hosts = self.get_hosts(target) + hosts = self.get_hosts(target, listener_type) try: if not hosts: err_msg = "No hosts were found for target %s." % target @@ -136,16 +141,16 @@ class DummyMatchMaker(MatchMakerBase): self._cache = collections.defaultdict(list) - def register(self, target, hostname): - key = str(target) + def register(self, target, hostname, listener_type): + key = zmq_address.target_to_key(target, listener_type) if hostname not in self._cache[key]: self._cache[key].append(hostname) - def unregister(self, target, hostname): - key = str(target) + def unregister(self, target, hostname, listener_type): + key = zmq_address.target_to_key(target, listener_type) if hostname in self._cache[key]: self._cache[key].remove(hostname) - def get_hosts(self, target): - key = str(target) + def get_hosts(self, target, listener_type): + key = zmq_address.target_to_key(target, listener_type) return self._cache[key] diff --git a/oslo_messaging/_drivers/zmq_driver/matchmaker/matchmaker_redis.py b/oslo_messaging/_drivers/zmq_driver/matchmaker/matchmaker_redis.py index cbf4e1066..576566a2f 100644 --- a/oslo_messaging/_drivers/zmq_driver/matchmaker/matchmaker_redis.py +++ b/oslo_messaging/_drivers/zmq_driver/matchmaker/matchmaker_redis.py @@ -52,27 +52,29 @@ class RedisMatchMaker(base.MatchMakerBase): def _get_hosts_by_key(self, key): return self._redis.lrange(key, 0, -1) - def register(self, target, hostname): + def register(self, target, hostname, listener_type): if target.topic and target.server: - key = zmq_address.target_to_key(target) + key = zmq_address.target_to_key(target, listener_type) if hostname not in self._get_hosts_by_key(key): self._redis.lpush(key, hostname) if target.topic: - if hostname not in self._get_hosts_by_key(target.topic): - self._redis.lpush(target.topic, hostname) + key = zmq_address.prefix_str(target.topic, listener_type) + if hostname not in self._get_hosts_by_key(key): + self._redis.lpush(key, hostname) if target.server: - if hostname not in self._get_hosts_by_key(target.server): - self._redis.lpush(target.server, hostname) + key = zmq_address.prefix_str(target.server, listener_type) + if hostname not in self._get_hosts_by_key(key): + self._redis.lpush(key, hostname) - def unregister(self, target, hostname): - key = zmq_address.target_to_key(target) + def unregister(self, target, hostname, listener_type): + key = zmq_address.target_to_key(target, listener_type) self._redis.lrem(key, 0, hostname) - def get_hosts(self, target): + def get_hosts(self, target, listener_type): hosts = [] - key = zmq_address.target_to_key(target) + key = zmq_address.target_to_key(target, listener_type) hosts.extend(self._get_hosts_by_key(key)) return hosts diff --git a/oslo_messaging/_drivers/zmq_driver/server/consumers/zmq_router_consumer.py b/oslo_messaging/_drivers/zmq_driver/server/consumers/zmq_router_consumer.py index 9c529dac0..a501ef7a7 100644 --- a/oslo_messaging/_drivers/zmq_driver/server/consumers/zmq_router_consumer.py +++ b/oslo_messaging/_drivers/zmq_driver/server/consumers/zmq_router_consumer.py @@ -64,13 +64,14 @@ class RouterConsumer(zmq_consumer_base.SingleSocketConsumer): LOG.info("[%s] Listen to target %s" % (self.host, target)) self.targets.append(target) - self.matchmaker.register(target=target, - hostname=self.host) + self.matchmaker.register(target, self.host, + zmq_names.socket_type_str(zmq.ROUTER)) def cleanup(self): super(RouterConsumer, self).cleanup() for target in self.targets: - self.matchmaker.unregister(target, self.host) + self.matchmaker.unregister(target, self.host, + zmq_names.socket_type_str(zmq.ROUTER)) def _receive_request(self, socket): reply_id = socket.recv() diff --git a/oslo_messaging/_drivers/zmq_driver/zmq_address.py b/oslo_messaging/_drivers/zmq_driver/zmq_address.py index afc92490f..b0ca2eba9 100644 --- a/oslo_messaging/_drivers/zmq_driver/zmq_address.py +++ b/oslo_messaging/_drivers/zmq_driver/zmq_address.py @@ -29,12 +29,20 @@ def get_broker_address(conf): return "ipc://%s/zmq-broker" % conf.rpc_zmq_ipc_dir -def target_to_key(target): +def prefix_str(key, listener_type): + return listener_type + "_" + key + + +def target_to_key(target, listener_type): + + def prefix(key): + return prefix_str(key, listener_type) + if target.topic and target.server: attributes = ['topic', 'server'] key = ".".join(getattr(target, attr) for attr in attributes) - return key + return prefix(key) if target.topic: - return target.topic + return prefix(target.topic) if target.server: - return target.server + return prefix(target.server) diff --git a/oslo_messaging/tests/drivers/zmq/matchmaker/test_impl_matchmaker.py b/oslo_messaging/tests/drivers/zmq/matchmaker/test_impl_matchmaker.py index ba5f1f399..5751e5ba3 100644 --- a/oslo_messaging/tests/drivers/zmq/matchmaker/test_impl_matchmaker.py +++ b/oslo_messaging/tests/drivers/zmq/matchmaker/test_impl_matchmaker.py @@ -62,47 +62,47 @@ class TestImplMatchmaker(test_utils.BaseTestCase): self.host2 = b"test_host2" def test_register(self): - self.test_matcher.register(self.target, self.host1) + self.test_matcher.register(self.target, self.host1, "test") - self.assertEqual(self.test_matcher.get_hosts(self.target), + self.assertEqual(self.test_matcher.get_hosts(self.target, "test"), [self.host1]) - self.assertEqual(self.test_matcher.get_single_host(self.target), + self.assertEqual(self.test_matcher.get_single_host(self.target, "test"), self.host1) def test_register_two_hosts(self): - self.test_matcher.register(self.target, self.host1) - self.test_matcher.register(self.target, self.host2) + self.test_matcher.register(self.target, self.host1, "test") + self.test_matcher.register(self.target, self.host2, "test") - self.assertItemsEqual(self.test_matcher.get_hosts(self.target), + self.assertItemsEqual(self.test_matcher.get_hosts(self.target, "test"), [self.host1, self.host2]) - self.assertIn(self.test_matcher.get_single_host(self.target), + self.assertIn(self.test_matcher.get_single_host(self.target, "test"), [self.host1, self.host2]) def test_register_unsibscribe(self): - self.test_matcher.register(self.target, self.host1) - self.test_matcher.register(self.target, self.host2) + self.test_matcher.register(self.target, self.host1, "test") + self.test_matcher.register(self.target, self.host2, "test") - self.test_matcher.unregister(self.target, self.host2) + self.test_matcher.unregister(self.target, self.host2, "test") - self.assertItemsEqual(self.test_matcher.get_hosts(self.target), + self.assertItemsEqual(self.test_matcher.get_hosts(self.target, "test"), [self.host1]) - self.assertNotIn(self.test_matcher.get_single_host(self.target), + self.assertNotIn(self.test_matcher.get_single_host(self.target, "test"), [self.host2]) def test_register_two_same_hosts(self): - self.test_matcher.register(self.target, self.host1) - self.test_matcher.register(self.target, self.host1) + self.test_matcher.register(self.target, self.host1, "test") + self.test_matcher.register(self.target, self.host1, "test") - self.assertEqual(self.test_matcher.get_hosts(self.target), + self.assertEqual(self.test_matcher.get_hosts(self.target, "test"), [self.host1]) - self.assertEqual(self.test_matcher.get_single_host(self.target), + self.assertEqual(self.test_matcher.get_single_host(self.target, "test"), self.host1) def test_get_hosts_wrong_topic(self): target = oslo_messaging.Target(topic="no_such_topic") - self.assertEqual(self.test_matcher.get_hosts(target), []) + self.assertEqual(self.test_matcher.get_hosts(target, "test"), []) def test_get_single_host_wrong_topic(self): target = oslo_messaging.Target(topic="no_such_topic") self.assertRaises(oslo_messaging.InvalidTarget, - self.test_matcher.get_single_host, target) + self.test_matcher.get_single_host, target, "test") From 018dfcd6415258c59df70d8012d8c7dc73a0752b Mon Sep 17 00:00:00 2001 From: Oleksii Zamiatin Date: Thu, 12 Nov 2015 15:14:01 +0200 Subject: [PATCH 25/74] [zmq] Make bind address configurable Makes use of exisiting 'rpc_zmq_bind_address' option in order to make binding address configurable. Change-Id: Ia46fa03e54b0e92d3504d9a0ebd65171a283e073 Closes-Bug: #1515267 --- oslo_messaging/_drivers/zmq_driver/zmq_address.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/oslo_messaging/_drivers/zmq_driver/zmq_address.py b/oslo_messaging/_drivers/zmq_driver/zmq_address.py index afc92490f..15bf20435 100644 --- a/oslo_messaging/_drivers/zmq_driver/zmq_address.py +++ b/oslo_messaging/_drivers/zmq_driver/zmq_address.py @@ -22,7 +22,7 @@ def get_tcp_direct_address(host): def get_tcp_random_address(conf): - return "tcp://*" + return "tcp://%s" % conf.rpc_zmq_bind_address def get_broker_address(conf): From 4a3ddce05bac17903fb768a47b042d4bc17fd0d9 Mon Sep 17 00:00:00 2001 From: Oleksii Zamiatin Date: Wed, 11 Nov 2015 14:12:20 +0200 Subject: [PATCH 26/74] [zmq] Add config options to specify dynamic ports range Operators may need a possibility to restrict ports ranges for a specified services in order to distinguish ports related to zmq messaging from all other ports in a system. Change-Id: Ibe5b02c1211b16859ff58bc02a59d96e1d2fa660 Closes-Bug: #1511181 --- oslo_messaging/_drivers/impl_zmq.py | 15 ++++++++++++- .../_drivers/zmq_driver/zmq_socket.py | 17 ++++++++++++++- .../tests/drivers/zmq/test_impl_zmq.py | 21 ++++++++++++++++++- 3 files changed, 50 insertions(+), 3 deletions(-) diff --git a/oslo_messaging/_drivers/impl_zmq.py b/oslo_messaging/_drivers/impl_zmq.py index aed6563a9..acff79c7b 100644 --- a/oslo_messaging/_drivers/impl_zmq.py +++ b/oslo_messaging/_drivers/impl_zmq.py @@ -79,7 +79,20 @@ zmq_opts = [ cfg.BoolOpt('zmq_use_broker', default=True, - help='Shows whether zmq-messaging uses broker or not.') + help='Shows whether zmq-messaging uses broker or not.'), + + cfg.IntOpt('rpc_zmq_min_port', + default=49152, + help='Minimal port number for random ports range.'), + + cfg.IntOpt('rpc_zmq_max_port', + default=65536, + help='Maximal port number for random ports range.'), + + cfg.IntOpt('rpc_zmq_bind_port_retries', + default=100, + help='Number of retries to find free port number before ' + 'fail with ZMQBindError.') ] diff --git a/oslo_messaging/_drivers/zmq_driver/zmq_socket.py b/oslo_messaging/_drivers/zmq_driver/zmq_socket.py index 8e51e30f1..8f79bd083 100644 --- a/oslo_messaging/_drivers/zmq_driver/zmq_socket.py +++ b/oslo_messaging/_drivers/zmq_driver/zmq_socket.py @@ -17,6 +17,8 @@ import logging from oslo_messaging._drivers.zmq_driver import zmq_address from oslo_messaging._drivers.zmq_driver import zmq_async from oslo_messaging._drivers.zmq_driver import zmq_names +from oslo_messaging._i18n import _LE +from oslo_messaging import exceptions LOG = logging.getLogger(__name__) @@ -79,10 +81,23 @@ class ZmqSocket(object): self.handle.close(*args, **kwargs) +class ZmqPortRangeExceededException(exceptions.MessagingException): + """Raised by ZmqRandomPortSocket - wrapping zmq.ZMQBindError""" + + class ZmqRandomPortSocket(ZmqSocket): def __init__(self, conf, context, socket_type): super(ZmqRandomPortSocket, self).__init__(context, socket_type) self.conf = conf self.bind_address = zmq_address.get_tcp_random_address(self.conf) - self.port = self.handle.bind_to_random_port(self.bind_address) + + try: + self.port = self.handle.bind_to_random_port( + self.bind_address, + min_port=conf.rpc_zmq_min_port, + max_port=conf.rpc_zmq_max_port, + max_tries=conf.rpc_zmq_bind_port_retries) + except zmq.ZMQBindError: + LOG.error(_LE("Random ports range exceeded!")) + raise ZmqPortRangeExceededException() diff --git a/oslo_messaging/tests/drivers/zmq/test_impl_zmq.py b/oslo_messaging/tests/drivers/zmq/test_impl_zmq.py index c40007523..feb1f7656 100644 --- a/oslo_messaging/tests/drivers/zmq/test_impl_zmq.py +++ b/oslo_messaging/tests/drivers/zmq/test_impl_zmq.py @@ -21,6 +21,7 @@ import testtools import oslo_messaging from oslo_messaging._drivers import impl_zmq from oslo_messaging._drivers.zmq_driver import zmq_async +from oslo_messaging._drivers.zmq_driver import zmq_socket from oslo_messaging._i18n import _ from oslo_messaging.tests import utils as test_utils @@ -78,7 +79,9 @@ class ZmqBaseTestCase(test_utils.BaseTestCase): 'rpc_response_timeout': 5, 'rpc_zmq_ipc_dir': self.internal_ipc_dir, 'zmq_use_broker': False, - 'rpc_zmq_matchmaker': 'dummy'} + 'rpc_zmq_matchmaker': 'dummy', + 'rpc_zmq_min_port': 5555, + 'rpc_zmq_max_port': 5560} self.config(**kwargs) # Get driver @@ -115,6 +118,22 @@ class stopRpc(object): class TestZmqBasics(ZmqBaseTestCase): + def test_ports_range(self): + listeners = [] + + for i in range(10): + try: + target = oslo_messaging.Target(topic='testtopic_'+str(i)) + new_listener = self.driver.listen(target) + listeners.append(new_listener) + except zmq_socket.ZmqPortRangeExceededException: + pass + + self.assertLessEqual(len(listeners), 5) + + for l in listeners: + l.cleanup() + def test_send_receive_raises(self): """Call() without method.""" target = oslo_messaging.Target(topic='testtopic') From a2ff73cb801344737032475820a0976e356f47ad Mon Sep 17 00:00:00 2001 From: Davanum Srinivas Date: Sat, 14 Nov 2015 23:06:55 -0500 Subject: [PATCH 27/74] cleanup tox.ini * Remove requirements.txt from deps, as this is already added automatically Change-Id: I696dd69ba1d59ab026180e8e3cb864fe37442e10 --- tox.ini | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tox.ini b/tox.ini index 1689b87a9..8f110014d 100644 --- a/tox.ini +++ b/tox.ini @@ -5,8 +5,7 @@ envlist = py34,py27,pep8,bandit setenv = VIRTUAL_ENV={envdir} OS_TEST_TIMEOUT=30 -deps = -r{toxinidir}/requirements.txt - -r{toxinidir}/test-requirements.txt +deps = -r{toxinidir}/test-requirements.txt commands = python setup.py testr --slowest --testr-args='{posargs}' [testenv:pep8] From bfa6f5af038ce9e86521dcfefb7f1536b9cf5b32 Mon Sep 17 00:00:00 2001 From: OpenStack Proposal Bot Date: Tue, 17 Nov 2015 02:30:04 +0000 Subject: [PATCH 28/74] Updated from global requirements Change-Id: Ic5436cf41d5e6c10e4080bc6c42694d2542c970c --- requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/requirements.txt b/requirements.txt index 7dc7a69b5..6eea9e1d3 100644 --- a/requirements.txt +++ b/requirements.txt @@ -10,7 +10,7 @@ oslo.context>=0.2.0 # Apache-2.0 oslo.log>=1.12.0 # Apache-2.0 oslo.utils!=2.6.0,>=2.4.0 # Apache-2.0 oslo.serialization>=1.10.0 # Apache-2.0 -oslo.service>=0.10.0 # Apache-2.0 +oslo.service>=0.12.0 # Apache-2.0 oslo.i18n>=1.5.0 # Apache-2.0 stevedore>=1.5.0 # Apache-2.0 From 00d07f5205c758757bb854372c5576e62a5f57d6 Mon Sep 17 00:00:00 2001 From: Matthew Booth Date: Mon, 19 Oct 2015 14:11:23 +0100 Subject: [PATCH 29/74] Robustify locking in MessageHandlingServer This change formalises locking in MessageHandlingServer, which closes several bugs: * It adds locking for internal state when using the blocking executor, which closes a number of races. * It does not hold a lock while executing server functions, which removes a potential cause of deadlock if the server does its own locking. * It fixes a regression introduced in change gI3cfbe1bf02d451e379b1dcc23dacb0139c03be76. If multiple threads called wait() simultaneously, only 1 of them would wait and the others would return immediately, despite message handling not having completed. With this change only 1 will call the underlying wait, but all will wait on its completion. Additionally, it introduces some new functionality: * It allows the user to make calls in any order and it will ensure, with locking, that these will be reordered appropriately. * The caller can pass a `timeout` argument to any server method, which will cause it to raise an exception if it waits too long. * The caller can pass a `log_after` argument to any server method, which will cause it to raise a log message if it waits too long. It can also be used to disable logging when waiting is intentional. We remove DummyCondition as it no longer has any users. This change was originally committed as change I9d516b208446963dcd80b75e2d5a2cecb1187efa, but was reverted as it caused a hang in a Nova test. This was caused by the locking behaviour for handling restarting a previously stopped server. The original patch caused the state to 'wrap' immediately after the user called wait(). This caused a hang in tests which redundantly called stop() and wait() multiple times. This new patch only wraps when the user calls start() again. Callers who do not restart a server will therefore not be affected by the wrapping behaviour. Callers who do restart a server will be no worse than before. We add a deprecation warning on restart, as this operation is inherently racy with this api and there is a simple, safe alternative. This new version has been successfully tested against the unit and functional tests of nova, cinder, glance, and ceilometer. Change-Id: Ic79f87e7b069c1f62d6121486fd6cafd732fdde7 --- oslo_messaging/_utils.py | 23 -- oslo_messaging/server.py | 352 +++++++++++++++++++----- oslo_messaging/tests/rpc/test_server.py | 302 ++++++++++++++++++++ 3 files changed, 585 insertions(+), 92 deletions(-) diff --git a/oslo_messaging/_utils.py b/oslo_messaging/_utils.py index 1bb20b089..cec94bb48 100644 --- a/oslo_messaging/_utils.py +++ b/oslo_messaging/_utils.py @@ -116,29 +116,6 @@ def fetch_current_thread_functor(): return lambda: threading.current_thread() -class DummyCondition(object): - def acquire(self): - pass - - def notify(self): - pass - - def notify_all(self): - pass - - def wait(self, timeout=None): - pass - - def release(self): - pass - - def __enter__(self): - self.acquire() - - def __exit__(self, type, value, traceback): - self.release() - - class DummyLock(object): def acquire(self): pass diff --git a/oslo_messaging/server.py b/oslo_messaging/server.py index 491ccbf52..804bcf105 100644 --- a/oslo_messaging/server.py +++ b/oslo_messaging/server.py @@ -23,20 +23,25 @@ __all__ = [ 'ServerListenError', ] +import functools +import inspect import logging import threading +import traceback from oslo_service import service from oslo_utils import timeutils from stevedore import driver from oslo_messaging._drivers import base as driver_base -from oslo_messaging._i18n import _LW -from oslo_messaging import _utils from oslo_messaging import exceptions LOG = logging.getLogger(__name__) +# The default number of seconds of waiting after which we will emit a log +# message +DEFAULT_LOG_AFTER = 30 + class MessagingServerError(exceptions.MessagingException): """Base class for all MessageHandlingServer exceptions.""" @@ -62,7 +67,223 @@ class ServerListenError(MessagingServerError): self.ex = ex -class MessageHandlingServer(service.ServiceBase): +class TaskTimeout(MessagingServerError): + """Raised if we timed out waiting for a task to complete.""" + + +class _OrderedTask(object): + """A task which must be executed in a particular order. + + A caller may wait for this task to complete by calling + `wait_for_completion`. + + A caller may run this task with `run_once`, which will ensure that however + many times the task is called it only runs once. Simultaneous callers will + block until the running task completes, which means that any caller can be + sure that the task has completed after run_once returns. + """ + + INIT = 0 # The task has not yet started + RUNNING = 1 # The task is running somewhere + COMPLETE = 2 # The task has run somewhere + + def __init__(self, name): + """Create a new _OrderedTask. + + :param name: The name of this task. Used in log messages. + """ + super(_OrderedTask, self).__init__() + + self._name = name + self._cond = threading.Condition() + self._state = self.INIT + + def _wait(self, condition, msg, log_after, timeout_timer): + """Wait while condition() is true. Write a log message if condition() + has not become false within `log_after` seconds. Raise TaskTimeout if + timeout_timer expires while waiting. + """ + + log_timer = None + if log_after != 0: + log_timer = timeutils.StopWatch(duration=log_after) + log_timer.start() + + while condition(): + if log_timer is not None and log_timer.expired(): + LOG.warn('Possible hang: %s' % msg) + LOG.debug(''.join(traceback.format_stack())) + # Only log once. After than we wait indefinitely without + # logging. + log_timer = None + + if timeout_timer is not None and timeout_timer.expired(): + raise TaskTimeout(msg) + + timeouts = [] + if log_timer is not None: + timeouts.append(log_timer.leftover()) + if timeout_timer is not None: + timeouts.append(timeout_timer.leftover()) + + wait = None + if timeouts: + wait = min(timeouts) + self._cond.wait(wait) + + @property + def complete(self): + return self._state == self.COMPLETE + + def wait_for_completion(self, caller, log_after, timeout_timer): + """Wait until this task has completed. + + :param caller: The name of the task which is waiting. + :param log_after: Emit a log message if waiting longer than `log_after` + seconds. + :param timeout_timer: Raise TaskTimeout if StopWatch object + `timeout_timer` expires while waiting. + """ + with self._cond: + msg = '%s is waiting for %s to complete' % (caller, self._name) + self._wait(lambda: not self.complete, + msg, log_after, timeout_timer) + + def run_once(self, fn, log_after, timeout_timer): + """Run a task exactly once. If it is currently running in another + thread, wait for it to complete. If it has already run, return + immediately without running it again. + + :param fn: The task to run. It must be a callable taking no arguments. + It may optionally return another callable, which also takes + no arguments, which will be executed after completion has + been signaled to other threads. + :param log_after: Emit a log message if waiting longer than `log_after` + seconds. + :param timeout_timer: Raise TaskTimeout if StopWatch object + `timeout_timer` expires while waiting. + """ + with self._cond: + if self._state == self.INIT: + self._state = self.RUNNING + # Note that nothing waits on RUNNING, so no need to notify + + # We need to release the condition lock before calling out to + # prevent deadlocks. Reacquire it immediately afterwards. + self._cond.release() + try: + post_fn = fn() + finally: + self._cond.acquire() + self._state = self.COMPLETE + self._cond.notify_all() + + if post_fn is not None: + # Release the condition lock before calling out to prevent + # deadlocks. Reacquire it immediately afterwards. + self._cond.release() + try: + post_fn() + finally: + self._cond.acquire() + elif self._state == self.RUNNING: + msg = ('%s is waiting for another thread to complete' + % self._name) + self._wait(lambda: self._state == self.RUNNING, + msg, log_after, timeout_timer) + + +class _OrderedTaskRunner(object): + """Mixin for a class which executes ordered tasks.""" + + def __init__(self, *args, **kwargs): + super(_OrderedTaskRunner, self).__init__(*args, **kwargs) + + # Get a list of methods on this object which have the _ordered + # attribute + self._tasks = [name + for (name, member) in inspect.getmembers(self) + if inspect.ismethod(member) and + getattr(member, '_ordered', False)] + self.reset_states() + + self._reset_lock = threading.Lock() + + def reset_states(self): + # Create new task states for tasks in reset + self._states = {task: _OrderedTask(task) for task in self._tasks} + + @staticmethod + def decorate_ordered(fn, state, after, reset_after): + + @functools.wraps(fn) + def wrapper(self, *args, **kwargs): + # If the reset_after state has already completed, reset state so + # we can run again. + # NOTE(mdbooth): This is ugly and requires external locking to be + # deterministic when using multiple threads. Consider a thread that + # does: server.stop(), server.wait(). If another thread causes a + # reset between stop() and wait(), this will not have the intended + # behaviour. It is safe without external locking, if the caller + # instantiates a new object. + with self._reset_lock: + if (reset_after is not None and + self._states[reset_after].complete): + self.reset_states() + + # Store the states we started with in case the state wraps on us + # while we're sleeping. We must wait and run_once in the same + # epoch. If the epoch ended while we were sleeping, run_once will + # safely do nothing. + states = self._states + + log_after = kwargs.pop('log_after', DEFAULT_LOG_AFTER) + timeout = kwargs.pop('timeout', None) + + timeout_timer = None + if timeout is not None: + timeout_timer = timeutils.StopWatch(duration=timeout) + timeout_timer.start() + + # Wait for the given preceding state to complete + if after is not None: + states[after].wait_for_completion(state, + log_after, timeout_timer) + + # Run this state + states[state].run_once(lambda: fn(self, *args, **kwargs), + log_after, timeout_timer) + return wrapper + + +def ordered(after=None, reset_after=None): + """A method which will be executed as an ordered task. The method will be + called exactly once, however many times it is called. If it is called + multiple times simultaneously it will only be called once, but all callers + will wait until execution is complete. + + If `after` is given, this method will not run until `after` has completed. + + If `reset_after` is given and the target method has completed, allow this + task to run again by resetting all task states. + + :param after: Optionally, the name of another `ordered` method. Wait for + the completion of `after` before executing this method. + :param reset_after: Optionally, the name of another `ordered` method. Reset + all states when calling this method if `reset_after` + has completed. + """ + def _ordered(fn): + # Set an attribute on the method so we can find it later + setattr(fn, '_ordered', True) + state = fn.__name__ + + return _OrderedTaskRunner.decorate_ordered(fn, state, after, + reset_after) + return _ordered + + +class MessageHandlingServer(service.ServiceBase, _OrderedTaskRunner): """Server for handling messages. Connect a transport to a dispatcher that knows how to process the @@ -94,29 +315,20 @@ class MessageHandlingServer(service.ServiceBase): self.dispatcher = dispatcher self.executor = executor - # NOTE(sileht): we use a lock to protect the state change of the - # server, we don't want to call stop until the transport driver - # is fully started. Except for the blocking executor that have - # start() that doesn't return - if self.executor != "blocking": - self._state_cond = threading.Condition() - self._dummy_cond = False - else: - self._state_cond = _utils.DummyCondition() - self._dummy_cond = True - try: mgr = driver.DriverManager('oslo.messaging.executors', self.executor) except RuntimeError as ex: raise ExecutorLoadFailure(self.executor, ex) - else: - self._executor_cls = mgr.driver - self._executor_obj = None - self._running = False + + self._executor_cls = mgr.driver + self._executor_obj = None + + self._started = False super(MessageHandlingServer, self).__init__() + @ordered(reset_after='stop') def start(self): """Start handling incoming messages. @@ -130,25 +342,39 @@ class MessageHandlingServer(service.ServiceBase): registering a callback with an event loop. Similarly, the executor may choose to dispatch messages in a new thread, coroutine or simply the current thread. + + :param log_after: Emit a log message if waiting longer than `log_after` + seconds to run this task. If set to zero, no log + message will be emitted. Defaults to 30 seconds. + :type log_after: int + :param timeout: Raise `TaskTimeout` if the task has to wait longer than + `timeout` seconds before executing. + :type timeout: int """ - if self._executor_obj is not None: - return - with self._state_cond: - if self._executor_obj is not None: - return - try: - listener = self.dispatcher._listen(self.transport) - except driver_base.TransportDriverError as ex: - raise ServerListenError(self.target, ex) - self._executor_obj = self._executor_cls(self.conf, listener, - self.dispatcher) - self._executor_obj.start() - self._running = True - self._state_cond.notify_all() + # Warn that restarting will be deprecated + if self._started: + LOG.warn('Restarting a MessageHandlingServer is inherently racy. ' + 'It is deprecated, and will become a noop in a future ' + 'release of oslo.messaging. If you need to restart ' + 'MessageHandlingServer you should instantiate a new ' + 'object.') + self._started = True + + try: + listener = self.dispatcher._listen(self.transport) + except driver_base.TransportDriverError as ex: + raise ServerListenError(self.target, ex) + executor = self._executor_cls(self.conf, listener, self.dispatcher) + executor.start() + self._executor_obj = executor if self.executor == 'blocking': - self._executor_obj.execute() + # N.B. This will be executed unlocked and unordered, so + # we can't rely on the value of self._executor_obj when this runs. + # We explicitly pass the local variable. + return lambda: executor.execute() + @ordered(after='start') def stop(self): """Stop handling incoming messages. @@ -156,13 +382,18 @@ class MessageHandlingServer(service.ServiceBase): the server. However, the server may still be in the process of handling some messages, and underlying driver resources associated to this server are still in use. See 'wait' for more details. - """ - with self._state_cond: - if self._executor_obj is not None: - self._running = False - self._executor_obj.stop() - self._state_cond.notify_all() + :param log_after: Emit a log message if waiting longer than `log_after` + seconds to run this task. If set to zero, no log + message will be emitted. Defaults to 30 seconds. + :type log_after: int + :param timeout: Raise `TaskTimeout` if the task has to wait longer than + `timeout` seconds before executing. + :type timeout: int + """ + self._executor_obj.stop() + + @ordered(after='stop') def wait(self): """Wait for message processing to complete. @@ -172,38 +403,21 @@ class MessageHandlingServer(service.ServiceBase): Once it's finished, the underlying driver resources associated to this server are released (like closing useless network connections). + + :param log_after: Emit a log message if waiting longer than `log_after` + seconds to run this task. If set to zero, no log + message will be emitted. Defaults to 30 seconds. + :type log_after: int + :param timeout: Raise `TaskTimeout` if the task has to wait longer than + `timeout` seconds before executing. + :type timeout: int """ - with self._state_cond: - if self._running: - LOG.warn(_LW("wait() should be called after stop() as it " - "waits for existing messages to finish " - "processing")) - w = timeutils.StopWatch() - w.start() - while self._running: - # NOTE(harlowja): 1.0 seconds was mostly chosen at - # random, but it seems like a reasonable value to - # use to avoid spamming the logs with to much - # information. - self._state_cond.wait(1.0) - if self._running and not self._dummy_cond: - LOG.warn( - _LW("wait() should have been called" - " after stop() as wait() waits for existing" - " messages to finish processing, it has" - " been %0.2f seconds and stop() still has" - " not been called"), w.elapsed()) - executor = self._executor_obj + try: + self._executor_obj.wait() + finally: + # Close listener connection after processing all messages + self._executor_obj.listener.cleanup() self._executor_obj = None - if executor is not None: - # We are the lucky calling thread to wait on the executor to - # actually finish. - try: - executor.wait() - finally: - # Close listener connection after processing all messages - executor.listener.cleanup() - executor = None def reset(self): """Reset service. diff --git a/oslo_messaging/tests/rpc/test_server.py b/oslo_messaging/tests/rpc/test_server.py index 258dacb24..846ea86e2 100644 --- a/oslo_messaging/tests/rpc/test_server.py +++ b/oslo_messaging/tests/rpc/test_server.py @@ -13,6 +13,8 @@ # License for the specific language governing permissions and limitations # under the License. +import eventlet +import time import threading from oslo_config import cfg @@ -20,6 +22,7 @@ import testscenarios import mock import oslo_messaging +from oslo_messaging import server as server_module from oslo_messaging.tests import utils as test_utils load_tests = testscenarios.load_tests_apply_scenarios @@ -528,3 +531,302 @@ class TestMultipleServers(test_utils.BaseTestCase, ServerSetupMixin): TestMultipleServers.generate_scenarios() + +class TestServerLocking(test_utils.BaseTestCase): + def setUp(self): + super(TestServerLocking, self).setUp(conf=cfg.ConfigOpts()) + + def _logmethod(name): + def method(self): + with self._lock: + self._calls.append(name) + return method + + executors = [] + class FakeExecutor(object): + def __init__(self, *args, **kwargs): + self._lock = threading.Lock() + self._calls = [] + self.listener = mock.MagicMock() + executors.append(self) + + start = _logmethod('start') + stop = _logmethod('stop') + wait = _logmethod('wait') + execute = _logmethod('execute') + self.executors = executors + + self.server = oslo_messaging.MessageHandlingServer(mock.Mock(), + mock.Mock()) + self.server._executor_cls = FakeExecutor + + def test_start_stop_wait(self): + # Test a simple execution of start, stop, wait in order + + thread = eventlet.spawn(self.server.start) + self.server.stop() + self.server.wait() + + self.assertEqual(len(self.executors), 1) + executor = self.executors[0] + self.assertEqual(executor._calls, + ['start', 'execute', 'stop', 'wait']) + self.assertTrue(executor.listener.cleanup.called) + + def test_reversed_order(self): + # Test that if we call wait, stop, start, these will be correctly + # reordered + + wait = eventlet.spawn(self.server.wait) + # This is non-deterministic, but there's not a great deal we can do + # about that + eventlet.sleep(0) + + stop = eventlet.spawn(self.server.stop) + eventlet.sleep(0) + + start = eventlet.spawn(self.server.start) + + self.server.wait() + + self.assertEqual(len(self.executors), 1) + executor = self.executors[0] + self.assertEqual(executor._calls, + ['start', 'execute', 'stop', 'wait']) + + def test_wait_for_running_task(self): + # Test that if 2 threads call a method simultaneously, both will wait, + # but only 1 will call the underlying executor method. + + start_event = threading.Event() + finish_event = threading.Event() + + running_event = threading.Event() + done_event = threading.Event() + + runner = [None] + class SteppingFakeExecutor(self.server._executor_cls): + def start(self): + # Tell the test which thread won the race + runner[0] = eventlet.getcurrent() + running_event.set() + + start_event.wait() + super(SteppingFakeExecutor, self).start() + done_event.set() + + finish_event.wait() + self.server._executor_cls = SteppingFakeExecutor + + start1 = eventlet.spawn(self.server.start) + start2 = eventlet.spawn(self.server.start) + + # Wait until one of the threads starts running + running_event.wait() + runner = runner[0] + waiter = start2 if runner == start1 else start2 + + waiter_finished = threading.Event() + waiter.link(lambda _: waiter_finished.set()) + + # At this point, runner is running start(), and waiter() is waiting for + # it to complete. runner has not yet logged anything. + self.assertEqual(1, len(self.executors)) + executor = self.executors[0] + + self.assertEqual(executor._calls, []) + self.assertFalse(waiter_finished.is_set()) + + # Let the runner log the call + start_event.set() + done_event.wait() + + # We haven't signalled completion yet, so execute shouldn't have run + self.assertEqual(executor._calls, ['start']) + self.assertFalse(waiter_finished.is_set()) + + # Let the runner complete + finish_event.set() + waiter.wait() + runner.wait() + + # Check that both threads have finished, start was only called once, + # and execute ran + self.assertTrue(waiter_finished.is_set()) + self.assertEqual(executor._calls, ['start', 'execute']) + + def test_start_stop_wait_stop_wait(self): + # Test that we behave correctly when calling stop/wait more than once. + # Subsequent calls should be noops. + + self.server.start() + self.server.stop() + self.server.wait() + self.server.stop() + self.server.wait() + + self.assertEqual(len(self.executors), 1) + executor = self.executors[0] + self.assertEqual(executor._calls, + ['start', 'execute', 'stop', 'wait']) + self.assertTrue(executor.listener.cleanup.called) + + def test_state_wrapping(self): + # Test that we behave correctly if a thread waits, and the server state + # has wrapped when it it next scheduled + + # Ensure that if 2 threads wait for the completion of 'start', the + # first will wait until complete_event is signalled, but the second + # will continue + complete_event = threading.Event() + complete_waiting_callback = threading.Event() + + start_state = self.server._states['start'] + old_wait_for_completion = start_state.wait_for_completion + waited = [False] + def new_wait_for_completion(*args, **kwargs): + if not waited[0]: + waited[0] = True + complete_waiting_callback.set() + complete_event.wait() + old_wait_for_completion(*args, **kwargs) + start_state.wait_for_completion = new_wait_for_completion + + # thread1 will wait for start to complete until we signal it + thread1 = eventlet.spawn(self.server.stop) + thread1_finished = threading.Event() + thread1.link(lambda _: thread1_finished.set()) + + self.server.start() + complete_waiting_callback.wait() + + # The server should have started, but stop should not have been called + self.assertEqual(1, len(self.executors)) + self.assertEqual(self.executors[0]._calls, ['start', 'execute']) + self.assertFalse(thread1_finished.is_set()) + + self.server.stop() + self.server.wait() + + # We should have gone through all the states, and thread1 should still + # be waiting + self.assertEqual(1, len(self.executors)) + self.assertEqual(self.executors[0]._calls, ['start', 'execute', + 'stop', 'wait']) + self.assertFalse(thread1_finished.is_set()) + + # Start again + self.server.start() + + # We should now record 2 executors + self.assertEqual(2, len(self.executors)) + self.assertEqual(self.executors[0]._calls, ['start', 'execute', + 'stop', 'wait']) + self.assertEqual(self.executors[1]._calls, ['start', 'execute']) + self.assertFalse(thread1_finished.is_set()) + + # Allow thread1 to complete + complete_event.set() + thread1_finished.wait() + + # thread1 should now have finished, and stop should not have been + # called again on either the first or second executor + self.assertEqual(2, len(self.executors)) + self.assertEqual(self.executors[0]._calls, ['start', 'execute', + 'stop', 'wait']) + self.assertEqual(self.executors[1]._calls, ['start', 'execute']) + self.assertTrue(thread1_finished.is_set()) + + @mock.patch.object(server_module, 'DEFAULT_LOG_AFTER', 1) + @mock.patch.object(server_module, 'LOG') + def test_logging(self, mock_log): + # Test that we generate a log message if we wait longer than + # DEFAULT_LOG_AFTER + + log_event = threading.Event() + mock_log.warn.side_effect = lambda _: log_event.set() + + # Call stop without calling start. We should log a wait after 1 second + thread = eventlet.spawn(self.server.stop) + log_event.wait() + + # Redundant given that we already waited, but it's nice to assert + self.assertTrue(mock_log.warn.called) + thread.kill() + + @mock.patch.object(server_module, 'LOG') + def test_logging_explicit_wait(self, mock_log): + # Test that we generate a log message if we wait longer than + # the number of seconds passed to log_after + + log_event = threading.Event() + mock_log.warn.side_effect = lambda _: log_event.set() + + # Call stop without calling start. We should log a wait after 1 second + thread = eventlet.spawn(self.server.stop, log_after=1) + log_event.wait() + + # Redundant given that we already waited, but it's nice to assert + self.assertTrue(mock_log.warn.called) + thread.kill() + + @mock.patch.object(server_module, 'LOG') + def test_logging_with_timeout(self, mock_log): + # Test that we log a message after log_after seconds if we've also + # specified an absolute timeout + + log_event = threading.Event() + mock_log.warn.side_effect = lambda _: log_event.set() + + # Call stop without calling start. We should log a wait after 1 second + thread = eventlet.spawn(self.server.stop, log_after=1, timeout=2) + log_event.wait() + + # Redundant given that we already waited, but it's nice to assert + self.assertTrue(mock_log.warn.called) + thread.kill() + + def test_timeout_wait(self): + # Test that we will eventually timeout when passing the timeout option + # if a preceding condition is not satisfied. + + self.assertRaises(server_module.TaskTimeout, + self.server.stop, timeout=1) + + def test_timeout_running(self): + # Test that we will eventually timeout if we're waiting for another + # thread to complete this task + + # Start the server, which will also instantiate an executor + self.server.start() + + stop_called = threading.Event() + + # Patch the executor's stop method to be very slow + def slow_stop(): + stop_called.set() + eventlet.sleep(10) + self.executors[0].stop = slow_stop + + # Call stop in a new thread + thread = eventlet.spawn(self.server.stop) + + # Wait until the thread is in the slow stop method + stop_called.wait() + + # Call stop again in the main thread with a timeout + self.assertRaises(server_module.TaskTimeout, + self.server.stop, timeout=1) + thread.kill() + + @mock.patch.object(server_module, 'LOG') + def test_log_after_zero(self, mock_log): + # Test that we do not log a message after DEFAULT_LOG_AFTER if the + # caller gave log_after=1 + + # Call stop without calling start. + self.assertRaises(server_module.TaskTimeout, + self.server.stop, log_after=0, timeout=2) + + # We timed out. Ensure we didn't log anything. + self.assertFalse(mock_log.warn.called) From 52be09af4da2264358f6ec4e413ea83a7a919f31 Mon Sep 17 00:00:00 2001 From: OpenStack Proposal Bot Date: Thu, 19 Nov 2015 16:00:26 +0000 Subject: [PATCH 30/74] Updated from global requirements Change-Id: Ib8cbf18578b4bfd1be8d5f529d646645bf3e8c71 --- requirements.txt | 4 ++-- test-requirements.txt | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/requirements.txt b/requirements.txt index 6eea9e1d3..2d53dd314 100644 --- a/requirements.txt +++ b/requirements.txt @@ -5,10 +5,10 @@ pbr>=1.6 futurist>=0.1.2 # Apache-2.0 -oslo.config>=2.6.0 # Apache-2.0 +oslo.config>=2.7.0 # Apache-2.0 oslo.context>=0.2.0 # Apache-2.0 oslo.log>=1.12.0 # Apache-2.0 -oslo.utils!=2.6.0,>=2.4.0 # Apache-2.0 +oslo.utils>=2.8.0 # Apache-2.0 oslo.serialization>=1.10.0 # Apache-2.0 oslo.service>=0.12.0 # Apache-2.0 oslo.i18n>=1.5.0 # Apache-2.0 diff --git a/test-requirements.txt b/test-requirements.txt index 53d199ceb..0c0f4e884 100644 --- a/test-requirements.txt +++ b/test-requirements.txt @@ -31,7 +31,7 @@ coverage>=3.6 # this is required for the docs build jobs sphinx!=1.2.0,!=1.3b1,<1.3,>=1.1.2 -oslosphinx>=2.5.0 # Apache-2.0 +oslosphinx!=3.4.0,>=2.5.0 # Apache-2.0 # AMQP 1.0 support depends on the Qpid Proton AMQP 1.0 # development libraries. From 9843641b862fa7a2ce07673af3d2e0ff39eb032f Mon Sep 17 00:00:00 2001 From: Sean Dague Date: Thu, 19 Nov 2015 14:09:38 -0500 Subject: [PATCH 31/74] make pep8 faster This builds a stripped down tox target for pep8 that doesn't need a giant venv with all the things. Works fast and lean, and makes julien fries. Change-Id: Id5b7671fb7f2b8cbf88745fd12f9238b3c0bb2dd --- tox.ini | 1 + 1 file changed, 1 insertion(+) diff --git a/tox.ini b/tox.ini index 8f110014d..7bc6cf0e5 100644 --- a/tox.ini +++ b/tox.ini @@ -10,6 +10,7 @@ commands = python setup.py testr --slowest --testr-args='{posargs}' [testenv:pep8] commands = flake8 +deps = hacking<0.11,>=0.10.0 [testenv:cover] setenv = VIRTUAL_ENV={envdir} From 15cd99050c6d2714b90059e0faad9f9e3409eaaa Mon Sep 17 00:00:00 2001 From: Matt Riedemann Date: Thu, 19 Nov 2015 10:42:34 -0800 Subject: [PATCH 32/74] Provide alias to oslo_messaging.notify._impl_messaging Ifb96c2ae9868426cac2700bf4917c27c02c90b15 moved the _impl_messaging module to oslo_messaging.notify.messaging which breaks neutron. Neutron is fixed on master for mitaka but neutron on stable/liberty is broken, and changing neutron on stable/liberty to use the new path would require a global-requirements minimum version bump for oslo.messaging to 2.6.0, which we want to avoid for people that have already shipped liberty. So provide an alias to the moved module so neutron in stable/liberty continues to work. We deprecate the module so consumers know they need to upgrade and move off this. We may need to cap oslo.messaging in global-requirements on stable/liberty at some point when we remove the deprecated alias module. Change-Id: I29453e0fbf30b0a571c2b1afc7cc81d1a11535f0 Closes-Bug: #1513630 --- oslo_messaging/notify/_impl_messaging.py | 24 +++++++++++++++ .../tests/notify/test_impl_messaging.py | 30 +++++++++++++++++++ requirements.txt | 2 ++ 3 files changed, 56 insertions(+) create mode 100644 oslo_messaging/notify/_impl_messaging.py create mode 100644 oslo_messaging/tests/notify/test_impl_messaging.py diff --git a/oslo_messaging/notify/_impl_messaging.py b/oslo_messaging/notify/_impl_messaging.py new file mode 100644 index 000000000..9f7c57113 --- /dev/null +++ b/oslo_messaging/notify/_impl_messaging.py @@ -0,0 +1,24 @@ +# Copyright 2015 IBM Corp. +# +# 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 debtcollector import removals + +from oslo_messaging.notify.messaging import * # noqa + + +# NOTE(mriedem): removal depends on how we can cap requirements in +# stable/liberty such that neutron does not try to load this +removals.removed_module(__name__, + oslo_messaging.notify.messaging.__name__, + removal_version='?') diff --git a/oslo_messaging/tests/notify/test_impl_messaging.py b/oslo_messaging/tests/notify/test_impl_messaging.py new file mode 100644 index 000000000..d2a9a2ee0 --- /dev/null +++ b/oslo_messaging/tests/notify/test_impl_messaging.py @@ -0,0 +1,30 @@ +# Copyright 2015 IBM Corp. +# +# 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 mock + +from oslo_messaging.tests import utils as test_utils + + +class TestDeprecationWarning(test_utils.BaseTestCase): + + @mock.patch('warnings.warn') + def test_impl_messaging_deprecation_warning(self, mock_warn): + # Tests that we get a deprecation warning when loading a messaging + # driver out of oslo_messaging.notify._impl_messaging. + from oslo_messaging.notify import _impl_messaging as messaging + driver = messaging.MessagingV2Driver( + conf={}, topics=['notifications'], transport='rpc') + # Make sure we got a deprecation warning by loading from the alias + self.assertEqual(1, mock_warn.call_count) diff --git a/requirements.txt b/requirements.txt index 6eea9e1d3..98f1daddb 100644 --- a/requirements.txt +++ b/requirements.txt @@ -18,6 +18,8 @@ stevedore>=1.5.0 # Apache-2.0 six>=1.9.0 cachetools>=1.0.0 # MIT License +debtcollector>=0.3.0 # Apache-2.0 + # FIXME(markmc): remove this when the drivers no longer # import eventlet From 925eb734a9d3cb46bb89a89ec1a78281d2d7afe9 Mon Sep 17 00:00:00 2001 From: Flavio Percoco Date: Tue, 3 Nov 2015 22:34:22 -0200 Subject: [PATCH 33/74] Remove qpidd's driver from the tree Back in liberty we marked this driver as deprecated. This patch removes it from the tree. The patch also removes tests, options and other references in the documentation. Note that one script is being kept because it's required by the amqp driver. Depends-On: If4b1773334e424d1f4a4e112bd1f10aca62682a9 Change-Id: I4a9cba314c4a2f24307504fa7b5427424268b114 --- doc/source/transport.rst | 2 - doc/source/zmq_driver.rst | 5 +- oslo_messaging/_drivers/amqp.py | 4 +- oslo_messaging/_drivers/impl_qpid.py | 800 ----------------- oslo_messaging/conffixture.py | 5 +- oslo_messaging/notify/logger.py | 2 +- oslo_messaging/opts.py | 3 - .../tests/drivers/test_impl_qpid.py | 850 ------------------ .../tests/functional/gate/post_test_hook.sh | 4 - oslo_messaging/tests/test_opts.py | 3 +- oslo_messaging/transport.py | 4 +- setup.cfg | 1 - test-requirements.txt | 3 - tox.ini | 4 - 14 files changed, 9 insertions(+), 1681 deletions(-) delete mode 100644 oslo_messaging/_drivers/impl_qpid.py delete mode 100644 oslo_messaging/tests/drivers/test_impl_qpid.py diff --git a/doc/source/transport.rst b/doc/source/transport.rst index 547198aa8..3449e9b7d 100644 --- a/doc/source/transport.rst +++ b/doc/source/transport.rst @@ -25,6 +25,4 @@ different 3rd party libraries that don't ensure that. In certain cases, with some drivers, it does work: * rabbit: works only if no connection have already been established. -* qpid: doesn't work (The qpid library has a global state that uses - file descriptors that can't be reset) * amqp1: works diff --git a/doc/source/zmq_driver.rst b/doc/source/zmq_driver.rst index 23c6d4d23..da4d0abc5 100644 --- a/doc/source/zmq_driver.rst +++ b/doc/source/zmq_driver.rst @@ -45,7 +45,7 @@ Juno release, as almost all the core projects in OpenStack have switched to oslo_messaging, ZeroMQ can be the only RPC driver across the OpenStack cluster. This document provides deployment information for this driver in oslo_messaging. -Other than AMQP-based drivers, like RabbitMQ or Qpid, ZeroMQ doesn't have +Other than AMQP-based drivers, like RabbitMQ, ZeroMQ doesn't have any central brokers in oslo.messaging, instead, each host (running OpenStack services) is both ZeroMQ client and server. As a result, each host needs to listen to a certain TCP port for incoming connections and directly connect @@ -172,7 +172,6 @@ The parameters for the script oslo-messaging-zmq-receiver should be:: You can specify ZeroMQ options in /etc/oslo/zeromq.conf if necessary. - Listening Address (optional) ---------------------------- @@ -204,7 +203,7 @@ DevStack Support ZeroMQ driver has been supported by DevStack. The configuration is as follows:: - ENABLED_SERVICES+=,-rabbit,-qpid,zeromq + ENABLED_SERVICES+=,-rabbit,zeromq ZEROMQ_MATCHMAKER=redis In local.conf [localrc] section need to enable zmq plugin which lives in diff --git a/oslo_messaging/_drivers/amqp.py b/oslo_messaging/_drivers/amqp.py index 55fb9d49f..2308b80d1 100644 --- a/oslo_messaging/_drivers/amqp.py +++ b/oslo_messaging/_drivers/amqp.py @@ -19,7 +19,7 @@ Shared code between AMQP based openstack.common.rpc implementations. The code in this module is shared between the rpc implementations based on -AMQP. Specifically, this includes impl_kombu and impl_qpid. impl_carrot also +AMQP. Specifically, this includes impl_kombu. impl_carrot also uses AMQP, but is deprecated and predates this code. """ @@ -66,7 +66,7 @@ amqp_opts = [ UNIQUE_ID = '_unique_id' LOG = logging.getLogger(__name__) -# NOTE(sileht): Even if rabbit/qpid have only one Connection class, +# NOTE(sileht): Even if rabbit has only one Connection class, # this connection can be used for two purposes: # * wait and receive amqp messages (only do read stuffs on the socket) # * send messages to the broker (only do write stuffs on the socket) diff --git a/oslo_messaging/_drivers/impl_qpid.py b/oslo_messaging/_drivers/impl_qpid.py deleted file mode 100644 index e0e901968..000000000 --- a/oslo_messaging/_drivers/impl_qpid.py +++ /dev/null @@ -1,800 +0,0 @@ -# Copyright 2011 OpenStack Foundation -# Copyright 2011 - 2012, Red Hat, Inc. -# -# 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 functools -import itertools -import logging -import os -import random -import time -import warnings - -from oslo_config import cfg -from oslo_serialization import jsonutils -from oslo_utils import importutils -from oslo_utils import netutils -import six - -from oslo_messaging._drivers import amqp as rpc_amqp -from oslo_messaging._drivers import amqpdriver -from oslo_messaging._drivers import base -from oslo_messaging._drivers import common as rpc_common -from oslo_messaging._i18n import _ -from oslo_messaging._i18n import _LE -from oslo_messaging._i18n import _LI -from oslo_messaging import exceptions - -qpid_codec = importutils.try_import("qpid.codec010") -qpid_messaging = importutils.try_import("qpid.messaging") -qpid_exceptions = importutils.try_import("qpid.messaging.exceptions") - -LOG = logging.getLogger(__name__) - -qpid_opts = [ - cfg.StrOpt('qpid_hostname', - default='localhost', - deprecated_group='DEFAULT', - help='Qpid broker hostname.'), - cfg.IntOpt('qpid_port', - default=5672, - deprecated_group='DEFAULT', - help='Qpid broker port.'), - cfg.ListOpt('qpid_hosts', - default=['$qpid_hostname:$qpid_port'], - deprecated_group='DEFAULT', - help='Qpid HA cluster host:port pairs.'), - cfg.StrOpt('qpid_username', - default='', - deprecated_group='DEFAULT', - help='Username for Qpid connection.'), - cfg.StrOpt('qpid_password', - default='', - deprecated_group='DEFAULT', - help='Password for Qpid connection.', - secret=True), - cfg.StrOpt('qpid_sasl_mechanisms', - default='', - deprecated_group='DEFAULT', - help='Space separated list of SASL mechanisms to use for ' - 'auth.'), - cfg.IntOpt('qpid_heartbeat', - default=60, - deprecated_group='DEFAULT', - help='Seconds between connection keepalive heartbeats.'), - cfg.StrOpt('qpid_protocol', - default='tcp', - deprecated_group='DEFAULT', - help="Transport to use, either 'tcp' or 'ssl'."), - cfg.BoolOpt('qpid_tcp_nodelay', - default=True, - deprecated_group='DEFAULT', - help='Whether to disable the Nagle algorithm.'), - cfg.IntOpt('qpid_receiver_capacity', - default=1, - deprecated_group='DEFAULT', - help='The number of prefetched messages held by receiver.'), - # NOTE(russellb) If any additional versions are added (beyond 1 and 2), - # this file could probably use some additional refactoring so that the - # differences between each version are split into different classes. - cfg.IntOpt('qpid_topology_version', - default=1, - deprecated_group='DEFAULT', - help="The qpid topology version to use. Version 1 is what " - "was originally used by impl_qpid. Version 2 includes " - "some backwards-incompatible changes that allow broker " - "federation to work. Users should update to version 2 " - "when they are able to take everything down, as it " - "requires a clean break."), -] - -JSON_CONTENT_TYPE = 'application/json; charset=utf8' - - -def raise_invalid_topology_version(conf): - msg = (_("Invalid value for qpid_topology_version: %d") % - conf.qpid_topology_version) - LOG.error(msg) - raise Exception(msg) - - -class QpidMessage(dict): - def __init__(self, session, raw_message): - super(QpidMessage, self).__init__( - rpc_common.deserialize_msg(raw_message.content)) - self._raw_message = raw_message - self._session = session - - def acknowledge(self): - self._session.acknowledge(self._raw_message) - - def requeue(self): - pass - - -class ConsumerBase(object): - """Consumer base class.""" - - def __init__(self, conf, session, callback, node_name, node_opts, - link_name, link_opts): - """Declare a queue on an amqp session. - - 'session' is the amqp session to use - 'callback' is the callback to call when messages are received - 'node_name' is the first part of the Qpid address string, before ';' - 'node_opts' will be applied to the "x-declare" section of "node" - in the address string. - 'link_name' goes into the "name" field of the "link" in the address - string - 'link_opts' will be applied to the "x-declare" section of "link" - in the address string. - """ - self.callback = callback - self.receiver = None - self.rcv_capacity = conf.qpid_receiver_capacity - self.session = None - - if conf.qpid_topology_version == 1: - addr_opts = { - "create": "always", - "node": { - "type": "topic", - "x-declare": { - "durable": True, - "auto-delete": True, - }, - }, - "link": { - "durable": True, - "x-declare": { - "durable": False, - "auto-delete": True, - "exclusive": False, - }, - }, - } - addr_opts["node"]["x-declare"].update(node_opts) - elif conf.qpid_topology_version == 2: - addr_opts = { - "link": { - "x-declare": { - "auto-delete": True, - "exclusive": False, - }, - }, - } - else: - raise_invalid_topology_version(conf) - - addr_opts["link"]["x-declare"].update(link_opts) - if link_name: - addr_opts["link"]["name"] = link_name - - self.address = "%s ; %s" % (node_name, jsonutils.dumps(addr_opts)) - - self.connect(session) - - def connect(self, session): - """Declare the receiver on connect.""" - self._declare_receiver(session) - - def reconnect(self, session): - """Re-declare the receiver after a Qpid reconnect.""" - self._declare_receiver(session) - - def _declare_receiver(self, session): - self.session = session - self.receiver = session.receiver(self.address) - self.receiver.capacity = self.rcv_capacity - - def _unpack_json_msg(self, msg): - """Load the JSON data in msg if msg.content_type indicates that it - is necessary. Put the loaded data back into msg.content and - update msg.content_type appropriately. - - A Qpid Message containing a dict will have a content_type of - 'amqp/map', whereas one containing a string that needs to be converted - back from JSON will have a content_type of JSON_CONTENT_TYPE. - - :param msg: a Qpid Message object - :returns: None - """ - if msg.content_type == JSON_CONTENT_TYPE: - msg.content = jsonutils.loads(msg.content) - msg.content_type = 'amqp/map' - - def consume(self): - """Fetch the message and pass it to the callback object.""" - message = self.receiver.fetch() - try: - self._unpack_json_msg(message) - self.callback(QpidMessage(self.session, message)) - except Exception: - LOG.exception(_LE("Failed to process message... skipping it.")) - self.session.acknowledge(message) - - def get_receiver(self): - return self.receiver - - def get_node_name(self): - return self.address.split(';')[0] - - -class DirectConsumer(ConsumerBase): - """Queue/consumer class for 'direct'.""" - - def __init__(self, conf, session, msg_id, callback): - """Init a 'direct' queue. - - 'session' is the amqp session to use - 'msg_id' is the msg_id to listen on - 'callback' is the callback to call when messages are received - """ - - link_opts = { - "exclusive": True, - "durable": conf.amqp_durable_queues, - } - - if conf.qpid_topology_version == 1: - node_name = "%s/%s" % (msg_id, msg_id) - node_opts = {"type": "direct"} - link_name = msg_id - elif conf.qpid_topology_version == 2: - node_name = "amq.direct/%s" % msg_id - node_opts = {} - link_name = msg_id - else: - raise_invalid_topology_version(conf) - - super(DirectConsumer, self).__init__(conf, session, callback, - node_name, node_opts, link_name, - link_opts) - - -class TopicConsumer(ConsumerBase): - """Consumer class for 'topic'.""" - - def __init__(self, conf, session, topic, callback, exchange_name, - name=None): - """Init a 'topic' queue. - - :param session: the amqp session to use - :param topic: is the topic to listen on - :paramtype topic: str - :param callback: the callback to call when messages are received - :param name: optional queue name, defaults to topic - """ - - link_opts = { - "auto-delete": conf.amqp_auto_delete, - "durable": conf.amqp_durable_queues, - } - - if conf.qpid_topology_version == 1: - node_name = "%s/%s" % (exchange_name, topic) - elif conf.qpid_topology_version == 2: - node_name = "amq.topic/topic/%s/%s" % (exchange_name, topic) - else: - raise_invalid_topology_version(conf) - - super(TopicConsumer, self).__init__(conf, session, callback, node_name, - {}, name or topic, link_opts) - - -class FanoutConsumer(ConsumerBase): - """Consumer class for 'fanout'.""" - - def __init__(self, conf, session, topic, callback): - """Init a 'fanout' queue. - - 'session' is the amqp session to use - 'topic' is the topic to listen on - 'callback' is the callback to call when messages are received - """ - self.conf = conf - - link_opts = {"exclusive": True} - - if conf.qpid_topology_version == 1: - node_name = "%s_fanout" % topic - node_opts = {"durable": False, "type": "fanout"} - elif conf.qpid_topology_version == 2: - node_name = "amq.topic/fanout/%s" % topic - node_opts = {} - else: - raise_invalid_topology_version(conf) - - super(FanoutConsumer, self).__init__(conf, session, callback, - node_name, node_opts, None, - link_opts) - - -class Publisher(object): - """Base Publisher class.""" - - def __init__(self, conf, session, node_name, node_opts=None): - """Init the Publisher class with the exchange_name, routing_key, - and other options - """ - self.sender = None - self.session = session - - if conf.qpid_topology_version == 1: - addr_opts = { - "create": "always", - "node": { - "type": "topic", - "x-declare": { - "durable": False, - # auto-delete isn't implemented for exchanges in qpid, - # but put in here anyway - "auto-delete": True, - }, - }, - } - if node_opts: - addr_opts["node"]["x-declare"].update(node_opts) - - self.address = "%s ; %s" % (node_name, jsonutils.dumps(addr_opts)) - elif conf.qpid_topology_version == 2: - self.address = node_name - else: - raise_invalid_topology_version(conf) - - self.reconnect(session) - - def reconnect(self, session): - """Re-establish the Sender after a reconnection.""" - self.sender = session.sender(self.address) - - def _pack_json_msg(self, msg): - """Qpid cannot serialize dicts containing strings longer than 65535 - characters. This function dumps the message content to a JSON - string, which Qpid is able to handle. - - :param msg: May be either a Qpid Message object or a bare dict. - :returns: A Qpid Message with its content field JSON encoded. - """ - try: - msg.content = jsonutils.dumps(msg.content) - except AttributeError: - # Need to have a Qpid message so we can set the content_type. - msg = qpid_messaging.Message(jsonutils.dumps(msg)) - msg.content_type = JSON_CONTENT_TYPE - return msg - - def send(self, msg): - """Send a message.""" - try: - # Check if Qpid can encode the message - check_msg = msg - if not hasattr(check_msg, 'content_type'): - check_msg = qpid_messaging.Message(msg) - content_type = check_msg.content_type - enc, dec = qpid_messaging.message.get_codec(content_type) - enc(check_msg.content) - except qpid_codec.CodecException: - # This means the message couldn't be serialized as a dict. - msg = self._pack_json_msg(msg) - self.sender.send(msg) - - -class DirectPublisher(Publisher): - """Publisher class for 'direct'.""" - def __init__(self, conf, session, topic): - """Init a 'direct' publisher.""" - - if conf.qpid_topology_version == 1: - node_name = "%s/%s" % (topic, topic) - node_opts = {"type": "direct"} - elif conf.qpid_topology_version == 2: - node_name = "amq.direct/%s" % topic - node_opts = {} - else: - raise_invalid_topology_version(conf) - - super(DirectPublisher, self).__init__(conf, session, node_name, - node_opts) - - -class TopicPublisher(Publisher): - """Publisher class for 'topic'.""" - def __init__(self, conf, session, exchange_name, topic): - """Init a 'topic' publisher. - """ - if conf.qpid_topology_version == 1: - node_name = "%s/%s" % (exchange_name, topic) - elif conf.qpid_topology_version == 2: - node_name = "amq.topic/topic/%s/%s" % (exchange_name, topic) - else: - raise_invalid_topology_version(conf) - - super(TopicPublisher, self).__init__(conf, session, node_name) - - -class FanoutPublisher(Publisher): - """Publisher class for 'fanout'.""" - def __init__(self, conf, session, topic): - """Init a 'fanout' publisher. - """ - - if conf.qpid_topology_version == 1: - node_name = "%s_fanout" % topic - node_opts = {"type": "fanout"} - elif conf.qpid_topology_version == 2: - node_name = "amq.topic/fanout/%s" % topic - node_opts = {} - else: - raise_invalid_topology_version(conf) - - super(FanoutPublisher, self).__init__(conf, session, node_name, - node_opts) - - -class NotifyPublisher(Publisher): - """Publisher class for notifications.""" - def __init__(self, conf, session, exchange_name, topic): - """Init a 'topic' publisher. - """ - node_opts = {"durable": True} - - if conf.qpid_topology_version == 1: - node_name = "%s/%s" % (exchange_name, topic) - elif conf.qpid_topology_version == 2: - node_name = "amq.topic/topic/%s/%s" % (exchange_name, topic) - else: - raise_invalid_topology_version(conf) - - super(NotifyPublisher, self).__init__(conf, session, node_name, - node_opts) - - -class Connection(object): - """Connection object.""" - - pools = {} - - def __init__(self, conf, url, purpose): - if not qpid_messaging: - raise ImportError("Failed to import qpid.messaging") - - self.connection = None - self.session = None - self.consumers = {} - self.conf = conf - self.driver_conf = conf.oslo_messaging_qpid - - self._consume_loop_stopped = False - - self.brokers_params = [] - if url.hosts: - for host in url.hosts: - params = { - 'username': host.username or '', - 'password': host.password or '', - } - if host.port is not None: - params['host'] = '%s:%d' % (host.hostname, host.port) - else: - params['host'] = host.hostname - self.brokers_params.append(params) - else: - # Old configuration format - for adr in self.driver_conf.qpid_hosts: - hostname, port = netutils.parse_host_port( - adr, default_port=5672) - - if ':' in hostname: - hostname = '[' + hostname + ']' - - params = { - 'host': '%s:%d' % (hostname, port), - 'username': self.driver_conf.qpid_username, - 'password': self.driver_conf.qpid_password, - } - self.brokers_params.append(params) - - random.shuffle(self.brokers_params) - self.brokers = itertools.cycle(self.brokers_params) - - self._initial_pid = os.getpid() - self.reconnect() - - def _connect(self, broker): - # Create the connection - this does not open the connection - self.connection = qpid_messaging.Connection(broker['host']) - - # Check if flags are set and if so set them for the connection - # before we call open - self.connection.username = broker['username'] - self.connection.password = broker['password'] - - self.connection.sasl_mechanisms = self.driver_conf.qpid_sasl_mechanisms - # Reconnection is done by self.reconnect() - self.connection.reconnect = False - self.connection.heartbeat = self.driver_conf.qpid_heartbeat - self.connection.transport = self.driver_conf.qpid_protocol - self.connection.tcp_nodelay = self.driver_conf.qpid_tcp_nodelay - self.connection.open() - - def _register_consumer(self, consumer): - self.consumers[six.text_type(consumer.get_receiver())] = consumer - - def _lookup_consumer(self, receiver): - return self.consumers[six.text_type(receiver)] - - def _disconnect(self): - # Close the session if necessary - if self.connection is not None and self.connection.opened(): - try: - self.connection.close() - except qpid_exceptions.MessagingError: - pass - self.connection = None - - def reconnect(self, retry=None): - """Handles reconnecting and re-establishing sessions and queues. - Will retry up to retry number of times. - retry = None or -1 means to retry forever - retry = 0 means no retry - retry = N means N retries - """ - delay = 1 - attempt = 0 - loop_forever = False - if retry is None or retry < 0: - loop_forever = True - - while True: - self._disconnect() - - attempt += 1 - broker = six.next(self.brokers) - try: - self._connect(broker) - except qpid_exceptions.MessagingError as e: - msg_dict = dict(e=e, - delay=delay, - retry=retry, - broker=broker) - if not loop_forever and attempt > retry: - msg = _('Unable to connect to AMQP server on ' - '%(broker)s after %(retry)d ' - 'tries: %(e)s') % msg_dict - LOG.error(msg) - raise exceptions.MessageDeliveryFailure(msg) - else: - msg = _LE("Unable to connect to AMQP server on " - "%(broker)s: %(e)s. Sleeping %(delay)s seconds") - LOG.error(msg, msg_dict) - time.sleep(delay) - delay = min(delay + 1, 5) - else: - LOG.info(_LI('Connected to AMQP server on %s'), broker['host']) - break - - self.session = self.connection.session() - - if self.consumers: - consumers = self.consumers - self.consumers = {} - - for consumer in six.itervalues(consumers): - consumer.reconnect(self.session) - self._register_consumer(consumer) - - LOG.debug("Re-established AMQP queues") - - def ensure(self, error_callback, method, retry=None): - - current_pid = os.getpid() - if self._initial_pid != current_pid: - # NOTE(sileht): - # to get the same level of fork support that rabbit driver have - # (ie: allow fork before the first connection established) - # we could use the kombu workaround: - # https://github.com/celery/kombu/blob/master/kombu/transport/ - # qpid_patches.py#L67 - LOG.warn("Process forked! " - "This can result in unpredictable behavior. " - "See: http://docs.openstack.org/developer/" - "oslo_messaging/transport.html") - self._initial_pid = current_pid - - while True: - try: - return method() - except (qpid_exceptions.Empty, - qpid_exceptions.MessagingError) as e: - if error_callback: - error_callback(e) - self.reconnect(retry=retry) - - def close(self): - """Close/release this connection.""" - try: - self.connection.close() - except Exception: - # NOTE(dripton) Logging exceptions that happen during cleanup just - # causes confusion; there's really nothing useful we can do with - # them. - pass - self.connection = None - - def reset(self): - """Reset a connection so it can be used again.""" - self.session.close() - self.session = self.connection.session() - self.consumers = {} - - def declare_consumer(self, consumer_cls, topic, callback): - """Create a Consumer using the class that was passed in and - add it to our list of consumers - """ - def _connect_error(exc): - log_info = {'topic': topic, 'err_str': exc} - LOG.error(_LE("Failed to declare consumer for topic '%(topic)s': " - "%(err_str)s"), log_info) - - def _declare_consumer(): - consumer = consumer_cls(self.driver_conf, self.session, topic, - callback) - self._register_consumer(consumer) - return consumer - - return self.ensure(_connect_error, _declare_consumer) - - def consume(self, timeout=None): - """Consume from all queues/consumers.""" - - timer = rpc_common.DecayingTimer(duration=timeout) - timer.start() - - def _raise_timeout(exc): - LOG.debug('Timed out waiting for RPC response: %s', exc) - raise rpc_common.Timeout() - - def _error_callback(exc): - timer.check_return(_raise_timeout, exc) - LOG.exception(_LE('Failed to consume message from queue: %s'), exc) - - def _consume(): - # NOTE(sileht): - # maximum value chosen according the best practice from kombu: - # http://kombu.readthedocs.org/en/latest/reference/kombu.common.html#kombu.common.eventloop - poll_timeout = 1 if timeout is None else min(timeout, 1) - - while True: - if self._consume_loop_stopped: - self._consume_loop_stopped = False - return - - try: - nxt_receiver = self.session.next_receiver( - timeout=poll_timeout) - except qpid_exceptions.Empty as exc: - poll_timeout = timer.check_return(_raise_timeout, exc, - maximum=1) - else: - break - - try: - self._lookup_consumer(nxt_receiver).consume() - except Exception: - LOG.exception(_LE("Error processing message. " - "Skipping it.")) - - self.ensure(_error_callback, _consume) - - def publisher_send(self, cls, topic, msg, retry=None, **kwargs): - """Send to a publisher based on the publisher class.""" - - def _connect_error(exc): - log_info = {'topic': topic, 'err_str': exc} - LOG.exception(_LE("Failed to publish message to topic " - "'%(topic)s': %(err_str)s"), log_info) - - def _publisher_send(): - publisher = cls(self.driver_conf, self.session, topic=topic, - **kwargs) - publisher.send(msg) - - return self.ensure(_connect_error, _publisher_send, retry=retry) - - def declare_direct_consumer(self, topic, callback): - """Create a 'direct' queue. - In nova's use, this is generally a msg_id queue used for - responses for call/multicall - """ - self.declare_consumer(DirectConsumer, topic, callback) - - def declare_topic_consumer(self, exchange_name, topic, callback=None, - queue_name=None): - """Create a 'topic' consumer.""" - self.declare_consumer(functools.partial(TopicConsumer, - name=queue_name, - exchange_name=exchange_name, - ), - topic, callback) - - def declare_fanout_consumer(self, topic, callback): - """Create a 'fanout' consumer.""" - self.declare_consumer(FanoutConsumer, topic, callback) - - def direct_send(self, msg_id, msg): - """Send a 'direct' message.""" - self.publisher_send(DirectPublisher, topic=msg_id, msg=msg) - - def topic_send(self, exchange_name, topic, msg, timeout=None, retry=None): - """Send a 'topic' message.""" - # - # We want to create a message with attributes, for example a TTL. We - # don't really need to keep 'msg' in its JSON format any longer - # so let's create an actual Qpid message here and get some - # value-add on the go. - # - # WARNING: Request timeout happens to be in the same units as - # Qpid's TTL (seconds). If this changes in the future, then this - # will need to be altered accordingly. - # - qpid_message = qpid_messaging.Message(content=msg, ttl=timeout) - self.publisher_send(TopicPublisher, topic=topic, msg=qpid_message, - exchange_name=exchange_name, retry=retry) - - def fanout_send(self, topic, msg, retry=None): - """Send a 'fanout' message.""" - self.publisher_send(FanoutPublisher, topic=topic, msg=msg, retry=retry) - - def notify_send(self, exchange_name, topic, msg, retry=None, **kwargs): - """Send a notify message on a topic.""" - self.publisher_send(NotifyPublisher, topic=topic, msg=msg, - exchange_name=exchange_name, retry=retry) - - def stop_consuming(self): - self._consume_loop_stopped = True - - -class QpidDriver(amqpdriver.AMQPDriverBase): - """qpidd Driver - - .. deprecated:: 1.16 (Liberty) - """ - - def __init__(self, conf, url, - default_exchange=None, allowed_remote_exmods=None): - - warnings.warn(_('The Qpid driver has been deprecated. ' - 'The driver is planned to be removed during the ' - '`Mitaka` development cycle.'), - DeprecationWarning, stacklevel=2) - - opt_group = cfg.OptGroup(name='oslo_messaging_qpid', - title='QPID driver options') - conf.register_group(opt_group) - conf.register_opts(qpid_opts, group=opt_group) - conf.register_opts(rpc_amqp.amqp_opts, group=opt_group) - conf.register_opts(base.base_opts, group=opt_group) - - connection_pool = rpc_amqp.ConnectionPool( - conf, conf.oslo_messaging_qpid.rpc_conn_pool_size, - url, Connection) - - super(QpidDriver, self).__init__( - conf, url, - connection_pool, - default_exchange, - allowed_remote_exmods, - conf.oslo_messaging_qpid.send_single_reply, - ) diff --git a/oslo_messaging/conffixture.py b/oslo_messaging/conffixture.py index 1312d66cf..35b6e2a27 100644 --- a/oslo_messaging/conffixture.py +++ b/oslo_messaging/conffixture.py @@ -50,9 +50,6 @@ class ConfFixture(fixtures.Fixture): _import_opts(self.conf, 'oslo_messaging._drivers.amqp', 'amqp_opts', 'oslo_messaging_rabbit') - _import_opts(self.conf, - 'oslo_messaging._drivers.impl_qpid', 'qpid_opts', - 'oslo_messaging_qpid') _import_opts(self.conf, 'oslo_messaging._drivers.amqp', 'amqp_opts', 'oslo_messaging_qpid') @@ -77,7 +74,7 @@ class ConfFixture(fixtures.Fixture): @property def transport_driver(self): - """The transport driver - for example 'rabbit', 'qpid' or 'fake'.""" + """The transport driver - for example 'rabbit', 'amqp' or 'fake'.""" return self.conf.rpc_backend @transport_driver.setter diff --git a/oslo_messaging/notify/logger.py b/oslo_messaging/notify/logger.py index 6b96b58d2..b4e48df0f 100644 --- a/oslo_messaging/notify/logger.py +++ b/oslo_messaging/notify/logger.py @@ -33,7 +33,7 @@ class LoggingNotificationHandler(logging.Handler): [handler_notifier] class=oslo_messaging.LoggingNotificationHandler level=ERROR - args=('qpid:///') + args=('rabbit:///') """ diff --git a/oslo_messaging/opts.py b/oslo_messaging/opts.py index c5856595d..263c59f68 100644 --- a/oslo_messaging/opts.py +++ b/oslo_messaging/opts.py @@ -22,7 +22,6 @@ import itertools from oslo_messaging._drivers import amqp from oslo_messaging._drivers import base as drivers_base -from oslo_messaging._drivers import impl_qpid from oslo_messaging._drivers import impl_rabbit from oslo_messaging._drivers import impl_zmq from oslo_messaging._drivers.protocols.amqp import opts as amqp_opts @@ -48,8 +47,6 @@ _opts = [ ('oslo_messaging_amqp', amqp_opts.amqp1_opts), ('oslo_messaging_rabbit', list(itertools.chain(amqp.amqp_opts, impl_rabbit.rabbit_opts))), - ('oslo_messaging_qpid', list(itertools.chain(amqp.amqp_opts, - impl_qpid.qpid_opts))) ] diff --git a/oslo_messaging/tests/drivers/test_impl_qpid.py b/oslo_messaging/tests/drivers/test_impl_qpid.py deleted file mode 100644 index 2eb0bb244..000000000 --- a/oslo_messaging/tests/drivers/test_impl_qpid.py +++ /dev/null @@ -1,850 +0,0 @@ -# Copyright (C) 2014 eNovance SAS -# -# 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 operator -import random -import threading -import time - -try: - import qpid -except ImportError: - qpid = None -from six.moves import _thread -import testscenarios -import testtools - -import oslo_messaging -from oslo_messaging._drivers import amqp -from oslo_messaging._drivers import impl_qpid as qpid_driver -from oslo_messaging.tests import utils as test_utils -from six.moves import mock - - -load_tests = testscenarios.load_tests_apply_scenarios - -QPID_BROKER = 'localhost:5672' - - -class TestQpidDriverLoad(test_utils.BaseTestCase): - - def setUp(self): - super(TestQpidDriverLoad, self).setUp() - self.messaging_conf.transport_driver = 'qpid' - - def test_driver_load(self): - transport = oslo_messaging.get_transport(self.conf) - self.assertIsInstance(transport._driver, qpid_driver.QpidDriver) - - -def _is_qpidd_service_running(): - - """this function checks if the qpid service is running or not.""" - - qpid_running = True - try: - broker = QPID_BROKER - connection = qpid.messaging.Connection(broker) - connection.open() - except Exception: - # qpid service is not running. - qpid_running = False - else: - connection.close() - - return qpid_running - - -class _QpidBaseTestCase(test_utils.BaseTestCase): - - @testtools.skipIf(qpid is None, "qpid not available") - def setUp(self): - super(_QpidBaseTestCase, self).setUp() - self.messaging_conf.transport_driver = 'qpid' - self.fake_qpid = not _is_qpidd_service_running() - - if self.fake_qpid: - self.session_receive = get_fake_qpid_session() - self.session_send = get_fake_qpid_session() - else: - self.broker = QPID_BROKER - # create connection from the qpid.messaging - # connection for the Consumer. - self.con_receive = qpid.messaging.Connection(self.broker) - self.con_receive.open() - # session to receive the messages - self.session_receive = self.con_receive.session() - - # connection for sending the message - self.con_send = qpid.messaging.Connection(self.broker) - self.con_send.open() - # session to send the messages - self.session_send = self.con_send.session() - - # list to store the expected messages and - # the actual received messages - self._expected = [] - self._messages = [] - self.initialized = True - - def tearDown(self): - super(_QpidBaseTestCase, self).tearDown() - - if self.initialized: - if self.fake_qpid: - _fake_session.flush_exchanges() - else: - self.con_receive.close() - self.con_send.close() - - -class TestQpidTransportURL(_QpidBaseTestCase): - - scenarios = [ - ('none', dict(url=None, - expected=[dict(host='localhost:5672', - username='', - password='')])), - ('empty', - dict(url='qpid:///', - expected=[dict(host='localhost:5672', - username='', - password='')])), - ('localhost', - dict(url='qpid://localhost/', - expected=[dict(host='localhost', - username='', - password='')])), - ('no_creds', - dict(url='qpid://host/', - expected=[dict(host='host', - username='', - password='')])), - ('no_port', - dict(url='qpid://user:password@host/', - expected=[dict(host='host', - username='user', - password='password')])), - ('full_url', - dict(url='qpid://user:password@host:10/', - expected=[dict(host='host:10', - username='user', - password='password')])), - ('full_two_url', - dict(url='qpid://user:password@host:10,' - 'user2:password2@host2:12/', - expected=[dict(host='host:10', - username='user', - password='password'), - dict(host='host2:12', - username='user2', - password='password2') - ] - )), - - ] - - @mock.patch.object(qpid_driver.Connection, 'reconnect') - def test_transport_url(self, *args): - transport = oslo_messaging.get_transport(self.conf, self.url) - self.addCleanup(transport.cleanup) - driver = transport._driver - - brokers_params = driver._get_connection().brokers_params - self.assertEqual(sorted(self.expected, - key=operator.itemgetter('host')), - sorted(brokers_params, - key=operator.itemgetter('host'))) - - -class TestQpidInvalidTopologyVersion(_QpidBaseTestCase): - """Unit test cases to test invalid qpid topology version.""" - - scenarios = [ - ('direct', dict(consumer_cls=qpid_driver.DirectConsumer, - consumer_kwargs={}, - publisher_cls=qpid_driver.DirectPublisher, - publisher_kwargs={})), - ('topic', dict(consumer_cls=qpid_driver.TopicConsumer, - consumer_kwargs={'exchange_name': 'openstack'}, - publisher_cls=qpid_driver.TopicPublisher, - publisher_kwargs={'exchange_name': 'openstack'})), - ('fanout', dict(consumer_cls=qpid_driver.FanoutConsumer, - consumer_kwargs={}, - publisher_cls=qpid_driver.FanoutPublisher, - publisher_kwargs={})), - ] - - def setUp(self): - super(TestQpidInvalidTopologyVersion, self).setUp() - self.config(qpid_topology_version=-1, - group='oslo_messaging_qpid') - - def test_invalid_topology_version(self): - def consumer_callback(msg): - pass - - msgid_or_topic = 'test' - - # not using self.assertRaises because - # 1. qpid driver raises Exception(msg) for invalid topology version - # 2. flake8 - H202 assertRaises Exception too broad - exception_msg = ("Invalid value for qpid_topology_version: %d" % - self.conf.oslo_messaging_qpid.qpid_topology_version) - recvd_exc_msg = '' - - try: - self.consumer_cls(self.conf.oslo_messaging_qpid, - self.session_receive, - msgid_or_topic, - consumer_callback, - **self.consumer_kwargs) - except Exception as e: - recvd_exc_msg = e.message - - self.assertEqual(exception_msg, recvd_exc_msg) - - recvd_exc_msg = '' - try: - self.publisher_cls(self.conf.oslo_messaging_qpid, - self.session_send, - topic=msgid_or_topic, - **self.publisher_kwargs) - except Exception as e: - recvd_exc_msg = e.message - - self.assertEqual(exception_msg, recvd_exc_msg) - - -class TestQpidDirectConsumerPublisher(_QpidBaseTestCase): - """Unit test cases to test DirectConsumer and Direct Publisher.""" - - _n_qpid_topology = [ - ('v1', dict(qpid_topology=1)), - ('v2', dict(qpid_topology=2)), - ] - - _n_msgs = [ - ('single', dict(no_msgs=1)), - ('multiple', dict(no_msgs=10)), - ] - - @classmethod - def generate_scenarios(cls): - cls.scenarios = testscenarios.multiply_scenarios(cls._n_qpid_topology, - cls._n_msgs) - - def consumer_callback(self, msg): - # This function will be called by the DirectConsumer - # when any message is received. - # Append the received message into the messages list - # so that the received messages can be validated - # with the expected messages - if isinstance(msg, dict): - self._messages.append(msg['content']) - else: - self._messages.append(msg) - - def test_qpid_direct_consumer_producer(self): - self.msgid = str(random.randint(1, 100)) - - # create a DirectConsumer and DirectPublisher class objects - self.dir_cons = qpid_driver.DirectConsumer( - self.conf.oslo_messaging_qpid, - self.session_receive, - self.msgid, - self.consumer_callback) - self.dir_pub = qpid_driver.DirectPublisher( - self.conf.oslo_messaging_qpid, - self.session_send, - self.msgid) - - def try_send_msg(no_msgs): - for i in range(no_msgs): - self._expected.append(str(i)) - snd_msg = {'content_type': 'text/plain', 'content': str(i)} - self.dir_pub.send(snd_msg) - - def try_receive_msg(no_msgs): - for i in range(no_msgs): - self.dir_cons.consume() - - thread1 = threading.Thread(target=try_receive_msg, - args=(self.no_msgs,)) - thread2 = threading.Thread(target=try_send_msg, - args=(self.no_msgs,)) - - thread1.start() - thread2.start() - thread1.join() - thread2.join() - - self.assertEqual(self.no_msgs, len(self._messages)) - self.assertEqual(self._expected, self._messages) - - -TestQpidDirectConsumerPublisher.generate_scenarios() - - -class TestQpidTopicAndFanout(_QpidBaseTestCase): - """Unit Test cases to test TopicConsumer and - TopicPublisher classes of the qpid driver - and FanoutConsumer and FanoutPublisher classes - of the qpid driver - """ - - _n_qpid_topology = [ - ('v1', dict(qpid_topology=1)), - ('v2', dict(qpid_topology=2)), - ] - - _n_msgs = [ - ('single', dict(no_msgs=1)), - ('multiple', dict(no_msgs=10)), - ] - - _n_senders = [ - ('single', dict(no_senders=1)), - ('multiple', dict(no_senders=10)), - ] - - _n_receivers = [ - ('single', dict(no_receivers=1)), - ] - _exchange_class = [ - ('topic', dict(consumer_cls=qpid_driver.TopicConsumer, - consumer_kwargs={'exchange_name': 'openstack'}, - publisher_cls=qpid_driver.TopicPublisher, - publisher_kwargs={'exchange_name': 'openstack'}, - topic='topictest.test', - receive_topic='topictest.test')), - ('fanout', dict(consumer_cls=qpid_driver.FanoutConsumer, - consumer_kwargs={}, - publisher_cls=qpid_driver.FanoutPublisher, - publisher_kwargs={}, - topic='fanouttest', - receive_topic='fanouttest')), - ] - - @classmethod - def generate_scenarios(cls): - cls.scenarios = testscenarios.multiply_scenarios(cls._n_qpid_topology, - cls._n_msgs, - cls._n_senders, - cls._n_receivers, - cls._exchange_class) - - def setUp(self): - super(TestQpidTopicAndFanout, self).setUp() - - # to store the expected messages and the - # actual received messages - # - # NOTE(dhellmann): These are dicts, where the base class uses - # lists. - self._expected = {} - self._messages = {} - - self._senders = [] - self._receivers = [] - - self._sender_threads = [] - self._receiver_threads = [] - - def consumer_callback(self, msg): - """callback function called by the ConsumerBase class of - qpid driver. - Message will be received in the format x-y - where x is the sender id and y is the msg number of the sender - extract the sender id 'x' and store the msg 'x-y' with 'x' as - the key - """ - - if isinstance(msg, dict): - msgcontent = msg['content'] - else: - msgcontent = msg - - splitmsg = msgcontent.split('-') - key = _thread.get_ident() - - if key not in self._messages: - self._messages[key] = dict() - - tdict = self._messages[key] - - if splitmsg[0] not in tdict: - tdict[splitmsg[0]] = [] - - tdict[splitmsg[0]].append(msgcontent) - - def _try_send_msg(self, sender_id, no_msgs): - for i in range(no_msgs): - sendmsg = '%s-%s' % (str(sender_id), str(i)) - key = str(sender_id) - # Store the message in the self._expected for each sender. - # This will be used later to - # validate the test by comparing it with the - # received messages by all the receivers - if key not in self._expected: - self._expected[key] = [] - self._expected[key].append(sendmsg) - send_dict = {'content_type': 'text/plain', 'content': sendmsg} - self._senders[sender_id].send(send_dict) - - def _try_receive_msg(self, receiver_id, no_msgs): - for i in range(self.no_senders * no_msgs): - no_of_attempts = 0 - - # ConsumerBase.consume blocks indefinitely until a message - # is received. - # So qpid_receiver.available() is called before calling - # ConsumerBase.consume() so that we are not - # blocked indefinitely - qpid_receiver = self._receivers[receiver_id].get_receiver() - while no_of_attempts < 50: - if qpid_receiver.available() > 0: - self._receivers[receiver_id].consume() - break - no_of_attempts += 1 - time.sleep(0.05) - - def test_qpid_topic_and_fanout(self): - for receiver_id in range(self.no_receivers): - consumer = self.consumer_cls(self.conf.oslo_messaging_qpid, - self.session_receive, - self.receive_topic, - self.consumer_callback, - **self.consumer_kwargs) - self._receivers.append(consumer) - - # create receivers threads - thread = threading.Thread(target=self._try_receive_msg, - args=(receiver_id, self.no_msgs,)) - self._receiver_threads.append(thread) - - for sender_id in range(self.no_senders): - publisher = self.publisher_cls(self.conf.oslo_messaging_qpid, - self.session_send, - topic=self.topic, - **self.publisher_kwargs) - self._senders.append(publisher) - - # create sender threads - thread = threading.Thread(target=self._try_send_msg, - args=(sender_id, self.no_msgs,)) - self._sender_threads.append(thread) - - for thread in self._receiver_threads: - thread.start() - - for thread in self._sender_threads: - thread.start() - - for thread in self._receiver_threads: - thread.join() - - for thread in self._sender_threads: - thread.join() - - # Each receiver should receive all the messages sent by - # the sender(s). - # So, Iterate through each of the receiver items in - # self._messages and compare with the expected messages - # messages. - - self.assertEqual(self.no_senders, len(self._expected)) - self.assertEqual(self.no_receivers, len(self._messages)) - - for key, messages in self._messages.iteritems(): - self.assertEqual(self._expected, messages) - -TestQpidTopicAndFanout.generate_scenarios() - - -class AddressNodeMatcher(object): - def __init__(self, node): - self.node = node - - def __eq__(self, address): - return address.split(';')[0].strip() == self.node - - -class TestDriverInterface(_QpidBaseTestCase): - """Unit Test cases to test the amqpdriver with qpid - """ - - def setUp(self): - super(TestDriverInterface, self).setUp() - self.config(qpid_topology_version=2, - group='oslo_messaging_qpid') - transport = oslo_messaging.get_transport(self.conf) - self.driver = transport._driver - - original_get_connection = self.driver._get_connection - p = mock.patch.object(self.driver, '_get_connection', - side_effect=lambda pooled=True: - original_get_connection(False)) - p.start() - self.addCleanup(p.stop) - - def test_listen_and_direct_send(self): - target = oslo_messaging.Target(exchange="exchange_test", - topic="topic_test", - server="server_test") - - with mock.patch('qpid.messaging.Connection') as conn_cls: - conn = conn_cls.return_value - session = conn.session.return_value - session.receiver.side_effect = [mock.Mock(), mock.Mock(), - mock.Mock()] - - listener = self.driver.listen(target) - listener.conn.direct_send("msg_id", {}) - - self.assertEqual(3, len(listener.conn.consumers)) - - expected_calls = [ - mock.call(AddressNodeMatcher( - 'amq.topic/topic/exchange_test/topic_test')), - mock.call(AddressNodeMatcher( - 'amq.topic/topic/exchange_test/topic_test.server_test')), - mock.call(AddressNodeMatcher('amq.topic/fanout/topic_test')), - ] - session.receiver.assert_has_calls(expected_calls) - session.sender.assert_called_with( - AddressNodeMatcher("amq.direct/msg_id")) - - def test_send(self): - target = oslo_messaging.Target(exchange="exchange_test", - topic="topic_test", - server="server_test") - with mock.patch('qpid.messaging.Connection') as conn_cls: - conn = conn_cls.return_value - session = conn.session.return_value - - self.driver.send(target, {}, {}) - session.sender.assert_called_with(AddressNodeMatcher( - "amq.topic/topic/exchange_test/topic_test.server_test")) - - def test_send_notification(self): - target = oslo_messaging.Target(exchange="exchange_test", - topic="topic_test.info") - with mock.patch('qpid.messaging.Connection') as conn_cls: - conn = conn_cls.return_value - session = conn.session.return_value - - self.driver.send_notification(target, {}, {}, "2.0") - session.sender.assert_called_with(AddressNodeMatcher( - "amq.topic/topic/exchange_test/topic_test.info")) - - -class TestQpidReconnectOrder(test_utils.BaseTestCase): - """Unit Test cases to test reconnection - """ - - @testtools.skipIf(qpid is None, "qpid not available") - def test_reconnect_order(self): - brokers = ['host1', 'host2', 'host3', 'host4', 'host5'] - brokers_count = len(brokers) - - self.config(qpid_hosts=brokers, - group='oslo_messaging_qpid') - - with mock.patch('qpid.messaging.Connection') as conn_mock: - # starting from the first broker in the list - url = oslo_messaging.TransportURL.parse(self.conf, None) - connection = qpid_driver.Connection(self.conf, url, - amqp.PURPOSE_SEND) - - # reconnect will advance to the next broker, one broker per - # attempt, and then wrap to the start of the list once the end is - # reached - for _ in range(brokers_count): - connection.reconnect() - - expected = [] - for broker in brokers: - expected.extend([mock.call("%s:5672" % broker), - mock.call().open(), - mock.call().session(), - mock.call().opened(), - mock.call().opened().__nonzero__(), - mock.call().close()]) - - conn_mock.assert_has_calls(expected, any_order=True) - - -def synchronized(func): - func.__lock__ = threading.Lock() - - def synced_func(*args, **kws): - with func.__lock__: - return func(*args, **kws) - - return synced_func - - -class FakeQpidMsgManager(object): - def __init__(self): - self._exchanges = {} - - @synchronized - def add_exchange(self, exchange): - if exchange not in self._exchanges: - self._exchanges[exchange] = {'msgs': [], 'consumers': {}} - - @synchronized - def add_exchange_consumer(self, exchange, consumer_id): - exchange_info = self._exchanges[exchange] - cons_dict = exchange_info['consumers'] - cons_dict[consumer_id] = 0 - - @synchronized - def add_exchange_msg(self, exchange, msg): - exchange_info = self._exchanges[exchange] - exchange_info['msgs'].append(msg) - - def get_exchange_msg(self, exchange, index): - exchange_info = self._exchanges[exchange] - return exchange_info['msgs'][index] - - def get_no_exch_msgs(self, exchange): - exchange_info = self._exchanges[exchange] - return len(exchange_info['msgs']) - - def get_exch_cons_index(self, exchange, consumer_id): - exchange_info = self._exchanges[exchange] - cons_dict = exchange_info['consumers'] - return cons_dict[consumer_id] - - @synchronized - def inc_consumer_index(self, exchange, consumer_id): - exchange_info = self._exchanges[exchange] - cons_dict = exchange_info['consumers'] - cons_dict[consumer_id] += 1 - -_fake_qpid_msg_manager = FakeQpidMsgManager() - - -class FakeQpidSessionSender(object): - def __init__(self, session, id, target, options): - self.session = session - self.id = id - self.target = target - self.options = options - - @synchronized - def send(self, object, sync=True, timeout=None): - _fake_qpid_msg_manager.add_exchange_msg(self.target, object) - - def close(self, timeout=None): - pass - - -class FakeQpidSessionReceiver(object): - - def __init__(self, session, id, source, options): - self.session = session - self.id = id - self.source = source - self.options = options - - @synchronized - def fetch(self, timeout=None): - if timeout is None: - # if timeout is not given, take a default time out - # of 30 seconds to avoid indefinite loop - _timeout = 30 - else: - _timeout = timeout - - deadline = time.time() + _timeout - while time.time() <= deadline: - index = _fake_qpid_msg_manager.get_exch_cons_index(self.source, - self.id) - try: - msg = _fake_qpid_msg_manager.get_exchange_msg(self.source, - index) - except IndexError: - pass - else: - _fake_qpid_msg_manager.inc_consumer_index(self.source, - self.id) - return qpid.messaging.Message(msg) - time.sleep(0.050) - - if timeout is None: - raise Exception('timed out waiting for reply') - - def close(self, timeout=None): - pass - - @synchronized - def available(self): - no_msgs = _fake_qpid_msg_manager.get_no_exch_msgs(self.source) - index = _fake_qpid_msg_manager.get_exch_cons_index(self.source, - self.id) - if no_msgs == 0 or index >= no_msgs: - return 0 - else: - return no_msgs - index - - -class FakeQpidSession(object): - - def __init__(self, connection=None, name=None, transactional=None): - self.connection = connection - self.name = name - self.transactional = transactional - self._receivers = {} - self.conf = None - self.url = None - self._senders = {} - self._sender_id = 0 - self._receiver_id = 0 - - @synchronized - def sender(self, target, **options): - exchange_key = self._extract_exchange_key(target) - _fake_qpid_msg_manager.add_exchange(exchange_key) - - sendobj = FakeQpidSessionSender(self, self._sender_id, - exchange_key, options) - self._senders[self._sender_id] = sendobj - self._sender_id = self._sender_id + 1 - return sendobj - - @synchronized - def receiver(self, source, **options): - exchange_key = self._extract_exchange_key(source) - _fake_qpid_msg_manager.add_exchange(exchange_key) - recvobj = FakeQpidSessionReceiver(self, self._receiver_id, - exchange_key, options) - self._receivers[self._receiver_id] = recvobj - _fake_qpid_msg_manager.add_exchange_consumer(exchange_key, - self._receiver_id) - self._receiver_id += 1 - return recvobj - - def acknowledge(self, message=None, disposition=None, sync=True): - pass - - @synchronized - def flush_exchanges(self): - _fake_qpid_msg_manager._exchanges = {} - - def _extract_exchange_key(self, exchange_msg): - """This function extracts a unique key for the exchange. - This key is used in the dictionary as a 'key' for - this exchange. - Eg. if the exchange_msg (for qpid topology version 1) - is 33/33 ; {"node": {"x-declare": {"auto-delete": true, .... - then 33 is returned as the key. - Eg 2. For topology v2, if the - exchange_msg is - amq.direct/44 ; {"link": {"x-dec....... - then 44 is returned - """ - # first check for ';' - semicolon_split = exchange_msg.split(';') - - # split the first item of semicolon_split with '/' - slash_split = semicolon_split[0].split('/') - # return the last element of the list as the key - key = slash_split[-1] - return key.strip() - - def close(self): - pass - -_fake_session = FakeQpidSession() - - -def get_fake_qpid_session(): - return _fake_session - - -class QPidHATestCase(test_utils.BaseTestCase): - - @testtools.skipIf(qpid is None, "qpid not available") - def setUp(self): - super(QPidHATestCase, self).setUp() - self.brokers = ['host1', 'host2', 'host3', 'host4', 'host5'] - - self.config(qpid_hosts=self.brokers, - qpid_username=None, - qpid_password=None, - group='oslo_messaging_qpid') - - hostname_sets = set() - self.info = {'attempt': 0, - 'fail': False} - - def _connect(myself, broker): - # do as little work that is enough to pass connection attempt - myself.connection = mock.Mock() - hostname = broker['host'] - self.assertNotIn(hostname, hostname_sets) - hostname_sets.add(hostname) - - self.info['attempt'] += 1 - if self.info['fail']: - raise qpid.messaging.exceptions.ConnectionError - - # just make sure connection instantiation does not fail with an - # exception - self.stubs.Set(qpid_driver.Connection, '_connect', _connect) - - # starting from the first broker in the list - url = oslo_messaging.TransportURL.parse(self.conf, None) - self.connection = qpid_driver.Connection(self.conf, url, - amqp.PURPOSE_SEND) - self.addCleanup(self.connection.close) - - self.info.update({'attempt': 0, - 'fail': True}) - hostname_sets.clear() - - def test_reconnect_order(self): - self.assertRaises(oslo_messaging.MessageDeliveryFailure, - self.connection.reconnect, - retry=len(self.brokers) - 1) - self.assertEqual(len(self.brokers), self.info['attempt']) - - def test_ensure_four_retries(self): - mock_callback = mock.Mock( - side_effect=qpid.messaging.exceptions.ConnectionError) - self.assertRaises(oslo_messaging.MessageDeliveryFailure, - self.connection.ensure, None, mock_callback, - retry=4) - self.assertEqual(5, self.info['attempt']) - self.assertEqual(1, mock_callback.call_count) - - def test_ensure_one_retry(self): - mock_callback = mock.Mock( - side_effect=qpid.messaging.exceptions.ConnectionError) - self.assertRaises(oslo_messaging.MessageDeliveryFailure, - self.connection.ensure, None, mock_callback, - retry=1) - self.assertEqual(2, self.info['attempt']) - self.assertEqual(1, mock_callback.call_count) - - def test_ensure_no_retry(self): - mock_callback = mock.Mock( - side_effect=qpid.messaging.exceptions.ConnectionError) - self.assertRaises(oslo_messaging.MessageDeliveryFailure, - self.connection.ensure, None, mock_callback, - retry=0) - self.assertEqual(1, self.info['attempt']) - self.assertEqual(1, mock_callback.call_count) diff --git a/oslo_messaging/tests/functional/gate/post_test_hook.sh b/oslo_messaging/tests/functional/gate/post_test_hook.sh index 276129cdd..23ee6ab48 100755 --- a/oslo_messaging/tests/functional/gate/post_test_hook.sh +++ b/oslo_messaging/tests/functional/gate/post_test_hook.sh @@ -46,10 +46,6 @@ case $RPC_BACKEND in sudo apt-get update -y sudo apt-get install -y redis-server python-redis ;; - qpid) - sudo apt-get update -y - sudo apt-get install -y qpidd sasl2-bin - ;; amqp1) sudo yum install -y qpid-cpp-server qpid-proton-c-devel python-qpid-proton cyrus-sasl-lib cyrus-sasl-plain ;; diff --git a/oslo_messaging/tests/test_opts.py b/oslo_messaging/tests/test_opts.py index e16145b94..931ded80f 100644 --- a/oslo_messaging/tests/test_opts.py +++ b/oslo_messaging/tests/test_opts.py @@ -32,14 +32,13 @@ class OptsTestCase(test_utils.BaseTestCase): super(OptsTestCase, self).setUp() def _test_list_opts(self, result): - self.assertEqual(5, len(result)) + self.assertEqual(4, len(result)) groups = [g for (g, l) in result] self.assertIn(None, groups) self.assertIn('matchmaker_redis', groups) self.assertIn('oslo_messaging_amqp', groups) self.assertIn('oslo_messaging_rabbit', groups) - self.assertIn('oslo_messaging_qpid', groups) opt_names = [o.name for (g, l) in result for o in l] self.assertIn('rpc_backend', opt_names) diff --git a/oslo_messaging/transport.py b/oslo_messaging/transport.py index 6fb3c8e42..144d1a7f3 100644 --- a/oslo_messaging/transport.py +++ b/oslo_messaging/transport.py @@ -43,7 +43,7 @@ _transport_opts = [ cfg.StrOpt('rpc_backend', default='rabbit', help='The messaging driver to use, defaults to rabbit. Other ' - 'drivers include qpid and zmq.'), + 'drivers include amqp and zmq.'), cfg.StrOpt('control_exchange', default='openstack', help='The default exchange under which topics are scoped. May ' @@ -232,7 +232,7 @@ class TransportURL(object): :param conf: a ConfigOpts instance :type conf: oslo.config.cfg.ConfigOpts - :param transport: a transport name for example 'rabbit' or 'qpid' + :param transport: a transport name for example 'rabbit' :type transport: str :param virtual_host: a virtual host path for example '/' :type virtual_host: str diff --git a/setup.cfg b/setup.cfg index ee63dc5bb..cbed37743 100644 --- a/setup.cfg +++ b/setup.cfg @@ -26,7 +26,6 @@ console_scripts = oslo.messaging.drivers = rabbit = oslo_messaging._drivers.impl_rabbit:RabbitDriver - qpid = oslo_messaging._drivers.impl_qpid:QpidDriver zmq = oslo_messaging._drivers.impl_zmq:ZmqDriver amqp = oslo_messaging._drivers.protocols.amqp.driver:ProtonDriver diff --git a/test-requirements.txt b/test-requirements.txt index 0c0f4e884..89cda423c 100644 --- a/test-requirements.txt +++ b/test-requirements.txt @@ -15,9 +15,6 @@ testscenarios>=0.4 testtools>=1.4.0 oslotest>=1.10.0 # Apache-2.0 -# for test_qpid -qpid-python;python_version=='2.7' - # for test_matchmaker_redis redis>=2.10.0 diff --git a/tox.ini b/tox.ini index 8f110014d..648962712 100644 --- a/tox.ini +++ b/tox.ini @@ -22,10 +22,6 @@ commands = {posargs} [testenv:docs] commands = python setup.py build_sphinx -[testenv:py27-func-qpid] -setenv = TRANSPORT_URL=qpid://stackqpid:secretqpid@127.0.0.1:65123// -commands = {toxinidir}/setup-test-env-qpid.sh 0-10 python setup.py testr --slowest --testr-args='oslo_messaging.tests.functional' - [testenv:py27-func-rabbit] commands = {toxinidir}/setup-test-env-rabbit.sh python setup.py testr --slowest --testr-args='oslo_messaging.tests.functional' From 47a906aff3cee8bc2f3e8102bf9f30802cf7974d Mon Sep 17 00:00:00 2001 From: Julien Danjou Date: Thu, 12 Nov 2015 17:24:55 +0100 Subject: [PATCH 34/74] middleware: remove oslo.context usage There's nothing interesting in the fake context we're passing. Let's stop depending on oslo.context all together here. Change-Id: I7784bc3b818e67118e03905857c39eac66765fad --- oslo_messaging/notify/middleware.py | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/oslo_messaging/notify/middleware.py b/oslo_messaging/notify/middleware.py index b5d3092b5..60aab278b 100644 --- a/oslo_messaging/notify/middleware.py +++ b/oslo_messaging/notify/middleware.py @@ -22,7 +22,6 @@ import sys import traceback as tb from oslo_config import cfg -from oslo_context import context from oslo_middleware import base import six import webob.dec @@ -85,7 +84,7 @@ class RequestNotifier(base.Middleware): 'request': self.environ_to_dict(request.environ), } - self.notifier.info(context.get_admin_context(), + self.notifier.info({}, 'http.request', payload) @@ -108,7 +107,7 @@ class RequestNotifier(base.Middleware): 'traceback': tb.format_tb(traceback) } - self.notifier.info(context.get_admin_context(), + self.notifier.info({}, 'http.response', payload) From 196980dace199c412dfaec34568c2a2d66b95a45 Mon Sep 17 00:00:00 2001 From: Julien Danjou Date: Thu, 12 Nov 2015 12:02:56 +0100 Subject: [PATCH 35/74] serializer: deprecate RequestContextSerializer We plan to remove it to avoid dependency on oslo.context. Change-Id: I21d76abf1c30b9e89cb8a6c20d1c0cc79cd83a3f --- oslo_messaging/serializer.py | 2 ++ requirements.txt | 1 + 2 files changed, 3 insertions(+) diff --git a/oslo_messaging/serializer.py b/oslo_messaging/serializer.py index b1761fd83..8b7c0a7a3 100644 --- a/oslo_messaging/serializer.py +++ b/oslo_messaging/serializer.py @@ -19,6 +19,7 @@ __all__ = ['Serializer', 'NoOpSerializer', 'JsonPayloadSerializer', import abc +from debtcollector import removals from oslo_context import context as common_context from oslo_serialization import jsonutils import six @@ -63,6 +64,7 @@ class Serializer(object): """ +@removals.remove(version="2.9", removal_version="3.0") class RequestContextSerializer(Serializer): def __init__(self, base): diff --git a/requirements.txt b/requirements.txt index 819fc247a..8175ec409 100644 --- a/requirements.txt +++ b/requirements.txt @@ -13,6 +13,7 @@ oslo.serialization>=1.10.0 # Apache-2.0 oslo.service>=0.12.0 # Apache-2.0 oslo.i18n>=1.5.0 # Apache-2.0 stevedore>=1.5.0 # Apache-2.0 +debtcollector>=0.3.0 # Apache-2.0 # for jsonutils six>=1.9.0 From 16f956d6f336bee259f194db2480821777a8058b Mon Sep 17 00:00:00 2001 From: ZhiQiang Fan Date: Mon, 23 Nov 2015 18:32:35 -0700 Subject: [PATCH 36/74] ignore .eggs directory This directory contains eggs that were downloaded by setuptools to build, test, and run plug-ins. This directory caches those eggs to prevent repeated downloads. We need to ignore it. Change-Id: Idd164c7c8952c70487253e5691ba2da33345059a --- .gitignore | 1 + 1 file changed, 1 insertion(+) diff --git a/.gitignore b/.gitignore index 9cc41b7a7..215d3f3cc 100644 --- a/.gitignore +++ b/.gitignore @@ -7,6 +7,7 @@ ChangeLog .tox .coverage *.egg-info/ +.eggs *.egg build/ doc/build/ From 6dba2ed591c357a722076db498a6c29657f8760d Mon Sep 17 00:00:00 2001 From: Davanum Srinivas Date: Sat, 21 Nov 2015 09:50:35 -0500 Subject: [PATCH 37/74] Add Warning when we cannot notify Calling Notifier.audit() won't work with LogDriver as the logger does not have a audit level anymore, So let's at least not fail silently and let the operators know that we are dropping stuff on the floor. Closes-Bug: #1518170 Change-Id: I74002c72e6763ea8b5df7d97d722619bd4a1950b --- oslo_messaging/notify/_impl_log.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/oslo_messaging/notify/_impl_log.py b/oslo_messaging/notify/_impl_log.py index 40833e96b..fa6e1f3cf 100644 --- a/oslo_messaging/notify/_impl_log.py +++ b/oslo_messaging/notify/_impl_log.py @@ -16,6 +16,7 @@ # under the License. import logging +import warnings from oslo_serialization import jsonutils from oslo_utils import strutils @@ -40,3 +41,7 @@ class LogDriver(notifier.Driver): method = getattr(logger, priority.lower(), None) if method: method(strutils.mask_password(jsonutils.dumps(message))) + else: + warnings.warn('Unable to log message as notify cannot find a ' + 'logger with the priority specified ' + '%s' % priority.lower()) From 4ea583b718e87a875567ae33690bc1f87df3eed9 Mon Sep 17 00:00:00 2001 From: OpenStack Proposal Bot Date: Tue, 24 Nov 2015 14:45:11 +0000 Subject: [PATCH 38/74] Updated from global requirements Change-Id: I6bc59397619d4cf06b8e18b8f48e27b2e0c8c862 --- requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/requirements.txt b/requirements.txt index 819fc247a..a9dda43f3 100644 --- a/requirements.txt +++ b/requirements.txt @@ -18,7 +18,7 @@ stevedore>=1.5.0 # Apache-2.0 six>=1.9.0 cachetools>=1.0.0 # MIT License -debtcollector>=0.3.0 # Apache-2.0 +debtcollector>=0.3.0 # Apache-2.0 # FIXME(markmc): remove this when the drivers no longer # import eventlet From a33c761e6261347864f0284f59022c1d340611d1 Mon Sep 17 00:00:00 2001 From: Kui Shi Date: Wed, 25 Nov 2015 09:02:45 +0000 Subject: [PATCH 39/74] Add log info for AMQP client If librabbitmq is installed and can be imported correctly, it will be used as AMQP client to connect RabbitMQ. Mark it in the log file. Change-Id: I33807df67dd2fffa13c675324c8cc7ae716a210e Signed-off-by: Kui Shi --- oslo_messaging/_drivers/impl_rabbit.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/oslo_messaging/_drivers/impl_rabbit.py b/oslo_messaging/_drivers/impl_rabbit.py index be41cc0f5..b0b82fade 100644 --- a/oslo_messaging/_drivers/impl_rabbit.py +++ b/oslo_messaging/_drivers/impl_rabbit.py @@ -491,7 +491,8 @@ class Connection(object): if purpose == rpc_amqp.PURPOSE_SEND: self._heartbeat_start() - LOG.debug('Connected to AMQP server on %(hostname)s:%(port)s', + LOG.debug('Connected to AMQP server on %(hostname)s:%(port)s ' + 'via [%(transport)s] client', self.connection.info()) # NOTE(sileht): value chosen according the best practice from kombu @@ -649,7 +650,7 @@ class Connection(object): consumer.declare(self) LOG.info(_LI('Reconnected to AMQP server on ' - '%(hostname)s:%(port)s'), + '%(hostname)s:%(port)s via [%(transport)s] client'), self.connection.info()) def execute_method(channel): From fb2037bcb492137ee7de5488c30ef8941b914e13 Mon Sep 17 00:00:00 2001 From: Julien Danjou Date: Thu, 12 Nov 2015 17:25:36 +0100 Subject: [PATCH 40/74] serializer: remove deprecated RequestContextSerializer This also allows us to drop the oslo.context dependency! Change-Id: I1434caf6323fb417ff99ceff865a0d43799e89b2 --- oslo_messaging/serializer.py | 28 +-------- oslo_messaging/tests/test_serializer.py | 75 ------------------------- requirements.txt | 1 - 3 files changed, 1 insertion(+), 103 deletions(-) delete mode 100644 oslo_messaging/tests/test_serializer.py diff --git a/oslo_messaging/serializer.py b/oslo_messaging/serializer.py index 8b7c0a7a3..78bf983b6 100644 --- a/oslo_messaging/serializer.py +++ b/oslo_messaging/serializer.py @@ -12,15 +12,12 @@ # License for the specific language governing permissions and limitations # under the License. -__all__ = ['Serializer', 'NoOpSerializer', 'JsonPayloadSerializer', - 'RequestContextSerializer'] +__all__ = ['Serializer', 'NoOpSerializer', 'JsonPayloadSerializer'] """Provides the definition of a message serialization handler""" import abc -from debtcollector import removals -from oslo_context import context as common_context from oslo_serialization import jsonutils import six @@ -64,29 +61,6 @@ class Serializer(object): """ -@removals.remove(version="2.9", removal_version="3.0") -class RequestContextSerializer(Serializer): - - def __init__(self, base): - self._base = base - - def serialize_entity(self, context, entity): - if not self._base: - return entity - return self._base.serialize_entity(context, entity) - - def deserialize_entity(self, context, entity): - if not self._base: - return entity - return self._base.deserialize_entity(context, entity) - - def serialize_context(self, context): - return context.to_dict() - - def deserialize_context(self, context): - return common_context.RequestContext.from_dict(context) - - class NoOpSerializer(Serializer): """A serializer that does nothing.""" diff --git a/oslo_messaging/tests/test_serializer.py b/oslo_messaging/tests/test_serializer.py deleted file mode 100644 index 329d9de71..000000000 --- a/oslo_messaging/tests/test_serializer.py +++ /dev/null @@ -1,75 +0,0 @@ -# Copyright 2015 Mirantis Inc. -# -# 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 oslo_context import context as common_context -from six.moves import mock - -from oslo_messaging import serializer -from oslo_messaging.tests import utils as test_utils - - -class TestRequestContextSerializer(test_utils.BaseTestCase): - - def setUp(self): - super(TestRequestContextSerializer, self).setUp() - - self.serializer = serializer.RequestContextSerializer(mock.MagicMock()) - self.context = common_context.RequestContext() - self.entity = {'foo': 'bar'} - - def test_serialize_entity(self): - self.serializer.serialize_entity(self.context, self.entity) - self.serializer._base.serialize_entity.assert_called_with( - self.context, self.entity) - - def test_serialize_entity_empty_base(self): - # NOTE(viktors): Return False for check `if self.serializer._base:` - bool_args = {'__bool__': lambda *args: False, - '__nonzero__': lambda *args: False} - self.serializer._base.configure_mock(**bool_args) - - entity = self.serializer.serialize_entity(self.context, self.entity) - self.assertFalse(self.serializer._base.serialize_entity.called) - self.assertEqual(entity, self.entity) - - def test_deserialize_entity(self): - self.serializer.deserialize_entity(self.context, self.entity) - self.serializer._base.deserialize_entity.assert_called_with( - self.context, self.entity) - - def test_deserialize_entity_empty_base(self): - # NOTE(viktors): Return False for check `if self.serializer._base:` - bool_args = {'__bool__': lambda *args: False, - '__nonzero__': lambda *args: False} - self.serializer._base.configure_mock(**bool_args) - - entity = self.serializer.deserialize_entity(self.context, self.entity) - self.assertFalse(self.serializer._base.serialize_entity.called) - self.assertEqual(entity, self.entity) - - def test_serialize_context(self): - new_context = self.serializer.serialize_context(self.context) - - self.assertEqual(new_context, self.context.to_dict()) - - @mock.patch.object(common_context.RequestContext, 'from_dict', - return_value='foobar') - def test_deserialize_context(self, mock_to_dict): - new_context = self.serializer.deserialize_context(self.context) - - mock_to_dict.assert_called_with(self.context) - self.assertEqual( - new_context, - common_context.RequestContext.from_dict(self.context) - ) diff --git a/requirements.txt b/requirements.txt index f7a5c87f3..c93f35eda 100644 --- a/requirements.txt +++ b/requirements.txt @@ -6,7 +6,6 @@ pbr>=1.6 futurist>=0.1.2 # Apache-2.0 oslo.config>=2.7.0 # Apache-2.0 -oslo.context>=0.2.0 # Apache-2.0 oslo.log>=1.12.0 # Apache-2.0 oslo.utils>=2.8.0 # Apache-2.0 oslo.serialization>=1.10.0 # Apache-2.0 From 533a0f8f3de497a7c46a97c98df89029703ae173 Mon Sep 17 00:00:00 2001 From: ZhiQiang Fan Date: Thu, 22 Oct 2015 01:30:33 -0600 Subject: [PATCH 41/74] Use oslo_config new type PortOpt for port options The oslo_config library provides new type PortOpt to validate the range of port now. Note, rpc_zmq_max_port is a socket port upper limit, so leave it untouched. Change-Id: Icac28141bcb09e2b894662de1a6497766351919d Closes-Bug: #1518256 ref: https://github.com/openstack/oslo.config/blob/2.6.0/oslo_config/cfg.py#L1114 --- oslo_messaging/_drivers/impl_rabbit.py | 8 ++++---- oslo_messaging/_drivers/impl_zmq.py | 8 +++++--- .../_drivers/zmq_driver/matchmaker/matchmaker_redis.py | 6 +++--- 3 files changed, 12 insertions(+), 10 deletions(-) diff --git a/oslo_messaging/_drivers/impl_rabbit.py b/oslo_messaging/_drivers/impl_rabbit.py index be41cc0f5..da708d6bf 100644 --- a/oslo_messaging/_drivers/impl_rabbit.py +++ b/oslo_messaging/_drivers/impl_rabbit.py @@ -89,10 +89,10 @@ rabbit_opts = [ deprecated_group='DEFAULT', help='The RabbitMQ broker address where a single node is ' 'used.'), - cfg.IntOpt('rabbit_port', - default=5672, - deprecated_group='DEFAULT', - help='The RabbitMQ broker port where a single node is used.'), + cfg.PortOpt('rabbit_port', + default=5672, + deprecated_group='DEFAULT', + help='The RabbitMQ broker port where a single node is used.'), cfg.ListOpt('rabbit_hosts', default=['$rabbit_host:$rabbit_port'], deprecated_group='DEFAULT', diff --git a/oslo_messaging/_drivers/impl_zmq.py b/oslo_messaging/_drivers/impl_zmq.py index acff79c7b..a77b6d4c9 100644 --- a/oslo_messaging/_drivers/impl_zmq.py +++ b/oslo_messaging/_drivers/impl_zmq.py @@ -81,11 +81,13 @@ zmq_opts = [ default=True, help='Shows whether zmq-messaging uses broker or not.'), - cfg.IntOpt('rpc_zmq_min_port', - default=49152, - help='Minimal port number for random ports range.'), + cfg.PortOpt('rpc_zmq_min_port', + default=49152, + help='Minimal port number for random ports range.'), cfg.IntOpt('rpc_zmq_max_port', + min=1, + max=65536, default=65536, help='Maximal port number for random ports range.'), diff --git a/oslo_messaging/_drivers/zmq_driver/matchmaker/matchmaker_redis.py b/oslo_messaging/_drivers/zmq_driver/matchmaker/matchmaker_redis.py index 576566a2f..3bbcf321a 100644 --- a/oslo_messaging/_drivers/zmq_driver/matchmaker/matchmaker_redis.py +++ b/oslo_messaging/_drivers/zmq_driver/matchmaker/matchmaker_redis.py @@ -27,9 +27,9 @@ matchmaker_redis_opts = [ cfg.StrOpt('host', default='127.0.0.1', help='Host to locate redis.'), - cfg.IntOpt('port', - default=6379, - help='Use this port to connect to redis host.'), + cfg.PortOpt('port', + default=6379, + help='Use this port to connect to redis host.'), cfg.StrOpt('password', default='', secret=True, From a811cf3e8bcc1704604082075a9c89d828a0e72d Mon Sep 17 00:00:00 2001 From: OpenStack Proposal Bot Date: Wed, 25 Nov 2015 22:27:46 +0000 Subject: [PATCH 42/74] Updated from global requirements Change-Id: Idc6649634438a2a0f33cc463594ad347a3674bc2 --- requirements.txt | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/requirements.txt b/requirements.txt index f7a5c87f3..e6c7e7741 100644 --- a/requirements.txt +++ b/requirements.txt @@ -13,13 +13,12 @@ oslo.serialization>=1.10.0 # Apache-2.0 oslo.service>=0.12.0 # Apache-2.0 oslo.i18n>=1.5.0 # Apache-2.0 stevedore>=1.5.0 # Apache-2.0 -debtcollector>=0.3.0 # Apache-2.0 +debtcollector>=0.3.0 # Apache-2.0 # for jsonutils six>=1.9.0 cachetools>=1.0.0 # MIT License -debtcollector>=0.3.0 # Apache-2.0 # FIXME(markmc): remove this when the drivers no longer # import eventlet From eea60cfb36e1f429185da0d801b9818322e5a73b Mon Sep 17 00:00:00 2001 From: Oleksii Zamiatin Date: Mon, 16 Nov 2015 15:07:15 +0200 Subject: [PATCH 43/74] [zmq] Driver optimizations for CALL New DEALER-based publisher provided for CALL. Futures-based reply waiting makes possible to refuse using of REQ blocking socket and also reduce number of openned sockets to a socket-per-target instead of socket-per-message as it was for CALL. Closes-Bug: #1517999 Optimized redis requests - request once instead of per each message. This should be elaborated with an autonomous nodes discovery mechanism to be correct in general case. Closes-Bug: #1517993 Reduced number of INFO log messages. Most of them switched to the DEBUG level. Closes-Bug: #1517997 Change-Id: Id017e79368cdc68613ddd7adef26411ea422dc8c --- oslo_messaging/_drivers/impl_zmq.py | 19 +- .../zmq_driver/broker/zmq_queue_proxy.py | 20 +- .../client/publishers/dealer/__init__.py | 0 .../dealer/zmq_dealer_call_publisher.py | 194 ++++++++++++++++++ .../{ => dealer}/zmq_dealer_publisher.py | 76 +------ .../dealer/zmq_dealer_publisher_proxy.py | 87 ++++++++ .../client/publishers/zmq_pub_publisher.py | 9 +- .../client/publishers/zmq_publisher_base.py | 36 ++-- .../client/publishers/zmq_push_publisher.py | 10 +- .../client/publishers/zmq_req_publisher.py | 19 +- .../_drivers/zmq_driver/client/zmq_client.py | 75 ++----- .../zmq_driver/client/zmq_client_base.py | 77 +++++++ .../zmq_driver/client/zmq_client_light.py | 46 +++++ .../zmq_driver/poller/threading_poller.py | 3 + .../server/consumers/zmq_consumer_base.py | 10 +- .../server/consumers/zmq_pull_consumer.py | 6 +- .../server/consumers/zmq_router_consumer.py | 14 +- .../zmq_driver/server/zmq_incoming_message.py | 5 +- .../_drivers/zmq_driver/zmq_async.py | 10 + .../_drivers/zmq_driver/zmq_socket.py | 3 + 20 files changed, 528 insertions(+), 191 deletions(-) create mode 100644 oslo_messaging/_drivers/zmq_driver/client/publishers/dealer/__init__.py create mode 100644 oslo_messaging/_drivers/zmq_driver/client/publishers/dealer/zmq_dealer_call_publisher.py rename oslo_messaging/_drivers/zmq_driver/client/publishers/{ => dealer}/zmq_dealer_publisher.py (58%) create mode 100644 oslo_messaging/_drivers/zmq_driver/client/publishers/dealer/zmq_dealer_publisher_proxy.py create mode 100644 oslo_messaging/_drivers/zmq_driver/client/zmq_client_base.py create mode 100644 oslo_messaging/_drivers/zmq_driver/client/zmq_client_light.py diff --git a/oslo_messaging/_drivers/impl_zmq.py b/oslo_messaging/_drivers/impl_zmq.py index a77b6d4c9..17bfe41e4 100644 --- a/oslo_messaging/_drivers/impl_zmq.py +++ b/oslo_messaging/_drivers/impl_zmq.py @@ -13,6 +13,7 @@ # under the License. import logging +import os import pprint import socket import threading @@ -23,6 +24,7 @@ from stevedore import driver from oslo_messaging._drivers import base from oslo_messaging._drivers import common as rpc_common from oslo_messaging._drivers.zmq_driver.client import zmq_client +from oslo_messaging._drivers.zmq_driver.client import zmq_client_light from oslo_messaging._drivers.zmq_driver.server import zmq_server from oslo_messaging._executors import impl_pooledexecutor # FIXME(markmc) @@ -78,8 +80,8 @@ zmq_opts = [ 'Poll raises timeout exception when timeout expired.'), cfg.BoolOpt('zmq_use_broker', - default=True, - help='Shows whether zmq-messaging uses broker or not.'), + default=False, + help='Configures zmq-messaging to use broker or not.'), cfg.PortOpt('rpc_zmq_min_port', default=49152, @@ -106,6 +108,7 @@ class LazyDriverItem(object): self.item_class = item_cls self.args = args self.kwargs = kwargs + self.process_id = os.getpid() def get(self): # NOTE(ozamiatin): Lazy initialization. @@ -114,11 +117,12 @@ class LazyDriverItem(object): # __init__, but 'fork' extensively used by services # breaks all things. - if self.item is not None: + if self.item is not None and os.getpid() == self.process_id: return self.item self._lock.acquire() - if self.item is None: + if self.item is None or os.getpid() != self.process_id: + self.process_id = os.getpid() self.item = self.item_class(*self.args, **self.kwargs) self._lock.release() return self.item @@ -175,12 +179,15 @@ class ZmqDriver(base.BaseDriver): self.notify_server = LazyDriverItem( zmq_server.ZmqServer, self, self.conf, self.matchmaker) + client_cls = zmq_client_light.ZmqClientLight \ + if conf.zmq_use_broker else zmq_client.ZmqClient + self.client = LazyDriverItem( - zmq_client.ZmqClient, self.conf, self.matchmaker, + client_cls, self.conf, self.matchmaker, self.allowed_remote_exmods) self.notifier = LazyDriverItem( - zmq_client.ZmqClient, self.conf, self.matchmaker, + client_cls, self.conf, self.matchmaker, self.allowed_remote_exmods) super(ZmqDriver, self).__init__(conf, url, default_exchange, diff --git a/oslo_messaging/_drivers/zmq_driver/broker/zmq_queue_proxy.py b/oslo_messaging/_drivers/zmq_driver/broker/zmq_queue_proxy.py index dd6665a03..eb752bed7 100644 --- a/oslo_messaging/_drivers/zmq_driver/broker/zmq_queue_proxy.py +++ b/oslo_messaging/_drivers/zmq_driver/broker/zmq_queue_proxy.py @@ -15,8 +15,8 @@ import logging from oslo_messaging._drivers.zmq_driver.broker import zmq_base_proxy -from oslo_messaging._drivers.zmq_driver.client.publishers\ - import zmq_dealer_publisher +from oslo_messaging._drivers.zmq_driver.client.publishers.dealer \ + import zmq_dealer_publisher_proxy from oslo_messaging._drivers.zmq_driver import zmq_address from oslo_messaging._drivers.zmq_driver import zmq_async from oslo_messaging._drivers.zmq_driver import zmq_names @@ -39,8 +39,8 @@ class UniversalQueueProxy(zmq_base_proxy.BaseProxy): LOG.info(_LI("Polling at universal proxy")) self.matchmaker = matchmaker - reply_receiver = zmq_dealer_publisher.ReplyReceiver(self.poller) - self.publisher = zmq_dealer_publisher.DealerPublisherProxy( + reply_receiver = zmq_dealer_publisher_proxy.ReplyReceiver(self.poller) + self.publisher = zmq_dealer_publisher_proxy.DealerPublisherProxy( conf, matchmaker, reply_receiver) def run(self): @@ -54,18 +54,18 @@ class UniversalQueueProxy(zmq_base_proxy.BaseProxy): self._redirect_reply(message) def _redirect_in_request(self, request): - LOG.info(_LI("-> Redirecting request %s to TCP publisher") - % request) + LOG.debug("-> Redirecting request %s to TCP publisher" + % request) self.publisher.send_request(request) def _redirect_reply(self, reply): - LOG.info(_LI("Reply proxy %s") % reply) + LOG.debug("Reply proxy %s" % reply) if reply[zmq_names.IDX_REPLY_TYPE] == zmq_names.ACK_TYPE: - LOG.info(_LI("Acknowledge dropped %s") % reply) + LOG.debug("Acknowledge dropped %s" % reply) return - LOG.info(_LI("<- Redirecting reply to ROUTER: reply: %s") - % reply[zmq_names.IDX_REPLY_BODY:]) + LOG.debug("<- Redirecting reply to ROUTER: reply: %s" + % reply[zmq_names.IDX_REPLY_BODY:]) self.router_socket.send_multipart(reply[zmq_names.IDX_REPLY_BODY:]) diff --git a/oslo_messaging/_drivers/zmq_driver/client/publishers/dealer/__init__.py b/oslo_messaging/_drivers/zmq_driver/client/publishers/dealer/__init__.py new file mode 100644 index 000000000..e69de29bb diff --git a/oslo_messaging/_drivers/zmq_driver/client/publishers/dealer/zmq_dealer_call_publisher.py b/oslo_messaging/_drivers/zmq_driver/client/publishers/dealer/zmq_dealer_call_publisher.py new file mode 100644 index 000000000..0c4e7536d --- /dev/null +++ b/oslo_messaging/_drivers/zmq_driver/client/publishers/dealer/zmq_dealer_call_publisher.py @@ -0,0 +1,194 @@ +# Copyright 2015 Mirantis, Inc. +# +# 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 logging +import threading + +from concurrent import futures +import futurist + +import oslo_messaging +from oslo_messaging._drivers import common as rpc_common +from oslo_messaging._drivers.zmq_driver.client.publishers\ + import zmq_publisher_base +from oslo_messaging._drivers.zmq_driver import zmq_address +from oslo_messaging._drivers.zmq_driver import zmq_async +from oslo_messaging._drivers.zmq_driver import zmq_names +from oslo_messaging._drivers.zmq_driver import zmq_socket + +LOG = logging.getLogger(__name__) + +zmq = zmq_async.import_zmq() + + +class DealerCallPublisher(zmq_publisher_base.PublisherBase): + """Thread-safe CALL publisher + + Used as faster and thread-safe publisher for CALL + instead of ReqPublisher. + """ + + def __init__(self, conf, matchmaker): + super(DealerCallPublisher, self).__init__(conf) + self.matchmaker = matchmaker + self.reply_waiter = ReplyWaiter(conf) + self.sender = RequestSender(conf, matchmaker, self.reply_waiter) \ + if not conf.zmq_use_broker else \ + RequestSenderLight(conf, matchmaker, self.reply_waiter) + + def send_request(self, request): + reply_future = self.sender.send_request(request) + try: + reply = reply_future.result(timeout=request.timeout) + except futures.TimeoutError: + raise oslo_messaging.MessagingTimeout( + "Timeout %s seconds was reached" % request.timeout) + finally: + self.reply_waiter.untrack_id(request.message_id) + + LOG.debug("Received reply %s" % reply) + if reply[zmq_names.FIELD_FAILURE]: + raise rpc_common.deserialize_remote_exception( + reply[zmq_names.FIELD_FAILURE], + request.allowed_remote_exmods) + else: + return reply[zmq_names.FIELD_REPLY] + + +class RequestSender(zmq_publisher_base.PublisherMultisend): + + def __init__(self, conf, matchmaker, reply_waiter): + super(RequestSender, self).__init__(conf, matchmaker, zmq.DEALER) + self.reply_waiter = reply_waiter + self.queue, self.empty_except = zmq_async.get_queue() + self.executor = zmq_async.get_executor(self.run_loop) + self.executor.execute() + + def send_request(self, request): + reply_future = futurist.Future() + self.reply_waiter.track_reply(reply_future, request.message_id) + self.queue.put(request) + return reply_future + + def _do_send_request(self, socket, request): + socket.send(b'', zmq.SNDMORE) + socket.send_pyobj(request) + + LOG.debug("Sending message_id %(message)s to a target %(target)s" + % {"message": request.message_id, + "target": request.target}) + + def _check_hosts_connections(self, target, listener_type): + if str(target) in self.outbound_sockets: + socket = self.outbound_sockets[str(target)] + else: + hosts = self.matchmaker.get_hosts( + target, listener_type) + socket = zmq_socket.ZmqSocket(self.zmq_context, self.socket_type) + self.outbound_sockets[str(target)] = socket + + for host in hosts: + self._connect_to_host(socket, host, target) + + return socket + + def run_loop(self): + try: + request = self.queue.get(timeout=self.conf.rpc_poll_timeout) + except self.empty_except: + return + + socket = self._check_hosts_connections( + request.target, zmq_names.socket_type_str(zmq.ROUTER)) + + self._do_send_request(socket, request) + self.reply_waiter.poll_socket(socket) + + +class RequestSenderLight(RequestSender): + """This class used with proxy. + + Simplified address matching because there is only + one proxy IPC address. + """ + + def __init__(self, conf, matchmaker, reply_waiter): + if not conf.zmq_use_broker: + raise rpc_common.RPCException("RequestSenderLight needs a proxy!") + + super(RequestSenderLight, self).__init__( + conf, matchmaker, reply_waiter) + + self.socket = None + + def _check_hosts_connections(self, target, listener_type): + if self.socket is None: + self.socket = zmq_socket.ZmqSocket(self.zmq_context, + self.socket_type) + self.outbound_sockets[str(target)] = self.socket + address = zmq_address.get_broker_address(self.conf) + self._connect_to_address(self.socket, address, target) + return self.socket + + def _do_send_request(self, socket, request): + LOG.debug("Sending %(type)s message_id %(message)s" + " to a target %(target)s" + % {"type": request.msg_type, + "message": request.message_id, + "target": request.target}) + + envelope = request.create_envelope() + + socket.send(b'', zmq.SNDMORE) + socket.send_pyobj(envelope, zmq.SNDMORE) + socket.send_pyobj(request) + + +class ReplyWaiter(object): + + def __init__(self, conf): + self.conf = conf + self.replies = {} + self.poller = zmq_async.get_poller() + self.executor = zmq_async.get_executor(self.run_loop) + self.executor.execute() + self._lock = threading.Lock() + + def track_reply(self, reply_future, message_id): + self._lock.acquire() + self.replies[message_id] = reply_future + self._lock.release() + + def untrack_id(self, message_id): + self._lock.acquire() + self.replies.pop(message_id) + self._lock.release() + + def poll_socket(self, socket): + + def _receive_method(socket): + empty = socket.recv() + assert empty == b'', "Empty expected!" + reply = socket.recv_pyobj() + LOG.debug("Received reply %s" % reply) + return reply + + self.poller.register(socket, recv_method=_receive_method) + + def run_loop(self): + reply, socket = self.poller.poll( + timeout=self.conf.rpc_poll_timeout) + if reply is not None: + call_future = self.replies[reply[zmq_names.FIELD_MSG_ID]] + call_future.set_result(reply) diff --git a/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_dealer_publisher.py b/oslo_messaging/_drivers/zmq_driver/client/publishers/dealer/zmq_dealer_publisher.py similarity index 58% rename from oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_dealer_publisher.py rename to oslo_messaging/_drivers/zmq_driver/client/publishers/dealer/zmq_dealer_publisher.py index 922607c7e..22e09c3fe 100644 --- a/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_dealer_publisher.py +++ b/oslo_messaging/_drivers/zmq_driver/client/publishers/dealer/zmq_dealer_publisher.py @@ -18,7 +18,7 @@ from oslo_messaging._drivers.zmq_driver.client.publishers\ import zmq_publisher_base from oslo_messaging._drivers.zmq_driver import zmq_async from oslo_messaging._drivers.zmq_driver import zmq_names -from oslo_messaging._i18n import _LI, _LW +from oslo_messaging._i18n import _LW LOG = logging.getLogger(__name__) @@ -34,7 +34,7 @@ class DealerPublisher(zmq_publisher_base.PublisherMultisend): self._check_request_pattern(request) - dealer_socket, hosts = self._check_hosts_connections( + dealer_socket = self._check_hosts_connections( request.target, zmq_names.socket_type_str(zmq.ROUTER)) if not dealer_socket.connections: @@ -61,15 +61,16 @@ class DealerPublisher(zmq_publisher_base.PublisherMultisend): socket.send(b'', zmq.SNDMORE) socket.send_pyobj(request) - LOG.info(_LI("Sending message_id %(message)s to a target %(target)s") - % {"message": request.message_id, - "target": request.target}) + LOG.debug("Sending message_id %(message)s to a target %(target)s" + % {"message": request.message_id, + "target": request.target}) def cleanup(self): super(DealerPublisher, self).cleanup() class DealerPublisherLight(zmq_publisher_base.PublisherBase): + """Used when publishing to proxy. """ def __init__(self, conf, address): super(DealerPublisherLight, self).__init__(conf) @@ -92,68 +93,6 @@ class DealerPublisherLight(zmq_publisher_base.PublisherBase): self.socket.close() -class DealerPublisherProxy(DealerPublisher): - - def __init__(self, conf, matchmaker, reply_receiver): - super(DealerPublisherProxy, self).__init__(conf, matchmaker) - self.reply_receiver = reply_receiver - - def send_request(self, multipart_message): - - envelope = multipart_message[zmq_names.MULTIPART_IDX_ENVELOPE] - - LOG.info(_LI("Envelope: %s") % envelope) - - target = envelope[zmq_names.FIELD_TARGET] - dealer_socket, hosts = self._check_hosts_connections( - target, zmq_names.socket_type_str(zmq.ROUTER)) - - if not dealer_socket.connections: - # NOTE(ozamiatin): Here we can provide - # a queue for keeping messages to send them later - # when some listener appears. However such approach - # being more reliable will consume additional memory. - LOG.warning(_LW("Request %s was dropped because no connection") - % envelope[zmq_names.FIELD_MSG_TYPE]) - return - - self.reply_receiver.track_socket(dealer_socket.handle) - - LOG.info(_LI("Sending message %(message)s to a target %(target)s") - % {"message": envelope[zmq_names.FIELD_MSG_ID], - "target": envelope[zmq_names.FIELD_TARGET]}) - - if envelope[zmq_names.FIELD_MSG_TYPE] in zmq_names.MULTISEND_TYPES: - for _ in range(dealer_socket.connections_count()): - self._send_request(dealer_socket, multipart_message) - else: - self._send_request(dealer_socket, multipart_message) - - def _send_request(self, socket, multipart_message): - - socket.send(b'', zmq.SNDMORE) - socket.send_pyobj( - multipart_message[zmq_names.MULTIPART_IDX_ENVELOPE], - zmq.SNDMORE) - socket.send(multipart_message[zmq_names.MULTIPART_IDX_BODY]) - - -class ReplyReceiver(object): - - def __init__(self, poller): - self.poller = poller - LOG.info(_LI("Reply waiter created in broker")) - - def _receive_reply(self, socket): - return socket.recv_multipart() - - def track_socket(self, socket): - self.poller.register(socket, self._receive_reply) - - def cleanup(self): - self.poller.close() - - class AcknowledgementReceiver(object): def __init__(self): @@ -172,8 +111,7 @@ class AcknowledgementReceiver(object): def poll_for_acknowledgements(self): ack_message, socket = self.poller.poll() - LOG.info(_LI("Message %s acknowledged") - % ack_message[zmq_names.FIELD_ID]) + LOG.debug("Message %s acknowledged" % ack_message[zmq_names.FIELD_ID]) def cleanup(self): self.thread.stop() diff --git a/oslo_messaging/_drivers/zmq_driver/client/publishers/dealer/zmq_dealer_publisher_proxy.py b/oslo_messaging/_drivers/zmq_driver/client/publishers/dealer/zmq_dealer_publisher_proxy.py new file mode 100644 index 000000000..c8ad98345 --- /dev/null +++ b/oslo_messaging/_drivers/zmq_driver/client/publishers/dealer/zmq_dealer_publisher_proxy.py @@ -0,0 +1,87 @@ +# Copyright 2015 Mirantis, Inc. +# +# 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 logging + +from oslo_messaging._drivers.zmq_driver.client.publishers.dealer \ + import zmq_dealer_publisher +from oslo_messaging._drivers.zmq_driver import zmq_async +from oslo_messaging._drivers.zmq_driver import zmq_names +from oslo_messaging._i18n import _LI, _LW + +zmq = zmq_async.import_zmq() + +LOG = logging.getLogger(__name__) + + +class DealerPublisherProxy(zmq_dealer_publisher.DealerPublisher): + + def __init__(self, conf, matchmaker, reply_receiver): + super(DealerPublisherProxy, self).__init__(conf, matchmaker) + self.reply_receiver = reply_receiver + + def send_request(self, multipart_message): + + envelope = multipart_message[zmq_names.MULTIPART_IDX_ENVELOPE] + + LOG.debug("Envelope: %s" % envelope) + + target = envelope[zmq_names.FIELD_TARGET] + dealer_socket = self._check_hosts_connections( + target, zmq_names.socket_type_str(zmq.ROUTER)) + + if not dealer_socket.connections: + # NOTE(ozamiatin): Here we can provide + # a queue for keeping messages to send them later + # when some listener appears. However such approach + # being more reliable will consume additional memory. + LOG.warning(_LW("Request %s was dropped because no connection") + % envelope[zmq_names.FIELD_MSG_TYPE]) + return + + self.reply_receiver.track_socket(dealer_socket.handle) + + LOG.debug("Sending message %(message)s to a target %(target)s" + % {"message": envelope[zmq_names.FIELD_MSG_ID], + "target": envelope[zmq_names.FIELD_TARGET]}) + + if envelope[zmq_names.FIELD_MSG_TYPE] in zmq_names.MULTISEND_TYPES: + for _ in range(dealer_socket.connections_count()): + self._send_request(dealer_socket, multipart_message) + else: + self._send_request(dealer_socket, multipart_message) + + def _send_request(self, socket, multipart_message): + + socket.send(b'', zmq.SNDMORE) + socket.send_pyobj( + multipart_message[zmq_names.MULTIPART_IDX_ENVELOPE], + zmq.SNDMORE) + socket.send(multipart_message[zmq_names.MULTIPART_IDX_BODY]) + + +class ReplyReceiver(object): + + def __init__(self, poller): + self.poller = poller + LOG.info(_LI("Reply waiter created in broker")) + + def _receive_reply(self, socket): + return socket.recv_multipart() + + def track_socket(self, socket): + self.poller.register(socket, self._receive_reply) + + def cleanup(self): + self.poller.close() diff --git a/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_pub_publisher.py b/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_pub_publisher.py index 1cd3360eb..6ecfb2f77 100644 --- a/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_pub_publisher.py +++ b/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_pub_publisher.py @@ -18,7 +18,6 @@ from oslo_messaging._drivers.zmq_driver.client.publishers\ import zmq_publisher_base from oslo_messaging._drivers.zmq_driver import zmq_async from oslo_messaging._drivers.zmq_driver import zmq_names -from oslo_messaging._i18n import _LI LOG = logging.getLogger(__name__) @@ -35,7 +34,7 @@ class PubPublisher(zmq_publisher_base.PublisherMultisend): if request.msg_type not in zmq_names.NOTIFY_TYPES: raise zmq_publisher_base.UnsupportedSendPattern(request.msg_type) - pub_socket, hosts = self._check_hosts_connections( + pub_socket = self._check_hosts_connections( request.target, zmq_names.socket_type_str(zmq.SUB)) self._send_request(pub_socket, request) @@ -43,6 +42,6 @@ class PubPublisher(zmq_publisher_base.PublisherMultisend): super(PubPublisher, self)._send_request(socket, request) - LOG.info(_LI("Publishing message %(message)s to a target %(target)s") - % {"message": request.message, - "target": request.target}) + LOG.debug("Publishing message %(message)s to a target %(target)s" + % {"message": request.message, + "target": request.target}) diff --git a/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_publisher_base.py b/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_publisher_base.py index 46e8ef535..6daf6db18 100644 --- a/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_publisher_base.py +++ b/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_publisher_base.py @@ -14,6 +14,7 @@ import abc import logging +import uuid import six @@ -89,12 +90,11 @@ class PublisherBase(object): :param request: Message data and destination container object :type request: zmq_request.Request """ - LOG.info(_LI("Sending %(type)s message_id %(message)s to a target" - "%(target)s host:%(host)s") - % {"type": request.msg_type, - "message": request.message_id, - "target": request.target, - "host": request.host}) + LOG.debug("Sending %(type)s message_id %(message)s to a target" + "%(target)s" + % {"type": request.msg_type, + "message": request.message_id, + "target": request.target}) socket.send_pyobj(request) def cleanup(self): @@ -124,28 +124,30 @@ class PublisherMultisend(PublisherBase): def _check_hosts_connections(self, target, listener_type): # TODO(ozamiatin): Place for significant optimization # Matchmaker cache should be implemented - hosts = self.matchmaker.get_hosts( - target, listener_type) if str(target) in self.outbound_sockets: socket = self.outbound_sockets[str(target)] else: + hosts = self.matchmaker.get_hosts(target, listener_type) socket = zmq_socket.ZmqSocket(self.zmq_context, self.socket_type) self.outbound_sockets[str(target)] = socket + for host in hosts: + self._connect_to_host(socket, host, target) - for host in hosts: - self._connect_to_host(socket, host, target) + return socket - return socket, hosts - - def _connect_to_host(self, socket, host, target): - address = zmq_address.get_tcp_direct_address(host) - LOG.info(address) + def _connect_to_address(self, socket, address, target): stype = zmq_names.socket_type_str(self.socket_type) try: LOG.info(_LI("Connecting %(stype)s to %(address)s for %(target)s") % {"stype": stype, "address": address, "target": target}) + + if six.PY3: + socket.setsockopt_string(zmq.IDENTITY, str(uuid.uuid1())) + else: + socket.handle.identity = str(uuid.uuid1()) + socket.connect(address) except zmq.ZMQError as e: errmsg = _LE("Failed connecting %(stype) to %(address)s: %(e)s")\ @@ -153,3 +155,7 @@ class PublisherMultisend(PublisherBase): LOG.error(_LE("Failed connecting %(stype) to %(address)s: %(e)s") % (stype, address, e)) raise rpc_common.RPCException(errmsg) + + def _connect_to_host(self, socket, host, target): + address = zmq_address.get_tcp_direct_address(host) + self._connect_to_address(socket, address, target) diff --git a/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_push_publisher.py b/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_push_publisher.py index 7fcb46961..3a38cfd43 100644 --- a/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_push_publisher.py +++ b/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_push_publisher.py @@ -18,7 +18,7 @@ from oslo_messaging._drivers.zmq_driver.client.publishers\ import zmq_publisher_base from oslo_messaging._drivers.zmq_driver import zmq_async from oslo_messaging._drivers.zmq_driver import zmq_names -from oslo_messaging._i18n import _LI, _LW +from oslo_messaging._i18n import _LW LOG = logging.getLogger(__name__) @@ -35,7 +35,7 @@ class PushPublisher(zmq_publisher_base.PublisherMultisend): if request.msg_type == zmq_names.CALL_TYPE: raise zmq_publisher_base.UnsupportedSendPattern(request.msg_type) - push_socket, hosts = self._check_hosts_connections( + push_socket = self._check_hosts_connections( request.target, zmq_names.socket_type_str(zmq.PULL)) if not push_socket.connections: @@ -53,6 +53,6 @@ class PushPublisher(zmq_publisher_base.PublisherMultisend): super(PushPublisher, self)._send_request(socket, request) - LOG.info(_LI("Publishing message %(message)s to a target %(target)s") - % {"message": request.message, - "target": request.target}) + LOG.debug("Publishing message %(message)s to a target %(target)s" + % {"message": request.message, + "target": request.target}) diff --git a/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_req_publisher.py b/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_req_publisher.py index ace229ba5..78330f3a3 100644 --- a/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_req_publisher.py +++ b/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_req_publisher.py @@ -82,8 +82,11 @@ class ReqPublisher(zmq_publisher_base.PublisherBase): def _receive_reply(socket, request): def _receive_method(socket): - return socket.recv_pyobj() + reply = socket.recv_pyobj() + LOG.debug("Received reply %s" % reply) + return reply + LOG.debug("Start waiting reply") # NOTE(ozamiatin): Check for retry here (no retries now) with contextlib.closing(zmq_async.get_reply_poller()) as poller: poller.register(socket, recv_method=_receive_method) @@ -91,7 +94,7 @@ class ReqPublisher(zmq_publisher_base.PublisherBase): if reply is None: raise oslo_messaging.MessagingTimeout( "Timeout %s seconds was reached" % request.timeout) - LOG.info(_LI("Received reply %s") % reply) + LOG.debug("Received reply %s" % reply) if reply[zmq_names.FIELD_FAILURE]: raise rpc_common.deserialize_remote_exception( reply[zmq_names.FIELD_FAILURE], @@ -114,12 +117,12 @@ class ReqPublisherLight(ReqPublisher): def _send_request(self, socket, request): - LOG.info(_LI("Sending %(type)s message_id %(message)s" - " to a target %(target)s, host:%(host)s") - % {"type": request.msg_type, - "message": request.message_id, - "target": request.target, - "host": request.host}) + LOG.debug("Sending %(type)s message_id %(message)s" + " to a target %(target)s, host:%(host)s" + % {"type": request.msg_type, + "message": request.message_id, + "target": request.target, + "host": request.host}) envelope = request.create_envelope() diff --git a/oslo_messaging/_drivers/zmq_driver/client/zmq_client.py b/oslo_messaging/_drivers/zmq_driver/client/zmq_client.py index 9fb38224d..c6e895863 100644 --- a/oslo_messaging/_drivers/zmq_driver/client/zmq_client.py +++ b/oslo_messaging/_drivers/zmq_driver/client/zmq_client.py @@ -12,70 +12,33 @@ # License for the specific language governing permissions and limitations # under the License. -import contextlib -from oslo_messaging._drivers.zmq_driver.client.publishers\ +from oslo_messaging._drivers import common as rpc_common +from oslo_messaging._drivers.zmq_driver.client.publishers.dealer \ + import zmq_dealer_call_publisher +from oslo_messaging._drivers.zmq_driver.client.publishers.dealer \ import zmq_dealer_publisher -from oslo_messaging._drivers.zmq_driver.client.publishers\ - import zmq_req_publisher -from oslo_messaging._drivers.zmq_driver.client import zmq_request -from oslo_messaging._drivers.zmq_driver import zmq_address +from oslo_messaging._drivers.zmq_driver.client import zmq_client_base from oslo_messaging._drivers.zmq_driver import zmq_async +from oslo_messaging._drivers.zmq_driver import zmq_names zmq = zmq_async.import_zmq() -class ZmqClient(object): +class ZmqClient(zmq_client_base.ZmqClientBase): def __init__(self, conf, matchmaker=None, allowed_remote_exmods=None): - self.conf = conf - self.context = zmq.Context() - self.matchmaker = matchmaker - self.allowed_remote_exmods = allowed_remote_exmods or [] + if conf.zmq_use_broker: + raise rpc_common.RPCException("This client doesn't need proxy!") - self.dealer_publisher = None - if self.conf.zmq_use_broker: - self.dealer_publisher = zmq_dealer_publisher.DealerPublisherLight( - conf, zmq_address.get_broker_address(self.conf)) - self.req_publisher_cls = zmq_req_publisher.ReqPublisherLight - else: - self.dealer_publisher = zmq_dealer_publisher.DealerPublisher( - conf, matchmaker) - self.req_publisher_cls = zmq_req_publisher.ReqPublisher + super(ZmqClient, self).__init__( + conf, matchmaker, allowed_remote_exmods, + publishers={ + zmq_names.CALL_TYPE: + zmq_dealer_call_publisher.DealerCallPublisher( + conf, matchmaker), - def send_call(self, target, context, message, timeout=None, retry=None): - with contextlib.closing(zmq_request.CallRequest( - target, context=context, message=message, - timeout=timeout, retry=retry, - allowed_remote_exmods=self.allowed_remote_exmods)) as request: - with contextlib.closing(self.req_publisher_cls( - self.conf, self.matchmaker)) as req_publisher: - return req_publisher.send_request(request) - - def send_cast(self, target, context, message, timeout=None, retry=None): - with contextlib.closing(zmq_request.CastRequest( - target, context=context, message=message, - timeout=timeout, retry=retry)) as request: - self.dealer_publisher.send_request(request) - - def send_fanout(self, target, context, message, timeout=None, retry=None): - with contextlib.closing(zmq_request.FanoutRequest( - target, context=context, message=message, - timeout=timeout, retry=retry)) as request: - self.dealer_publisher.send_request(request) - - def send_notify(self, target, context, message, version, retry=None): - with contextlib.closing(zmq_request.NotificationRequest( - target, context, message, version=version, - retry=retry)) as request: - self.dealer_publisher.send_request(request) - - def send_notify_fanout(self, target, context, message, version, - retry=None): - with contextlib.closing(zmq_request.NotificationFanoutRequest( - target, context, message, version=version, - retry=retry)) as request: - self.dealer_publisher.send_request(request) - - def cleanup(self): - self.dealer_publisher.cleanup() + "default": zmq_dealer_publisher.DealerPublisher( + conf, matchmaker) + } + ) diff --git a/oslo_messaging/_drivers/zmq_driver/client/zmq_client_base.py b/oslo_messaging/_drivers/zmq_driver/client/zmq_client_base.py new file mode 100644 index 000000000..aa7cd12d1 --- /dev/null +++ b/oslo_messaging/_drivers/zmq_driver/client/zmq_client_base.py @@ -0,0 +1,77 @@ +# Copyright 2015 Mirantis, Inc. +# +# 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 contextlib + +from oslo_messaging._drivers.zmq_driver.client import zmq_request +from oslo_messaging._drivers.zmq_driver import zmq_async +from oslo_messaging._drivers.zmq_driver import zmq_names + +zmq = zmq_async.import_zmq() + + +class ZmqClientBase(object): + + def __init__(self, conf, matchmaker=None, allowed_remote_exmods=None, + publishers=None): + self.conf = conf + self.context = zmq.Context() + self.matchmaker = matchmaker + self.allowed_remote_exmods = allowed_remote_exmods or [] + + self.publishers = publishers + self.call_publisher = publishers.get(zmq_names.CALL_TYPE) \ + or publishers["default"] + self.cast_publisher = publishers.get(zmq_names.CAST_TYPE) \ + or publishers["default"] + self.fanout_publisher = publishers.get(zmq_names.CAST_FANOUT_TYPE) \ + or publishers["default"] + self.notify_publisher = publishers.get(zmq_names.NOTIFY_TYPE) \ + or publishers["default"] + + def send_call(self, target, context, message, timeout=None, retry=None): + with contextlib.closing(zmq_request.CallRequest( + target, context=context, message=message, + timeout=timeout, retry=retry, + allowed_remote_exmods=self.allowed_remote_exmods)) as request: + return self.call_publisher.send_request(request) + + def send_cast(self, target, context, message, timeout=None, retry=None): + with contextlib.closing(zmq_request.CastRequest( + target, context=context, message=message, + timeout=timeout, retry=retry)) as request: + self.cast_publisher.send_request(request) + + def send_fanout(self, target, context, message, timeout=None, retry=None): + with contextlib.closing(zmq_request.FanoutRequest( + target, context=context, message=message, + timeout=timeout, retry=retry)) as request: + self.fanout_publisher.send_request(request) + + def send_notify(self, target, context, message, version, retry=None): + with contextlib.closing(zmq_request.NotificationRequest( + target, context, message, version=version, + retry=retry)) as request: + self.notify_publisher.send_request(request) + + def send_notify_fanout(self, target, context, message, version, + retry=None): + with contextlib.closing(zmq_request.NotificationFanoutRequest( + target, context, message, version=version, + retry=retry)) as request: + self.notify_publisher.send_request(request) + + def cleanup(self): + for publisher in self.publishers.values(): + publisher.cleanup() diff --git a/oslo_messaging/_drivers/zmq_driver/client/zmq_client_light.py b/oslo_messaging/_drivers/zmq_driver/client/zmq_client_light.py new file mode 100644 index 000000000..873911f8d --- /dev/null +++ b/oslo_messaging/_drivers/zmq_driver/client/zmq_client_light.py @@ -0,0 +1,46 @@ +# Copyright 2015 Mirantis, Inc. +# +# 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 oslo_messaging._drivers import common as rpc_common +from oslo_messaging._drivers.zmq_driver.client.publishers.dealer \ + import zmq_dealer_call_publisher +from oslo_messaging._drivers.zmq_driver.client.publishers.dealer \ + import zmq_dealer_publisher +from oslo_messaging._drivers.zmq_driver.client import zmq_client_base +from oslo_messaging._drivers.zmq_driver import zmq_address +from oslo_messaging._drivers.zmq_driver import zmq_async +from oslo_messaging._drivers.zmq_driver import zmq_names + +zmq = zmq_async.import_zmq() + + +class ZmqClientLight(zmq_client_base.ZmqClientBase): + + def __init__(self, conf, matchmaker=None, allowed_remote_exmods=None): + if not conf.zmq_use_broker: + raise rpc_common.RPCException( + "This client needs proxy to be configured!") + + super(ZmqClientLight, self).__init__( + conf, matchmaker, allowed_remote_exmods, + publishers={ + zmq_names.CALL_TYPE: + zmq_dealer_call_publisher.DealerCallPublisher( + conf, matchmaker), + + "default": zmq_dealer_publisher.DealerPublisherLight( + conf, zmq_address.get_broker_address(self.conf)) + } + ) diff --git a/oslo_messaging/_drivers/zmq_driver/poller/threading_poller.py b/oslo_messaging/_drivers/zmq_driver/poller/threading_poller.py index 518d32e49..8167715f1 100644 --- a/oslo_messaging/_drivers/zmq_driver/poller/threading_poller.py +++ b/oslo_messaging/_drivers/zmq_driver/poller/threading_poller.py @@ -38,6 +38,7 @@ class ThreadingPoller(zmq_poller.ZmqPoller): self.recv_methods = {} def register(self, socket, recv_method=None): + LOG.debug("Registering socket") if socket in self.recv_methods: return if recv_method is not None: @@ -46,6 +47,8 @@ class ThreadingPoller(zmq_poller.ZmqPoller): def poll(self, timeout=None): + LOG.debug("Entering poll method") + if timeout: timeout *= 1000 # zmq poller waits milliseconds diff --git a/oslo_messaging/_drivers/zmq_driver/server/consumers/zmq_consumer_base.py b/oslo_messaging/_drivers/zmq_driver/server/consumers/zmq_consumer_base.py index f002c45af..b7532a74a 100644 --- a/oslo_messaging/_drivers/zmq_driver/server/consumers/zmq_consumer_base.py +++ b/oslo_messaging/_drivers/zmq_driver/server/consumers/zmq_consumer_base.py @@ -21,7 +21,7 @@ from oslo_messaging._drivers import common as rpc_common from oslo_messaging._drivers.zmq_driver import zmq_async from oslo_messaging._drivers.zmq_driver import zmq_names from oslo_messaging._drivers.zmq_driver import zmq_socket -from oslo_messaging._i18n import _LE, _LI +from oslo_messaging._i18n import _LE LOG = logging.getLogger(__name__) @@ -44,10 +44,10 @@ class ConsumerBase(object): self.conf, self.context, socket_type) self.sockets.append(socket) self.poller.register(socket, self.receive_message) - LOG.info(_LI("Run %(stype)s consumer on %(addr)s:%(port)d"), - {"stype": zmq_names.socket_type_str(socket_type), - "addr": socket.bind_address, - "port": socket.port}) + LOG.debug("Run %(stype)s consumer on %(addr)s:%(port)d", + {"stype": zmq_names.socket_type_str(socket_type), + "addr": socket.bind_address, + "port": socket.port}) return socket except zmq.ZMQError as e: errmsg = _LE("Failed binding to port %(port)d: %(e)s")\ diff --git a/oslo_messaging/_drivers/zmq_driver/server/consumers/zmq_pull_consumer.py b/oslo_messaging/_drivers/zmq_driver/server/consumers/zmq_pull_consumer.py index 98ef3a73c..81cf7fde0 100644 --- a/oslo_messaging/_drivers/zmq_driver/server/consumers/zmq_pull_consumer.py +++ b/oslo_messaging/_drivers/zmq_driver/server/consumers/zmq_pull_consumer.py @@ -56,9 +56,9 @@ class PullConsumer(zmq_consumer_base.SingleSocketConsumer): assert msg_type is not None, 'Bad format: msg type expected' context = socket.recv_pyobj() message = socket.recv_pyobj() - LOG.info(_LI("Received %(msg_type)s message %(msg)s") - % {"msg_type": msg_type, - "msg": str(message)}) + LOG.debug("Received %(msg_type)s message %(msg)s" + % {"msg_type": msg_type, + "msg": str(message)}) if msg_type in (zmq_names.CAST_TYPES + zmq_names.NOTIFY_TYPES): return PullIncomingMessage(self.server, context, message) diff --git a/oslo_messaging/_drivers/zmq_driver/server/consumers/zmq_router_consumer.py b/oslo_messaging/_drivers/zmq_driver/server/consumers/zmq_router_consumer.py index a501ef7a7..f5885c55a 100644 --- a/oslo_messaging/_drivers/zmq_driver/server/consumers/zmq_router_consumer.py +++ b/oslo_messaging/_drivers/zmq_driver/server/consumers/zmq_router_consumer.py @@ -21,7 +21,7 @@ from oslo_messaging._drivers.zmq_driver.server import zmq_incoming_message from oslo_messaging._drivers.zmq_driver import zmq_address from oslo_messaging._drivers.zmq_driver import zmq_async from oslo_messaging._drivers.zmq_driver import zmq_names -from oslo_messaging._i18n import _LE, _LI +from oslo_messaging._i18n import _LE LOG = logging.getLogger(__name__) @@ -43,7 +43,7 @@ class RouterIncomingMessage(base.IncomingMessage): """Reply is not needed for non-call messages""" def acknowledge(self): - LOG.info("Not sending acknowledge for %s", self.msg_id) + LOG.debug("Not sending acknowledge for %s", self.msg_id) def requeue(self): """Requeue is not supported""" @@ -83,11 +83,11 @@ class RouterConsumer(zmq_consumer_base.SingleSocketConsumer): def receive_message(self, socket): try: request, reply_id = self._receive_request(socket) - LOG.info(_LI("[%(host)s] Received %(type)s, %(id)s, %(target)s") - % {"host": self.host, - "type": request.msg_type, - "id": request.message_id, - "target": request.target}) + LOG.debug("[%(host)s] Received %(type)s, %(id)s, %(target)s" + % {"host": self.host, + "type": request.msg_type, + "id": request.message_id, + "target": request.target}) if request.msg_type == zmq_names.CALL_TYPE: return zmq_incoming_message.ZmqIncomingRequest( diff --git a/oslo_messaging/_drivers/zmq_driver/server/zmq_incoming_message.py b/oslo_messaging/_drivers/zmq_driver/server/zmq_incoming_message.py index 5e932493f..e009d55c9 100644 --- a/oslo_messaging/_drivers/zmq_driver/server/zmq_incoming_message.py +++ b/oslo_messaging/_drivers/zmq_driver/server/zmq_incoming_message.py @@ -45,9 +45,10 @@ class ZmqIncomingRequest(base.IncomingMessage): zmq_names.FIELD_REPLY: reply, zmq_names.FIELD_FAILURE: failure, zmq_names.FIELD_LOG_FAILURE: log_failure, - zmq_names.FIELD_ID: self.request.proxy_reply_id} + zmq_names.FIELD_ID: self.request.proxy_reply_id, + zmq_names.FIELD_MSG_ID: self.request.message_id} - LOG.info("Replying %s REP", (str(self.request.message_id))) + LOG.debug("Replying %s", (str(self.request.message_id))) self.received = True self.reply_socket.send(self.reply_id, zmq.SNDMORE) diff --git a/oslo_messaging/_drivers/zmq_driver/zmq_async.py b/oslo_messaging/_drivers/zmq_driver/zmq_async.py index 7a993a285..4de248f39 100644 --- a/oslo_messaging/_drivers/zmq_driver/zmq_async.py +++ b/oslo_messaging/_drivers/zmq_driver/zmq_async.py @@ -80,3 +80,13 @@ def _raise_error_if_invalid_config_value(zmq_concurrency): if zmq_concurrency not in ZMQ_MODULES: errmsg = _('Invalid zmq_concurrency value: %s') raise ValueError(errmsg % zmq_concurrency) + + +def get_queue(zmq_concurrency='eventlet'): + _raise_error_if_invalid_config_value(zmq_concurrency) + if zmq_concurrency == 'eventlet' and _is_eventlet_zmq_available(): + import eventlet + return eventlet.queue.Queue(), eventlet.queue.Empty + else: + import six + return six.moves.queue.Queue(), six.moves.queue.Empty diff --git a/oslo_messaging/_drivers/zmq_driver/zmq_socket.py b/oslo_messaging/_drivers/zmq_driver/zmq_socket.py index 8f79bd083..4119e5735 100644 --- a/oslo_messaging/_drivers/zmq_driver/zmq_socket.py +++ b/oslo_messaging/_drivers/zmq_driver/zmq_socket.py @@ -47,6 +47,9 @@ class ZmqSocket(object): def setsockopt(self, *args, **kwargs): self.handle.setsockopt(*args, **kwargs) + def setsockopt_string(self, *args, **kwargs): + self.handle.setsockopt_string(*args, **kwargs) + def send(self, *args, **kwargs): self.handle.send(*args, **kwargs) From 47baebde1f04b1dbd657250084b64f0990b768c2 Mon Sep 17 00:00:00 2001 From: Oleksii Zamiatin Date: Tue, 24 Nov 2015 13:21:54 +0200 Subject: [PATCH 44/74] [zmq] Random failure with ZmqPortRangeExceededException Don't restrict ports range for all the other unit tests. Change-Id: I47d566f3488d371cff604dac72c94c775d729487 Closes-Bug: #1519312 --- .../tests/drivers/zmq/test_impl_zmq.py | 47 +++++++++++-------- 1 file changed, 28 insertions(+), 19 deletions(-) diff --git a/oslo_messaging/tests/drivers/zmq/test_impl_zmq.py b/oslo_messaging/tests/drivers/zmq/test_impl_zmq.py index feb1f7656..a11b61f59 100644 --- a/oslo_messaging/tests/drivers/zmq/test_impl_zmq.py +++ b/oslo_messaging/tests/drivers/zmq/test_impl_zmq.py @@ -79,9 +79,7 @@ class ZmqBaseTestCase(test_utils.BaseTestCase): 'rpc_response_timeout': 5, 'rpc_zmq_ipc_dir': self.internal_ipc_dir, 'zmq_use_broker': False, - 'rpc_zmq_matchmaker': 'dummy', - 'rpc_zmq_min_port': 5555, - 'rpc_zmq_max_port': 5560} + 'rpc_zmq_matchmaker': 'dummy'} self.config(**kwargs) # Get driver @@ -93,6 +91,33 @@ class ZmqBaseTestCase(test_utils.BaseTestCase): self.addCleanup(stopRpc(self.__dict__)) +class ZmqTestPortsRange(ZmqBaseTestCase): + + @testtools.skipIf(zmq is None, "zmq not available") + def setUp(self): + super(ZmqTestPortsRange, self).setUp() + + # Set config values + kwargs = {'rpc_zmq_min_port': 5555, + 'rpc_zmq_max_port': 5560} + self.config(**kwargs) + + def test_ports_range(self): + listeners = [] + + for i in range(10): + try: + target = oslo_messaging.Target(topic='testtopic_'+str(i)) + new_listener = self.driver.listen(target) + listeners.append(new_listener) + except zmq_socket.ZmqPortRangeExceededException: + pass + + self.assertLessEqual(len(listeners), 5) + + for l in listeners: + l.cleanup() + class TestConfZmqDriverLoad(test_utils.BaseTestCase): @testtools.skipIf(zmq is None, "zmq not available") @@ -118,22 +143,6 @@ class stopRpc(object): class TestZmqBasics(ZmqBaseTestCase): - def test_ports_range(self): - listeners = [] - - for i in range(10): - try: - target = oslo_messaging.Target(topic='testtopic_'+str(i)) - new_listener = self.driver.listen(target) - listeners.append(new_listener) - except zmq_socket.ZmqPortRangeExceededException: - pass - - self.assertLessEqual(len(listeners), 5) - - for l in listeners: - l.cleanup() - def test_send_receive_raises(self): """Call() without method.""" target = oslo_messaging.Target(topic='testtopic') From 067cac36be9d57fe4d92490cb1f17b683213a81b Mon Sep 17 00:00:00 2001 From: OpenStack Proposal Bot Date: Fri, 27 Nov 2015 17:47:01 +0000 Subject: [PATCH 45/74] Updated from global requirements Change-Id: I6670a181e2e2fe3ed1c96d0a16525c2d6eada436 --- requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/requirements.txt b/requirements.txt index 9a5f77788..9e5fb84b1 100644 --- a/requirements.txt +++ b/requirements.txt @@ -36,7 +36,7 @@ amqp>=1.4.0 kombu>=3.0.7 # middleware -oslo.middleware>=2.9.0 # Apache-2.0 +oslo.middleware>=3.0.0 # Apache-2.0 # needed by the aioeventlet executor aioeventlet>=0.4 From 6cd1dcebc0801dc16db5f64c81baf1fe17165c88 Mon Sep 17 00:00:00 2001 From: "Davanum Srinivas (dims)" Date: Sun, 29 Nov 2015 02:21:46 +0000 Subject: [PATCH 46/74] Revert "serializer: remove deprecated RequestContextSerializer" This reverts commit fb2037bcb492137ee7de5488c30ef8941b914e13. Change-Id: I9b32708340c232369940738ade14cb6cbb02b331 --- oslo_messaging/serializer.py | 28 ++++++++- oslo_messaging/tests/test_serializer.py | 75 +++++++++++++++++++++++++ requirements.txt | 1 + 3 files changed, 103 insertions(+), 1 deletion(-) create mode 100644 oslo_messaging/tests/test_serializer.py diff --git a/oslo_messaging/serializer.py b/oslo_messaging/serializer.py index 78bf983b6..8b7c0a7a3 100644 --- a/oslo_messaging/serializer.py +++ b/oslo_messaging/serializer.py @@ -12,12 +12,15 @@ # License for the specific language governing permissions and limitations # under the License. -__all__ = ['Serializer', 'NoOpSerializer', 'JsonPayloadSerializer'] +__all__ = ['Serializer', 'NoOpSerializer', 'JsonPayloadSerializer', + 'RequestContextSerializer'] """Provides the definition of a message serialization handler""" import abc +from debtcollector import removals +from oslo_context import context as common_context from oslo_serialization import jsonutils import six @@ -61,6 +64,29 @@ class Serializer(object): """ +@removals.remove(version="2.9", removal_version="3.0") +class RequestContextSerializer(Serializer): + + def __init__(self, base): + self._base = base + + def serialize_entity(self, context, entity): + if not self._base: + return entity + return self._base.serialize_entity(context, entity) + + def deserialize_entity(self, context, entity): + if not self._base: + return entity + return self._base.deserialize_entity(context, entity) + + def serialize_context(self, context): + return context.to_dict() + + def deserialize_context(self, context): + return common_context.RequestContext.from_dict(context) + + class NoOpSerializer(Serializer): """A serializer that does nothing.""" diff --git a/oslo_messaging/tests/test_serializer.py b/oslo_messaging/tests/test_serializer.py new file mode 100644 index 000000000..329d9de71 --- /dev/null +++ b/oslo_messaging/tests/test_serializer.py @@ -0,0 +1,75 @@ +# Copyright 2015 Mirantis Inc. +# +# 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 oslo_context import context as common_context +from six.moves import mock + +from oslo_messaging import serializer +from oslo_messaging.tests import utils as test_utils + + +class TestRequestContextSerializer(test_utils.BaseTestCase): + + def setUp(self): + super(TestRequestContextSerializer, self).setUp() + + self.serializer = serializer.RequestContextSerializer(mock.MagicMock()) + self.context = common_context.RequestContext() + self.entity = {'foo': 'bar'} + + def test_serialize_entity(self): + self.serializer.serialize_entity(self.context, self.entity) + self.serializer._base.serialize_entity.assert_called_with( + self.context, self.entity) + + def test_serialize_entity_empty_base(self): + # NOTE(viktors): Return False for check `if self.serializer._base:` + bool_args = {'__bool__': lambda *args: False, + '__nonzero__': lambda *args: False} + self.serializer._base.configure_mock(**bool_args) + + entity = self.serializer.serialize_entity(self.context, self.entity) + self.assertFalse(self.serializer._base.serialize_entity.called) + self.assertEqual(entity, self.entity) + + def test_deserialize_entity(self): + self.serializer.deserialize_entity(self.context, self.entity) + self.serializer._base.deserialize_entity.assert_called_with( + self.context, self.entity) + + def test_deserialize_entity_empty_base(self): + # NOTE(viktors): Return False for check `if self.serializer._base:` + bool_args = {'__bool__': lambda *args: False, + '__nonzero__': lambda *args: False} + self.serializer._base.configure_mock(**bool_args) + + entity = self.serializer.deserialize_entity(self.context, self.entity) + self.assertFalse(self.serializer._base.serialize_entity.called) + self.assertEqual(entity, self.entity) + + def test_serialize_context(self): + new_context = self.serializer.serialize_context(self.context) + + self.assertEqual(new_context, self.context.to_dict()) + + @mock.patch.object(common_context.RequestContext, 'from_dict', + return_value='foobar') + def test_deserialize_context(self, mock_to_dict): + new_context = self.serializer.deserialize_context(self.context) + + mock_to_dict.assert_called_with(self.context) + self.assertEqual( + new_context, + common_context.RequestContext.from_dict(self.context) + ) diff --git a/requirements.txt b/requirements.txt index c93f35eda..f7a5c87f3 100644 --- a/requirements.txt +++ b/requirements.txt @@ -6,6 +6,7 @@ pbr>=1.6 futurist>=0.1.2 # Apache-2.0 oslo.config>=2.7.0 # Apache-2.0 +oslo.context>=0.2.0 # Apache-2.0 oslo.log>=1.12.0 # Apache-2.0 oslo.utils>=2.8.0 # Apache-2.0 oslo.serialization>=1.10.0 # Apache-2.0 From 6ae46796a61fc97467450b5bdd51dc6a0c86f9f4 Mon Sep 17 00:00:00 2001 From: Dmitry Mescheryakov Date: Mon, 23 Nov 2015 17:27:24 +0300 Subject: [PATCH 47/74] Use round robin failover strategy for Kombu driver Shuffle strategy we use right now leads to increased reconnection time and provides no benefit. Sometimes it might lead to RPC operations timeout because the strategy provides no guarantee on how long the reconnection process will take. See the referenced bug for details. On the other side, round-robin strategy provides least achievable reconnection time. It also provides guarantee that if K of N RabbitMQ hosts are alive, it will take at most N - K + 1 attempts to successfully reconnect to RabbitMQ cluster. With shuffle strategy during failover clients connect to random hosts and so the load is distributed evenly between alive RabbitMQs. But since we shuffle list of hosts before providing it to Kombu, load will be distributed evenly with round-robin strategy as well. DocImpact A new configuration option kombu_failover_strategy for Kombu driver is added. It determines how the next RabbitMQ node is chosen in case the one we are currently connected to becomes unavailable. It takes effect only if more than one RabbitMQ node is provided in config. Available options are: * round-robin: each RabbitMQ host in the list is tried in cycle until oslo.messaging successfully connects. Since oslo.messaging shuffles list of RabbitMQ hosts, the order of hosts in the cycle will be random and will not depend on order provided in config. * shuffle: oslo.messaging selects a random host from the list and tries to connect to it. If connection fails, oslo.messaging repeats attempt to connect to another random host. Oslo.messaging stops once it successfully connects to a host. Note that in each iteration a host to connect is selected independently of previous iterations, i.e. it might happen that oslo.messaging will try to connect to the same host several times in a row. The option's default value is round-robin. Before the option was introduced, the default strategy was shuffle. For the reasoning, see the main body of the commit message and the referenced bug. Closes-Bug: #1519851 Change-Id: I9a510c86bd5a6ce8b707734385af1a83de82804e --- oslo_messaging/_drivers/impl_rabbit.py | 10 +++++++++- oslo_messaging/tests/drivers/test_impl_rabbit.py | 2 +- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/oslo_messaging/_drivers/impl_rabbit.py b/oslo_messaging/_drivers/impl_rabbit.py index be41cc0f5..892961c42 100644 --- a/oslo_messaging/_drivers/impl_rabbit.py +++ b/oslo_messaging/_drivers/impl_rabbit.py @@ -84,6 +84,13 @@ rabbit_opts = [ help='How long to wait before considering a reconnect ' 'attempt to have failed. This value should not be ' 'longer than rpc_response_timeout.'), + cfg.StrOpt('kombu_failover_strategy', + choices=('round-robin', 'shuffle'), + default='round-robin', + help='Determines how the next RabbitMQ node is chosen in case ' + 'the one we are currently connected to becomes ' + 'unavailable. Takes effect only if more than one ' + 'RabbitMQ node is provided in config.'), cfg.StrOpt('rabbit_host', default='localhost', deprecated_group='DEFAULT', @@ -377,6 +384,7 @@ class Connection(object): self.amqp_auto_delete = driver_conf.amqp_auto_delete self.rabbit_use_ssl = driver_conf.rabbit_use_ssl self.kombu_reconnect_timeout = driver_conf.kombu_reconnect_timeout + self.kombu_failover_strategy = driver_conf.kombu_failover_strategy if self.rabbit_use_ssl: self.kombu_ssl_version = driver_conf.kombu_ssl_version @@ -456,8 +464,8 @@ class Connection(object): self.connection = kombu.connection.Connection( self._url, ssl=self._fetch_ssl_params(), login_method=self.login_method, - failover_strategy="shuffle", heartbeat=self.heartbeat_timeout_threshold, + failover_strategy=self.kombu_failover_strategy, transport_options={ 'confirm_publish': True, 'on_blocked': self._on_connection_blocked, diff --git a/oslo_messaging/tests/drivers/test_impl_rabbit.py b/oslo_messaging/tests/drivers/test_impl_rabbit.py index 06c78982a..db1a8c3df 100644 --- a/oslo_messaging/tests/drivers/test_impl_rabbit.py +++ b/oslo_messaging/tests/drivers/test_impl_rabbit.py @@ -169,7 +169,7 @@ class TestRabbitDriverLoadSSL(test_utils.BaseTestCase): 'on_blocked': mock.ANY, 'on_unblocked': mock.ANY}, ssl=self.expected, login_method='AMQPLAIN', - heartbeat=60, failover_strategy="shuffle") + heartbeat=60, failover_strategy='round-robin') class TestRabbitPublisher(test_utils.BaseTestCase): From 357dcb75abdfe1fc78e034d1913f478357cde18f Mon Sep 17 00:00:00 2001 From: Davanum Srinivas Date: Sun, 29 Nov 2015 18:26:32 -0500 Subject: [PATCH 48/74] Move ConnectionPool and ConnectionContext outside amqp.py ConnectionPool and ConnectionContext can be used by other drivers (like Kafka) and hence should be outside of amqp.py. * Moving ConnectionPool to pool.py * Moving ConnectionContext to common.py * Moving a couple of global variables to common.py No other logic changes, just refactoring Change-Id: I85154509a361690426772ef116590d38a965ca8d --- oslo_messaging/_drivers/amqp.py | 117 ------------------ oslo_messaging/_drivers/amqpdriver.py | 16 +-- oslo_messaging/_drivers/common.py | 96 ++++++++++++++ oslo_messaging/_drivers/impl_rabbit.py | 7 +- oslo_messaging/_drivers/pool.py | 26 ++++ .../tests/drivers/test_impl_rabbit.py | 12 +- 6 files changed, 140 insertions(+), 134 deletions(-) diff --git a/oslo_messaging/_drivers/amqp.py b/oslo_messaging/_drivers/amqp.py index 2308b80d1..86f41adde 100644 --- a/oslo_messaging/_drivers/amqp.py +++ b/oslo_messaging/_drivers/amqp.py @@ -31,7 +31,6 @@ from oslo_config import cfg import six from oslo_messaging._drivers import common as rpc_common -from oslo_messaging._drivers import pool deprecated_durable_opts = [ cfg.DeprecatedOpt('amqp_durable_queues', @@ -66,122 +65,6 @@ amqp_opts = [ UNIQUE_ID = '_unique_id' LOG = logging.getLogger(__name__) -# NOTE(sileht): Even if rabbit has only one Connection class, -# this connection can be used for two purposes: -# * wait and receive amqp messages (only do read stuffs on the socket) -# * send messages to the broker (only do write stuffs on the socket) -# The code inside a connection class is not concurrency safe. -# Using one Connection class instance for doing both, will result -# of eventlet complaining of multiple greenthreads that read/write the -# same fd concurrently... because 'send' and 'listen' run in different -# greenthread. -# So, a connection cannot be shared between thread/greenthread and -# this two variables permit to define the purpose of the connection -# to allow drivers to add special handling if needed (like heatbeat). -# amqp drivers create 3 kind of connections: -# * driver.listen*(): each call create a new 'PURPOSE_LISTEN' connection -# * driver.send*(): a pool of 'PURPOSE_SEND' connections is used -# * driver internally have another 'PURPOSE_LISTEN' connection dedicated -# to wait replies of rpc call -PURPOSE_LISTEN = 'listen' -PURPOSE_SEND = 'send' - - -class ConnectionPool(pool.Pool): - """Class that implements a Pool of Connections.""" - def __init__(self, conf, rpc_conn_pool_size, url, connection_cls): - self.connection_cls = connection_cls - self.conf = conf - self.url = url - super(ConnectionPool, self).__init__(rpc_conn_pool_size) - self.reply_proxy = None - - # TODO(comstud): Timeout connections not used in a while - def create(self, purpose=None): - if purpose is None: - purpose = PURPOSE_SEND - LOG.debug('Pool creating new connection') - return self.connection_cls(self.conf, self.url, purpose) - - def empty(self): - for item in self.iter_free(): - item.close() - - -class ConnectionContext(rpc_common.Connection): - """The class that is actually returned to the create_connection() caller. - - This is essentially a wrapper around Connection that supports 'with'. - It can also return a new Connection, or one from a pool. - - The function will also catch when an instance of this class is to be - deleted. With that we can return Connections to the pool on exceptions - and so forth without making the caller be responsible for catching them. - If possible the function makes sure to return a connection to the pool. - """ - - def __init__(self, connection_pool, purpose): - """Create a new connection, or get one from the pool.""" - self.connection = None - self.connection_pool = connection_pool - pooled = purpose == PURPOSE_SEND - if pooled: - self.connection = connection_pool.get() - else: - # a non-pooled connection is requested, so create a new connection - self.connection = connection_pool.create(purpose) - self.pooled = pooled - self.connection.pooled = pooled - - def __enter__(self): - """When with ConnectionContext() is used, return self.""" - return self - - def _done(self): - """If the connection came from a pool, clean it up and put it back. - If it did not come from a pool, close it. - """ - if self.connection: - if self.pooled: - # Reset the connection so it's ready for the next caller - # to grab from the pool - try: - self.connection.reset() - except Exception: - LOG.exception("Fail to reset the connection, drop it") - try: - self.connection.close() - except Exception: - pass - self.connection = self.connection_pool.create() - finally: - self.connection_pool.put(self.connection) - else: - try: - self.connection.close() - except Exception: - pass - self.connection = None - - def __exit__(self, exc_type, exc_value, tb): - """End of 'with' statement. We're done here.""" - self._done() - - def __del__(self): - """Caller is done with this connection. Make sure we cleaned up.""" - self._done() - - def close(self): - """Caller is done with this connection.""" - self._done() - - def __getattr__(self, key): - """Proxy all other calls to the Connection instance.""" - if self.connection: - return getattr(self.connection, key) - else: - raise rpc_common.InvalidRPCConnectionReuse() - class RpcContext(rpc_common.CommonRpcContext): """Context that supports replying to a rpc.call.""" diff --git a/oslo_messaging/_drivers/amqpdriver.py b/oslo_messaging/_drivers/amqpdriver.py index d3405086a..420587c45 100644 --- a/oslo_messaging/_drivers/amqpdriver.py +++ b/oslo_messaging/_drivers/amqpdriver.py @@ -100,7 +100,7 @@ class AMQPIncomingMessage(base.IncomingMessage): return with self.listener.driver._get_connection( - rpc_amqp.PURPOSE_SEND) as conn: + rpc_common.PURPOSE_SEND) as conn: if self.listener.driver.send_single_reply: self._send_reply(conn, reply, failure, log_failure=log_failure, ending=True) @@ -366,9 +366,9 @@ class AMQPDriverBase(base.BaseDriver): def _get_exchange(self, target): return target.exchange or self._default_exchange - def _get_connection(self, purpose=rpc_amqp.PURPOSE_SEND): - return rpc_amqp.ConnectionContext(self._connection_pool, - purpose=purpose) + def _get_connection(self, purpose=rpc_common.PURPOSE_SEND): + return rpc_common.ConnectionContext(self._connection_pool, + purpose=purpose) def _get_reply_q(self): with self._reply_q_lock: @@ -377,7 +377,7 @@ class AMQPDriverBase(base.BaseDriver): reply_q = 'reply_' + uuid.uuid4().hex - conn = self._get_connection(rpc_amqp.PURPOSE_LISTEN) + conn = self._get_connection(rpc_common.PURPOSE_LISTEN) self._waiter = ReplyWaiter(reply_q, conn, self._allowed_remote_exmods) @@ -422,7 +422,7 @@ class AMQPDriverBase(base.BaseDriver): log_msg = "CAST unique_id: %s " % unique_id try: - with self._get_connection(rpc_amqp.PURPOSE_SEND) as conn: + with self._get_connection(rpc_common.PURPOSE_SEND) as conn: if notify: exchange = self._get_exchange(target) log_msg += "NOTIFY exchange '%(exchange)s'" \ @@ -468,7 +468,7 @@ class AMQPDriverBase(base.BaseDriver): envelope=(version == 2.0), notify=True, retry=retry) def listen(self, target): - conn = self._get_connection(rpc_amqp.PURPOSE_LISTEN) + conn = self._get_connection(rpc_common.PURPOSE_LISTEN) listener = AMQPListener(self, conn) @@ -484,7 +484,7 @@ class AMQPDriverBase(base.BaseDriver): return listener def listen_for_notifications(self, targets_and_priorities, pool): - conn = self._get_connection(rpc_amqp.PURPOSE_LISTEN) + conn = self._get_connection(rpc_common.PURPOSE_LISTEN) listener = AMQPListener(self, conn) for target, priority in targets_and_priorities: diff --git a/oslo_messaging/_drivers/common.py b/oslo_messaging/_drivers/common.py index 78bdd9239..02c04805d 100644 --- a/oslo_messaging/_drivers/common.py +++ b/oslo_messaging/_drivers/common.py @@ -348,3 +348,99 @@ class DecayingTimer(object): if left <= 0 and timeout_callback is not None: timeout_callback(*args, **kwargs) return left if maximum is None else min(left, maximum) + + +# NOTE(sileht): Even if rabbit has only one Connection class, +# this connection can be used for two purposes: +# * wait and receive amqp messages (only do read stuffs on the socket) +# * send messages to the broker (only do write stuffs on the socket) +# The code inside a connection class is not concurrency safe. +# Using one Connection class instance for doing both, will result +# of eventlet complaining of multiple greenthreads that read/write the +# same fd concurrently... because 'send' and 'listen' run in different +# greenthread. +# So, a connection cannot be shared between thread/greenthread and +# this two variables permit to define the purpose of the connection +# to allow drivers to add special handling if needed (like heatbeat). +# amqp drivers create 3 kind of connections: +# * driver.listen*(): each call create a new 'PURPOSE_LISTEN' connection +# * driver.send*(): a pool of 'PURPOSE_SEND' connections is used +# * driver internally have another 'PURPOSE_LISTEN' connection dedicated +# to wait replies of rpc call +PURPOSE_LISTEN = 'listen' +PURPOSE_SEND = 'send' + + +class ConnectionContext(Connection): + """The class that is actually returned to the create_connection() caller. + + This is essentially a wrapper around Connection that supports 'with'. + It can also return a new Connection, or one from a pool. + + The function will also catch when an instance of this class is to be + deleted. With that we can return Connections to the pool on exceptions + and so forth without making the caller be responsible for catching them. + If possible the function makes sure to return a connection to the pool. + """ + + def __init__(self, connection_pool, purpose): + """Create a new connection, or get one from the pool.""" + self.connection = None + self.connection_pool = connection_pool + pooled = purpose == PURPOSE_SEND + if pooled: + self.connection = connection_pool.get() + else: + # a non-pooled connection is requested, so create a new connection + self.connection = connection_pool.create(purpose) + self.pooled = pooled + self.connection.pooled = pooled + + def __enter__(self): + """When with ConnectionContext() is used, return self.""" + return self + + def _done(self): + """If the connection came from a pool, clean it up and put it back. + If it did not come from a pool, close it. + """ + if self.connection: + if self.pooled: + # Reset the connection so it's ready for the next caller + # to grab from the pool + try: + self.connection.reset() + except Exception: + LOG.exception("Fail to reset the connection, drop it") + try: + self.connection.close() + except Exception: + pass + self.connection = self.connection_pool.create() + finally: + self.connection_pool.put(self.connection) + else: + try: + self.connection.close() + except Exception: + pass + self.connection = None + + def __exit__(self, exc_type, exc_value, tb): + """End of 'with' statement. We're done here.""" + self._done() + + def __del__(self): + """Caller is done with this connection. Make sure we cleaned up.""" + self._done() + + def close(self): + """Caller is done with this connection.""" + self._done() + + def __getattr__(self, key): + """Proxy all other calls to the Connection instance.""" + if self.connection: + return getattr(self.connection, key) + else: + raise InvalidRPCConnectionReuse() diff --git a/oslo_messaging/_drivers/impl_rabbit.py b/oslo_messaging/_drivers/impl_rabbit.py index bfcc1c63a..69f413540 100644 --- a/oslo_messaging/_drivers/impl_rabbit.py +++ b/oslo_messaging/_drivers/impl_rabbit.py @@ -37,6 +37,7 @@ from oslo_messaging._drivers import amqp as rpc_amqp from oslo_messaging._drivers import amqpdriver from oslo_messaging._drivers import base from oslo_messaging._drivers import common as rpc_common +from oslo_messaging._drivers import pool from oslo_messaging._i18n import _ from oslo_messaging._i18n import _LE from oslo_messaging._i18n import _LI @@ -448,7 +449,7 @@ class Connection(object): # NOTE(sileht): if purpose is PURPOSE_LISTEN # we don't need the lock because we don't # have a heartbeat thread - if purpose == rpc_amqp.PURPOSE_SEND: + if purpose == rpc_common.PURPOSE_SEND: self._connection_lock = ConnectionLock() else: self._connection_lock = DummyConnectionLock() @@ -488,7 +489,7 @@ class Connection(object): # the consume code does the heartbeat stuff # we don't need a thread self._heartbeat_thread = None - if purpose == rpc_amqp.PURPOSE_SEND: + if purpose == rpc_common.PURPOSE_SEND: self._heartbeat_start() LOG.debug('Connected to AMQP server on %(hostname)s:%(port)s ' @@ -1151,7 +1152,7 @@ class RabbitDriver(amqpdriver.AMQPDriverBase): conf.register_opts(rpc_amqp.amqp_opts, group=opt_group) conf.register_opts(base.base_opts, group=opt_group) - connection_pool = rpc_amqp.ConnectionPool( + connection_pool = pool.ConnectionPool( conf, conf.oslo_messaging_rabbit.rpc_conn_pool_size, url, Connection) diff --git a/oslo_messaging/_drivers/pool.py b/oslo_messaging/_drivers/pool.py index e689d678a..699ce5c10 100644 --- a/oslo_messaging/_drivers/pool.py +++ b/oslo_messaging/_drivers/pool.py @@ -17,8 +17,13 @@ import abc import collections import threading +from oslo_log import log as logging import six +from oslo_messaging._drivers import common + +LOG = logging.getLogger(__name__) + @six.add_metaclass(abc.ABCMeta) class Pool(object): @@ -86,3 +91,24 @@ class Pool(object): @abc.abstractmethod def create(self): """Construct a new item.""" + + +class ConnectionPool(Pool): + """Class that implements a Pool of Connections.""" + def __init__(self, conf, rpc_conn_pool_size, url, connection_cls): + self.connection_cls = connection_cls + self.conf = conf + self.url = url + super(ConnectionPool, self).__init__(rpc_conn_pool_size) + self.reply_proxy = None + + # TODO(comstud): Timeout connections not used in a while + def create(self, purpose=None): + if purpose is None: + purpose = common.PURPOSE_SEND + LOG.debug('Pool creating new connection') + return self.connection_cls(self.conf, self.url, purpose) + + def empty(self): + for item in self.iter_free(): + item.close() diff --git a/oslo_messaging/tests/drivers/test_impl_rabbit.py b/oslo_messaging/tests/drivers/test_impl_rabbit.py index 06c78982a..441548d9a 100644 --- a/oslo_messaging/tests/drivers/test_impl_rabbit.py +++ b/oslo_messaging/tests/drivers/test_impl_rabbit.py @@ -177,7 +177,7 @@ class TestRabbitPublisher(test_utils.BaseTestCase): def test_send_with_timeout(self, fake_publish): transport = oslo_messaging.get_transport(self.conf, 'kombu+memory:////') - with transport._driver._get_connection(amqp.PURPOSE_SEND) as pool_conn: + with transport._driver._get_connection(driver_common.PURPOSE_SEND) as pool_conn: conn = pool_conn.connection conn._publish(mock.Mock(), 'msg', routing_key='routing_key', timeout=1) @@ -187,7 +187,7 @@ class TestRabbitPublisher(test_utils.BaseTestCase): def test_send_no_timeout(self, fake_publish): transport = oslo_messaging.get_transport(self.conf, 'kombu+memory:////') - with transport._driver._get_connection(amqp.PURPOSE_SEND) as pool_conn: + with transport._driver._get_connection(driver_common.PURPOSE_SEND) as pool_conn: conn = pool_conn.connection conn._publish(mock.Mock(), 'msg', routing_key='routing_key') fake_publish.assert_called_with('msg', expiration=None) @@ -207,7 +207,7 @@ class TestRabbitPublisher(test_utils.BaseTestCase): type='topic', passive=False) - with transport._driver._get_connection(amqp.PURPOSE_SEND) as pool_conn: + with transport._driver._get_connection(driver_common.PURPOSE_SEND) as pool_conn: conn = pool_conn.connection exc = conn.connection.channel_errors[0] @@ -240,7 +240,7 @@ class TestRabbitConsume(test_utils.BaseTestCase): 'kombu+memory:////') self.addCleanup(transport.cleanup) deadline = time.time() + 6 - with transport._driver._get_connection(amqp.PURPOSE_LISTEN) as conn: + with transport._driver._get_connection(driver_common.PURPOSE_LISTEN) as conn: self.assertRaises(driver_common.Timeout, conn.consume, timeout=3) @@ -259,7 +259,7 @@ class TestRabbitConsume(test_utils.BaseTestCase): transport = oslo_messaging.get_transport(self.conf, 'kombu+memory:////') self.addCleanup(transport.cleanup) - with transport._driver._get_connection(amqp.PURPOSE_LISTEN) as conn: + with transport._driver._get_connection(driver_common.PURPOSE_LISTEN) as conn: channel = conn.connection.channel with mock.patch('kombu.connection.Connection.connected', new_callable=mock.PropertyMock, @@ -902,7 +902,7 @@ class RpcKombuHATestCase(test_utils.BaseTestCase): # starting from the first broker in the list url = oslo_messaging.TransportURL.parse(self.conf, None) self.connection = rabbit_driver.Connection(self.conf, url, - amqp.PURPOSE_SEND) + driver_common.PURPOSE_SEND) self.addCleanup(self.connection.close) def test_ensure_four_retry(self): From 33c1010c3281804456a22b769c4bac5ac6a7cca1 Mon Sep 17 00:00:00 2001 From: Davanum Srinivas Date: Tue, 24 Nov 2015 19:56:16 -0500 Subject: [PATCH 49/74] Option group for notifications In change Ief6f95ea906bfd95b3218a930c9db5d8a764beb9, we decoupled RPC and Notifications a bit. We should take another step and separate out the options for notifications into its own group. Change-Id: Ib51e2839f9035d0cc0e3f459939d9f9003a8c810 --- doc/source/FAQ.rst | 9 ++-- oslo_messaging/conffixture.py | 4 +- oslo_messaging/notify/_impl_routing.py | 8 ++-- oslo_messaging/notify/log_handler.py | 3 +- oslo_messaging/notify/notifier.py | 43 ++++++++++++------- .../tests/functional/notify/test_logger.py | 5 ++- .../tests/notify/test_log_handler.py | 3 +- oslo_messaging/tests/notify/test_logger.py | 3 +- oslo_messaging/tests/notify/test_notifier.py | 22 ++++++---- tools/simulator.py | 4 +- 10 files changed, 65 insertions(+), 39 deletions(-) diff --git a/doc/source/FAQ.rst b/doc/source/FAQ.rst index ebb6fe3c2..2a67bba8c 100644 --- a/doc/source/FAQ.rst +++ b/doc/source/FAQ.rst @@ -6,7 +6,8 @@ I don't need notifications on the message bus. How do I disable them? ===================================================================== Notification messages can be disabled using the ``noop`` notify -driver. Set ``notification_driver = noop`` in your configuration file. +driver. Set ``driver = noop`` in your configuration file under the +[oslo_messaging_notifications] section. Why does the notification publisher create queues, too? Shouldn't the subscriber do that? ========================================================================================= @@ -26,9 +27,9 @@ notification "level". The default topic is ``notifications``, so an info-level notification is published to the topic ``notifications.info``. A subscriber queue of the same name is created automatically for each of these topics. To change the queue names, -change the notification topic using the ``notification_topics`` -configuration option. The option accepts a list of values, so it is -possible to publish to multiple topics. +change the notification topic using the ``topics`` +configuration option in ``[oslo_messaging_notifications]``. The option +accepts a list of values, so it is possible to publish to multiple topics. What are the other choices of notification drivers available? ============================================================= diff --git a/oslo_messaging/conffixture.py b/oslo_messaging/conffixture.py index 35b6e2a27..2f53d3f9e 100644 --- a/oslo_messaging/conffixture.py +++ b/oslo_messaging/conffixture.py @@ -66,7 +66,9 @@ class ConfFixture(fixtures.Fixture): _import_opts(self.conf, 'oslo_messaging.rpc.client', '_client_opts') _import_opts(self.conf, 'oslo_messaging.transport', '_transport_opts') _import_opts(self.conf, - 'oslo_messaging.notify.notifier', '_notifier_opts') + 'oslo_messaging.notify.notifier', + '_notifier_opts', + 'oslo_messaging_notifications') def setUp(self): super(ConfFixture, self).setUp() diff --git a/oslo_messaging/notify/_impl_routing.py b/oslo_messaging/notify/_impl_routing.py index 6c5fd18d2..3e89f8770 100644 --- a/oslo_messaging/notify/_impl_routing.py +++ b/oslo_messaging/notify/_impl_routing.py @@ -27,11 +27,13 @@ from oslo_messaging.notify import notifier LOG = logging.getLogger(__name__) -router_config = cfg.StrOpt('routing_notifier_config', default='', +router_config = cfg.StrOpt('routing_config', default='', + deprecated_group='DEFAULT', + deprecated_name='routing_notifier_config', help='RoutingNotifier configuration file location.') CONF = cfg.CONF -CONF.register_opt(router_config) +CONF.register_opt(router_config, group='oslo_messaging_notifications') class RoutingDriver(notifier.Driver): @@ -56,7 +58,7 @@ class RoutingDriver(notifier.Driver): """One-time load of notifier config file.""" self.routing_groups = {} self.used_drivers = set() - filename = CONF.routing_notifier_config + filename = CONF.oslo_messaging_notifications.routing_config if not filename: return diff --git a/oslo_messaging/notify/log_handler.py b/oslo_messaging/notify/log_handler.py index 7fefb9b29..8dc8454b5 100644 --- a/oslo_messaging/notify/log_handler.py +++ b/oslo_messaging/notify/log_handler.py @@ -27,8 +27,9 @@ class LoggingErrorNotificationHandler(logging.Handler): publisher_id='error.publisher') def emit(self, record): + conf = self._transport.conf # NOTE(bnemec): Notifier registers this opt with the transport. - if ('log' in self._transport.conf.notification_driver): + if ('log' in conf.oslo_messaging_notifications.driver): # NOTE(lbragstad): If we detect that log is one of the # notification drivers, then return. This protects from infinite # recursion where something bad happens, it gets logged, the log diff --git a/oslo_messaging/notify/notifier.py b/oslo_messaging/notify/notifier.py index dec25b16f..fb841644f 100644 --- a/oslo_messaging/notify/notifier.py +++ b/oslo_messaging/notify/notifier.py @@ -28,19 +28,27 @@ from oslo_messaging import serializer as msg_serializer from oslo_messaging import transport as msg_transport _notifier_opts = [ - cfg.MultiStrOpt('notification_driver', + cfg.MultiStrOpt('driver', default=[], + deprecated_name='notification_driver', + deprecated_group='DEFAULT', help='The Drivers(s) to handle sending notifications. ' 'Possible values are messaging, messagingv2, ' 'routing, log, test, noop'), - cfg.StrOpt('notification_transport_url', + cfg.StrOpt('transport_url', + deprecated_name='notification_transport_url', + deprecated_group='DEFAULT', help='A URL representing the messaging driver to use for ' 'notifications. If not set, we fall back to the same ' 'configuration used for RPC.'), - cfg.ListOpt('notification_topics', + cfg.ListOpt('topics', default=['notifications', ], - deprecated_name='topics', - deprecated_group='rpc_notifier2', + deprecated_opts=[ + cfg.DeprecatedOpt('topics', + group='rpc_notifier2'), + cfg.DeprecatedOpt('notification_topics', + group='DEFAULT') + ], help='AMQP topic used for OpenStack notifications.'), ] @@ -83,8 +91,9 @@ class Driver(object): def get_notification_transport(conf, url=None, allowed_remote_exmods=None, aliases=None): if url is None: - conf.register_opts(_notifier_opts) - url = conf.notification_transport_url + conf.register_opts(_notifier_opts, + group='oslo_messaging_notifications') + url = conf.oslo_messaging_notifications.transport_url return msg_transport.get_transport(conf, url, allowed_remote_exmods, aliases) @@ -111,9 +120,9 @@ class Notifier(object): notifier = messaging.Notifier(get_notification_transport(CONF), 'compute') - and notifications are sent via drivers chosen with the notification_driver - config option and on the topics chosen with the notification_topics config - option. + and notifications are sent via drivers chosen with the driver + config option and on the topics chosen with the topics config + option in [oslo_messaging_notifications] section. Alternatively, a Notifier object can be instantiated with a specific driver or topic:: @@ -154,24 +163,26 @@ class Notifier(object): N means N retries :type retry: int """ - transport.conf.register_opts(_notifier_opts) + conf = transport.conf + conf.register_opts(_notifier_opts, + group='oslo_messaging_notifications') self.transport = transport self.publisher_id = publisher_id self.retry = retry - self._driver_names = ([driver] if driver is not None - else transport.conf.notification_driver) + self._driver_names = ([driver] if driver is not None else + conf.oslo_messaging_notifications.driver) - self._topics = ([topic] if topic is not None - else transport.conf.notification_topics) + self._topics = ([topic] if topic is not None else + conf.oslo_messaging_notifications.topics) self._serializer = serializer or msg_serializer.NoOpSerializer() self._driver_mgr = named.NamedExtensionManager( 'oslo.messaging.notify.drivers', names=self._driver_names, invoke_on_load=True, - invoke_args=[transport.conf], + invoke_args=[conf], invoke_kwds={ 'topics': self._topics, 'transport': self.transport, diff --git a/oslo_messaging/tests/functional/notify/test_logger.py b/oslo_messaging/tests/functional/notify/test_logger.py index ad30e8850..a7f580bc3 100644 --- a/oslo_messaging/tests/functional/notify/test_logger.py +++ b/oslo_messaging/tests/functional/notify/test_logger.py @@ -51,8 +51,9 @@ class LoggingNotificationHandlerTestCase(utils.SkipIfNoTransportURL): # NOTE(gtt): Using different topic to make tests run in parallel topic = 'test_logging_%s_driver_%s' % (self.priority, self.driver) - self.conf.notification_driver = [self.driver] - self.conf.notification_topics = [topic] + self.config(driver=[self.driver], + topics=[topic], + group='oslo_messaging_notifications') listener = self.useFixture( utils.NotificationFixture(self.conf, self.url, [topic])) diff --git a/oslo_messaging/tests/notify/test_log_handler.py b/oslo_messaging/tests/notify/test_log_handler.py index dfc214aed..1851321dd 100644 --- a/oslo_messaging/tests/notify/test_log_handler.py +++ b/oslo_messaging/tests/notify/test_log_handler.py @@ -28,7 +28,8 @@ class PublishErrorsHandlerTestCase(test_utils.BaseTestCase): def test_emit_cfg_log_notifier_in_notifier_drivers(self): drivers = ['messaging', 'log'] - self.config(notification_driver=drivers) + self.config(driver=drivers, + group='oslo_messaging_notifications') self.stub_flg = True transport = test_notifier._FakeTransport(self.conf) diff --git a/oslo_messaging/tests/notify/test_logger.py b/oslo_messaging/tests/notify/test_logger.py index f0ea8e8b8..12d1cd100 100644 --- a/oslo_messaging/tests/notify/test_logger.py +++ b/oslo_messaging/tests/notify/test_logger.py @@ -49,7 +49,8 @@ class TestLogNotifier(test_utils.BaseTestCase): def setUp(self): super(TestLogNotifier, self).setUp() self.addCleanup(oslo_messaging.notify._impl_test.reset) - self.config(notification_driver=['test']) + self.config(driver=['test'], + group='oslo_messaging_notifications') # NOTE(jamespage) disable thread information logging for testing # as this causes test failures when zmq tests monkey_patch via # eventlet diff --git a/oslo_messaging/tests/notify/test_notifier.py b/oslo_messaging/tests/notify/test_notifier.py index e55913d51..0b2da7fbb 100644 --- a/oslo_messaging/tests/notify/test_notifier.py +++ b/oslo_messaging/tests/notify/test_notifier.py @@ -156,8 +156,9 @@ class TestMessagingNotifier(test_utils.BaseTestCase): if self.v2: drivers.append('messagingv2') - self.config(notification_driver=drivers, - notification_topics=self.topics) + self.config(driver=drivers, + topics=self.topics, + group='oslo_messaging_notifications') transport = _FakeTransport(self.conf) @@ -269,7 +270,8 @@ class TestLogNotifier(test_utils.BaseTestCase): @mock.patch('oslo_utils.timeutils.utcnow') def test_notifier(self, mock_utcnow): - self.config(notification_driver=['log']) + self.config(driver=['log'], + group='oslo_messaging_notifications') transport = _FakeTransport(self.conf) @@ -338,7 +340,8 @@ class TestLogNotifier(test_utils.BaseTestCase): class TestRoutingNotifier(test_utils.BaseTestCase): def setUp(self): super(TestRoutingNotifier, self).setUp() - self.config(notification_driver=['routing']) + self.config(driver=['routing'], + group='oslo_messaging_notifications') transport = _FakeTransport(self.conf) self.notifier = oslo_messaging.Notifier(transport) @@ -360,13 +363,14 @@ class TestRoutingNotifier(test_utils.BaseTestCase): self.assertTrue(self.router._should_load_plugin(ext)) def test_load_notifiers_no_config(self): - # default routing_notifier_config="" + # default routing_config="" self.router._load_notifiers() self.assertEqual({}, self.router.routing_groups) self.assertEqual(0, len(self.router.used_drivers)) def test_load_notifiers_no_extensions(self): - self.config(routing_notifier_config="routing_notifier.yaml") + self.config(routing_config="routing_notifier.yaml", + group='oslo_messaging_notifications') routing_config = r"" config_file = mock.MagicMock() config_file.return_value = routing_config @@ -382,7 +386,8 @@ class TestRoutingNotifier(test_utils.BaseTestCase): self.assertEqual({}, self.router.routing_groups) def test_load_notifiers_config(self): - self.config(routing_notifier_config="routing_notifier.yaml") + self.config(routing_config="routing_notifier.yaml", + group='oslo_messaging_notifications') routing_config = r""" group_1: rpc : foo @@ -519,7 +524,8 @@ group_1: sorted(pm.map.call_args[0][6])) def test_notify_filtered(self): - self.config(routing_notifier_config="routing_notifier.yaml") + self.config(routing_config="routing_notifier.yaml", + group='oslo_messaging_notifications') routing_config = r""" group_1: rpc: diff --git a/tools/simulator.py b/tools/simulator.py index d3dda3eec..3c587d5a0 100755 --- a/tools/simulator.py +++ b/tools/simulator.py @@ -283,8 +283,8 @@ def main(): # oslo.config defaults cfg.CONF.heartbeat_interval = 5 - cfg.CONF.notification_topics = "notif" - cfg.CONF.notification_driver = "messaging" + cfg.CONF.oslo_messaging_notifications.topics = "notif" + cfg.CONF.oslo_messaging_notifications.driver = "messaging" cfg.CONF.prog = os.path.basename(__file__) cfg.CONF.project = 'oslo.messaging' From 67c63031f5cb1a675686fc2648ce27f6e36ee254 Mon Sep 17 00:00:00 2001 From: Komei Shimamura Date: Fri, 5 Jun 2015 23:05:29 -0700 Subject: [PATCH 50/74] Add a driver for Apache Kafka Adding a driver for Apache Kafka connection, supporting notification via Kafka. This driver is experimental until having functional and integration tests Change-Id: I7a5d8e3259b21d5e29ed3b795d04952e1d13ad08 Implements: blueprint adding-kafka-support --- oslo_messaging/_drivers/impl_kafka.py | 363 ++++++++++++++++++ .../tests/drivers/test_impl_kafka.py | 288 ++++++++++++++ setup.cfg | 3 + test-requirements.txt | 3 + 4 files changed, 657 insertions(+) create mode 100644 oslo_messaging/_drivers/impl_kafka.py create mode 100644 oslo_messaging/tests/drivers/test_impl_kafka.py diff --git a/oslo_messaging/_drivers/impl_kafka.py b/oslo_messaging/_drivers/impl_kafka.py new file mode 100644 index 000000000..9be417bdd --- /dev/null +++ b/oslo_messaging/_drivers/impl_kafka.py @@ -0,0 +1,363 @@ +# Copyright (C) 2015 Cisco Systems, Inc. +# +# 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 + +from oslo_messaging._drivers import base +from oslo_messaging._drivers import common as driver_common +from oslo_messaging._drivers import pool as driver_pool +from oslo_messaging._i18n import _LE +from oslo_messaging._i18n import _LW +from oslo_serialization import jsonutils + +import kafka +from kafka.common import KafkaError +from oslo_config import cfg +from oslo_log import log as logging + + +LOG = logging.getLogger(__name__) + +PURPOSE_SEND = 'send' +PURPOSE_LISTEN = 'listen' + +kafka_opts = [ + cfg.StrOpt('kafka_default_host', default='localhost', + help='Default Kafka broker Host'), + + cfg.IntOpt('kafka_default_port', default=9092, + help='Default Kafka broker Port'), + + cfg.IntOpt('kafka_max_fetch_bytes', default=1024 * 1024, + help='Max fetch bytes of Kafka consumer'), + + cfg.IntOpt('kafka_consumer_timeout', default=1.0, + help='Default timeout(s) for Kafka consumers'), + + cfg.IntOpt('pool_size', default=10, + help='Pool Size for Kafka Consumers'), +] + +CONF = cfg.CONF + + +def pack_context_with_message(ctxt, msg): + """Pack context into msg.""" + if isinstance(ctxt, dict): + context_d = ctxt + else: + context_d = ctxt.to_dict() + + return {'message': msg, 'context': context_d} + + +def target_to_topic(target): + """Convert target into topic string + + :param target: Message destination target + :type target: oslo_messaging.Target + """ + if target.exchange is None: + return target.topic + return "%s_%s" % (target.exchange, target.topic) + + +class Connection(object): + + def __init__(self, conf, url, purpose): + + driver_conf = conf.oslo_messaging_kafka + + self.conf = conf + self.kafka_client = None + self.producer = None + self.consumer = None + self.fetch_messages_max_bytes = driver_conf.kafka_max_fetch_bytes + self.consumer_timeout = float(driver_conf.kafka_consumer_timeout) + self.url = url + self._parse_url() + # TODO(Support for manual/auto_commit functionality) + # When auto_commit is False, consumer can manually notify + # the completion of the subscription. + # Currently we don't support for non auto commit option + self.auto_commit = True + self._consume_loop_stopped = False + + def _parse_url(self): + driver_conf = self.conf.oslo_messaging_kafka + try: + self.host = self.url.hosts[0].hostname + except (NameError, IndexError): + self.host = driver_conf.kafka_default_host + + try: + self.port = self.url.hosts[0].port + except (NameError, IndexError): + self.port = driver_conf.kafka_default_port + + if self.host is None: + self.host = driver_conf.kafka_default_host + + if self.port is None: + self.port = driver_conf.kafka_default_port + + def notify_send(self, topic, ctxt, msg, retry): + """Send messages to Kafka broker. + + :param topic: String of the topic + :param ctxt: context for the messages + :param msg: messages for publishing + :param retry: the number of retry + """ + message = pack_context_with_message(ctxt, msg) + self._ensure_connection() + self._send_and_retry(message, topic, retry) + + def _send_and_retry(self, message, topic, retry): + current_retry = 0 + if not isinstance(message, str): + message = jsonutils.dumps(message) + while message is not None: + try: + self._send(message, topic) + message = None + except Exception: + LOG.warn(_LW("Failed to publish a message of topic %s"), topic) + current_retry += 1 + if retry is not None and current_retry >= retry: + LOG.exception(_LE("Failed to retry to send data " + "with max retry times")) + message = None + + def _send(self, message, topic): + self.producer.send_messages(topic, message) + + def consume(self, timeout=None): + """recieve messages as many as max_fetch_messages. + + In this functions, there are no while loop to subscribe. + This would be helpful when we wants to control the velocity of + subscription. + """ + duration = (self.consumer_timeout if timeout is None else timeout) + timer = driver_common.DecayingTimer(duration=duration) + timer.start() + + def _raise_timeout(): + LOG.debug('Timed out waiting for Kafka response') + raise driver_common.Timeout() + + poll_timeout = (self.consumer_timeout if timeout is None + else min(timeout, self.consumer_timeout)) + + while True: + if self._consume_loop_stopped: + return + try: + next_timeout = poll_timeout * 1000.0 + # TODO(use configure() method instead) + # Currently KafkaConsumer does not support for + # the case of updating only fetch_max_wait_ms parameter + self.consumer._config['fetch_max_wait_ms'] = next_timeout + messages = list(self.consumer.fetch_messages()) + except Exception as e: + LOG.exception(_LE("Failed to consume messages: %s"), e) + messages = None + + if not messages: + poll_timeout = timer.check_return( + _raise_timeout, maximum=self.consumer_timeout) + continue + + return messages + + def stop_consuming(self): + self._consume_loop_stopped = True + + def reset(self): + """Reset a connection so it can be used again.""" + if self.kafka_client: + self.kafka_client.close() + self.kafka_client = None + if self.producer: + self.producer.stop() + self.producer = None + self.consumer = None + + def close(self): + if self.kafka_client: + self.kafka_client.close() + self.kafka_client = None + if self.producer: + self.producer.stop() + self.consumer = None + + def commit(self): + """Commit is used by subscribers belonging to the same group. + After subscribing messages, commit is called to prevent + the other subscribers which belong to the same group + from re-subscribing the same messages. + + Currently self.auto_commit option is always True, + so we don't need to call this function. + """ + self.consumer.commit() + + def _ensure_connection(self): + if self.kafka_client: + return + try: + self.kafka_client = kafka.KafkaClient( + "%s:%s" % (self.host, str(self.port))) + self.producer = kafka.SimpleProducer(self.kafka_client) + except KafkaError as e: + LOG.exception(_LE("Kafka Connection is not available: %s"), e) + self.kafka_client = None + + def declare_topic_consumer(self, topics, group=None): + self.consumer = kafka.KafkaConsumer( + *topics, group_id=group, + metadata_broker_list=["%s:%s" % (self.host, str(self.port))], + # auto_commit_enable=self.auto_commit, + fetch_message_max_bytes=self.fetch_messages_max_bytes) + + +class OsloKafkaMessage(base.IncomingMessage): + + def __init__(self, listener, ctxt, message): + super(OsloKafkaMessage, self).__init__(listener, ctxt, message) + + def requeue(self): + LOG.warn(_LW("requeue is not supported")) + + def reply(self, reply=None, failure=None, log_failure=True): + LOG.warn(_LW("reply is not supported")) + + +class KafkaListener(base.Listener): + + def __init__(self, driver, conn): + super(KafkaListener, self).__init__(driver) + self._stopped = threading.Event() + self.conn = conn + self.incoming_queue = [] + + def poll(self, timeout=None): + while not self._stopped.is_set(): + if self.incoming_queue: + return self.incoming_queue.pop(0) + try: + messages = self.conn.consume(timeout=timeout) + for msg in messages: + message = msg.value + message = jsonutils.loads(message) + self.incoming_queue.append(OsloKafkaMessage( + listener=self, ctxt=message['context'], + message=message['message'])) + except driver_common.Timeout: + return None + + def stop(self): + self._stopped.set() + self.conn.stop_consuming() + + def cleanup(self): + self.conn.close() + + def commit(self): + # TODO(Support for manually/auto commit functionality) + # It's better to allow users to commit manually and support for + # self.auto_commit = False option. For now, this commit function + # is meaningless since user couldn't call this function and + # auto_commit option is always True. + self.conn.commit() + + +class KafkaDriver(base.BaseDriver): + """Note: Current implementation of this driver is experimental. + We will have functional and/or integrated testing enabled for this driver. + """ + + def __init__(self, conf, url, default_exchange=None, + allowed_remote_exmods=None): + + opt_group = cfg.OptGroup(name='oslo_messaging_kafka', + title='Kafka driver options') + conf.register_group(opt_group) + conf.register_opts(kafka_opts, group=opt_group) + + super(KafkaDriver, self).__init__( + conf, url, default_exchange, allowed_remote_exmods) + + self.connection_pool = driver_pool.ConnectionPool( + self.conf, self.conf.oslo_messaging_kafka.pool_size, + self._url, Connection) + self.listeners = [] + + def cleanup(self): + for c in self.listeners: + c.close() + self.listeners = [] + + def send(self, target, ctxt, message, wait_for_reply=None, timeout=None, + retry=None): + raise NotImplementedError( + 'The RPC implementation for Kafka is not implemented') + + def send_notification(self, target, ctxt, message, version, retry=None): + """Send notification to Kafka brokers + + :param target: Message destination target + :type target: oslo_messaging.Target + :param ctxt: Message context + :type ctxt: dict + :param message: Message payload to pass + :type message: dict + :param version: Messaging API version (currently not used) + :type version: str + :param retry: an optional default kafka consumer retries configuration + None means to retry forever + 0 means no retry + N means N retries + :type retry: int + """ + with self._get_connection(purpose=PURPOSE_SEND) as conn: + conn.notify_send(target_to_topic(target), ctxt, message, retry) + + def listen(self, target): + raise NotImplementedError( + 'The RPC implementation for Kafka is not implemented') + + def listen_for_notifications(self, targets_and_priorities, pool=None): + """Listen to a specified list of targets on Kafka brokers + + :param targets_and_priorities: List of pairs (target, priority) + priority is not used for kafka driver + target.exchange_target.topic is used as + a kafka topic + :type targets_and_priorities: list + :param pool: consumer group of Kafka consumers + :type pool: string + """ + conn = self._get_connection(purpose=PURPOSE_LISTEN) + topics = [] + for target, priority in targets_and_priorities: + topics.append(target_to_topic(target)) + + conn.declare_topic_consumer(topics, pool) + + listener = KafkaListener(self, conn) + return listener + + def _get_connection(self, purpose): + return driver_common.ConnectionContext(self.connection_pool, purpose) diff --git a/oslo_messaging/tests/drivers/test_impl_kafka.py b/oslo_messaging/tests/drivers/test_impl_kafka.py new file mode 100644 index 000000000..6f25b2c64 --- /dev/null +++ b/oslo_messaging/tests/drivers/test_impl_kafka.py @@ -0,0 +1,288 @@ +# Copyright (C) 2015 Cisco Systems, Inc. +# +# 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 json +import kafka +from kafka.common import KafkaError +import mock +import testscenarios +from testtools.testcase import unittest +import time + +import oslo_messaging +from oslo_messaging._drivers import common as driver_common +from oslo_messaging._drivers import impl_kafka as kafka_driver +from oslo_messaging.tests import utils as test_utils + +load_tests = testscenarios.load_tests_apply_scenarios + +KAFKA_BROKER = 'localhost:9092' +KAFKA_BROKER_URL = 'kafka://localhost:9092' + + +def _is_kafka_service_running(): + """Checks whether the Kafka service is running or not""" + kafka_running = True + try: + broker = KAFKA_BROKER + kafka.KafkaClient(broker) + except KafkaError: + # Kafka service is not running. + kafka_running = False + return kafka_running + + +class TestKafkaDriverLoad(test_utils.BaseTestCase): + + def setUp(self): + super(TestKafkaDriverLoad, self).setUp() + self.messaging_conf.transport_driver = 'kafka' + + def test_driver_load(self): + transport = oslo_messaging.get_transport(self.conf) + self.assertIsInstance(transport._driver, kafka_driver.KafkaDriver) + + +class TestKafkaTransportURL(test_utils.BaseTestCase): + + scenarios = [ + ('none', dict(url=None, + expected=[dict(host='localhost', port=9092)])), + ('empty', dict(url='kafka:///', + expected=[dict(host='localhost', port=9092)])), + ('host', dict(url='kafka://127.0.0.1', + expected=[dict(host='127.0.0.1', port=9092)])), + ('port', dict(url='kafka://localhost:1234', + expected=[dict(host='localhost', port=1234)])), + ] + + def setUp(self): + super(TestKafkaTransportURL, self).setUp() + self.messaging_conf.transport_driver = 'kafka' + + def test_transport_url(self): + transport = oslo_messaging.get_transport(self.conf, self.url) + self.addCleanup(transport.cleanup) + driver = transport._driver + + conn = driver._get_connection(kafka_driver.PURPOSE_SEND) + self.assertEqual(self.expected[0]['host'], conn.host) + self.assertEqual(self.expected[0]['port'], conn.port) + + +class TestKafkaDriver(test_utils.BaseTestCase): + """Unit Test cases to test the kafka driver + """ + + def setUp(self): + super(TestKafkaDriver, self).setUp() + self.messaging_conf.transport_driver = 'kafka' + transport = oslo_messaging.get_transport(self.conf) + self.driver = transport._driver + + def test_send(self): + target = oslo_messaging.Target(topic="topic_test") + self.assertRaises(NotImplementedError, + self.driver.send, target, {}, {}) + + def test_send_notification(self): + target = oslo_messaging.Target(topic="topic_test") + + with mock.patch.object( + kafka_driver.Connection, 'notify_send') as fake_send: + self.driver.send_notification(target, {}, {}, None) + self.assertEquals(1, len(fake_send.mock_calls)) + + def test_listen(self): + target = oslo_messaging.Target(topic="topic_test") + self.assertRaises(NotImplementedError, self.driver.listen, target) + + +class TestKafkaConnection(test_utils.BaseTestCase): + + def setUp(self): + super(TestKafkaConnection, self).setUp() + self.messaging_conf.transport_driver = 'kafka' + transport = oslo_messaging.get_transport(self.conf) + self.driver = transport._driver + + @mock.patch.object(kafka_driver.Connection, '_ensure_connection') + @mock.patch.object(kafka_driver.Connection, '_send') + def test_notify(self, fake_send, fake_ensure_connection): + conn = self.driver._get_connection(kafka_driver.PURPOSE_SEND) + conn.notify_send("fake_topic", {"fake_ctxt": "fake_param"}, + {"fake_text": "fake_message_1"}, 10) + self.assertEqual(1, len(fake_send.mock_calls)) + + @mock.patch.object(kafka_driver.Connection, '_ensure_connection') + @mock.patch.object(kafka_driver.Connection, '_send') + def test_notify_with_retry(self, fake_send, fake_ensure_connection): + conn = self.driver._get_connection(kafka_driver.PURPOSE_SEND) + fake_send.side_effect = KafkaError("fake_exception") + conn.notify_send("fake_topic", {"fake_ctxt": "fake_param"}, + {"fake_text": "fake_message_2"}, 10) + self.assertEqual(10, len(fake_send.mock_calls)) + + @mock.patch.object(kafka_driver.Connection, '_ensure_connection') + @mock.patch.object(kafka_driver.Connection, '_parse_url') + def test_consume(self, fake_parse_url, fake_ensure_connection): + fake_message = { + "context": {"fake": "fake_context_1"}, + "message": {"fake": "fake_message_1"}} + + conn = kafka_driver.Connection( + self.conf, '', kafka_driver.PURPOSE_LISTEN) + + conn.consumer = mock.MagicMock() + conn.consumer.fetch_messages = mock.MagicMock( + return_value=iter([json.dumps(fake_message)])) + + self.assertEqual(fake_message, json.loads(conn.consume()[0])) + self.assertEqual(1, len(conn.consumer.fetch_messages.mock_calls)) + + @mock.patch.object(kafka_driver.Connection, '_ensure_connection') + @mock.patch.object(kafka_driver.Connection, '_parse_url') + def test_consume_timeout(self, fake_parse_url, fake_ensure_connection): + deadline = time.time() + 3 + conn = kafka_driver.Connection( + self.conf, '', kafka_driver.PURPOSE_LISTEN) + + conn.consumer = mock.MagicMock() + conn.consumer.fetch_messages = mock.MagicMock(return_value=iter([])) + + self.assertRaises(driver_common.Timeout, conn.consume, timeout=3) + self.assertEqual(0, int(deadline - time.time())) + + @mock.patch.object(kafka_driver.Connection, '_ensure_connection') + @mock.patch.object(kafka_driver.Connection, '_parse_url') + def test_consume_with_default_timeout( + self, fake_parse_url, fake_ensure_connection): + deadline = time.time() + 1 + conn = kafka_driver.Connection( + self.conf, '', kafka_driver.PURPOSE_LISTEN) + + conn.consumer = mock.MagicMock() + conn.consumer.fetch_messages = mock.MagicMock(return_value=iter([])) + + self.assertRaises(driver_common.Timeout, conn.consume) + self.assertEqual(0, int(deadline - time.time())) + + @mock.patch.object(kafka_driver.Connection, '_ensure_connection') + @mock.patch.object(kafka_driver.Connection, '_parse_url') + def test_consume_timeout_without_consumers( + self, fake_parse_url, fake_ensure_connection): + deadline = time.time() + 3 + conn = kafka_driver.Connection( + self.conf, '', kafka_driver.PURPOSE_LISTEN) + conn.consumer = mock.MagicMock(return_value=None) + + self.assertRaises(driver_common.Timeout, conn.consume, timeout=3) + self.assertEqual(0, int(deadline - time.time())) + + +class TestKafkaListener(test_utils.BaseTestCase): + + def setUp(self): + super(TestKafkaListener, self).setUp() + self.messaging_conf.transport_driver = 'kafka' + transport = oslo_messaging.get_transport(self.conf) + self.driver = transport._driver + + @mock.patch.object(kafka_driver.Connection, '_ensure_connection') + @mock.patch.object(kafka_driver.Connection, 'declare_topic_consumer') + def test_create_listener(self, fake_consumer, fake_ensure_connection): + fake_target = oslo_messaging.Target(topic='fake_topic') + fake_targets_and_priorities = [(fake_target, 'info')] + listener = self.driver.listen_for_notifications( + fake_targets_and_priorities) + self.assertEqual(1, len(fake_consumer.mock_calls)) + + @mock.patch.object(kafka_driver.Connection, '_ensure_connection') + @mock.patch.object(kafka_driver.Connection, 'declare_topic_consumer') + def test_stop_listener(self, fake_consumer, fake_client): + fake_target = oslo_messaging.Target(topic='fake_topic') + fake_targets_and_priorities = [(fake_target, 'info')] + listener = self.driver.listen_for_notifications( + fake_targets_and_priorities) + listener.conn.consume = mock.MagicMock() + listener.conn.consume.return_value = ( + iter([kafka.common.KafkaMessage( + topic='fake_topic', partition=0, offset=0, + key=None, value='{"message": {"fake": "fake_message_1"},' + '"context": {"fake": "fake_context_1"}}')])) + listener.poll() + self.assertEqual(1, len(listener.conn.consume.mock_calls)) + listener.conn.stop_consuming = mock.MagicMock() + listener.stop() + fake_response = listener.poll() + self.assertEqual(1, len(listener.conn.consume.mock_calls)) + self.assertEqual(fake_response, None) + + +class TestWithRealKafkaBroker(test_utils.BaseTestCase): + + def setUp(self): + super(TestWithRealKafkaBroker, self).setUp() + self.messaging_conf.transport_driver = 'kafka' + transport = oslo_messaging.get_transport(self.conf, KAFKA_BROKER_URL) + self.driver = transport._driver + + @unittest.skipUnless( + _is_kafka_service_running(), "Kafka service is not available") + def test_send_and_recieve_message(self): + target = oslo_messaging.Target( + topic="fake_topic", exchange='fake_exchange') + targets_and_priorities = [(target, 'fake_info')] + + listener = self.driver.listen_for_notifications( + targets_and_priorities) + fake_context = {"fake_context_key": "fake_context_value"} + fake_message = {"fake_message_key": "fake_message_value"} + self.driver.send_notification( + target, fake_context, fake_message, None) + + received_message = listener.poll() + self.assertEqual(fake_context, received_message.ctxt) + self.assertEqual(fake_message, received_message.message) + + @unittest.skipUnless( + _is_kafka_service_running(), "Kafka service is not available") + def test_send_and_recieve_message_without_exchange(self): + target = oslo_messaging.Target(topic="fake_no_exchange_topic") + targets_and_priorities = [(target, 'fake_info')] + + listener = self.driver.listen_for_notifications( + targets_and_priorities) + fake_context = {"fake_context_key": "fake_context_value"} + fake_message = {"fake_message_key": "fake_message_value"} + self.driver.send_notification( + target, fake_context, fake_message, None) + + received_message = listener.poll() + self.assertEqual(fake_context, received_message.ctxt) + self.assertEqual(fake_message, received_message.message) + + @unittest.skipUnless( + _is_kafka_service_running(), "Kafka service is not available") + def test_recieve_message_from_empty_topic_with_timeout(self): + target = oslo_messaging.Target( + topic="fake_empty_topic", exchange='fake_empty_exchange') + targets_and_priorities = [(target, 'fake_info')] + + listener = self.driver.listen_for_notifications( + targets_and_priorities) + + deadline = time.time() + 3 + received_message = listener.poll(timeout=3) + self.assertEqual(0, int(deadline - time.time())) + self.assertEqual(None, received_message) diff --git a/setup.cfg b/setup.cfg index cbed37743..b45466ca9 100644 --- a/setup.cfg +++ b/setup.cfg @@ -29,6 +29,9 @@ oslo.messaging.drivers = zmq = oslo_messaging._drivers.impl_zmq:ZmqDriver amqp = oslo_messaging._drivers.protocols.amqp.driver:ProtonDriver + # This driver is supporting for only notification usage + kafka = oslo_messaging._drivers.impl_kafka:KafkaDriver + # To avoid confusion kombu = oslo_messaging._drivers.impl_rabbit:RabbitDriver diff --git a/test-requirements.txt b/test-requirements.txt index 89cda423c..1387e1a6b 100644 --- a/test-requirements.txt +++ b/test-requirements.txt @@ -21,6 +21,9 @@ redis>=2.10.0 # for test_impl_zmq pyzmq>=14.3.1 # LGPL+BSD +# for test_impl_kafka +kafka-python>=0.9.2 # Apache-2.0 + # when we can require tox>= 1.4, this can go into tox.ini: # [testenv:cover] # deps = {[testenv]deps} coverage From cc97ba2e17d48cc5fa02f8d9c32b2e0ffacab1a6 Mon Sep 17 00:00:00 2001 From: Mehdi Abaakouk Date: Tue, 1 Dec 2015 15:50:50 +0100 Subject: [PATCH 51/74] doc: explain rpc call/cast expection This change adds some doc about remote method execution expectation when rpc call/cast is used. Change-Id: Idb26413fc9a6747ebcd6fd32b82f63ea97bfae16 --- oslo_messaging/rpc/client.py | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/oslo_messaging/rpc/client.py b/oslo_messaging/rpc/client.py index f67f673dd..7beea841e 100644 --- a/oslo_messaging/rpc/client.py +++ b/oslo_messaging/rpc/client.py @@ -356,6 +356,10 @@ class RPCClient(object): Similarly, the request context must be a dict unless the client's serializer supports serializing another type. + Note: cast doesn't ensure the remote method to be been executed + on each destination. But ensures that it will be not executed twice + on a destination. + :param ctxt: a request context dict :type ctxt: dict :param method: the method name @@ -392,6 +396,12 @@ class RPCClient(object): allowed_remote_exmods list, then a messaging.RemoteError exception is raised with all details of the remote exception. + Note: call is done 'at-most-once'. In case of we can't known + if the call have been done correctly, because we didn't get the + response on time, MessagingTimeout exception is raised. + The real reason can vary, transport failure, worker + doesn't answer in time or crash, ... + :param ctxt: a request context dict :type ctxt: dict :param method: the method name From daddb82788918296f8b34d6cdeb40d01620fb183 Mon Sep 17 00:00:00 2001 From: Mehdi Abaakouk Date: Wed, 2 Dec 2015 11:38:27 +0100 Subject: [PATCH 52/74] Don't hold the connection when reply fail This change moves the reply retry code to upper layer to be able to release the connection while we wait between two retries. In the worse scenario, a client waits for more than 30 replies and died/restart, the server tries to send this 30 replies to this this client and can wait too 60s per replies. During this replies for other clients are just stuck. This change fixes that. Related-bug: #1477914 Closes-bug: #1521958 Change-Id: I0d3c16ea6d2c1da143de4924b3be41d1cea159bd --- oslo_messaging/_drivers/amqpdriver.py | 69 +++++++++++++++++++------- oslo_messaging/_drivers/impl_rabbit.py | 58 +++++++--------------- 2 files changed, 67 insertions(+), 60 deletions(-) diff --git a/oslo_messaging/_drivers/amqpdriver.py b/oslo_messaging/_drivers/amqpdriver.py index 420587c45..e95edfc2e 100644 --- a/oslo_messaging/_drivers/amqpdriver.py +++ b/oslo_messaging/_drivers/amqpdriver.py @@ -17,6 +17,7 @@ __all__ = ['AMQPDriverBase'] import logging import threading +import time import uuid import cachetools @@ -70,16 +71,13 @@ class AMQPIncomingMessage(base.IncomingMessage): # Otherwise use the msg_id for backward compatibility. if self.reply_q: msg['_msg_id'] = self.msg_id - try: - if ending: - LOG.debug("sending reply msg_id: %(msg_id)s " - "reply queue: %(reply_q)s" % { - 'msg_id': self.msg_id, - 'unique_id': unique_id, - 'reply_q': self.reply_q}) - conn.direct_send(self.reply_q, rpc_common.serialize_msg(msg)) - except rpc_amqp.AMQPDestinationNotFound: - self._obsolete_reply_queues.add(self.reply_q, self.msg_id) + if ending: + LOG.debug("sending reply msg_id: %(msg_id)s " + "reply queue: %(reply_q)s" % { + 'msg_id': self.msg_id, + 'unique_id': unique_id, + 'reply_q': self.reply_q}) + conn.direct_send(self.reply_q, rpc_common.serialize_msg(msg)) else: # TODO(sileht): look at which version of oslo-incubator rpc # send need this, but I guess this is older than icehouse @@ -93,20 +91,52 @@ class AMQPIncomingMessage(base.IncomingMessage): # because reply should not be expected by caller side return - # NOTE(sileht): return without hold the a connection if possible + # NOTE(sileht): return without using a connection if possible if (self.reply_q and not self._obsolete_reply_queues.reply_q_valid(self.reply_q, self.msg_id)): return - with self.listener.driver._get_connection( - rpc_common.PURPOSE_SEND) as conn: - if self.listener.driver.send_single_reply: - self._send_reply(conn, reply, failure, log_failure=log_failure, - ending=True) - else: - self._send_reply(conn, reply, failure, log_failure=log_failure) - self._send_reply(conn, ending=True) + # NOTE(sileht): we read the configuration value from the driver + # to be able to backport this change in previous version that + # still have the qpid driver + duration = self.listener.driver.missing_destination_retry_timeout + timer = rpc_common.DecayingTimer(duration=duration) + timer.start() + + first_reply_sent = False + while True: + try: + with self.listener.driver._get_connection( + rpc_common.PURPOSE_SEND) as conn: + if self.listener.driver.send_single_reply: + self._send_reply(conn, reply, failure, + log_failure=log_failure, + ending=True) + else: + if not first_reply_sent: + self._send_reply(conn, reply, failure, + log_failure=log_failure) + first_reply_sent = True + self._send_reply(conn, ending=True) + return + except rpc_amqp.AMQPDestinationNotFound: + if timer.check_return() > 0: + LOG.info(_LI("The reply %(msg_id)s cannot be sent " + "%(reply_q)s reply queue don't exist, " + "retrying...") % { + 'msg_id': self.msg_id, + 'reply_q': self.reply_q}) + time.sleep(0.25) + else: + self._obsolete_reply_queues.add(self.reply_q, self.msg_id) + LOG.info(_LI("The reply %(msg_id)s cannot be sent " + "%(reply_q)s reply queue don't exist after " + "%(duration)s sec abandoning...") % { + 'msg_id': self.msg_id, + 'reply_q': self.reply_q, + 'duration': duration}) + return def acknowledge(self): self.acknowledge_callback() @@ -345,6 +375,7 @@ class ReplyWaiter(object): class AMQPDriverBase(base.BaseDriver): + missing_destination_retry_timeout = 0 def __init__(self, conf, url, connection_pool, default_exchange=None, allowed_remote_exmods=None, diff --git a/oslo_messaging/_drivers/impl_rabbit.py b/oslo_messaging/_drivers/impl_rabbit.py index d96a39838..cdd642ea6 100644 --- a/oslo_messaging/_drivers/impl_rabbit.py +++ b/oslo_messaging/_drivers/impl_rabbit.py @@ -1043,32 +1043,20 @@ class Connection(object): self._publish(exchange, msg, routing_key=routing_key, timeout=timeout) - def _publish_and_retry_on_missing_exchange(self, exchange, msg, - routing_key=None, timeout=None): - """Publisher that retry if the exchange is missing. - """ - + def _publish_and_raises_on_missing_exchange(self, exchange, msg, + routing_key=None, + timeout=None): + """Publisher that raises exception if exchange is missing.""" if not exchange.passive: raise RuntimeError("_publish_and_retry_on_missing_exchange() must " "be called with an passive exchange.") - # TODO(sileht): use @retrying - # NOTE(sileht): no need to wait the application expect a response - # before timeout is exshauted - duration = ( - timeout if timeout is not None - else self.kombu_reconnect_timeout - ) - - timer = rpc_common.DecayingTimer(duration=duration) - timer.start() - - while True: - try: - self._publish(exchange, msg, routing_key=routing_key, - timeout=timeout) - return - except self.connection.channel_errors as exc: + try: + self._publish(exchange, msg, routing_key=routing_key, + timeout=timeout) + return + except self.connection.channel_errors as exc: + if exc.code == 404: # NOTE(noelbk/sileht): # If rabbit dies, the consumer can be disconnected before the # publisher sends, and if the consumer hasn't declared the @@ -1077,24 +1065,9 @@ class Connection(object): # So we set passive=True to the publisher exchange and catch # the 404 kombu ChannelError and retry until the exchange # appears - if exc.code == 404 and timer.check_return() > 0: - LOG.info(_LI("The exchange %(exchange)s to send to " - "%(routing_key)s doesn't exist yet, " - "retrying...") % { - 'exchange': exchange.name, - 'routing_key': routing_key}) - time.sleep(0.25) - continue - elif exc.code == 404: - msg = _("The exchange %(exchange)s to send to " - "%(routing_key)s still doesn't exist after " - "%(duration)s sec abandoning...") % { - 'duration': duration, - 'exchange': exchange.name, - 'routing_key': routing_key} - LOG.info(msg) - raise rpc_amqp.AMQPDestinationNotFound(msg) - raise + raise rpc_amqp.AMQPDestinationNotFound( + "exchange %s doesn't exists" % exchange.name) + raise def direct_send(self, msg_id, msg): """Send a 'direct' message.""" @@ -1104,7 +1077,7 @@ class Connection(object): auto_delete=True, passive=True) - self._ensure_publishing(self._publish_and_retry_on_missing_exchange, + self._ensure_publishing(self._publish_and_raises_on_missing_exchange, exchange, msg, routing_key=msg_id) def topic_send(self, exchange_name, topic, msg, timeout=None, retry=None): @@ -1160,6 +1133,9 @@ class RabbitDriver(amqpdriver.AMQPDriverBase): conf.register_opts(rpc_amqp.amqp_opts, group=opt_group) conf.register_opts(base.base_opts, group=opt_group) + self.missing_destination_retry_timeout = ( + conf.oslo_messaging_rabbit.kombu_reconnect_timeout) + connection_pool = pool.ConnectionPool( conf, conf.oslo_messaging_rabbit.rpc_conn_pool_size, url, Connection) From b6ad95e1caa19a755e11077facaa2022b64d0cf0 Mon Sep 17 00:00:00 2001 From: Davanum Srinivas Date: Tue, 1 Dec 2015 17:23:20 -0500 Subject: [PATCH 53/74] Support older notifications set_override keys Neutron and Ceilometer use set_override to set the older deprecated key. We should support them using the ConfFixture Closes-Bug: #1521776 Change-Id: I2bd77284f80bc4525f062f313b1ec74f2b54b395 --- oslo_messaging/conffixture.py | 19 ++++++++++++++++ oslo_messaging/tests/test_fixture.py | 33 ++++++++++++++++++++++++++++ 2 files changed, 52 insertions(+) create mode 100644 oslo_messaging/tests/test_fixture.py diff --git a/oslo_messaging/conffixture.py b/oslo_messaging/conffixture.py index 2f53d3f9e..42e9d51b0 100644 --- a/oslo_messaging/conffixture.py +++ b/oslo_messaging/conffixture.py @@ -18,6 +18,7 @@ __all__ = ['ConfFixture'] import sys import fixtures +from functools import wraps def _import_opts(conf, module, opts, group=None): @@ -70,6 +71,24 @@ class ConfFixture(fixtures.Fixture): '_notifier_opts', 'oslo_messaging_notifications') + # Support older test cases that still use the set_override + # with the old config key names + def decorator_for_set_override(wrapped_function): + @wraps(wrapped_function) + def _wrapper(*args, **kwargs): + group = 'oslo_messaging_notifications' + if args[0] == 'notification_driver': + args = ('driver', args[1], group) + elif args[0] == 'notification_transport_url': + args = ('transport_url', args[1], group) + elif args[0] == 'notification_topics': + args = ('topics', args[1], group) + return wrapped_function(*args, **kwargs) + return _wrapper + + self.conf.set_override = decorator_for_set_override( + self.conf.set_override) + def setUp(self): super(ConfFixture, self).setUp() self.addCleanup(self.conf.reset) diff --git a/oslo_messaging/tests/test_fixture.py b/oslo_messaging/tests/test_fixture.py new file mode 100644 index 000000000..dfa78b63d --- /dev/null +++ b/oslo_messaging/tests/test_fixture.py @@ -0,0 +1,33 @@ +# Copyright 2015 Mirantis Inc. +# +# 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 oslo_messaging.tests import utils as test_utils + + +class TestConfFixture(test_utils.BaseTestCase): + def test_old_notifications_config_override(self): + conf = self.messaging_conf.conf + conf.set_override( + "notification_driver", "messaging") + conf.set_override( + "notification_transport_url", "http://xyz") + conf.set_override( + "notification_topics", ['topic1']) + + self.assertEqual("messaging", + conf.oslo_messaging_notifications.driver) + self.assertEqual("http://xyz", + conf.oslo_messaging_notifications.transport_url) + self.assertEqual(['topic1'], + conf.oslo_messaging_notifications.topics) From 74a0ec8b1c4b3ca6c700d110d6bfd77348cc970a Mon Sep 17 00:00:00 2001 From: Davanum Srinivas Date: Wed, 2 Dec 2015 14:43:32 -0500 Subject: [PATCH 54/74] setUp/tearDown decorator for set/clear override Problem with recursion shows up only in full runs of Nova for example. So split the code that sets up the decorator and add a method to cleanup the decorated set_override during teardown. Also add a decorator for clear_override similar to the one for set_override. Added more tests for all the above. Change-Id: Ib16af2e770e96d971aef7f5c5d48ffd781477cfe --- oslo_messaging/conffixture.py | 32 ++++++++++++++++-- oslo_messaging/tests/test_fixture.py | 49 ++++++++++++++++++++++++++++ 2 files changed, 79 insertions(+), 2 deletions(-) diff --git a/oslo_messaging/conffixture.py b/oslo_messaging/conffixture.py index 42e9d51b0..0d05a5881 100644 --- a/oslo_messaging/conffixture.py +++ b/oslo_messaging/conffixture.py @@ -71,6 +71,7 @@ class ConfFixture(fixtures.Fixture): '_notifier_opts', 'oslo_messaging_notifications') + def _setup_decorator(self): # Support older test cases that still use the set_override # with the old config key names def decorator_for_set_override(wrapped_function): @@ -84,13 +85,40 @@ class ConfFixture(fixtures.Fixture): elif args[0] == 'notification_topics': args = ('topics', args[1], group) return wrapped_function(*args, **kwargs) + _wrapper.wrapped = wrapped_function return _wrapper - self.conf.set_override = decorator_for_set_override( - self.conf.set_override) + def decorator_for_clear_override(wrapped_function): + @wraps(wrapped_function) + def _wrapper(*args, **kwargs): + group = 'oslo_messaging_notifications' + if args[0] == 'notification_driver': + args = ('driver', group) + elif args[0] == 'notification_transport_url': + args = ('transport_url', group) + elif args[0] == 'notification_topics': + args = ('topics', group) + return wrapped_function(*args, **kwargs) + _wrapper.wrapped = wrapped_function + return _wrapper + + if not hasattr(self.conf.set_override, 'wrapped'): + self.conf.set_override = decorator_for_set_override( + self.conf.set_override) + if not hasattr(self.conf.clear_override, 'wrapped'): + self.conf.clear_override = decorator_for_clear_override( + self.conf.clear_override) + + def _teardown_decorator(self): + if hasattr(self.conf.set_override, 'wrapped'): + self.conf.set_override = self.conf.set_override.wrapped + if hasattr(self.conf.clear_override, 'wrapped'): + self.conf.clear_override = self.conf.clear_override.wrapped def setUp(self): super(ConfFixture, self).setUp() + self._setup_decorator() + self.addCleanup(self._teardown_decorator) self.addCleanup(self.conf.reset) @property diff --git a/oslo_messaging/tests/test_fixture.py b/oslo_messaging/tests/test_fixture.py index dfa78b63d..2ad1117d6 100644 --- a/oslo_messaging/tests/test_fixture.py +++ b/oslo_messaging/tests/test_fixture.py @@ -12,10 +12,48 @@ # License for the specific language governing permissions and limitations # under the License. +from oslo_config import cfg + +from oslo_messaging import conffixture from oslo_messaging.tests import utils as test_utils class TestConfFixture(test_utils.BaseTestCase): + + def test_fixture_wraps_set_override(self): + conf = self.messaging_conf.conf + self.assertIsNotNone(conf.set_override.wrapped) + self.messaging_conf._teardown_decorator() + self.assertFalse(hasattr(conf.set_override, 'wrapped')) + + def test_fixture_wraps_clear_override(self): + conf = self.messaging_conf.conf + self.assertIsNotNone(conf.clear_override.wrapped) + self.messaging_conf._teardown_decorator() + self.assertFalse(hasattr(conf.clear_override, 'wrapped')) + + def test_fixture_setup_teardown_decorator(self): + conf = cfg.ConfigOpts() + self.assertFalse(hasattr(conf.set_override, 'wrapped')) + self.assertFalse(hasattr(conf.clear_override, 'wrapped')) + fixture = conffixture.ConfFixture(conf) + self.assertFalse(hasattr(conf.set_override, 'wrapped')) + self.assertFalse(hasattr(conf.clear_override, 'wrapped')) + self.useFixture(fixture) + self.assertTrue(hasattr(conf.set_override, 'wrapped')) + self.assertTrue(hasattr(conf.clear_override, 'wrapped')) + fixture._teardown_decorator() + self.assertFalse(hasattr(conf.set_override, 'wrapped')) + self.assertFalse(hasattr(conf.clear_override, 'wrapped')) + + def test_fixture_properties(self): + conf = self.messaging_conf.conf + self.messaging_conf.transport_driver = 'fake' + self.assertEqual('fake', + self.messaging_conf.transport_driver) + self.assertEqual('fake', + conf.rpc_backend) + def test_old_notifications_config_override(self): conf = self.messaging_conf.conf conf.set_override( @@ -31,3 +69,14 @@ class TestConfFixture(test_utils.BaseTestCase): conf.oslo_messaging_notifications.transport_url) self.assertEqual(['topic1'], conf.oslo_messaging_notifications.topics) + + conf.clear_override("notification_driver") + conf.clear_override("notification_transport_url") + conf.clear_override("notification_topics") + + self.assertEqual([], + conf.oslo_messaging_notifications.driver) + self.assertEqual(None, + conf.oslo_messaging_notifications.transport_url) + self.assertEqual(['notifications'], + conf.oslo_messaging_notifications.topics) \ No newline at end of file From 822b803fb0d7e260628230c680b4975c1a3e0900 Mon Sep 17 00:00:00 2001 From: Kenneth Giusti Date: Thu, 3 Dec 2015 14:22:50 -0500 Subject: [PATCH 55/74] Skip Cyrus SASL tests if proton does not support Cyrus SASL Change-Id: I265d17a2c92b97777a5a97683b95427825872d3a Closes-Bug: #1508523 --- oslo_messaging/tests/test_amqp_driver.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/oslo_messaging/tests/test_amqp_driver.py b/oslo_messaging/tests/test_amqp_driver.py index 4e3b750fe..e73f4916c 100644 --- a/oslo_messaging/tests/test_amqp_driver.py +++ b/oslo_messaging/tests/test_amqp_driver.py @@ -38,6 +38,11 @@ pyngus = importutils.try_import("pyngus") if pyngus: from oslo_messaging._drivers.protocols.amqp import driver as amqp_driver +# The Cyrus-based SASL tests can only be run if the installed version of proton +# has been built with Cyrus SASL support. +_proton = importutils.try_import("proton") +CYRUS_ENABLED = (pyngus and pyngus.VERSION >= (2, 0, 0) and _proton + and getattr(_proton.SASL, "extended", lambda: False)()) LOG = logging.getLogger(__name__) @@ -354,8 +359,7 @@ class TestAuthentication(test_utils.BaseTestCase): driver.cleanup() -@testtools.skipUnless(pyngus and pyngus.VERSION >= (2, 0, 0), - "pyngus module not present") +@testtools.skipUnless(CYRUS_ENABLED, "Cyrus SASL not supported") class TestCyrusAuthentication(test_utils.BaseTestCase): """Test the driver's Cyrus SASL integration""" From 18d1708711191a8cfee479499ac066828355d47f Mon Sep 17 00:00:00 2001 From: Mehdi Abaakouk Date: Wed, 2 Dec 2015 09:36:02 +0100 Subject: [PATCH 56/74] rename kombu_reconnect_timeout option This change renames kombu_reconnect_timeout to missing_consumer_retry_timeout. And improves its documentation. Change-Id: I961cf96108db2f392b13d159f516baac9ff4e989 --- oslo_messaging/_drivers/impl_rabbit.py | 21 +++++++------------ .../tests/drivers/test_impl_rabbit.py | 6 +++--- 2 files changed, 11 insertions(+), 16 deletions(-) diff --git a/oslo_messaging/_drivers/impl_rabbit.py b/oslo_messaging/_drivers/impl_rabbit.py index cdd642ea6..72dd62af3 100644 --- a/oslo_messaging/_drivers/impl_rabbit.py +++ b/oslo_messaging/_drivers/impl_rabbit.py @@ -73,18 +73,12 @@ rabbit_opts = [ deprecated_group='DEFAULT', help='How long to wait before reconnecting in response to an ' 'AMQP consumer cancel notification.'), - cfg.IntOpt('kombu_reconnect_timeout', - # NOTE(dhellmann): We want this to be similar to - # rpc_response_timeout, but we can't use - # "$rpc_response_timeout" as a default because that - # option may not have been defined by the time this - # option is accessed. Instead, document the intent in - # the help text for this option and provide a separate - # literal default value. + cfg.IntOpt('kombu_missing_consumer_retry_timeout', + deprecated_name="kombu_reconnect_timeout", default=60, - help='How long to wait before considering a reconnect ' - 'attempt to have failed. This value should not be ' - 'longer than rpc_response_timeout.'), + help='How long to wait a missing client beforce abandoning to ' + 'send it its replies. This value should not be longer ' + 'than rpc_response_timeout.'), cfg.StrOpt('kombu_failover_strategy', choices=('round-robin', 'shuffle'), default='round-robin', @@ -384,7 +378,8 @@ class Connection(object): self.amqp_durable_queues = driver_conf.amqp_durable_queues self.amqp_auto_delete = driver_conf.amqp_auto_delete self.rabbit_use_ssl = driver_conf.rabbit_use_ssl - self.kombu_reconnect_timeout = driver_conf.kombu_reconnect_timeout + self.kombu_missing_consumer_retry_timeout = \ + driver_conf.kombu_missing_consumer_retry_timeout self.kombu_failover_strategy = driver_conf.kombu_failover_strategy if self.rabbit_use_ssl: @@ -1134,7 +1129,7 @@ class RabbitDriver(amqpdriver.AMQPDriverBase): conf.register_opts(base.base_opts, group=opt_group) self.missing_destination_retry_timeout = ( - conf.oslo_messaging_rabbit.kombu_reconnect_timeout) + conf.oslo_messaging_rabbit.kombu_missing_consumer_retry_timeout) connection_pool = pool.ConnectionPool( conf, conf.oslo_messaging_rabbit.rpc_conn_pool_size, diff --git a/oslo_messaging/tests/drivers/test_impl_rabbit.py b/oslo_messaging/tests/drivers/test_impl_rabbit.py index 63ce0662a..52cbfe1cb 100644 --- a/oslo_messaging/tests/drivers/test_impl_rabbit.py +++ b/oslo_messaging/tests/drivers/test_impl_rabbit.py @@ -379,7 +379,7 @@ class TestSendReceive(test_utils.BaseTestCase): cls._reply_ending) def test_send_receive(self): - self.config(kombu_reconnect_timeout=0.5, + self.config(kombu_missing_consumer_retry_timeout=0.5, group="oslo_messaging_rabbit") self.config(heartbeat_timeout_threshold=0, group="oslo_messaging_rabbit") @@ -474,10 +474,10 @@ class TestSendReceive(test_utils.BaseTestCase): if self.reply_failure_404: # NOTE(sileht) all reply fail, first take - # kombu_reconnect_timeout seconds to fail + # kombu_missing_consumer_retry_timeout seconds to fail # next immediately fail dt = time.time() - start - timeout = self.conf.oslo_messaging_rabbit.kombu_reconnect_timeout + timeout = self.conf.oslo_messaging_rabbit.kombu_missing_consumer_retry_timeout self.assertTrue(timeout <= dt < (timeout + 0.100), dt) self.assertEqual(len(senders), len(replies)) From 8c03a6db6c0396099e7425834998da5478a1df7c Mon Sep 17 00:00:00 2001 From: Mehdi Abaakouk Date: Wed, 2 Dec 2015 09:39:37 +0100 Subject: [PATCH 57/74] default of kombu_missing_consumer_retry_timeout This change the default of kombu_missing_consumer_retry_timeout The initial value of 60 seconds, have been chosen because the default rpc_response_timeout is 60. That means, the client doesn't wait for its reply after rpc_response_timeout is reach, so we don't need to retry it send it its reply more than rpc_response_timeout. But the real intent of kombu_missing_consumer_retry_timeout is to mitigate the side effect when the rabbitmq server(s) died/failover/restart. So the question is more how long we expect the server(s) to come back and all the oslo.messaging applications to reconnect. In that case 60 seconds looks a bit high. Also this 60 seconds have a sad side effect when we can't send the reply when the rpc client is really gone (like nova-compute restart). The rabbitmq connection to send the reply is hold during 60 seconds. I propose 5 seconds because,i n case of failover or restart I expect everything because normal in less that 5 seconds. Change-Id: I2ec174e440eb91e950d9453a9de8b97ed5888968 --- oslo_messaging/_drivers/impl_rabbit.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/oslo_messaging/_drivers/impl_rabbit.py b/oslo_messaging/_drivers/impl_rabbit.py index 72dd62af3..b9ff36353 100644 --- a/oslo_messaging/_drivers/impl_rabbit.py +++ b/oslo_messaging/_drivers/impl_rabbit.py @@ -75,7 +75,7 @@ rabbit_opts = [ 'AMQP consumer cancel notification.'), cfg.IntOpt('kombu_missing_consumer_retry_timeout', deprecated_name="kombu_reconnect_timeout", - default=60, + default=5, help='How long to wait a missing client beforce abandoning to ' 'send it its replies. This value should not be longer ' 'than rpc_response_timeout.'), From 6ad70713a3316dd2003ff1f73db573b674a6f20f Mon Sep 17 00:00:00 2001 From: Mehdi Abaakouk Date: Wed, 2 Dec 2015 10:02:01 +0100 Subject: [PATCH 58/74] Follow the plan about the single reply message This change removes the "send_single_reply" option as planned in the bp: http://specs.openstack.org/openstack/oslo-specs/specs/liberty/oslo.messaging-remove-double-reply.html Change-Id: Ib88de71cb2008a49a25f302d5e47ed587154d402 --- oslo_messaging/_drivers/amqp.py | 12 ------ oslo_messaging/_drivers/amqpdriver.py | 40 ++++++------------- oslo_messaging/_drivers/impl_rabbit.py | 3 +- .../tests/drivers/test_impl_rabbit.py | 11 +---- 4 files changed, 14 insertions(+), 52 deletions(-) diff --git a/oslo_messaging/_drivers/amqp.py b/oslo_messaging/_drivers/amqp.py index 86f41adde..06a59f846 100644 --- a/oslo_messaging/_drivers/amqp.py +++ b/oslo_messaging/_drivers/amqp.py @@ -48,18 +48,6 @@ amqp_opts = [ default=False, deprecated_group='DEFAULT', help='Auto-delete queues in AMQP.'), - cfg.BoolOpt('send_single_reply', - default=False, - help='Send a single AMQP reply to call message. The current ' - 'behaviour since oslo-incubator is to send two AMQP ' - 'replies - first one with the payload, a second one to ' - 'ensure the other have finish to send the payload. We ' - 'are going to remove it in the N release, but we must ' - 'keep backward compatible at the same time. This option ' - 'provides such compatibility - it defaults to False in ' - 'Liberty and can be turned on for early adopters with a ' - 'new installations or for testing. Please note, that ' - 'this option will be removed in the Mitaka release.') ] UNIQUE_ID = '_unique_id' diff --git a/oslo_messaging/_drivers/amqpdriver.py b/oslo_messaging/_drivers/amqpdriver.py index e95edfc2e..d03fa6529 100644 --- a/oslo_messaging/_drivers/amqpdriver.py +++ b/oslo_messaging/_drivers/amqpdriver.py @@ -48,8 +48,7 @@ class AMQPIncomingMessage(base.IncomingMessage): self.requeue_callback = message.requeue self._obsolete_reply_queues = obsolete_reply_queues - def _send_reply(self, conn, reply=None, failure=None, - ending=False, log_failure=True): + def _send_reply(self, conn, reply=None, failure=None, log_failure=True): if (self.reply_q and not self._obsolete_reply_queues.reply_q_valid(self.reply_q, self.msg_id)): @@ -58,11 +57,9 @@ class AMQPIncomingMessage(base.IncomingMessage): if failure: failure = rpc_common.serialize_remote_exception(failure, log_failure) - - msg = {'result': reply, 'failure': failure} - if ending: - msg['ending'] = True - + # NOTE(sileht): ending can be removed in N*, see Listener.wait() + # for more detail. + msg = {'result': reply, 'failure': failure, 'ending': True} rpc_amqp._add_unique_id(msg) unique_id = msg[rpc_amqp.UNIQUE_ID] @@ -71,12 +68,11 @@ class AMQPIncomingMessage(base.IncomingMessage): # Otherwise use the msg_id for backward compatibility. if self.reply_q: msg['_msg_id'] = self.msg_id - if ending: - LOG.debug("sending reply msg_id: %(msg_id)s " - "reply queue: %(reply_q)s" % { - 'msg_id': self.msg_id, - 'unique_id': unique_id, - 'reply_q': self.reply_q}) + LOG.debug("sending reply msg_id: %(msg_id)s " + "reply queue: %(reply_q)s" % { + 'msg_id': self.msg_id, + 'unique_id': unique_id, + 'reply_q': self.reply_q}) conn.direct_send(self.reply_q, rpc_common.serialize_msg(msg)) else: # TODO(sileht): look at which version of oslo-incubator rpc @@ -104,21 +100,12 @@ class AMQPIncomingMessage(base.IncomingMessage): timer = rpc_common.DecayingTimer(duration=duration) timer.start() - first_reply_sent = False while True: try: with self.listener.driver._get_connection( rpc_common.PURPOSE_SEND) as conn: - if self.listener.driver.send_single_reply: - self._send_reply(conn, reply, failure, - log_failure=log_failure, - ending=True) - else: - if not first_reply_sent: - self._send_reply(conn, reply, failure, - log_failure=log_failure) - first_reply_sent = True - self._send_reply(conn, ending=True) + self._send_reply(conn, reply, failure, + log_failure=log_failure) return except rpc_amqp.AMQPDestinationNotFound: if timer.check_return() > 0: @@ -378,8 +365,7 @@ class AMQPDriverBase(base.BaseDriver): missing_destination_retry_timeout = 0 def __init__(self, conf, url, connection_pool, - default_exchange=None, allowed_remote_exmods=None, - send_single_reply=False): + default_exchange=None, allowed_remote_exmods=None): super(AMQPDriverBase, self).__init__(conf, url, default_exchange, allowed_remote_exmods) @@ -392,8 +378,6 @@ class AMQPDriverBase(base.BaseDriver): self._reply_q_conn = None self._waiter = None - self.send_single_reply = send_single_reply - def _get_exchange(self, target): return target.exchange or self._default_exchange diff --git a/oslo_messaging/_drivers/impl_rabbit.py b/oslo_messaging/_drivers/impl_rabbit.py index b9ff36353..f0110d940 100644 --- a/oslo_messaging/_drivers/impl_rabbit.py +++ b/oslo_messaging/_drivers/impl_rabbit.py @@ -1139,8 +1139,7 @@ class RabbitDriver(amqpdriver.AMQPDriverBase): conf, url, connection_pool, default_exchange, - allowed_remote_exmods, - conf.oslo_messaging_rabbit.send_single_reply, + allowed_remote_exmods ) def require_features(self, requeue=True): diff --git a/oslo_messaging/tests/drivers/test_impl_rabbit.py b/oslo_messaging/tests/drivers/test_impl_rabbit.py index 52cbfe1cb..9fdd211a8 100644 --- a/oslo_messaging/tests/drivers/test_impl_rabbit.py +++ b/oslo_messaging/tests/drivers/test_impl_rabbit.py @@ -28,7 +28,6 @@ from oslotest import mockpatch import testscenarios import oslo_messaging -from oslo_messaging._drivers import amqp from oslo_messaging._drivers import amqpdriver from oslo_messaging._drivers import common as driver_common from oslo_messaging._drivers import impl_rabbit as rabbit_driver @@ -363,11 +362,6 @@ class TestSendReceive(test_utils.BaseTestCase): ('timeout', dict(timeout=0.01)), # FIXME(markmc): timeout=0 is broken? ] - _reply_ending = [ - ('old_behavior', dict(send_single_reply=False)), - ('new_behavior', dict(send_single_reply=True)), - ] - @classmethod def generate_scenarios(cls): cls.scenarios = testscenarios.multiply_scenarios(cls._n_senders, @@ -375,16 +369,13 @@ class TestSendReceive(test_utils.BaseTestCase): cls._reply, cls._reply_fail, cls._failure, - cls._timeout, - cls._reply_ending) + cls._timeout) def test_send_receive(self): self.config(kombu_missing_consumer_retry_timeout=0.5, group="oslo_messaging_rabbit") self.config(heartbeat_timeout_threshold=0, group="oslo_messaging_rabbit") - self.config(send_single_reply=self.send_single_reply, - group="oslo_messaging_rabbit") transport = oslo_messaging.get_transport(self.conf, 'kombu+memory:////') self.addCleanup(transport.cleanup) From c1d0412e2d5b437b06d8729bbe2cdaea594427be Mon Sep 17 00:00:00 2001 From: Mehdi Abaakouk Date: Wed, 2 Dec 2015 10:13:18 +0100 Subject: [PATCH 59/74] kombu: remove compat of folsom reply format This change removes codepath where _reply_q is not present in the message dict. This kind of messages have been deprecated in grizzly and cannot be emitted since havana. https://github.com/openstack/oslo-incubator/commit/70891c271e011f59792933eaf65c3214493ef14a Change-Id: I20558d9fae8f56970c967aa0def77cfb2a1ca3ec --- oslo_messaging/_drivers/amqpdriver.py | 46 +++++++++------------------ 1 file changed, 15 insertions(+), 31 deletions(-) diff --git a/oslo_messaging/_drivers/amqpdriver.py b/oslo_messaging/_drivers/amqpdriver.py index d03fa6529..f8deac836 100644 --- a/oslo_messaging/_drivers/amqpdriver.py +++ b/oslo_messaging/_drivers/amqpdriver.py @@ -49,9 +49,8 @@ class AMQPIncomingMessage(base.IncomingMessage): self._obsolete_reply_queues = obsolete_reply_queues def _send_reply(self, conn, reply=None, failure=None, log_failure=True): - if (self.reply_q and - not self._obsolete_reply_queues.reply_q_valid(self.reply_q, - self.msg_id)): + if not self._obsolete_reply_queues.reply_q_valid(self.reply_q, + self.msg_id): return if failure: @@ -59,27 +58,17 @@ class AMQPIncomingMessage(base.IncomingMessage): log_failure) # NOTE(sileht): ending can be removed in N*, see Listener.wait() # for more detail. - msg = {'result': reply, 'failure': failure, 'ending': True} + msg = {'result': reply, 'failure': failure, 'ending': True, + '_msg_id': self.msg_id} rpc_amqp._add_unique_id(msg) unique_id = msg[rpc_amqp.UNIQUE_ID] - # If a reply_q exists, add the msg_id to the reply and pass the - # reply_q to direct_send() to use it as the response queue. - # Otherwise use the msg_id for backward compatibility. - if self.reply_q: - msg['_msg_id'] = self.msg_id - LOG.debug("sending reply msg_id: %(msg_id)s " - "reply queue: %(reply_q)s" % { - 'msg_id': self.msg_id, - 'unique_id': unique_id, - 'reply_q': self.reply_q}) - conn.direct_send(self.reply_q, rpc_common.serialize_msg(msg)) - else: - # TODO(sileht): look at which version of oslo-incubator rpc - # send need this, but I guess this is older than icehouse - # if this is icehouse, we can drop this at Mitaka - # if this is havana, we can drop this now. - conn.direct_send(self.msg_id, rpc_common.serialize_msg(msg)) + LOG.debug("sending reply msg_id: %(msg_id)s " + "reply queue: %(reply_q)s" % { + 'msg_id': self.msg_id, + 'unique_id': unique_id, + 'reply_q': self.reply_q}) + conn.direct_send(self.reply_q, rpc_common.serialize_msg(msg)) def reply(self, reply=None, failure=None, log_failure=True): if not self.msg_id: @@ -87,10 +76,9 @@ class AMQPIncomingMessage(base.IncomingMessage): # because reply should not be expected by caller side return - # NOTE(sileht): return without using a connection if possible - if (self.reply_q and - not self._obsolete_reply_queues.reply_q_valid(self.reply_q, - self.msg_id)): + # NOTE(sileht): return without hold the a connection if possible + if not self._obsolete_reply_queues.reply_q_valid(self.reply_q, + self.msg_id): return # NOTE(sileht): we read the configuration value from the driver @@ -204,12 +192,8 @@ class AMQPListener(base.Listener): unique_id = self.msg_id_cache.check_duplicate_message(message) - if ctxt.reply_q: - LOG.debug( - "received message msg_id: %(msg_id)s reply to %(queue)s" % { - 'queue': ctxt.reply_q, 'msg_id': ctxt.msg_id}) - else: - LOG.debug("received message unique_id: %s " % unique_id) + LOG.debug("received message msg_id: %(msg_id)s reply to %(queue)s" % { + 'queue': ctxt.reply_q, 'msg_id': ctxt.msg_id}) self.incoming.append(AMQPIncomingMessage(self, ctxt.to_dict(), From 52ccff7cbc26af0738d7a0a7d6e99330421b61d1 Mon Sep 17 00:00:00 2001 From: Oleksii Zamiatin Date: Fri, 4 Dec 2015 21:47:18 +0200 Subject: [PATCH 60/74] Properly skip zmq tests without ZeroMQ being installed In this change import_zmq() doesn't raise ImportError any more for the benefit of skipping tests. Alarm about zmq unavailability moved to driver's init. Change-Id: I6e6acc39f42c979333510064d9e845228400d233 Closes-Bug: #1522920 --- oslo_messaging/_drivers/impl_zmq.py | 6 ++++++ .../_drivers/zmq_driver/zmq_async.py | 4 +--- .../tests/drivers/zmq/test_impl_zmq.py | 1 + .../tests/drivers/zmq/test_zmq_async.py | 20 +++++++++---------- 4 files changed, 18 insertions(+), 13 deletions(-) diff --git a/oslo_messaging/_drivers/impl_zmq.py b/oslo_messaging/_drivers/impl_zmq.py index 17bfe41e4..f8bf1378e 100644 --- a/oslo_messaging/_drivers/impl_zmq.py +++ b/oslo_messaging/_drivers/impl_zmq.py @@ -26,7 +26,9 @@ from oslo_messaging._drivers import common as rpc_common from oslo_messaging._drivers.zmq_driver.client import zmq_client from oslo_messaging._drivers.zmq_driver.client import zmq_client_light from oslo_messaging._drivers.zmq_driver.server import zmq_server +from oslo_messaging._drivers.zmq_driver import zmq_async from oslo_messaging._executors import impl_pooledexecutor # FIXME(markmc) +from oslo_messaging._i18n import _LE pformat = pprint.pformat @@ -162,6 +164,10 @@ class ZmqDriver(base.BaseDriver): :param allowed_remote_exmods: remote exception passing options :type allowed_remote_exmods: list """ + zmq = zmq_async.import_zmq() + if zmq is None: + raise ImportError(_LE("ZeroMQ is not available!")) + conf.register_opts(zmq_opts) conf.register_opts(impl_pooledexecutor._pool_opts) conf.register_opts(base.base_opts) diff --git a/oslo_messaging/_drivers/zmq_driver/zmq_async.py b/oslo_messaging/_drivers/zmq_driver/zmq_async.py index 4de248f39..093544118 100644 --- a/oslo_messaging/_drivers/zmq_driver/zmq_async.py +++ b/oslo_messaging/_drivers/zmq_driver/zmq_async.py @@ -30,12 +30,10 @@ def import_zmq(zmq_concurrency='eventlet'): _raise_error_if_invalid_config_value(zmq_concurrency) imported_zmq = importutils.try_import(ZMQ_MODULES[zmq_concurrency], - default='zmq') + default=None) if imported_zmq is None: - errmsg = _LE("ZeroMQ not found!") LOG.error(_LE("ZeroMQ not found!")) - raise ImportError(errmsg) return imported_zmq diff --git a/oslo_messaging/tests/drivers/zmq/test_impl_zmq.py b/oslo_messaging/tests/drivers/zmq/test_impl_zmq.py index a11b61f59..4062f17c7 100644 --- a/oslo_messaging/tests/drivers/zmq/test_impl_zmq.py +++ b/oslo_messaging/tests/drivers/zmq/test_impl_zmq.py @@ -224,6 +224,7 @@ class TestZmqBasics(ZmqBaseTestCase): class TestPoller(test_utils.BaseTestCase): + @testtools.skipIf(zmq is None, "zmq not available") def setUp(self): super(TestPoller, self).setUp() self.poller = zmq_async.get_poller() diff --git a/oslo_messaging/tests/drivers/zmq/test_zmq_async.py b/oslo_messaging/tests/drivers/zmq/test_zmq_async.py index 28e091a0e..ccfae3348 100644 --- a/oslo_messaging/tests/drivers/zmq/test_zmq_async.py +++ b/oslo_messaging/tests/drivers/zmq/test_zmq_async.py @@ -11,15 +11,19 @@ # under the License. import mock +import testtools from oslo_messaging._drivers.zmq_driver.poller import green_poller from oslo_messaging._drivers.zmq_driver.poller import threading_poller from oslo_messaging._drivers.zmq_driver import zmq_async from oslo_messaging.tests import utils as test_utils +zmq = zmq_async.import_zmq() + class TestImportZmq(test_utils.BaseTestCase): + @testtools.skipIf(zmq is None, "zmq not available") def setUp(self): super(TestImportZmq, self).setUp() @@ -29,12 +33,12 @@ class TestImportZmq(test_utils.BaseTestCase): zmq_async.importutils.try_import.return_value = 'mock zmq module' self.assertEqual('mock zmq module', zmq_async.import_zmq('native')) - mock_try_import.assert_called_with('zmq', default='zmq') + mock_try_import.assert_called_with('zmq', default=None) zmq_async.importutils.try_import.return_value = 'mock eventlet module' self.assertEqual('mock eventlet module', zmq_async.import_zmq('eventlet')) - mock_try_import.assert_called_with('eventlet.green.zmq', default='zmq') + mock_try_import.assert_called_with('eventlet.green.zmq', default=None) def test_when_no_args_then_default_zmq_module_is_loaded(self): mock_try_import = mock.Mock() @@ -42,14 +46,7 @@ class TestImportZmq(test_utils.BaseTestCase): zmq_async.import_zmq() - mock_try_import.assert_called_with('eventlet.green.zmq', default='zmq') - - def test_when_import_fails_then_raise_ImportError(self): - zmq_async.importutils.try_import = mock.Mock() - zmq_async.importutils.try_import.return_value = None - - with self.assertRaisesRegexp(ImportError, "ZeroMQ not found!"): - zmq_async.import_zmq('native') + mock_try_import.assert_called_with('eventlet.green.zmq', default=None) def test_invalid_config_value_raise_ValueError(self): invalid_opt = 'x' @@ -61,6 +58,7 @@ class TestImportZmq(test_utils.BaseTestCase): class TestGetPoller(test_utils.BaseTestCase): + @testtools.skipIf(zmq is None, "zmq not available") def setUp(self): super(TestGetPoller, self).setUp() @@ -100,6 +98,7 @@ class TestGetPoller(test_utils.BaseTestCase): class TestGetReplyPoller(test_utils.BaseTestCase): + @testtools.skipIf(zmq is None, "zmq not available") def setUp(self): super(TestGetReplyPoller, self).setUp() @@ -134,6 +133,7 @@ class TestGetReplyPoller(test_utils.BaseTestCase): class TestGetExecutor(test_utils.BaseTestCase): + @testtools.skipIf(zmq is None, "zmq not available") def setUp(self): super(TestGetExecutor, self).setUp() From 5561a6fd0fbe8dc7defb8fb1f7432fc460954aa4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Stanis=C5=82aw=20Pitucha?= Date: Mon, 7 Dec 2015 15:09:43 +1100 Subject: [PATCH 61/74] Fix multiline strings with missing spaces Change-Id: Ide9999f6bb80f0f87500270a4fc024462bce0dbf --- oslo_messaging/_drivers/protocols/amqp/driver.py | 6 +++--- .../zmq_driver/client/publishers/zmq_publisher_base.py | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/oslo_messaging/_drivers/protocols/amqp/driver.py b/oslo_messaging/_drivers/protocols/amqp/driver.py index cf1c9127b..e5a1bd4e9 100644 --- a/oslo_messaging/_drivers/protocols/amqp/driver.py +++ b/oslo_messaging/_drivers/protocols/amqp/driver.py @@ -180,7 +180,7 @@ class ProtonDriver(base.BaseDriver): """Send a message to the given target.""" # TODO(kgiusti) need to add support for retry if retry is not None: - raise NotImplementedError('"retry" not implemented by' + raise NotImplementedError('"retry" not implemented by ' 'this transport driver') request = marshal_request(message, ctxt, envelope) @@ -210,7 +210,7 @@ class ProtonDriver(base.BaseDriver): """Send a notification message to the given target.""" # TODO(kgiusti) need to add support for retry if retry is not None: - raise NotImplementedError('"retry" not implemented by' + raise NotImplementedError('"retry" not implemented by ' 'this transport driver') return self.send(target, ctxt, message, envelope=(version == 2.0)) @@ -226,7 +226,7 @@ class ProtonDriver(base.BaseDriver): def listen_for_notifications(self, targets_and_priorities, pool): LOG.debug("Listen for notifications %s", targets_and_priorities) if pool: - raise NotImplementedError('"pool" not implemented by' + raise NotImplementedError('"pool" not implemented by ' 'this transport driver') listener = ProtonListener(self) for target, priority in targets_and_priorities: diff --git a/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_publisher_base.py b/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_publisher_base.py index 6daf6db18..e2f898550 100644 --- a/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_publisher_base.py +++ b/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_publisher_base.py @@ -90,7 +90,7 @@ class PublisherBase(object): :param request: Message data and destination container object :type request: zmq_request.Request """ - LOG.debug("Sending %(type)s message_id %(message)s to a target" + LOG.debug("Sending %(type)s message_id %(message)s to a target " "%(target)s" % {"type": request.msg_type, "message": request.message_id, From 2a4f915891eec8adabc1caaff3948dbde0ef6bbe Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Stanis=C5=82aw=20Pitucha?= Date: Mon, 7 Dec 2015 15:11:50 +1100 Subject: [PATCH 62/74] Remove unnecessary quote Change-Id: I6ec2297495c1a7ce409ea0de9a92a9720b6e2dca --- oslo_messaging/_drivers/protocols/amqp/drivertasks.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/oslo_messaging/_drivers/protocols/amqp/drivertasks.py b/oslo_messaging/_drivers/protocols/amqp/drivertasks.py index 5d9e2ed4d..385241334 100644 --- a/oslo_messaging/_drivers/protocols/amqp/drivertasks.py +++ b/oslo_messaging/_drivers/protocols/amqp/drivertasks.py @@ -41,7 +41,7 @@ class SendTask(controller.Task): """Wait for the send to complete, and, optionally, a reply message from the remote. Will raise MessagingTimeout if the send does not complete or no reply is received within timeout seconds. If the request has - failed for any other reason, a MessagingException is raised." + failed for any other reason, a MessagingException is raised. """ try: result = self._results_queue.get(timeout=timeout) From bdf287e847024368e20f5f806380e97070c9561c Mon Sep 17 00:00:00 2001 From: Mehdi Abaakouk Date: Tue, 1 Dec 2015 09:34:57 +0100 Subject: [PATCH 63/74] creates a dispatcher abstraction This change creates a dispatcher abstraction to document the interface of a dispatcher. And also allows in the futur to have attributes with default values. Change-Id: I9a7e5e03f89635a3790b3851f492a1a7aab58feb --- oslo_messaging/_utils.py | 51 --------- oslo_messaging/dispatcher.py | 105 ++++++++++++++++++ oslo_messaging/notify/dispatcher.py | 6 +- oslo_messaging/rpc/dispatcher.py | 5 +- .../tests/executors/test_executor.py | 7 +- 5 files changed, 114 insertions(+), 60 deletions(-) create mode 100644 oslo_messaging/dispatcher.py diff --git a/oslo_messaging/_utils.py b/oslo_messaging/_utils.py index cec94bb48..021fea26c 100644 --- a/oslo_messaging/_utils.py +++ b/oslo_messaging/_utils.py @@ -46,57 +46,6 @@ def version_is_compatible(imp_version, version): return True -class DispatcherExecutorContext(object): - """Dispatcher executor context helper - - A dispatcher can have work to do before and after the dispatch of the - request in the main server thread while the dispatcher itself can be - done in its own thread. - - The executor can use the helper like this: - - callback = dispatcher(incoming) - callback.prepare() - thread = MyWhateverThread() - thread.on_done(callback.done) - thread.run(callback.run) - - """ - def __init__(self, incoming, dispatch, executor_callback=None, - post=None): - self._result = None - self._incoming = incoming - self._dispatch = dispatch - self._post = post - self._executor_callback = executor_callback - - def run(self): - """The incoming message dispath itself - - Can be run in an other thread/greenlet/corotine if the executor is - able to do it. - """ - try: - self._result = self._dispatch(self._incoming, - self._executor_callback) - except Exception: - msg = 'The dispatcher method must catches all exceptions' - LOG.exception(msg) - raise RuntimeError(msg) - - def done(self): - """Callback after the incoming message have been dispathed - - Should be ran in the main executor thread/greenlet/corotine - """ - # FIXME(sileht): this is not currently true, this works only because - # the driver connection used for polling write on the wire only to - # ack/requeue message, but what if one day, the driver do something - # else - if self._post is not None: - self._post(self._incoming, self._result) - - def fetch_current_thread_functor(): # Until https://github.com/eventlet/eventlet/issues/172 is resolved # or addressed we have to use complicated workaround to get a object diff --git a/oslo_messaging/dispatcher.py b/oslo_messaging/dispatcher.py new file mode 100644 index 000000000..5cdd14748 --- /dev/null +++ b/oslo_messaging/dispatcher.py @@ -0,0 +1,105 @@ + +# 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 abc +import logging + +import six + +__all__ = [ + "DispatcherBase", + "DispatcherExecutorContext" +] + +LOG = logging.getLogger(__name__) + + +class DispatcherExecutorContext(object): + """Dispatcher executor context helper + + A dispatcher can have work to do before and after the dispatch of the + request in the main server thread while the dispatcher itself can be + done in its own thread. + + The executor can use the helper like this: + + callback = dispatcher(incoming) + callback.prepare() + thread = MyWhateverThread() + thread.on_done(callback.done) + thread.run(callback.run) + + """ + def __init__(self, incoming, dispatch, executor_callback=None, + post=None): + self._result = None + self._incoming = incoming + self._dispatch = dispatch + self._post = post + self._executor_callback = executor_callback + + def run(self): + """The incoming message dispath itself + + Can be run in an other thread/greenlet/corotine if the executor is + able to do it. + """ + try: + self._result = self._dispatch(self._incoming, + self._executor_callback) + except Exception: + msg = 'The dispatcher method must catches all exceptions' + LOG.exception(msg) + raise RuntimeError(msg) + + def done(self): + """Callback after the incoming message have been dispathed + + Should be ran in the main executor thread/greenlet/corotine + """ + # FIXME(sileht): this is not currently true, this works only because + # the driver connection used for polling write on the wire only to + # ack/requeue message, but what if one day, the driver do something + # else + if self._post is not None: + self._post(self._incoming, self._result) + + +@six.add_metaclass(abc.ABCMeta) +class DispatcherBase(object): + "Base class for dispatcher" + + @abc.abstractmethod + def _listen(self, transport): + """Initiate the driver Listener + + Usualy the driver Listener is start with the transport helper methods: + + * transport._listen() + * transport._listen_for_notifications() + + :param transport: the transport object + :type transport: oslo_messaging.transport.Transport + :returns: a driver Listener object + :rtype: oslo_messaging._drivers.base.Listener + """ + + @abc.abstractmethod + def __call__(self, incoming, executor_callback=None): + """Called by the executor to get the DispatcherExecutorContext + + :param incoming: message or list of messages + :type incoming: oslo_messging._drivers.base.IncomingMessage + :returns: DispatcherExecutorContext + :rtype: DispatcherExecutorContext + """ diff --git a/oslo_messaging/notify/dispatcher.py b/oslo_messaging/notify/dispatcher.py index 46d53035e..33aeea8ff 100644 --- a/oslo_messaging/notify/dispatcher.py +++ b/oslo_messaging/notify/dispatcher.py @@ -18,7 +18,7 @@ import itertools import logging import sys -from oslo_messaging import _utils as utils +from oslo_messaging import dispatcher from oslo_messaging import localcontext from oslo_messaging import serializer as msg_serializer @@ -33,7 +33,7 @@ class NotificationResult(object): REQUEUE = 'requeue' -class NotificationDispatcher(object): +class NotificationDispatcher(dispatcher.DispatcherBase): """A message dispatcher which understands Notification messages. A MessageHandlingServer is constructed by passing a callable dispatcher @@ -69,7 +69,7 @@ class NotificationDispatcher(object): pool=self.pool) def __call__(self, incoming, executor_callback=None): - return utils.DispatcherExecutorContext( + return dispatcher.DispatcherExecutorContext( incoming, self._dispatch_and_handle_error, executor_callback=executor_callback, post=self._post_dispatch) diff --git a/oslo_messaging/rpc/dispatcher.py b/oslo_messaging/rpc/dispatcher.py index 6913e7afe..9d640c425 100644 --- a/oslo_messaging/rpc/dispatcher.py +++ b/oslo_messaging/rpc/dispatcher.py @@ -31,6 +31,7 @@ import six from oslo_messaging._i18n import _LE from oslo_messaging import _utils as utils +from oslo_messaging import dispatcher from oslo_messaging import localcontext from oslo_messaging import serializer as msg_serializer from oslo_messaging import server as msg_server @@ -75,7 +76,7 @@ class UnsupportedVersion(RPCDispatcherError): self.method = method -class RPCDispatcher(object): +class RPCDispatcher(dispatcher.DispatcherBase): """A message dispatcher which understands RPC messages. A MessageHandlingServer is constructed by passing a callable dispatcher @@ -131,7 +132,7 @@ class RPCDispatcher(object): def __call__(self, incoming, executor_callback=None): incoming.acknowledge() - return utils.DispatcherExecutorContext( + return dispatcher.DispatcherExecutorContext( incoming, self._dispatch_and_reply, executor_callback=executor_callback) diff --git a/oslo_messaging/tests/executors/test_executor.py b/oslo_messaging/tests/executors/test_executor.py index 1e175fdf8..cd338a4e3 100644 --- a/oslo_messaging/tests/executors/test_executor.py +++ b/oslo_messaging/tests/executors/test_executor.py @@ -44,7 +44,7 @@ try: except ImportError: impl_eventlet = None from oslo_messaging._executors import impl_thread -from oslo_messaging import _utils as utils +from oslo_messaging import dispatcher as dispatcher_base from oslo_messaging.tests import utils as test_utils from six.moves import mock @@ -151,9 +151,8 @@ class TestExecutor(test_utils.BaseTestCase): return result def __call__(self, incoming, executor_callback=None): - return utils.DispatcherExecutorContext(incoming, - self.callback, - executor_callback) + return dispatcher_base.DispatcherExecutorContext( + incoming, self.callback, executor_callback) return Dispatcher(endpoint), endpoint, event, run_executor From a1fb6b9776dd635cf53d3bb03867de879cb4ed89 Mon Sep 17 00:00:00 2001 From: OpenStack Proposal Bot Date: Tue, 8 Dec 2015 02:32:50 +0000 Subject: [PATCH 64/74] Updated from global requirements Change-Id: Ie3e254b5b37a1d74eeb24ce1ae179ca9b4e84707 --- requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/requirements.txt b/requirements.txt index 2d23543c3..65d3200a5 100644 --- a/requirements.txt +++ b/requirements.txt @@ -10,7 +10,7 @@ oslo.context>=0.2.0 # Apache-2.0 oslo.log>=1.12.0 # Apache-2.0 oslo.utils>=2.8.0 # Apache-2.0 oslo.serialization>=1.10.0 # Apache-2.0 -oslo.service>=0.12.0 # Apache-2.0 +oslo.service>=1.0.0 # Apache-2.0 oslo.i18n>=1.5.0 # Apache-2.0 stevedore>=1.5.0 # Apache-2.0 debtcollector>=0.3.0 # Apache-2.0 From 4dd644ac201ee0fe247d648a2f735998416bf2c7 Mon Sep 17 00:00:00 2001 From: Mehdi Abaakouk Date: Sun, 2 Aug 2015 10:26:02 +0200 Subject: [PATCH 65/74] batch notification listener Gnocchi performs better if measurements are write in batch When Ceilometer is used with Gnocchi, this is not possible. This change introduce a new notification listener that allows that. On the driver side, a default batch implementation is provided. It's just call the legacy poll method many times. Driver can override it to provide a better implementation. For example, kafka handles batch natively and take benefit of this. Change-Id: I16184da24b8661aff7f4fba6196ecf33165f1a77 --- oslo_messaging/_drivers/amqpdriver.py | 1 + oslo_messaging/_drivers/base.py | 33 +++- oslo_messaging/_drivers/impl_fake.py | 1 + oslo_messaging/_drivers/impl_kafka.py | 1 + oslo_messaging/_drivers/impl_rabbit.py | 3 +- .../_drivers/protocols/amqp/driver.py | 8 +- .../_drivers/zmq_driver/server/zmq_server.py | 1 + .../_executors/impl_pooledexecutor.py | 7 +- oslo_messaging/dispatcher.py | 8 +- oslo_messaging/notify/__init__.py | 1 + oslo_messaging/notify/dispatcher.py | 161 ++++++++++++------ oslo_messaging/notify/listener.py | 43 +++++ oslo_messaging/rpc/dispatcher.py | 4 +- .../tests/drivers/test_impl_kafka.py | 6 +- .../tests/drivers/test_impl_rabbit.py | 12 +- .../tests/drivers/zmq/test_impl_zmq.py | 3 +- .../tests/executors/test_executor.py | 13 +- .../tests/functional/test_functional.py | 20 ++- oslo_messaging/tests/functional/utils.py | 50 +++++- .../tests/notify/test_dispatcher.py | 6 +- oslo_messaging/tests/notify/test_listener.py | 93 +++++++++- oslo_messaging/tests/rpc/test_dispatcher.py | 2 +- oslo_messaging/tests/test_amqp_driver.py | 2 +- tools/simulator.py | 97 ++++++++--- tox.ini | 1 + 25 files changed, 450 insertions(+), 127 deletions(-) diff --git a/oslo_messaging/_drivers/amqpdriver.py b/oslo_messaging/_drivers/amqpdriver.py index e95edfc2e..e47e7e3b8 100644 --- a/oslo_messaging/_drivers/amqpdriver.py +++ b/oslo_messaging/_drivers/amqpdriver.py @@ -232,6 +232,7 @@ class AMQPListener(base.Listener): ctxt.reply_q, self._obsolete_reply_queues)) + @base.batch_poll_helper def poll(self, timeout=None): while not self._stopped.is_set(): if self.incoming: diff --git a/oslo_messaging/_drivers/base.py b/oslo_messaging/_drivers/base.py index 607821faa..9c2cb87ba 100644 --- a/oslo_messaging/_drivers/base.py +++ b/oslo_messaging/_drivers/base.py @@ -15,9 +15,12 @@ import abc -import six - from oslo_config import cfg +from oslo_utils import timeutils +import six +from six.moves import range as compat_range + + from oslo_messaging import exceptions base_opts = [ @@ -28,6 +31,27 @@ base_opts = [ ] +def batch_poll_helper(func): + """Decorator to poll messages in batch + + This decorator helps driver that polls message one by one, + to returns a list of message. + """ + def wrapper(in_self, timeout=None, prefetch_size=1): + incomings = [] + watch = timeutils.StopWatch(duration=timeout) + with watch: + for __ in compat_range(prefetch_size): + msg = func(in_self, timeout=watch.leftover(return_none=True)) + if msg is not None: + incomings.append(msg) + else: + # timeout reached or listener stopped + break + return incomings + return wrapper + + class TransportDriverError(exceptions.MessagingException): """Base class for transport driver specific exceptions.""" @@ -61,8 +85,9 @@ class Listener(object): self.driver = driver @abc.abstractmethod - def poll(self, timeout=None): - """Blocking until a message is pending and return IncomingMessage. + def poll(self, timeout=None, prefetch_size=1): + """Blocking until 'prefetch_size' message is pending and return + [IncomingMessage]. Return None after timeout seconds if timeout is set and no message is ending or if the listener have been stopped. """ diff --git a/oslo_messaging/_drivers/impl_fake.py b/oslo_messaging/_drivers/impl_fake.py index 36365e91c..4dea4df79 100644 --- a/oslo_messaging/_drivers/impl_fake.py +++ b/oslo_messaging/_drivers/impl_fake.py @@ -54,6 +54,7 @@ class FakeListener(base.Listener): exchange = self._exchange_manager.get_exchange(target.exchange) exchange.ensure_queue(target, pool) + @base.batch_poll_helper def poll(self, timeout=None): if timeout is not None: deadline = time.time() + timeout diff --git a/oslo_messaging/_drivers/impl_kafka.py b/oslo_messaging/_drivers/impl_kafka.py index 9be417bdd..ce6452dc0 100644 --- a/oslo_messaging/_drivers/impl_kafka.py +++ b/oslo_messaging/_drivers/impl_kafka.py @@ -252,6 +252,7 @@ class KafkaListener(base.Listener): self.conn = conn self.incoming_queue = [] + @base.batch_poll_helper def poll(self, timeout=None): while not self._stopped.is_set(): if self.incoming_queue: diff --git a/oslo_messaging/_drivers/impl_rabbit.py b/oslo_messaging/_drivers/impl_rabbit.py index b9ff36353..384eb7f0d 100644 --- a/oslo_messaging/_drivers/impl_rabbit.py +++ b/oslo_messaging/_drivers/impl_rabbit.py @@ -853,7 +853,8 @@ class Connection(object): raise rpc_common.Timeout() def _recoverable_error_callback(exc): - self._new_consumers = self._consumers + if not isinstance(exc, rpc_common.Timeout): + self._new_consumers = self._consumers timer.check_return(_raise_timeout, exc) def _error_callback(exc): diff --git a/oslo_messaging/_drivers/protocols/amqp/driver.py b/oslo_messaging/_drivers/protocols/amqp/driver.py index cf1c9127b..e951294ea 100644 --- a/oslo_messaging/_drivers/protocols/amqp/driver.py +++ b/oslo_messaging/_drivers/protocols/amqp/driver.py @@ -117,8 +117,12 @@ class ProtonListener(base.Listener): super(ProtonListener, self).__init__(driver) self.incoming = moves.queue.Queue() - def poll(self): - message = self.incoming.get() + @base.batch_poll_helper + def poll(self, timeout=None): + try: + message = self.incoming.get(True, timeout) + except moves.queue.Empty: + return request, ctxt = unmarshal_request(message) LOG.debug("Returning incoming message") return ProtonIncomingMessage(self, ctxt, request, message) diff --git a/oslo_messaging/_drivers/zmq_driver/server/zmq_server.py b/oslo_messaging/_drivers/zmq_driver/server/zmq_server.py index 0680e7fa5..c37aef047 100644 --- a/oslo_messaging/_drivers/zmq_driver/server/zmq_server.py +++ b/oslo_messaging/_drivers/zmq_driver/server/zmq_server.py @@ -40,6 +40,7 @@ class ZmqServer(base.Listener): self.notify_consumer = self.rpc_consumer self.consumers = [self.rpc_consumer] + @base.batch_poll_helper def poll(self, timeout=None): message, socket = self.poller.poll( timeout or self.conf.rpc_poll_timeout) diff --git a/oslo_messaging/_executors/impl_pooledexecutor.py b/oslo_messaging/_executors/impl_pooledexecutor.py index 9962370ec..f442c3ae9 100644 --- a/oslo_messaging/_executors/impl_pooledexecutor.py +++ b/oslo_messaging/_executors/impl_pooledexecutor.py @@ -93,8 +93,11 @@ class PooledExecutor(base.ExecutorBase): @excutils.forever_retry_uncaught_exceptions def _runner(self): while not self._tombstone.is_set(): - incoming = self.listener.poll() - if incoming is None: + incoming = self.listener.poll( + timeout=self.dispatcher.batch_timeout, + prefetch_size=self.dispatcher.batch_size) + + if not incoming: continue callback = self.dispatcher(incoming, self._executor_callback) was_submitted = self._do_submit(callback) diff --git a/oslo_messaging/dispatcher.py b/oslo_messaging/dispatcher.py index 5cdd14748..780724416 100644 --- a/oslo_messaging/dispatcher.py +++ b/oslo_messaging/dispatcher.py @@ -79,6 +79,12 @@ class DispatcherExecutorContext(object): class DispatcherBase(object): "Base class for dispatcher" + batch_size = 1 + "Number of messages to wait before calling endpoints callacks" + + batch_timeout = None + "Number of seconds to wait before calling endpoints callacks" + @abc.abstractmethod def _listen(self, transport): """Initiate the driver Listener @@ -98,7 +104,7 @@ class DispatcherBase(object): def __call__(self, incoming, executor_callback=None): """Called by the executor to get the DispatcherExecutorContext - :param incoming: message or list of messages + :param incoming: list of messages :type incoming: oslo_messging._drivers.base.IncomingMessage :returns: DispatcherExecutorContext :rtype: DispatcherExecutorContext diff --git a/oslo_messaging/notify/__init__.py b/oslo_messaging/notify/__init__.py index c946e34f8..912e63398 100644 --- a/oslo_messaging/notify/__init__.py +++ b/oslo_messaging/notify/__init__.py @@ -17,6 +17,7 @@ __all__ = ['Notifier', 'LoggingNotificationHandler', 'get_notification_transport', 'get_notification_listener', + 'get_batch_notification_listener', 'NotificationResult', 'NotificationFilter', 'PublishErrorsHandler', diff --git a/oslo_messaging/notify/dispatcher.py b/oslo_messaging/notify/dispatcher.py index 33aeea8ff..afd34a270 100644 --- a/oslo_messaging/notify/dispatcher.py +++ b/oslo_messaging/notify/dispatcher.py @@ -16,7 +16,8 @@ import itertools import logging -import sys + +import six from oslo_messaging import dispatcher from oslo_messaging import localcontext @@ -33,17 +34,7 @@ class NotificationResult(object): REQUEUE = 'requeue' -class NotificationDispatcher(dispatcher.DispatcherBase): - """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 - """ - +class _NotificationDispatcherBase(dispatcher.DispatcherBase): def __init__(self, targets, endpoints, serializer, allow_requeue, pool=None): self.targets = targets @@ -74,12 +65,15 @@ class NotificationDispatcher(dispatcher.DispatcherBase): executor_callback=executor_callback, post=self._post_dispatch) - @staticmethod - def _post_dispatch(incoming, result): - if result == NotificationResult.HANDLED: - incoming.acknowledge() - else: - incoming.requeue() + def _post_dispatch(self, incoming, requeues): + for m in incoming: + try: + if requeues and m in requeues: + m.requeue() + else: + m.acknowledge() + except Exception: + LOG.error("Fail to ack/requeue message", exc_info=True) def _dispatch_and_handle_error(self, incoming, executor_callback): """Dispatch a notification message to the appropriate endpoint method. @@ -88,24 +82,59 @@ class NotificationDispatcher(dispatcher.DispatcherBase): :type ctxt: IncomingMessage """ try: - return self._dispatch(incoming.ctxt, incoming.message, - executor_callback) + return self._dispatch(incoming, executor_callback) except Exception: - # sys.exc_info() is deleted by LOG.exception(). - exc_info = sys.exc_info() - LOG.error('Exception during message handling', - exc_info=exc_info) - return NotificationResult.HANDLED + LOG.error('Exception during message handling', exc_info=True) - def _dispatch(self, ctxt, message, executor_callback=None): - """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 + def _dispatch(self, incoming, executor_callback=None): + """Dispatch notification messages to the appropriate endpoint method. """ - ctxt = self.serializer.deserialize_context(ctxt) + + messages_grouped = itertools.groupby(( + self._extract_user_message(m) + for m in incoming), lambda x: x[0]) + + requeues = set() + for priority, messages in messages_grouped: + __, raw_messages, messages = six.moves.zip(*messages) + raw_messages = list(raw_messages) + messages = list(messages) + if priority not in PRIORITIES: + LOG.warning('Unknown priority "%s"', priority) + continue + for screen, callback in self._callbacks_by_priority.get(priority, + []): + if screen: + filtered_messages = [message for message in messages + if screen.match( + message["ctxt"], + message["publisher_id"], + message["event_type"], + message["metadata"], + message["payload"])] + else: + filtered_messages = messages + + if not filtered_messages: + continue + + ret = self._exec_callback(executor_callback, callback, + filtered_messages) + if self.allow_requeue and ret == NotificationResult.REQUEUE: + requeues.update(raw_messages) + break + return requeues + + def _exec_callback(self, executor_callback, callback, *args): + if executor_callback: + ret = executor_callback(callback, *args) + else: + ret = callback(*args) + return NotificationResult.HANDLED if ret is None else ret + + def _extract_user_message(self, incoming): + ctxt = self.serializer.deserialize_context(incoming.ctxt) + message = incoming.message publisher_id = message.get('publisher_id') event_type = message.get('event_type') @@ -114,28 +143,50 @@ class NotificationDispatcher(dispatcher.DispatcherBase): 'timestamp': message.get('timestamp') } 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')) + return priority, incoming, dict(ctxt=ctxt, + publisher_id=publisher_id, + event_type=event_type, + payload=payload, + metadata=metadata) - for screen, callback in self._callbacks_by_priority.get(priority, []): - if screen and not screen.match(ctxt, publisher_id, event_type, - metadata, payload): - continue - localcontext._set_local_context(ctxt) - try: - if executor_callback: - ret = executor_callback(callback, ctxt, publisher_id, - event_type, payload, metadata) - else: - ret = callback(ctxt, publisher_id, event_type, payload, - metadata) - ret = NotificationResult.HANDLED if ret is None else ret - if self.allow_requeue and ret == NotificationResult.REQUEUE: - return ret - finally: - localcontext._clear_local_context() - return NotificationResult.HANDLED + +class NotificationDispatcher(_NotificationDispatcherBase): + """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. + """ + def _exec_callback(self, executor_callback, callback, messages): + localcontext._set_local_context( + messages[0]["ctxt"]) + try: + return super(NotificationDispatcher, self)._exec_callback( + executor_callback, callback, + messages[0]["ctxt"], + messages[0]["publisher_id"], + messages[0]["event_type"], + messages[0]["payload"], + messages[0]["metadata"]) + finally: + localcontext._clear_local_context() + + +class BatchNotificationDispatcher(_NotificationDispatcherBase): + """A message dispatcher which understands Notification messages. + + A MessageHandlingServer is constructed by passing a callable dispatcher + which is invoked with a list of message dictionaries each time 'batch_size' + messages are received or 'batch_timeout' seconds is reached. + """ + + def __init__(self, targets, endpoints, serializer, allow_requeue, + pool=None, batch_size=None, batch_timeout=None): + super(BatchNotificationDispatcher, self).__init__(targets, endpoints, + serializer, + allow_requeue, + pool) + self.batch_size = batch_size + self.batch_timeout = batch_timeout diff --git a/oslo_messaging/notify/listener.py b/oslo_messaging/notify/listener.py index 8786e9d4a..275da20c0 100644 --- a/oslo_messaging/notify/listener.py +++ b/oslo_messaging/notify/listener.py @@ -142,3 +142,46 @@ def get_notification_listener(transport, targets, endpoints, serializer, allow_requeue, pool) return msg_server.MessageHandlingServer(transport, dispatcher, executor) + + +def get_batch_notification_listener(transport, targets, endpoints, + executor='blocking', serializer=None, + allow_requeue=False, pool=None, + batch_size=None, batch_timeout=None): + """Construct a batch 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. + + If the eventlet executor is used, the threading and time library need to be + monkeypatched. + + :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 - for example + 'eventlet', 'blocking' + :type executor: str + :param serializer: an optional entity serializer + :type serializer: Serializer + :param allow_requeue: whether NotificationResult.REQUEUE support is needed + :type allow_requeue: bool + :param pool: the pool name + :type pool: str + :param batch_size: number of messages to wait before calling + endpoints callacks + :type batch_size: int + :param batch_timeout: number of seconds to wait before calling + endpoints callacks + :type batch_timeout: int + :raises: NotImplementedError + """ + transport._require_driver_features(requeue=allow_requeue) + dispatcher = notify_dispatcher.BatchNotificationDispatcher( + targets, endpoints, serializer, allow_requeue, pool, + batch_size, batch_timeout) + return msg_server.MessageHandlingServer(transport, dispatcher, executor) diff --git a/oslo_messaging/rpc/dispatcher.py b/oslo_messaging/rpc/dispatcher.py index 9d640c425..9d5dbe13d 100644 --- a/oslo_messaging/rpc/dispatcher.py +++ b/oslo_messaging/rpc/dispatcher.py @@ -131,9 +131,9 @@ class RPCDispatcher(dispatcher.DispatcherBase): return self.serializer.serialize_entity(ctxt, result) def __call__(self, incoming, executor_callback=None): - incoming.acknowledge() + incoming[0].acknowledge() return dispatcher.DispatcherExecutorContext( - incoming, self._dispatch_and_reply, + incoming[0], self._dispatch_and_reply, executor_callback=executor_callback) def _dispatch_and_reply(self, incoming, executor_callback): diff --git a/oslo_messaging/tests/drivers/test_impl_kafka.py b/oslo_messaging/tests/drivers/test_impl_kafka.py index 6f25b2c64..dcbab0a88 100644 --- a/oslo_messaging/tests/drivers/test_impl_kafka.py +++ b/oslo_messaging/tests/drivers/test_impl_kafka.py @@ -226,7 +226,7 @@ class TestKafkaListener(test_utils.BaseTestCase): listener.stop() fake_response = listener.poll() self.assertEqual(1, len(listener.conn.consume.mock_calls)) - self.assertEqual(fake_response, None) + self.assertEqual([], fake_response) class TestWithRealKafkaBroker(test_utils.BaseTestCase): @@ -251,7 +251,7 @@ class TestWithRealKafkaBroker(test_utils.BaseTestCase): self.driver.send_notification( target, fake_context, fake_message, None) - received_message = listener.poll() + received_message = listener.poll()[0] self.assertEqual(fake_context, received_message.ctxt) self.assertEqual(fake_message, received_message.message) @@ -268,7 +268,7 @@ class TestWithRealKafkaBroker(test_utils.BaseTestCase): self.driver.send_notification( target, fake_context, fake_message, None) - received_message = listener.poll() + received_message = listener.poll()[0] self.assertEqual(fake_context, received_message.ctxt) self.assertEqual(fake_message, received_message.message) diff --git a/oslo_messaging/tests/drivers/test_impl_rabbit.py b/oslo_messaging/tests/drivers/test_impl_rabbit.py index 52cbfe1cb..c8db8ea8b 100644 --- a/oslo_messaging/tests/drivers/test_impl_rabbit.py +++ b/oslo_messaging/tests/drivers/test_impl_rabbit.py @@ -432,7 +432,7 @@ class TestSendReceive(test_utils.BaseTestCase): for i in range(len(senders)): senders[i].start() - received = listener.poll() + received = listener.poll()[0] self.assertIsNotNone(received) self.assertEqual(self.ctxt, received.ctxt) self.assertEqual({'tx_id': i}, received.message) @@ -510,7 +510,7 @@ class TestPollAsync(test_utils.BaseTestCase): target = oslo_messaging.Target(topic='testtopic') listener = driver.listen(target) received = listener.poll(timeout=0.050) - self.assertIsNone(received) + self.assertEqual([], received) class TestRacyWaitForReply(test_utils.BaseTestCase): @@ -570,13 +570,13 @@ class TestRacyWaitForReply(test_utils.BaseTestCase): senders[0].start() notify_condition.wait() - msgs.append(listener.poll()) + msgs.extend(listener.poll()) self.assertEqual({'tx_id': 0}, msgs[-1].message) # Start the second guy, receive his message senders[1].start() - msgs.append(listener.poll()) + msgs.extend(listener.poll()) self.assertEqual({'tx_id': 1}, msgs[-1].message) # Reply to both in order, making the second thread queue @@ -590,7 +590,7 @@ class TestRacyWaitForReply(test_utils.BaseTestCase): # Start the 3rd guy, receive his message senders[2].start() - msgs.append(listener.poll()) + msgs.extend(listener.poll()) self.assertEqual({'tx_id': 2}, msgs[-1].message) # Verify the _send_reply was not invoked by driver: @@ -871,7 +871,7 @@ class TestReplyWireFormat(test_utils.BaseTestCase): producer.publish(msg) - received = listener.poll() + received = listener.poll()[0] self.assertIsNotNone(received) self.assertEqual(self.expected_ctxt, received.ctxt) self.assertEqual(self.expected, received.message) diff --git a/oslo_messaging/tests/drivers/zmq/test_impl_zmq.py b/oslo_messaging/tests/drivers/zmq/test_impl_zmq.py index 4062f17c7..1d710d3a9 100644 --- a/oslo_messaging/tests/drivers/zmq/test_impl_zmq.py +++ b/oslo_messaging/tests/drivers/zmq/test_impl_zmq.py @@ -52,7 +52,8 @@ class TestServerListener(object): def _run(self): try: message = self.listener.poll() - if message is not None: + if message: + message = message[0] message.acknowledge() self._received.set() self.message = message diff --git a/oslo_messaging/tests/executors/test_executor.py b/oslo_messaging/tests/executors/test_executor.py index cd338a4e3..fb91c11be 100644 --- a/oslo_messaging/tests/executors/test_executor.py +++ b/oslo_messaging/tests/executors/test_executor.py @@ -132,11 +132,14 @@ class TestExecutor(test_utils.BaseTestCase): endpoint = mock.MagicMock(return_value='result') event = None - class Dispatcher(object): + class Dispatcher(dispatcher_base.DispatcherBase): def __init__(self, endpoint): self.endpoint = endpoint self.result = "not set" + def _listen(self, transport): + pass + def callback(self, incoming, executor_callback): if executor_callback is None: result = self.endpoint(incoming.ctxt, @@ -152,7 +155,7 @@ class TestExecutor(test_utils.BaseTestCase): def __call__(self, incoming, executor_callback=None): return dispatcher_base.DispatcherExecutorContext( - incoming, self.callback, executor_callback) + incoming[0], self.callback, executor_callback) return Dispatcher(endpoint), endpoint, event, run_executor @@ -162,7 +165,7 @@ class TestExecutor(test_utils.BaseTestCase): executor = self.executor(self.conf, listener, dispatcher) incoming_message = mock.MagicMock(ctxt={}, message={'payload': 'data'}) - def fake_poll(timeout=None): + def fake_poll(timeout=None, prefetch_size=1): time.sleep(0.1) if listener.poll.call_count == 10: if event is not None: @@ -190,9 +193,9 @@ class TestExecutor(test_utils.BaseTestCase): executor = self.executor(self.conf, listener, dispatcher) incoming_message = mock.MagicMock(ctxt={}, message={'payload': 'data'}) - def fake_poll(timeout=None): + def fake_poll(timeout=None, prefetch_size=1): if listener.poll.call_count == 1: - return incoming_message + return [incoming_message] if event is not None: event.wait() executor.stop() diff --git a/oslo_messaging/tests/functional/test_functional.py b/oslo_messaging/tests/functional/test_functional.py index 5e50a399a..8ee758dfc 100644 --- a/oslo_messaging/tests/functional/test_functional.py +++ b/oslo_messaging/tests/functional/test_functional.py @@ -16,6 +16,7 @@ import uuid import concurrent.futures from oslo_config import cfg +import six.moves from testtools import matchers import oslo_messaging @@ -27,8 +28,8 @@ class CallTestCase(utils.SkipIfNoTransportURL): def setUp(self): super(CallTestCase, self).setUp(conf=cfg.ConfigOpts()) - self.conf.prog="test_prog" - self.conf.project="test_project" + self.conf.prog = "test_prog" + self.conf.project = "test_project" self.config(heartbeat_timeout_threshold=0, group='oslo_messaging_rabbit') @@ -324,3 +325,18 @@ class NotifyTestCase(utils.SkipIfNoTransportURL): self.assertEqual(expected[1], actual[0]) self.assertEqual(expected[2], actual[1]) self.assertEqual(expected[3], actual[2]) + + def test_simple_batch(self): + listener = self.useFixture( + utils.BatchNotificationFixture(self.conf, self.url, + ['test_simple_batch'], + batch_size=100, batch_timeout=2)) + notifier = listener.notifier('abc') + + for i in six.moves.range(0, 205): + notifier.info({}, 'test%s' % i, 'Hello World!') + events = listener.get_events(timeout=3) + self.assertEqual(3, len(events), events) + self.assertEqual(100, len(events[0][1])) + self.assertEqual(100, len(events[1][1])) + self.assertEqual(5, len(events[2][1])) diff --git a/oslo_messaging/tests/functional/utils.py b/oslo_messaging/tests/functional/utils.py index 99794ed37..2f9e7b9f7 100644 --- a/oslo_messaging/tests/functional/utils.py +++ b/oslo_messaging/tests/functional/utils.py @@ -293,13 +293,14 @@ class SkipIfNoTransportURL(test_utils.BaseTestCase): class NotificationFixture(fixtures.Fixture): - def __init__(self, conf, url, topics): + def __init__(self, conf, url, topics, batch=None): super(NotificationFixture, self).__init__() self.conf = conf self.url = url self.topics = topics self.events = moves.queue.Queue() self.name = str(id(self)) + self.batch = batch def setUp(self): super(NotificationFixture, self).setUp() @@ -307,10 +308,7 @@ class NotificationFixture(fixtures.Fixture): # add a special topic for internal notifications targets.append(oslo_messaging.Target(topic=self.name)) transport = self.useFixture(TransportFixture(self.conf, self.url)) - self.server = oslo_messaging.get_notification_listener( - transport.transport, - targets, - [self], 'eventlet') + self.server = self._get_server(transport, targets) self._ctrl = self.notifier('internal', topic=self.name) self._start() transport.wait() @@ -319,6 +317,12 @@ class NotificationFixture(fixtures.Fixture): self._stop() super(NotificationFixture, self).cleanUp() + def _get_server(self, transport, targets): + return oslo_messaging.get_notification_listener( + transport.transport, + targets, + [self], 'eventlet') + def _start(self): self.thread = test_utils.ServerThreadHelper(self.server) self.thread.start() @@ -366,3 +370,39 @@ class NotificationFixture(fixtures.Fixture): except moves.queue.Empty: pass return results + + +class BatchNotificationFixture(NotificationFixture): + def __init__(self, conf, url, topics, batch_size=5, batch_timeout=2): + super(BatchNotificationFixture, self).__init__(conf, url, topics) + self.batch_size = batch_size + self.batch_timeout = batch_timeout + + def _get_server(self, transport, targets): + return oslo_messaging.get_batch_notification_listener( + transport.transport, + targets, + [self], 'eventlet', + batch_timeout=self.batch_timeout, + batch_size=self.batch_size) + + def debug(self, messages): + self.events.put(['debug', messages]) + + def audit(self, messages): + self.events.put(['audit', messages]) + + def info(self, messages): + self.events.put(['info', messages]) + + def warn(self, messages): + self.events.put(['warn', messages]) + + def error(self, messages): + self.events.put(['error', messages]) + + def critical(self, messages): + self.events.put(['critical', messages]) + + def sample(self, messages): + pass # Just used for internal shutdown control diff --git a/oslo_messaging/tests/notify/test_dispatcher.py b/oslo_messaging/tests/notify/test_dispatcher.py index f0da90d89..2442d535e 100644 --- a/oslo_messaging/tests/notify/test_dispatcher.py +++ b/oslo_messaging/tests/notify/test_dispatcher.py @@ -107,7 +107,7 @@ class TestDispatcher(test_utils.BaseTestCase): sorted(dispatcher._targets_priorities)) incoming = mock.Mock(ctxt={}, message=msg) - callback = dispatcher(incoming) + callback = dispatcher([incoming]) callback.run() callback.done() @@ -144,7 +144,7 @@ class TestDispatcher(test_utils.BaseTestCase): msg['priority'] = 'what???' dispatcher = notify_dispatcher.NotificationDispatcher( [mock.Mock()], [mock.Mock()], None, allow_requeue=True, pool=None) - callback = dispatcher(mock.Mock(ctxt={}, message=msg)) + callback = dispatcher([mock.Mock(ctxt={}, message=msg)]) callback.run() callback.done() mylog.warning.assert_called_once_with('Unknown priority "%s"', @@ -246,7 +246,7 @@ class TestDispatcherFilter(test_utils.BaseTestCase): 'timestamp': '2014-03-03 18:21:04.369234', 'message_id': '99863dda-97f0-443a-a0c1-6ed317b7fd45'} incoming = mock.Mock(ctxt=self.context, message=message) - callback = dispatcher(incoming) + callback = dispatcher([incoming]) callback.run() callback.done() diff --git a/oslo_messaging/tests/notify/test_listener.py b/oslo_messaging/tests/notify/test_listener.py index 810933e31..3e534aaac 100644 --- a/oslo_messaging/tests/notify/test_listener.py +++ b/oslo_messaging/tests/notify/test_listener.py @@ -23,6 +23,7 @@ import oslo_messaging from oslo_messaging.notify import dispatcher from oslo_messaging.notify import notifier as msg_notifier from oslo_messaging.tests import utils as test_utils +import six from six.moves import mock load_tests = testscenarios.load_tests_apply_scenarios @@ -56,7 +57,7 @@ class ListenerSetupMixin(object): self.threads = [] self.lock = threading.Condition() - def info(self, ctxt, publisher_id, event_type, payload, metadata): + def info(self, *args, **kwargs): # NOTE(sileht): this run into an other thread with self.lock: self._received_msgs += 1 @@ -86,7 +87,7 @@ class ListenerSetupMixin(object): self.trackers = {} def _setup_listener(self, transport, endpoints, - targets=None, pool=None): + targets=None, pool=None, batch=False): if pool is None: tracker_name = '__default__' @@ -98,9 +99,15 @@ class ListenerSetupMixin(object): tracker = self.trackers.setdefault( tracker_name, self.ThreadTracker()) - listener = oslo_messaging.get_notification_listener( - transport, targets=targets, endpoints=[tracker] + endpoints, - allow_requeue=True, pool=pool, executor='eventlet') + if batch: + listener = oslo_messaging.get_batch_notification_listener( + transport, targets=targets, endpoints=[tracker] + endpoints, + allow_requeue=True, pool=pool, executor='eventlet', + batch_size=batch[0], batch_timeout=batch[1]) + else: + listener = oslo_messaging.get_notification_listener( + transport, targets=targets, endpoints=[tracker] + endpoints, + allow_requeue=True, pool=pool, executor='eventlet') thread = RestartableServerThread(listener) tracker.start(thread) @@ -170,6 +177,82 @@ class TestNotifyListener(test_utils.BaseTestCase, ListenerSetupMixin): else: self.assertTrue(False) + def test_batch_timeout(self): + transport = oslo_messaging.get_transport(self.conf, url='fake:') + + endpoint = mock.Mock() + endpoint.info.return_value = None + listener_thread = self._setup_listener(transport, [endpoint], + batch=(5, 1)) + + notifier = self._setup_notifier(transport) + for i in six.moves.range(12): + notifier.info({}, 'an_event.start', 'test message') + + self.wait_for_messages(3) + self.assertFalse(listener_thread.stop()) + + messages = [dict(ctxt={}, + publisher_id='testpublisher', + event_type='an_event.start', + payload='test message', + metadata={'message_id': mock.ANY, + 'timestamp': mock.ANY})] + + endpoint.info.assert_has_calls([mock.call(messages * 5), + mock.call(messages * 5), + mock.call(messages * 2)]) + + def test_batch_size(self): + transport = oslo_messaging.get_transport(self.conf, url='fake:') + + endpoint = mock.Mock() + endpoint.info.return_value = None + listener_thread = self._setup_listener(transport, [endpoint], + batch=(5, None)) + + notifier = self._setup_notifier(transport) + for i in six.moves.range(10): + notifier.info({}, 'an_event.start', 'test message') + + self.wait_for_messages(2) + self.assertFalse(listener_thread.stop()) + + messages = [dict(ctxt={}, + publisher_id='testpublisher', + event_type='an_event.start', + payload='test message', + metadata={'message_id': mock.ANY, + 'timestamp': mock.ANY})] + + endpoint.info.assert_has_calls([mock.call(messages * 5), + mock.call(messages * 5)]) + + def test_batch_size_exception_path(self): + transport = oslo_messaging.get_transport(self.conf, url='fake:') + + endpoint = mock.Mock() + endpoint.info.side_effect = [None, Exception('boom!')] + listener_thread = self._setup_listener(transport, [endpoint], + batch=(5, None)) + + notifier = self._setup_notifier(transport) + for i in six.moves.range(10): + notifier.info({}, 'an_event.start', 'test message') + + self.wait_for_messages(2) + self.assertFalse(listener_thread.stop()) + + messages = [dict(ctxt={}, + publisher_id='testpublisher', + event_type='an_event.start', + payload='test message', + metadata={'message_id': mock.ANY, + 'timestamp': mock.ANY})] + + endpoint.info.assert_has_calls([mock.call(messages * 5)]) + + def test_one_topic(self): transport = msg_notifier.get_notification_transport( self.conf, url='fake:') diff --git a/oslo_messaging/tests/rpc/test_dispatcher.py b/oslo_messaging/tests/rpc/test_dispatcher.py index f81be0b9c..672733a05 100644 --- a/oslo_messaging/tests/rpc/test_dispatcher.py +++ b/oslo_messaging/tests/rpc/test_dispatcher.py @@ -133,7 +133,7 @@ class TestDispatcher(test_utils.BaseTestCase): incoming = mock.Mock(ctxt=self.ctxt, message=self.msg) incoming.reply.side_effect = check_reply - callback = dispatcher(incoming) + callback = dispatcher([incoming]) callback.run() callback.done() diff --git a/oslo_messaging/tests/test_amqp_driver.py b/oslo_messaging/tests/test_amqp_driver.py index e73f4916c..909bc599d 100644 --- a/oslo_messaging/tests/test_amqp_driver.py +++ b/oslo_messaging/tests/test_amqp_driver.py @@ -60,7 +60,7 @@ class _ListenerThread(threading.Thread): def run(self): LOG.debug("Listener started") while self.msg_count > 0: - in_msg = self.listener.poll() + in_msg = self.listener.poll()[0] self.messages.put(in_msg) self.msg_count -= 1 if in_msg.message.get('method') == 'echo': diff --git a/tools/simulator.py b/tools/simulator.py index 3c587d5a0..5d9d54706 100755 --- a/tools/simulator.py +++ b/tools/simulator.py @@ -79,14 +79,34 @@ class LoggingNoParsingFilter(logging.Filter): return True -class NotifyEndpoint(object): - def __init__(self): +class Monitor(object): + def __init__(self, show_stats=False, *args, **kwargs): + self._count = self._prev_count = 0 + self.show_stats = show_stats + if self.show_stats: + self._monitor() + + def _monitor(self): + threading.Timer(1.0, self._monitor).start() + print ("%d msg was received per second" + % (self._count - self._prev_count)) + self._prev_count = self._count + + def info(self, *args, **kwargs): + self._count += 1 + + +class NotifyEndpoint(Monitor): + def __init__(self, *args, **kwargs): + super(NotifyEndpoint, self).__init__(*args, **kwargs) self.cache = [] def info(self, ctxt, publisher_id, event_type, payload, metadata): + super(NotifyEndpoint, self).info(ctxt, publisher_id, event_type, + payload, metadata) LOG.info('msg rcv') LOG.info("%s %s %s %s" % (ctxt, publisher_id, event_type, payload)) - if payload not in self.cache: + if not self.show_stats and payload not in self.cache: LOG.info('requeue msg') self.cache.append(payload) for i in range(15): @@ -97,8 +117,8 @@ class NotifyEndpoint(object): return messaging.NotificationResult.HANDLED -def notify_server(transport): - endpoints = [NotifyEndpoint()] +def notify_server(transport, show_stats): + endpoints = [NotifyEndpoint(show_stats)] target = messaging.Target(topic='n-t1') server = notify.get_notification_listener(transport, [target], endpoints, executor='eventlet') @@ -106,8 +126,41 @@ def notify_server(transport): server.wait() -class RpcEndpoint(object): - def __init__(self, wait_before_answer): +class BatchNotifyEndpoint(Monitor): + def __init__(self, *args, **kwargs): + super(BatchNotifyEndpoint, self).__init__(*args, **kwargs) + self.cache = [] + + def info(self, messages): + super(BatchNotifyEndpoint, self).info(messages) + self._count += len(messages) - 1 + + LOG.info('msg rcv') + LOG.info("%s" % messages) + if not self.show_stats and messages not in self.cache: + LOG.info('requeue msg') + self.cache.append(messages) + for i in range(15): + eventlet.sleep(1) + return messaging.NotificationResult.REQUEUE + else: + LOG.info('ack msg') + return messaging.NotificationResult.HANDLED + + +def batch_notify_server(transport, show_stats): + endpoints = [BatchNotifyEndpoint(show_stats)] + target = messaging.Target(topic='n-t1') + server = notify.get_batch_notification_listener( + transport, [target], + endpoints, executor='eventlet', + batch_size=1000, batch_time=5) + server.start() + server.wait() + + +class RpcEndpoint(Monitor): + def __init__(self, wait_before_answer, show_stats): self.count = None self.wait_before_answer = wait_before_answer @@ -126,27 +179,8 @@ class RpcEndpoint(object): return "OK: %s" % message -class RpcEndpointMonitor(RpcEndpoint): - def __init__(self, *args, **kwargs): - super(RpcEndpointMonitor, self).__init__(*args, **kwargs) - - self._count = self._prev_count = 0 - self._monitor() - - def _monitor(self): - threading.Timer(1.0, self._monitor).start() - print ("%d msg was received per second" - % (self._count - self._prev_count)) - self._prev_count = self._count - - def info(self, *args, **kwargs): - self._count += 1 - super(RpcEndpointMonitor, self).info(*args, **kwargs) - - def rpc_server(transport, target, wait_before_answer, executor, show_stats): - endpoint_cls = RpcEndpointMonitor if show_stats else RpcEndpoint - endpoints = [endpoint_cls(wait_before_answer)] + endpoints = [RpcEndpoint(wait_before_answer, show_stats)] server = rpc.get_rpc_server(transport, target, endpoints, executor=executor) server.start() @@ -244,6 +278,11 @@ def main(): help='notify/rpc server/client mode') server = subparsers.add_parser('notify-server') + server.add_argument('--show-stats', dest='show_stats', + type=bool, default=True) + server = subparsers.add_parser('batch-notify-server') + server.add_argument('--show-stats', dest='show_stats', + type=bool, default=True) client = subparsers.add_parser('notify-client') client.add_argument('-p', dest='threads', type=int, default=1, help='number of client threads') @@ -298,7 +337,9 @@ def main(): rpc_server(transport, target, args.wait_before_answer, args.executor, args.show_stats) elif args.mode == 'notify-server': - notify_server(transport) + notify_server(transport, args.show_stats) + elif args.mode == 'batch-notify-server': + batch_notify_server(transport, args.show_stats) elif args.mode == 'notify-client': threads_spawner(args.threads, notifier, transport, args.messages, args.wait_after_msg, args.timeout) diff --git a/tox.ini b/tox.ini index 8a1803c5c..7c30bd226 100644 --- a/tox.ini +++ b/tox.ini @@ -5,6 +5,7 @@ envlist = py34,py27,pep8,bandit setenv = VIRTUAL_ENV={envdir} OS_TEST_TIMEOUT=30 +passend = OS_* deps = -r{toxinidir}/test-requirements.txt commands = python setup.py testr --slowest --testr-args='{posargs}' From 185f94c013442d87edcea3d81b133d26fdf8a945 Mon Sep 17 00:00:00 2001 From: Mehdi Abaakouk Date: Tue, 1 Dec 2015 10:27:23 +0100 Subject: [PATCH 66/74] notif: Check the driver features in dispatcher The transport/driver features check is done into the get listener methods. So when these methods are not used the driver features checks is not done. This change moves it into the dispatcher layer to ensure the requirements are always checked. This changes a bit the behavior of when the check occurs. Before it was during the listener object initialisation. Now this when the listener server start. Change-Id: I4d81a4e8496f04d62e48317829d5dd8b942d501c --- oslo_messaging/notify/dispatcher.py | 1 + oslo_messaging/notify/listener.py | 2 -- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/oslo_messaging/notify/dispatcher.py b/oslo_messaging/notify/dispatcher.py index afd34a270..72287678d 100644 --- a/oslo_messaging/notify/dispatcher.py +++ b/oslo_messaging/notify/dispatcher.py @@ -56,6 +56,7 @@ class _NotificationDispatcherBase(dispatcher.DispatcherBase): priorities)) def _listen(self, transport): + transport._require_driver_features(requeue=self.allow_requeue) return transport._listen_for_notifications(self._targets_priorities, pool=self.pool) diff --git a/oslo_messaging/notify/listener.py b/oslo_messaging/notify/listener.py index 275da20c0..8855d5875 100644 --- a/oslo_messaging/notify/listener.py +++ b/oslo_messaging/notify/listener.py @@ -137,7 +137,6 @@ def get_notification_listener(transport, targets, endpoints, :type pool: str :raises: NotImplementedError """ - transport._require_driver_features(requeue=allow_requeue) dispatcher = notify_dispatcher.NotificationDispatcher(targets, endpoints, serializer, allow_requeue, pool) @@ -180,7 +179,6 @@ def get_batch_notification_listener(transport, targets, endpoints, :type batch_timeout: int :raises: NotImplementedError """ - transport._require_driver_features(requeue=allow_requeue) dispatcher = notify_dispatcher.BatchNotificationDispatcher( targets, endpoints, serializer, allow_requeue, pool, batch_size, batch_timeout) From 050024f7984397010c38cbfb8626112d33cbec43 Mon Sep 17 00:00:00 2001 From: Mehdi Abaakouk Date: Tue, 8 Dec 2015 08:03:55 +0000 Subject: [PATCH 67/74] Fix notifier options registration Change-Id: I37082f6f349e89af6b74e6ec5e5c416902299263 --- oslo_messaging/notify/notifier.py | 4 ++-- tools/simulator.py | 14 +++++++++----- 2 files changed, 11 insertions(+), 7 deletions(-) diff --git a/oslo_messaging/notify/notifier.py b/oslo_messaging/notify/notifier.py index fb841644f..cc4f2eb8f 100644 --- a/oslo_messaging/notify/notifier.py +++ b/oslo_messaging/notify/notifier.py @@ -90,9 +90,9 @@ class Driver(object): def get_notification_transport(conf, url=None, allowed_remote_exmods=None, aliases=None): + conf.register_opts(_notifier_opts, + group='oslo_messaging_notifications') if url is None: - conf.register_opts(_notifier_opts, - group='oslo_messaging_notifications') url = conf.oslo_messaging_notifications.transport_url return msg_transport.get_transport(conf, url, allowed_remote_exmods, aliases) diff --git a/tools/simulator.py b/tools/simulator.py index 3c587d5a0..03afe0884 100755 --- a/tools/simulator.py +++ b/tools/simulator.py @@ -281,16 +281,20 @@ def main(): _setup_logging(is_debug=args.debug) + if args.mode in ['rpc-server', 'rpc-client']: + transport = messaging.get_transport(cfg.CONF, url=args.url) + else: + transport = messaging.get_notification_transport(cfg.CONF, + url=args.url) + cfg.CONF.oslo_messaging_notifications.topics = "notif" + cfg.CONF.oslo_messaging_notifications.driver = "messaging" + target = messaging.Target(topic=args.topic, server='profiler_server') + # oslo.config defaults cfg.CONF.heartbeat_interval = 5 - cfg.CONF.oslo_messaging_notifications.topics = "notif" - cfg.CONF.oslo_messaging_notifications.driver = "messaging" cfg.CONF.prog = os.path.basename(__file__) cfg.CONF.project = 'oslo.messaging' - transport = messaging.get_transport(cfg.CONF, url=args.url) - target = messaging.Target(topic=args.topic, server='profiler_server') - if args.mode == 'rpc-server': if args.url.startswith('zmq'): cfg.CONF.rpc_zmq_matchmaker = "redis" From 148e8380ce1cc4f60716300b95104aaa2cf8c543 Mon Sep 17 00:00:00 2001 From: Mehdi Abaakouk Date: Fri, 4 Dec 2015 14:57:03 +0100 Subject: [PATCH 68/74] Fix reconnection when heartbeat is missed When a heartbeat is missing we call ensure_connection() that runs a dummy method to trigger the reconnection code in kombu. But also the code is triggered only if the channel is None. In case of the heartbeat threads we didn't reset the channel before reconnecting, so the dummy method doesn't do anything. This change sets the channel to None to ensure the connection is reestablished before the dummy method is run. Also it replaces the dummy method by checking the kombu connection object. So we are sure the connection is reestablished. Change-Id: I39f8cd23c5a5498e6f4c1aa3236ed27f3b5d7c9a Closes-bug: #1493890 --- oslo_messaging/_drivers/impl_rabbit.py | 6 ++++-- oslo_messaging/tests/drivers/test_impl_rabbit.py | 2 ++ 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/oslo_messaging/_drivers/impl_rabbit.py b/oslo_messaging/_drivers/impl_rabbit.py index b9ff36353..263b2043c 100644 --- a/oslo_messaging/_drivers/impl_rabbit.py +++ b/oslo_messaging/_drivers/impl_rabbit.py @@ -581,7 +581,10 @@ class Connection(object): LOG.info(_LI("The broker has unblocked the connection")) def ensure_connection(self): - self.ensure(method=lambda: True) + # NOTE(sileht): we reset the channel and ensure + # the kombu underlying connection works + self._set_current_channel(None) + self.ensure(method=lambda: self.connection.connection) def ensure(self, method, retry=None, recoverable_error_callback=None, error_callback=None, @@ -732,7 +735,6 @@ class Connection(object): for tag, consumer in enumerate(self._consumers): consumer.cancel(tag=tag) except recoverable_errors: - self._set_current_channel(None) self.ensure_connection() self._consumers = [] diff --git a/oslo_messaging/tests/drivers/test_impl_rabbit.py b/oslo_messaging/tests/drivers/test_impl_rabbit.py index 52cbfe1cb..114fc52e1 100644 --- a/oslo_messaging/tests/drivers/test_impl_rabbit.py +++ b/oslo_messaging/tests/drivers/test_impl_rabbit.py @@ -896,6 +896,8 @@ class RpcKombuHATestCase(test_utils.BaseTestCase): self.useFixture(mockpatch.Patch( 'kombu.connection.Connection.connect', side_effect=self.kombu_connect)) + self.useFixture(mockpatch.Patch( + 'kombu.connection.Connection.connection')) self.useFixture(mockpatch.Patch( 'kombu.connection.Connection.channel')) From 185693a6ed57e02b2f94b0fb8f14a91471605969 Mon Sep 17 00:00:00 2001 From: Mehdi Abaakouk Date: Wed, 9 Dec 2015 11:23:52 +0100 Subject: [PATCH 69/74] Improves comment Change-Id: Idc8002e6d622435aac48304857985c0f82be3e32 --- oslo_messaging/rpc/dispatcher.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/oslo_messaging/rpc/dispatcher.py b/oslo_messaging/rpc/dispatcher.py index 6913e7afe..87dd4090d 100644 --- a/oslo_messaging/rpc/dispatcher.py +++ b/oslo_messaging/rpc/dispatcher.py @@ -145,7 +145,9 @@ class RPCDispatcher(object): e.exc_info[1]) incoming.reply(failure=e.exc_info, log_failure=False) except Exception as e: - # sys.exc_info() is deleted by LOG.exception(). + # current sys.exc_info() content can be overriden + # by another exception raise by a log handler during + # LOG.exception(). So keep a copy and delete it later. exc_info = sys.exc_info() LOG.error(_LE('Exception during message handling: %s'), e, exc_info=exc_info) From e72599435c59c09277a9da7686b32aa4f9df7ba4 Mon Sep 17 00:00:00 2001 From: Mehdi Abaakouk Date: Wed, 9 Dec 2015 18:49:19 +0100 Subject: [PATCH 70/74] Don't trigger error_callback for known exc When AMQPDestinationNotFound is raised, we must not call the error_callback method. The exception is logged only if needed in upper layer (amqpdriver.py). Related-bug: #1524418 Change-Id: Ic1ddec2d13172532dbaa572d04a4c22c97ac4fe7 --- oslo_messaging/_drivers/impl_rabbit.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/oslo_messaging/_drivers/impl_rabbit.py b/oslo_messaging/_drivers/impl_rabbit.py index b9ff36353..f8b4aea7c 100644 --- a/oslo_messaging/_drivers/impl_rabbit.py +++ b/oslo_messaging/_drivers/impl_rabbit.py @@ -700,6 +700,10 @@ class Connection(object): 'tries: %(err_str)s') % info LOG.error(msg) raise exceptions.MessageDeliveryFailure(msg) + except rpc_amqp.AMQPDestinationNotFound: + # NOTE(sileht): we must reraise this without + # trigger error_callback + raise except Exception as exc: error_callback and error_callback(exc) raise From 3ee86964fa460882d8fcac8686edd0e6bfb12008 Mon Sep 17 00:00:00 2001 From: Mehdi Abaakouk Date: Wed, 9 Dec 2015 19:37:40 +0100 Subject: [PATCH 71/74] Revert "default of kombu_missing_consumer_retry_timeout" This reverts commit 8c03a6db6c0396099e7425834998da5478a1df7c. Closes-bug: #1524418 Change-Id: I35538a6c15d6402272e4513bc1beaa537b0dd7b9 --- oslo_messaging/_drivers/impl_rabbit.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/oslo_messaging/_drivers/impl_rabbit.py b/oslo_messaging/_drivers/impl_rabbit.py index f8b4aea7c..9f7a327a2 100644 --- a/oslo_messaging/_drivers/impl_rabbit.py +++ b/oslo_messaging/_drivers/impl_rabbit.py @@ -75,7 +75,7 @@ rabbit_opts = [ 'AMQP consumer cancel notification.'), cfg.IntOpt('kombu_missing_consumer_retry_timeout', deprecated_name="kombu_reconnect_timeout", - default=5, + default=60, help='How long to wait a missing client beforce abandoning to ' 'send it its replies. This value should not be longer ' 'than rpc_response_timeout.'), From 46daf858144202a072c4bf8580aeafec11d20e13 Mon Sep 17 00:00:00 2001 From: Davanum Srinivas Date: Fri, 11 Dec 2015 11:04:13 +0300 Subject: [PATCH 72/74] Cleanup parameter docstrings Change-Id: I301fdd51446bf0c0a6dd0d05b26da0556db8367d --- oslo_messaging/_drivers/common.py | 2 +- oslo_messaging/server.py | 24 ------------------------ 2 files changed, 1 insertion(+), 25 deletions(-) diff --git a/oslo_messaging/_drivers/common.py b/oslo_messaging/_drivers/common.py index 02c04805d..85d814da6 100644 --- a/oslo_messaging/_drivers/common.py +++ b/oslo_messaging/_drivers/common.py @@ -109,7 +109,7 @@ class Timeout(RPCException): :param info: Extra info to convey to the user :param topic: The topic that the rpc call was sent to - :param rpc_method_name: The name of the rpc method being + :param method: The name of the rpc method being called """ self.info = info diff --git a/oslo_messaging/server.py b/oslo_messaging/server.py index 804bcf105..6b4e50a0c 100644 --- a/oslo_messaging/server.py +++ b/oslo_messaging/server.py @@ -342,14 +342,6 @@ class MessageHandlingServer(service.ServiceBase, _OrderedTaskRunner): registering a callback with an event loop. Similarly, the executor may choose to dispatch messages in a new thread, coroutine or simply the current thread. - - :param log_after: Emit a log message if waiting longer than `log_after` - seconds to run this task. If set to zero, no log - message will be emitted. Defaults to 30 seconds. - :type log_after: int - :param timeout: Raise `TaskTimeout` if the task has to wait longer than - `timeout` seconds before executing. - :type timeout: int """ # Warn that restarting will be deprecated if self._started: @@ -382,14 +374,6 @@ class MessageHandlingServer(service.ServiceBase, _OrderedTaskRunner): the server. However, the server may still be in the process of handling some messages, and underlying driver resources associated to this server are still in use. See 'wait' for more details. - - :param log_after: Emit a log message if waiting longer than `log_after` - seconds to run this task. If set to zero, no log - message will be emitted. Defaults to 30 seconds. - :type log_after: int - :param timeout: Raise `TaskTimeout` if the task has to wait longer than - `timeout` seconds before executing. - :type timeout: int """ self._executor_obj.stop() @@ -403,14 +387,6 @@ class MessageHandlingServer(service.ServiceBase, _OrderedTaskRunner): Once it's finished, the underlying driver resources associated to this server are released (like closing useless network connections). - - :param log_after: Emit a log message if waiting longer than `log_after` - seconds to run this task. If set to zero, no log - message will be emitted. Defaults to 30 seconds. - :type log_after: int - :param timeout: Raise `TaskTimeout` if the task has to wait longer than - `timeout` seconds before executing. - :type timeout: int """ try: self._executor_obj.wait() From 17ccb2306d03a74304c57d31716a54ba2b3b4311 Mon Sep 17 00:00:00 2001 From: Mehdi Abaakouk Date: Fri, 11 Dec 2015 10:59:54 +0100 Subject: [PATCH 73/74] Move to debug a too verbose log When a client is gone (died/restart) and somes replies cannot be sent because the the exchange of this client will never comeback. We log one message per reply every 0.25 messages during 60 seconds. When the only useful log is the one where we decide to drop this replies. This change moves the less important message to debug level. Change-Id: I508787c0db4dcec2c0027b89eb4e65c4f98022b9 Related-bug: #1524418 --- oslo_messaging/_drivers/amqpdriver.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/oslo_messaging/_drivers/amqpdriver.py b/oslo_messaging/_drivers/amqpdriver.py index 7eaf0c8ed..21cda4373 100644 --- a/oslo_messaging/_drivers/amqpdriver.py +++ b/oslo_messaging/_drivers/amqpdriver.py @@ -97,11 +97,11 @@ class AMQPIncomingMessage(base.IncomingMessage): return except rpc_amqp.AMQPDestinationNotFound: if timer.check_return() > 0: - LOG.info(_LI("The reply %(msg_id)s cannot be sent " - "%(reply_q)s reply queue don't exist, " - "retrying...") % { - 'msg_id': self.msg_id, - 'reply_q': self.reply_q}) + LOG.debug(("The reply %(msg_id)s cannot be sent " + "%(reply_q)s reply queue don't exist, " + "retrying...") % { + 'msg_id': self.msg_id, + 'reply_q': self.reply_q}) time.sleep(0.25) else: self._obsolete_reply_queues.add(self.reply_q, self.msg_id) From a5d78891745b6b9e5827271dc305f00acae1392f Mon Sep 17 00:00:00 2001 From: OpenStack Proposal Bot Date: Fri, 11 Dec 2015 15:24:05 +0000 Subject: [PATCH 74/74] Updated from global requirements Change-Id: Ifd78016c067740477a82dbe06d74d5944ba91893 --- requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/requirements.txt b/requirements.txt index 65d3200a5..47b6a8d8d 100644 --- a/requirements.txt +++ b/requirements.txt @@ -8,7 +8,7 @@ futurist>=0.1.2 # Apache-2.0 oslo.config>=2.7.0 # Apache-2.0 oslo.context>=0.2.0 # Apache-2.0 oslo.log>=1.12.0 # Apache-2.0 -oslo.utils>=2.8.0 # Apache-2.0 +oslo.utils!=3.1.0,>=2.8.0 # Apache-2.0 oslo.serialization>=1.10.0 # Apache-2.0 oslo.service>=1.0.0 # Apache-2.0 oslo.i18n>=1.5.0 # Apache-2.0