Re-factor the AMQP 1.0 addressing semantics
This patch introduces a new routable addressing format. This new address format not only works with traditional broker-based messaging backends but can be used to optimally route messages across a mesh messaging topology. This new addressing format may be enabled either via configuration or by message bus identification. This patch moves all of the AMQP 1.0 addressing logic into a new Addresser class. This class is used to map an oslo.messaging Target address and notification flag into a corresponding AMQP 1.0 message/subscription address based on the addressing mode. This hides the addressing details from the rest of the driver. For backward compatibility with previous releases the Addresser can be configured to automatically detect a non-routable messaging back end and fall back to using the legacy addressing scheme. The intention is to use routable addressing as the only form of addressing supported by the driver regardless of messaging back end. However before that can happen the legacy addressing must follow the standard deprecation process. Until then legacy addressing will remain available for backwards compatibility. Change-Id: I6b60b6944a4c44da9b2e41ab2a83ab59f15e396e
This commit is contained in:
parent
35ea442a0e
commit
4c0674d194
273
oslo_messaging/_drivers/amqp1_driver/addressing.py
Normal file
273
oslo_messaging/_drivers/amqp1_driver/addressing.py
Normal file
@ -0,0 +1,273 @@
|
||||
# Copyright 2016, Red Hat, Inc.
|
||||
#
|
||||
# Licensed under the Apache License, Version 2.0 (the "License"); you may
|
||||
# not use this file except in compliance with the License. You may obtain
|
||||
# a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
||||
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
||||
# License for the specific language governing permissions and limitations
|
||||
# under the License.
|
||||
|
||||
"""
|
||||
Utilities that map from a Target address to a proper AMQP 1.0 address.
|
||||
|
||||
This module defines a utility class that translates a high level oslo.messaging
|
||||
address (Target) into the message-level address used on the message bus. This
|
||||
translation may be statically configured or determined when the connection to
|
||||
the message bus is made.
|
||||
|
||||
The Target members that are used to generate the address are:
|
||||
|
||||
* exchange
|
||||
* topic
|
||||
* server flag
|
||||
* fanout flag
|
||||
|
||||
In addition a 'service' tag is associated with the address. This tag determines
|
||||
the service associated with an address (e.g. rpc or notification) so
|
||||
that traffic can be partitioned based on its use.
|
||||
"""
|
||||
|
||||
import abc
|
||||
|
||||
from oslo_messaging.target import Target
|
||||
|
||||
__all__ = [
|
||||
"keyify",
|
||||
"AddresserFactory",
|
||||
"SERVICE_RPC",
|
||||
"SERVICE_NOTIFY"
|
||||
]
|
||||
|
||||
SERVICE_RPC = 0
|
||||
SERVICE_NOTIFY = 1
|
||||
|
||||
|
||||
def keyify(address, service=SERVICE_RPC):
|
||||
"""Create a hashable key from a Target and service that will uniquely
|
||||
identify the generated address. This key is used to map the abstract
|
||||
oslo.messaging address to its corresponding AMQP link(s). This mapping may
|
||||
be done before the connection is established.
|
||||
"""
|
||||
if isinstance(address, Target):
|
||||
# service is important because the resolved address may be
|
||||
# different based on whether or not this Target is used for
|
||||
# notifications or RPC
|
||||
return ("Target:{t={%s} e={%s} s={%s} f={%s} service={%s}}" %
|
||||
(address.topic, address.exchange, address.server,
|
||||
address.fanout, service))
|
||||
else:
|
||||
# absolute address can be used without modification
|
||||
return "String:{%s}" % address
|
||||
|
||||
|
||||
class Addresser(object):
|
||||
"""Base class message bus address generator. Used to convert an
|
||||
oslo.messaging address into an AMQP 1.0 address string used over the
|
||||
connection to the message bus.
|
||||
"""
|
||||
def __init__(self, default_exchange):
|
||||
self._default_exchange = default_exchange
|
||||
|
||||
def resolve(self, target, service):
|
||||
if not isinstance(target, Target):
|
||||
# an already resolved address
|
||||
return target
|
||||
# Return a link address for a given target
|
||||
if target.fanout:
|
||||
return self.multicast_address(target, service)
|
||||
elif target.server:
|
||||
return self.unicast_address(target, service)
|
||||
else:
|
||||
return self.anycast_address(target, service)
|
||||
|
||||
@abc.abstractmethod
|
||||
def multicast_address(self, target, service):
|
||||
"""Address used to broadcast to all subscribers
|
||||
"""
|
||||
|
||||
@abc.abstractmethod
|
||||
def unicast_address(self, target, service):
|
||||
"""Address used to target a specific subscriber (direct)
|
||||
"""
|
||||
|
||||
@abc.abstractmethod
|
||||
def anycast_address(self, target, service):
|
||||
"""Address used for shared subscribers (competing consumers)
|
||||
"""
|
||||
|
||||
|
||||
class LegacyAddresser(Addresser):
|
||||
"""Legacy addresses are in the following format:
|
||||
|
||||
multicast: '$broadcast_prefix.$exchange.$topic.all'
|
||||
unicast: '$server_prefix.$exchange.$topic.$server'
|
||||
anycast: '$group_prefix.$exchange.$topic'
|
||||
|
||||
Legacy addresses do not distinguish RPC traffic from Notification traffic
|
||||
"""
|
||||
def __init__(self, default_exchange, server_prefix, broadcast_prefix,
|
||||
group_prefix):
|
||||
super(LegacyAddresser, self).__init__(default_exchange)
|
||||
self._server_prefix = server_prefix
|
||||
self._broadcast_prefix = broadcast_prefix
|
||||
self._group_prefix = group_prefix
|
||||
|
||||
def multicast_address(self, target, service):
|
||||
return self._concatenate([self._broadcast_prefix,
|
||||
target.exchange or self._default_exchange,
|
||||
target.topic, "all"])
|
||||
|
||||
def unicast_address(self, target, service=SERVICE_RPC):
|
||||
return self._concatenate([self._server_prefix,
|
||||
target.exchange or self._default_exchange,
|
||||
target.topic, target.server])
|
||||
|
||||
def anycast_address(self, target, service=SERVICE_RPC):
|
||||
return self._concatenate([self._group_prefix,
|
||||
target.exchange or self._default_exchange,
|
||||
target.topic])
|
||||
|
||||
def _concatenate(self, items):
|
||||
return ".".join(filter(bool, items))
|
||||
|
||||
# for debug:
|
||||
def _is_multicast(self, address):
|
||||
return address.startswith(self._broadcast_prefix)
|
||||
|
||||
def _is_unicast(self, address):
|
||||
return address.startswith(self._server_prefix)
|
||||
|
||||
def _is_anycast(self, address):
|
||||
return address.startswith(self._group_prefix)
|
||||
|
||||
def _is_service(self, address, service):
|
||||
# legacy addresses are the same for RPC or Notifications
|
||||
return True
|
||||
|
||||
|
||||
class RoutableAddresser(Addresser):
|
||||
"""Routable addresses have different formats based their use. It starts
|
||||
with a prefix that is determined by the type of traffic (RPC or
|
||||
Notifications). The prefix is followed by a description of messaging
|
||||
delivery semantics. The delivery may be one of: 'multicast', 'unicast', or
|
||||
'anycast'. The delivery semantics are followed by information pulled from
|
||||
the Target. The template is:
|
||||
|
||||
$prefix/$semantics/$exchange/$topic[/$server]
|
||||
|
||||
Examples based on the default prefix and semantic values:
|
||||
|
||||
rpc-unicast: "openstack.org/om/rpc/unicast/$exchange/$topic/$server"
|
||||
notify-anycast: "openstack.org/om/notify/anycast/$exchange/$topic"
|
||||
"""
|
||||
|
||||
def __init__(self, default_exchange, rpc_exchange, rpc_prefix,
|
||||
notify_exchange, notify_prefix, unicast_tag, multicast_tag,
|
||||
anycast_tag):
|
||||
super(RoutableAddresser, self).__init__(default_exchange)
|
||||
if not self._default_exchange:
|
||||
self._default_exchange = "openstack"
|
||||
# templates for address generation:
|
||||
_rpc = rpc_prefix + "/"
|
||||
self._rpc_prefix = _rpc
|
||||
self._rpc_unicast = _rpc + unicast_tag
|
||||
self._rpc_multicast = _rpc + multicast_tag
|
||||
self._rpc_anycast = _rpc + anycast_tag
|
||||
|
||||
_notify = notify_prefix + "/"
|
||||
self._notify_prefix = _notify
|
||||
self._notify_unicast = _notify + unicast_tag
|
||||
self._notify_multicast = _notify + multicast_tag
|
||||
self._notify_anycast = _notify + anycast_tag
|
||||
|
||||
self._exchange = [
|
||||
# SERVICE_RPC:
|
||||
rpc_exchange or self._default_exchange or 'rpc',
|
||||
# SERVICE_NOTIFY:
|
||||
notify_exchange or self._default_exchange or 'notify'
|
||||
]
|
||||
|
||||
def multicast_address(self, target, service=SERVICE_RPC):
|
||||
if service == SERVICE_RPC:
|
||||
prefix = self._rpc_multicast
|
||||
else:
|
||||
prefix = self._notify_multicast
|
||||
return "%s/%s/%s" % (prefix,
|
||||
target.exchange or self._exchange[service],
|
||||
target.topic)
|
||||
|
||||
def unicast_address(self, target, service=SERVICE_RPC):
|
||||
if service == SERVICE_RPC:
|
||||
prefix = self._rpc_unicast
|
||||
else:
|
||||
prefix = self._notify_unicast
|
||||
if target.server:
|
||||
return "%s/%s/%s/%s" % (prefix,
|
||||
target.exchange or self._exchange[service],
|
||||
target.topic,
|
||||
target.server)
|
||||
return "%s/%s/%s" % (prefix,
|
||||
target.exchange or self._exchange[service],
|
||||
target.topic)
|
||||
|
||||
def anycast_address(self, target, service=SERVICE_RPC):
|
||||
if service == SERVICE_RPC:
|
||||
prefix = self._rpc_anycast
|
||||
else:
|
||||
prefix = self._notify_anycast
|
||||
return "%s/%s/%s" % (prefix,
|
||||
target.exchange or self._exchange[service],
|
||||
target.topic)
|
||||
|
||||
# for debug:
|
||||
def _is_multicast(self, address):
|
||||
return (address.startswith(self._rpc_multicast) or
|
||||
address.startswith(self._notify_multicast))
|
||||
|
||||
def _is_unicast(self, address):
|
||||
return (address.startswith(self._rpc_unicast) or
|
||||
address.startswith(self._notify_unicast))
|
||||
|
||||
def _is_anycast(self, address):
|
||||
return (address.startswith(self._rpc_anycast) or
|
||||
address.startswith(self._notify_anycast))
|
||||
|
||||
def _is_service(self, address, service):
|
||||
return address.startswith(self._rpc_prefix if service == SERVICE_RPC
|
||||
else self._notify_prefix)
|
||||
|
||||
|
||||
class AddresserFactory(object):
|
||||
"""Generates the proper Addresser based on configuration and the type of
|
||||
message bus the driver is connected to.
|
||||
"""
|
||||
def __init__(self, default_exchange, mode, **kwargs):
|
||||
self._default_exchange = default_exchange
|
||||
self._mode = mode
|
||||
self._kwargs = kwargs
|
||||
|
||||
def __call__(self, remote_properties):
|
||||
# for backwards compatiblity use legacy if dynamic and we're connected
|
||||
# to qpidd or we cannot identify the message bus. This can be
|
||||
# overridden via the configuration.
|
||||
product = remote_properties.get('product', 'qpid-cpp')
|
||||
if self._mode == 'legacy' or (self._mode == 'dynamic'
|
||||
and product == 'qpid-cpp'):
|
||||
return LegacyAddresser(self._default_exchange,
|
||||
self._kwargs['legacy_server_prefix'],
|
||||
self._kwargs['legacy_broadcast_prefix'],
|
||||
self._kwargs['legacy_group_prefix'])
|
||||
else:
|
||||
return RoutableAddresser(self._default_exchange,
|
||||
self._kwargs.get("rpc_exchange"),
|
||||
self._kwargs["rpc_prefix"],
|
||||
self._kwargs.get("notify_exchange"),
|
||||
self._kwargs["notify_prefix"],
|
||||
self._kwargs["unicast"],
|
||||
self._kwargs["multicast"],
|
||||
self._kwargs["anycast"])
|
@ -27,7 +27,10 @@ functions scheduled by the Controller.
|
||||
import abc
|
||||
import collections
|
||||
import logging
|
||||
import os
|
||||
import platform
|
||||
import random
|
||||
import sys
|
||||
import threading
|
||||
import time
|
||||
import uuid
|
||||
@ -38,10 +41,14 @@ from six import iteritems
|
||||
from six import itervalues
|
||||
from six import moves
|
||||
|
||||
from oslo_messaging._drivers.amqp1_driver.addressing import AddresserFactory
|
||||
from oslo_messaging._drivers.amqp1_driver.addressing import keyify
|
||||
from oslo_messaging._drivers.amqp1_driver.addressing import SERVICE_NOTIFY
|
||||
from oslo_messaging._drivers.amqp1_driver.addressing import SERVICE_RPC
|
||||
from oslo_messaging._drivers.amqp1_driver import eventloop
|
||||
from oslo_messaging._i18n import _LE, _LI, _LW
|
||||
from oslo_messaging import exceptions
|
||||
from oslo_messaging import target
|
||||
from oslo_messaging.target import Target
|
||||
from oslo_messaging import transport
|
||||
|
||||
LOG = logging.getLogger(__name__)
|
||||
@ -70,23 +77,17 @@ class SubscribeTask(Task):
|
||||
"""
|
||||
def __init__(self, target, listener, notifications=False):
|
||||
super(SubscribeTask, self).__init__()
|
||||
self._target = target
|
||||
self._listener = listener
|
||||
self._notifications = notifications
|
||||
self._target = target() # mutable - need a copy
|
||||
self._subscriber_id = listener.id
|
||||
self._in_queue = listener.incoming
|
||||
self._service = SERVICE_NOTIFY if notifications else SERVICE_RPC
|
||||
self._wakeup = threading.Event()
|
||||
|
||||
def wait(self):
|
||||
self._wakeup.wait()
|
||||
|
||||
def _execute(self, controller):
|
||||
if self._notifications:
|
||||
controller.subscribe_notifications(self._target,
|
||||
self._listener.incoming,
|
||||
self._listener.id)
|
||||
else:
|
||||
controller.subscribe(self._target,
|
||||
self._listener.incoming,
|
||||
self._listener.id)
|
||||
controller.subscribe(self)
|
||||
self._wakeup.set()
|
||||
|
||||
|
||||
@ -95,15 +96,17 @@ class SendTask(Task):
|
||||
destination.
|
||||
"""
|
||||
def __init__(self, name, message, target, deadline, retry,
|
||||
wait_for_ack):
|
||||
wait_for_ack, notification=False):
|
||||
super(SendTask, self).__init__()
|
||||
self.name = name
|
||||
# note: target can be either a Target class or a string
|
||||
self.target = target
|
||||
# target is mutable - make copy
|
||||
self.target = target() if isinstance(target, Target) else target
|
||||
self.message = message
|
||||
self.deadline = deadline
|
||||
self.retry = retry
|
||||
self.wait_for_ack = wait_for_ack
|
||||
self.service = SERVICE_NOTIFY if notification else SERVICE_RPC
|
||||
self.timer = None
|
||||
self._wakeup = threading.Event()
|
||||
self._controller = None
|
||||
@ -255,13 +258,14 @@ class MessageDispositionTask(Task):
|
||||
|
||||
|
||||
class Sender(pyngus.SenderEventHandler):
|
||||
"""A link for sending to a particular address on the message bus.
|
||||
"""A link for sending to a particular destination on the message bus.
|
||||
"""
|
||||
def __init__(self, address, scheduler, delay):
|
||||
def __init__(self, destination, scheduler, delay, service):
|
||||
super(Sender, self).__init__()
|
||||
self._address = address
|
||||
self._destination = destination
|
||||
self._service = service
|
||||
self._address = None
|
||||
self._link = None
|
||||
self._name = "Producer [target=%s:id=%s]" % (address, uuid.uuid4().hex)
|
||||
self._scheduler = scheduler
|
||||
self._delay = delay # for re-connecting
|
||||
# holds all pending SendTasks
|
||||
@ -275,13 +279,14 @@ class Sender(pyngus.SenderEventHandler):
|
||||
def pending_messages(self):
|
||||
return len(self._pending_sends)
|
||||
|
||||
def attach(self, connection, reply_link):
|
||||
def attach(self, connection, reply_link, addresser):
|
||||
"""Open the link. Called by the Controller when the AMQP connection
|
||||
becomes active.
|
||||
"""
|
||||
LOG.debug("Sender %s attached", self._address)
|
||||
self._connection = connection
|
||||
self._reply_link = reply_link
|
||||
self._address = addresser.resolve(self._destination, self._service)
|
||||
LOG.debug("Sender %s attached", self._address)
|
||||
self._link = self._open_link()
|
||||
|
||||
def detach(self):
|
||||
@ -289,6 +294,7 @@ class Sender(pyngus.SenderEventHandler):
|
||||
response to a close requested by the remote. May be re-attached later
|
||||
(after a reset is done)
|
||||
"""
|
||||
self._address = None
|
||||
self._connection = None
|
||||
self._reply_link = None
|
||||
if self._link:
|
||||
@ -299,6 +305,7 @@ class Sender(pyngus.SenderEventHandler):
|
||||
resources, abort any in-flight messages, and check the retry limit on
|
||||
all pending send requests.
|
||||
"""
|
||||
self._address = None
|
||||
self._connection = None
|
||||
self._reply_link = None
|
||||
if self._link:
|
||||
@ -326,10 +333,9 @@ class Sender(pyngus.SenderEventHandler):
|
||||
except ValueError:
|
||||
pass
|
||||
send_task._on_timeout()
|
||||
send_task.timer = self._scheduler.alarm(timer_callback,
|
||||
send_task.deadline)
|
||||
send_task._timer = self._scheduler.alarm(timer_callback,
|
||||
send_task.deadline)
|
||||
|
||||
send_task.message.address = self._address
|
||||
if not self._can_send:
|
||||
self._pending_sends.append(send_task)
|
||||
elif self._pending_sends:
|
||||
@ -405,6 +411,7 @@ class Sender(pyngus.SenderEventHandler):
|
||||
|
||||
def _send(self, send_task):
|
||||
send_task._prepare(self)
|
||||
send_task.message.address = self._address
|
||||
if send_task.wait_for_ack:
|
||||
def pyngus_callback(link, handle, state, info):
|
||||
# invoked when the message bus (n)acks this message
|
||||
@ -433,7 +440,9 @@ class Sender(pyngus.SenderEventHandler):
|
||||
self._send(self._pending_sends.popleft())
|
||||
|
||||
def _open_link(self):
|
||||
link = self._connection.create_sender(name=self._name,
|
||||
name = "openstack.org/om/sender/[%s]/%s" % (self._address,
|
||||
uuid.uuid4().hex)
|
||||
link = self._connection.create_sender(name=name,
|
||||
source_address=self._address,
|
||||
target_address=self._address,
|
||||
event_handler=self)
|
||||
@ -456,8 +465,9 @@ class Replies(pyngus.ReceiverEventHandler):
|
||||
self._correlation = {} # map of correlation-id to response queue
|
||||
self._on_ready = on_ready
|
||||
self._on_down = on_down
|
||||
rname = "RPC_Response-%s:src=[dynamic]:tgt=replies" % uuid.uuid4().hex
|
||||
self._receiver = connection.create_receiver("replies",
|
||||
rname = ("openstack.org/om/receiver/[rpc-response]/%s"
|
||||
% uuid.uuid4().hex)
|
||||
self._receiver = connection.create_receiver("rpc-response",
|
||||
event_handler=self,
|
||||
name=rname)
|
||||
|
||||
@ -567,12 +577,12 @@ class Server(pyngus.ReceiverEventHandler):
|
||||
from a given target. Messages arriving on the links are placed on the
|
||||
'incoming' queue.
|
||||
"""
|
||||
def __init__(self, addresses, incoming, subscription_id, scheduler, delay):
|
||||
def __init__(self, target, incoming, scheduler, delay):
|
||||
self._target = target
|
||||
self._incoming = incoming
|
||||
self._addresses = addresses
|
||||
self._addresses = []
|
||||
self._capacity = 500 # credit per link
|
||||
self._receivers = []
|
||||
self._id = subscription_id
|
||||
self._scheduler = scheduler
|
||||
self._delay = delay # for link re-attach
|
||||
self._connection = None
|
||||
@ -584,13 +594,14 @@ class Server(pyngus.ReceiverEventHandler):
|
||||
"""
|
||||
self._connection = connection
|
||||
for a in self._addresses:
|
||||
name = "Listener-%s:src=%s:tgt=%s" % (uuid.uuid4().hex, a, a)
|
||||
name = "openstack.org/om/receiver/[%s]/%s" % (a, uuid.uuid4().hex)
|
||||
r = self._open_link(a, name)
|
||||
self._receivers.append(r)
|
||||
|
||||
def detach(self):
|
||||
"""Attempt a clean shutdown of the links"""
|
||||
self._connection = None
|
||||
self._addresses = []
|
||||
for receiver in self._receivers:
|
||||
receiver.close()
|
||||
|
||||
@ -599,6 +610,7 @@ class Server(pyngus.ReceiverEventHandler):
|
||||
# failing over. Since links are destroyed, this cannot be called from
|
||||
# any of the following ReceiverLink callbacks.
|
||||
self._connection = None
|
||||
self._addresses = []
|
||||
self._reopen_scheduled = False
|
||||
for r in self._receivers:
|
||||
r.destroy()
|
||||
@ -686,6 +698,37 @@ class Server(pyngus.ReceiverEventHandler):
|
||||
self._receivers[i] = self._open_link(addr, name)
|
||||
|
||||
|
||||
class RPCServer(Server):
|
||||
"""Subscribes to RPC addresses"""
|
||||
def __init__(self, target, incoming, scheduler, delay):
|
||||
super(RPCServer, self).__init__(target, incoming, scheduler, delay)
|
||||
|
||||
def attach(self, connection, addresser):
|
||||
# Generate the AMQP 1.0 addresses for the base class
|
||||
self._addresses = [
|
||||
addresser.unicast_address(self._target, SERVICE_RPC),
|
||||
addresser.multicast_address(self._target, SERVICE_RPC),
|
||||
addresser.anycast_address(self._target, SERVICE_RPC)
|
||||
]
|
||||
# now invoke the base class with the generated addresses
|
||||
super(RPCServer, self).attach(connection)
|
||||
|
||||
|
||||
class NotificationServer(Server):
|
||||
"""Subscribes to Notification addresses"""
|
||||
def __init__(self, target, incoming, scheduler, delay):
|
||||
super(NotificationServer, self).__init__(target, incoming, scheduler,
|
||||
delay)
|
||||
|
||||
def attach(self, connection, addresser):
|
||||
# Generate the AMQP 1.0 addresses for the base class
|
||||
self._addresses = [
|
||||
addresser.anycast_address(self._target, SERVICE_NOTIFY)
|
||||
]
|
||||
# now invoke the base class with the generated addresses
|
||||
super(NotificationServer, self).attach(connection)
|
||||
|
||||
|
||||
class Hosts(object):
|
||||
"""An order list of TransportHost addresses. Connection failover
|
||||
progresses from one host to the next. username and password come from the
|
||||
@ -731,6 +774,9 @@ class Controller(pyngus.ConnectionEventHandler):
|
||||
def __init__(self, hosts, default_exchange, config):
|
||||
self.processor = None
|
||||
self._socket_connection = None
|
||||
self._node = platform.node() or "<UNKNOWN>"
|
||||
self._command = os.path.basename(sys.argv[0])
|
||||
self._pid = os.getpid()
|
||||
# queue of drivertask objects to execute on the eventloop thread
|
||||
self._tasks = moves.queue.Queue(maxsize=500)
|
||||
# limit the number of Task()'s to execute per call to _process_tasks().
|
||||
@ -743,11 +789,6 @@ class Controller(pyngus.ConnectionEventHandler):
|
||||
# specific ProtonListener's identifier:
|
||||
self._servers = {}
|
||||
|
||||
self.server_request_prefix = \
|
||||
config.oslo_messaging_amqp.server_request_prefix
|
||||
self.broadcast_prefix = config.oslo_messaging_amqp.broadcast_prefix
|
||||
self.group_request_prefix = \
|
||||
config.oslo_messaging_amqp.group_request_prefix
|
||||
self._container_name = config.oslo_messaging_amqp.container_name
|
||||
self.idle_timeout = config.oslo_messaging_amqp.idle_timeout
|
||||
self.trace_protocol = config.oslo_messaging_amqp.trace
|
||||
@ -773,9 +814,22 @@ class Controller(pyngus.ConnectionEventHandler):
|
||||
if self.max_send_retries <= 0:
|
||||
self.max_send_retries = None # None or 0 == forever, like rabbit
|
||||
|
||||
self.separator = "."
|
||||
self.fanout_qualifier = "all"
|
||||
self.default_exchange = default_exchange
|
||||
_opts = config.oslo_messaging_amqp
|
||||
factory_args = {"legacy_server_prefix": _opts.server_request_prefix,
|
||||
"legacy_broadcast_prefix": _opts.broadcast_prefix,
|
||||
"legacy_group_prefix": _opts.group_request_prefix,
|
||||
"rpc_prefix": _opts.rpc_address_prefix,
|
||||
"notify_prefix": _opts.notify_address_prefix,
|
||||
"multicast": _opts.multicast_address,
|
||||
"unicast": _opts.unicast_address,
|
||||
"anycast": _opts.anycast_address,
|
||||
"notify_exchange": _opts.default_notification_exchange,
|
||||
"rpc_exchange": _opts.default_rpc_exchange}
|
||||
|
||||
self.addresser_factory = AddresserFactory(default_exchange,
|
||||
_opts.addressing_mode,
|
||||
**factory_args)
|
||||
self.addresser = None
|
||||
|
||||
# cannot send an RPC request until the replies link is active, as we
|
||||
# need the peer assigned address, so need to delay sending any RPC
|
||||
@ -792,7 +846,8 @@ class Controller(pyngus.ConnectionEventHandler):
|
||||
|
||||
def connect(self):
|
||||
"""Connect to the messaging service."""
|
||||
self.processor = eventloop.Thread(self._container_name)
|
||||
self.processor = eventloop.Thread(self._container_name, self._node,
|
||||
self._command, self._pid)
|
||||
self.processor.wakeup(lambda: self._do_connect())
|
||||
|
||||
def add_task(self, task):
|
||||
@ -825,87 +880,56 @@ class Controller(pyngus.ConnectionEventHandler):
|
||||
if send_task.deadline and send_task.deadline <= time.time():
|
||||
send_task._on_timeout()
|
||||
return
|
||||
if isinstance(send_task.target, target.Target):
|
||||
address = self._resolve(send_task.target)
|
||||
else:
|
||||
address = send_task.target
|
||||
LOG.debug("Sending message to %s", address)
|
||||
LOG.debug("Sending message to %s", send_task.target)
|
||||
if send_task.retry is None or send_task.retry < 0:
|
||||
send_task.retry = self.max_send_retries
|
||||
sender = self._senders.get(address)
|
||||
key = keyify(send_task.target, send_task.service)
|
||||
sender = self._senders.get(key)
|
||||
if not sender:
|
||||
sender = Sender(address, self.processor, self.link_retry_delay)
|
||||
self._senders[address] = sender
|
||||
sender = Sender(send_task.target, self.processor,
|
||||
self.link_retry_delay, send_task.service)
|
||||
self._senders[key] = sender
|
||||
if self.reply_link and self.reply_link.active:
|
||||
sender.attach(self._socket_connection.connection,
|
||||
self.reply_link)
|
||||
self.reply_link, self.addresser)
|
||||
sender.send_message(send_task)
|
||||
|
||||
def subscribe(self, target, in_queue, subscription_id):
|
||||
"""Subscribe to messages sent to 'target', place received messages on
|
||||
'in_queue'.
|
||||
"""
|
||||
addresses = [
|
||||
self._server_address(target),
|
||||
self._broadcast_address(target),
|
||||
self._group_request_address(target)
|
||||
]
|
||||
self._subscribe(target, addresses, in_queue, subscription_id)
|
||||
def subscribe(self, subscribe_task):
|
||||
"""Subscribe to a given target"""
|
||||
if subscribe_task._service == SERVICE_NOTIFY:
|
||||
t = "notification"
|
||||
server = NotificationServer(subscribe_task._target,
|
||||
subscribe_task._in_queue,
|
||||
self.processor,
|
||||
self.link_retry_delay)
|
||||
else:
|
||||
t = "RPC"
|
||||
server = RPCServer(subscribe_task._target,
|
||||
subscribe_task._in_queue,
|
||||
self.processor,
|
||||
self.link_retry_delay)
|
||||
|
||||
def subscribe_notifications(self, target, in_queue, subscription_id):
|
||||
"""Subscribe for notifications on 'target', place received messages on
|
||||
'in_queue'.
|
||||
"""
|
||||
addresses = [self._group_request_address(target)]
|
||||
self._subscribe(target, addresses, in_queue, subscription_id)
|
||||
|
||||
def _subscribe(self, target, addresses, in_queue, subscription_id):
|
||||
LOG.debug("Subscribing to %(target)s (%(addresses)s)",
|
||||
{'target': target, 'addresses': addresses})
|
||||
server = Server(addresses, in_queue, subscription_id,
|
||||
self.processor,
|
||||
self.link_retry_delay)
|
||||
servers = self._servers.get(target)
|
||||
LOG.debug("Subscribing to %(type)s target %(target)s",
|
||||
{'type': t, 'target': subscribe_task._target})
|
||||
key = keyify(subscribe_task._target, subscribe_task._service)
|
||||
servers = self._servers.get(key)
|
||||
if servers is None:
|
||||
servers = {}
|
||||
self._servers[target] = servers
|
||||
servers[subscription_id] = server
|
||||
self._servers[key] = servers
|
||||
servers[subscribe_task._subscriber_id] = server
|
||||
if self._active:
|
||||
server.attach(self._socket_connection.connection)
|
||||
|
||||
def _resolve(self, target):
|
||||
"""Return a link address for a given target."""
|
||||
if target.fanout:
|
||||
return self._broadcast_address(target)
|
||||
elif target.server:
|
||||
return self._server_address(target)
|
||||
else:
|
||||
return self._group_request_address(target)
|
||||
|
||||
def _server_address(self, target):
|
||||
return self._concatenate([self.server_request_prefix,
|
||||
target.exchange or self.default_exchange,
|
||||
target.topic, target.server])
|
||||
|
||||
def _broadcast_address(self, target):
|
||||
return self._concatenate([self.broadcast_prefix,
|
||||
target.exchange or self.default_exchange,
|
||||
target.topic, self.fanout_qualifier])
|
||||
|
||||
def _group_request_address(self, target):
|
||||
return self._concatenate([self.group_request_prefix,
|
||||
target.exchange or self.default_exchange,
|
||||
target.topic])
|
||||
|
||||
def _concatenate(self, items):
|
||||
return self.separator.join(filter(bool, items))
|
||||
server.attach(self._socket_connection.connection,
|
||||
self.addresser)
|
||||
|
||||
# commands executed on the processor (eventloop) via 'wakeup()':
|
||||
|
||||
def _do_connect(self):
|
||||
"""Establish connection and reply subscription on processor thread."""
|
||||
host = self.hosts.current
|
||||
conn_props = {'hostname': host.hostname}
|
||||
conn_props = {'properties': {u'process': self._command,
|
||||
u'pid': self._pid,
|
||||
u'node': self._node},
|
||||
'hostname': host.hostname}
|
||||
if self.idle_timeout:
|
||||
conn_props["idle-time-out"] = float(self.idle_timeout)
|
||||
if self.trace_protocol:
|
||||
@ -989,7 +1013,8 @@ class Controller(pyngus.ConnectionEventHandler):
|
||||
{'hostname': self.hosts.current.hostname,
|
||||
'port': self.hosts.current.port})
|
||||
for sender in itervalues(self._senders):
|
||||
sender.attach(self._socket_connection.connection, self.reply_link)
|
||||
sender.attach(self._socket_connection.connection,
|
||||
self.reply_link, self.addresser)
|
||||
|
||||
def _reply_link_down(self):
|
||||
# Treat it as a recoverable failure because the RPC reply address is
|
||||
@ -1028,9 +1053,14 @@ class Controller(pyngus.ConnectionEventHandler):
|
||||
LOG.debug("Connection active (%(hostname)s:%(port)s), subscribing...",
|
||||
{'hostname': self.hosts.current.hostname,
|
||||
'port': self.hosts.current.port})
|
||||
# allocate an addresser based on the advertised properties of the
|
||||
# message bus
|
||||
props = connection.remote_properties or {}
|
||||
self.addresser = self.addresser_factory(props)
|
||||
for servers in itervalues(self._servers):
|
||||
for server in itervalues(servers):
|
||||
server.attach(self._socket_connection.connection)
|
||||
server.attach(self._socket_connection.connection,
|
||||
self.addresser)
|
||||
self.reply_link = Replies(self._socket_connection.connection,
|
||||
self._reply_link_ready,
|
||||
self._reply_link_down)
|
||||
@ -1078,6 +1108,7 @@ class Controller(pyngus.ConnectionEventHandler):
|
||||
"""The connection to the messaging service has been lost. Try to
|
||||
reestablish the connection/failover if not shutting down the driver.
|
||||
"""
|
||||
self.addresser = None
|
||||
if self._closing:
|
||||
# we're in the middle of shutting down the driver anyways,
|
||||
# just consider it done:
|
||||
|
@ -28,7 +28,6 @@ import logging
|
||||
import os
|
||||
import select
|
||||
import socket
|
||||
import sys
|
||||
import threading
|
||||
import time
|
||||
import uuid
|
||||
@ -48,8 +47,7 @@ class _SocketConnection(object):
|
||||
def __init__(self, name, container, properties, handler):
|
||||
self.name = name
|
||||
self.socket = None
|
||||
self._properties = properties or {}
|
||||
self._properties["properties"] = self._get_name_and_pid()
|
||||
self._properties = properties
|
||||
# The handler is a pyngus ConnectionEventHandler, which is invoked by
|
||||
# pyngus on connection-related events (active, closed, error, etc).
|
||||
# Currently it is the Controller object.
|
||||
@ -57,10 +55,6 @@ class _SocketConnection(object):
|
||||
self._container = container
|
||||
self.connection = None
|
||||
|
||||
def _get_name_and_pid(self):
|
||||
# helps identify the process that is using the connection
|
||||
return {u'process': os.path.basename(sys.argv[0]), u'pid': os.getpid()}
|
||||
|
||||
def fileno(self):
|
||||
"""Allows use of a _SocketConnection in a select() call.
|
||||
"""
|
||||
@ -252,7 +246,7 @@ class Thread(threading.Thread):
|
||||
"""Manages socket I/O and executes callables queued up by external
|
||||
threads.
|
||||
"""
|
||||
def __init__(self, container_name=None):
|
||||
def __init__(self, container_name, node, command, pid):
|
||||
super(Thread, self).__init__()
|
||||
|
||||
# callables from other threads:
|
||||
@ -262,7 +256,8 @@ class Thread(threading.Thread):
|
||||
|
||||
# Configure a container
|
||||
if container_name is None:
|
||||
container_name = "Container-" + uuid.uuid4().hex
|
||||
container_name = ("openstack.org/om/container/%s/%s/%s/%s" %
|
||||
(node, command, pid, uuid.uuid4().hex))
|
||||
self._container = pyngus.Container(container_name)
|
||||
|
||||
self.name = "Thread for Proton container: %s" % self._container.name
|
||||
@ -300,9 +295,9 @@ class Thread(threading.Thread):
|
||||
"""Invoke request at a particular time"""
|
||||
return self._scheduler.alarm(request, deadline)
|
||||
|
||||
def connect(self, host, handler, properties=None, name=None):
|
||||
def connect(self, host, handler, properties):
|
||||
"""Get a _SocketConnection to a peer represented by url."""
|
||||
key = name or "%s:%i" % (host.hostname, host.port)
|
||||
key = "openstack.org/om/connection/%s:%s/" % (host.hostname, host.port)
|
||||
# return pre-existing
|
||||
conn = self._container.get_connection(key)
|
||||
if conn:
|
||||
|
@ -16,25 +16,11 @@ from oslo_config import cfg
|
||||
|
||||
|
||||
amqp1_opts = [
|
||||
cfg.StrOpt('server_request_prefix',
|
||||
default='exclusive',
|
||||
deprecated_group='amqp1',
|
||||
help="address prefix used when sending to a specific server"),
|
||||
|
||||
cfg.StrOpt('broadcast_prefix',
|
||||
default='broadcast',
|
||||
deprecated_group='amqp1',
|
||||
help="address prefix used when broadcasting to all servers"),
|
||||
|
||||
cfg.StrOpt('group_request_prefix',
|
||||
default='unicast',
|
||||
deprecated_group='amqp1',
|
||||
help="address prefix when sending to any server in group"),
|
||||
|
||||
cfg.StrOpt('container_name',
|
||||
default=None,
|
||||
deprecated_group='amqp1',
|
||||
help='Name for the AMQP container'),
|
||||
help='Name for the AMQP container. must be globally unique.'
|
||||
' Defaults to a generated UUID'),
|
||||
|
||||
cfg.IntOpt('idle_timeout',
|
||||
default=0, # disabled
|
||||
@ -130,5 +116,86 @@ amqp1_opts = [
|
||||
default=10,
|
||||
min=1,
|
||||
help='Time to pause between re-connecting an AMQP 1.0 link that'
|
||||
' failed due to a recoverable error.')
|
||||
' failed due to a recoverable error.'),
|
||||
|
||||
# Addressing:
|
||||
|
||||
cfg.StrOpt('addressing_mode',
|
||||
default='dynamic',
|
||||
help="Indicates the addressing mode used by the driver.\n"
|
||||
"Permitted values:\n"
|
||||
"'legacy' - use legacy non-routable addressing\n"
|
||||
"'routable' - use routable addresses\n"
|
||||
"'dynamic' - use legacy addresses if the message bus does not"
|
||||
" support routing otherwise use routable addressing"),
|
||||
|
||||
# Legacy addressing customization:
|
||||
|
||||
cfg.StrOpt('server_request_prefix',
|
||||
default='exclusive',
|
||||
deprecated_group='amqp1',
|
||||
help="address prefix used when sending to a specific server"),
|
||||
|
||||
cfg.StrOpt('broadcast_prefix',
|
||||
default='broadcast',
|
||||
deprecated_group='amqp1',
|
||||
help="address prefix used when broadcasting to all servers"),
|
||||
|
||||
cfg.StrOpt('group_request_prefix',
|
||||
default='unicast',
|
||||
deprecated_group='amqp1',
|
||||
help="address prefix when sending to any server in group"),
|
||||
|
||||
# Routable addressing customization:
|
||||
#
|
||||
# Addresses a composed of the following string values using a template in
|
||||
# the form of:
|
||||
# $(address_prefix)/$(*cast)/$(exchange)/$(topic)[/$(server-name)]
|
||||
# where *cast is one of the multicast/unicast/anycast values used to
|
||||
# identify the delivery pattern used for the addressed message
|
||||
|
||||
cfg.StrOpt('rpc_address_prefix',
|
||||
default='openstack.org/om/rpc',
|
||||
help="Address prefix for all generated RPC addresses"),
|
||||
|
||||
cfg.StrOpt('notify_address_prefix',
|
||||
default='openstack.org/om/notify',
|
||||
help="Address prefix for all generated Notification addresses"),
|
||||
|
||||
cfg.StrOpt('multicast_address',
|
||||
default='multicast',
|
||||
help="Appended to the address prefix when sending a fanout"
|
||||
" message. Used by the message bus to identify fanout"
|
||||
" messages."),
|
||||
|
||||
cfg.StrOpt('unicast_address',
|
||||
default='unicast',
|
||||
help="Appended to the address prefix when sending to a"
|
||||
" particular RPC/Notification server. Used by the message bus"
|
||||
" to identify messages sent to a single destination."),
|
||||
|
||||
cfg.StrOpt('anycast_address',
|
||||
default='anycast',
|
||||
help="Appended to the address prefix when sending to a group of"
|
||||
" consumers. Used by the message bus to identify messages that"
|
||||
" should be delivered in a round-robin fashion across"
|
||||
" consumers."),
|
||||
|
||||
cfg.StrOpt('default_notification_exchange',
|
||||
default=None,
|
||||
help="Exchange name used in notification addresses.\n"
|
||||
"Exchange name resolution precedence:\n"
|
||||
"Target.exchange if set\n"
|
||||
"else default_notification_exchange if set\n"
|
||||
"else control_exchange if set\n"
|
||||
"else 'notify'"),
|
||||
|
||||
cfg.StrOpt('default_rpc_exchange',
|
||||
default=None,
|
||||
help="Exchange name used in RPC addresses.\n"
|
||||
"Exchange name resolution precedence:\n"
|
||||
"Target.exchange if set\n"
|
||||
"else default_rpc_exchange if set\n"
|
||||
"else control_exchange if set\n"
|
||||
"else 'rpc'")
|
||||
]
|
||||
|
@ -28,6 +28,7 @@ import time
|
||||
import uuid
|
||||
|
||||
from oslo_config import cfg
|
||||
from oslo_messaging.target import Target
|
||||
from oslo_serialization import jsonutils
|
||||
from oslo_utils import importutils
|
||||
from oslo_utils import timeutils
|
||||
@ -36,7 +37,6 @@ from oslo_messaging._drivers.amqp1_driver import opts
|
||||
from oslo_messaging._drivers import base
|
||||
from oslo_messaging._drivers import common
|
||||
from oslo_messaging._i18n import _LI, _LW
|
||||
from oslo_messaging import target as messaging_target
|
||||
|
||||
|
||||
proton = importutils.try_import('proton')
|
||||
@ -344,7 +344,7 @@ class ProtonDriver(base.BaseDriver):
|
||||
LOG.debug("Send notification to %s", target)
|
||||
task = controller.SendTask("Notify", request, target,
|
||||
time.time() + self._default_notify_timeout,
|
||||
retry, wait_for_ack=True)
|
||||
retry, wait_for_ack=True, notification=True)
|
||||
self._ctrl.add_task(task)
|
||||
rc = task.wait()
|
||||
if isinstance(rc, Exception):
|
||||
@ -374,10 +374,13 @@ class ProtonDriver(base.BaseDriver):
|
||||
raise NotImplementedError('"pool" not implemented by '
|
||||
'this transport driver')
|
||||
listener = ProtonListener(self)
|
||||
# this is how the destination target is created by the notifier,
|
||||
# see MessagingDriver.notify in oslo_messaging/notify/messaging.py
|
||||
for target, priority in targets_and_priorities:
|
||||
topic = '%s.%s' % (target.topic, priority)
|
||||
t = messaging_target.Target(topic=topic)
|
||||
task = controller.SubscribeTask(t, listener, notifications=True)
|
||||
# Sooo... the exchange is simply discarded? (see above comment)
|
||||
task = controller.SubscribeTask(Target(topic=topic),
|
||||
listener, notifications=True)
|
||||
self._ctrl.add_task(task)
|
||||
task.wait()
|
||||
return base.PollStyleListenerAdapter(listener, batch_size,
|
||||
|
@ -37,6 +37,12 @@ from oslo_messaging.tests import utils as test_utils
|
||||
# are available in the base repos for all supported platforms.
|
||||
pyngus = importutils.try_import("pyngus")
|
||||
if pyngus:
|
||||
from oslo_messaging._drivers.amqp1_driver.addressing \
|
||||
import AddresserFactory
|
||||
from oslo_messaging._drivers.amqp1_driver.addressing \
|
||||
import LegacyAddresser
|
||||
from oslo_messaging._drivers.amqp1_driver.addressing \
|
||||
import RoutableAddresser
|
||||
import oslo_messaging._drivers.impl_amqp1 as amqp_driver
|
||||
|
||||
# The Cyrus-based SASL tests can only be run if the installed version of proton
|
||||
@ -120,7 +126,7 @@ class _AmqpBrokerTestCase(test_utils.BaseTestCase):
|
||||
@testtools.skipUnless(pyngus, "proton modules not present")
|
||||
def setUp(self):
|
||||
super(_AmqpBrokerTestCase, self).setUp()
|
||||
self._broker = FakeBroker()
|
||||
self._broker = FakeBroker(self.conf.oslo_messaging_amqp)
|
||||
self._broker_addr = "amqp://%s:%d" % (self._broker.host,
|
||||
self._broker.port)
|
||||
self._broker_url = oslo_messaging.TransportURL.parse(
|
||||
@ -531,7 +537,8 @@ class TestAuthentication(test_utils.BaseTestCase):
|
||||
# for simplicity, encode the credentials as they would appear 'on the
|
||||
# wire' in a SASL frame - username and password prefixed by zero.
|
||||
user_credentials = ["\0joe\0secret"]
|
||||
self._broker = FakeBroker(sasl_mechanisms="PLAIN",
|
||||
self._broker = FakeBroker(self.conf.oslo_messaging_amqp,
|
||||
sasl_mechanisms="PLAIN",
|
||||
user_credentials=user_credentials)
|
||||
self._broker.start()
|
||||
|
||||
@ -614,7 +621,8 @@ mech_list: ${mechs}
|
||||
with open(conf, 'w') as f:
|
||||
f.write(t.substitute(db=db, mechs=mechs))
|
||||
|
||||
self._broker = FakeBroker(sasl_mechanisms=mechs,
|
||||
self._broker = FakeBroker(self.conf.oslo_messaging_amqp,
|
||||
sasl_mechanisms=mechs,
|
||||
user_credentials=["\0joe\0secret"],
|
||||
sasl_config_dir=self._conf_dir,
|
||||
sasl_config_name="openstack")
|
||||
@ -716,7 +724,13 @@ class TestFailover(test_utils.BaseTestCase):
|
||||
|
||||
def setUp(self):
|
||||
super(TestFailover, self).setUp()
|
||||
self._brokers = [FakeBroker(), FakeBroker()]
|
||||
# configure different addressing modes on the brokers to test failing
|
||||
# over from one type of backend to another
|
||||
self.config(addressing_mode='dynamic', group="oslo_messaging_amqp")
|
||||
self._brokers = [FakeBroker(self.conf.oslo_messaging_amqp,
|
||||
product="qpid-cpp"),
|
||||
FakeBroker(self.conf.oslo_messaging_amqp,
|
||||
product="routable")]
|
||||
self._primary = 0
|
||||
self._backup = 1
|
||||
hosts = []
|
||||
@ -1025,6 +1039,240 @@ class TestLinkRecovery(_AmqpBrokerTestCase):
|
||||
driver.cleanup()
|
||||
|
||||
|
||||
@testtools.skipUnless(pyngus, "proton modules not present")
|
||||
class TestAddressing(test_utils.BaseTestCase):
|
||||
# Verify the addressing modes supported by the driver
|
||||
def _address_test(self, rpc_target, targets_priorities):
|
||||
# verify proper messaging semantics for a given addressing mode
|
||||
broker = FakeBroker(self.conf.oslo_messaging_amqp)
|
||||
broker.start()
|
||||
url = oslo_messaging.TransportURL.parse(self.conf,
|
||||
"amqp://%s:%d" %
|
||||
(broker.host, broker.port))
|
||||
driver = amqp_driver.ProtonDriver(self.conf, url)
|
||||
|
||||
rl = []
|
||||
for server in ["Server1", "Server2"]:
|
||||
_ = driver.listen(rpc_target(server=server), None,
|
||||
None)._poll_style_listener
|
||||
# 3 == 1 msg to server + 1 fanout msg + 1 anycast msg
|
||||
rl.append(_ListenerThread(_, 3))
|
||||
|
||||
nl = []
|
||||
for n in range(2):
|
||||
_ = driver.listen_for_notifications(targets_priorities, None, None,
|
||||
None)._poll_style_listener
|
||||
nl.append(_ListenerThread(_, len(targets_priorities)))
|
||||
|
||||
driver.send(rpc_target(server="Server1"), {"context": "whatever"},
|
||||
{"msg": "Server1"})
|
||||
driver.send(rpc_target(server="Server2"), {"context": "whatever"},
|
||||
{"msg": "Server2"})
|
||||
driver.send(rpc_target(fanout=True), {"context": "whatever"},
|
||||
{"msg": "Fanout"})
|
||||
# FakeBroker should evenly distribute these across the servers
|
||||
driver.send(rpc_target(server=None), {"context": "whatever"},
|
||||
{"msg": "Anycast1"})
|
||||
driver.send(rpc_target(server=None), {"context": "whatever"},
|
||||
{"msg": "Anycast2"})
|
||||
|
||||
expected = []
|
||||
for n in targets_priorities:
|
||||
# this is how the notifier creates an address:
|
||||
topic = "%s.%s" % (n[0].topic, n[1])
|
||||
target = oslo_messaging.Target(topic=topic)
|
||||
driver.send_notification(target, {"context": "whatever"},
|
||||
{"msg": topic}, 2.0)
|
||||
expected.append(topic)
|
||||
|
||||
for l in rl:
|
||||
l.join(timeout=30)
|
||||
|
||||
# anycast will not evenly distribute an odd number of msgs
|
||||
predicate = lambda: len(expected) == (nl[0].messages.qsize() +
|
||||
nl[1].messages.qsize())
|
||||
_wait_until(predicate, 30)
|
||||
for l in nl:
|
||||
l.kill(timeout=30)
|
||||
|
||||
s1_payload = [m.message.get('msg') for m in rl[0].get_messages()]
|
||||
s2_payload = [m.message.get('msg') for m in rl[1].get_messages()]
|
||||
|
||||
self.assertTrue("Server1" in s1_payload
|
||||
and "Server2" not in s1_payload)
|
||||
self.assertTrue("Server2" in s2_payload
|
||||
and "Server1" not in s2_payload)
|
||||
self.assertEqual(s1_payload.count("Fanout"), 1)
|
||||
self.assertEqual(s2_payload.count("Fanout"), 1)
|
||||
self.assertEqual((s1_payload + s2_payload).count("Anycast1"), 1)
|
||||
self.assertEqual((s1_payload + s2_payload).count("Anycast2"), 1)
|
||||
|
||||
n1_payload = [m.message.get('msg') for m in nl[0].get_messages()]
|
||||
n2_payload = [m.message.get('msg') for m in nl[1].get_messages()]
|
||||
|
||||
self.assertEqual((n1_payload + n2_payload).sort(), expected.sort())
|
||||
|
||||
driver.cleanup()
|
||||
broker.stop()
|
||||
return broker.message_log
|
||||
|
||||
def test_routable_address(self):
|
||||
# verify routable address mode
|
||||
self.config(addressing_mode='routable', group="oslo_messaging_amqp")
|
||||
_opts = self.conf.oslo_messaging_amqp
|
||||
notifications = [(oslo_messaging.Target(topic="test-topic"), 'info'),
|
||||
(oslo_messaging.Target(topic="test-topic"), 'error'),
|
||||
(oslo_messaging.Target(topic="test-topic"), 'debug')]
|
||||
|
||||
msgs = self._address_test(oslo_messaging.Target(exchange="ex",
|
||||
topic="test-topic"),
|
||||
notifications)
|
||||
addrs = [m.address for m in msgs]
|
||||
|
||||
notify_addrs = [a for a in addrs
|
||||
if a.startswith(_opts.notify_address_prefix)]
|
||||
self.assertEqual(len(notify_addrs), len(notifications))
|
||||
# expect all notifications to be 'anycast'
|
||||
self.assertEqual(len(notifications),
|
||||
len([a for a in notify_addrs
|
||||
if _opts.anycast_address in a]))
|
||||
|
||||
rpc_addrs = [a for a in addrs
|
||||
if a.startswith(_opts.rpc_address_prefix)]
|
||||
# 2 anycast messages
|
||||
self.assertEqual(2,
|
||||
len([a for a in rpc_addrs
|
||||
if _opts.anycast_address in a]))
|
||||
# 1 fanout sent
|
||||
self.assertEqual(1,
|
||||
len([a for a in rpc_addrs
|
||||
if _opts.multicast_address in a]))
|
||||
# 2 unicast messages (1 for each server)
|
||||
self.assertEqual(2,
|
||||
len([a for a in rpc_addrs
|
||||
if _opts.unicast_address in a]))
|
||||
|
||||
def test_legacy_address(self):
|
||||
# verify legacy address mode
|
||||
self.config(addressing_mode='legacy', group="oslo_messaging_amqp")
|
||||
_opts = self.conf.oslo_messaging_amqp
|
||||
notifications = [(oslo_messaging.Target(topic="test-topic"), 'info'),
|
||||
(oslo_messaging.Target(topic="test-topic"), 'error'),
|
||||
(oslo_messaging.Target(topic="test-topic"), 'debug')]
|
||||
|
||||
msgs = self._address_test(oslo_messaging.Target(exchange="ex",
|
||||
topic="test-topic"),
|
||||
notifications)
|
||||
addrs = [m.address for m in msgs]
|
||||
|
||||
server_addrs = [a for a in addrs
|
||||
if a.startswith(_opts.server_request_prefix)]
|
||||
broadcast_addrs = [a for a in addrs
|
||||
if a.startswith(_opts.broadcast_prefix)]
|
||||
group_addrs = [a for a in addrs
|
||||
if a.startswith(_opts.group_request_prefix)]
|
||||
# 2 server address messages sent
|
||||
self.assertEqual(len(server_addrs), 2)
|
||||
# 1 fanout address message sent
|
||||
self.assertEqual(len(broadcast_addrs), 1)
|
||||
# group messages: 2 rpc + all notifications
|
||||
self.assertEqual(len(group_addrs),
|
||||
2 + len(notifications))
|
||||
|
||||
def test_address_options(self):
|
||||
# verify addressing configuration options
|
||||
self.config(addressing_mode='routable', group="oslo_messaging_amqp")
|
||||
self.config(rpc_address_prefix="RPC-PREFIX",
|
||||
group="oslo_messaging_amqp")
|
||||
self.config(notify_address_prefix="NOTIFY-PREFIX",
|
||||
group="oslo_messaging_amqp")
|
||||
|
||||
self.config(multicast_address="MULTI-CAST",
|
||||
group="oslo_messaging_amqp")
|
||||
self.config(unicast_address="UNI-CAST",
|
||||
group="oslo_messaging_amqp")
|
||||
self.config(anycast_address="ANY-CAST",
|
||||
group="oslo_messaging_amqp")
|
||||
|
||||
self.config(default_notification_exchange="NOTIFY-EXCHANGE",
|
||||
group="oslo_messaging_amqp")
|
||||
self.config(default_rpc_exchange="RPC-EXCHANGE",
|
||||
group="oslo_messaging_amqp")
|
||||
|
||||
notifications = [(oslo_messaging.Target(topic="test-topic"), 'info'),
|
||||
(oslo_messaging.Target(topic="test-topic"), 'error'),
|
||||
(oslo_messaging.Target(topic="test-topic"), 'debug')]
|
||||
|
||||
msgs = self._address_test(oslo_messaging.Target(exchange=None,
|
||||
topic="test-topic"),
|
||||
notifications)
|
||||
addrs = [m.address for m in msgs]
|
||||
|
||||
notify_addrs = [a for a in addrs
|
||||
if a.startswith("NOTIFY-PREFIX")]
|
||||
self.assertEqual(len(notify_addrs), len(notifications))
|
||||
# expect all notifications to be 'anycast'
|
||||
self.assertEqual(len(notifications),
|
||||
len([a for a in notify_addrs
|
||||
if "ANY-CAST" in a]))
|
||||
# and all should contain the default exchange:
|
||||
self.assertEqual(len(notifications),
|
||||
len([a for a in notify_addrs
|
||||
if "NOTIFY-EXCHANGE" in a]))
|
||||
|
||||
rpc_addrs = [a for a in addrs
|
||||
if a.startswith("RPC-PREFIX")]
|
||||
# 2 RPC anycast messages
|
||||
self.assertEqual(2,
|
||||
len([a for a in rpc_addrs
|
||||
if "ANY-CAST" in a]))
|
||||
# 1 RPC fanout sent
|
||||
self.assertEqual(1,
|
||||
len([a for a in rpc_addrs
|
||||
if "MULTI-CAST" in a]))
|
||||
# 2 RPC unicast messages (1 for each server)
|
||||
self.assertEqual(2,
|
||||
len([a for a in rpc_addrs
|
||||
if "UNI-CAST" in a]))
|
||||
|
||||
self.assertEqual(len(rpc_addrs),
|
||||
len([a for a in rpc_addrs
|
||||
if "RPC-EXCHANGE" in a]))
|
||||
|
||||
def _dynamic_test(self, product):
|
||||
# return the addresser used when connected to 'product'
|
||||
broker = FakeBroker(self.conf.oslo_messaging_amqp,
|
||||
product=product)
|
||||
broker.start()
|
||||
url = oslo_messaging.TransportURL.parse(self.conf,
|
||||
"amqp://%s:%d" %
|
||||
(broker.host, broker.port))
|
||||
driver = amqp_driver.ProtonDriver(self.conf, url)
|
||||
|
||||
# need to send a message to initate the connection to the broker
|
||||
target = oslo_messaging.Target(topic="test-topic",
|
||||
server="Server")
|
||||
listener = _ListenerThread(
|
||||
driver.listen(target, None, None)._poll_style_listener, 1)
|
||||
driver.send(target, {"context": True}, {"msg": "value"},
|
||||
wait_for_reply=False)
|
||||
listener.join(timeout=30)
|
||||
|
||||
addresser = driver._ctrl.addresser
|
||||
driver.cleanup()
|
||||
broker.stop() # clears the driver's addresser
|
||||
return addresser
|
||||
|
||||
def test_dynamic_addressing(self):
|
||||
# simply check that the correct addresser is provided based on the
|
||||
# identity of the messaging back-end
|
||||
self.config(addressing_mode='dynamic', group="oslo_messaging_amqp")
|
||||
self.assertTrue(isinstance(self._dynamic_test("router"),
|
||||
RoutableAddresser))
|
||||
self.assertTrue(isinstance(self._dynamic_test("qpid-cpp"),
|
||||
LegacyAddresser))
|
||||
|
||||
|
||||
class FakeBroker(threading.Thread):
|
||||
"""A test AMQP message 'broker'."""
|
||||
|
||||
@ -1032,7 +1280,7 @@ class FakeBroker(threading.Thread):
|
||||
class Connection(pyngus.ConnectionEventHandler):
|
||||
"""A single AMQP connection."""
|
||||
|
||||
def __init__(self, server, socket_, name,
|
||||
def __init__(self, server, socket_, name, product,
|
||||
sasl_mechanisms, user_credentials,
|
||||
sasl_config_dir, sasl_config_name):
|
||||
"""Create a Connection using socket_."""
|
||||
@ -1050,6 +1298,8 @@ class FakeBroker(threading.Thread):
|
||||
properties['x-sasl-config-dir'] = sasl_config_dir
|
||||
if sasl_config_name:
|
||||
properties['x-sasl-config-name'] = sasl_config_name
|
||||
if product:
|
||||
properties['properties'] = {'product': product}
|
||||
|
||||
self.connection = server.container.create_connection(
|
||||
name, self, properties)
|
||||
@ -1248,11 +1498,10 @@ class FakeBroker(threading.Thread):
|
||||
if self.link.capacity < 1:
|
||||
self.server.credit_exhausted(self.link)
|
||||
|
||||
def __init__(self, server_prefix="exclusive",
|
||||
broadcast_prefix="broadcast",
|
||||
group_prefix="unicast",
|
||||
address_separator=".",
|
||||
def __init__(self, cfg,
|
||||
sock_addr="", sock_port=0,
|
||||
product=None,
|
||||
default_exchange="Test-Exchange",
|
||||
sasl_mechanisms="ANONYMOUS",
|
||||
user_credentials=None,
|
||||
sasl_config_dir=None,
|
||||
@ -1261,10 +1510,8 @@ class FakeBroker(threading.Thread):
|
||||
if not pyngus:
|
||||
raise AssertionError("pyngus module not present")
|
||||
threading.Thread.__init__(self)
|
||||
self._server_prefix = server_prefix + address_separator
|
||||
self._broadcast_prefix = broadcast_prefix + address_separator
|
||||
self._group_prefix = group_prefix + address_separator
|
||||
self._address_separator = address_separator
|
||||
self._config = cfg
|
||||
self._product = product
|
||||
self._sasl_mechanisms = sasl_mechanisms
|
||||
self._sasl_config_dir = sasl_config_dir
|
||||
self._sasl_config_name = sasl_config_name
|
||||
@ -1275,6 +1522,22 @@ class FakeBroker(threading.Thread):
|
||||
self.host, self.port = self._my_socket.getsockname()
|
||||
self.container = pyngus.Container("test_server_%s:%d"
|
||||
% (self.host, self.port))
|
||||
|
||||
# create an addresser using the test client's config and expected
|
||||
# message bus so the broker can parse the message addresses
|
||||
af = AddresserFactory(default_exchange,
|
||||
cfg.addressing_mode,
|
||||
legacy_server_prefix=cfg.server_request_prefix,
|
||||
legacy_broadcast_prefix=cfg.broadcast_prefix,
|
||||
legacy_group_prefix=cfg.group_request_prefix,
|
||||
rpc_prefix=cfg.rpc_address_prefix,
|
||||
notify_prefix=cfg.notify_address_prefix,
|
||||
multicast=cfg.multicast_address,
|
||||
unicast=cfg.unicast_address,
|
||||
anycast=cfg.anycast_address)
|
||||
props = {'product': product} if product else {}
|
||||
self._addresser = af(props)
|
||||
|
||||
self._connections = {}
|
||||
self._sources = {}
|
||||
self._pause = threading.Event()
|
||||
@ -1290,6 +1553,8 @@ class FakeBroker(threading.Thread):
|
||||
self.receiver_link_count = 0
|
||||
self.sender_link_ack_count = 0
|
||||
self.sender_link_requeue_count = 0
|
||||
# log of all messages received by the broker
|
||||
self.message_log = []
|
||||
# callback hooks
|
||||
self.on_sender_active = lambda link: None
|
||||
self.on_receiver_active = lambda link: link.add_capacity(10)
|
||||
@ -1358,6 +1623,7 @@ class FakeBroker(threading.Thread):
|
||||
# create a new Connection for it:
|
||||
name = str(addr)
|
||||
conn = FakeBroker.Connection(self, sock, name,
|
||||
self._product,
|
||||
self._sasl_mechanisms,
|
||||
self._user_credentials,
|
||||
self._sasl_config_dir,
|
||||
@ -1427,6 +1693,7 @@ class FakeBroker(threading.Thread):
|
||||
|
||||
def forward_message(self, message, handle, rlink):
|
||||
# returns True if message was routed
|
||||
self.message_log.append(message)
|
||||
dest = message.address
|
||||
if dest not in self._sources:
|
||||
# can't forward
|
||||
@ -1437,14 +1704,14 @@ class FakeBroker(threading.Thread):
|
||||
return
|
||||
|
||||
LOG.debug("Forwarding [%s]", dest)
|
||||
# route "behavior" determined by prefix:
|
||||
if dest.startswith(self._broadcast_prefix):
|
||||
# route "behavior" determined by address prefix:
|
||||
if self._addresser._is_multicast(dest):
|
||||
self.fanout_count += 1
|
||||
for link in self._sources[dest]:
|
||||
self.fanout_sent_count += 1
|
||||
LOG.debug("Broadcast to %s", dest)
|
||||
link.send_message(message)
|
||||
elif dest.startswith(self._group_prefix):
|
||||
elif self._addresser._is_anycast(dest):
|
||||
# round-robin:
|
||||
self.topic_count += 1
|
||||
link = self._sources[dest].pop(0)
|
||||
|
Loading…
x
Reference in New Issue
Block a user