diff --git a/doc/source/index.rst b/doc/source/index.rst index 775f5a818..3c609cd87 100644 --- a/doc/source/index.rst +++ b/doc/source/index.rst @@ -2,7 +2,7 @@ oslo.messaging ============== The Oslo messaging API supports RPC and notifications over a number of -different messsaging transports. +different messaging transports. Contents ======== diff --git a/oslo_messaging/_cmd/zmq_receiver.py b/oslo_messaging/_cmd/zmq_receiver.py deleted file mode 100644 index d65576ac4..000000000 --- a/oslo_messaging/_cmd/zmq_receiver.py +++ /dev/null @@ -1,40 +0,0 @@ -#!/usr/bin/env python - -# Copyright 2011 OpenStack Foundation -# -# 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 eventlet -eventlet.monkey_patch() - -import contextlib -import sys - -from oslo_config import cfg -from oslo_log import log - -from oslo_messaging._drivers import impl_zmq -from oslo_messaging._executors import base # FIXME(markmc) - -CONF = cfg.CONF -CONF.register_opts(impl_zmq.zmq_opts) -CONF.register_opts(base._pool_opts) - - -def main(): - CONF(sys.argv[1:], project='oslo') - log.setup(CONF, 'oslo.messaging') - - with contextlib.closing(impl_zmq.ZmqProxy(CONF)) as reactor: - reactor.consume_in_thread() - reactor.wait() diff --git a/oslo_messaging/_drivers/base.py b/oslo_messaging/_drivers/base.py index 0c0008269..607821faa 100644 --- a/oslo_messaging/_drivers/base.py +++ b/oslo_messaging/_drivers/base.py @@ -75,8 +75,7 @@ class Listener(object): def cleanup(self): """Cleanup listener. - Close connection used by listener if any. For some listeners like - zmq there is no connection so no need to close connection. + Close connection (socket) used by listener if any. As this is listener specific method, overwrite it in to derived class if cleanup of listener required. """ diff --git a/oslo_messaging/_drivers/impl_zmq.py b/oslo_messaging/_drivers/impl_zmq.py index 420626424..e39b7cfba 100644 --- a/oslo_messaging/_drivers/impl_zmq.py +++ b/oslo_messaging/_drivers/impl_zmq.py @@ -12,39 +12,22 @@ # License for the specific language governing permissions and limitations # under the License. -import collections import logging -import os import pprint -import re import socket -import sys import threading -import types -import uuid -import eventlet -import greenlet from oslo_config import cfg -from oslo_serialization import jsonutils -from oslo_utils import excutils -from oslo_utils import importutils -import six -from six import moves 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.server import zmq_server from oslo_messaging._executors import impl_pooledexecutor # FIXME(markmc) -from oslo_messaging._i18n import _, _LE, _LI, _LW -from oslo_messaging._drivers import pool -zmq = importutils.try_import('eventlet.green.zmq') - -# for convenience, are not modified. pformat = pprint.pformat -Timeout = eventlet.timeout.Timeout LOG = logging.getLogger(__name__) RPCException = rpc_common.RPCException @@ -58,10 +41,17 @@ zmq_opts = [ # The module.Class to use for matchmaking. cfg.StrOpt( 'rpc_zmq_matchmaker', - default='local', + default='redis', help='MatchMaker driver.', ), + cfg.BoolOpt('rpc_zmq_all_req_rep', + default=True, + help='Use REQ/REP pattern for all methods CALL/CAST/FANOUT.'), + + cfg.StrOpt('rpc_zmq_concurrency', default='eventlet', + help='Type of concurrency used. Either "native" or "eventlet"'), + # The following port is unassigned by IANA as of 2012-05-21 cfg.IntOpt('rpc_zmq_port', default=9501, help='ZeroMQ receiver listening port.'), @@ -85,1052 +75,180 @@ zmq_opts = [ default=30, help='Seconds to wait before a cast expires (TTL). ' 'Only supported by impl_zmq.'), + + cfg.IntOpt('rpc_poll_timeout', + default=1, + help='The default number of seconds that poll should wait. ' + 'Poll raises timeout exception when timeout expired.'), ] -CONF = cfg.CONF -matchmaker = None # memoized matchmaker object +class LazyDriverItem(object): - -def _serialize(data): - """Serialization wrapper. - - We prefer using JSON, but it cannot encode all types. - Error if a developer passes us bad data. - """ - try: - return jsonutils.dumps(data, ensure_ascii=True) - except TypeError: - with excutils.save_and_reraise_exception(): - LOG.error(_LE("JSON serialization failed.")) - - -def _deserialize(data): - """Deserialization wrapper.""" - LOG.debug("Deserializing: %r", data) - return jsonutils.loads(data) - - -class ZmqSocket(object): - """A tiny wrapper around ZeroMQ. - - Simplifies the send/recv protocol and connection management. - Can be used as a Context (supports the 'with' statement). - """ - - def __init__(self, addr, zmq_type, bind=True, subscribe=None, ctxt=None): - self.ctxt = ctxt or zmq.Context(CONF.rpc_zmq_contexts) - self.sock = self.ctxt.socket(zmq_type) - - # Enable IPv6-support in libzmq. - # When IPv6 is enabled, a socket will connect to, or accept - # connections from, both IPv4 and IPv6 hosts. - try: - self.sock.ipv6 = True - except AttributeError: - # NOTE(dhellmann): Sometimes the underlying library does - # not recognize the IPV6 option. There's nothing we can - # really do in that case, so ignore the error and keep - # trying to work. - pass - - self.addr = addr - self.type = zmq_type - self.subscriptions = [] - - # Support failures on sending/receiving on wrong socket type. - self.can_recv = zmq_type in (zmq.PULL, zmq.SUB) - self.can_send = zmq_type in (zmq.PUSH, zmq.PUB) - self.can_sub = zmq_type in (zmq.SUB, ) - - # Support list, str, & None for subscribe arg (cast to list) - do_sub = { - list: subscribe, - str: [subscribe], - type(None): [] - }[type(subscribe)] - - for f in do_sub: - self.subscribe(f) - - str_data = {'addr': addr, 'type': self.socket_s(), - 'subscribe': subscribe, 'bind': bind} - - LOG.debug("Connecting to %(addr)s with %(type)s", str_data) - LOG.debug("-> Subscribed to %(subscribe)s", str_data) - LOG.debug("-> bind: %(bind)s", str_data) - - try: - if bind: - self.sock.bind(addr) - else: - self.sock.connect(addr) - except Exception: - raise RPCException(_("Could not open socket.")) - - def socket_s(self): - """Get socket type as string.""" - t_enum = ('PUSH', 'PULL', 'PUB', 'SUB', 'REP', 'REQ', 'ROUTER', - 'DEALER') - return dict(map(lambda t: (getattr(zmq, t), t), t_enum))[self.type] - - def subscribe(self, msg_filter): - """Subscribe.""" - if not self.can_sub: - raise RPCException("Cannot subscribe on this socket.") - LOG.debug("Subscribing to %s", msg_filter) - - try: - arg = msg_filter - if six.PY3: - arg = arg.encode('utf-8') - self.sock.setsockopt(zmq.SUBSCRIBE, arg) - except Exception: - return - - self.subscriptions.append(msg_filter) - - def unsubscribe(self, msg_filter): - """Unsubscribe.""" - if msg_filter not in self.subscriptions: - return - arg = msg_filter - if six.PY3: - arg = arg.encode('utf-8') - self.sock.setsockopt(zmq.UNSUBSCRIBE, arg) - self.subscriptions.remove(msg_filter) - - @property - def closed(self): - return self.sock is None or self.sock.closed - - def close(self): - if self.sock is None or self.sock.closed: - return - - # We must unsubscribe, or we'll leak descriptors. - if self.subscriptions: - for f in self.subscriptions: - try: - self.sock.setsockopt(zmq.UNSUBSCRIBE, f) - except Exception: - pass - self.subscriptions = [] - - try: - # Default is to linger - self.sock.close() - self.ctxt.term() - except Exception: - # While this is a bad thing to happen, - # it would be much worse if some of the code calling this - # were to fail. For now, lets log, and later evaluate - # if we can safely raise here. - LOG.error("ZeroMQ socket could not be closed.") - self.sock = None - - def recv(self, **kwargs): - if not self.can_recv: - raise RPCException(_("You cannot recv on this socket.")) - return self.sock.recv_multipart(**kwargs) - - def send(self, data, **kwargs): - if not self.can_send: - raise RPCException(_("You cannot send on this socket.")) - self.sock.send_multipart(data, **kwargs) - - -class ZmqClient(object): - """Client for ZMQ sockets.""" - - def __init__(self, addr, ctxt=None): - self.address = addr - self.outq = ZmqSocket(addr, zmq.PUSH, bind=False, ctxt=ctxt) - - def cast(self, msg_id, topic, data, envelope): - msg_id = msg_id or '0' - - if six.PY3: - msg_id = msg_id.encode('utf-8') - - if not envelope: - data = _serialize(data) - if six.PY3: - data = data.encode('utf-8') - data = (msg_id, topic, b'cast', data) - self.outq.send([bytes(item) for item in data]) - return - - rpc_envelope = rpc_common.serialize_msg(data[1]) - zmq_msg = moves.reduce(lambda x, y: x + y, rpc_envelope.items()) - data = (msg_id, topic, b'impl_zmq_v2', data[0]) + zmq_msg - self.outq.send([bytes(item) for item in data]) - - def close(self): - self.outq.close() - - -class ZmqClientContext(object): - """This is essentially a wrapper around ZmqClient that supports 'with'. - It can also return a new ZmqClient, 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 ZmqClients 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 client to the pool. - - Based on amqp.ConnectionContext. - """ - - def __init__(self, address, connection_pool=None, pooled=False): - self.connection = None - self.connection_pool = connection_pool - self.pooled = pooled - if self.pooled and self.connection_pool is not None: - self.connection = self.connection_pool.get(address) - else: - self.connection = ZmqClient(address) - - def __enter__(self): - """When with ZmqClientContext() is used, return self.""" - return self - - def _done(self): - """If the client 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 and self.connection_pool is not None: - # Reset the connection so it's ready for the next caller - # to grab from the pool - 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 client. Make sure we cleaned up.""" - self._done() - - def close(self): - """Caller is done with this client.""" - self._done() - - def __getattr__(self, key): - """Proxy all other calls to the ZmqClient 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.""" - def __init__(self, **kwargs): - self.replies = [] - super(RpcContext, self).__init__(**kwargs) - - def deepcopy(self): - values = self.to_dict() - values['replies'] = self.replies - return self.__class__(**values) - - def reply(self, reply=None, failure=None, ending=False): - if ending: - return - self.replies.append(reply) - - @classmethod - def marshal(self, ctx): - if not isinstance(ctx, dict): - ctx_data = ctx.to_dict() - else: - ctx_data = ctx - return _serialize(ctx_data) - - @classmethod - def unmarshal(self, data): - return RpcContext.from_dict(_deserialize(data)) - - -class InternalContext(object): - """Used by ConsumerBase as a private context for - methods.""" - - def __init__(self, proxy): - self.proxy = proxy - self.msg_waiter = None - - def _get_response(self, ctx, proxy, topic, data): - """Process a curried message and cast the result to topic.""" - LOG.debug("Running func with context: %s", ctx.to_dict()) - data.setdefault('version', None) - data.setdefault('args', {}) - - try: - if not data.get("method"): - raise KeyError - result = proxy.dispatch(ctx, data) - return ConsumerBase.normalize_reply(result, ctx.replies) - except greenlet.GreenletExit: - # ignore these since they are just from shutdowns - pass - except rpc_common.ClientException as e: - LOG.debug("Expected exception during message handling (%s)", - e._exc_info[1]) - return {'exc': - rpc_common.serialize_remote_exception(e._exc_info, - log_failure=False)} - except Exception: - LOG.error(_LE("Exception during message handling")) - return {'exc': - rpc_common.serialize_remote_exception(sys.exc_info())} - - def reply(self, driver, ctx, proxy, - msg_id=None, context=None, topic=None, msg=None): - """Reply to a casted call.""" - # NOTE(ewindisch): context kwarg exists for Grizzly compat. - # this may be able to be removed earlier than - # 'I' if ConsumerBase.process were refactored. - if type(msg) is list: - payload = msg[-1] - else: - payload = msg - - response = ConsumerBase.normalize_reply( - self._get_response(ctx, proxy, topic, payload), - ctx.replies) - - LOG.debug("Sending reply") - _multi_send(driver, _cast, ctx, topic, { - 'method': '-process_reply', - 'args': { - 'msg_id': msg_id, # Include for Folsom compat. - 'response': response - } - }, _msg_id=msg_id, pooled=True) - - -class ConsumerBase(object): - """Base Consumer.""" - - def __init__(self, driver): - self.driver = driver - self.private_ctx = InternalContext(None) - - @classmethod - def normalize_reply(self, result, replies): - # TODO(ewindisch): re-evaluate and document this method. - if isinstance(result, types.GeneratorType): - return list(result) - elif replies: - return replies - else: - return [result] - - def process(self, proxy, ctx, data): - data.setdefault('version', None) - data.setdefault('args', {}) - - # Method starting with - are - # processed internally. (non-valid method name) - method = data.get('method') - # Internal method - # uses internal context for safety. - if method == '-reply': - self.private_ctx.reply(self.driver, ctx, proxy, **data['args']) - return - - proxy.dispatch(ctx, data) - - -class ZmqBaseReactor(ConsumerBase): - """A consumer class implementing a centralized casting broker (PULL-PUSH). - - Used for RoundRobin requests. - """ - - def __init__(self, conf, driver=None): - super(ZmqBaseReactor, self).__init__(driver) - - self.driver = driver - self.proxies = {} - self.threads = [] - self.sockets = [] - self.subscribe = {} - - self.pool = eventlet.greenpool.GreenPool( - conf.executor_thread_pool_size) - - def register(self, proxy, in_addr, zmq_type_in, - in_bind=True, subscribe=None): - - LOG.info(_LI("Registering reactor")) - - if zmq_type_in not in (zmq.PULL, zmq.SUB): - raise RPCException("Bad input socktype") - - # Items push in. - inq = ZmqSocket(in_addr, zmq_type_in, bind=in_bind, - subscribe=subscribe) - - self.proxies[inq] = proxy - self.sockets.append(inq) - - LOG.info(_LI("In reactor registered")) - - def consume_in_thread(self): - def _consume(sock): - LOG.info(_LI("Consuming socket")) - while not sock.closed: - self.consume(sock) - - for k in self.proxies.keys(): - self.threads.append( - self.pool.spawn(_consume, k) - ) - - def wait(self): - for t in self.threads: - t.wait() - - def close(self): - for t in self.threads: - t.kill() - - for s in self.sockets: - s.close() - - -class ZmqProxy(ZmqBaseReactor): - """A consumer class implementing a topic-based proxy. - - Forwards to IPC sockets. - """ - - def __init__(self, conf): - super(ZmqProxy, self).__init__(conf) - pathsep = set((os.path.sep or '', os.path.altsep or '', '/', '\\')) - self.badchars = re.compile(r'[%s]' % re.escape(''.join(pathsep))) - - self.topic_proxy = {} - - def consume(self, sock): - ipc_dir = CONF.rpc_zmq_ipc_dir - - data = sock.recv(copy=False) - topic = data[1].bytes - if six.PY3: - topic = topic.decode('utf-8') - - if topic.startswith('fanout~'): - sock_type = zmq.PUB - topic = topic.split('.', 1)[0] - elif topic.startswith('zmq_replies'): - sock_type = zmq.PUB - else: - sock_type = zmq.PUSH - - if topic not in self.topic_proxy: - def publisher(waiter): - LOG.info(_LI("Creating proxy for topic: %s"), topic) - - try: - # The topic is received over the network, - # don't trust this input. - if self.badchars.search(topic) is not None: - emsg = _("Topic contained dangerous characters.") - LOG.warn(emsg) - raise RPCException(emsg) - - out_sock = ZmqSocket("ipc://%s/zmq_topic_%s" % - (ipc_dir, topic), - sock_type, bind=True) - except RPCException: - waiter.send_exception(*sys.exc_info()) - return - - self.topic_proxy[topic] = eventlet.queue.LightQueue( - CONF.rpc_zmq_topic_backlog) - self.sockets.append(out_sock) - - # It takes some time for a pub socket to open, - # before we can have any faith in doing a send() to it. - if sock_type == zmq.PUB: - eventlet.sleep(.5) - - waiter.send(True) - - while(True): - data = self.topic_proxy[topic].get() - out_sock.send(data, copy=False) - - wait_sock_creation = eventlet.event.Event() - eventlet.spawn(publisher, wait_sock_creation) - - try: - wait_sock_creation.wait() - except RPCException: - LOG.error(_LE("Topic socket file creation failed.")) - return - - try: - self.topic_proxy[topic].put_nowait(data) - except eventlet.queue.Full: - LOG.error(_LE("Local per-topic backlog buffer full for topic " - "%s. Dropping message."), topic) - - def consume_in_thread(self): - """Runs the ZmqProxy service.""" - ipc_dir = CONF.rpc_zmq_ipc_dir - consume_in = "tcp://%s:%s" % \ - (CONF.rpc_zmq_bind_address, - CONF.rpc_zmq_port) - consumption_proxy = InternalContext(None) - - try: - os.makedirs(ipc_dir) - except os.error: - if not os.path.isdir(ipc_dir): - with excutils.save_and_reraise_exception(): - LOG.error(_LE("Required IPC directory does not exist at" - " %s"), ipc_dir) - try: - self.register(consumption_proxy, - consume_in, - zmq.PULL) - except zmq.ZMQError: - if os.access(ipc_dir, os.X_OK): - with excutils.save_and_reraise_exception(): - LOG.error(_LE("Permission denied to IPC directory at" - " %s"), ipc_dir) - with excutils.save_and_reraise_exception(): - LOG.error(_LE("Could not create ZeroMQ receiver daemon. " - "Socket may already be in use.")) - - super(ZmqProxy, self).consume_in_thread() - - -def unflatten_envelope(packenv): - """Unflattens the RPC envelope. - - Takes a list and returns a dictionary. - i.e. [1,2,3,4] => {1: 2, 3: 4} - """ - i = iter(packenv) - h = {} - try: - while True: - k = six.next(i) - h[k] = six.next(i) - except StopIteration: - return h - - -class ZmqReactor(ZmqBaseReactor): - """A consumer class implementing a consumer for messages. - - Can also be used as a 1:1 proxy - """ - - def __init__(self, conf, driver): - super(ZmqReactor, self).__init__(conf, driver) - - def consume(self, sock): - # TODO(ewindisch): use zero-copy (i.e. references, not copying) - data = sock.recv() - LOG.debug("CONSUMER RECEIVED DATA: %s", data) - - proxy = self.proxies[sock] - - if data[2] == b'cast': # Legacy protocol - packenv = data[3] - - ctx, msg = _deserialize(packenv) - request = rpc_common.deserialize_msg(msg) - ctx = RpcContext.unmarshal(ctx) - elif data[2] == b'impl_zmq_v2': - packenv = data[4:] - - msg = unflatten_envelope(packenv) - request = rpc_common.deserialize_msg(msg) - - # Unmarshal only after verifying the message. - ctx = RpcContext.unmarshal(data[3]) - else: - LOG.error(_LE("ZMQ Envelope version unsupported or unknown.")) - return - - self.pool.spawn_n(self.process, proxy, ctx, request) - - -class Connection(rpc_common.Connection): - """Manages connections and threads.""" - - def __init__(self, conf, driver): - self.topics = [] - self.reactor = ZmqReactor(conf, driver) - - def create_consumer(self, topic, proxy, fanout=False): - # Register with matchmaker. - _get_matchmaker().register(topic, CONF.rpc_zmq_host) - - # Subscription scenarios - if fanout: - sock_type = zmq.SUB - subscribe = ('', fanout)[type(fanout) == str] - topic = 'fanout~' + topic.split('.', 1)[0] - else: - sock_type = zmq.PULL - subscribe = None - topic = '.'.join((topic.split('.', 1)[0], CONF.rpc_zmq_host)) - - if topic in self.topics: - LOG.info(_LI("Skipping topic registration. Already registered.")) - return - - # Receive messages from (local) proxy - inaddr = "ipc://%s/zmq_topic_%s" % \ - (CONF.rpc_zmq_ipc_dir, topic) - - LOG.debug("Consumer is a zmq.%s", - ['PULL', 'SUB'][sock_type == zmq.SUB]) - - self.reactor.register(proxy, inaddr, sock_type, - subscribe=subscribe, in_bind=False) - self.topics.append(topic) - - def close(self): - mm = _get_matchmaker() - mm.stop_heartbeat() - for topic in self.topics: - try: - mm.unregister(topic, CONF.rpc_zmq_host) - except Exception as err: - LOG.error(_LE('Unable to unregister topic %(topic)s' - ' from matchmaker: %(err)s') % - {'topic': topic, 'err': err}) - - self.reactor.close() - self.topics = [] - - def wait(self): - self.reactor.wait() - - def consume_in_thread(self): - _get_matchmaker().start_heartbeat() - self.reactor.consume_in_thread() - - -def _cast(driver, addr, context, topic, msg, timeout=None, envelope=False, - _msg_id=None, allowed_remote_exmods=None, pooled=False): - allowed_remote_exmods = allowed_remote_exmods or [] - timeout_cast = timeout or CONF.rpc_cast_timeout - payload = [RpcContext.marshal(context), msg] - if six.PY3: - topic = topic.encode('utf-8') - - with Timeout(timeout_cast, exception=rpc_common.Timeout): - with driver.get_connection(addr, pooled) as conn: - try: - # assumes cast can't return an exception - conn.cast(_msg_id, topic, payload, envelope) - except zmq.ZMQError: - raise RPCException("Cast failed. ZMQ Socket Exception") - - -def _call(driver, addr, context, topic, msg, timeout=None, - envelope=False, allowed_remote_exmods=None, pooled=False): - allowed_remote_exmods = allowed_remote_exmods or [] - # timeout_response is how long we wait for a response - timeout = timeout or CONF.rpc_response_timeout - - # The msg_id is used to track replies. - msg_id = uuid.uuid4().hex - - # Replies always come into the reply service. - reply_topic = "zmq_replies.%s" % CONF.rpc_zmq_host - - LOG.debug("Creating payload") - # Curry the original request into a reply method. - mcontext = RpcContext.marshal(context) - payload = { - 'method': '-reply', - 'args': { - 'msg_id': msg_id, - 'topic': reply_topic, - # TODO(ewindisch): safe to remove mcontext in I. - 'msg': [mcontext, msg] - } - } - - LOG.debug("Creating queue socket for reply waiter") - - # Messages arriving async. - # TODO(ewindisch): have reply consumer with dynamic subscription mgmt - with Timeout(timeout, exception=rpc_common.Timeout): - try: - msg_waiter = ZmqSocket( - "ipc://%s/zmq_topic_zmq_replies.%s" % - (CONF.rpc_zmq_ipc_dir, - CONF.rpc_zmq_host), - zmq.SUB, subscribe=msg_id, bind=False - ) - - LOG.debug("Sending cast: %s", topic) - _cast(driver, addr, context, topic, payload, envelope=envelope, - pooled=pooled) - - LOG.debug("Cast sent; Waiting reply") - # Blocks until receives reply - msg = msg_waiter.recv() - if msg is None: - raise rpc_common.Timeout() - LOG.debug("Received message: %s", msg) - LOG.debug("Unpacking response") - - if msg[2] == b'cast': # Legacy version - raw_msg = _deserialize(msg[-1])[-1] - elif msg[2] == b'impl_zmq_v2': - rpc_envelope = unflatten_envelope(msg[4:]) - raw_msg = rpc_common.deserialize_msg(rpc_envelope) - else: - raise rpc_common.UnsupportedRpcEnvelopeVersion( - _("Unsupported or unknown ZMQ envelope returned.")) - - responses = raw_msg['args']['response'] - # ZMQError trumps the Timeout error. - except zmq.ZMQError: - raise RPCException("ZMQ Socket Error") - except (IndexError, KeyError): - raise RPCException(_("RPC Message Invalid.")) - finally: - if 'msg_waiter' in vars(): - msg_waiter.close() - - # It seems we don't need to do all of the following, - # but perhaps it would be useful for multicall? - # One effect of this is that we're checking all - # responses for Exceptions. - for resp in responses: - if isinstance(resp, dict) and 'exc' in resp: - raise rpc_common.deserialize_remote_exception( - resp['exc'], allowed_remote_exmods) - - return responses[-1] - - -def _multi_send(driver, method, context, topic, msg, timeout=None, - envelope=False, _msg_id=None, allowed_remote_exmods=None, - pooled=False): - """Wraps the sending of messages. - - Dispatches to the matchmaker and sends message to all relevant hosts. - """ - allowed_remote_exmods = allowed_remote_exmods or [] - conf = CONF - LOG.debug(' '.join(map(pformat, (topic, msg)))) - - queues = _get_matchmaker().queues(topic) - LOG.debug("Sending message(s) to: %s", queues) - - # Don't stack if we have no matchmaker results - if not queues: - warn_log = _LW("No matchmaker results. Not sending.") - - if method.__name__ == '_cast': - LOG.warn(warn_log) - return - - # While not strictly a timeout, callers know how to handle - # this exception and a timeout isn't too big a lie. - raise rpc_common.Timeout(warn_log) - - # This supports brokerless fanout (addresses > 1) - return_val = None - for queue in queues: - _topic, ip_addr = queue - _addr = "tcp://%s:%s" % (ip_addr, conf.rpc_zmq_port) - - if method.__name__ == '_cast': - eventlet.spawn_n(method, driver, _addr, context, - _topic, msg, timeout, envelope, _msg_id, - None, pooled) - else: - return_val = method(driver, _addr, context, _topic, msg, timeout, - envelope, allowed_remote_exmods, pooled) - - return return_val - - -def _get_matchmaker(*args, **kwargs): - global matchmaker - mm_name = CONF.rpc_zmq_matchmaker - - # Back compatibility for old class names - mm_mapping = { - 'oslo_messaging._drivers.matchmaker_redis.MatchMakerRedis': 'redis', - 'oslo_messaging._drivers.matchmaker_ring.MatchMakerRing': 'ring', - 'oslo_messaging._drivers.matchmaker.MatchMakerLocalhost': 'local', - 'oslo.messaging._drivers.matchmaker_redis.MatchMakerRedis': 'redis', - 'oslo.messaging._drivers.matchmaker_ring.MatchMakerRing': 'ring', - 'oslo.messaging._drivers.matchmaker.MatchMakerLocalhost': 'local'} - if mm_name in mm_mapping: - LOG.warn(_LW('rpc_zmq_matchmaker = %(old_val)s is deprecated. ' - 'It is suggested to change the value to %(new_val)s.'), - {'old_val': mm_name, 'new_val': mm_mapping[mm_name]}) - mm_name = mm_mapping[mm_name] - - if not matchmaker: - mgr = driver.DriverManager('oslo.messaging.zmq.matchmaker', - mm_name) - matchmaker = mgr.driver(*args, **kwargs) - return matchmaker - - -class ZmqIncomingMessage(base.IncomingMessage): - - ReceivedReply = collections.namedtuple( - 'ReceivedReply', ['reply', 'failure', 'log_failure']) - - def __init__(self, listener, ctxt, message): - super(ZmqIncomingMessage, self).__init__(listener, ctxt, message) - self.condition = threading.Condition() - self.received = None - - def reply(self, reply=None, failure=None, log_failure=True): - self.received = self.ReceivedReply(reply, failure, log_failure) - with self.condition: - self.condition.notify() - - def requeue(self): - LOG.debug("WARNING: requeue not supported") - - -class ZmqListener(base.Listener): - - def __init__(self, driver): - super(ZmqListener, self).__init__(driver) - self.incoming_queue = moves.queue.Queue() - - def dispatch(self, ctxt, message): - incoming = ZmqIncomingMessage(self, - ctxt.to_dict(), - message) - - self.incoming_queue.put(incoming) - - with incoming.condition: - incoming.condition.wait() - - assert incoming.received - - if incoming.received.failure: - raise incoming.received.failure - else: - return incoming.received.reply - - def poll(self, timeout=None): - try: - return self.incoming_queue.get(timeout=timeout) - except six.moves.queue.Empty: - # timeout - return None - - -class ZmqClientPool(pool.Pool): - """Class that implements a pool of Zmq Clients for a single endpoint""" - def __init__(self, conf, address, connection_cls, ctxt): - self.connection_cls = connection_cls - self.ctxt = ctxt - self.address = address - super(ZmqClientPool, self).__init__(conf.rpc_conn_pool_size) - - def create(self): - LOG.debug('Pool creating new ZMQ connection for %s' % self.address) - return self.connection_cls(self.address, self.ctxt) - - def empty(self): - for item in self.iter_free(): - item.close() - - -class ZmqClientPoolManager(object): - """Class that manages pools of clients for Zmq endpoints""" - - def __init__(self, conf, ctxt=None): - self._pools = {} + def __init__(self, item_cls, *args, **kwargs): self._lock = threading.Lock() - self.conf = conf - self.ctxt = ctxt + self.item = None + self.item_class = item_cls + self.args = args + self.kwargs = kwargs - def get(self, address): - if address not in self._pools: - with self._lock: - if address not in self._pools: - self._pools[address] = ZmqClientPool(self.conf, - address, - ZmqClient, - self.ctxt) - return self._pools[address].get() + def get(self): + # NOTE(ozamiatin): Lazy initialization. + # All init stuff moved closer to usage point - lazy init. + # Better design approach is to initialize in the driver's + # __init__, but 'fork' extensively used by services + # breaks all things. - def put(self, item): - self._pools[item.address].put(item) + if self.item is not None: + return self.item - def empty(self): - for p in self._pools: - self._pools[p].empty() + self._lock.acquire() + if self.item is None: + self.item = self.item_class(*self.args, **self.kwargs) + self._lock.release() + return self.item + + def cleanup(self): + if self.item: + self.item.cleanup() class ZmqDriver(base.BaseDriver): - """ZeroMQ Driver + + """ZeroMQ Driver implementation. + + Provides implementation of RPC and Notifier APIs by means + of ZeroMQ library. See :doc:`zmq_driver` for details. - """ - # FIXME(markmc): allow this driver to be used without eventlet - def __init__(self, conf, url, default_exchange=None, allowed_remote_exmods=None): - if not zmq: - raise ImportError("Failed to import eventlet.green.zmq") + """Construct ZeroMQ driver. + + Intialize driver options. + + Construct matchmaker - pluggable interface to targets management + Name Service + + Construct client and server controllers + + :param conf: oslo messaging configuration object + :type conf: oslo_config.CONF + :param url: transport URL + :type url: TransportUrl + :param default_exchange: Not used in zmq implementation + :type default_exchange: None + :param allowed_remote_exmods: remote exception passing options + :type allowed_remote_exmods: list + """ conf.register_opts(zmq_opts) conf.register_opts(impl_pooledexecutor._pool_opts) conf.register_opts(base.base_opts) + self.conf = conf + self.allowed_remote_exmods = allowed_remote_exmods + + self.matchmaker = driver.DriverManager( + 'oslo.messaging.zmq.matchmaker', + self.conf.rpc_zmq_matchmaker, + ).driver(self.conf) + + self.server = LazyDriverItem( + zmq_server.ZmqServer, self, self.conf, self.matchmaker) + + self.notify_server = LazyDriverItem( + zmq_server.ZmqServer, self, self.conf, self.matchmaker) + + self.client = LazyDriverItem( + zmq_client.ZmqClient, self.conf, self.matchmaker, + self.allowed_remote_exmods) + + self.notifier = LazyDriverItem( + zmq_client.ZmqClient, self.conf, self.matchmaker, + self.allowed_remote_exmods) super(ZmqDriver, self).__init__(conf, url, default_exchange, allowed_remote_exmods) - # FIXME(markmc): handle default_exchange - - # FIXME(markmc): handle transport URL - if self._url.hosts: - raise NotImplementedError('The ZeroMQ driver does not yet support ' - 'transport URLs') - - # FIXME(markmc): use self.conf everywhere - if self.conf is not CONF: - raise NotImplementedError('The ZeroMQ driver currently only works ' - 'with oslo.config.cfg.CONF') - - self.listeners = [] - - # NOTE(jamespage): Create pool manager on first use to deal with - # os.fork calls in openstack daemons. - self._pool = None - self._pid = None - self._lock = threading.Lock() - - def _configure_pool_manager(func): - """Causes a new pool manager to be created when the messaging service - is first used by the current process. This is important as all - connections in the pools manager by the pool manager will share the - same ZMQ context, which must not be shared across OS processes. - """ - def wrap(self, *args, **kws): - with self._lock: - old_pid = self._pid - self._pid = os.getpid() - - if old_pid != self._pid: - # Create fresh pool manager for the current process - # along with a new ZMQ context. - self._pool = ZmqClientPoolManager( - self.conf, - zmq.Context(self.conf.rpc_zmq_contexts) - ) - return func(self, *args, **kws) - return wrap - - def _send(self, target, ctxt, message, - wait_for_reply=None, timeout=None, envelope=False): - - if wait_for_reply: - method = _call - else: - method = _cast - - topic = target.topic - if target.fanout: - # NOTE(ewindisch): fanout~ is used because it avoid splitting on - # and acts as a non-subtle hint to the matchmaker and ZmqProxy. - topic = 'fanout~' + topic - elif target.server: - topic = '%s.%s' % (topic, target.server) - - reply = _multi_send(self, method, ctxt, topic, message, - envelope=envelope, - allowed_remote_exmods=self._allowed_remote_exmods, - pooled=True) - - if wait_for_reply: - return reply[-1] - - @_configure_pool_manager def send(self, target, ctxt, message, wait_for_reply=None, timeout=None, retry=None): - # NOTE(sileht): retry is not implemented because this driver never - # retry anything - return self._send(target, ctxt, message, wait_for_reply, timeout) + """Send RPC message to server + + :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 wait_for_reply: Waiting for reply flag + :type wait_for_reply: bool + :param timeout: Reply waiting timeout in seconds + :type timeout: int + :param retry: an optional default connection retries configuration + None or -1 means to retry forever + 0 means no retry + N means N retries + :type retry: int + """ + client = self.client.get() + timeout = timeout or self.conf.rpc_response_timeout + if wait_for_reply: + return client.send_call(target, ctxt, message, timeout, retry) + elif target.fanout: + client.send_fanout(target, ctxt, message, timeout, retry) + else: + client.send_cast(target, ctxt, message, timeout, retry) - @_configure_pool_manager def send_notification(self, target, ctxt, message, version, retry=None): - # NOTE(ewindisch): dot-priority in rpc notifier does not - # work with our assumptions. - # NOTE(sileht): retry is not implemented because this driver never - # retry anything - target = target(topic=target.topic.replace('.', '-')) - return self._send(target, ctxt, message, envelope=(version == 2.0)) + """Send notification to server + + :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 + :type version: str + :param retry: an optional default connection retries configuration + None or -1 means to retry forever + 0 means no retry + N means N retries + :type retry: int + """ + client = self.notifier.get() + if target.fanout: + client.send_notify_fanout(target, ctxt, message, version, retry) + else: + client.send_notify(target, ctxt, message, version, retry) - @_configure_pool_manager def listen(self, target): - conn = Connection(self.conf, self) + """Listen to a specified target on a server side - listener = ZmqListener(self) + :param target: Message destination target + :type target: oslo_messaging.Target + """ + server = self.server.get() + server.listen(target) + return server - conn.create_consumer(target.topic, listener) - conn.create_consumer('%s.%s' % (target.topic, target.server), - listener) - conn.create_consumer(target.topic, listener, fanout=True) - - conn.consume_in_thread() - self.listeners.append(conn) - - return listener - - @_configure_pool_manager def listen_for_notifications(self, targets_and_priorities, pool): - # NOTE(sileht): this listener implementation is limited - # because zeromq doesn't support: - # * requeing message - # * pool - conn = Connection(self.conf, self) + """Listen to a specified list of targets on a server side - listener = ZmqListener(self) - for target, priority in targets_and_priorities: - # NOTE(ewindisch): dot-priority in rpc notifier does not - # work with our assumptions. - # NOTE(sileht): create_consumer doesn't support target.exchange - conn.create_consumer('%s-%s' % (target.topic, priority), - listener) - conn.consume_in_thread() - self.listeners.append(conn) - - return listener + :param targets_and_priorities: List of pairs (target, priority) + :type targets_and_priorities: list + :param pool: Not used for zmq implementation + :type pool: object + """ + server = self.notify_server.get() + server.listen_notification(targets_and_priorities) + return server def cleanup(self): - for c in self.listeners: - c.close() - self.listeners = [] - if self._pool: - self._pool.empty() - - def get_connection(self, address, pooled=False): - return ZmqClientContext(address, self._pool, pooled) + """Cleanup all driver's connections finally + """ + self.client.cleanup() + self.server.cleanup() + self.notify_server.cleanup() + self.notifier.cleanup() diff --git a/oslo_messaging/_drivers/matchmaker.py b/oslo_messaging/_drivers/matchmaker.py deleted file mode 100644 index bd3a9ebbd..000000000 --- a/oslo_messaging/_drivers/matchmaker.py +++ /dev/null @@ -1,322 +0,0 @@ -# Copyright 2011 Cloudscaling Group, 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. -""" -The MatchMaker classes should except a Topic or Fanout exchange key and -return keys for direct exchanges, per (approximate) AMQP parlance. -""" - -import contextlib -import logging - -import eventlet -from oslo_config import cfg - -from oslo_messaging._i18n import _ -from oslo_messaging._i18n import _LI - -matchmaker_opts = [ - cfg.IntOpt('matchmaker_heartbeat_freq', - default=300, - help='Heartbeat frequency.'), - cfg.IntOpt('matchmaker_heartbeat_ttl', - default=600, - help='Heartbeat time-to-live.'), -] - -CONF = cfg.CONF -CONF.register_opts(matchmaker_opts) -LOG = logging.getLogger(__name__) -contextmanager = contextlib.contextmanager - - -class MatchMakerException(Exception): - """Signified a match could not be found.""" - message = _("Match not found by MatchMaker.") - - -class Exchange(object): - """Implements lookups. - - Subclass this to support hashtables, dns, etc. - """ - def __init__(self): - pass - - def run(self, key): - raise NotImplementedError() - - -class Binding(object): - """A binding on which to perform a lookup.""" - def __init__(self): - pass - - def test(self, key): - raise NotImplementedError() - - -class MatchMakerBase(object): - """Match Maker Base Class. - - Build off HeartbeatMatchMakerBase if building a heartbeat-capable - MatchMaker. - """ - def __init__(self): - # Array of tuples. Index [2] toggles negation, [3] is last-if-true - self.bindings = [] - - self.no_heartbeat_msg = _('Matchmaker does not implement ' - 'registration or heartbeat.') - - def register(self, key, host): - """Register a host on a backend. - - Heartbeats, if applicable, may keepalive registration. - """ - pass - - def ack_alive(self, key, host): - """Acknowledge that a key.host is alive. - - Used internally for updating heartbeats, but may also be used - publicly to acknowledge a system is alive (i.e. rpc message - successfully sent to host) - """ - pass - - def is_alive(self, topic, host): - """Checks if a host is alive.""" - pass - - def expire(self, topic, host): - """Explicitly expire a host's registration.""" - pass - - def send_heartbeats(self): - """Send all heartbeats. - - Use start_heartbeat to spawn a heartbeat greenthread, - which loops this method. - """ - pass - - def unregister(self, key, host): - """Unregister a topic.""" - pass - - def start_heartbeat(self): - """Spawn heartbeat greenthread.""" - pass - - def stop_heartbeat(self): - """Destroys the heartbeat greenthread.""" - pass - - def add_binding(self, binding, rule, last=True): - self.bindings.append((binding, rule, False, last)) - - # NOTE(ewindisch): kept the following method in case we implement the - # underlying support. - # def add_negate_binding(self, binding, rule, last=True): - # self.bindings.append((binding, rule, True, last)) - - def queues(self, key): - workers = [] - - # bit is for negate bindings - if we choose to implement it. - # last stops processing rules if this matches. - for (binding, exchange, bit, last) in self.bindings: - if binding.test(key): - workers.extend(exchange.run(key)) - - # Support last. - if last: - return workers - return workers - - -class HeartbeatMatchMakerBase(MatchMakerBase): - """Base for a heart-beat capable MatchMaker. - - Provides common methods for registering, unregistering, and maintaining - heartbeats. - """ - def __init__(self): - self.hosts = set() - self._heart = None - self.host_topic = {} - - super(HeartbeatMatchMakerBase, self).__init__() - - def send_heartbeats(self): - """Send all heartbeats. - - Use start_heartbeat to spawn a heartbeat greenthread, - which loops this method. - """ - for key, host in self.host_topic.keys(): - self.ack_alive(key, host) - - def ack_alive(self, key, host): - """Acknowledge that a host.topic is alive. - - Used internally for updating heartbeats, but may also be used - publicly to acknowledge a system is alive (i.e. rpc message - successfully sent to host) - """ - raise NotImplementedError("Must implement ack_alive") - - def backend_register(self, key, host): - """Implements registration logic. - - Called by register(self,key,host) - """ - raise NotImplementedError("Must implement backend_register") - - def backend_unregister(self, key, key_host): - """Implements de-registration logic. - - Called by unregister(self,key,host) - """ - raise NotImplementedError("Must implement backend_unregister") - - def register(self, key, host): - """Register a host on a backend. - - Heartbeats, if applicable, may keepalive registration. - """ - self.hosts.add(host) - self.host_topic[(key, host)] = host - key_host = '.'.join((key, host)) - - self.backend_register(key, key_host) - - self.ack_alive(key, host) - - def unregister(self, key, host): - """Unregister a topic.""" - if (key, host) in self.host_topic: - del self.host_topic[(key, host)] - - self.hosts.discard(host) - self.backend_unregister(key, '.'.join((key, host))) - - LOG.info(_LI("Matchmaker unregistered: %(key)s, %(host)s"), - {'key': key, 'host': host}) - - def start_heartbeat(self): - """Implementation of MatchMakerBase.start_heartbeat. - - Launches greenthread looping send_heartbeats(), - yielding for CONF.matchmaker_heartbeat_freq seconds - between iterations. - """ - if not self.hosts: - raise MatchMakerException( - _("Register before starting heartbeat.")) - - def do_heartbeat(): - while True: - self.send_heartbeats() - eventlet.sleep(CONF.matchmaker_heartbeat_freq) - - self._heart = eventlet.spawn(do_heartbeat) - - def stop_heartbeat(self): - """Destroys the heartbeat greenthread.""" - if self._heart: - self._heart.kill() - - -class DirectBinding(Binding): - """Specifies a host in the key via a '.' character. - - Although dots are used in the key, the behavior here is - that it maps directly to a host, thus direct. - """ - def test(self, key): - return '.' in key - - -class TopicBinding(Binding): - """Where a 'bare' key without dots. - - AMQP generally considers topic exchanges to be those *with* dots, - but we deviate here in terminology as the behavior here matches - that of a topic exchange (whereas where there are dots, behavior - matches that of a direct exchange. - """ - def test(self, key): - return '.' not in key - - -class FanoutBinding(Binding): - """Match on fanout keys, where key starts with 'fanout.' string.""" - def test(self, key): - return key.startswith('fanout~') - - -class StubExchange(Exchange): - """Exchange that does nothing.""" - def run(self, key): - return [(key, None)] - - -class LocalhostExchange(Exchange): - """Exchange where all direct topics are local.""" - def __init__(self, host='localhost'): - self.host = host - super(Exchange, self).__init__() - - def run(self, key): - return [('.'.join((key.split('.')[0], self.host)), self.host)] - - -class DirectExchange(Exchange): - """Exchange where all topic keys are split, sending to second half. - - i.e. "compute.host" sends a message to "compute.host" running on "host" - """ - def __init__(self): - super(Exchange, self).__init__() - - def run(self, key): - e = key.split('.', 1)[1] - return [(key, e)] - - -class MatchMakerLocalhost(MatchMakerBase): - """Match Maker where all bare topics resolve to localhost. - - Useful for testing. - """ - def __init__(self, host='localhost'): - super(MatchMakerLocalhost, self).__init__() - self.add_binding(FanoutBinding(), LocalhostExchange(host)) - self.add_binding(DirectBinding(), DirectExchange()) - self.add_binding(TopicBinding(), LocalhostExchange(host)) - - -class MatchMakerStub(MatchMakerBase): - """Match Maker where topics are untouched. - - Useful for testing, or for AMQP/brokered queues. - Will not work where knowledge of hosts is known (i.e. zeromq) - """ - def __init__(self): - super(MatchMakerStub, self).__init__() - - self.add_binding(FanoutBinding(), StubExchange()) - self.add_binding(DirectBinding(), StubExchange()) - self.add_binding(TopicBinding(), StubExchange()) diff --git a/oslo_messaging/_drivers/matchmaker_redis.py b/oslo_messaging/_drivers/matchmaker_redis.py deleted file mode 100644 index 290b60351..000000000 --- a/oslo_messaging/_drivers/matchmaker_redis.py +++ /dev/null @@ -1,145 +0,0 @@ -# Copyright 2013 Cloudscaling Group, 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. -""" -The MatchMaker classes should accept a Topic or Fanout exchange key and -return keys for direct exchanges, per (approximate) AMQP parlance. -""" - -from oslo_config import cfg -from oslo_utils import importutils - -from oslo_messaging._drivers import matchmaker as mm_common - -redis = importutils.try_import('redis') - - -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.StrOpt('password', - help='Password for Redis server (optional).'), -] - -CONF = cfg.CONF -opt_group = cfg.OptGroup(name='matchmaker_redis', - title='Options for Redis-based MatchMaker') -CONF.register_group(opt_group) -CONF.register_opts(matchmaker_redis_opts, opt_group) - - -class RedisExchange(mm_common.Exchange): - def __init__(self, matchmaker): - self.matchmaker = matchmaker - self.redis = matchmaker.redis - super(RedisExchange, self).__init__() - - -class RedisTopicExchange(RedisExchange): - """Exchange where all topic keys are split, sending to second half. - - i.e. "compute.host" sends a message to "compute" running on "host" - """ - def run(self, topic): - while True: - member_name = self.redis.srandmember(topic) - - if not member_name: - # If this happens, there are no - # longer any members. - break - - if not self.matchmaker.is_alive(topic, member_name): - continue - - host = member_name.split('.', 1)[1] - return [(member_name, host)] - return [] - - -class RedisFanoutExchange(RedisExchange): - """Return a list of all hosts.""" - def run(self, topic): - topic = topic.split('~', 1)[1] - hosts = self.redis.smembers(topic) - good_hosts = filter( - lambda host: self.matchmaker.is_alive(topic, host), hosts) - - return [(x, x.split('.', 1)[1]) for x in good_hosts] - - -class MatchMakerRedis(mm_common.HeartbeatMatchMakerBase): - """MatchMaker registering and looking-up hosts with a Redis server.""" - def __init__(self): - super(MatchMakerRedis, self).__init__() - - if not redis: - raise ImportError("Failed to import module redis.") - - self.redis = redis.StrictRedis( - host=CONF.matchmaker_redis.host, - port=CONF.matchmaker_redis.port, - password=CONF.matchmaker_redis.password) - - self.add_binding(mm_common.FanoutBinding(), RedisFanoutExchange(self)) - self.add_binding(mm_common.DirectBinding(), mm_common.DirectExchange()) - self.add_binding(mm_common.TopicBinding(), RedisTopicExchange(self)) - - def ack_alive(self, key, host): - topic = "%s.%s" % (key, host) - if not self.redis.expire(topic, CONF.matchmaker_heartbeat_ttl): - # If we could not update the expiration, the key - # might have been pruned. Re-register, creating a new - # key in Redis. - self.register(key, host) - - def is_alive(self, topic, host): - # After redis 2.8, if the specialized key doesn't exist, - # TTL fuction would return -2. If key exists, - # but doesn't have expiration associated, - # TTL func would return -1. For more information, - # please visit http://redis.io/commands/ttl - if self.redis.ttl(host) == -2: - self.expire(topic, host) - return False - return True - - def expire(self, topic, host): - with self.redis.pipeline() as pipe: - pipe.multi() - pipe.delete(host) - pipe.srem(topic, host) - pipe.execute() - - def backend_register(self, key, key_host): - with self.redis.pipeline() as pipe: - pipe.multi() - pipe.sadd(key, key_host) - - # No value is needed, we just - # care if it exists. Sets aren't viable - # because only keys can expire. - pipe.sadd(key_host, '') - - pipe.execute() - - def backend_unregister(self, key, key_host): - with self.redis.pipeline() as pipe: - pipe.multi() - pipe.srem(key, key_host) - pipe.delete(key_host) - pipe.execute() diff --git a/oslo_messaging/_drivers/matchmaker_ring.py b/oslo_messaging/_drivers/matchmaker_ring.py deleted file mode 100644 index 8678f9d01..000000000 --- a/oslo_messaging/_drivers/matchmaker_ring.py +++ /dev/null @@ -1,105 +0,0 @@ -# Copyright 2011-2013 Cloudscaling Group, 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. -""" -The MatchMaker classes should except a Topic or Fanout exchange key and -return keys for direct exchanges, per (approximate) AMQP parlance. -""" - -import itertools -import json -import logging - -from oslo_config import cfg - -from oslo_messaging._drivers import matchmaker as mm -from oslo_messaging._i18n import _LW - -matchmaker_opts = [ - # Matchmaker ring file - cfg.StrOpt('ringfile', - deprecated_name='matchmaker_ringfile', - deprecated_group='DEFAULT', - default='/etc/oslo/matchmaker_ring.json', - help='Matchmaker ring file (JSON).'), -] - -CONF = cfg.CONF -CONF.register_opts(matchmaker_opts, 'matchmaker_ring') -LOG = logging.getLogger(__name__) - - -class RingExchange(mm.Exchange): - """Match Maker where hosts are loaded from a static JSON formatted file. - - __init__ takes optional ring dictionary argument, otherwise - loads the ringfile from CONF.mathcmaker_ringfile. - """ - def __init__(self, ring=None): - super(RingExchange, self).__init__() - - if ring: - self.ring = ring - else: - fh = open(CONF.matchmaker_ring.ringfile, 'r') - self.ring = json.load(fh) - fh.close() - - self.ring0 = {} - for k in self.ring.keys(): - self.ring0[k] = itertools.cycle(self.ring[k]) - - def _ring_has(self, key): - return key in self.ring0 - - -class RoundRobinRingExchange(RingExchange): - """A Topic Exchange based on a hashmap.""" - def __init__(self, ring=None): - super(RoundRobinRingExchange, self).__init__(ring) - - def run(self, key): - if not self._ring_has(key): - LOG.warn( - _LW("No key defining hosts for topic '%s', " - "see ringfile"), key - ) - return [] - host = next(self.ring0[key]) - return [(key + '.' + host, host)] - - -class FanoutRingExchange(RingExchange): - """Fanout Exchange based on a hashmap.""" - def __init__(self, ring=None): - super(FanoutRingExchange, self).__init__(ring) - - def run(self, key): - # Assume starts with "fanout~", strip it for lookup. - nkey = key.split('fanout~')[1:][0] - if not self._ring_has(nkey): - LOG.warn( - _LW("No key defining hosts for topic '%s', " - "see ringfile"), nkey - ) - return [] - return map(lambda x: (key + '.' + x, x), self.ring[nkey]) - - -class MatchMakerRing(mm.MatchMakerBase): - """Match Maker where hosts are loaded from a static hashmap.""" - def __init__(self, ring=None): - super(MatchMakerRing, self).__init__() - self.add_binding(mm.FanoutBinding(), FanoutRingExchange(ring)) - self.add_binding(mm.DirectBinding(), mm.DirectExchange()) - self.add_binding(mm.TopicBinding(), RoundRobinRingExchange(ring)) diff --git a/oslo_messaging/_drivers/zmq_driver/__init__.py b/oslo_messaging/_drivers/zmq_driver/__init__.py new file mode 100644 index 000000000..e69de29bb diff --git a/oslo_messaging/_drivers/zmq_driver/client/__init__.py b/oslo_messaging/_drivers/zmq_driver/client/__init__.py new file mode 100644 index 000000000..e69de29bb diff --git a/oslo_messaging/_drivers/zmq_driver/client/publishers/__init__.py b/oslo_messaging/_drivers/zmq_driver/client/publishers/__init__.py new file mode 100644 index 000000000..e69de29bb 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 new file mode 100644 index 000000000..a5c3f0fdf --- /dev/null +++ b/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_dealer_publisher.py @@ -0,0 +1,101 @@ +# 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 uuid + +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 + +LOG = logging.getLogger(__name__) + +zmq = zmq_async.import_zmq() + + +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) + + dealer_socket, hosts = self._check_hosts_connections(request.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") + % 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 _send_request(self, socket, request): + + message_id = str(uuid.uuid1()) + + socket.send(b'', zmq.SNDMORE) + socket.send_string(request.msg_type, zmq.SNDMORE) + socket.send_string(message_id, zmq.SNDMORE) + socket.send_pyobj(request.context, zmq.SNDMORE) + socket.send_pyobj(request.message) + + LOG.info(_LI("Sending message %(message)s to a target %(target)s") + % {"message": request.message, + "target": request.target}) + + def cleanup(self): + self.ack_receiver.cleanup() + super(DealerPublisher, self).cleanup() + + +class AcknowledgementReceiver(object): + + def __init__(self): + self.poller = zmq_async.get_poller() + self.thread = zmq_async.get_executor(self.poll_for_acknowledgements) + self.thread.execute() + + def _receive_acknowledgement(self, socket): + empty = socket.recv() + assert empty == b"", "Empty delimiter expected" + ack_message = socket.recv_pyobj() + return ack_message + + def track_socket(self, socket): + self.poller.register(socket, self._receive_acknowledgement) + + def poll_for_acknowledgements(self): + ack_message, socket = self.poller.poll() + LOG.info(_LI("Message %s acknowledged") + % ack_message[zmq_names.FIELD_ID]) + + def cleanup(self): + self.thread.stop() + 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 new file mode 100644 index 000000000..228724b6c --- /dev/null +++ b/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_pub_publisher.py @@ -0,0 +1,47 @@ +# 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\ + 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__) + +zmq = zmq_async.import_zmq() + + +class PubPublisher(zmq_publisher_base.PublisherMultisend): + + def __init__(self, conf, matchmaker): + super(PubPublisher, self).__init__(conf, matchmaker, zmq.PUB) + + def send_request(self, request): + + 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) + self._send_request(pub_socket, request) + + def _send_request(self, socket, request): + + 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}) 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 new file mode 100644 index 000000000..5ff1a41c6 --- /dev/null +++ b/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_publisher_base.py @@ -0,0 +1,141 @@ +# 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 abc +import logging + +import six + +from oslo_messaging._drivers import common as rpc_common +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 +from oslo_messaging._i18n import _LE, _LI + +LOG = logging.getLogger(__name__) + +zmq = zmq_async.import_zmq() + + +class UnsupportedSendPattern(rpc_common.RPCException): + + """Exception to raise from publishers in case of unsupported + sending pattern called. + """ + + def __init__(self, pattern_name): + """Construct exception object + + :param pattern_name: Message type name from zmq_names + :type pattern_name: str + """ + errmsg = _LE("Sending pattern %s is unsupported.") % pattern_name + super(UnsupportedSendPattern, self).__init__(errmsg) + + +@six.add_metaclass(abc.ABCMeta) +class PublisherBase(object): + + """Abstract publisher class + + Each publisher from zmq-driver client should implement + this interface to serve as a messages publisher. + + Publisher can send request objects from zmq_request. + """ + + def __init__(self, conf, matchmaker): + + """Construct publisher + + Accept configuration object and Name Service interface object. + Create zmq.Context and connected sockets dictionary. + + :param conf: configuration object + :type conf: oslo_config.CONF + :param matchmaker: Name Service interface object + :type matchmaker: matchmaker.MatchMakerBase + """ + + self.conf = conf + self.zmq_context = zmq.Context() + self.matchmaker = matchmaker + self.outbound_sockets = {} + super(PublisherBase, self).__init__() + + @abc.abstractmethod + def send_request(self, request): + """Send request to consumer + + :param request: Message data and destination container object + :type request: zmq_request.Request + """ + + def _send_request(self, socket, request): + """Send request to consumer. + Helper private method which defines basic sending behavior. + + :param socket: Socket to publish message on + :type socket: zmq.Socket + :param request: Message data and destination container object + :type request: zmq_request.Request + """ + socket.send_string(request.msg_type, zmq.SNDMORE) + socket.send_pyobj(request.context, zmq.SNDMORE) + socket.send_pyobj(request.message) + + def cleanup(self): + """Cleanup publisher. Close allocated connections.""" + for socket in self.outbound_sockets.values(): + socket.setsockopt(zmq.LINGER, 0) + socket.close() + + +class PublisherMultisend(PublisherBase): + + def __init__(self, conf, matchmaker, socket_type): + self.socket_type = socket_type + super(PublisherMultisend, self).__init__(conf, matchmaker) + + def _check_hosts_connections(self, target): + # TODO(ozamiatin): Place for significant optimization + # Matchmaker cache should be implemented + hosts = self.matchmaker.get_hosts(target) + if str(target) in self.outbound_sockets: + socket = self.outbound_sockets[str(target)] + else: + 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, hosts + + def _connect_to_host(self, socket, host, target): + address = zmq_address.get_tcp_direct_address(host) + 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}) + socket.connect(address) + except zmq.ZMQError as e: + errmsg = _LE("Failed connecting %(stype) to %(address)s: %(e)s")\ + % (stype, address, e) + LOG.error(_LE("Failed connecting %(stype) to %(address)s: %(e)s") + % (stype, address, e)) + raise rpc_common.RPCException(errmsg) 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 new file mode 100644 index 000000000..b8fc4fe51 --- /dev/null +++ b/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_push_publisher.py @@ -0,0 +1,57 @@ +# 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\ + 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 + +LOG = logging.getLogger(__name__) + +zmq = zmq_async.import_zmq() + + +class PushPublisher(zmq_publisher_base.PublisherMultisend): + + def __init__(self, conf, matchmaker): + super(PushPublisher, self).__init__(conf, matchmaker, zmq.PUSH) + + def send_request(self, request): + + 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) + + if not push_socket.connections: + LOG.warning(_LW("Request %s was dropped because no connection") + % request.msg_type) + return + + if request.msg_type in zmq_names.MULTISEND_TYPES: + for _ in range(push_socket.connections_count()): + self._send_request(push_socket, request) + else: + self._send_request(push_socket, request) + + def _send_request(self, socket, request): + + 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}) 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 new file mode 100644 index 000000000..001fe026f --- /dev/null +++ b/oslo_messaging/_drivers/zmq_driver/client/publishers/zmq_req_publisher.py @@ -0,0 +1,85 @@ +# 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 +import logging + +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._i18n import _LE, _LI + +LOG = logging.getLogger(__name__) + +zmq = zmq_async.import_zmq() + + +class ReqPublisher(zmq_publisher_base.PublisherBase): + + def send_request(self, request): + + if request.msg_type != zmq_names.CALL_TYPE: + raise zmq_publisher_base.UnsupportedSendPattern(request.msg_type) + + socket = self._connect_to_host(request.target) + self._send_request(socket, request) + return self._receive_reply(socket, request) + + def _connect_to_host(self, target): + + try: + self.zmq_context = zmq.Context() + socket = self.zmq_context.socket(zmq.REQ) + + host = self.matchmaker.get_single_host(target) + connect_address = zmq_address.get_tcp_direct_address(host) + + LOG.info(_LI("Connecting REQ to %s") % connect_address) + + socket.connect(connect_address) + self.outbound_sockets[str(target)] = socket + return socket + + except zmq.ZMQError as e: + errmsg = _LE("Error connecting to socket: %s") % str(e) + LOG.error(_LE("Error connecting to socket: %s") % str(e)) + raise rpc_common.RPCException(errmsg) + + @staticmethod + def _receive_reply(socket, request): + + def _receive_method(socket): + return socket.recv_pyobj() + + # 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) + reply, socket = poller.poll(timeout=request.timeout) + if reply is None: + raise oslo_messaging.MessagingTimeout( + "Timeout %s seconds was reached" % request.timeout) + 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] + + def close(self): + # For contextlib compatibility + self.cleanup() diff --git a/oslo_messaging/_drivers/zmq_driver/client/zmq_client.py b/oslo_messaging/_drivers/zmq_driver/client/zmq_client.py new file mode 100644 index 000000000..26a358f67 --- /dev/null +++ b/oslo_messaging/_drivers/zmq_driver/client/zmq_client.py @@ -0,0 +1,72 @@ +# 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.publishers\ + 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_async + +zmq = zmq_async.import_zmq() + + +class ZmqClient(object): + + 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 [] + self.dealer_publisher = zmq_dealer_publisher.DealerPublisher( + 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(zmq_req_publisher.ReqPublisher( + 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() diff --git a/oslo_messaging/_drivers/zmq_driver/client/zmq_request.py b/oslo_messaging/_drivers/zmq_driver/client/zmq_request.py new file mode 100644 index 000000000..e692a3aab --- /dev/null +++ b/oslo_messaging/_drivers/zmq_driver/client/zmq_request.py @@ -0,0 +1,118 @@ +# 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 abc +import logging + +import six + +from oslo_messaging._drivers.zmq_driver import zmq_async +from oslo_messaging._drivers.zmq_driver import zmq_names +from oslo_messaging._i18n import _LE + +LOG = logging.getLogger(__name__) + +zmq = zmq_async.import_zmq() + + +@six.add_metaclass(abc.ABCMeta) +class Request(object): + + """Zmq request abstract class + + Represents socket (publisher) independent data object to publish. + Request object should contain all needed information for a publisher + to publish it, for instance: message payload, target, timeout + and retries etc. + """ + + def __init__(self, target, context=None, message=None, retry=None): + + """Construct request object + + :param target: Message destination target + :type target: oslo_messaging.Target + :param context: Message context + :type context: dict + :param message: Message payload to pass + :type message: dict + :param retry: an optional default connection retries configuration + None or -1 means to retry forever + 0 means no retry + N means N retries + :type retry: int + """ + + if self.msg_type not in zmq_names.MESSAGE_TYPES: + raise RuntimeError("Unknown message type!") + + self.target = target + self.context = context + self.message = message + self.retry = retry + + @abc.abstractproperty + def msg_type(self): + """ZMQ message type""" + + def close(self): + """Nothing to close in base request""" + + +class RpcRequest(Request): + + def __init__(self, *args, **kwargs): + message = kwargs.get("message") + if message['method'] is None: + errmsg = _LE("No method specified for RPC call") + LOG.error(_LE("No method specified for RPC call")) + raise KeyError(errmsg) + + self.timeout = kwargs.pop("timeout") + assert self.timeout is not None, "Timeout should be specified!" + + super(RpcRequest, self).__init__(*args, **kwargs) + + +class CallRequest(RpcRequest): + + msg_type = zmq_names.CALL_TYPE + + def __init__(self, *args, **kwargs): + self.allowed_remote_exmods = kwargs.pop("allowed_remote_exmods") + super(CallRequest, self).__init__(*args, **kwargs) + + +class CastRequest(RpcRequest): + + msg_type = zmq_names.CAST_TYPE + + +class FanoutRequest(RpcRequest): + + msg_type = zmq_names.CAST_FANOUT_TYPE + + +class NotificationRequest(Request): + + msg_type = zmq_names.NOTIFY_TYPE + + def __init__(self, *args, **kwargs): + self.version = kwargs.pop("version") + super(NotificationRequest, self).__init__(*args, **kwargs) + + +class NotificationFanoutRequest(NotificationRequest): + + msg_type = zmq_names.NOTIFY_FANOUT_TYPE diff --git a/oslo_messaging/_drivers/zmq_driver/matchmaker/__init__.py b/oslo_messaging/_drivers/zmq_driver/matchmaker/__init__.py new file mode 100644 index 000000000..e69de29bb diff --git a/oslo_messaging/_drivers/zmq_driver/matchmaker/base.py b/oslo_messaging/_drivers/zmq_driver/matchmaker/base.py new file mode 100644 index 000000000..f4480f0be --- /dev/null +++ b/oslo_messaging/_drivers/zmq_driver/matchmaker/base.py @@ -0,0 +1,95 @@ +# +# 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 collections +import logging +import random + +import six + +import oslo_messaging +from oslo_messaging._i18n import _LI, _LW + + +LOG = logging.getLogger(__name__) + + +@six.add_metaclass(abc.ABCMeta) +class MatchMakerBase(object): + + def __init__(self, conf, *args, **kwargs): + super(MatchMakerBase, self).__init__(*args, **kwargs) + + self.conf = conf + + @abc.abstractmethod + def register(self, target, hostname): + """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 + """ + + @abc.abstractmethod + def get_hosts(self, target): + """Get all hosts from nameserver by target. + + :param target: the default target for invocations + :type target: Target + :returns: a list of "hostname:port" hosts + """ + + def get_single_host(self, target): + """Get a single host by target. + + :param target: the target for messages + :type target: Target + :returns: a "hostname:port" host + """ + + hosts = self.get_hosts(target) + if not hosts: + err_msg = "No hosts were found for target %s." % target + LOG.error(err_msg) + raise oslo_messaging.InvalidTarget(err_msg, target) + + if len(hosts) == 1: + host = hosts[0] + LOG.info(_LI("A single host %(host)s found for target %(target)s.") + % {"host": host, "target": target}) + else: + host = random.choice(hosts) + LOG.warning(_LW("Multiple hosts %(hosts)s were found for target " + " %(target)s. Using the random one - %(host)s.") + % {"hosts": hosts, "target": target, "host": host}) + return host + + +class DummyMatchMaker(MatchMakerBase): + + def __init__(self, conf, *args, **kwargs): + super(DummyMatchMaker, self).__init__(conf, *args, **kwargs) + + self._cache = collections.defaultdict(list) + + def register(self, target, hostname): + key = str(target) + if hostname not in self._cache[key]: + self._cache[key].append(hostname) + + def get_hosts(self, target): + key = str(target) + 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 new file mode 100644 index 000000000..a2ee9bccf --- /dev/null +++ b/oslo_messaging/_drivers/zmq_driver/matchmaker/matchmaker_redis.py @@ -0,0 +1,77 @@ +# +# 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_config import cfg +import redis + +from oslo_messaging._drivers.zmq_driver.matchmaker import base + + +LOG = logging.getLogger(__name__) + + +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.StrOpt('password', + default='', + secret=True, + help='Password for Redis server (optional).'), +] + + +class RedisMatchMaker(base.MatchMakerBase): + + def __init__(self, conf, *args, **kwargs): + super(RedisMatchMaker, self).__init__(conf, *args, **kwargs) + self.conf.register_opts(matchmaker_redis_opts, "matchmaker_redis") + + self._redis = redis.StrictRedis( + host=self.conf.matchmaker_redis.host, + port=self.conf.matchmaker_redis.port, + 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) + + 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)) + return hosts diff --git a/oslo_messaging/_drivers/zmq_driver/poller/__init__.py b/oslo_messaging/_drivers/zmq_driver/poller/__init__.py new file mode 100644 index 000000000..e69de29bb diff --git a/oslo_messaging/_drivers/zmq_driver/poller/green_poller.py b/oslo_messaging/_drivers/zmq_driver/poller/green_poller.py new file mode 100644 index 000000000..58f8d8af1 --- /dev/null +++ b/oslo_messaging/_drivers/zmq_driver/poller/green_poller.py @@ -0,0 +1,114 @@ +# 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 + +import eventlet + +from oslo_messaging._drivers.zmq_driver import zmq_poller + +LOG = logging.getLogger(__name__) + + +class GreenPoller(zmq_poller.ZmqPoller): + + def __init__(self): + self.incoming_queue = eventlet.queue.LightQueue() + self.green_pool = eventlet.GreenPool() + self.thread_by_socket = {} + + def register(self, socket, recv_method=None): + if socket not in self.thread_by_socket: + self.thread_by_socket[socket] = self.green_pool.spawn( + self._socket_receive, socket, recv_method) + + def _socket_receive(self, socket, recv_method=None): + while True: + if recv_method: + incoming = recv_method(socket) + else: + incoming = socket.recv_multipart() + self.incoming_queue.put((incoming, socket)) + eventlet.sleep() + + def poll(self, timeout=None): + try: + return self.incoming_queue.get(timeout=timeout) + except eventlet.queue.Empty: + return (None, None) + + def close(self): + for thread in self.thread_by_socket.values(): + thread.kill() + + self.thread_by_socket = {} + + +class HoldReplyPoller(GreenPoller): + + def __init__(self): + super(HoldReplyPoller, self).__init__() + self.event_by_socket = {} + self._is_running = threading.Event() + + def register(self, socket, recv_method=None): + super(HoldReplyPoller, self).register(socket, recv_method) + self.event_by_socket[socket] = threading.Event() + + def resume_polling(self, socket): + pause = self.event_by_socket[socket] + pause.set() + + def _socket_receive(self, socket, recv_method=None): + pause = self.event_by_socket[socket] + while not self._is_running.is_set(): + pause.clear() + if recv_method: + incoming = recv_method(socket) + else: + incoming = socket.recv_multipart() + self.incoming_queue.put((incoming, socket)) + pause.wait() + + def close(self): + self._is_running.set() + for pause in self.event_by_socket.values(): + pause.set() + eventlet.sleep() + + super(HoldReplyPoller, self).close() + + +class GreenExecutor(zmq_poller.Executor): + + def __init__(self, method): + self._method = method + super(GreenExecutor, self).__init__(None) + + def _loop(self): + while True: + self._method() + eventlet.sleep() + + def execute(self): + self.thread = eventlet.spawn(self._loop) + + def wait(self): + if self.thread is not None: + self.thread.wait() + + def stop(self): + if self.thread is not None: + self.thread.kill() diff --git a/oslo_messaging/_drivers/zmq_driver/poller/threading_poller.py b/oslo_messaging/_drivers/zmq_driver/poller/threading_poller.py new file mode 100644 index 000000000..7719310cf --- /dev/null +++ b/oslo_messaging/_drivers/zmq_driver/poller/threading_poller.py @@ -0,0 +1,86 @@ +# 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 oslo_utils import eventletutils +import zmq + +from oslo_messaging._drivers.zmq_driver import zmq_poller + +LOG = logging.getLogger(__name__) + +_threading = threading + +if eventletutils.EVENTLET_AVAILABLE: + import eventlet + _threading = eventlet.patcher.original('threading') + + +class ThreadingPoller(zmq_poller.ZmqPoller): + + def __init__(self): + self.poller = zmq.Poller() + self.recv_methods = {} + + def register(self, socket, recv_method=None): + if recv_method is not None: + self.recv_methods[socket] = recv_method + self.poller.register(socket, zmq.POLLIN) + + def poll(self, timeout=None): + timeout = timeout * 1000 # zmq poller waits milliseconds + sockets = None + + try: + sockets = dict(self.poller.poll(timeout=timeout)) + except zmq.ZMQError as e: + LOG.debug("Polling terminated with error: %s" % e) + + if not sockets: + return None, None + for socket in sockets: + if socket in self.recv_methods: + return self.recv_methods[socket](socket), socket + else: + return socket.recv_multipart(), socket + + def resume_polling(self, socket): + pass # Nothing to do for threading poller + + def close(self): + pass # Nothing to do for threading poller + + +class ThreadingExecutor(zmq_poller.Executor): + + def __init__(self, method): + self._method = method + super(ThreadingExecutor, self).__init__( + _threading.Thread(target=self._loop)) + self._stop = _threading.Event() + + def _loop(self): + while not self._stop.is_set(): + self._method() + + def execute(self): + self.thread.start() + + def stop(self): + self._stop.set() + + def wait(self): + self.thread.join() diff --git a/oslo_messaging/_drivers/zmq_driver/server/__init__.py b/oslo_messaging/_drivers/zmq_driver/server/__init__.py new file mode 100644 index 000000000..e69de29bb diff --git a/oslo_messaging/_drivers/zmq_driver/server/consumers/__init__.py b/oslo_messaging/_drivers/zmq_driver/server/consumers/__init__.py new file mode 100644 index 000000000..e69de29bb 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 new file mode 100644 index 000000000..8bb2461e7 --- /dev/null +++ b/oslo_messaging/_drivers/zmq_driver/server/consumers/zmq_consumer_base.py @@ -0,0 +1,86 @@ +# 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 abc +import logging + +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_socket +from oslo_messaging._i18n import _LE, _LI + +LOG = logging.getLogger(__name__) + +zmq = zmq_async.import_zmq() + + +@six.add_metaclass(abc.ABCMeta) +class ConsumerBase(object): + + def __init__(self, conf, poller, server): + self.conf = conf + self.poller = poller + self.server = server + self.sockets = [] + self.context = zmq.Context() + + def subscribe_socket(self, socket_type): + try: + socket = zmq_socket.ZmqRandomPortSocket( + 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": 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")\ + % (self.port, e) + LOG.error(_LE("Failed binding to port %(port)d: %(e)s") + % (self.port, e)) + raise rpc_common.RPCException(errmsg) + + @abc.abstractmethod + def listen(self, target): + """Associate new sockets with targets here""" + + @abc.abstractmethod + def receive_message(self, target): + """Method for poller - receiving message routine""" + + def cleanup(self): + for socket in self.sockets: + if not socket.handle.closed: + socket.setsockopt(zmq.LINGER, 0) + socket.close() + self.sockets = [] + + +class SingleSocketConsumer(ConsumerBase): + + def __init__(self, conf, poller, server, socket_type): + super(SingleSocketConsumer, self).__init__(conf, poller, server) + self.socket = self.subscribe_socket(socket_type) + + @property + def address(self): + return self.socket.bind_address + + @property + def port(self): + return self.socket.port 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 new file mode 100644 index 000000000..98ef3a73c --- /dev/null +++ b/oslo_messaging/_drivers/zmq_driver/server/consumers/zmq_pull_consumer.py @@ -0,0 +1,69 @@ +# 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 import base +from oslo_messaging._drivers.zmq_driver.server.consumers\ + import zmq_consumer_base +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 + +LOG = logging.getLogger(__name__) + +zmq = zmq_async.import_zmq() + + +class PullIncomingMessage(base.IncomingMessage): + + def __init__(self, listener, context, message): + super(PullIncomingMessage, self).__init__(listener, context, message) + + def reply(self, reply=None, failure=None, log_failure=True): + """Reply is not needed for non-call messages.""" + + def acknowledge(self): + """Acknowledgments are not supported by this type of consumer.""" + + def requeue(self): + """Requeueing is not supported.""" + + +class PullConsumer(zmq_consumer_base.SingleSocketConsumer): + + def __init__(self, conf, poller, server): + super(PullConsumer, self).__init__(conf, poller, server, zmq.PULL) + + def listen(self, target): + LOG.info(_LI("Listen to target %s") % str(target)) + # Do nothing here because we have a single socket + + def receive_message(self, socket): + try: + msg_type = socket.recv_string() + 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)}) + + if msg_type in (zmq_names.CAST_TYPES + zmq_names.NOTIFY_TYPES): + return PullIncomingMessage(self.server, context, message) + else: + LOG.error(_LE("Unknown message type: %s") % msg_type) + + except zmq.ZMQError as e: + LOG.error(_LE("Receiving message failed: %s") % str(e)) 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 new file mode 100644 index 000000000..bfbfe9fdc --- /dev/null +++ b/oslo_messaging/_drivers/zmq_driver/server/consumers/zmq_router_consumer.py @@ -0,0 +1,94 @@ +# 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 import base +from oslo_messaging._drivers.zmq_driver.server.consumers\ + import zmq_consumer_base +from oslo_messaging._drivers.zmq_driver.server import zmq_incoming_message +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 + +LOG = logging.getLogger(__name__) + +zmq = zmq_async.import_zmq() + + +class RouterIncomingMessage(base.IncomingMessage): + + def __init__(self, listener, context, message, socket, reply_id, msg_id, + poller): + super(RouterIncomingMessage, self).__init__(listener, context, message) + self.socket = socket + self.reply_id = reply_id + self.msg_id = msg_id + self.message = message + poller.resume_polling(socket) + + def reply(self, reply=None, failure=None, log_failure=True): + """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) + + def requeue(self): + """Requeue is not supported""" + + +class RouterConsumer(zmq_consumer_base.SingleSocketConsumer): + + def __init__(self, conf, poller, server): + super(RouterConsumer, self).__init__(conf, poller, server, zmq.ROUTER) + + def listen(self, target): + LOG.info(_LI("Listen to target %s") % str(target)) + # Do nothing here because we have a single socket + + def receive_message(self, socket): + try: + reply_id = socket.recv() + empty = socket.recv() + assert empty == b'', 'Bad format: empty delimiter expected' + msg_type = socket.recv_string() + assert msg_type is not None, 'Bad format: msg type expected' + + msg_id = None + if msg_type != zmq_names.CALL_TYPE: + msg_id = socket.recv_string() + + 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)}) + + if msg_type == zmq_names.CALL_TYPE: + return zmq_incoming_message.ZmqIncomingRequest( + self.server, context, message, socket, reply_id, + self.poller) + elif msg_type in (zmq_names.CAST_TYPES + zmq_names.NOTIFY_TYPES): + return RouterIncomingMessage( + self.server, context, message, socket, reply_id, + msg_id, self.poller) + else: + LOG.error(_LE("Unknown message type: %s") % msg_type) + + except zmq.ZMQError as e: + LOG.error(_LE("Receiving message failed: %s") % str(e)) diff --git a/oslo_messaging/_drivers/zmq_driver/server/zmq_incoming_message.py b/oslo_messaging/_drivers/zmq_driver/server/zmq_incoming_message.py new file mode 100644 index 000000000..f43ec2325 --- /dev/null +++ b/oslo_messaging/_drivers/zmq_driver/server/zmq_incoming_message.py @@ -0,0 +1,55 @@ +# 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 import base +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 + + +LOG = logging.getLogger(__name__) + +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) + self.reply_socket = socket + self.reply_id = rep_id + self.received = None + self.poller = poller + + def reply(self, reply=None, failure=None, log_failure=True): + if failure is not None: + failure = rpc_common.serialize_remote_exception(failure, + log_failure) + message_reply = {zmq_names.FIELD_REPLY: reply, + zmq_names.FIELD_FAILURE: failure, + zmq_names.FIELD_LOG_FAILURE: log_failure} + + LOG.info("Replying %s REP", (str(message_reply))) + + self.received = True + self.reply_socket.send(self.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) + + def requeue(self): + """Requeue is not supported""" diff --git a/oslo_messaging/_drivers/zmq_driver/server/zmq_server.py b/oslo_messaging/_drivers/zmq_driver/server/zmq_server.py new file mode 100644 index 000000000..8f7f12657 --- /dev/null +++ b/oslo_messaging/_drivers/zmq_driver/server/zmq_server.py @@ -0,0 +1,80 @@ +# 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 copy +import logging + +from oslo_messaging._drivers import base +from oslo_messaging._drivers.zmq_driver.server.consumers\ + import zmq_router_consumer +from oslo_messaging._drivers.zmq_driver import zmq_address +from oslo_messaging._drivers.zmq_driver import zmq_async + +LOG = logging.getLogger(__name__) + +zmq = zmq_async.import_zmq() + + +class ZmqServer(base.Listener): + + def __init__(self, driver, conf, matchmaker=None): + 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) + self.notify_consumer = self.rpc_consumer + self.consumers = [self.rpc_consumer] + + def poll(self, timeout=None): + message, socket = self.poller.poll( + timeout or self.conf.rpc_poll_timeout) + return message + + def stop(self): + consumer = self.rpc_consumer + LOG.info("Stop server %s:%d" % (consumer.address, consumer.port)) + + def cleanup(self): + self.poller.close() + for consumer in self.consumers: + consumer.cleanup() + + def listen(self, target): + + consumer = self.rpc_consumer + consumer.listen(target) + + LOG.info("Listen to target %s on %s:%d" % + (target, consumer.address, consumer.port)) + + host = zmq_address.combine_address(self.conf.rpc_zmq_host, + consumer.port) + self.matchmaker.register(target=target, + hostname=host) + + def listen_notification(self, targets_and_priorities): + + consumer = self.notify_consumer + + LOG.info("Listen for notifications on %s:%d" + % (consumer.address, consumer.port)) + + for target, priority in targets_and_priorities: + host = zmq_address.combine_address(self.conf.rpc_zmq_host, + consumer.port) + t = copy.deepcopy(target) + t.topic = target.topic + '.' + priority + self.matchmaker.register(target=t, hostname=host) + consumer.listen(t) diff --git a/oslo_messaging/_drivers/zmq_driver/zmq_address.py b/oslo_messaging/_drivers/zmq_driver/zmq_address.py new file mode 100644 index 000000000..7feb05d89 --- /dev/null +++ b/oslo_messaging/_drivers/zmq_driver/zmq_address.py @@ -0,0 +1,25 @@ +# 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. + + +def combine_address(host, port): + return "%s:%s" % (host, port) + + +def get_tcp_direct_address(host): + return "tcp://%s" % (host) + + +def get_tcp_random_address(conf): + return "tcp://*" diff --git a/oslo_messaging/_drivers/zmq_driver/zmq_async.py b/oslo_messaging/_drivers/zmq_driver/zmq_async.py new file mode 100644 index 000000000..c228e7215 --- /dev/null +++ b/oslo_messaging/_drivers/zmq_driver/zmq_async.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. + +import logging + +from oslo_messaging._drivers.zmq_driver.poller import green_poller +from oslo_messaging._drivers.zmq_driver.poller import threading_poller +from oslo_messaging._i18n import _, _LE +from oslo_utils import importutils + +LOG = logging.getLogger(__name__) + +# Map zmq_concurrency config option names to the actual module name. +ZMQ_MODULES = { + 'native': 'zmq', + 'eventlet': 'eventlet.green.zmq', +} + + +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') + + if imported_zmq is None: + errmsg = _LE("ZeroMQ not found!") + LOG.error(_LE("ZeroMQ not found!")) + raise ImportError(errmsg) + return imported_zmq + + +def get_poller(zmq_concurrency='eventlet'): + _raise_error_if_invalid_config_value(zmq_concurrency) + + if zmq_concurrency == 'eventlet' and _is_eventlet_zmq_available(): + return green_poller.GreenPoller() + return threading_poller.ThreadingPoller() + + +def get_reply_poller(zmq_concurrency='eventlet'): + _raise_error_if_invalid_config_value(zmq_concurrency) + + if zmq_concurrency == 'eventlet' and _is_eventlet_zmq_available(): + return green_poller.HoldReplyPoller() + return threading_poller.ThreadingPoller() + + +def get_executor(method, zmq_concurrency='eventlet'): + _raise_error_if_invalid_config_value(zmq_concurrency) + + if zmq_concurrency == 'eventlet' and _is_eventlet_zmq_available(): + return green_poller.GreenExecutor(method) + return threading_poller.ThreadingExecutor(method) + + +def _is_eventlet_zmq_available(): + return importutils.try_import('eventlet.green.zmq') + + +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) diff --git a/oslo_messaging/_drivers/zmq_driver/zmq_names.py b/oslo_messaging/_drivers/zmq_driver/zmq_names.py new file mode 100644 index 000000000..1c3c33440 --- /dev/null +++ b/oslo_messaging/_drivers/zmq_driver/zmq_names.py @@ -0,0 +1,53 @@ +# 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.zmq_driver import zmq_async + +zmq = zmq_async.import_zmq() + + +ZMQ_SOCKET_STR = {zmq.DEALER: "DEALER", + zmq.ROUTER: "ROUTER", + zmq.PUSH: "PUSH", + zmq.PULL: "PULL", + zmq.REQ: "REQ", + zmq.REP: "REP", + zmq.PUB: "PUB", + zmq.SUB: "SUB"} + +FIELD_FAILURE = 'failure' +FIELD_REPLY = 'reply' +FIELD_LOG_FAILURE = 'log_failure' +FIELD_ID = 'id' + +CALL_TYPE = 'call' +CAST_TYPE = 'cast' +CAST_FANOUT_TYPE = 'cast-f' +NOTIFY_TYPE = 'notify' +NOTIFY_FANOUT_TYPE = 'notify-f' + +MESSAGE_TYPES = (CALL_TYPE, + CAST_TYPE, + CAST_FANOUT_TYPE, + NOTIFY_TYPE, + NOTIFY_FANOUT_TYPE) + +MULTISEND_TYPES = (CAST_FANOUT_TYPE, NOTIFY_FANOUT_TYPE) +DIRECT_TYPES = (CALL_TYPE, CAST_TYPE, NOTIFY_TYPE) +CAST_TYPES = (CAST_TYPE, CAST_FANOUT_TYPE) +NOTIFY_TYPES = (NOTIFY_TYPE, NOTIFY_FANOUT_TYPE) + + +def socket_type_str(socket_type): + return ZMQ_SOCKET_STR[socket_type] diff --git a/oslo_messaging/_drivers/zmq_driver/zmq_poller.py b/oslo_messaging/_drivers/zmq_driver/zmq_poller.py new file mode 100644 index 000000000..a62ea8a6f --- /dev/null +++ b/oslo_messaging/_drivers/zmq_driver/zmq_poller.py @@ -0,0 +1,106 @@ +# 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 abc + +import six + + +@six.add_metaclass(abc.ABCMeta) +class ZmqPoller(object): + + """Base poller interface + + Needed to poll on zmq sockets in green and native async manner. + Native poller implementation wraps zmq.Poller helper class. + Wrapping is needed to provide unified poller interface + in zmq-driver (for both native and zmq pollers). It makes some + difference with poller-helper from zmq library which doesn't actually + receive message. + + The poller object should be obtained over: + + poller = zmq_async.get_poller() + + Then we have to register sockets for polling. We are able + to provide specific receiving method. By default poller calls + socket.recv_multipart. + + def receive_message(socket): + id = socket.recv_string() + ctxt = socket.recv_json() + msg = socket.recv_json() + return (id, ctxt, msg) + + poller.register(socket, recv_method=receive_message) + + Further to receive a message we should call: + + message, socket = poller.poll() + + The 'message' here contains (id, ctxt, msg) tuple. + """ + + @abc.abstractmethod + def register(self, socket, recv_method=None): + """Register socket to poll + + :param socket: Socket to subscribe for polling + :type socket: zmq.Socket + :param recv_method: Optional specific receiver procedure + Should return received message object + :type recv_method: callable + """ + + @abc.abstractmethod + def poll(self, timeout=None): + """Poll for messages + + :param timeout: Optional polling timeout + None or -1 means poll forever + any positive value means timeout in seconds + :type timeout: int + :returns: (message, socket) tuple + """ + + @abc.abstractmethod + def close(self): + """Terminate polling""" + + def resume_polling(self, socket): + """Resume with polling + + Some implementations of poller may provide hold polling before reply + This method is intended to excplicitly resume polling aftewards. + """ + + +@six.add_metaclass(abc.ABCMeta) +class Executor(object): + """Base executor interface for threading/green async executors""" + + def __init__(self, thread): + self.thread = thread + + @abc.abstractmethod + def execute(self): + """Run execution""" + + @abc.abstractmethod + def stop(self): + """Stop execution""" + + @abc.abstractmethod + def wait(self): + """Wait until pass""" diff --git a/oslo_messaging/_drivers/zmq_driver/zmq_socket.py b/oslo_messaging/_drivers/zmq_driver/zmq_socket.py new file mode 100644 index 000000000..2a4144c5a --- /dev/null +++ b/oslo_messaging/_drivers/zmq_driver/zmq_socket.py @@ -0,0 +1,82 @@ +# 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 import zmq_address +from oslo_messaging._drivers.zmq_driver import zmq_async +from oslo_messaging._drivers.zmq_driver import zmq_names + +LOG = logging.getLogger(__name__) + +zmq = zmq_async.import_zmq() + + +class ZmqSocket(object): + + def __init__(self, context, socket_type): + self.context = context + self.socket_type = socket_type + self.handle = context.socket(socket_type) + self.connections = set() + + def type_name(self): + return zmq_names.socket_type_str(self.socket_type) + + def connections_count(self): + return len(self.connections) + + def connect(self, address): + if address not in self.connections: + self.handle.connect(address) + self.connections.add(address) + + def setsockopt(self, *args, **kwargs): + self.handle.setsockopt(*args, **kwargs) + + def send(self, *args, **kwargs): + self.handle.send(*args, **kwargs) + + def send_string(self, *args, **kwargs): + self.handle.send_string(*args, **kwargs) + + def send_json(self, *args, **kwargs): + self.handle.send_json(*args, **kwargs) + + def send_pyobj(self, *args, **kwargs): + self.handle.send_pyobj(*args, **kwargs) + + def recv(self, *args, **kwargs): + return self.handle.recv(*args, **kwargs) + + def recv_string(self, *args, **kwargs): + return self.handle.recv_string(*args, **kwargs) + + def recv_json(self, *args, **kwargs): + return self.handle.recv_json(*args, **kwargs) + + def recv_pyobj(self, *args, **kwargs): + return self.handle.recv_pyobj(*args, **kwargs) + + def close(self, *args, **kwargs): + self.handle.close(*args, **kwargs) + + +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) diff --git a/oslo_messaging/conffixture.py b/oslo_messaging/conffixture.py index 8a44f9e62..bbf92caf0 100644 --- a/oslo_messaging/conffixture.py +++ b/oslo_messaging/conffixture.py @@ -59,7 +59,8 @@ class ConfFixture(fixtures.Fixture): _import_opts(self.conf, 'oslo_messaging._drivers.impl_zmq', 'zmq_opts') _import_opts(self.conf, - 'oslo_messaging._drivers.matchmaker_redis', + 'oslo_messaging._drivers.zmq_driver.' + 'matchmaker.matchmaker_redis', 'matchmaker_redis_opts', 'matchmaker_redis') _import_opts(self.conf, 'oslo_messaging.rpc.client', '_client_opts') diff --git a/oslo_messaging/opts.py b/oslo_messaging/opts.py index 267b7ec16..c5856595d 100644 --- a/oslo_messaging/opts.py +++ b/oslo_messaging/opts.py @@ -25,10 +25,8 @@ 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 import matchmaker -from oslo_messaging._drivers import matchmaker_redis -from oslo_messaging._drivers import matchmaker_ring from oslo_messaging._drivers.protocols.amqp import opts as amqp_opts +from oslo_messaging._drivers.zmq_driver.matchmaker import matchmaker_redis from oslo_messaging._executors import impl_pooledexecutor from oslo_messaging.notify import notifier from oslo_messaging.rpc import client @@ -37,7 +35,7 @@ from oslo_messaging import transport _global_opt_lists = [ drivers_base.base_opts, impl_zmq.zmq_opts, - matchmaker.matchmaker_opts, + matchmaker_redis.matchmaker_redis_opts, impl_pooledexecutor._pool_opts, notifier._notifier_opts, client._client_opts, @@ -47,7 +45,6 @@ _global_opt_lists = [ _opts = [ (None, list(itertools.chain(*_global_opt_lists))), ('matchmaker_redis', matchmaker_redis.matchmaker_redis_opts), - ('matchmaker_ring', matchmaker_ring.matchmaker_opts), ('oslo_messaging_amqp', amqp_opts.amqp1_opts), ('oslo_messaging_rabbit', list(itertools.chain(amqp.amqp_opts, impl_rabbit.rabbit_opts))), diff --git a/oslo_messaging/tests/drivers/test_impl_zmq.py b/oslo_messaging/tests/drivers/test_impl_zmq.py deleted file mode 100644 index 85e5dd377..000000000 --- a/oslo_messaging/tests/drivers/test_impl_zmq.py +++ /dev/null @@ -1,473 +0,0 @@ -# Copyright 2014 Canonical, Ltd. -# All Rights Reserved. -# -# 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 socket - -import fixtures -from oslo_utils import importutils -import testtools - -try: - import zmq -except ImportError: - zmq = None - -import oslo_messaging -from oslo_messaging._drivers import common as rpc_common -from oslo_messaging.tests import utils as test_utils -from six.moves import mock - -# eventlet is not yet py3 compatible, so skip if not installed -eventlet = importutils.try_import('eventlet') - -impl_zmq = importutils.try_import('oslo_messaging._drivers.impl_zmq') - -LOG = logging.getLogger(__name__) - - -def get_unused_port(): - """Returns an unused port on localhost.""" - s = socket.socket(socket.AF_INET, socket.SOCK_STREAM) - s.setsockopt(socket.SOL_SOCKET, socket.SO_REUSEADDR, 1) - s.bind(('localhost', 0)) - port = s.getsockname()[1] - s.close() - return port - - -class ZmqBaseTestCase(test_utils.BaseTestCase): - """Base test case for all ZMQ tests that make use of the ZMQ Proxy""" - - @testtools.skipIf(zmq is None, "zmq not available") - def setUp(self): - super(ZmqBaseTestCase, self).setUp() - self.messaging_conf.transport_driver = 'zmq' - # Get driver - transport = oslo_messaging.get_transport(self.conf) - self.driver = transport._driver - - # Set config values - self.internal_ipc_dir = self.useFixture(fixtures.TempDir()).path - kwargs = {'rpc_zmq_bind_address': '127.0.0.1', - 'rpc_zmq_host': '127.0.0.1', - 'rpc_response_timeout': 5, - 'rpc_zmq_port': get_unused_port(), - 'rpc_zmq_ipc_dir': self.internal_ipc_dir} - self.config(**kwargs) - - # Start RPC - LOG.info("Running internal zmq receiver.") - self.reactor = impl_zmq.ZmqProxy(self.conf) - self.reactor.consume_in_thread() - - self.matchmaker = impl_zmq._get_matchmaker(host='127.0.0.1') - self.addCleanup(stopRpc(self.__dict__)) - - -class TestConfZmqDriverLoad(test_utils.BaseTestCase): - - @testtools.skipIf(zmq is None, "zmq not available") - def setUp(self): - super(TestConfZmqDriverLoad, self).setUp() - self.messaging_conf.transport_driver = 'zmq' - - def test_driver_load(self): - transport = oslo_messaging.get_transport(self.conf) - self.assertIsInstance(transport._driver, impl_zmq.ZmqDriver) - - -class stopRpc(object): - def __init__(self, attrs): - self.attrs = attrs - - def __call__(self): - if self.attrs['reactor']: - self.attrs['reactor'].close() - if self.attrs['driver']: - self.attrs['driver'].cleanup() - - -class TestZmqBasics(ZmqBaseTestCase): - - def test_start_stop_listener(self): - target = oslo_messaging.Target(topic='testtopic') - listener = self.driver.listen(target) - result = listener.poll(0.01) - self.assertEqual(result, None) - - def test_send_receive_raises(self): - """Call() without method.""" - target = oslo_messaging.Target(topic='testtopic') - self.driver.listen(target) - self.assertRaises( - KeyError, - self.driver.send, - target, {}, {'tx_id': 1}, wait_for_reply=True) - - @mock.patch('oslo_messaging._drivers.impl_zmq.ZmqIncomingMessage') - def test_send_receive_topic(self, mock_msg): - """Call() with method.""" - mock_msg.return_value = msg = mock.MagicMock() - msg.received = received = mock.MagicMock() - received.failure = False - received.reply = True - msg.condition = condition = mock.MagicMock() - condition.wait.return_value = True - - target = oslo_messaging.Target(topic='testtopic') - self.driver.listen(target) - result = self.driver.send( - target, {}, - {'method': 'hello-world', 'tx_id': 1}, - wait_for_reply=True) - self.assertEqual(result, True) - - @mock.patch('oslo_messaging._drivers.impl_zmq._call', autospec=True) - def test_send_receive_fanout(self, mock_call): - target = oslo_messaging.Target(topic='testtopic', fanout=True) - self.driver.listen(target) - - mock_call.__name__ = '_call' - mock_call.return_value = [True] - - result = self.driver.send( - target, {}, - {'method': 'hello-world', 'tx_id': 1}, - wait_for_reply=True) - - self.assertEqual(result, True) - mock_call.assert_called_once_with( - self.driver, - 'tcp://127.0.0.1:%s' % self.conf['rpc_zmq_port'], - {}, 'fanout~testtopic.127.0.0.1', - {'tx_id': 1, 'method': 'hello-world'}, - None, False, [], True) - - @mock.patch('oslo_messaging._drivers.impl_zmq._call', autospec=True) - def test_send_receive_direct(self, mock_call): - # Also verifies fix for bug http://pad.lv/1301723 - target = oslo_messaging.Target(topic='testtopic', server='localhost') - self.driver.listen(target) - - mock_call.__name__ = '_call' - mock_call.return_value = [True] - - result = self.driver.send( - target, {}, - {'method': 'hello-world', 'tx_id': 1}, - wait_for_reply=True) - - self.assertEqual(result, True) - mock_call.assert_called_once_with( - self.driver, - 'tcp://localhost:%s' % self.conf['rpc_zmq_port'], - {}, 'testtopic.localhost', - {'tx_id': 1, 'method': 'hello-world'}, - None, False, [], True) - - -class TestZmqSocket(test_utils.BaseTestCase): - - @testtools.skipIf(zmq is None, "zmq not available") - def setUp(self): - super(TestZmqSocket, self).setUp() - self.messaging_conf.transport_driver = 'zmq' - # Get driver - transport = oslo_messaging.get_transport(self.conf) - self.driver = transport._driver - - @mock.patch('oslo_messaging._drivers.impl_zmq.ZmqSocket.subscribe') - @mock.patch('oslo_messaging._drivers.impl_zmq.zmq.Context') - def test_zmqsocket_init_type_pull(self, mock_context, mock_subscribe): - mock_ctxt = mock.Mock() - mock_context.return_value = mock_ctxt - mock_sock = mock.Mock() - mock_ctxt.socket = mock.Mock(return_value=mock_sock) - mock_sock.connect = mock.Mock() - mock_sock.bind = mock.Mock() - addr = '127.0.0.1' - - sock = impl_zmq.ZmqSocket(addr, impl_zmq.zmq.PULL, bind=False, - subscribe=None) - self.assertTrue(sock.can_recv) - self.assertFalse(sock.can_send) - self.assertFalse(sock.can_sub) - self.assertTrue(mock_sock.connect.called) - self.assertFalse(mock_sock.bind.called) - - @mock.patch('oslo_messaging._drivers.impl_zmq.ZmqSocket.subscribe') - @mock.patch('oslo_messaging._drivers.impl_zmq.zmq.Context') - def test_zmqsocket_init_type_sub(self, mock_context, mock_subscribe): - mock_ctxt = mock.Mock() - mock_context.return_value = mock_ctxt - mock_sock = mock.Mock() - mock_ctxt.socket = mock.Mock(return_value=mock_sock) - mock_sock.connect = mock.Mock() - mock_sock.bind = mock.Mock() - addr = '127.0.0.1' - - sock = impl_zmq.ZmqSocket(addr, impl_zmq.zmq.SUB, bind=False, - subscribe=None) - self.assertTrue(sock.can_recv) - self.assertFalse(sock.can_send) - self.assertTrue(sock.can_sub) - self.assertTrue(mock_sock.connect.called) - self.assertFalse(mock_sock.bind.called) - - @mock.patch('oslo_messaging._drivers.impl_zmq.ZmqSocket.subscribe') - @mock.patch('oslo_messaging._drivers.impl_zmq.zmq.Context') - def test_zmqsocket_init_type_push(self, mock_context, mock_subscribe): - mock_ctxt = mock.Mock() - mock_context.return_value = mock_ctxt - mock_sock = mock.Mock() - mock_ctxt.socket = mock.Mock(return_value=mock_sock) - mock_sock.connect = mock.Mock() - mock_sock.bind = mock.Mock() - addr = '127.0.0.1' - - sock = impl_zmq.ZmqSocket(addr, impl_zmq.zmq.PUSH, bind=False, - subscribe=None) - self.assertFalse(sock.can_recv) - self.assertTrue(sock.can_send) - self.assertFalse(sock.can_sub) - self.assertTrue(mock_sock.connect.called) - self.assertFalse(mock_sock.bind.called) - - @mock.patch('oslo_messaging._drivers.impl_zmq.ZmqSocket.subscribe') - @mock.patch('oslo_messaging._drivers.impl_zmq.zmq.Context') - def test_zmqsocket_init_type_pub(self, mock_context, mock_subscribe): - mock_ctxt = mock.Mock() - mock_context.return_value = mock_ctxt - mock_sock = mock.Mock() - mock_ctxt.socket = mock.Mock(return_value=mock_sock) - mock_sock.connect = mock.Mock() - mock_sock.bind = mock.Mock() - addr = '127.0.0.1' - - sock = impl_zmq.ZmqSocket(addr, impl_zmq.zmq.PUB, bind=False, - subscribe=None) - self.assertFalse(sock.can_recv) - self.assertTrue(sock.can_send) - self.assertFalse(sock.can_sub) - self.assertTrue(mock_sock.connect.called) - self.assertFalse(mock_sock.bind.called) - - -class TestZmqIncomingMessage(test_utils.BaseTestCase): - - @testtools.skipIf(zmq is None, "zmq not available") - def setUp(self): - super(TestZmqIncomingMessage, self).setUp() - self.messaging_conf.transport_driver = 'zmq' - # Get driver - transport = oslo_messaging.get_transport(self.conf) - self.driver = transport._driver - - def test_zmqincomingmessage(self): - msg = impl_zmq.ZmqIncomingMessage(mock.Mock(), None, 'msg.foo') - msg.reply("abc") - self.assertIsInstance( - msg.received, impl_zmq.ZmqIncomingMessage.ReceivedReply) - self.assertIsInstance( - msg.received, impl_zmq.ZmqIncomingMessage.ReceivedReply) - self.assertEqual(msg.received.reply, "abc") - msg.requeue() - - -class TestZmqConnection(ZmqBaseTestCase): - - @mock.patch('oslo_messaging._drivers.impl_zmq.ZmqReactor', autospec=True) - def test_zmqconnection_create_consumer(self, mock_reactor): - - mock_reactor.register = mock.Mock() - conn = impl_zmq.Connection(self.driver.conf, self.driver) - topic = 'topic.foo' - context = mock.Mock() - inaddr = ('ipc://%s/zmq_topic_topic.127.0.0.1' % - (self.internal_ipc_dir)) - # No Fanout - conn.create_consumer(topic, context) - conn.reactor.register.assert_called_with(context, inaddr, - impl_zmq.zmq.PULL, - subscribe=None, in_bind=False) - - # Reset for next bunch of checks - conn.reactor.register.reset_mock() - - # Fanout - inaddr = ('ipc://%s/zmq_topic_fanout~topic' % - (self.internal_ipc_dir)) - conn.create_consumer(topic, context, fanout='subscriber.foo') - conn.reactor.register.assert_called_with(context, inaddr, - impl_zmq.zmq.SUB, - subscribe='subscriber.foo', - in_bind=False) - - @mock.patch('oslo_messaging._drivers.impl_zmq.ZmqReactor', autospec=True) - def test_zmqconnection_create_consumer_topic_exists(self, mock_reactor): - mock_reactor.register = mock.Mock() - conn = impl_zmq.Connection(self.driver.conf, self.driver) - topic = 'topic.foo' - context = mock.Mock() - inaddr = ('ipc://%s/zmq_topic_topic.127.0.0.1' % - (self.internal_ipc_dir)) - - conn.create_consumer(topic, context) - conn.reactor.register.assert_called_with( - context, inaddr, impl_zmq.zmq.PULL, subscribe=None, in_bind=False) - conn.reactor.register.reset_mock() - # Call again with same topic - conn.create_consumer(topic, context) - self.assertFalse(conn.reactor.register.called) - - @mock.patch('oslo_messaging._drivers.impl_zmq._get_matchmaker', - autospec=True) - @mock.patch('oslo_messaging._drivers.impl_zmq.ZmqReactor', autospec=True) - def test_zmqconnection_close(self, mock_reactor, mock_getmatchmaker): - conn = impl_zmq.Connection(self.driver.conf, self.driver) - conn.reactor.close = mock.Mock() - mock_getmatchmaker.return_value.stop_heartbeat = mock.Mock() - conn.close() - self.assertTrue(mock_getmatchmaker.return_value.stop_heartbeat.called) - self.assertTrue(conn.reactor.close.called) - - @mock.patch('oslo_messaging._drivers.impl_zmq.ZmqReactor', autospec=True) - def test_zmqconnection_wait(self, mock_reactor): - conn = impl_zmq.Connection(self.driver, self.driver) - conn.reactor.wait = mock.Mock() - conn.wait() - self.assertTrue(conn.reactor.wait.called) - - @mock.patch('oslo_messaging._drivers.impl_zmq._get_matchmaker', - autospec=True) - @mock.patch('oslo_messaging._drivers.impl_zmq.ZmqReactor', autospec=True) - def test_zmqconnection_consume_in_thread(self, mock_reactor, - mock_getmatchmaker): - mock_getmatchmaker.return_value.start_heartbeat = mock.Mock() - conn = impl_zmq.Connection(self.driver, self.driver) - conn.reactor.consume_in_thread = mock.Mock() - conn.consume_in_thread() - self.assertTrue(mock_getmatchmaker.return_value.start_heartbeat.called) - self.assertTrue(conn.reactor.consume_in_thread.called) - - -class TestZmqListener(ZmqBaseTestCase): - - def test_zmqlistener_no_msg(self): - listener = impl_zmq.ZmqListener(self.driver) - # Timeout = 0 should return straight away since the queue is empty - listener.poll(timeout=0) - - def test_zmqlistener_w_msg(self): - listener = impl_zmq.ZmqListener(self.driver) - kwargs = {'a': 1, 'b': 2} - m = mock.Mock() - ctxt = mock.Mock(autospec=impl_zmq.RpcContext) - message = {'namespace': 'name.space', 'method': m.fake_method, - 'args': kwargs} - eventlet.spawn_n(listener.dispatch, ctxt, message) - resp = listener.poll(timeout=10) - msg = {'method': m.fake_method, 'namespace': 'name.space', - 'args': kwargs} - self.assertEqual(resp.message, msg) - - -class TestZmqDriver(ZmqBaseTestCase): - - @mock.patch('oslo_messaging._drivers.impl_zmq._cast', autospec=True) - @mock.patch('oslo_messaging._drivers.matchmaker.MatchMakerBase.queues', - autospec=True) - def test_zmqdriver_multi_send_cast_with_no_queues(self, - mock_queues, - mock_cast): - context = mock.Mock(autospec=impl_zmq.RpcContext) - topic = 'testtopic' - msg = 'jeronimo' - - with mock.patch.object(impl_zmq.LOG, 'warn') as flog: - mock_queues.return_value = None - impl_zmq._multi_send(self.driver, mock_cast, - context, topic, msg) - self.assertEqual(1, flog.call_count) - args, kwargs = flog.call_args - self.assertIn('No matchmaker results', args[0]) - - @mock.patch('oslo_messaging._drivers.impl_zmq._call', autospec=True) - @mock.patch('oslo_messaging._drivers.matchmaker.MatchMakerBase.queues', - autospec=True) - def test_zmqdriver_multi_send_call_with_no_queues(self, - mock_queues, - mock_call): - context = mock.Mock(autospec=impl_zmq.RpcContext) - topic = 'testtopic' - msg = 'jeronimo' - - mock_queues.return_value = None - self.assertRaises(rpc_common.Timeout, - impl_zmq._multi_send, self.driver, - mock_call, context, topic, msg) - - @mock.patch('oslo_messaging._drivers.impl_zmq._cast', autospec=True) - @mock.patch('oslo_messaging._drivers.impl_zmq._multi_send', autospec=True) - def test_zmqdriver_send(self, mock_multi_send, mock_cast): - context = mock.Mock(autospec=impl_zmq.RpcContext) - topic = 'testtopic' - msg = 'jeronimo' - self.driver.send(oslo_messaging.Target(topic=topic), context, msg, - False, 0, False) - mock_multi_send.assert_called_with(self.driver, mock_cast, context, - topic, msg, - allowed_remote_exmods=[], - envelope=False, pooled=True) - - @mock.patch('oslo_messaging._drivers.impl_zmq._cast', autospec=True) - @mock.patch('oslo_messaging._drivers.impl_zmq._multi_send', autospec=True) - def test_zmqdriver_send_notification(self, mock_multi_send, mock_cast): - context = mock.Mock(autospec=impl_zmq.RpcContext) - topic = 'testtopic.foo' - topic_reformat = 'testtopic-foo' - msg = 'jeronimo' - self.driver.send_notification(oslo_messaging.Target(topic=topic), - context, msg, False, False) - mock_multi_send.assert_called_with(self.driver, mock_cast, context, - topic_reformat, msg, - allowed_remote_exmods=[], - envelope=False, pooled=True) - - @mock.patch('oslo_messaging._drivers.impl_zmq.ZmqListener', autospec=True) - @mock.patch('oslo_messaging._drivers.impl_zmq.Connection', autospec=True) - def test_zmqdriver_listen(self, mock_connection, mock_listener): - mock_listener.return_value = listener = mock.Mock() - mock_connection.return_value = conn = mock.Mock() - conn.create_consumer = mock.Mock() - conn.consume_in_thread = mock.Mock() - topic = 'testtopic.foo' - self.driver.listen(oslo_messaging.Target(topic=topic)) - conn.create_consumer.assert_called_with(topic, listener, fanout=True) - - @mock.patch('oslo_messaging._drivers.impl_zmq.ZmqListener', autospec=True) - @mock.patch('oslo_messaging._drivers.impl_zmq.Connection', autospec=True) - def test_zmqdriver_listen_for_notification(self, mock_connection, - mock_listener): - mock_listener.return_value = listener = mock.Mock() - mock_connection.return_value = conn = mock.Mock() - conn.create_consumer = mock.Mock() - conn.consume_in_thread = mock.Mock() - topic = 'testtopic.foo' - data = [(oslo_messaging.Target(topic=topic), 0)] - # NOTE(jamespage): Pooling not supported, just pass None for now. - self.driver.listen_for_notifications(data, None) - conn.create_consumer.assert_called_with("%s-%s" % (topic, 0), listener) diff --git a/oslo_messaging/tests/drivers/test_matchmaker.py b/oslo_messaging/tests/drivers/test_matchmaker.py deleted file mode 100644 index 61c37a92b..000000000 --- a/oslo_messaging/tests/drivers/test_matchmaker.py +++ /dev/null @@ -1,69 +0,0 @@ -# Copyright 2014 Canonical, Ltd. -# -# 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_utils import importutils -import testtools - -from oslo_messaging.tests import utils as test_utils - -# NOTE(jamespage) matchmaker tied directly to eventlet -# which is not yet py3 compatible - skip if import fails -matchmaker = ( - importutils.try_import('oslo_messaging._drivers.matchmaker')) - - -@testtools.skipIf(not matchmaker, "matchmaker/eventlet unavailable") -class MatchmakerTest(test_utils.BaseTestCase): - - def test_fanout_binding(self): - matcher = matchmaker.MatchMakerBase() - matcher.add_binding( - matchmaker.FanoutBinding(), matchmaker.DirectExchange()) - self.assertEqual(matcher.queues('hello.world'), []) - self.assertEqual( - matcher.queues('fanout~fantasy.unicorn'), - [('fanout~fantasy.unicorn', 'unicorn')]) - self.assertEqual( - matcher.queues('fanout~fantasy.pony'), - [('fanout~fantasy.pony', 'pony')]) - - def test_topic_binding(self): - matcher = matchmaker.MatchMakerBase() - matcher.add_binding( - matchmaker.TopicBinding(), matchmaker.StubExchange()) - self.assertEqual( - matcher.queues('hello-world'), [('hello-world', None)]) - - def test_direct_binding(self): - matcher = matchmaker.MatchMakerBase() - matcher.add_binding( - matchmaker.DirectBinding(), matchmaker.StubExchange()) - self.assertEqual( - matcher.queues('hello.server'), [('hello.server', None)]) - self.assertEqual(matcher.queues('hello-world'), []) - - def test_localhost_match(self): - matcher = matchmaker.MatchMakerLocalhost() - self.assertEqual( - matcher.queues('hello.server'), [('hello.server', 'server')]) - - # Gets remapped due to localhost exchange - # all bindings default to first match. - self.assertEqual( - matcher.queues('fanout~testing.server'), - [('fanout~testing.localhost', 'localhost')]) - - self.assertEqual( - matcher.queues('hello-world'), - [('hello-world.localhost', 'localhost')]) diff --git a/oslo_messaging/tests/drivers/test_matchmaker_redis.py b/oslo_messaging/tests/drivers/test_matchmaker_redis.py deleted file mode 100644 index f2498cd6d..000000000 --- a/oslo_messaging/tests/drivers/test_matchmaker_redis.py +++ /dev/null @@ -1,97 +0,0 @@ -# Copyright 2014 Canonical, Ltd. -# -# 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_utils import importutils -import testtools - -from oslo_messaging.tests import utils as test_utils - -redis = importutils.try_import('redis') -matchmaker_redis = ( - importutils.try_import('oslo_messaging._drivers.matchmaker_redis')) - - -def redis_available(): - '''Helper to see if local redis server is running''' - if not redis: - return False - try: - c = redis.StrictRedis(socket_timeout=1) - c.ping() - return True - except redis.exceptions.ConnectionError: - return False - - -@testtools.skipIf(not matchmaker_redis, "matchmaker/eventlet unavailable") -@testtools.skipIf(not redis_available(), "redis unavailable") -class RedisMatchMakerTest(test_utils.BaseTestCase): - - def setUp(self): - super(RedisMatchMakerTest, self).setUp() - self.ring_data = { - "conductor": ["controller1", "node1", "node2", "node3"], - "scheduler": ["controller1", "node1", "node2", "node3"], - "network": ["controller1", "node1", "node2", "node3"], - "cert": ["controller1"], - "console": ["controller1"], - "l3_agent.node1": ["node1"], - "consoleauth": ["controller1"]} - self.matcher = matchmaker_redis.MatchMakerRedis() - self.populate() - - def tearDown(self): - super(RedisMatchMakerTest, self).tearDown() - c = redis.StrictRedis() - c.flushdb() - - def populate(self): - for k, hosts in self.ring_data.items(): - for h in hosts: - self.matcher.register(k, h) - - def test_direct(self): - self.assertEqual( - self.matcher.queues('cert.controller1'), - [('cert.controller1', 'controller1')]) - - def test_register(self): - self.matcher.register('cert', 'keymaster') - self.assertEqual( - sorted(self.matcher.redis.smembers('cert')), - [b'cert.controller1', b'cert.keymaster']) - self.matcher.register('l3_agent.node1', 'node1') - self.assertEqual( - sorted(self.matcher.redis.smembers('l3_agent.node1')), - [b'l3_agent.node1.node1']) - - def test_unregister(self): - self.matcher.unregister('conductor', 'controller1') - self.assertEqual( - sorted(self.matcher.redis.smembers('conductor')), - [b'conductor.node1', b'conductor.node2', b'conductor.node3']) - - def test_ack_alive(self): - self.matcher.ack_alive('ack_alive', 'controller1') - self.assertEqual( - sorted(self.matcher.redis.smembers('ack_alive')), - [b'ack_alive.controller1']) - - def test_is_alive(self): - self.assertEqual( - self.matcher.is_alive('conductor', 'conductor.controller1'), - True) - self.assertEqual( - self.matcher.is_alive('conductor', 'conductor.controller2'), - False) diff --git a/oslo_messaging/tests/drivers/test_matchmaker_ring.py b/oslo_messaging/tests/drivers/test_matchmaker_ring.py deleted file mode 100644 index 5f156007a..000000000 --- a/oslo_messaging/tests/drivers/test_matchmaker_ring.py +++ /dev/null @@ -1,73 +0,0 @@ -# Copyright 2014 Canonical, Ltd. -# -# 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_utils import importutils -import testtools - -from oslo_messaging.tests import utils as test_utils - -# NOTE(jamespage) matchmaker tied directly to eventlet -# which is not yet py3 compatible - skip if import fails -matchmaker_ring = ( - importutils.try_import('oslo_messaging._drivers.matchmaker_ring')) - - -@testtools.skipIf(not matchmaker_ring, "matchmaker/eventlet unavailable") -class MatchmakerRingTest(test_utils.BaseTestCase): - - def setUp(self): - super(MatchmakerRingTest, self).setUp() - self.ring_data = { - "conductor": ["controller1", "node1", "node2", "node3"], - "scheduler": ["controller1", "node1", "node2", "node3"], - "network": ["controller1", "node1", "node2", "node3"], - "cert": ["controller1"], - "console": ["controller1"], - "consoleauth": ["controller1"]} - self.matcher = matchmaker_ring.MatchMakerRing(self.ring_data) - - def test_direct(self): - self.assertEqual( - self.matcher.queues('cert.controller1'), - [('cert.controller1', 'controller1')]) - self.assertEqual( - self.matcher.queues('conductor.node1'), - [('conductor.node1', 'node1')]) - - def test_fanout(self): - self.assertEqual( - self.matcher.queues('fanout~conductor'), - [('fanout~conductor.controller1', 'controller1'), - ('fanout~conductor.node1', 'node1'), - ('fanout~conductor.node2', 'node2'), - ('fanout~conductor.node3', 'node3')]) - - def test_bare_topic(self): - # Round robins through the hosts on the topic - self.assertEqual( - self.matcher.queues('scheduler'), - [('scheduler.controller1', 'controller1')]) - self.assertEqual( - self.matcher.queues('scheduler'), - [('scheduler.node1', 'node1')]) - self.assertEqual( - self.matcher.queues('scheduler'), - [('scheduler.node2', 'node2')]) - self.assertEqual( - self.matcher.queues('scheduler'), - [('scheduler.node3', 'node3')]) - # Cycles loop - self.assertEqual( - self.matcher.queues('scheduler'), - [('scheduler.controller1', 'controller1')]) diff --git a/oslo_messaging/tests/drivers/zmq/__init__.py b/oslo_messaging/tests/drivers/zmq/__init__.py new file mode 100644 index 000000000..e69de29bb diff --git a/oslo_messaging/_cmd/__init__.py b/oslo_messaging/tests/drivers/zmq/matchmaker/__init__.py similarity index 100% rename from oslo_messaging/_cmd/__init__.py rename to oslo_messaging/tests/drivers/zmq/matchmaker/__init__.py diff --git a/oslo_messaging/tests/drivers/zmq/matchmaker/test_impl_matchmaker.py b/oslo_messaging/tests/drivers/zmq/matchmaker/test_impl_matchmaker.py new file mode 100644 index 000000000..1f04920c8 --- /dev/null +++ b/oslo_messaging/tests/drivers/zmq/matchmaker/test_impl_matchmaker.py @@ -0,0 +1,80 @@ +# Copyright 2014 Canonical, Ltd. +# +# 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 stevedore import driver +import testscenarios + +import oslo_messaging +from oslo_messaging.tests import utils as test_utils + + +load_tests = testscenarios.load_tests_apply_scenarios + + +class TestImplMatchmaker(test_utils.BaseTestCase): + + scenarios = [ + ("dummy", {"rpc_zmq_matchmaker": "dummy"}), + ("redis", {"rpc_zmq_matchmaker": "redis"}), + ] + + def setUp(self): + super(TestImplMatchmaker, self).setUp() + + self.test_matcher = driver.DriverManager( + 'oslo.messaging.zmq.matchmaker', + self.rpc_zmq_matchmaker, + ).driver(self.conf) + + if self.rpc_zmq_matchmaker == "redis": + self.addCleanup(self.test_matcher._redis.flushdb) + + self.target = oslo_messaging.Target(topic="test_topic") + self.host1 = b"test_host1" + self.host2 = b"test_host2" + + def test_register(self): + self.test_matcher.register(self.target, self.host1) + + self.assertEqual(self.test_matcher.get_hosts(self.target), + [self.host1]) + self.assertEqual(self.test_matcher.get_single_host(self.target), + 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.assertItemsEqual(self.test_matcher.get_hosts(self.target), + [self.host1, self.host2]) + self.assertIn(self.test_matcher.get_single_host(self.target), + [self.host1, 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.assertEqual(self.test_matcher.get_hosts(self.target), + [self.host1]) + self.assertEqual(self.test_matcher.get_single_host(self.target), + 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), []) + + 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) diff --git a/oslo_messaging/tests/drivers/zmq/test_impl_zmq.py b/oslo_messaging/tests/drivers/zmq/test_impl_zmq.py new file mode 100644 index 000000000..21641dd51 --- /dev/null +++ b/oslo_messaging/tests/drivers/zmq/test_impl_zmq.py @@ -0,0 +1,246 @@ +# 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 + +import fixtures +import testtools + +import oslo_messaging +from oslo_messaging._drivers import impl_zmq +from oslo_messaging._drivers.zmq_driver import zmq_async +from oslo_messaging._i18n import _ +from oslo_messaging.tests import utils as test_utils + +LOG = logging.getLogger(__name__) + +zmq = zmq_async.import_zmq() + + +class TestServerListener(object): + + def __init__(self, driver): + self.driver = driver + self.listener = None + self.executor = zmq_async.get_executor(self._run) + self._stop = threading.Event() + self._received = threading.Event() + self.message = None + + def listen(self, target): + self.listener = self.driver.listen(target) + self.executor.execute() + + def listen_notifications(self, targets_and_priorities): + self.listener = self.driver.listen_for_notifications( + targets_and_priorities, {}) + self.executor.execute() + + def _run(self): + try: + message = self.listener.poll() + if message is not None: + message.acknowledge() + self._received.set() + self.message = message + message.reply(reply=True) + except Exception: + LOG.exception(_("Unexpected exception occurred.")) + + def stop(self): + self.executor.stop() + + +class ZmqBaseTestCase(test_utils.BaseTestCase): + """Base test case for all ZMQ tests """ + + @testtools.skipIf(zmq is None, "zmq not available") + def setUp(self): + super(ZmqBaseTestCase, self).setUp() + self.messaging_conf.transport_driver = 'zmq' + + # Set config values + self.internal_ipc_dir = self.useFixture(fixtures.TempDir()).path + kwargs = {'rpc_zmq_bind_address': '127.0.0.1', + 'rpc_zmq_host': '127.0.0.1', + 'rpc_response_timeout': 5, + 'rpc_zmq_ipc_dir': self.internal_ipc_dir, + 'rpc_zmq_matchmaker': 'dummy'} + self.config(**kwargs) + + # Get driver + transport = oslo_messaging.get_transport(self.conf) + self.driver = transport._driver + + self.listener = TestServerListener(self.driver) + + self.addCleanup(stopRpc(self.__dict__)) + + +class TestConfZmqDriverLoad(test_utils.BaseTestCase): + + @testtools.skipIf(zmq is None, "zmq not available") + def setUp(self): + super(TestConfZmqDriverLoad, self).setUp() + self.messaging_conf.transport_driver = 'zmq' + + def test_driver_load(self): + transport = oslo_messaging.get_transport(self.conf) + self.assertIsInstance(transport._driver, impl_zmq.ZmqDriver) + + +class stopRpc(object): + def __init__(self, attrs): + self.attrs = attrs + + def __call__(self): + if self.attrs['driver']: + self.attrs['driver'].cleanup() + if self.attrs['listener']: + self.attrs['listener'].stop() + + +class TestZmqBasics(ZmqBaseTestCase): + + def test_send_receive_raises(self): + """Call() without method.""" + target = oslo_messaging.Target(topic='testtopic') + self.listener.listen(target) + self.assertRaises( + KeyError, + self.driver.send, + target, {}, {'tx_id': 1}, wait_for_reply=True) + + def test_send_receive_topic(self): + """Call() with topic.""" + + target = oslo_messaging.Target(topic='testtopic') + self.listener.listen(target) + result = self.driver.send( + target, {}, + {'method': 'hello-world', 'tx_id': 1}, + wait_for_reply=True) + self.assertTrue(result) + + def test_send_noreply(self): + """Cast() with topic.""" + + target = oslo_messaging.Target(topic='testtopic', server="my@server") + self.listener.listen(target) + result = self.driver.send( + target, {}, + {'method': 'hello-world', 'tx_id': 1}, + wait_for_reply=False) + + self.listener._received.wait() + + self.assertIsNone(result) + self.assertEqual(True, self.listener._received.isSet()) + method = self.listener.message.message[u'method'] + self.assertEqual(u'hello-world', method) + + def test_send_fanout(self): + target = oslo_messaging.Target(topic='testtopic', fanout=True) + self.listener.listen(target) + + result = self.driver.send( + target, {}, + {'method': 'hello-world', 'tx_id': 1}, + wait_for_reply=False) + + self.listener._received.wait() + + self.assertIsNone(result) + self.assertEqual(True, self.listener._received.isSet()) + method = self.listener.message.message[u'method'] + self.assertEqual(u'hello-world', method) + + def test_send_receive_direct(self): + """Call() without topic.""" + + target = oslo_messaging.Target(server='127.0.0.1') + self.listener.listen(target) + message = {'method': 'hello-world', 'tx_id': 1} + context = {} + result = self.driver.send(target, context, message, + wait_for_reply=True) + self.assertTrue(result) + + def test_send_receive_notification(self): + """Notify() test""" + + target = oslo_messaging.Target(topic='t1', + server='notification@server') + self.listener.listen_notifications([(target, 'info')]) + + message = {'method': 'hello-world', 'tx_id': 1} + context = {} + target.topic = target.topic + '.info' + self.driver.send_notification(target, context, message, '3.0') + self.listener._received.wait(5) + self.assertTrue(self.listener._received.isSet()) + + +class TestPoller(test_utils.BaseTestCase): + + def setUp(self): + super(TestPoller, self).setUp() + self.poller = zmq_async.get_poller() + self.ctx = zmq.Context() + self.internal_ipc_dir = self.useFixture(fixtures.TempDir()).path + self.ADDR_REQ = "ipc://%s/request1" % self.internal_ipc_dir + + def test_poll_blocking(self): + + rep = self.ctx.socket(zmq.REP) + rep.bind(self.ADDR_REQ) + + reply_poller = zmq_async.get_reply_poller() + reply_poller.register(rep) + + def listener(): + incoming, socket = reply_poller.poll() + self.assertEqual(b'Hello', incoming[0]) + socket.send_string('Reply') + reply_poller.resume_polling(socket) + + executor = zmq_async.get_executor(listener) + executor.execute() + + req1 = self.ctx.socket(zmq.REQ) + req1.connect(self.ADDR_REQ) + + req2 = self.ctx.socket(zmq.REQ) + req2.connect(self.ADDR_REQ) + + req1.send_string('Hello') + req2.send_string('Hello') + + reply = req1.recv_string() + self.assertEqual('Reply', reply) + + reply = req2.recv_string() + self.assertEqual('Reply', reply) + + def test_poll_timeout(self): + rep = self.ctx.socket(zmq.REP) + rep.bind(self.ADDR_REQ) + + reply_poller = zmq_async.get_reply_poller() + reply_poller.register(rep) + + incoming, socket = reply_poller.poll(1) + self.assertIsNone(incoming) + self.assertIsNone(socket) diff --git a/oslo_messaging/tests/drivers/zmq/test_zmq_async.py b/oslo_messaging/tests/drivers/zmq/test_zmq_async.py new file mode 100644 index 000000000..28e091a0e --- /dev/null +++ b/oslo_messaging/tests/drivers/zmq/test_zmq_async.py @@ -0,0 +1,170 @@ +# 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._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 + + +class TestImportZmq(test_utils.BaseTestCase): + + def setUp(self): + super(TestImportZmq, self).setUp() + + def test_config_short_names_are_converted_to_correct_module_names(self): + mock_try_import = mock.Mock() + zmq_async.importutils.try_import = mock_try_import + + 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') + + 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') + + def test_when_no_args_then_default_zmq_module_is_loaded(self): + mock_try_import = mock.Mock() + zmq_async.importutils.try_import = mock_try_import + + 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') + + def test_invalid_config_value_raise_ValueError(self): + invalid_opt = 'x' + + errmsg = 'Invalid zmq_concurrency value: x' + with self.assertRaisesRegexp(ValueError, errmsg): + zmq_async.import_zmq(invalid_opt) + + +class TestGetPoller(test_utils.BaseTestCase): + + def setUp(self): + super(TestGetPoller, self).setUp() + + def test_when_no_arg_to_get_poller_then_return_default_poller(self): + zmq_async._is_eventlet_zmq_available = lambda: True + + actual = zmq_async.get_poller() + + self.assertTrue(isinstance(actual, green_poller.GreenPoller)) + + def test_when_native_poller_requested_then_return_ThreadingPoller(self): + actual = zmq_async.get_poller('native') + + self.assertTrue(isinstance(actual, threading_poller.ThreadingPoller)) + + def test_when_eventlet_is_unavailable_then_return_ThreadingPoller(self): + zmq_async._is_eventlet_zmq_available = lambda: False + + actual = zmq_async.get_poller('eventlet') + + self.assertTrue(isinstance(actual, threading_poller.ThreadingPoller)) + + def test_when_eventlet_is_available_then_return_GreenPoller(self): + zmq_async._is_eventlet_zmq_available = lambda: True + + actual = zmq_async.get_poller('eventlet') + + self.assertTrue(isinstance(actual, green_poller.GreenPoller)) + + def test_invalid_config_value_raise_ValueError(self): + invalid_opt = 'x' + + errmsg = 'Invalid zmq_concurrency value: x' + with self.assertRaisesRegexp(ValueError, errmsg): + zmq_async.get_poller(invalid_opt) + + +class TestGetReplyPoller(test_utils.BaseTestCase): + + def setUp(self): + super(TestGetReplyPoller, self).setUp() + + def test_default_reply_poller_is_HoldReplyPoller(self): + zmq_async._is_eventlet_zmq_available = lambda: True + + actual = zmq_async.get_reply_poller() + + self.assertTrue(isinstance(actual, green_poller.HoldReplyPoller)) + + def test_when_eventlet_is_available_then_return_HoldReplyPoller(self): + zmq_async._is_eventlet_zmq_available = lambda: True + + actual = zmq_async.get_reply_poller('eventlet') + + self.assertTrue(isinstance(actual, green_poller.HoldReplyPoller)) + + def test_when_eventlet_is_unavailable_then_return_ThreadingPoller(self): + zmq_async._is_eventlet_zmq_available = lambda: False + + actual = zmq_async.get_reply_poller('eventlet') + + self.assertTrue(isinstance(actual, threading_poller.ThreadingPoller)) + + def test_invalid_config_value_raise_ValueError(self): + invalid_opt = 'x' + + errmsg = 'Invalid zmq_concurrency value: x' + with self.assertRaisesRegexp(ValueError, errmsg): + zmq_async.get_reply_poller(invalid_opt) + + +class TestGetExecutor(test_utils.BaseTestCase): + + def setUp(self): + super(TestGetExecutor, self).setUp() + + def test_default_executor_is_GreenExecutor(self): + zmq_async._is_eventlet_zmq_available = lambda: True + + executor = zmq_async.get_executor('any method') + + self.assertTrue(isinstance(executor, green_poller.GreenExecutor)) + self.assertEqual('any method', executor._method) + + def test_when_eventlet_module_is_available_then_return_GreenExecutor(self): + zmq_async._is_eventlet_zmq_available = lambda: True + + executor = zmq_async.get_executor('any method', 'eventlet') + + self.assertTrue(isinstance(executor, green_poller.GreenExecutor)) + self.assertEqual('any method', executor._method) + + def test_when_eventlet_is_unavailable_then_return_ThreadingExecutor(self): + zmq_async._is_eventlet_zmq_available = lambda: False + + executor = zmq_async.get_executor('any method', 'eventlet') + + self.assertTrue(isinstance(executor, + threading_poller.ThreadingExecutor)) + self.assertEqual('any method', executor._method) + + def test_invalid_config_value_raise_ValueError(self): + invalid_opt = 'x' + + errmsg = 'Invalid zmq_concurrency value: x' + with self.assertRaisesRegexp(ValueError, errmsg): + zmq_async.get_executor('any method', invalid_opt) diff --git a/oslo_messaging/tests/functional/test_functional.py b/oslo_messaging/tests/functional/test_functional.py index c381e5bd9..4ff5d5867 100644 --- a/oslo_messaging/tests/functional/test_functional.py +++ b/oslo_messaging/tests/functional/test_functional.py @@ -101,6 +101,8 @@ class CallTestCase(utils.SkipIfNoTransportURL): self.assertEqual(0, s.endpoint.ival) def test_timeout(self): + if self.url.startswith("zmq"): + self.skipTest("Skip CallTestCase.test_timeout for ZMQ driver") transport = self.useFixture(utils.TransportFixture(self.url)) target = oslo_messaging.Target(topic="no_such_topic") c = utils.ClientStub(transport.transport, target, timeout=1) @@ -111,8 +113,7 @@ class CallTestCase(utils.SkipIfNoTransportURL): group = self.useFixture(utils.RpcServerGroupFixture(self.url)) client = group.client(1) client.add(increment=2) - f = lambda: client.subtract(increment=3) - self.assertThat(f, matchers.raises(ValueError)) + self.assertRaises(ValueError, client.subtract, increment=3) def test_timeout_with_concurrently_queues(self): transport = self.useFixture(utils.TransportFixture(self.url)) diff --git a/oslo_messaging/tests/functional/utils.py b/oslo_messaging/tests/functional/utils.py index ad9fd11a9..1de3d921e 100644 --- a/oslo_messaging/tests/functional/utils.py +++ b/oslo_messaging/tests/functional/utils.py @@ -125,7 +125,7 @@ class RpcServerGroupFixture(fixtures.Fixture): # NOTE(sileht): topic and servier_name must be uniq # to be able to run all tests in parallel self.topic = topic or str(uuid.uuid4()) - self.names = names or ["server_%i_%s" % (i, uuid.uuid4()) + self.names = names or ["server_%i_%s" % (i, str(uuid.uuid4())[:8]) for i in range(3)] self.exchange = exchange self.targets = [self._target(server=n) for n in self.names] diff --git a/oslo_messaging/tests/test_opts.py b/oslo_messaging/tests/test_opts.py index 5e4f241f9..e16145b94 100644 --- a/oslo_messaging/tests/test_opts.py +++ b/oslo_messaging/tests/test_opts.py @@ -32,11 +32,10 @@ class OptsTestCase(test_utils.BaseTestCase): super(OptsTestCase, self).setUp() def _test_list_opts(self, result): - self.assertEqual(6, len(result)) + self.assertEqual(5, len(result)) groups = [g for (g, l) in result] self.assertIn(None, groups) - self.assertIn('matchmaker_ring', groups) self.assertIn('matchmaker_redis', groups) self.assertIn('oslo_messaging_amqp', groups) self.assertIn('oslo_messaging_rabbit', groups) diff --git a/oslo_messaging/tests/test_utils.py b/oslo_messaging/tests/test_utils.py index 1dd9d4a8d..9c9e2f09a 100644 --- a/oslo_messaging/tests/test_utils.py +++ b/oslo_messaging/tests/test_utils.py @@ -97,3 +97,4 @@ class TimerTestCase(test_utils.BaseTestCase): remaining = t.check_return(callback, 1, a='b') self.assertEqual(0, remaining) callback.assert_called_once_with(1, a='b') + diff --git a/setup-test-env-zmq.sh b/setup-test-env-zmq.sh index c3c8e33c1..353c2602c 100755 --- a/setup-test-env-zmq.sh +++ b/setup-test-env-zmq.sh @@ -22,9 +22,4 @@ EOF redis-server --port $ZMQ_REDIS_PORT & -oslo-messaging-zmq-receiver --config-file ${DATADIR}/zmq.conf > ${DATADIR}/receiver.log 2>&1 & - -# FIXME(sileht): This does the same kind of setup that devstack does -# But this doesn't work yet, a zeromq maintener should take a look on that - $* diff --git a/setup.cfg b/setup.cfg index bb6e4ac46..f584e21ef 100644 --- a/setup.cfg +++ b/setup.cfg @@ -52,9 +52,8 @@ oslo.messaging.notify.drivers = oslo.messaging.zmq.matchmaker = # Matchmakers for ZeroMQ - redis = oslo_messaging._drivers.matchmaker_redis:MatchMakerRedis - ring = oslo_messaging._drivers.matchmaker_ring:MatchMakerRing - local = oslo_messaging._drivers.matchmaker:MatchMakerLocalhost + dummy = oslo_messaging._drivers.zmq_driver.matchmaker.base:DummyMatchMaker + redis = oslo_messaging._drivers.zmq_driver.matchmaker.matchmaker_redis:RedisMatchMaker oslo.config.opts = oslo.messaging = oslo_messaging.opts:list_opts diff --git a/tools/simulator.py b/tools/simulator.py index 843abc8ce..f3d72419f 100755 --- a/tools/simulator.py +++ b/tools/simulator.py @@ -25,7 +25,6 @@ import oslo_messaging as messaging from oslo_messaging import notify # noqa from oslo_messaging import rpc # noqa - LOG = logging.getLogger() USAGE = """ Usage: ./simulator.py [-h] [--url URL] [-d DEBUG]\