Integration with oslo.messaging library

Port Trove to use oslo messaging library instead of obsolete messaging
code from oslo incubator.

Change-Id: Ibd886f3cb4a45250c7c434b3af711abee266671c
Implements: blueprint rpc-versioning
This commit is contained in:
Sergey Gotliv 2014-09-07 11:45:23 +03:00
parent 3e56c2177b
commit ea148d7dfe
66 changed files with 862 additions and 6733 deletions

View File

@ -5,6 +5,23 @@ control_exchange = trove
trove_auth_url = http://0.0.0.0:5000/v2.0
sql_connection = mysql://root:e1a2c042c828d3566d0a@localhost/trove
#===================== RPC Configuration =================================
# URL representing the messaging driver to use and its full configuration.
# If not set, we fall back to the 'rpc_backend' option and driver specific
# configuration.
#transport_url=<None>
# The messaging driver to use. Options include rabbit, qpid and zmq.
# Default is rabbit. (string value)
#rpc_backend=rabbit
# The default exchange under which topics are scoped. May be
# overridden by an exchange name specified in the 'transport_url option.
#control_exchange=openstack
# ================ RabbitMQ Configuration ===============================
# The RabbitMQ broker address where a single node is used.
# (string value)
#rabbit_host=localhost
@ -28,10 +45,5 @@ rabbit_password=f7999d1955c5014aa32c
# The RabbitMQ virtual host. (string value)
#rabbit_virtual_host=/
# RabbitMQ topic used for OpenStack notifications. (list value)
#rabbit_notification_topic = ['notifications']
rpc_backend = trove.openstack.common.rpc.impl_kombu
# The manager class to use for conductor. (string value)
conductor_manager = trove.conductor.manager.Manager

View File

@ -1,5 +1,20 @@
[DEFAULT]
#=========== RPC Configuration ======================
# URL representing the messaging driver to use and its full configuration.
# If not set, we fall back to the 'rpc_backend' option and driver specific
# configuration.
#transport_url=<None>
# The messaging driver to use. Options include rabbit, qpid and zmq.
# Default is rabbit. (string value)
#rpc_backend=rabbit
# The default exchange under which topics are scoped. May be
# overridden by an exchange name specified in the 'transport_url option.
#control_exchange=openstack
# ========== Sample RabbitMQ Configuration ==========
# The RabbitMQ broker address where a single node is used.
@ -25,10 +40,6 @@ rabbit_password=f7999d1955c5014aa32c
# The RabbitMQ virtual host. (string value)
# rabbit_virtual_host=/
# RabbitMQ topic used for OpenStack notifications. (list value)
# rabbit_notification_topic = ['notifications']
# ========== Configuration options for Swift ==========
# The swift_url can be specified directly or fetched from Keystone catalog.

View File

@ -11,6 +11,23 @@ debug = True
# restore_usage_timeout = 36000
update_status_on_fail = True
#================= RPC Configuration ================================
# URL representing the messaging driver to use and its full configuration.
# If not set, we fall back to the 'rpc_backend' option and driver specific
# configuration.
#transport_url=<None>
# The messaging driver to use. Options include rabbit, qpid and zmq.
# Default is rabbit. (string value)
#rpc_backend=rabbit
# The default exchange under which topics are scoped. May be
# overridden by an exchange name specified in the 'transport_url option.
#control_exchange=openstack
# ================ RabbitMQ Configuration ===========================
# The RabbitMQ broker address where a single node is used.
# (string value)
#rabbit_host=localhost
@ -34,11 +51,6 @@ rabbit_password=f7999d1955c5014aa32c
# The RabbitMQ virtual host. (string value)
#rabbit_virtual_host=/
# RabbitMQ topic used for OpenStack notifications. (list value)
#rabbit_notification_topic = ['notifications']
rpc_backend = trove.openstack.common.rpc.impl_kombu
# SQLAlchemy connection string for the reference implementation
# registry server. Any valid SQLAlchemy connection string is fine.
# See: http://www.sqlalchemy.org/docs/05/reference/sqlalchemy/connections.html#sqlalchemy.create_engine
@ -153,10 +165,15 @@ network_label_regex = ^private$
# Datastore templates
template_path = /etc/trove/templates/
# ============ notifer queue kombu connection options ========================
# ============ Notification System configuration ===========================
# Sets the notification driver used by oslo.messaging. Options include
# messaging, messagingv2, log and routing. Default is 'noop'
# notification_driver=noop
# Topics used for OpenStack notifications, list value. Default is 'notifications'.
# notification_topics=['notifications', ]
# usage notifications
notification_driver=trove.openstack.common.notifier.rpc_notifier
control_exchange=trove
# ============ Logging information =============================

View File

@ -15,6 +15,23 @@ bind_port = 8779
# be the number of CPUs available. (integer value)
#trove_api_workers=None
#===================== RPC Configuration =================================
# URL representing the messaging driver to use and its full configuration.
# If not set, we fall back to the 'rpc_backend' option and driver specific
# configuration.
#transport_url=<None>
# The messaging driver to use. Options include rabbit, qpid and zmq.
# Default is rabbit. (string value)
#rpc_backend=rabbit
# The default exchange under which topics are scoped. May be
# overridden by an exchange name specified in the 'transport_url option.
#control_exchange=openstack
# ==================== RabbitMQ Configuration =======================
# The RabbitMQ broker address where a single node is used.
# (string value)
#rabbit_host=localhost
@ -38,11 +55,6 @@ rabbit_password=f7999d1955c5014aa32c
# The RabbitMQ virtual host. (string value)
#rabbit_virtual_host=/
# RabbitMQ topic used for OpenStack notifications. (list value)
#rabbit_notification_topic = ['notifications']
rpc_backend = trove.openstack.common.rpc.impl_kombu
# SQLAlchemy connection string for the reference implementation
# registry server. Any valid SQLAlchemy connection string is fine.
# See: http://www.sqlalchemy.org/docs/05/reference/sqlalchemy/connections.html#sqlalchemy.create_engine
@ -162,7 +174,14 @@ reboot_time_out = 60
api_paste_config = api-paste.ini
# ============ notifer queue kombu connection options ========================
# ============ Notification System configuration ===========================
# Sets the notification driver used by oslo.messaging. Options include
# messaging, messagingv2, log and routing. Default is 'noop'
# notification_driver=noop
# Topics used for OpenStack notifications, list value. Default is 'notifications'.
# notification_topics=['notifications', ]
control_exchange = trove

View File

@ -7,7 +7,10 @@ remote_swift_client = trove.tests.fakes.swift.fake_create_swift_client
remote_cinder_client = trove.tests.fakes.nova.fake_create_cinder_client
# Fake out the RPC implementation
rpc_backend = trove.common.rpc.impl_fake
rpc_backend = fake
# Skip running periodic tasks
report_interval = 0
# Fake out DNS.
trove_dns_support = True

View File

@ -36,3 +36,5 @@ Babel>=1.3
six>=1.7.0
stevedore>=1.1.0 # Apache-2.0
ordereddict
oslo.messaging>=1.4.0,!=1.5.0

View File

@ -25,12 +25,14 @@ import traceback
from trove.common import cfg
from trove.common import utils
from trove.common.rpc import service as rpc_service
from trove.common.rpc import version as rpc_version
from trove.openstack.common import log as logging
from trove.tests.config import CONFIG
from trove import rpc
from wsgi_intercept.httplib2_intercept import install as wsgi_install
import proboscis
import wsgi_intercept
from trove.openstack.common.rpc import service as rpc_service
import eventlet
eventlet.monkey_patch(thread=False)
@ -63,11 +65,11 @@ def initialize_trove(config_file):
default_config_files=[config_file])
logging.setup(None)
topic = CONF.taskmanager_queue
rpc.init(CONF)
from trove.taskmanager import manager
manager_impl = manager.Manager()
taskman_service = rpc_service.Service(None, topic=topic,
manager=manager_impl)
taskman_service = rpc_service.RpcService(
None, topic=topic, rpc_api_version=rpc_version.RPC_API_VERSION,
manager='trove.taskmanager.manager.Manager')
taskman_service.start()
return pastedeploy.paste_deploy_app(config_file, 'trove', {})

View File

@ -37,6 +37,14 @@ trove.api.extensions =
mysql = trove.extensions.routes.mysql:Mysql
security_group = trove.extensions.routes.security_group:Security_group
# These are for backwards compatibility with Havana notification_driver configuration values
oslo.messaging.notify.drivers =
trove.openstack.common.notifier.log_notifier = oslo.messaging.notify._impl_log:LogDriver
trove.openstack.common.notifier.no_op_notifier = oslo.messaging.notify._impl_noop:NoOpDriver
trove.openstack.common.notifier.rpc_notifier2 = oslo.messaging.notify._impl_messaging:MessagingV2Driver
trove.openstack.common.notifier.rpc_notifier = oslo.messaging.notify._impl_messaging:MessagingDriver
trove.openstack.common.notifier.test_notifier = oslo.messaging.notify._impl_test:TestDriver
[global]
setup-hooks =
pbr.hooks.setup_hook

View File

@ -43,10 +43,16 @@ def initialize(extra_opts=None, pre_logging=None):
logging.setup(None)
debug_utils.setup()
# Patch 'thread' module if debug is disabled
# Patch 'thread' module if debug is disabled.
if not debug_utils.enabled():
eventlet.monkey_patch(thread=True)
# rpc module must be loaded after decision about thread monkeypatching
# because if thread module is not monkeypatched we can't use eventlet
# executor from oslo.messaging library.
from trove import rpc
rpc.init(conf)
# Initialize Trove database.
from trove.db import get_db_api
get_db_api().configure_db(conf)

View File

@ -19,11 +19,13 @@ from trove.openstack.common import processutils
@with_initialize
def main(conf):
from trove.common.rpc import service as rpc_service
from trove.common.rpc import version as rpc_version
from trove.openstack.common import service as openstack_service
topic = conf.conductor_queue
server = rpc_service.RpcService(manager=conf.conductor_manager,
topic=topic)
server = rpc_service.RpcService(
manager=conf.conductor_manager, topic=topic,
rpc_api_version=rpc_version.RPC_API_VERSION)
workers = conf.trove_conductor_workers or processutils.get_worker_count()
launcher = openstack_service.launch(server, workers=workers)
launcher.wait()

View File

@ -47,7 +47,7 @@ def main(conf):
def start_fake_taskmanager(conf):
topic = conf.taskmanager_queue
from trove.openstack.common.rpc import service as rpc_service
from trove.common.rpc import service as rpc_service
from trove.taskmanager import manager
manager_impl = manager.Manager()
taskman_service = rpc_service.Service(None, topic=topic,

View File

@ -14,26 +14,23 @@
# under the License.
import eventlet
import gettext
import sys
gettext.install('trove', unicode=1)
from trove.common import cfg
from trove.common import debug_utils
from trove.common.rpc import service as rpc_service
from oslo.config import cfg as openstack_cfg
from trove.openstack.common import log as logging
from trove.openstack.common import service as openstack_service
# Apply whole eventlet.monkey_patch excluding 'thread' module.
# Decision for 'thread' module patching will be made
# after debug_utils setting up
eventlet.monkey_patch(all=True, thread=False)
import gettext
gettext.install('trove', unicode=1)
import sys
from oslo.config import cfg as openstack_cfg
from trove.common import cfg
from trove.common import debug_utils
from trove.openstack.common import log as logging
from trove.openstack.common import service as openstack_service
CONF = cfg.CONF
# The guest_id opt definition must match the one in common/cfg.py
CONF.register_opts([openstack_cfg.StrOpt('guest_id', default=None,
@ -42,7 +39,6 @@ CONF.register_opts([openstack_cfg.StrOpt('guest_id', default=None,
def main():
cfg.parse_args(sys.argv)
from trove.guestagent import dbaas
logging.setup(None)
debug_utils.setup()
@ -51,11 +47,24 @@ def main():
if not debug_utils.enabled():
eventlet.monkey_patch(thread=True)
from trove.guestagent import dbaas
manager = dbaas.datastore_registry().get(CONF.datastore_manager)
if not manager:
msg = ("Manager class not registered for datastore manager %s" %
CONF.datastore_manager)
raise RuntimeError(msg)
server = rpc_service.RpcService(manager=manager, host=CONF.guest_id)
# rpc module must be loaded after decision about thread monkeypatching
# because if thread module is not monkeypatched we can't use eventlet
# executor from oslo.messaging library.
from trove import rpc
rpc.init(CONF)
from trove.common.rpc import service as rpc_service
from trove.common.rpc import version as rpc_version
server = rpc_service.RpcService(
manager=manager, host=CONF.guest_id,
rpc_api_version=rpc_version.RPC_API_VERSION)
launcher = openstack_service.launch(server)
launcher.wait()

View File

@ -21,10 +21,12 @@ extra_opts = [openstack_cfg.StrOpt('taskmanager_manager')]
def startup(conf, topic):
from trove.common.rpc import service as rpc_service
from trove.common.rpc import version as rpc_version
from trove.openstack.common import service as openstack_service
server = rpc_service.RpcService(manager=conf.taskmanager_manager,
topic=topic)
server = rpc_service.RpcService(
manager=conf.taskmanager_manager, topic=topic,
rpc_api_version=rpc_version.RPC_API_VERSION)
launcher = openstack_service.launch(server)
launcher.wait()

View File

@ -15,11 +15,12 @@
# under the License.
"""Routines for configuring Trove."""
import trove
from oslo.config import cfg
import os.path
from oslo.config import cfg
import trove
UNKNOWN_SERVICE_ID = 'unknown-service-id-error'
path_opts = [
@ -740,6 +741,24 @@ postgresql_opts = [
cfg.ListOpt('ignore_dbs', default=['postgres']),
]
# RPC version groups
upgrade_levels = cfg.OptGroup(
'upgrade_levels',
title='RPC upgrade levels group for handling versions',
help='Contains the support version caps for each RPC API')
rpcapi_cap_opts = [
cfg.StrOpt(
'taskmanager', default="icehouse",
help='Set a version cap for messages sent to taskmanager services'),
cfg.StrOpt(
'guestagent', default="icehouse",
help='Set a version cap for messages sent to guestagent services'),
cfg.StrOpt(
'conductor', default="icehouse",
help='Set a version cap for messages sent to conductor services'),
]
CONF = cfg.CONF
CONF.register_opts(path_opts)
@ -761,6 +780,8 @@ CONF.register_opts(couchbase_opts, couchbase_group)
CONF.register_opts(mongodb_opts, mongodb_group)
CONF.register_opts(postgresql_opts, postgresql_group)
CONF.register_opts(rpcapi_cap_opts, upgrade_levels)
def custom_parser(parsername, parser):
CONF.register_cli_opt(cfg.SubCommandOpt(parsername, handler=parser))

View File

@ -33,6 +33,10 @@ class TroveContext(context.RequestContext):
self.limit = kwargs.pop('limit', None)
self.marker = kwargs.pop('marker', None)
self.service_catalog = kwargs.pop('service_catalog', None)
self.user_identity = kwargs.pop('user_identity', None)
# TODO(esp): not sure we need this
self.timeout = kwargs.pop('timeout', None)
super(TroveContext, self).__init__(**kwargs)
if not hasattr(local.store, 'context'):

View File

@ -1,36 +0,0 @@
#!/usr/bin/env python
# Copyright 2011 OpenStack Foundation
# 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.
"""RPC helper for launching a rpc service."""
import kombu
from trove.openstack.common import rpc as openstack_rpc
from trove.common import cfg
CONF = cfg.CONF
def delete_queue(context, topic):
if CONF.rpc_backend == "trove.openstack.common.rpc.impl_kombu":
connection = openstack_rpc.create_connection()
channel = connection.channel
durable = connection.conf.amqp_durable_queues
queue = kombu.entity.Queue(name=topic, channel=channel,
auto_delete=False, exclusive=False,
durable=durable)
queue.delete()

View File

@ -1,37 +0,0 @@
# Copyright 2011 OpenStack Foundation
# 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.
"""
Standard openstack.common.rpc.impl_fake with nonblocking cast
"""
import eventlet
from trove.openstack.common.rpc.impl_fake import cast
from trove.openstack.common.rpc.impl_fake import create_connection
original_cast = cast
def non_blocking_cast(*args, **kwargs):
eventlet.spawn_n(original_cast, *args, **kwargs)
cast = non_blocking_cast
# Asserting create_connection, workaround for pep8-F401 for unused import.
assert create_connection

View File

@ -18,31 +18,61 @@
import inspect
import os
from oslo import messaging
from oslo.utils import importutils
from trove.openstack.common.gettextutils import _
from trove.openstack.common import log as logging
from trove.openstack.common import loopingcall
from trove.openstack.common.rpc import service as rpc_service
from trove.openstack.common import service
from trove.common import cfg
from trove import rpc
CONF = cfg.CONF
LOG = logging.getLogger(__name__)
class RpcService(rpc_service.Service):
class RpcService(service.Service):
def __init__(self, host=None, binary=None, topic=None, manager=None):
host = host or CONF.host
binary = binary or os.path.basename(inspect.stack()[-1][1])
topic = topic or binary.rpartition('trove-')[2]
def __init__(self, host=None, binary=None, topic=None, manager=None,
rpc_api_version=None):
super(RpcService, self).__init__()
self.host = host or CONF.host
self.binary = binary or os.path.basename(inspect.stack()[-1][1])
self.topic = topic or self.binary.rpartition('trove-')[2]
self.manager_impl = importutils.import_object(manager)
self.report_interval = CONF.report_interval
super(RpcService, self).__init__(host, topic,
manager=self.manager_impl)
self.rpc_api_version = rpc_api_version or \
self.manager_impl.RPC_API_VERSION
def start(self):
super(RpcService, self).start()
LOG.debug("Creating RPC server for service %s", self.topic)
target = messaging.Target(topic=self.topic, server=self.host,
version=self.rpc_api_version)
if not hasattr(self.manager_impl, 'target'):
self.manager_impl.target = target
endpoints = [self.manager_impl]
self.rpcserver = rpc.get_server(target, endpoints)
self.rpcserver.start()
# TODO(hub-cap): Currently the context is none... do we _need_ it here?
pulse = loopingcall.FixedIntervalLoopingCall(
self.manager_impl.run_periodic_tasks, context=None)
pulse.start(interval=self.report_interval,
initial_delay=self.report_interval)
pulse.wait()
if self.report_interval > 0:
pulse = loopingcall.FixedIntervalLoopingCall(
self.manager_impl.run_periodic_tasks, context=None)
pulse.start(interval=self.report_interval,
initial_delay=self.report_interval)
pulse.wait()
def stop(self):
# Try to shut the connection down, but if we get any sort of
# errors, go ahead and ignore them.. as we're shutting down anyway
try:
self.rpcserver.stop()
except Exception:
LOG.info(_("Failed to stop RPC server before shutdown. "))
pass
super(RpcService, self).stop()

View File

@ -13,10 +13,14 @@
# License for the specific language governing permissions and limitations
# under the License.
# based on configured release version
RPC_API_VERSION = "1.0"
NOTIFICATIONS = []
def notify(_context, message):
"""Test notifier, stores notifications in memory for unittests."""
NOTIFICATIONS.append(message)
# API version history:
#
# 1.0 - Initial version. (We started keeping track at icehouse-3)
# 1.1 -
# 1.2 - ...
VERSION_ALIASES = {
'icehouse': '1.0'
}

View File

@ -12,49 +12,62 @@
# License for the specific language governing permissions and limitations
# under the License.
from oslo import messaging
from trove import rpc
from trove.common import cfg
from trove.openstack.common.rpc import proxy
from trove.common.rpc import version as rpc_version
from trove.openstack.common import log as logging
CONF = cfg.CONF
LOG = logging.getLogger(__name__)
RPC_API_VERSION = "1.0"
class API(proxy.RpcProxy):
class API(object):
"""API for interacting with trove conductor."""
def __init__(self, context):
self.context = context
super(API, self).__init__(self._get_routing_key(), RPC_API_VERSION)
super(API, self).__init__()
def _get_routing_key(self):
"""Create the routing key for conductor."""
return CONF.conductor_queue
target = messaging.Target(topic=CONF.conductor_queue,
version=rpc_version.RPC_API_VERSION)
self.version_cap = rpc_version.VERSION_ALIASES.get(
CONF.upgrade_levels.conductor)
self.client = self.get_client(target, self.version_cap)
def get_client(self, target, version_cap, serializer=None):
return rpc.get_client(target,
version_cap=version_cap,
serializer=serializer)
def heartbeat(self, instance_id, payload, sent=None):
LOG.debug("Making async call to cast heartbeat for instance: %s"
% instance_id)
self.cast(self.context, self.make_msg("heartbeat",
instance_id=instance_id,
sent=sent,
payload=payload))
cctxt = self.client.prepare(version=self.version_cap)
cctxt.cast(self.context, "heartbeat",
instance_id=instance_id,
sent=sent,
payload=payload)
def update_backup(self, instance_id, backup_id, sent=None,
**backup_fields):
LOG.debug("Making async call to cast update_backup for instance: %s"
% instance_id)
self.cast(self.context, self.make_msg("update_backup",
instance_id=instance_id,
backup_id=backup_id,
sent=sent,
**backup_fields))
cctxt = self.client.prepare(version=self.version_cap)
cctxt.cast(self.context, "update_backup",
instance_id=instance_id,
backup_id=backup_id,
sent=sent,
**backup_fields)
def report_root(self, instance_id, user):
LOG.debug("Making async call to cast report_root for instance: %s"
% instance_id)
self.cast(self.context, self.make_msg("report_root",
instance_id=instance_id,
user=user))
cctxt = self.client.prepare(version=self.version_cap)
cctxt.cast(self.context, "report_root",
instance_id=instance_id,
user=user)

View File

@ -12,10 +12,13 @@
# License for the specific language governing permissions and limitations
# under the License.
from oslo import messaging
from trove.backup import models as bkup_models
from trove.common import cfg
from trove.common import exception
from trove.common.instance import ServiceStatus
from trove.common.rpc import version as rpc_version
from trove.conductor.models import LastSeen
from trove.extensions.mysql import models as mysql_models
from trove.instance import models as t_models
@ -24,12 +27,13 @@ from trove.openstack.common import periodic_task
from trove.common.i18n import _
LOG = logging.getLogger(__name__)
RPC_API_VERSION = "1.0"
CONF = cfg.CONF
class Manager(periodic_task.PeriodicTasks):
target = messaging.Target(version=rpc_version.RPC_API_VERSION)
def __init__(self):
super(Manager, self).__init__()

View File

@ -17,11 +17,11 @@ from trove.common import cfg
from trove.common import remote
from trove.common import utils
from trove.openstack.common import log as logging
from trove.openstack.common.notifier import api as notifier
from trove.instance import models as imodels
from trove.instance.models import load_instance, InstanceServiceStatus
from trove.instance import models as instance_models
from trove.extensions.mysql import models as mysql_models
from trove import rpc
LOG = logging.getLogger(__name__)
CONF = cfg.CONF
@ -166,15 +166,12 @@ def _load_servers(instances, find_server):
def publish_exist_events(transformer, admin_context):
notifier = rpc.get_notifier("taskmanager")
notifications = transformer()
# clear out admin_context.auth_token so it does not get logged
admin_context.auth_token = None
for notification in notifications:
notifier.notify(admin_context,
CONF.host,
"trove.instance.exists",
'INFO',
notification)
notifier.info(admin_context, "trove.instance.exists", notification)
class NotificationTransformer(object):

View File

@ -18,43 +18,52 @@ Handles all request to the Platform or Guest VM
"""
from eventlet import Timeout
from oslo.messaging.rpc.client import RemoteError
from oslo import messaging
from trove.common import cfg
from trove.common import exception
from trove.common import rpc as rd_rpc
from trove.openstack.common import rpc
from trove.openstack.common import log as logging
from trove.openstack.common.rpc import proxy
from trove.openstack.common.rpc import common
from trove.common.i18n import _
import trove.common.rpc.version as rpc_version
from trove.openstack.common import log as logging
from trove import rpc
CONF = cfg.CONF
LOG = logging.getLogger(__name__)
AGENT_LOW_TIMEOUT = CONF.agent_call_low_timeout
AGENT_HIGH_TIMEOUT = CONF.agent_call_high_timeout
AGENT_SNAPSHOT_TIMEOUT = CONF.agent_replication_snapshot_timeout
RPC_API_VERSION = "1.0"
class API(proxy.RpcProxy):
class API(object):
"""API for interacting with the guest manager."""
def __init__(self, context, id):
self.context = context
self.id = id
super(API, self).__init__(self._get_routing_key(),
RPC_API_VERSION)
super(API, self).__init__()
def _call(self, method_name, timeout_sec, **kwargs):
LOG.debug("Calling %s with timeout %s." % (method_name, timeout_sec))
target = messaging.Target(topic=self._get_routing_key(),
version=rpc_version.RPC_API_VERSION)
self.version_cap = rpc_version.VERSION_ALIASES.get(
CONF.upgrade_levels.guestagent)
self.client = self.get_client(target, self.version_cap)
def get_client(self, target, version_cap, serializer=None):
return rpc.get_client(target,
version_cap=version_cap,
serializer=serializer)
def _call(self, method_name, timeout_sec, version, **kwargs):
LOG.debug("Calling %s with timeout %s" % (method_name, timeout_sec))
try:
result = self.call(self.context,
self.make_msg(method_name, **kwargs),
timeout=timeout_sec)
cctxt = self.client.prepare(version=version, timeout=timeout_sec)
result = cctxt.call(self.context, method_name, **kwargs)
LOG.debug("Result is %s." % result)
return result
except common.RemoteError as r:
except RemoteError as r:
LOG.exception(_("Error calling %s") % method_name)
raise exception.GuestError(original_message=r.value)
except Exception as e:
@ -63,43 +72,18 @@ class API(proxy.RpcProxy):
except Timeout:
raise exception.GuestTimeout()
def _cast(self, method_name, **kwargs):
LOG.debug("Casting %s." % method_name)
def _cast(self, method_name, version, **kwargs):
LOG.debug("Casting %s" % method_name)
try:
self.cast(self.context, self.make_msg(method_name, **kwargs),
topic=kwargs.get('topic'),
version=kwargs.get('version'))
except common.RemoteError as r:
cctxt = self.client.prepare(version=version)
cctxt.cast(self.context, method_name, **kwargs)
except RemoteError as r:
LOG.exception(_("Error calling %s") % method_name)
raise exception.GuestError(original_message=r.value)
except Exception as e:
LOG.exception(_("Error calling %s") % method_name)
raise exception.GuestError(original_message=str(e))
def _cast_with_consumer(self, method_name, **kwargs):
conn = None
try:
conn = rpc.create_connection(new=True)
conn.create_consumer(self._get_routing_key(), None, fanout=False)
except common.RemoteError as r:
LOG.exception(_("Error calling %s") % method_name)
raise exception.GuestError(original_message=r.value)
except Exception as e:
LOG.exception(_("Error calling %s") % method_name)
raise exception.GuestError(original_message=str(e))
finally:
if conn:
conn.close()
# leave the cast call out of the hackity consumer create
self._cast(method_name, **kwargs)
def delete_queue(self):
"""Deletes the queue."""
topic = self._get_routing_key()
LOG.debug("Deleting queue with name %s." % topic)
rd_rpc.delete_queue(self.context, topic)
def _get_routing_key(self):
"""Create the routing key based on the container id."""
return "guestagent.%s" % self.id
@ -109,31 +93,31 @@ class API(proxy.RpcProxy):
users.
"""
LOG.debug("Changing passwords for users on instance %s.", self.id)
self._cast("change_passwords", users=users)
self._cast("change_passwords", self.version_cap, users=users)
def update_attributes(self, username, hostname, user_attrs):
"""Update user attributes."""
LOG.debug("Changing user attributes on instance %s.", self.id)
self._cast("update_attributes", username=username, hostname=hostname,
user_attrs=user_attrs)
self._cast("update_attributes", self.version_cap, username=username,
hostname=hostname, user_attrs=user_attrs)
def create_user(self, users):
"""Make an asynchronous call to create a new database user"""
LOG.debug("Creating Users for instance %s.", self.id)
self._cast("create_user", users=users)
self._cast("create_user", self.version_cap, users=users)
def get_user(self, username, hostname):
"""Make an asynchronous call to get a single database user."""
LOG.debug("Getting a user %(username)s on instance %(id)s.",
{'username': username, 'id': self.id})
return self._call("get_user", AGENT_LOW_TIMEOUT,
return self._call("get_user", AGENT_LOW_TIMEOUT, self.version_cap,
username=username, hostname=hostname)
def list_access(self, username, hostname):
"""Show all the databases to which a user has more than USAGE."""
LOG.debug("Showing user %(username)s grants on instance %(id)s.",
{'username': username, 'id': self.id})
return self._call("list_access", AGENT_LOW_TIMEOUT,
return self._call("list_access", AGENT_LOW_TIMEOUT, self.version_cap,
username=username, hostname=hostname)
def grant_access(self, username, hostname, databases):
@ -142,7 +126,7 @@ class API(proxy.RpcProxy):
"%(username)s on instance %(id)s.", {'username': username,
'databases': databases,
'id': self.id})
return self._call("grant_access", AGENT_LOW_TIMEOUT,
return self._call("grant_access", AGENT_LOW_TIMEOUT, self.version_cap,
username=username, hostname=hostname,
databases=databases)
@ -152,34 +136,36 @@ class API(proxy.RpcProxy):
"%(username)s on instance %(id)s.", {'username': username,
'database': database,
'id': self.id})
return self._call("revoke_access", AGENT_LOW_TIMEOUT,
return self._call("revoke_access", AGENT_LOW_TIMEOUT, self.version_cap,
username=username, hostname=hostname,
database=database)
def list_users(self, limit=None, marker=None, include_marker=False):
"""Make an asynchronous call to list database users."""
LOG.debug("Listing Users for instance %s.", self.id)
return self._call("list_users", AGENT_HIGH_TIMEOUT, limit=limit,
marker=marker, include_marker=include_marker)
return self._call("list_users", AGENT_HIGH_TIMEOUT, self.version_cap,
limit=limit, marker=marker,
include_marker=include_marker)
def delete_user(self, user):
"""Make an asynchronous call to delete an existing database user."""
LOG.debug("Deleting user %(user)s for instance %(instance_id)s." %
{'user': user, 'instance_id': self.id})
self._cast("delete_user", user=user)
self._cast("delete_user", self.version_cap, user=user)
def create_database(self, databases):
"""Make an asynchronous call to create a new database
within the specified container
"""
LOG.debug("Creating databases for instance %s.", self.id)
self._cast("create_database", databases=databases)
self._cast("create_database", self.version_cap, databases=databases)
def list_databases(self, limit=None, marker=None, include_marker=False):
"""Make an asynchronous call to list databases."""
LOG.debug("Listing databases for instance %s.", self.id)
return self._call("list_databases", AGENT_LOW_TIMEOUT, limit=limit,
marker=marker, include_marker=include_marker)
return self._call("list_databases", AGENT_LOW_TIMEOUT,
self.version_cap, limit=limit, marker=marker,
include_marker=include_marker)
def delete_database(self, database):
"""Make an asynchronous call to delete an existing database
@ -188,38 +174,40 @@ class API(proxy.RpcProxy):
LOG.debug("Deleting database %(database)s for "
"instance %(instance_id)s." % {'database': database,
'instance_id': self.id})
self._cast("delete_database", database=database)
self._cast("delete_database", self.version_cap, database=database)
def enable_root(self):
"""Make a synchronous call to enable the root user for
access from anywhere
"""
LOG.debug("Enable root user for instance %s.", self.id)
return self._call("enable_root", AGENT_HIGH_TIMEOUT)
return self._call("enable_root", AGENT_HIGH_TIMEOUT, self.version_cap)
def disable_root(self):
"""Make a synchronous call to disable the root user for
access from anywhere
"""
LOG.debug("Disable root user for instance %s.", self.id)
return self._call("disable_root", AGENT_LOW_TIMEOUT)
return self._call("disable_root", AGENT_LOW_TIMEOUT, self.version_cap)
def is_root_enabled(self):
"""Make a synchronous call to check if root access is
available for the container
"""
LOG.debug("Check root access for instance %s.", self.id)
return self._call("is_root_enabled", AGENT_LOW_TIMEOUT)
return self._call("is_root_enabled", AGENT_LOW_TIMEOUT,
self.version_cap)
def get_hwinfo(self):
"""Make a synchronous call to get hardware info for the container"""
LOG.debug("Check hwinfo on instance %s.", self.id)
return self._call("get_hwinfo", AGENT_LOW_TIMEOUT)
return self._call("get_hwinfo", AGENT_LOW_TIMEOUT, self.version_cap)
def get_diagnostics(self):
"""Make a synchronous call to get diagnostics for the container"""
LOG.debug("Check diagnostics on instance %s.", self.id)
return self._call("get_diagnostics", AGENT_LOW_TIMEOUT)
return self._call("get_diagnostics", AGENT_LOW_TIMEOUT,
self.version_cap)
def prepare(self, memory_mb, packages, databases, users,
device_path='/dev/vdb', mount_point='/mnt/volume',
@ -229,25 +217,50 @@ class API(proxy.RpcProxy):
as a database container optionally includes a backup id for restores
"""
LOG.debug("Sending the call to prepare the Guest.")
# Taskmanager is a publisher, guestagent is a consumer. Usually
# consumer creates a queue, but in this case we have to make sure
# "prepare" doesn't get lost if for some reason guest was delayed and
# didn't create a queue on time.
self._create_guest_queue()
packages = packages.split()
self._cast_with_consumer(
"prepare", packages=packages, databases=databases,
memory_mb=memory_mb, users=users, device_path=device_path,
mount_point=mount_point, backup_info=backup_info,
config_contents=config_contents, root_password=root_password,
overrides=overrides, cluster_config=cluster_config)
self._cast(
"prepare", self.version_cap, packages=packages,
databases=databases, memory_mb=memory_mb, users=users,
device_path=device_path, mount_point=mount_point,
backup_info=backup_info, config_contents=config_contents,
root_password=root_password, overrides=overrides,
cluster_config=cluster_config)
def _create_guest_queue(self):
"""Call to construct, start and immediately stop rpc server in order
to create a queue to communicate with the guestagent. This is
method do nothing in case a queue is already created by
the guest
"""
server = None
target = messaging.Target(topic=self._get_routing_key(),
server=self.id,
version=rpc_version.RPC_API_VERSION)
try:
server = rpc.get_server(target, [])
server.start()
finally:
if server is not None:
server.stop()
def restart(self):
"""Restart the MySQL server."""
LOG.debug("Sending the call to restart MySQL on the Guest.")
self._call("restart", AGENT_HIGH_TIMEOUT)
self._call("restart", AGENT_HIGH_TIMEOUT, self.version_cap)
def start_db_with_conf_changes(self, config_contents):
"""Start the MySQL server."""
LOG.debug("Sending the call to start MySQL on the Guest with "
"a timeout of %s." % AGENT_HIGH_TIMEOUT)
self._call("start_db_with_conf_changes", AGENT_HIGH_TIMEOUT,
config_contents=config_contents)
self.version_cap, config_contents=config_contents)
def reset_configuration(self, configuration):
"""Ignore running state of MySQL, and just change the config file
@ -256,18 +269,18 @@ class API(proxy.RpcProxy):
LOG.debug("Sending the call to change MySQL conf file on the Guest "
"with a timeout of %s." % AGENT_HIGH_TIMEOUT)
self._call("reset_configuration", AGENT_HIGH_TIMEOUT,
configuration=configuration)
self.version_cap, configuration=configuration)
def stop_db(self, do_not_start_on_reboot=False):
"""Stop the MySQL server."""
LOG.debug("Sending the call to stop MySQL on the Guest.")
self._call("stop_db", AGENT_HIGH_TIMEOUT,
self._call("stop_db", AGENT_HIGH_TIMEOUT, self.version_cap,
do_not_start_on_reboot=do_not_start_on_reboot)
def upgrade(self, instance_version, location, metadata=None):
"""Make an asynchronous call to self upgrade the guest agent."""
LOG.debug("Sending an upgrade call to nova-guest.")
self._cast("upgrade",
self._cast("upgrade", self.version_cap,
instance_version=instance_version,
location=location,
metadata=metadata)
@ -276,74 +289,77 @@ class API(proxy.RpcProxy):
"""Make a synchronous call to get volume info for the container."""
LOG.debug("Check Volume Info on instance %s.", self.id)
return self._call("get_filesystem_stats", AGENT_LOW_TIMEOUT,
fs_path=None)
self.version_cap, fs_path=None)
def update_guest(self):
"""Make a synchronous call to update the guest agent."""
LOG.debug("Updating guest agent on instance %s.", self.id)
self._call("update_guest", AGENT_HIGH_TIMEOUT)
self._call("update_guest", AGENT_HIGH_TIMEOUT, self.version_cap)
def create_backup(self, backup_info):
"""Make async call to create a full backup of this instance."""
LOG.debug("Create Backup %(backup_id)s "
"for instance %(instance_id)s." %
{'backup_id': backup_info['id'], 'instance_id': self.id})
self._cast("create_backup", backup_info=backup_info)
self._cast("create_backup", self.version_cap, backup_info=backup_info)
def mount_volume(self, device_path=None, mount_point=None):
"""Mount the volume."""
LOG.debug("Mount volume %(mount)s on instance %(id)s." % {
'mount': mount_point, 'id': self.id})
self._call("mount_volume", AGENT_LOW_TIMEOUT,
self._call("mount_volume", AGENT_LOW_TIMEOUT, self.version_cap,
device_path=device_path, mount_point=mount_point)
def unmount_volume(self, device_path=None, mount_point=None):
"""Unmount the volume."""
LOG.debug("Unmount volume %(device)s on instance %(id)s." % {
'device': device_path, 'id': self.id})
self._call("unmount_volume", AGENT_LOW_TIMEOUT,
self._call("unmount_volume", AGENT_LOW_TIMEOUT, self.version_cap,
device_path=device_path, mount_point=mount_point)
def resize_fs(self, device_path=None, mount_point=None):
"""Resize the filesystem."""
LOG.debug("Resize device %(device)s on instance %(id)s." % {
'device': device_path, 'id': self.id})
self._call("resize_fs", AGENT_HIGH_TIMEOUT, device_path=device_path,
mount_point=mount_point)
self._call("resize_fs", AGENT_HIGH_TIMEOUT, self.version_cap,
device_path=device_path, mount_point=mount_point)
def update_overrides(self, overrides, remove=False):
"""Update the overrides."""
LOG.debug("Updating overrides values %(overrides)s on instance "
"%(id)s.", {'overrides': overrides, 'id': self.id})
self._cast("update_overrides", overrides=overrides, remove=remove)
self._cast("update_overrides", self.version_cap, overrides=overrides,
remove=remove)
def apply_overrides(self, overrides):
LOG.debug("Applying overrides values %(overrides)s on instance "
"%(id)s.", {'overrides': overrides, 'id': self.id})
self._cast("apply_overrides", overrides=overrides)
self._cast("apply_overrides", self.version_cap, overrides=overrides)
def get_replication_snapshot(self, snapshot_info=None,
replica_source_config=None):
LOG.debug("Retrieving replication snapshot from instance %s.", self.id)
return self._call("get_replication_snapshot", AGENT_SNAPSHOT_TIMEOUT,
snapshot_info=snapshot_info,
self.version_cap, snapshot_info=snapshot_info,
replica_source_config=replica_source_config)
def attach_replication_slave(self, snapshot, replica_config=None):
LOG.debug("Configuring instance %s to replicate from %s.",
self.id, snapshot.get('master').get('id'))
self._cast("attach_replication_slave", snapshot=snapshot,
slave_config=replica_config)
self._cast("attach_replication_slave", self.version_cap,
snapshot=snapshot, slave_config=replica_config)
def detach_replica(self):
LOG.debug("Detaching replica %s from its replication source.", self.id)
return self._call("detach_replica", AGENT_HIGH_TIMEOUT)
return self._call("detach_replica", AGENT_HIGH_TIMEOUT,
self.version_cap)
def cleanup_source_on_replica_detach(self, replica_info):
LOG.debug("Cleaning up master %s on detach of replica.", self.id)
self._call("cleanup_source_on_replica_detach", AGENT_HIGH_TIMEOUT,
replica_info=replica_info)
self.version_cap, replica_info=replica_info)
def demote_replication_master(self):
LOG.debug("Demoting instance %s to non-master.", self.id)
self._call("demote_replication_master", AGENT_LOW_TIMEOUT)
self._call("demote_replication_master", AGENT_LOW_TIMEOUT,
self.version_cap)

View File

@ -577,12 +577,6 @@ class BaseInstance(SimpleInstance):
deleted_at = datetime.utcnow()
self._delete_resources(deleted_at)
LOG.debug("Setting instance %s to be deleted.", self.id)
# Delete guest queue.
try:
guest = self.get_guest()
guest.delete_queue()
except Exception as ex:
LOG.warn(ex)
self.update_db(deleted=True, deleted_at=deleted_at,
task_status=InstanceTasks.NONE)
self.set_servicestatus_deleted()

View File

@ -1,14 +0,0 @@
# Copyright 2011 OpenStack Foundation.
# 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.

View File

@ -1,182 +0,0 @@
# Copyright 2011 OpenStack Foundation.
# 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 uuid
from oslo.config import cfg
from trove.openstack.common import context
from trove.openstack.common.gettextutils import _
from trove.openstack.common import importutils
from trove.openstack.common import jsonutils
from trove.openstack.common import log as logging
from trove.openstack.common import timeutils
LOG = logging.getLogger(__name__)
notifier_opts = [
cfg.MultiStrOpt('notification_driver',
default=[],
help='Driver or drivers to handle sending notifications'),
cfg.StrOpt('default_notification_level',
default='INFO',
help='Default notification level for outgoing notifications'),
cfg.StrOpt('default_publisher_id',
default='$host',
help='Default publisher_id for outgoing notifications'),
]
CONF = cfg.CONF
CONF.register_opts(notifier_opts)
WARN = 'WARN'
INFO = 'INFO'
ERROR = 'ERROR'
CRITICAL = 'CRITICAL'
DEBUG = 'DEBUG'
log_levels = (DEBUG, WARN, INFO, ERROR, CRITICAL)
class BadPriorityException(Exception):
pass
def notify_decorator(name, fn):
""" decorator for notify which is used from utils.monkey_patch()
:param name: name of the function
:param function: - object of the function
:returns: function -- decorated function
"""
def wrapped_func(*args, **kwarg):
body = {}
body['args'] = []
body['kwarg'] = {}
for arg in args:
body['args'].append(arg)
for key in kwarg:
body['kwarg'][key] = kwarg[key]
ctxt = context.get_context_from_function_and_args(fn, args, kwarg)
notify(ctxt,
CONF.default_publisher_id,
name,
CONF.default_notification_level,
body)
return fn(*args, **kwarg)
return wrapped_func
def publisher_id(service, host=None):
if not host:
host = CONF.host
return "%s.%s" % (service, host)
def notify(context, publisher_id, event_type, priority, payload):
"""Sends a notification using the specified driver
:param publisher_id: the source worker_type.host of the message
:param event_type: the literal type of event (ex. Instance Creation)
:param priority: patterned after the enumeration of Python logging
levels in the set (DEBUG, WARN, INFO, ERROR, CRITICAL)
:param payload: A python dictionary of attributes
Outgoing message format includes the above parameters, and appends the
following:
message_id
a UUID representing the id for this notification
timestamp
the GMT timestamp the notification was sent at
The composite message will be constructed as a dictionary of the above
attributes, which will then be sent via the transport mechanism defined
by the driver.
Message example::
{'message_id': str(uuid.uuid4()),
'publisher_id': 'compute.host1',
'timestamp': timeutils.utcnow(),
'priority': 'WARN',
'event_type': 'compute.create_instance',
'payload': {'instance_id': 12, ... }}
"""
if priority not in log_levels:
raise BadPriorityException(
_('%s not in valid priorities') % priority)
# Ensure everything is JSON serializable.
payload = jsonutils.to_primitive(payload, convert_instances=True)
msg = dict(message_id=str(uuid.uuid4()),
publisher_id=publisher_id,
event_type=event_type,
priority=priority,
payload=payload,
timestamp=str(timeutils.utcnow()))
for driver in _get_drivers():
try:
driver.notify(context, msg)
except Exception as e:
LOG.exception(_("Problem '%(e)s' attempting to "
"send to notification system. "
"Payload=%(payload)s")
% dict(e=e, payload=payload))
_drivers = None
def _get_drivers():
"""Instantiate, cache, and return drivers based on the CONF."""
global _drivers
if _drivers is None:
_drivers = {}
for notification_driver in CONF.notification_driver:
add_driver(notification_driver)
return _drivers.values()
def add_driver(notification_driver):
"""Add a notification driver at runtime."""
# Make sure the driver list is initialized.
_get_drivers()
if isinstance(notification_driver, basestring):
# Load and add
try:
driver = importutils.import_module(notification_driver)
_drivers[notification_driver] = driver
except ImportError:
LOG.exception(_("Failed to load notifier %s. "
"These notifications will not be sent.") %
notification_driver)
else:
# Driver is already loaded; just add the object.
_drivers[notification_driver] = notification_driver
def _reset_drivers():
"""Used by unit tests to reset the drivers."""
global _drivers
_drivers = None

View File

@ -1,35 +0,0 @@
# Copyright 2011 OpenStack Foundation.
# 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.
from oslo.config import cfg
from trove.openstack.common import jsonutils
from trove.openstack.common import log as logging
CONF = cfg.CONF
def notify(_context, message):
"""Notifies the recipient of the desired event given the model.
Log notifications using openstack's default logging system"""
priority = message.get('priority',
CONF.default_notification_level)
priority = priority.lower()
logger = logging.getLogger(
'trove.openstack.common.notification.%s' %
message['event_type'])
getattr(logger, priority)(jsonutils.dumps(message))

View File

@ -1,19 +0,0 @@
# Copyright 2011 OpenStack Foundation.
# 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.
def notify(_context, message):
"""Notifies the recipient of the desired event given the model"""
pass

View File

@ -1,29 +0,0 @@
# Copyright 2012 Red Hat, Inc.
# 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.
from trove.openstack.common.gettextutils import _
from trove.openstack.common import log as logging
from trove.openstack.common.notifier import rpc_notifier
LOG = logging.getLogger(__name__)
def notify(context, message):
"""Deprecated in Grizzly. Please use rpc_notifier instead."""
LOG.deprecated(_("The rabbit_notifier is now deprecated."
" Please use rpc_notifier instead."))
rpc_notifier.notify(context, message)

View File

@ -1,46 +0,0 @@
# Copyright 2011 OpenStack Foundation.
# 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.
from oslo.config import cfg
from trove.openstack.common import context as req_context
from trove.openstack.common.gettextutils import _
from trove.openstack.common import log as logging
from trove.openstack.common import rpc
LOG = logging.getLogger(__name__)
notification_topic_opt = cfg.ListOpt(
'notification_topics', default=['notifications', ],
help='AMQP topic used for openstack notifications')
CONF = cfg.CONF
CONF.register_opt(notification_topic_opt)
def notify(context, message):
"""Sends a notification via RPC"""
if not context:
context = req_context.get_admin_context()
priority = message.get('priority',
CONF.default_notification_level)
priority = priority.lower()
for topic in CONF.notification_topics:
topic = '%s.%s' % (topic, priority)
try:
rpc.notify(context, topic, message)
except Exception:
LOG.exception(_("Could not send notification to %(topic)s. "
"Payload=%(message)s"), locals())

View File

@ -1,52 +0,0 @@
# Copyright 2011 OpenStack Foundation.
# 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.
'''messaging based notification driver, with message envelopes'''
from oslo.config import cfg
from trove.openstack.common import context as req_context
from trove.openstack.common.gettextutils import _
from trove.openstack.common import log as logging
from trove.openstack.common import rpc
LOG = logging.getLogger(__name__)
notification_topic_opt = cfg.ListOpt(
'topics', default=['notifications', ],
help='AMQP topic(s) used for openstack notifications')
opt_group = cfg.OptGroup(name='rpc_notifier2',
title='Options for rpc_notifier2')
CONF = cfg.CONF
CONF.register_group(opt_group)
CONF.register_opt(notification_topic_opt, opt_group)
def notify(context, message):
"""Sends a notification via RPC"""
if not context:
context = req_context.get_admin_context()
priority = message.get('priority',
CONF.default_notification_level)
priority = priority.lower()
for topic in CONF.rpc_notifier2.topics:
topic = '%s.%s' % (topic, priority)
try:
rpc.notify(context, topic, message, envelope=True)
except Exception:
LOG.exception(_("Could not send notification to %(topic)s. "
"Payload=%(message)s"), locals())

View File

@ -1,306 +0,0 @@
# vim: tabstop=4 shiftwidth=4 softtabstop=4
# Copyright 2010 United States Government as represented by the
# Administrator of the National Aeronautics and Space Administration.
# All Rights Reserved.
# Copyright 2011 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.
"""
A remote procedure call (rpc) abstraction.
For some wrappers that add message versioning to rpc, see:
rpc.dispatcher
rpc.proxy
"""
import inspect
from oslo.config import cfg
from trove.openstack.common.gettextutils import _ # noqa
from trove.openstack.common import importutils
from trove.openstack.common import local
from trove.openstack.common import log as logging
LOG = logging.getLogger(__name__)
rpc_opts = [
cfg.StrOpt('rpc_backend',
default='%s.impl_kombu' % __package__,
help="The messaging module to use, defaults to kombu."),
cfg.IntOpt('rpc_thread_pool_size',
default=64,
help='Size of RPC thread pool'),
cfg.IntOpt('rpc_conn_pool_size',
default=30,
help='Size of RPC connection pool'),
cfg.IntOpt('rpc_response_timeout',
default=60,
help='Seconds to wait for a response from call or multicall'),
cfg.IntOpt('rpc_cast_timeout',
default=30,
help='Seconds to wait before a cast expires (TTL). '
'Only supported by impl_zmq.'),
cfg.ListOpt('allowed_rpc_exception_modules',
default=['nova.exception',
'cinder.exception',
'exceptions',
],
help='Modules of exceptions that are permitted to be recreated'
'upon receiving exception data from an rpc call.'),
cfg.BoolOpt('fake_rabbit',
default=False,
help='If passed, use a fake RabbitMQ provider'),
cfg.StrOpt('control_exchange',
default='openstack',
help='AMQP exchange to connect to if using RabbitMQ or Qpid'),
]
CONF = cfg.CONF
CONF.register_opts(rpc_opts)
def set_defaults(control_exchange):
cfg.set_defaults(rpc_opts,
control_exchange=control_exchange)
def create_connection(new=True):
"""Create a connection to the message bus used for rpc.
For some example usage of creating a connection and some consumers on that
connection, see nova.service.
:param new: Whether or not to create a new connection. A new connection
will be created by default. If new is False, the
implementation is free to return an existing connection from a
pool.
:returns: An instance of openstack.common.rpc.common.Connection
"""
return _get_impl().create_connection(CONF, new=new)
def _check_for_lock():
if not CONF.debug:
return None
if ((hasattr(local.strong_store, 'locks_held')
and local.strong_store.locks_held)):
stack = ' :: '.join([frame[3] for frame in inspect.stack()])
LOG.warn(_('A RPC is being made while holding a lock. The locks '
'currently held are %(locks)s. This is probably a bug. '
'Please report it. Include the following: [%(stack)s].'),
{'locks': local.strong_store.locks_held,
'stack': stack})
return True
return False
def call(context, topic, msg, timeout=None, check_for_lock=False):
"""Invoke a remote method that returns something.
:param context: Information that identifies the user that has made this
request.
:param topic: The topic to send the rpc message to. This correlates to the
topic argument of
openstack.common.rpc.common.Connection.create_consumer()
and only applies when the consumer was created with
fanout=False.
:param msg: This is a dict in the form { "method" : "method_to_invoke",
"args" : dict_of_kwargs }
:param timeout: int, number of seconds to use for a response timeout.
If set, this overrides the rpc_response_timeout option.
:param check_for_lock: if True, a warning is emitted if a RPC call is made
with a lock held.
:returns: A dict from the remote method.
:raises: openstack.common.rpc.common.Timeout if a complete response
is not received before the timeout is reached.
"""
if check_for_lock:
_check_for_lock()
return _get_impl().call(CONF, context, topic, msg, timeout)
def cast(context, topic, msg):
"""Invoke a remote method that does not return anything.
:param context: Information that identifies the user that has made this
request.
:param topic: The topic to send the rpc message to. This correlates to the
topic argument of
openstack.common.rpc.common.Connection.create_consumer()
and only applies when the consumer was created with
fanout=False.
:param msg: This is a dict in the form { "method" : "method_to_invoke",
"args" : dict_of_kwargs }
:returns: None
"""
return _get_impl().cast(CONF, context, topic, msg)
def fanout_cast(context, topic, msg):
"""Broadcast a remote method invocation with no return.
This method will get invoked on all consumers that were set up with this
topic name and fanout=True.
:param context: Information that identifies the user that has made this
request.
:param topic: The topic to send the rpc message to. This correlates to the
topic argument of
openstack.common.rpc.common.Connection.create_consumer()
and only applies when the consumer was created with
fanout=True.
:param msg: This is a dict in the form { "method" : "method_to_invoke",
"args" : dict_of_kwargs }
:returns: None
"""
return _get_impl().fanout_cast(CONF, context, topic, msg)
def multicall(context, topic, msg, timeout=None, check_for_lock=False):
"""Invoke a remote method and get back an iterator.
In this case, the remote method will be returning multiple values in
separate messages, so the return values can be processed as the come in via
an iterator.
:param context: Information that identifies the user that has made this
request.
:param topic: The topic to send the rpc message to. This correlates to the
topic argument of
openstack.common.rpc.common.Connection.create_consumer()
and only applies when the consumer was created with
fanout=False.
:param msg: This is a dict in the form { "method" : "method_to_invoke",
"args" : dict_of_kwargs }
:param timeout: int, number of seconds to use for a response timeout.
If set, this overrides the rpc_response_timeout option.
:param check_for_lock: if True, a warning is emitted if a RPC call is made
with a lock held.
:returns: An iterator. The iterator will yield a tuple (N, X) where N is
an index that starts at 0 and increases by one for each value
returned and X is the Nth value that was returned by the remote
method.
:raises: openstack.common.rpc.common.Timeout if a complete response
is not received before the timeout is reached.
"""
if check_for_lock:
_check_for_lock()
return _get_impl().multicall(CONF, context, topic, msg, timeout)
def notify(context, topic, msg, envelope=False):
"""Send notification event.
:param context: Information that identifies the user that has made this
request.
:param topic: The topic to send the notification to.
:param msg: This is a dict of content of event.
:param envelope: Set to True to enable message envelope for notifications.
:returns: None
"""
return _get_impl().notify(cfg.CONF, context, topic, msg, envelope)
def cleanup():
"""Clean up resoruces in use by implementation.
Clean up any resources that have been allocated by the RPC implementation.
This is typically open connections to a messaging service. This function
would get called before an application using this API exits to allow
connections to get torn down cleanly.
:returns: None
"""
return _get_impl().cleanup()
def cast_to_server(context, server_params, topic, msg):
"""Invoke a remote method that does not return anything.
:param context: Information that identifies the user that has made this
request.
:param server_params: Connection information
:param topic: The topic to send the notification to.
:param msg: This is a dict in the form { "method" : "method_to_invoke",
"args" : dict_of_kwargs }
:returns: None
"""
return _get_impl().cast_to_server(CONF, context, server_params, topic,
msg)
def fanout_cast_to_server(context, server_params, topic, msg):
"""Broadcast to a remote method invocation with no return.
:param context: Information that identifies the user that has made this
request.
:param server_params: Connection information
:param topic: The topic to send the notification to.
:param msg: This is a dict in the form { "method" : "method_to_invoke",
"args" : dict_of_kwargs }
:returns: None
"""
return _get_impl().fanout_cast_to_server(CONF, context, server_params,
topic, msg)
def queue_get_for(context, topic, host):
"""Get a queue name for a given topic + host.
This function only works if this naming convention is followed on the
consumer side, as well. For example, in nova, every instance of the
nova-foo service calls create_consumer() for two topics:
foo
foo.<host>
Messages sent to the 'foo' topic are distributed to exactly one instance of
the nova-foo service. The services are chosen in a round-robin fashion.
Messages sent to the 'foo.<host>' topic are sent to the nova-foo service on
<host>.
"""
return '%s.%s' % (topic, host) if host else topic
_RPCIMPL = None
def _get_impl():
"""Delay import of rpc_backend until configuration is loaded."""
global _RPCIMPL
if _RPCIMPL is None:
try:
_RPCIMPL = importutils.import_module(CONF.rpc_backend)
except ImportError:
# For backwards compatibility with older nova config.
impl = CONF.rpc_backend.replace('nova.rpc',
'nova.openstack.common.rpc')
_RPCIMPL = importutils.import_module(impl)
return _RPCIMPL

View File

@ -1,615 +0,0 @@
# vim: tabstop=4 shiftwidth=4 softtabstop=4
# Copyright 2010 United States Government as represented by the
# Administrator of the National Aeronautics and Space Administration.
# All Rights Reserved.
# Copyright 2011 - 2012, Red Hat, Inc.
#
# Licensed under the Apache License, Version 2.0 (the "License"); you may
# not use this file except in compliance with the License. You may obtain
# a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
# License for the specific language governing permissions and limitations
# under the License.
"""
Shared code between AMQP based openstack.common.rpc implementations.
The code in this module is shared between the rpc implemenations based on AMQP.
Specifically, this includes impl_kombu and impl_qpid. impl_carrot also uses
AMQP, but is deprecated and predates this code.
"""
import collections
import inspect
import sys
import uuid
from eventlet import greenpool
from eventlet import pools
from eventlet import queue
from eventlet import semaphore
from oslo.config import cfg
from trove.openstack.common import excutils
from trove.openstack.common.gettextutils import _ # noqa
from trove.openstack.common import local
from trove.openstack.common import log as logging
from trove.openstack.common.rpc import common as rpc_common
amqp_opts = [
cfg.BoolOpt('amqp_durable_queues',
default=False,
deprecated_name='rabbit_durable_queues',
deprecated_group='DEFAULT',
help='Use durable queues in amqp.'),
cfg.BoolOpt('amqp_auto_delete',
default=False,
help='Auto-delete queues in amqp.'),
]
cfg.CONF.register_opts(amqp_opts)
UNIQUE_ID = '_unique_id'
LOG = logging.getLogger(__name__)
class Pool(pools.Pool):
"""Class that implements a Pool of Connections."""
def __init__(self, conf, connection_cls, *args, **kwargs):
self.connection_cls = connection_cls
self.conf = conf
kwargs.setdefault("max_size", self.conf.rpc_conn_pool_size)
kwargs.setdefault("order_as_stack", True)
super(Pool, self).__init__(*args, **kwargs)
self.reply_proxy = None
# TODO(comstud): Timeout connections not used in a while
def create(self):
LOG.debug(_('Pool creating new connection'))
return self.connection_cls(self.conf)
def empty(self):
while self.free_items:
self.get().close()
# Force a new connection pool to be created.
# Note that this was added due to failing unit test cases. The issue
# is the above "while loop" gets all the cached connections from the
# pool and closes them, but never returns them to the pool, a pool
# leak. The unit tests hang waiting for an item to be returned to the
# pool. The unit tests get here via the tearDown() method. In the run
# time code, it gets here via cleanup() and only appears in service.py
# just before doing a sys.exit(), so cleanup() only happens once and
# the leakage is not a problem.
self.connection_cls.pool = None
_pool_create_sem = semaphore.Semaphore()
def get_connection_pool(conf, connection_cls):
with _pool_create_sem:
# Make sure only one thread tries to create the connection pool.
if not connection_cls.pool:
connection_cls.pool = Pool(conf, connection_cls)
return connection_cls.pool
class ConnectionContext(rpc_common.Connection):
"""The class that is actually returned to the create_connection() caller.
This is essentially a wrapper around Connection that supports 'with'.
It can also return a new Connection, or one from a pool.
The function will also catch when an instance of this class is to be
deleted. With that we can return Connections to the pool on exceptions
and so forth without making the caller be responsible for catching them.
If possible the function makes sure to return a connection to the pool.
"""
def __init__(self, conf, connection_pool, pooled=True, server_params=None):
"""Create a new connection, or get one from the pool."""
self.connection = None
self.conf = conf
self.connection_pool = connection_pool
if pooled:
self.connection = connection_pool.get()
else:
self.connection = connection_pool.connection_cls(
conf,
server_params=server_params)
self.pooled = pooled
def __enter__(self):
"""When with ConnectionContext() is used, return self."""
return self
def _done(self):
"""If the connection came from a pool, clean it up and put it back.
If it did not come from a pool, close it.
"""
if self.connection:
if self.pooled:
# Reset the connection so it's ready for the next caller
# to grab from the pool
self.connection.reset()
self.connection_pool.put(self.connection)
else:
try:
self.connection.close()
except Exception:
pass
self.connection = None
def __exit__(self, exc_type, exc_value, tb):
"""End of 'with' statement. We're done here."""
self._done()
def __del__(self):
"""Caller is done with this connection. Make sure we cleaned up."""
self._done()
def close(self):
"""Caller is done with this connection."""
self._done()
def create_consumer(self, topic, proxy, fanout=False):
self.connection.create_consumer(topic, proxy, fanout)
def create_worker(self, topic, proxy, pool_name):
self.connection.create_worker(topic, proxy, pool_name)
def join_consumer_pool(self, callback, pool_name, topic, exchange_name,
ack_on_error=True):
self.connection.join_consumer_pool(callback,
pool_name,
topic,
exchange_name,
ack_on_error)
def consume_in_thread(self):
self.connection.consume_in_thread()
def __getattr__(self, key):
"""Proxy all other calls to the Connection instance."""
if self.connection:
return getattr(self.connection, key)
else:
raise rpc_common.InvalidRPCConnectionReuse()
class ReplyProxy(ConnectionContext):
"""Connection class for RPC replies / callbacks."""
def __init__(self, conf, connection_pool):
self._call_waiters = {}
self._num_call_waiters = 0
self._num_call_waiters_wrn_threshhold = 10
self._reply_q = 'reply_' + uuid.uuid4().hex
super(ReplyProxy, self).__init__(conf, connection_pool, pooled=False)
self.declare_direct_consumer(self._reply_q, self._process_data)
self.consume_in_thread()
def _process_data(self, message_data):
msg_id = message_data.pop('_msg_id', None)
waiter = self._call_waiters.get(msg_id)
if not waiter:
LOG.warn(_('No calling threads waiting for msg_id : %(msg_id)s'
', message : %(data)s'), {'msg_id': msg_id,
'data': message_data})
LOG.warn(_('_call_waiters: %s') % str(self._call_waiters))
else:
waiter.put(message_data)
def add_call_waiter(self, waiter, msg_id):
self._num_call_waiters += 1
if self._num_call_waiters > self._num_call_waiters_wrn_threshhold:
LOG.warn(_('Number of call waiters is greater than warning '
'threshhold: %d. There could be a MulticallProxyWaiter '
'leak.') % self._num_call_waiters_wrn_threshhold)
self._num_call_waiters_wrn_threshhold *= 2
self._call_waiters[msg_id] = waiter
def del_call_waiter(self, msg_id):
self._num_call_waiters -= 1
del self._call_waiters[msg_id]
def get_reply_q(self):
return self._reply_q
def msg_reply(conf, msg_id, reply_q, connection_pool, reply=None,
failure=None, ending=False, log_failure=True):
"""Sends a reply or an error on the channel signified by msg_id.
Failure should be a sys.exc_info() tuple.
"""
with ConnectionContext(conf, connection_pool) as conn:
if failure:
failure = rpc_common.serialize_remote_exception(failure,
log_failure)
msg = {'result': reply, 'failure': failure}
if ending:
msg['ending'] = True
_add_unique_id(msg)
# If a reply_q exists, add the msg_id to the reply and pass the
# reply_q to direct_send() to use it as the response queue.
# Otherwise use the msg_id for backward compatibilty.
if reply_q:
msg['_msg_id'] = msg_id
conn.direct_send(reply_q, rpc_common.serialize_msg(msg))
else:
conn.direct_send(msg_id, rpc_common.serialize_msg(msg))
class RpcContext(rpc_common.CommonRpcContext):
"""Context that supports replying to a rpc.call."""
def __init__(self, **kwargs):
self.msg_id = kwargs.pop('msg_id', None)
self.reply_q = kwargs.pop('reply_q', None)
self.conf = kwargs.pop('conf')
super(RpcContext, self).__init__(**kwargs)
def deepcopy(self):
values = self.to_dict()
values['conf'] = self.conf
values['msg_id'] = self.msg_id
values['reply_q'] = self.reply_q
return self.__class__(**values)
def reply(self, reply=None, failure=None, ending=False,
connection_pool=None, log_failure=True):
if self.msg_id:
msg_reply(self.conf, self.msg_id, self.reply_q, connection_pool,
reply, failure, ending, log_failure)
if ending:
self.msg_id = None
def unpack_context(conf, msg):
"""Unpack context from msg."""
context_dict = {}
for key in list(msg.keys()):
# NOTE(vish): Some versions of python don't like unicode keys
# in kwargs.
key = str(key)
if key.startswith('_context_'):
value = msg.pop(key)
context_dict[key[9:]] = value
context_dict['msg_id'] = msg.pop('_msg_id', None)
context_dict['reply_q'] = msg.pop('_reply_q', None)
context_dict['conf'] = conf
ctx = RpcContext.from_dict(context_dict)
rpc_common._safe_log(LOG.debug, _('unpacked context: %s'), ctx.to_dict())
return ctx
def pack_context(msg, context):
"""Pack context into msg.
Values for message keys need to be less than 255 chars, so we pull
context out into a bunch of separate keys. If we want to support
more arguments in rabbit messages, we may want to do the same
for args at some point.
"""
if isinstance(context, dict):
context_d = dict([('_context_%s' % key, value)
for (key, value) in context.iteritems()])
else:
context_d = dict([('_context_%s' % key, value)
for (key, value) in context.to_dict().iteritems()])
msg.update(context_d)
class _MsgIdCache(object):
"""This class checks any duplicate messages."""
# NOTE: This value is considered can be a configuration item, but
# it is not necessary to change its value in most cases,
# so let this value as static for now.
DUP_MSG_CHECK_SIZE = 16
def __init__(self, **kwargs):
self.prev_msgids = collections.deque([],
maxlen=self.DUP_MSG_CHECK_SIZE)
def check_duplicate_message(self, message_data):
"""AMQP consumers may read same message twice when exceptions occur
before ack is returned. This method prevents doing it.
"""
if UNIQUE_ID in message_data:
msg_id = message_data[UNIQUE_ID]
if msg_id not in self.prev_msgids:
self.prev_msgids.append(msg_id)
else:
raise rpc_common.DuplicateMessageError(msg_id=msg_id)
def _add_unique_id(msg):
"""Add unique_id for checking duplicate messages."""
unique_id = uuid.uuid4().hex
msg.update({UNIQUE_ID: unique_id})
LOG.debug(_('UNIQUE_ID is %s.') % (unique_id))
class _ThreadPoolWithWait(object):
"""Base class for a delayed invocation manager.
Used by the Connection class to start up green threads
to handle incoming messages.
"""
def __init__(self, conf, connection_pool):
self.pool = greenpool.GreenPool(conf.rpc_thread_pool_size)
self.connection_pool = connection_pool
self.conf = conf
def wait(self):
"""Wait for all callback threads to exit."""
self.pool.waitall()
class CallbackWrapper(_ThreadPoolWithWait):
"""Wraps a straight callback.
Allows it to be invoked in a green thread.
"""
def __init__(self, conf, callback, connection_pool):
"""Initiates CallbackWrapper object.
:param conf: cfg.CONF instance
:param callback: a callable (probably a function)
:param connection_pool: connection pool as returned by
get_connection_pool()
"""
super(CallbackWrapper, self).__init__(
conf=conf,
connection_pool=connection_pool,
)
self.callback = callback
def __call__(self, message_data):
self.pool.spawn_n(self.callback, message_data)
class ProxyCallback(_ThreadPoolWithWait):
"""Calls methods on a proxy object based on method and args."""
def __init__(self, conf, proxy, connection_pool):
super(ProxyCallback, self).__init__(
conf=conf,
connection_pool=connection_pool,
)
self.proxy = proxy
self.msg_id_cache = _MsgIdCache()
def __call__(self, message_data):
"""Consumer callback to call a method on a proxy object.
Parses the message for validity and fires off a thread to call the
proxy object method.
Message data should be a dictionary with two keys:
method: string representing the method to call
args: dictionary of arg: value
Example: {'method': 'echo', 'args': {'value': 42}}
"""
# It is important to clear the context here, because at this point
# the previous context is stored in local.store.context
if hasattr(local.store, 'context'):
del local.store.context
rpc_common._safe_log(LOG.debug, _('received %s'), message_data)
self.msg_id_cache.check_duplicate_message(message_data)
ctxt = unpack_context(self.conf, message_data)
method = message_data.get('method')
args = message_data.get('args', {})
version = message_data.get('version')
namespace = message_data.get('namespace')
if not method:
LOG.warn(_('no method for message: %s') % message_data)
ctxt.reply(_('No method for message: %s') % message_data,
connection_pool=self.connection_pool)
return
self.pool.spawn_n(self._process_data, ctxt, version, method,
namespace, args)
def _process_data(self, ctxt, version, method, namespace, args):
"""Process a message in a new thread.
If the proxy object we have has a dispatch method
(see rpc.dispatcher.RpcDispatcher), pass it the version,
method, and args and let it dispatch as appropriate. If not, use
the old behavior of magically calling the specified method on the
proxy we have here.
"""
ctxt.update_store()
try:
rval = self.proxy.dispatch(ctxt, version, method, namespace,
**args)
# Check if the result was a generator
if inspect.isgenerator(rval):
for x in rval:
ctxt.reply(x, None, connection_pool=self.connection_pool)
else:
ctxt.reply(rval, None, connection_pool=self.connection_pool)
# This final None tells multicall that it is done.
ctxt.reply(ending=True, connection_pool=self.connection_pool)
except rpc_common.ClientException as e:
LOG.debug(_('Expected exception during message handling (%s)') %
e._exc_info[1])
ctxt.reply(None, e._exc_info,
connection_pool=self.connection_pool,
log_failure=False)
except Exception:
# sys.exc_info() is deleted by LOG.exception().
exc_info = sys.exc_info()
LOG.error(_('Exception during message handling'),
exc_info=exc_info)
ctxt.reply(None, exc_info, connection_pool=self.connection_pool)
class MulticallProxyWaiter(object):
def __init__(self, conf, msg_id, timeout, connection_pool):
self._msg_id = msg_id
self._timeout = timeout or conf.rpc_response_timeout
self._reply_proxy = connection_pool.reply_proxy
self._done = False
self._got_ending = False
self._conf = conf
self._dataqueue = queue.LightQueue()
# Add this caller to the reply proxy's call_waiters
self._reply_proxy.add_call_waiter(self, self._msg_id)
self.msg_id_cache = _MsgIdCache()
def put(self, data):
self._dataqueue.put(data)
def done(self):
if self._done:
return
self._done = True
# Remove this caller from reply proxy's call_waiters
self._reply_proxy.del_call_waiter(self._msg_id)
def _process_data(self, data):
result = None
self.msg_id_cache.check_duplicate_message(data)
if data['failure']:
failure = data['failure']
result = rpc_common.deserialize_remote_exception(self._conf,
failure)
elif data.get('ending', False):
self._got_ending = True
else:
result = data['result']
return result
def __iter__(self):
"""Return a result until we get a reply with an 'ending' flag."""
if self._done:
raise StopIteration
while True:
try:
data = self._dataqueue.get(timeout=self._timeout)
result = self._process_data(data)
except queue.Empty:
self.done()
raise rpc_common.Timeout()
except Exception:
with excutils.save_and_reraise_exception():
self.done()
if self._got_ending:
self.done()
raise StopIteration
if isinstance(result, Exception):
self.done()
raise result
yield result
def create_connection(conf, new, connection_pool):
"""Create a connection."""
return ConnectionContext(conf, connection_pool, pooled=not new)
_reply_proxy_create_sem = semaphore.Semaphore()
def multicall(conf, context, topic, msg, timeout, connection_pool):
"""Make a call that returns multiple times."""
LOG.debug(_('Making synchronous call on %s ...'), topic)
msg_id = uuid.uuid4().hex
msg.update({'_msg_id': msg_id})
LOG.debug(_('MSG_ID is %s') % (msg_id))
_add_unique_id(msg)
pack_context(msg, context)
with _reply_proxy_create_sem:
if not connection_pool.reply_proxy:
connection_pool.reply_proxy = ReplyProxy(conf, connection_pool)
msg.update({'_reply_q': connection_pool.reply_proxy.get_reply_q()})
wait_msg = MulticallProxyWaiter(conf, msg_id, timeout, connection_pool)
with ConnectionContext(conf, connection_pool) as conn:
conn.topic_send(topic, rpc_common.serialize_msg(msg), timeout)
return wait_msg
def call(conf, context, topic, msg, timeout, connection_pool):
"""Sends a message on a topic and wait for a response."""
rv = multicall(conf, context, topic, msg, timeout, connection_pool)
# NOTE(vish): return the last result from the multicall
rv = list(rv)
if not rv:
return
return rv[-1]
def cast(conf, context, topic, msg, connection_pool):
"""Sends a message on a topic without waiting for a response."""
LOG.debug(_('Making asynchronous cast on %s...'), topic)
_add_unique_id(msg)
pack_context(msg, context)
with ConnectionContext(conf, connection_pool) as conn:
conn.topic_send(topic, rpc_common.serialize_msg(msg))
def fanout_cast(conf, context, topic, msg, connection_pool):
"""Sends a message on a fanout exchange without waiting for a response."""
LOG.debug(_('Making asynchronous fanout cast...'))
_add_unique_id(msg)
pack_context(msg, context)
with ConnectionContext(conf, connection_pool) as conn:
conn.fanout_send(topic, rpc_common.serialize_msg(msg))
def cast_to_server(conf, context, server_params, topic, msg, connection_pool):
"""Sends a message on a topic to a specific server."""
_add_unique_id(msg)
pack_context(msg, context)
with ConnectionContext(conf, connection_pool, pooled=False,
server_params=server_params) as conn:
conn.topic_send(topic, rpc_common.serialize_msg(msg))
def fanout_cast_to_server(conf, context, server_params, topic, msg,
connection_pool):
"""Sends a message on a fanout exchange to a specific server."""
_add_unique_id(msg)
pack_context(msg, context)
with ConnectionContext(conf, connection_pool, pooled=False,
server_params=server_params) as conn:
conn.fanout_send(topic, rpc_common.serialize_msg(msg))
def notify(conf, context, topic, msg, connection_pool, envelope):
"""Sends a notification event on a topic."""
LOG.debug(_('Sending %(event_type)s on %(topic)s'),
dict(event_type=msg.get('event_type'),
topic=topic))
_add_unique_id(msg)
pack_context(msg, context)
with ConnectionContext(conf, connection_pool) as conn:
if envelope:
msg = rpc_common.serialize_msg(msg)
conn.notify_send(topic, msg)
def cleanup(connection_pool):
if connection_pool:
connection_pool.empty()
def get_control_exchange(conf):
return conf.control_exchange

View File

@ -1,508 +0,0 @@
# Copyright 2010 United States Government as represented by the
# Administrator of the National Aeronautics and Space Administration.
# All Rights Reserved.
# Copyright 2011 Red Hat, Inc.
#
# Licensed under the Apache License, Version 2.0 (the "License"); you may
# not use this file except in compliance with the License. You may obtain
# a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
# License for the specific language governing permissions and limitations
# under the License.
import copy
import sys
import traceback
from oslo.config import cfg
import six
from trove.openstack.common.gettextutils import _
from trove.openstack.common import importutils
from trove.openstack.common import jsonutils
from trove.openstack.common import local
from trove.openstack.common import log as logging
from trove.openstack.common import versionutils
CONF = cfg.CONF
LOG = logging.getLogger(__name__)
_RPC_ENVELOPE_VERSION = '2.0'
'''RPC Envelope Version.
This version number applies to the top level structure of messages sent out.
It does *not* apply to the message payload, which must be versioned
independently. For example, when using rpc APIs, a version number is applied
for changes to the API being exposed over rpc. This version number is handled
in the rpc proxy and dispatcher modules.
This version number applies to the message envelope that is used in the
serialization done inside the rpc layer. See serialize_msg() and
deserialize_msg().
The current message format (version 2.0) is very simple. It is::
{
'oslo.version': <RPC Envelope Version as a String>,
'oslo.message': <Application Message Payload, JSON encoded>
}
Message format version '1.0' is just considered to be the messages we sent
without a message envelope.
So, the current message envelope just includes the envelope version. It may
eventually contain additional information, such as a signature for the message
payload.
We will JSON encode the application message payload. The message envelope,
which includes the JSON encoded application message body, will be passed down
to the messaging libraries as a dict.
'''
_VERSION_KEY = 'oslo.version'
_MESSAGE_KEY = 'oslo.message'
_REMOTE_POSTFIX = '_Remote'
class RPCException(Exception):
msg_fmt = _("An unknown RPC related exception occurred.")
def __init__(self, message=None, **kwargs):
self.kwargs = kwargs
if not message:
try:
message = self.msg_fmt % kwargs
except Exception:
# kwargs doesn't match a variable in the message
# log the issue and the kwargs
LOG.exception(_('Exception in string format operation'))
for name, value in six.iteritems(kwargs):
LOG.error("%s: %s" % (name, value))
# at least get the core message out if something happened
message = self.msg_fmt
super(RPCException, self).__init__(message)
class RemoteError(RPCException):
"""Signifies that a remote class has raised an exception.
Contains a string representation of the type of the original exception,
the value of the original exception, and the traceback. These are
sent to the parent as a joined string so printing the exception
contains all of the relevant info.
"""
msg_fmt = _("Remote error: %(exc_type)s %(value)s\n%(traceback)s.")
def __init__(self, exc_type=None, value=None, traceback=None):
self.exc_type = exc_type
self.value = value
self.traceback = traceback
super(RemoteError, self).__init__(exc_type=exc_type,
value=value,
traceback=traceback)
class Timeout(RPCException):
"""Signifies that a timeout has occurred.
This exception is raised if the rpc_response_timeout is reached while
waiting for a response from the remote side.
"""
msg_fmt = _('Timeout while waiting on RPC response - '
'topic: "%(topic)s", RPC method: "%(method)s" '
'info: "%(info)s"')
def __init__(self, info=None, topic=None, method=None):
"""Initiates Timeout object.
:param info: Extra info to convey to the user
:param topic: The topic that the rpc call was sent to
:param rpc_method_name: The name of the rpc method being
called
"""
self.info = info
self.topic = topic
self.method = method
super(Timeout, self).__init__(
None,
info=info or _('<unknown>'),
topic=topic or _('<unknown>'),
method=method or _('<unknown>'))
class DuplicateMessageError(RPCException):
msg_fmt = _("Found duplicate message(%(msg_id)s). Skipping it.")
class InvalidRPCConnectionReuse(RPCException):
msg_fmt = _("Invalid reuse of an RPC connection.")
class UnsupportedRpcVersion(RPCException):
msg_fmt = _("Specified RPC version, %(version)s, not supported by "
"this endpoint.")
class UnsupportedRpcEnvelopeVersion(RPCException):
msg_fmt = _("Specified RPC envelope version, %(version)s, "
"not supported by this endpoint.")
class RpcVersionCapError(RPCException):
msg_fmt = _("Specified RPC version cap, %(version_cap)s, is too low")
class Connection(object):
"""A connection, returned by rpc.create_connection().
This class represents a connection to the message bus used for rpc.
An instance of this class should never be created by users of the rpc API.
Use rpc.create_connection() instead.
"""
def close(self):
"""Close the connection.
This method must be called when the connection will no longer be used.
It will ensure that any resources associated with the connection, such
as a network connection, and cleaned up.
"""
raise NotImplementedError()
def create_consumer(self, topic, proxy, fanout=False):
"""Create a consumer on this connection.
A consumer is associated with a message queue on the backend message
bus. The consumer will read messages from the queue, unpack them, and
dispatch them to the proxy object. The contents of the message pulled
off of the queue will determine which method gets called on the proxy
object.
:param topic: This is a name associated with what to consume from.
Multiple instances of a service may consume from the same
topic. For example, all instances of nova-compute consume
from a queue called "compute". In that case, the
messages will get distributed amongst the consumers in a
round-robin fashion if fanout=False. If fanout=True,
every consumer associated with this topic will get a
copy of every message.
:param proxy: The object that will handle all incoming messages.
:param fanout: Whether or not this is a fanout topic. See the
documentation for the topic parameter for some
additional comments on this.
"""
raise NotImplementedError()
def create_worker(self, topic, proxy, pool_name):
"""Create a worker on this connection.
A worker is like a regular consumer of messages directed to a
topic, except that it is part of a set of such consumers (the
"pool") which may run in parallel. Every pool of workers will
receive a given message, but only one worker in the pool will
be asked to process it. Load is distributed across the members
of the pool in round-robin fashion.
:param topic: This is a name associated with what to consume from.
Multiple instances of a service may consume from the same
topic.
:param proxy: The object that will handle all incoming messages.
:param pool_name: String containing the name of the pool of workers
"""
raise NotImplementedError()
def join_consumer_pool(self, callback, pool_name, topic, exchange_name):
"""Register as a member of a group of consumers.
Uses given topic from the specified exchange.
Exactly one member of a given pool will receive each message.
A message will be delivered to multiple pools, if more than
one is created.
:param callback: Callable to be invoked for each message.
:type callback: callable accepting one argument
:param pool_name: The name of the consumer pool.
:type pool_name: str
:param topic: The routing topic for desired messages.
:type topic: str
:param exchange_name: The name of the message exchange where
the client should attach. Defaults to
the configured exchange.
:type exchange_name: str
"""
raise NotImplementedError()
def consume_in_thread(self):
"""Spawn a thread to handle incoming messages.
Spawn a thread that will be responsible for handling all incoming
messages for consumers that were set up on this connection.
Message dispatching inside of this is expected to be implemented in a
non-blocking manner. An example implementation would be having this
thread pull messages in for all of the consumers, but utilize a thread
pool for dispatching the messages to the proxy objects.
"""
raise NotImplementedError()
def _safe_log(log_func, msg, msg_data):
"""Sanitizes the msg_data field before logging."""
SANITIZE = ['_context_auth_token', 'auth_token', 'new_pass']
def _fix_passwords(d):
"""Sanitizes the password fields in the dictionary."""
for k in six.iterkeys(d):
if k.lower().find('password') != -1:
d[k] = '<SANITIZED>'
elif k.lower() in SANITIZE:
d[k] = '<SANITIZED>'
elif isinstance(d[k], list):
for e in d[k]:
if isinstance(e, dict):
_fix_passwords(e)
elif isinstance(d[k], dict):
_fix_passwords(d[k])
return d
return log_func(msg, _fix_passwords(copy.deepcopy(msg_data)))
def serialize_remote_exception(failure_info, log_failure=True):
"""Prepares exception data to be sent over rpc.
Failure_info should be a sys.exc_info() tuple.
"""
tb = traceback.format_exception(*failure_info)
failure = failure_info[1]
if log_failure:
LOG.error(_("Returning exception %s to caller"),
six.text_type(failure))
LOG.error(tb)
kwargs = {}
if hasattr(failure, 'kwargs'):
kwargs = failure.kwargs
# NOTE(matiu): With cells, it's possible to re-raise remote, remote
# exceptions. Lets turn it back into the original exception type.
cls_name = str(failure.__class__.__name__)
mod_name = str(failure.__class__.__module__)
if (cls_name.endswith(_REMOTE_POSTFIX) and
mod_name.endswith(_REMOTE_POSTFIX)):
cls_name = cls_name[:-len(_REMOTE_POSTFIX)]
mod_name = mod_name[:-len(_REMOTE_POSTFIX)]
data = {
'class': cls_name,
'module': mod_name,
'message': six.text_type(failure),
'tb': tb,
'args': failure.args,
'kwargs': kwargs
}
json_data = jsonutils.dumps(data)
return json_data
def deserialize_remote_exception(conf, data):
failure = jsonutils.loads(str(data))
trace = failure.get('tb', [])
message = failure.get('message', "") + "\n" + "\n".join(trace)
name = failure.get('class')
module = failure.get('module')
# NOTE(ameade): We DO NOT want to allow just any module to be imported, in
# order to prevent arbitrary code execution.
if module not in conf.allowed_rpc_exception_modules:
return RemoteError(name, failure.get('message'), trace)
try:
mod = importutils.import_module(module)
klass = getattr(mod, name)
if not issubclass(klass, Exception):
raise TypeError("Can only deserialize Exceptions")
failure = klass(*failure.get('args', []), **failure.get('kwargs', {}))
except (AttributeError, TypeError, ImportError):
return RemoteError(name, failure.get('message'), trace)
ex_type = type(failure)
str_override = lambda self: message
new_ex_type = type(ex_type.__name__ + _REMOTE_POSTFIX, (ex_type,),
{'__str__': str_override, '__unicode__': str_override})
new_ex_type.__module__ = '%s%s' % (module, _REMOTE_POSTFIX)
try:
# NOTE(ameade): Dynamically create a new exception type and swap it in
# as the new type for the exception. This only works on user defined
# Exceptions and not core python exceptions. This is important because
# we cannot necessarily change an exception message so we must override
# the __str__ method.
failure.__class__ = new_ex_type
except TypeError:
# NOTE(ameade): If a core exception then just add the traceback to the
# first exception argument.
failure.args = (message,) + failure.args[1:]
return failure
class CommonRpcContext(object):
def __init__(self, **kwargs):
self.values = kwargs
def __getattr__(self, key):
try:
return self.values[key]
except KeyError:
raise AttributeError(key)
def to_dict(self):
return copy.deepcopy(self.values)
@classmethod
def from_dict(cls, values):
return cls(**values)
def deepcopy(self):
return self.from_dict(self.to_dict())
def update_store(self):
local.store.context = self
def elevated(self, read_deleted=None, overwrite=False):
"""Return a version of this context with admin flag set."""
# TODO(russellb) This method is a bit of a nova-ism. It makes
# some assumptions about the data in the request context sent
# across rpc, while the rest of this class does not. We could get
# rid of this if we changed the nova code that uses this to
# convert the RpcContext back to its native RequestContext doing
# something like nova.context.RequestContext.from_dict(ctxt.to_dict())
context = self.deepcopy()
context.values['is_admin'] = True
context.values.setdefault('roles', [])
if 'admin' not in context.values['roles']:
context.values['roles'].append('admin')
if read_deleted is not None:
context.values['read_deleted'] = read_deleted
return context
class ClientException(Exception):
"""Encapsulates actual exception expected to be hit by a RPC proxy object.
Merely instantiating it records the current exception information, which
will be passed back to the RPC client without exceptional logging.
"""
def __init__(self):
self._exc_info = sys.exc_info()
def catch_client_exception(exceptions, func, *args, **kwargs):
try:
return func(*args, **kwargs)
except Exception as e:
if type(e) in exceptions:
raise ClientException()
else:
raise
def client_exceptions(*exceptions):
"""Decorator for manager methods that raise expected exceptions.
Marking a Manager method with this decorator allows the declaration
of expected exceptions that the RPC layer should not consider fatal,
and not log as if they were generated in a real error scenario. Note
that this will cause listed exceptions to be wrapped in a
ClientException, which is used internally by the RPC layer.
"""
def outer(func):
def inner(*args, **kwargs):
return catch_client_exception(exceptions, func, *args, **kwargs)
return inner
return outer
# TODO(sirp): we should deprecate this in favor of
# using `versionutils.is_compatible` directly
def version_is_compatible(imp_version, version):
"""Determine whether versions are compatible.
:param imp_version: The version implemented
:param version: The version requested by an incoming message.
"""
return versionutils.is_compatible(version, imp_version)
def serialize_msg(raw_msg):
# NOTE(russellb) See the docstring for _RPC_ENVELOPE_VERSION for more
# information about this format.
msg = {_VERSION_KEY: _RPC_ENVELOPE_VERSION,
_MESSAGE_KEY: jsonutils.dumps(raw_msg)}
return msg
def deserialize_msg(msg):
# NOTE(russellb): Hang on to your hats, this road is about to
# get a little bumpy.
#
# Robustness Principle:
# "Be strict in what you send, liberal in what you accept."
#
# At this point we have to do a bit of guessing about what it
# is we just received. Here is the set of possibilities:
#
# 1) We received a dict. This could be 2 things:
#
# a) Inspect it to see if it looks like a standard message envelope.
# If so, great!
#
# b) If it doesn't look like a standard message envelope, it could either
# be a notification, or a message from before we added a message
# envelope (referred to as version 1.0).
# Just return the message as-is.
#
# 2) It's any other non-dict type. Just return it and hope for the best.
# This case covers return values from rpc.call() from before message
# envelopes were used. (messages to call a method were always a dict)
if not isinstance(msg, dict):
# See #2 above.
return msg
base_envelope_keys = (_VERSION_KEY, _MESSAGE_KEY)
if not all(map(lambda key: key in msg, base_envelope_keys)):
# See #1.b above.
return msg
# At this point we think we have the message envelope
# format we were expecting. (#1.a above)
if not version_is_compatible(_RPC_ENVELOPE_VERSION, msg[_VERSION_KEY]):
raise UnsupportedRpcEnvelopeVersion(version=msg[_VERSION_KEY])
raw_msg = jsonutils.loads(msg[_MESSAGE_KEY])
return raw_msg

View File

@ -1,178 +0,0 @@
# vim: tabstop=4 shiftwidth=4 softtabstop=4
# Copyright 2012 Red Hat, Inc.
#
# Licensed under the Apache License, Version 2.0 (the "License"); you may
# not use this file except in compliance with the License. You may obtain
# a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
# License for the specific language governing permissions and limitations
# under the License.
"""
Code for rpc message dispatching.
Messages that come in have a version number associated with them. RPC API
version numbers are in the form:
Major.Minor
For a given message with version X.Y, the receiver must be marked as able to
handle messages of version A.B, where:
A = X
B >= Y
The Major version number would be incremented for an almost completely new API.
The Minor version number would be incremented for backwards compatible changes
to an existing API. A backwards compatible change could be something like
adding a new method, adding an argument to an existing method (but not
requiring it), or changing the type for an existing argument (but still
handling the old type as well).
The conversion over to a versioned API must be done on both the client side and
server side of the API at the same time. However, as the code stands today,
there can be both versioned and unversioned APIs implemented in the same code
base.
EXAMPLES
========
Nova was the first project to use versioned rpc APIs. Consider the compute rpc
API as an example. The client side is in nova/compute/rpcapi.py and the server
side is in nova/compute/manager.py.
Example 1) Adding a new method.
-------------------------------
Adding a new method is a backwards compatible change. It should be added to
nova/compute/manager.py, and RPC_API_VERSION should be bumped from X.Y to
X.Y+1. On the client side, the new method in nova/compute/rpcapi.py should
have a specific version specified to indicate the minimum API version that must
be implemented for the method to be supported. For example::
def get_host_uptime(self, ctxt, host):
topic = _compute_topic(self.topic, ctxt, host, None)
return self.call(ctxt, self.make_msg('get_host_uptime'), topic,
version='1.1')
In this case, version '1.1' is the first version that supported the
get_host_uptime() method.
Example 2) Adding a new parameter.
----------------------------------
Adding a new parameter to an rpc method can be made backwards compatible. The
RPC_API_VERSION on the server side (nova/compute/manager.py) should be bumped.
The implementation of the method must not expect the parameter to be present.::
def some_remote_method(self, arg1, arg2, newarg=None):
# The code needs to deal with newarg=None for cases
# where an older client sends a message without it.
pass
On the client side, the same changes should be made as in example 1. The
minimum version that supports the new parameter should be specified.
"""
from trove.openstack.common.rpc import common as rpc_common
from trove.openstack.common.rpc import serializer as rpc_serializer
class RpcDispatcher(object):
"""Dispatch rpc messages according to the requested API version.
This class can be used as the top level 'manager' for a service. It
contains a list of underlying managers that have an API_VERSION attribute.
"""
def __init__(self, callbacks, serializer=None):
"""Initialize the rpc dispatcher.
:param callbacks: List of proxy objects that are an instance
of a class with rpc methods exposed. Each proxy
object should have an RPC_API_VERSION attribute.
:param serializer: The Serializer object that will be used to
deserialize arguments before the method call and
to serialize the result after it returns.
"""
self.callbacks = callbacks
if serializer is None:
serializer = rpc_serializer.NoOpSerializer()
self.serializer = serializer
super(RpcDispatcher, self).__init__()
def _deserialize_args(self, context, kwargs):
"""Helper method called to deserialize args before dispatch.
This calls our serializer on each argument, returning a new set of
args that have been deserialized.
:param context: The request context
:param kwargs: The arguments to be deserialized
:returns: A new set of deserialized args
"""
new_kwargs = dict()
for argname, arg in kwargs.iteritems():
new_kwargs[argname] = self.serializer.deserialize_entity(context,
arg)
return new_kwargs
def dispatch(self, ctxt, version, method, namespace, **kwargs):
"""Dispatch a message based on a requested version.
:param ctxt: The request context
:param version: The requested API version from the incoming message
:param method: The method requested to be called by the incoming
message.
:param namespace: The namespace for the requested method. If None,
the dispatcher will look for a method on a callback
object with no namespace set.
:param kwargs: A dict of keyword arguments to be passed to the method.
:returns: Whatever is returned by the underlying method that gets
called.
"""
if not version:
version = '1.0'
had_compatible = False
for proxyobj in self.callbacks:
# Check for namespace compatibility
try:
cb_namespace = proxyobj.RPC_API_NAMESPACE
except AttributeError:
cb_namespace = None
if namespace != cb_namespace:
continue
# Check for version compatibility
try:
rpc_api_version = proxyobj.RPC_API_VERSION
except AttributeError:
rpc_api_version = '1.0'
is_compatible = rpc_common.version_is_compatible(rpc_api_version,
version)
had_compatible = had_compatible or is_compatible
if not hasattr(proxyobj, method):
continue
if is_compatible:
kwargs = self._deserialize_args(ctxt, kwargs)
result = getattr(proxyobj, method)(ctxt, **kwargs)
return self.serializer.serialize_entity(ctxt, result)
if had_compatible:
raise AttributeError("No such RPC function '%s'" % method)
else:
raise rpc_common.UnsupportedRpcVersion(version=version)

View File

@ -1,195 +0,0 @@
# vim: tabstop=4 shiftwidth=4 softtabstop=4
# 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.
"""Fake RPC implementation which calls proxy methods directly with no
queues. Casts will block, but this is very useful for tests.
"""
import inspect
# NOTE(russellb): We specifically want to use json, not our own jsonutils.
# jsonutils has some extra logic to automatically convert objects to primitive
# types so that they can be serialized. We want to catch all cases where
# non-primitive types make it into this code and treat it as an error.
import json
import time
import eventlet
from trove.openstack.common.rpc import common as rpc_common
CONSUMERS = {}
class RpcContext(rpc_common.CommonRpcContext):
def __init__(self, **kwargs):
super(RpcContext, self).__init__(**kwargs)
self._response = []
self._done = False
def deepcopy(self):
values = self.to_dict()
new_inst = self.__class__(**values)
new_inst._response = self._response
new_inst._done = self._done
return new_inst
def reply(self, reply=None, failure=None, ending=False):
if ending:
self._done = True
if not self._done:
self._response.append((reply, failure))
class Consumer(object):
def __init__(self, topic, proxy):
self.topic = topic
self.proxy = proxy
def call(self, context, version, method, namespace, args, timeout):
done = eventlet.event.Event()
def _inner():
ctxt = RpcContext.from_dict(context.to_dict())
try:
rval = self.proxy.dispatch(context, version, method,
namespace, **args)
res = []
# Caller might have called ctxt.reply() manually
for (reply, failure) in ctxt._response:
if failure:
raise failure[0], failure[1], failure[2]
res.append(reply)
# if ending not 'sent'...we might have more data to
# return from the function itself
if not ctxt._done:
if inspect.isgenerator(rval):
for val in rval:
res.append(val)
else:
res.append(rval)
done.send(res)
except rpc_common.ClientException as e:
done.send_exception(e._exc_info[1])
except Exception as e:
done.send_exception(e)
thread = eventlet.greenthread.spawn(_inner)
if timeout:
start_time = time.time()
while not done.ready():
eventlet.greenthread.sleep(1)
cur_time = time.time()
if (cur_time - start_time) > timeout:
thread.kill()
raise rpc_common.Timeout()
return done.wait()
class Connection(object):
"""Connection object."""
def __init__(self):
self.consumers = []
def create_consumer(self, topic, proxy, fanout=False):
consumer = Consumer(topic, proxy)
self.consumers.append(consumer)
if topic not in CONSUMERS:
CONSUMERS[topic] = []
CONSUMERS[topic].append(consumer)
def close(self):
for consumer in self.consumers:
CONSUMERS[consumer.topic].remove(consumer)
self.consumers = []
def consume_in_thread(self):
pass
def create_connection(conf, new=True):
"""Create a connection."""
return Connection()
def check_serialize(msg):
"""Make sure a message intended for rpc can be serialized."""
json.dumps(msg)
def multicall(conf, context, topic, msg, timeout=None):
"""Make a call that returns multiple times."""
check_serialize(msg)
method = msg.get('method')
if not method:
return
args = msg.get('args', {})
version = msg.get('version', None)
namespace = msg.get('namespace', None)
try:
consumer = CONSUMERS[topic][0]
except (KeyError, IndexError):
raise rpc_common.Timeout("No consumers available")
else:
return consumer.call(context, version, method, namespace, args,
timeout)
def call(conf, context, topic, msg, timeout=None):
"""Sends a message on a topic and wait for a response."""
rv = multicall(conf, context, topic, msg, timeout)
# NOTE(vish): return the last result from the multicall
rv = list(rv)
if not rv:
return
return rv[-1]
def cast(conf, context, topic, msg):
check_serialize(msg)
try:
call(conf, context, topic, msg)
except Exception:
pass
def notify(conf, context, topic, msg, envelope):
check_serialize(msg)
def cleanup():
pass
def fanout_cast(conf, context, topic, msg):
"""Cast to all consumers of a topic."""
check_serialize(msg)
method = msg.get('method')
if not method:
return
args = msg.get('args', {})
version = msg.get('version', None)
namespace = msg.get('namespace', None)
for consumer in CONSUMERS.get(topic, []):
try:
consumer.call(context, version, method, namespace, args, None)
except Exception:
pass

View File

@ -1,855 +0,0 @@
# vim: tabstop=4 shiftwidth=4 softtabstop=4
# 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 functools
import itertools
import socket
import ssl
import time
import uuid
import eventlet
import greenlet
import kombu
import kombu.connection
import kombu.entity
import kombu.messaging
from oslo.config import cfg
from trove.openstack.common import excutils
from trove.openstack.common.gettextutils import _ # noqa
from trove.openstack.common import network_utils
from trove.openstack.common.rpc import amqp as rpc_amqp
from trove.openstack.common.rpc import common as rpc_common
from trove.openstack.common import sslutils
kombu_opts = [
cfg.StrOpt('kombu_ssl_version',
default='',
help='SSL version to use (valid only if SSL enabled). '
'valid values are TLSv1, SSLv23 and SSLv3. SSLv2 may '
'be available on some distributions'
),
cfg.StrOpt('kombu_ssl_keyfile',
default='',
help='SSL key file (valid only if SSL enabled)'),
cfg.StrOpt('kombu_ssl_certfile',
default='',
help='SSL cert file (valid only if SSL enabled)'),
cfg.StrOpt('kombu_ssl_ca_certs',
default='',
help=('SSL certification authority file '
'(valid only if SSL enabled)')),
cfg.StrOpt('rabbit_host',
default='localhost',
help='The RabbitMQ broker address where a single node is used'),
cfg.IntOpt('rabbit_port',
default=5672,
help='The RabbitMQ broker port where a single node is used'),
cfg.ListOpt('rabbit_hosts',
default=['$rabbit_host:$rabbit_port'],
help='RabbitMQ HA cluster host:port pairs'),
cfg.BoolOpt('rabbit_use_ssl',
default=False,
help='connect over SSL for RabbitMQ'),
cfg.StrOpt('rabbit_userid',
default='guest',
help='the RabbitMQ userid'),
cfg.StrOpt('rabbit_password',
default='guest',
help='the RabbitMQ password',
secret=True),
cfg.StrOpt('rabbit_virtual_host',
default='/',
help='the RabbitMQ virtual host'),
cfg.IntOpt('rabbit_retry_interval',
default=1,
help='how frequently to retry connecting with RabbitMQ'),
cfg.IntOpt('rabbit_retry_backoff',
default=2,
help='how long to backoff for between retries when connecting '
'to RabbitMQ'),
cfg.IntOpt('rabbit_max_retries',
default=0,
help='maximum retries with trying to connect to RabbitMQ '
'(the default of 0 implies an infinite retry count)'),
cfg.BoolOpt('rabbit_ha_queues',
default=False,
help='use H/A queues in RabbitMQ (x-ha-policy: all).'
'You need to wipe RabbitMQ database when '
'changing this option.'),
]
cfg.CONF.register_opts(kombu_opts)
LOG = rpc_common.LOG
def _get_queue_arguments(conf):
"""Construct the arguments for declaring a queue.
If the rabbit_ha_queues option is set, we declare a mirrored queue
as described here:
http://www.rabbitmq.com/ha.html
Setting x-ha-policy to all means that the queue will be mirrored
to all nodes in the cluster.
"""
return {'x-ha-policy': 'all'} if conf.rabbit_ha_queues else {}
class ConsumerBase(object):
"""Consumer base class."""
def __init__(self, channel, callback, tag, **kwargs):
"""Declare a queue on an amqp channel.
'channel' is the amqp channel to use
'callback' is the callback to call when messages are received
'tag' is a unique ID for the consumer on the channel
queue name, exchange name, and other kombu options are
passed in here as a dictionary.
"""
self.callback = callback
self.tag = str(tag)
self.kwargs = kwargs
self.queue = None
self.ack_on_error = kwargs.get('ack_on_error', True)
self.reconnect(channel)
def reconnect(self, channel):
"""Re-declare the queue after a rabbit reconnect."""
self.channel = channel
self.kwargs['channel'] = channel
self.queue = kombu.entity.Queue(**self.kwargs)
self.queue.declare()
def _callback_handler(self, message, callback):
"""Call callback with deserialized message.
Messages that are processed without exception are ack'ed.
If the message processing generates an exception, it will be
ack'ed if ack_on_error=True. Otherwise it will be .requeue()'ed.
"""
try:
msg = rpc_common.deserialize_msg(message.payload)
callback(msg)
except Exception:
if self.ack_on_error:
LOG.exception(_("Failed to process message"
" ... skipping it."))
message.ack()
else:
LOG.exception(_("Failed to process message"
" ... will requeue."))
message.requeue()
else:
message.ack()
def consume(self, *args, **kwargs):
"""Actually declare the consumer on the amqp channel. This will
start the flow of messages from the queue. Using the
Connection.iterconsume() iterator will process the messages,
calling the appropriate callback.
If a callback is specified in kwargs, use that. Otherwise,
use the callback passed during __init__()
If kwargs['nowait'] is True, then this call will block until
a message is read.
"""
options = {'consumer_tag': self.tag}
options['nowait'] = kwargs.get('nowait', False)
callback = kwargs.get('callback', self.callback)
if not callback:
raise ValueError("No callback defined")
def _callback(raw_message):
message = self.channel.message_to_python(raw_message)
self._callback_handler(message, callback)
self.queue.consume(*args, callback=_callback, **options)
def cancel(self):
"""Cancel the consuming from the queue, if it has started."""
try:
self.queue.cancel(self.tag)
except KeyError as e:
# NOTE(comstud): Kludge to get around a amqplib bug
if str(e) != "u'%s'" % self.tag:
raise
self.queue = None
class DirectConsumer(ConsumerBase):
"""Queue/consumer class for 'direct'."""
def __init__(self, conf, channel, msg_id, callback, tag, **kwargs):
"""Init a 'direct' queue.
'channel' is the amqp channel to use
'msg_id' is the msg_id to listen on
'callback' is the callback to call when messages are received
'tag' is a unique ID for the consumer on the channel
Other kombu options may be passed
"""
# Default options
options = {'durable': False,
'queue_arguments': _get_queue_arguments(conf),
'auto_delete': True,
'exclusive': False}
options.update(kwargs)
exchange = kombu.entity.Exchange(name=msg_id,
type='direct',
durable=options['durable'],
auto_delete=options['auto_delete'])
super(DirectConsumer, self).__init__(channel,
callback,
tag,
name=msg_id,
exchange=exchange,
routing_key=msg_id,
**options)
class TopicConsumer(ConsumerBase):
"""Consumer class for 'topic'."""
def __init__(self, conf, channel, topic, callback, tag, name=None,
exchange_name=None, **kwargs):
"""Init a 'topic' queue.
:param channel: the amqp channel to use
:param topic: the topic to listen on
:paramtype topic: str
:param callback: the callback to call when messages are received
:param tag: a unique ID for the consumer on the channel
:param name: optional queue name, defaults to topic
:paramtype name: str
Other kombu options may be passed as keyword arguments
"""
# Default options
options = {'durable': conf.amqp_durable_queues,
'queue_arguments': _get_queue_arguments(conf),
'auto_delete': conf.amqp_auto_delete,
'exclusive': False}
options.update(kwargs)
exchange_name = exchange_name or rpc_amqp.get_control_exchange(conf)
exchange = kombu.entity.Exchange(name=exchange_name,
type='topic',
durable=options['durable'],
auto_delete=options['auto_delete'])
super(TopicConsumer, self).__init__(channel,
callback,
tag,
name=name or topic,
exchange=exchange,
routing_key=topic,
**options)
class FanoutConsumer(ConsumerBase):
"""Consumer class for 'fanout'."""
def __init__(self, conf, channel, topic, callback, tag, **kwargs):
"""Init a 'fanout' queue.
'channel' is the amqp channel to use
'topic' is the topic to listen on
'callback' is the callback to call when messages are received
'tag' is a unique ID for the consumer on the channel
Other kombu options may be passed
"""
unique = uuid.uuid4().hex
exchange_name = '%s_fanout' % topic
queue_name = '%s_fanout_%s' % (topic, unique)
# Default options
options = {'durable': False,
'queue_arguments': _get_queue_arguments(conf),
'auto_delete': True,
'exclusive': False}
options.update(kwargs)
exchange = kombu.entity.Exchange(name=exchange_name, type='fanout',
durable=options['durable'],
auto_delete=options['auto_delete'])
super(FanoutConsumer, self).__init__(channel, callback, tag,
name=queue_name,
exchange=exchange,
routing_key=topic,
**options)
class Publisher(object):
"""Base Publisher class."""
def __init__(self, channel, exchange_name, routing_key, **kwargs):
"""Init the Publisher class with the exchange_name, routing_key,
and other options
"""
self.exchange_name = exchange_name
self.routing_key = routing_key
self.kwargs = kwargs
self.reconnect(channel)
def reconnect(self, channel):
"""Re-establish the Producer after a rabbit reconnection."""
self.exchange = kombu.entity.Exchange(name=self.exchange_name,
**self.kwargs)
self.producer = kombu.messaging.Producer(exchange=self.exchange,
channel=channel,
routing_key=self.routing_key)
def send(self, msg, timeout=None):
"""Send a message."""
if timeout:
#
# AMQP TTL is in milliseconds when set in the header.
#
self.producer.publish(msg, headers={'ttl': (timeout * 1000)})
else:
self.producer.publish(msg)
class DirectPublisher(Publisher):
"""Publisher class for 'direct'."""
def __init__(self, conf, channel, msg_id, **kwargs):
"""init a 'direct' publisher.
Kombu options may be passed as keyword args to override defaults
"""
options = {'durable': False,
'auto_delete': True,
'exclusive': False}
options.update(kwargs)
super(DirectPublisher, self).__init__(channel, msg_id, msg_id,
type='direct', **options)
class TopicPublisher(Publisher):
"""Publisher class for 'topic'."""
def __init__(self, conf, channel, topic, **kwargs):
"""init a 'topic' publisher.
Kombu options may be passed as keyword args to override defaults
"""
options = {'durable': conf.amqp_durable_queues,
'auto_delete': conf.amqp_auto_delete,
'exclusive': False}
options.update(kwargs)
exchange_name = rpc_amqp.get_control_exchange(conf)
super(TopicPublisher, self).__init__(channel,
exchange_name,
topic,
type='topic',
**options)
class FanoutPublisher(Publisher):
"""Publisher class for 'fanout'."""
def __init__(self, conf, channel, topic, **kwargs):
"""init a 'fanout' publisher.
Kombu options may be passed as keyword args to override defaults
"""
options = {'durable': False,
'auto_delete': True,
'exclusive': False}
options.update(kwargs)
super(FanoutPublisher, self).__init__(channel, '%s_fanout' % topic,
None, type='fanout', **options)
class NotifyPublisher(TopicPublisher):
"""Publisher class for 'notify'."""
def __init__(self, conf, channel, topic, **kwargs):
self.durable = kwargs.pop('durable', conf.amqp_durable_queues)
self.queue_arguments = _get_queue_arguments(conf)
super(NotifyPublisher, self).__init__(conf, channel, topic, **kwargs)
def reconnect(self, channel):
super(NotifyPublisher, self).reconnect(channel)
# NOTE(jerdfelt): Normally the consumer would create the queue, but
# we do this to ensure that messages don't get dropped if the
# consumer is started after we do
queue = kombu.entity.Queue(channel=channel,
exchange=self.exchange,
durable=self.durable,
name=self.routing_key,
routing_key=self.routing_key,
queue_arguments=self.queue_arguments)
queue.declare()
class Connection(object):
"""Connection object."""
pool = None
def __init__(self, conf, server_params=None):
self.consumers = []
self.consumer_thread = None
self.proxy_callbacks = []
self.conf = conf
self.max_retries = self.conf.rabbit_max_retries
# Try forever?
if self.max_retries <= 0:
self.max_retries = None
self.interval_start = self.conf.rabbit_retry_interval
self.interval_stepping = self.conf.rabbit_retry_backoff
# max retry-interval = 30 seconds
self.interval_max = 30
self.memory_transport = False
if server_params is None:
server_params = {}
# Keys to translate from server_params to kombu params
server_params_to_kombu_params = {'username': 'userid'}
ssl_params = self._fetch_ssl_params()
params_list = []
for adr in self.conf.rabbit_hosts:
hostname, port = network_utils.parse_host_port(
adr, default_port=self.conf.rabbit_port)
params = {
'hostname': hostname,
'port': port,
'userid': self.conf.rabbit_userid,
'password': self.conf.rabbit_password,
'virtual_host': self.conf.rabbit_virtual_host,
}
for sp_key, value in server_params.iteritems():
p_key = server_params_to_kombu_params.get(sp_key, sp_key)
params[p_key] = value
if self.conf.fake_rabbit:
params['transport'] = 'memory'
if self.conf.rabbit_use_ssl:
params['ssl'] = ssl_params
params_list.append(params)
self.params_list = params_list
self.memory_transport = self.conf.fake_rabbit
self.connection = None
self.reconnect()
def _fetch_ssl_params(self):
"""Handles fetching what ssl params should be used for the connection
(if any).
"""
ssl_params = dict()
# http://docs.python.org/library/ssl.html - ssl.wrap_socket
if self.conf.kombu_ssl_version:
ssl_params['ssl_version'] = sslutils.validate_ssl_version(
self.conf.kombu_ssl_version)
if self.conf.kombu_ssl_keyfile:
ssl_params['keyfile'] = self.conf.kombu_ssl_keyfile
if self.conf.kombu_ssl_certfile:
ssl_params['certfile'] = self.conf.kombu_ssl_certfile
if self.conf.kombu_ssl_ca_certs:
ssl_params['ca_certs'] = self.conf.kombu_ssl_ca_certs
# We might want to allow variations in the
# future with this?
ssl_params['cert_reqs'] = ssl.CERT_REQUIRED
# Return the extended behavior or just have the default behavior
return ssl_params or True
def _connect(self, params):
"""Connect to rabbit. Re-establish any queues that may have
been declared before if we are reconnecting. Exceptions should
be handled by the caller.
"""
if self.connection:
LOG.info(_("Reconnecting to AMQP server on "
"%(hostname)s:%(port)d") % params)
try:
self.connection.release()
except self.connection_errors:
pass
# Setting this in case the next statement fails, though
# it shouldn't be doing any network operations, yet.
self.connection = None
self.connection = kombu.connection.BrokerConnection(**params)
self.connection_errors = self.connection.connection_errors
if self.memory_transport:
# Kludge to speed up tests.
self.connection.transport.polling_interval = 0.0
self.consumer_num = itertools.count(1)
self.connection.connect()
self.channel = self.connection.channel()
# work around 'memory' transport bug in 1.1.3
if self.memory_transport:
self.channel._new_queue('ae.undeliver')
for consumer in self.consumers:
consumer.reconnect(self.channel)
LOG.info(_('Connected to AMQP server on %(hostname)s:%(port)d') %
params)
def reconnect(self):
"""Handles reconnecting and re-establishing queues.
Will retry up to self.max_retries number of times.
self.max_retries = 0 means to retry forever.
Sleep between tries, starting at self.interval_start
seconds, backing off self.interval_stepping number of seconds
each attempt.
"""
attempt = 0
while True:
params = self.params_list[attempt % len(self.params_list)]
attempt += 1
try:
self._connect(params)
return
except (IOError, self.connection_errors) as e:
pass
except Exception as e:
# NOTE(comstud): Unfortunately it's possible for amqplib
# to return an error not covered by its transport
# connection_errors in the case of a timeout waiting for
# a protocol response. (See paste link in LP888621)
# So, we check all exceptions for 'timeout' in them
# and try to reconnect in this case.
if 'timeout' not in str(e):
raise
log_info = {}
log_info['err_str'] = str(e)
log_info['max_retries'] = self.max_retries
log_info.update(params)
if self.max_retries and attempt == self.max_retries:
msg = _('Unable to connect to AMQP server on '
'%(hostname)s:%(port)d after %(max_retries)d '
'tries: %(err_str)s') % log_info
LOG.error(msg)
raise rpc_common.RPCException(msg)
if attempt == 1:
sleep_time = self.interval_start or 1
elif attempt > 1:
sleep_time += self.interval_stepping
if self.interval_max:
sleep_time = min(sleep_time, self.interval_max)
log_info['sleep_time'] = sleep_time
LOG.error(_('AMQP server on %(hostname)s:%(port)d is '
'unreachable: %(err_str)s. Trying again in '
'%(sleep_time)d seconds.') % log_info)
time.sleep(sleep_time)
def ensure(self, error_callback, method, *args, **kwargs):
while True:
try:
return method(*args, **kwargs)
except (self.connection_errors, socket.timeout, IOError) as e:
if error_callback:
error_callback(e)
except Exception as e:
# NOTE(comstud): Unfortunately it's possible for amqplib
# to return an error not covered by its transport
# connection_errors in the case of a timeout waiting for
# a protocol response. (See paste link in LP888621)
# So, we check all exceptions for 'timeout' in them
# and try to reconnect in this case.
if 'timeout' not in str(e):
raise
if error_callback:
error_callback(e)
self.reconnect()
def get_channel(self):
"""Convenience call for bin/clear_rabbit_queues."""
return self.channel
def close(self):
"""Close/release this connection."""
self.cancel_consumer_thread()
self.wait_on_proxy_callbacks()
self.connection.release()
self.connection = None
def reset(self):
"""Reset a connection so it can be used again."""
self.cancel_consumer_thread()
self.wait_on_proxy_callbacks()
self.channel.close()
self.channel = self.connection.channel()
# work around 'memory' transport bug in 1.1.3
if self.memory_transport:
self.channel._new_queue('ae.undeliver')
self.consumers = []
def declare_consumer(self, consumer_cls, topic, callback):
"""Create a Consumer using the class that was passed in and
add it to our list of consumers
"""
def _connect_error(exc):
log_info = {'topic': topic, 'err_str': str(exc)}
LOG.error(_("Failed to declare consumer for topic '%(topic)s': "
"%(err_str)s") % log_info)
def _declare_consumer():
consumer = consumer_cls(self.conf, self.channel, topic, callback,
self.consumer_num.next())
self.consumers.append(consumer)
return consumer
return self.ensure(_connect_error, _declare_consumer)
def iterconsume(self, limit=None, timeout=None):
"""Return an iterator that will consume from all queues/consumers."""
info = {'do_consume': True}
def _error_callback(exc):
if isinstance(exc, socket.timeout):
LOG.debug(_('Timed out waiting for RPC response: %s') %
str(exc))
raise rpc_common.Timeout()
else:
LOG.exception(_('Failed to consume message from queue: %s') %
str(exc))
info['do_consume'] = True
def _consume():
if info['do_consume']:
queues_head = self.consumers[:-1] # not fanout.
queues_tail = self.consumers[-1] # fanout
for queue in queues_head:
queue.consume(nowait=True)
queues_tail.consume(nowait=False)
info['do_consume'] = False
return self.connection.drain_events(timeout=timeout)
for iteration in itertools.count(0):
if limit and iteration >= limit:
raise StopIteration
yield self.ensure(_error_callback, _consume)
def cancel_consumer_thread(self):
"""Cancel a consumer thread."""
if self.consumer_thread is not None:
self.consumer_thread.kill()
try:
self.consumer_thread.wait()
except greenlet.GreenletExit:
pass
self.consumer_thread = None
def wait_on_proxy_callbacks(self):
"""Wait for all proxy callback threads to exit."""
for proxy_cb in self.proxy_callbacks:
proxy_cb.wait()
def publisher_send(self, cls, topic, msg, timeout=None, **kwargs):
"""Send to a publisher based on the publisher class."""
def _error_callback(exc):
log_info = {'topic': topic, 'err_str': str(exc)}
LOG.exception(_("Failed to publish message to topic "
"'%(topic)s': %(err_str)s") % log_info)
def _publish():
publisher = cls(self.conf, self.channel, topic, **kwargs)
publisher.send(msg, timeout)
self.ensure(_error_callback, _publish)
def declare_direct_consumer(self, topic, callback):
"""Create a 'direct' queue.
In nova's use, this is generally a msg_id queue used for
responses for call/multicall
"""
self.declare_consumer(DirectConsumer, topic, callback)
def declare_topic_consumer(self, topic, callback=None, queue_name=None,
exchange_name=None, ack_on_error=True):
"""Create a 'topic' consumer."""
self.declare_consumer(functools.partial(TopicConsumer,
name=queue_name,
exchange_name=exchange_name,
ack_on_error=ack_on_error,
),
topic, callback)
def declare_fanout_consumer(self, topic, callback):
"""Create a 'fanout' consumer."""
self.declare_consumer(FanoutConsumer, topic, callback)
def direct_send(self, msg_id, msg):
"""Send a 'direct' message."""
self.publisher_send(DirectPublisher, msg_id, msg)
def topic_send(self, topic, msg, timeout=None):
"""Send a 'topic' message."""
self.publisher_send(TopicPublisher, topic, msg, timeout)
def fanout_send(self, topic, msg):
"""Send a 'fanout' message."""
self.publisher_send(FanoutPublisher, topic, msg)
def notify_send(self, topic, msg, **kwargs):
"""Send a notify message on a topic."""
self.publisher_send(NotifyPublisher, topic, msg, None, **kwargs)
def consume(self, limit=None):
"""Consume from all queues/consumers."""
it = self.iterconsume(limit=limit)
while True:
try:
it.next()
except StopIteration:
return
def consume_in_thread(self):
"""Consumer from all queues/consumers in a greenthread."""
@excutils.forever_retry_uncaught_exceptions
def _consumer_thread():
try:
self.consume()
except greenlet.GreenletExit:
return
if self.consumer_thread is None:
self.consumer_thread = eventlet.spawn(_consumer_thread)
return self.consumer_thread
def create_consumer(self, topic, proxy, fanout=False):
"""Create a consumer that calls a method in a proxy object."""
proxy_cb = rpc_amqp.ProxyCallback(
self.conf, proxy,
rpc_amqp.get_connection_pool(self.conf, Connection))
self.proxy_callbacks.append(proxy_cb)
if fanout:
self.declare_fanout_consumer(topic, proxy_cb)
else:
self.declare_topic_consumer(topic, proxy_cb)
def create_worker(self, topic, proxy, pool_name):
"""Create a worker that calls a method in a proxy object."""
proxy_cb = rpc_amqp.ProxyCallback(
self.conf, proxy,
rpc_amqp.get_connection_pool(self.conf, Connection))
self.proxy_callbacks.append(proxy_cb)
self.declare_topic_consumer(topic, proxy_cb, pool_name)
def join_consumer_pool(self, callback, pool_name, topic,
exchange_name=None, ack_on_error=True):
"""Register as a member of a group of consumers for a given topic from
the specified exchange.
Exactly one member of a given pool will receive each message.
A message will be delivered to multiple pools, if more than
one is created.
"""
callback_wrapper = rpc_amqp.CallbackWrapper(
conf=self.conf,
callback=callback,
connection_pool=rpc_amqp.get_connection_pool(self.conf,
Connection),
)
self.proxy_callbacks.append(callback_wrapper)
self.declare_topic_consumer(
queue_name=pool_name,
topic=topic,
exchange_name=exchange_name,
callback=callback_wrapper,
ack_on_error=ack_on_error,
)
def create_connection(conf, new=True):
"""Create a connection."""
return rpc_amqp.create_connection(
conf, new,
rpc_amqp.get_connection_pool(conf, Connection))
def multicall(conf, context, topic, msg, timeout=None):
"""Make a call that returns multiple times."""
return rpc_amqp.multicall(
conf, context, topic, msg, timeout,
rpc_amqp.get_connection_pool(conf, Connection))
def call(conf, context, topic, msg, timeout=None):
"""Sends a message on a topic and wait for a response."""
return rpc_amqp.call(
conf, context, topic, msg, timeout,
rpc_amqp.get_connection_pool(conf, Connection))
def cast(conf, context, topic, msg):
"""Sends a message on a topic without waiting for a response."""
return rpc_amqp.cast(
conf, context, topic, msg,
rpc_amqp.get_connection_pool(conf, Connection))
def fanout_cast(conf, context, topic, msg):
"""Sends a message on a fanout exchange without waiting for a response."""
return rpc_amqp.fanout_cast(
conf, context, topic, msg,
rpc_amqp.get_connection_pool(conf, Connection))
def cast_to_server(conf, context, server_params, topic, msg):
"""Sends a message on a topic to a specific server."""
return rpc_amqp.cast_to_server(
conf, context, server_params, topic, msg,
rpc_amqp.get_connection_pool(conf, Connection))
def fanout_cast_to_server(conf, context, server_params, topic, msg):
"""Sends a message on a fanout exchange to a specific server."""
return rpc_amqp.fanout_cast_to_server(
conf, context, server_params, topic, msg,
rpc_amqp.get_connection_pool(conf, Connection))
def notify(conf, context, topic, msg, envelope):
"""Sends a notification event on a topic."""
return rpc_amqp.notify(
conf, context, topic, msg,
rpc_amqp.get_connection_pool(conf, Connection),
envelope)
def cleanup():
return rpc_amqp.cleanup(Connection.pool)

View File

@ -1,740 +0,0 @@
# vim: tabstop=4 shiftwidth=4 softtabstop=4
# Copyright 2011 OpenStack Foundation
# Copyright 2011 - 2012, Red Hat, Inc.
#
# Licensed under the Apache License, Version 2.0 (the "License"); you may
# not use this file except in compliance with the License. You may obtain
# a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
# License for the specific language governing permissions and limitations
# under the License.
import functools
import itertools
import time
import uuid
import eventlet
import greenlet
from oslo.config import cfg
from trove.openstack.common import excutils
from trove.openstack.common.gettextutils import _ # noqa
from trove.openstack.common import importutils
from trove.openstack.common import jsonutils
from trove.openstack.common import log as logging
from trove.openstack.common.rpc import amqp as rpc_amqp
from trove.openstack.common.rpc import common as rpc_common
qpid_codec = importutils.try_import("qpid.codec010")
qpid_messaging = importutils.try_import("qpid.messaging")
qpid_exceptions = importutils.try_import("qpid.messaging.exceptions")
LOG = logging.getLogger(__name__)
qpid_opts = [
cfg.StrOpt('qpid_hostname',
default='localhost',
help='Qpid broker hostname'),
cfg.IntOpt('qpid_port',
default=5672,
help='Qpid broker port'),
cfg.ListOpt('qpid_hosts',
default=['$qpid_hostname:$qpid_port'],
help='Qpid HA cluster host:port pairs'),
cfg.StrOpt('qpid_username',
default='',
help='Username for qpid connection'),
cfg.StrOpt('qpid_password',
default='',
help='Password for qpid connection',
secret=True),
cfg.StrOpt('qpid_sasl_mechanisms',
default='',
help='Space separated list of SASL mechanisms to use for auth'),
cfg.IntOpt('qpid_heartbeat',
default=60,
help='Seconds between connection keepalive heartbeats'),
cfg.StrOpt('qpid_protocol',
default='tcp',
help="Transport to use, either 'tcp' or 'ssl'"),
cfg.BoolOpt('qpid_tcp_nodelay',
default=True,
help='Disable Nagle algorithm'),
]
cfg.CONF.register_opts(qpid_opts)
JSON_CONTENT_TYPE = 'application/json; charset=utf8'
class ConsumerBase(object):
"""Consumer base class."""
def __init__(self, session, callback, node_name, node_opts,
link_name, link_opts):
"""Declare a queue on an amqp session.
'session' is the amqp session to use
'callback' is the callback to call when messages are received
'node_name' is the first part of the Qpid address string, before ';'
'node_opts' will be applied to the "x-declare" section of "node"
in the address string.
'link_name' goes into the "name" field of the "link" in the address
string
'link_opts' will be applied to the "x-declare" section of "link"
in the address string.
"""
self.callback = callback
self.receiver = None
self.session = None
addr_opts = {
"create": "always",
"node": {
"type": "topic",
"x-declare": {
"durable": True,
"auto-delete": True,
},
},
"link": {
"name": link_name,
"durable": True,
"x-declare": {
"durable": False,
"auto-delete": True,
"exclusive": False,
},
},
}
addr_opts["node"]["x-declare"].update(node_opts)
addr_opts["link"]["x-declare"].update(link_opts)
self.address = "%s ; %s" % (node_name, jsonutils.dumps(addr_opts))
self.connect(session)
def connect(self, session):
"""Declare the reciever on connect."""
self._declare_receiver(session)
def reconnect(self, session):
"""Re-declare the receiver after a qpid reconnect."""
self._declare_receiver(session)
def _declare_receiver(self, session):
self.session = session
self.receiver = session.receiver(self.address)
self.receiver.capacity = 1
def _unpack_json_msg(self, msg):
"""Load the JSON data in msg if msg.content_type indicates that it
is necessary. Put the loaded data back into msg.content and
update msg.content_type appropriately.
A Qpid Message containing a dict will have a content_type of
'amqp/map', whereas one containing a string that needs to be converted
back from JSON will have a content_type of JSON_CONTENT_TYPE.
:param msg: a Qpid Message object
:returns: None
"""
if msg.content_type == JSON_CONTENT_TYPE:
msg.content = jsonutils.loads(msg.content)
msg.content_type = 'amqp/map'
def consume(self):
"""Fetch the message and pass it to the callback object."""
message = self.receiver.fetch()
try:
self._unpack_json_msg(message)
msg = rpc_common.deserialize_msg(message.content)
self.callback(msg)
except Exception:
LOG.exception(_("Failed to process message... skipping it."))
finally:
# TODO(sandy): Need support for optional ack_on_error.
self.session.acknowledge(message)
def get_receiver(self):
return self.receiver
def get_node_name(self):
return self.address.split(';')[0]
class DirectConsumer(ConsumerBase):
"""Queue/consumer class for 'direct'."""
def __init__(self, conf, session, msg_id, callback):
"""Init a 'direct' queue.
'session' is the amqp session to use
'msg_id' is the msg_id to listen on
'callback' is the callback to call when messages are received
"""
super(DirectConsumer, self).__init__(
session, callback,
"%s/%s" % (msg_id, msg_id),
{"type": "direct"},
msg_id,
{
"auto-delete": conf.amqp_auto_delete,
"exclusive": True,
"durable": conf.amqp_durable_queues,
})
class TopicConsumer(ConsumerBase):
"""Consumer class for 'topic'."""
def __init__(self, conf, session, topic, callback, name=None,
exchange_name=None):
"""Init a 'topic' queue.
:param session: the amqp session to use
:param topic: is the topic to listen on
:paramtype topic: str
:param callback: the callback to call when messages are received
:param name: optional queue name, defaults to topic
"""
exchange_name = exchange_name or rpc_amqp.get_control_exchange(conf)
super(TopicConsumer, self).__init__(
session, callback,
"%s/%s" % (exchange_name, topic),
{}, name or topic,
{
"auto-delete": conf.amqp_auto_delete,
"durable": conf.amqp_durable_queues,
})
class FanoutConsumer(ConsumerBase):
"""Consumer class for 'fanout'."""
def __init__(self, conf, session, topic, callback):
"""Init a 'fanout' queue.
'session' is the amqp session to use
'topic' is the topic to listen on
'callback' is the callback to call when messages are received
"""
self.conf = conf
super(FanoutConsumer, self).__init__(
session, callback,
"%s_fanout" % topic,
{"durable": False, "type": "fanout"},
"%s_fanout_%s" % (topic, uuid.uuid4().hex),
{"exclusive": True})
def reconnect(self, session):
topic = self.get_node_name().rpartition('_fanout')[0]
params = {
'session': session,
'topic': topic,
'callback': self.callback,
}
self.__init__(conf=self.conf, **params)
super(FanoutConsumer, self).reconnect(session)
class Publisher(object):
"""Base Publisher class."""
def __init__(self, session, node_name, node_opts=None):
"""Init the Publisher class with the exchange_name, routing_key,
and other options
"""
self.sender = None
self.session = session
addr_opts = {
"create": "always",
"node": {
"type": "topic",
"x-declare": {
"durable": False,
# auto-delete isn't implemented for exchanges in qpid,
# but put in here anyway
"auto-delete": True,
},
},
}
if node_opts:
addr_opts["node"]["x-declare"].update(node_opts)
self.address = "%s ; %s" % (node_name, jsonutils.dumps(addr_opts))
self.reconnect(session)
def reconnect(self, session):
"""Re-establish the Sender after a reconnection."""
self.sender = session.sender(self.address)
def _pack_json_msg(self, msg):
"""Qpid cannot serialize dicts containing strings longer than 65535
characters. This function dumps the message content to a JSON
string, which Qpid is able to handle.
:param msg: May be either a Qpid Message object or a bare dict.
:returns: A Qpid Message with its content field JSON encoded.
"""
try:
msg.content = jsonutils.dumps(msg.content)
except AttributeError:
# Need to have a Qpid message so we can set the content_type.
msg = qpid_messaging.Message(jsonutils.dumps(msg))
msg.content_type = JSON_CONTENT_TYPE
return msg
def send(self, msg):
"""Send a message."""
try:
# Check if Qpid can encode the message
check_msg = msg
if not hasattr(check_msg, 'content_type'):
check_msg = qpid_messaging.Message(msg)
content_type = check_msg.content_type
enc, dec = qpid_messaging.message.get_codec(content_type)
enc(check_msg.content)
except qpid_codec.CodecException:
# This means the message couldn't be serialized as a dict.
msg = self._pack_json_msg(msg)
self.sender.send(msg)
class DirectPublisher(Publisher):
"""Publisher class for 'direct'."""
def __init__(self, conf, session, msg_id):
"""Init a 'direct' publisher."""
node_name = "%s/%s" % (msg_id, msg_id)
super(DirectPublisher, self).__init__(session, node_name,
{"type": "direct"})
class TopicPublisher(Publisher):
"""Publisher class for 'topic'."""
def __init__(self, conf, session, topic):
"""init a 'topic' publisher.
"""
exchange_name = rpc_amqp.get_control_exchange(conf)
super(TopicPublisher, self).__init__(session,
"%s/%s" % (exchange_name, topic))
class FanoutPublisher(Publisher):
"""Publisher class for 'fanout'."""
def __init__(self, conf, session, topic):
"""init a 'fanout' publisher.
"""
super(FanoutPublisher, self).__init__(
session,
"%s_fanout" % topic, {"type": "fanout"})
class NotifyPublisher(Publisher):
"""Publisher class for notifications."""
def __init__(self, conf, session, topic):
"""init a 'topic' publisher.
"""
exchange_name = rpc_amqp.get_control_exchange(conf)
super(NotifyPublisher, self).__init__(session,
"%s/%s" % (exchange_name, topic),
{"durable": True})
class Connection(object):
"""Connection object."""
pool = None
def __init__(self, conf, server_params=None):
if not qpid_messaging:
raise ImportError("Failed to import qpid.messaging")
self.session = None
self.consumers = {}
self.consumer_thread = None
self.proxy_callbacks = []
self.conf = conf
if server_params and 'hostname' in server_params:
# NOTE(russellb) This enables support for cast_to_server.
server_params['qpid_hosts'] = [
'%s:%d' % (server_params['hostname'],
server_params.get('port', 5672))
]
params = {
'qpid_hosts': self.conf.qpid_hosts,
'username': self.conf.qpid_username,
'password': self.conf.qpid_password,
}
params.update(server_params or {})
self.brokers = params['qpid_hosts']
self.username = params['username']
self.password = params['password']
self.connection_create(self.brokers[0])
self.reconnect()
def connection_create(self, broker):
# Create the connection - this does not open the connection
self.connection = qpid_messaging.Connection(broker)
# Check if flags are set and if so set them for the connection
# before we call open
self.connection.username = self.username
self.connection.password = self.password
self.connection.sasl_mechanisms = self.conf.qpid_sasl_mechanisms
# Reconnection is done by self.reconnect()
self.connection.reconnect = False
self.connection.heartbeat = self.conf.qpid_heartbeat
self.connection.transport = self.conf.qpid_protocol
self.connection.tcp_nodelay = self.conf.qpid_tcp_nodelay
def _register_consumer(self, consumer):
self.consumers[str(consumer.get_receiver())] = consumer
def _lookup_consumer(self, receiver):
return self.consumers[str(receiver)]
def reconnect(self):
"""Handles reconnecting and re-establishing sessions and queues."""
attempt = 0
delay = 1
while True:
# Close the session if necessary
if self.connection.opened():
try:
self.connection.close()
except qpid_exceptions.ConnectionError:
pass
broker = self.brokers[attempt % len(self.brokers)]
attempt += 1
try:
self.connection_create(broker)
self.connection.open()
except qpid_exceptions.ConnectionError as e:
msg_dict = dict(e=e, delay=delay)
msg = _("Unable to connect to AMQP server: %(e)s. "
"Sleeping %(delay)s seconds") % msg_dict
LOG.error(msg)
time.sleep(delay)
delay = min(2 * delay, 60)
else:
LOG.info(_('Connected to AMQP server on %s'), broker)
break
self.session = self.connection.session()
if self.consumers:
consumers = self.consumers
self.consumers = {}
for consumer in consumers.itervalues():
consumer.reconnect(self.session)
self._register_consumer(consumer)
LOG.debug(_("Re-established AMQP queues"))
def ensure(self, error_callback, method, *args, **kwargs):
while True:
try:
return method(*args, **kwargs)
except (qpid_exceptions.Empty,
qpid_exceptions.ConnectionError) as e:
if error_callback:
error_callback(e)
self.reconnect()
def close(self):
"""Close/release this connection."""
self.cancel_consumer_thread()
self.wait_on_proxy_callbacks()
try:
self.connection.close()
except Exception:
# NOTE(dripton) Logging exceptions that happen during cleanup just
# causes confusion; there's really nothing useful we can do with
# them.
pass
self.connection = None
def reset(self):
"""Reset a connection so it can be used again."""
self.cancel_consumer_thread()
self.wait_on_proxy_callbacks()
self.session.close()
self.session = self.connection.session()
self.consumers = {}
def declare_consumer(self, consumer_cls, topic, callback):
"""Create a Consumer using the class that was passed in and
add it to our list of consumers
"""
def _connect_error(exc):
log_info = {'topic': topic, 'err_str': str(exc)}
LOG.error(_("Failed to declare consumer for topic '%(topic)s': "
"%(err_str)s") % log_info)
def _declare_consumer():
consumer = consumer_cls(self.conf, self.session, topic, callback)
self._register_consumer(consumer)
return consumer
return self.ensure(_connect_error, _declare_consumer)
def iterconsume(self, limit=None, timeout=None):
"""Return an iterator that will consume from all queues/consumers."""
def _error_callback(exc):
if isinstance(exc, qpid_exceptions.Empty):
LOG.debug(_('Timed out waiting for RPC response: %s') %
str(exc))
raise rpc_common.Timeout()
else:
LOG.exception(_('Failed to consume message from queue: %s') %
str(exc))
def _consume():
nxt_receiver = self.session.next_receiver(timeout=timeout)
try:
self._lookup_consumer(nxt_receiver).consume()
except Exception:
LOG.exception(_("Error processing message. Skipping it."))
for iteration in itertools.count(0):
if limit and iteration >= limit:
raise StopIteration
yield self.ensure(_error_callback, _consume)
def cancel_consumer_thread(self):
"""Cancel a consumer thread."""
if self.consumer_thread is not None:
self.consumer_thread.kill()
try:
self.consumer_thread.wait()
except greenlet.GreenletExit:
pass
self.consumer_thread = None
def wait_on_proxy_callbacks(self):
"""Wait for all proxy callback threads to exit."""
for proxy_cb in self.proxy_callbacks:
proxy_cb.wait()
def publisher_send(self, cls, topic, msg):
"""Send to a publisher based on the publisher class."""
def _connect_error(exc):
log_info = {'topic': topic, 'err_str': str(exc)}
LOG.exception(_("Failed to publish message to topic "
"'%(topic)s': %(err_str)s") % log_info)
def _publisher_send():
publisher = cls(self.conf, self.session, topic)
publisher.send(msg)
return self.ensure(_connect_error, _publisher_send)
def declare_direct_consumer(self, topic, callback):
"""Create a 'direct' queue.
In nova's use, this is generally a msg_id queue used for
responses for call/multicall
"""
self.declare_consumer(DirectConsumer, topic, callback)
def declare_topic_consumer(self, topic, callback=None, queue_name=None,
exchange_name=None):
"""Create a 'topic' consumer."""
self.declare_consumer(functools.partial(TopicConsumer,
name=queue_name,
exchange_name=exchange_name,
),
topic, callback)
def declare_fanout_consumer(self, topic, callback):
"""Create a 'fanout' consumer."""
self.declare_consumer(FanoutConsumer, topic, callback)
def direct_send(self, msg_id, msg):
"""Send a 'direct' message."""
self.publisher_send(DirectPublisher, msg_id, msg)
def topic_send(self, topic, msg, timeout=None):
"""Send a 'topic' message."""
#
# We want to create a message with attributes, e.g. a TTL. We
# don't really need to keep 'msg' in its JSON format any longer
# so let's create an actual qpid message here and get some
# value-add on the go.
#
# WARNING: Request timeout happens to be in the same units as
# qpid's TTL (seconds). If this changes in the future, then this
# will need to be altered accordingly.
#
qpid_message = qpid_messaging.Message(content=msg, ttl=timeout)
self.publisher_send(TopicPublisher, topic, qpid_message)
def fanout_send(self, topic, msg):
"""Send a 'fanout' message."""
self.publisher_send(FanoutPublisher, topic, msg)
def notify_send(self, topic, msg, **kwargs):
"""Send a notify message on a topic."""
self.publisher_send(NotifyPublisher, topic, msg)
def consume(self, limit=None):
"""Consume from all queues/consumers."""
it = self.iterconsume(limit=limit)
while True:
try:
it.next()
except StopIteration:
return
def consume_in_thread(self):
"""Consumer from all queues/consumers in a greenthread."""
@excutils.forever_retry_uncaught_exceptions
def _consumer_thread():
try:
self.consume()
except greenlet.GreenletExit:
return
if self.consumer_thread is None:
self.consumer_thread = eventlet.spawn(_consumer_thread)
return self.consumer_thread
def create_consumer(self, topic, proxy, fanout=False):
"""Create a consumer that calls a method in a proxy object."""
proxy_cb = rpc_amqp.ProxyCallback(
self.conf, proxy,
rpc_amqp.get_connection_pool(self.conf, Connection))
self.proxy_callbacks.append(proxy_cb)
if fanout:
consumer = FanoutConsumer(self.conf, self.session, topic, proxy_cb)
else:
consumer = TopicConsumer(self.conf, self.session, topic, proxy_cb)
self._register_consumer(consumer)
return consumer
def create_worker(self, topic, proxy, pool_name):
"""Create a worker that calls a method in a proxy object."""
proxy_cb = rpc_amqp.ProxyCallback(
self.conf, proxy,
rpc_amqp.get_connection_pool(self.conf, Connection))
self.proxy_callbacks.append(proxy_cb)
consumer = TopicConsumer(self.conf, self.session, topic, proxy_cb,
name=pool_name)
self._register_consumer(consumer)
return consumer
def join_consumer_pool(self, callback, pool_name, topic,
exchange_name=None, ack_on_error=True):
"""Register as a member of a group of consumers for a given topic from
the specified exchange.
Exactly one member of a given pool will receive each message.
A message will be delivered to multiple pools, if more than
one is created.
"""
callback_wrapper = rpc_amqp.CallbackWrapper(
conf=self.conf,
callback=callback,
connection_pool=rpc_amqp.get_connection_pool(self.conf,
Connection),
)
self.proxy_callbacks.append(callback_wrapper)
consumer = TopicConsumer(conf=self.conf,
session=self.session,
topic=topic,
callback=callback_wrapper,
name=pool_name,
exchange_name=exchange_name)
self._register_consumer(consumer)
return consumer
def create_connection(conf, new=True):
"""Create a connection."""
return rpc_amqp.create_connection(
conf, new,
rpc_amqp.get_connection_pool(conf, Connection))
def multicall(conf, context, topic, msg, timeout=None):
"""Make a call that returns multiple times."""
return rpc_amqp.multicall(
conf, context, topic, msg, timeout,
rpc_amqp.get_connection_pool(conf, Connection))
def call(conf, context, topic, msg, timeout=None):
"""Sends a message on a topic and wait for a response."""
return rpc_amqp.call(
conf, context, topic, msg, timeout,
rpc_amqp.get_connection_pool(conf, Connection))
def cast(conf, context, topic, msg):
"""Sends a message on a topic without waiting for a response."""
return rpc_amqp.cast(
conf, context, topic, msg,
rpc_amqp.get_connection_pool(conf, Connection))
def fanout_cast(conf, context, topic, msg):
"""Sends a message on a fanout exchange without waiting for a response."""
return rpc_amqp.fanout_cast(
conf, context, topic, msg,
rpc_amqp.get_connection_pool(conf, Connection))
def cast_to_server(conf, context, server_params, topic, msg):
"""Sends a message on a topic to a specific server."""
return rpc_amqp.cast_to_server(
conf, context, server_params, topic, msg,
rpc_amqp.get_connection_pool(conf, Connection))
def fanout_cast_to_server(conf, context, server_params, topic, msg):
"""Sends a message on a fanout exchange to a specific server."""
return rpc_amqp.fanout_cast_to_server(
conf, context, server_params, topic, msg,
rpc_amqp.get_connection_pool(conf, Connection))
def notify(conf, context, topic, msg, envelope):
"""Sends a notification event on a topic."""
return rpc_amqp.notify(conf, context, topic, msg,
rpc_amqp.get_connection_pool(conf, Connection),
envelope)
def cleanup():
return rpc_amqp.cleanup(Connection.pool)

View File

@ -1,818 +0,0 @@
# vim: tabstop=4 shiftwidth=4 softtabstop=4
# 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.
import os
import pprint
import re
import socket
import sys
import types
import uuid
import eventlet
import greenlet
from oslo.config import cfg
from trove.openstack.common import excutils
from trove.openstack.common.gettextutils import _ # noqa
from trove.openstack.common import importutils
from trove.openstack.common import jsonutils
from trove.openstack.common.rpc import common as rpc_common
zmq = importutils.try_import('eventlet.green.zmq')
# for convenience, are not modified.
pformat = pprint.pformat
Timeout = eventlet.timeout.Timeout
LOG = rpc_common.LOG
RemoteError = rpc_common.RemoteError
RPCException = rpc_common.RPCException
zmq_opts = [
cfg.StrOpt('rpc_zmq_bind_address', default='*',
help='ZeroMQ bind address. Should be a wildcard (*), '
'an ethernet interface, or IP. '
'The "host" option should point or resolve to this '
'address.'),
# The module.Class to use for matchmaking.
cfg.StrOpt(
'rpc_zmq_matchmaker',
default=('trove.openstack.common.rpc.'
'matchmaker.MatchMakerLocalhost'),
help='MatchMaker driver',
),
# The following port is unassigned by IANA as of 2012-05-21
cfg.IntOpt('rpc_zmq_port', default=9501,
help='ZeroMQ receiver listening port'),
cfg.IntOpt('rpc_zmq_contexts', default=1,
help='Number of ZeroMQ contexts, defaults to 1'),
cfg.IntOpt('rpc_zmq_topic_backlog', default=None,
help='Maximum number of ingress messages to locally buffer '
'per topic. Default is unlimited.'),
cfg.StrOpt('rpc_zmq_ipc_dir', default='/var/run/openstack',
help='Directory for holding IPC sockets'),
cfg.StrOpt('rpc_zmq_host', default=socket.gethostname(),
help='Name of this node. Must be a valid hostname, FQDN, or '
'IP address. Must match "host" option, if running Nova.')
]
CONF = cfg.CONF
CONF.register_opts(zmq_opts)
ZMQ_CTX = None # ZeroMQ Context, must be global.
matchmaker = None # memoized matchmaker 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(_("JSON serialization failed."))
def _deserialize(data):
"""Deserialization wrapper."""
LOG.debug(_("Deserializing: %s"), 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):
self.sock = _get_ctxt().socket(zmq_type)
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:
self.sock.setsockopt(zmq.SUBSCRIBE, msg_filter)
except Exception:
return
self.subscriptions.append(msg_filter)
def unsubscribe(self, msg_filter):
"""Unsubscribe."""
if msg_filter not in self.subscriptions:
return
self.sock.setsockopt(zmq.UNSUBSCRIBE, msg_filter)
self.subscriptions.remove(msg_filter)
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()
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):
self.outq = ZmqSocket(addr, zmq.PUSH, bind=False)
def cast(self, msg_id, topic, data, envelope):
msg_id = msg_id or 0
if not envelope:
self.outq.send(map(bytes,
(msg_id, topic, 'cast', _serialize(data))))
return
rpc_envelope = rpc_common.serialize_msg(data[1], envelope)
zmq_msg = reduce(lambda x, y: x + y, rpc_envelope.items())
self.outq.send(map(bytes,
(msg_id, topic, 'impl_zmq_v2', data[0]) + zmq_msg))
def close(self):
self.outq.close()
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):
ctx_data = ctx.to_dict()
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:
result = proxy.dispatch(
ctx, data['version'], data['method'],
data.get('namespace'), **data['args'])
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(_("Exception during message handling"))
return {'exc':
rpc_common.serialize_remote_exception(sys.exc_info())}
def reply(self, 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(_cast, ctx, topic, {
'method': '-process_reply',
'args': {
'msg_id': msg_id, # Include for Folsom compat.
'response': response
}
}, _msg_id=msg_id)
class ConsumerBase(object):
"""Base Consumer."""
def __init__(self):
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')
if not method:
LOG.error(_("RPC message did not include method."))
return
# Internal method
# uses internal context for safety.
if method == '-reply':
self.private_ctx.reply(ctx, proxy, **data['args'])
return
proxy.dispatch(ctx, data['version'],
data['method'], data.get('namespace'), **data['args'])
class ZmqBaseReactor(ConsumerBase):
"""A consumer class implementing a centralized casting broker (PULL-PUSH).
Used for RoundRobin requests.
"""
def __init__(self, conf):
super(ZmqBaseReactor, self).__init__()
self.proxies = {}
self.threads = []
self.sockets = []
self.subscribe = {}
self.pool = eventlet.greenpool.GreenPool(conf.rpc_thread_pool_size)
def register(self, proxy, in_addr, zmq_type_in,
in_bind=True, subscribe=None):
LOG.info(_("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(_("In reactor registered"))
def consume_in_thread(self):
@excutils.forever_retry_uncaught_exceptions
def _consume(sock):
LOG.info(_("Consuming socket"))
while True:
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 s in self.sockets:
s.close()
for t in self.threads:
t.kill()
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 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(_("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(_("Topic socket file creation failed."))
return
try:
self.topic_proxy[topic].put_nowait(data)
except eventlet.queue.Full:
LOG.error(_("Local per-topic backlog buffer full for topic "
"%(topic)s. Dropping message.") % {'topic': 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(_("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(_("Permission denied to IPC directory at"
" %s") % (ipc_dir, ))
with excutils.save_and_reraise_exception():
LOG.error(_("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 = i.next()
h[k] = i.next()
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):
super(ZmqReactor, self).__init__(conf)
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] == 'cast': # Legacy protocol
packenv = data[3]
ctx, msg = _deserialize(packenv)
request = rpc_common.deserialize_msg(msg)
ctx = RpcContext.unmarshal(ctx)
elif data[2] == '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(_("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):
self.topics = []
self.reactor = ZmqReactor(conf)
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(_("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):
_get_matchmaker().stop_heartbeat()
for topic in self.topics:
_get_matchmaker().unregister(topic, CONF.rpc_zmq_host)
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(addr, context, topic, msg, timeout=None, envelope=False,
_msg_id=None):
timeout_cast = timeout or CONF.rpc_cast_timeout
payload = [RpcContext.marshal(context), msg]
with Timeout(timeout_cast, exception=rpc_common.Timeout):
try:
conn = ZmqClient(addr)
# assumes cast can't return an exception
conn.cast(_msg_id, topic, payload, envelope)
except zmq.ZMQError:
raise RPCException("Cast failed. ZMQ Socket Exception")
finally:
if 'conn' in vars():
conn.close()
def _call(addr, context, topic, msg, timeout=None,
envelope=False):
# 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"))
_cast(addr, context, topic, payload, envelope)
LOG.debug(_("Cast sent; Waiting reply"))
# Blocks until receives reply
msg = msg_waiter.recv()
LOG.debug(_("Received message: %s"), msg)
LOG.debug(_("Unpacking response"))
if msg[2] == 'cast': # Legacy version
raw_msg = _deserialize(msg[-1])[-1]
elif msg[2] == '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, types.DictType) and 'exc' in resp:
raise rpc_common.deserialize_remote_exception(CONF, resp['exc'])
return responses[-1]
def _multi_send(method, context, topic, msg, timeout=None,
envelope=False, _msg_id=None):
"""Wraps the sending of messages.
Dispatches to the matchmaker and sends message to all relevant hosts.
"""
conf = CONF
LOG.debug(_("%(msg)s") % {'msg': ' '.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:
LOG.warn(_("No matchmaker results. Not casting."))
# 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(_("No match from matchmaker."))
# This supports brokerless fanout (addresses > 1)
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, _addr, context,
_topic, msg, timeout, envelope,
_msg_id)
return
return method(_addr, context, _topic, msg, timeout,
envelope)
def create_connection(conf, new=True):
return Connection(conf)
def multicall(conf, *args, **kwargs):
"""Multiple calls."""
return _multi_send(_call, *args, **kwargs)
def call(conf, *args, **kwargs):
"""Send a message, expect a response."""
data = _multi_send(_call, *args, **kwargs)
return data[-1]
def cast(conf, *args, **kwargs):
"""Send a message expecting no reply."""
_multi_send(_cast, *args, **kwargs)
def fanout_cast(conf, context, topic, msg, **kwargs):
"""Send a message to all listening and expect no reply."""
# NOTE(ewindisch): fanout~ is used because it avoid splitting on .
# and acts as a non-subtle hint to the matchmaker and ZmqProxy.
_multi_send(_cast, context, 'fanout~' + str(topic), msg, **kwargs)
def notify(conf, context, topic, msg, envelope):
"""Send notification event.
Notifications are sent to topic-priority.
This differs from the AMQP drivers which send to topic.priority.
"""
# NOTE(ewindisch): dot-priority in rpc notifier does not
# work with our assumptions.
topic = topic.replace('.', '-')
cast(conf, context, topic, msg, envelope=envelope)
def cleanup():
"""Clean up resources in use by implementation."""
global ZMQ_CTX
if ZMQ_CTX:
ZMQ_CTX.term()
ZMQ_CTX = None
global matchmaker
matchmaker = None
def _get_ctxt():
if not zmq:
raise ImportError("Failed to import eventlet.green.zmq")
global ZMQ_CTX
if not ZMQ_CTX:
ZMQ_CTX = zmq.Context(CONF.rpc_zmq_contexts)
return ZMQ_CTX
def _get_matchmaker(*args, **kwargs):
global matchmaker
if not matchmaker:
mm = CONF.rpc_zmq_matchmaker
if mm.endswith('matchmaker.MatchMakerRing'):
mm.replace('matchmaker', 'matchmaker_ring')
LOG.warn(_('rpc_zmq_matchmaker = %(orig)s is deprecated; use'
' %(new)s instead') % dict(
orig=CONF.rpc_zmq_matchmaker, new=mm))
matchmaker = importutils.import_object(mm, *args, **kwargs)
return matchmaker

View File

@ -1,324 +0,0 @@
# vim: tabstop=4 shiftwidth=4 softtabstop=4
# 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 eventlet
from oslo.config import cfg
from trove.openstack.common.gettextutils import _ # noqa
from trove.openstack.common import log as logging
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
publically 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:
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
publically 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(_("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())

View File

@ -1,145 +0,0 @@
# vim: tabstop=4 shiftwidth=4 softtabstop=4
# 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 trove.openstack.common import importutils
from trove.openstack.common import log as logging
from trove.openstack.common.rpc 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',
default=None,
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)
LOG = logging.getLogger(__name__)
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(self.topic_host[host], host)
def is_alive(self, topic, host):
if self.redis.ttl(host) == -1:
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.set(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()

View File

@ -1,108 +0,0 @@
# vim: tabstop=4 shiftwidth=4 softtabstop=4
# 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
from oslo.config import cfg
from trove.openstack.common.gettextutils import _ # noqa
from trove.openstack.common import log as logging
from trove.openstack.common.rpc import matchmaker as mm
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(
_("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(
_("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))

View File

@ -1,226 +0,0 @@
# vim: tabstop=4 shiftwidth=4 softtabstop=4
# Copyright 2012-2013 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.
"""
A helper class for proxy objects to remote APIs.
For more information about rpc API version numbers, see:
rpc/dispatcher.py
"""
from trove.openstack.common import rpc
from trove.openstack.common.rpc import common as rpc_common
from trove.openstack.common.rpc import serializer as rpc_serializer
class RpcProxy(object):
"""A helper class for rpc clients.
This class is a wrapper around the RPC client API. It allows you to
specify the topic and API version in a single place. This is intended to
be used as a base class for a class that implements the client side of an
rpc API.
"""
# The default namespace, which can be overriden in a subclass.
RPC_API_NAMESPACE = None
def __init__(self, topic, default_version, version_cap=None,
serializer=None):
"""Initialize an RpcProxy.
:param topic: The topic to use for all messages.
:param default_version: The default API version to request in all
outgoing messages. This can be overridden on a per-message
basis.
:param version_cap: Optionally cap the maximum version used for sent
messages.
:param serializer: Optionaly (de-)serialize entities with a
provided helper.
"""
self.topic = topic
self.default_version = default_version
self.version_cap = version_cap
if serializer is None:
serializer = rpc_serializer.NoOpSerializer()
self.serializer = serializer
super(RpcProxy, self).__init__()
def _set_version(self, msg, vers):
"""Helper method to set the version in a message.
:param msg: The message having a version added to it.
:param vers: The version number to add to the message.
"""
v = vers if vers else self.default_version
if (self.version_cap and not
rpc_common.version_is_compatible(self.version_cap, v)):
raise rpc_common.RpcVersionCapError(version_cap=self.version_cap)
msg['version'] = v
def _get_topic(self, topic):
"""Return the topic to use for a message."""
return topic if topic else self.topic
def can_send_version(self, version):
"""Check to see if a version is compatible with the version cap."""
return (not self.version_cap or
rpc_common.version_is_compatible(self.version_cap, version))
@staticmethod
def make_namespaced_msg(method, namespace, **kwargs):
return {'method': method, 'namespace': namespace, 'args': kwargs}
def make_msg(self, method, **kwargs):
return self.make_namespaced_msg(method, self.RPC_API_NAMESPACE,
**kwargs)
def _serialize_msg_args(self, context, kwargs):
"""Helper method called to serialize message arguments.
This calls our serializer on each argument, returning a new
set of args that have been serialized.
:param context: The request context
:param kwargs: The arguments to serialize
:returns: A new set of serialized arguments
"""
new_kwargs = dict()
for argname, arg in kwargs.iteritems():
new_kwargs[argname] = self.serializer.serialize_entity(context,
arg)
return new_kwargs
def call(self, context, msg, topic=None, version=None, timeout=None):
"""rpc.call() a remote method.
:param context: The request context
:param msg: The message to send, including the method and args.
:param topic: Override the topic for this message.
:param version: (Optional) Override the requested API version in this
message.
:param timeout: (Optional) A timeout to use when waiting for the
response. If no timeout is specified, a default timeout will be
used that is usually sufficient.
:returns: The return value from the remote method.
"""
self._set_version(msg, version)
msg['args'] = self._serialize_msg_args(context, msg['args'])
real_topic = self._get_topic(topic)
try:
result = rpc.call(context, real_topic, msg, timeout)
return self.serializer.deserialize_entity(context, result)
except rpc.common.Timeout as exc:
raise rpc.common.Timeout(
exc.info, real_topic, msg.get('method'))
def multicall(self, context, msg, topic=None, version=None, timeout=None):
"""rpc.multicall() a remote method.
:param context: The request context
:param msg: The message to send, including the method and args.
:param topic: Override the topic for this message.
:param version: (Optional) Override the requested API version in this
message.
:param timeout: (Optional) A timeout to use when waiting for the
response. If no timeout is specified, a default timeout will be
used that is usually sufficient.
:returns: An iterator that lets you process each of the returned values
from the remote method as they arrive.
"""
self._set_version(msg, version)
msg['args'] = self._serialize_msg_args(context, msg['args'])
real_topic = self._get_topic(topic)
try:
result = rpc.multicall(context, real_topic, msg, timeout)
return self.serializer.deserialize_entity(context, result)
except rpc.common.Timeout as exc:
raise rpc.common.Timeout(
exc.info, real_topic, msg.get('method'))
def cast(self, context, msg, topic=None, version=None):
"""rpc.cast() a remote method.
:param context: The request context
:param msg: The message to send, including the method and args.
:param topic: Override the topic for this message.
:param version: (Optional) Override the requested API version in this
message.
:returns: None. rpc.cast() does not wait on any return value from the
remote method.
"""
self._set_version(msg, version)
msg['args'] = self._serialize_msg_args(context, msg['args'])
rpc.cast(context, self._get_topic(topic), msg)
def fanout_cast(self, context, msg, topic=None, version=None):
"""rpc.fanout_cast() a remote method.
:param context: The request context
:param msg: The message to send, including the method and args.
:param topic: Override the topic for this message.
:param version: (Optional) Override the requested API version in this
message.
:returns: None. rpc.fanout_cast() does not wait on any return value
from the remote method.
"""
self._set_version(msg, version)
msg['args'] = self._serialize_msg_args(context, msg['args'])
rpc.fanout_cast(context, self._get_topic(topic), msg)
def cast_to_server(self, context, server_params, msg, topic=None,
version=None):
"""rpc.cast_to_server() a remote method.
:param context: The request context
:param server_params: Server parameters. See rpc.cast_to_server() for
details.
:param msg: The message to send, including the method and args.
:param topic: Override the topic for this message.
:param version: (Optional) Override the requested API version in this
message.
:returns: None. rpc.cast_to_server() does not wait on any
return values.
"""
self._set_version(msg, version)
msg['args'] = self._serialize_msg_args(context, msg['args'])
rpc.cast_to_server(context, server_params, self._get_topic(topic), msg)
def fanout_cast_to_server(self, context, server_params, msg, topic=None,
version=None):
"""rpc.fanout_cast_to_server() a remote method.
:param context: The request context
:param server_params: Server parameters. See rpc.cast_to_server() for
details.
:param msg: The message to send, including the method and args.
:param topic: Override the topic for this message.
:param version: (Optional) Override the requested API version in this
message.
:returns: None. rpc.fanout_cast_to_server() does not wait on any
return values.
"""
self._set_version(msg, version)
msg['args'] = self._serialize_msg_args(context, msg['args'])
rpc.fanout_cast_to_server(context, server_params,
self._get_topic(topic), msg)

View File

@ -1,521 +0,0 @@
# vim: tabstop=4 shiftwidth=4 softtabstop=4
# Copyright 2013 Red Hat, Inc.
#
# Licensed under the Apache License, Version 2.0 (the "License"); you may
# not use this file except in compliance with the License. You may obtain
# a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
# License for the specific language governing permissions and limitations
# under the License.
import base64
import collections
import os
import struct
import time
import requests
from oslo.config import cfg
from trove.openstack.common.crypto import utils as cryptoutils
from trove.openstack.common import jsonutils
from trove.openstack.common import log as logging
secure_message_opts = [
cfg.BoolOpt('enabled', default=True,
help='Whether Secure Messaging (Signing) is enabled,'
' defaults to enabled'),
cfg.BoolOpt('enforced', default=False,
help='Whether Secure Messaging (Signing) is enforced,'
' defaults to not enforced'),
cfg.BoolOpt('encrypt', default=False,
help='Whether Secure Messaging (Encryption) is enabled,'
' defaults to not enabled'),
cfg.StrOpt('secret_keys_file',
help='Path to the file containing the keys, takes precedence'
' over secret_key'),
cfg.MultiStrOpt('secret_key',
help='A list of keys: (ex: name:<base64 encoded key>),'
' ignored if secret_keys_file is set'),
cfg.StrOpt('kds_endpoint',
help='KDS endpoint (ex: http://kds.example.com:35357/v3)'),
]
secure_message_group = cfg.OptGroup('secure_messages',
title='Secure Messaging options')
LOG = logging.getLogger(__name__)
class SecureMessageException(Exception):
"""Generic Exception for Secure Messages."""
msg = "An unknown Secure Message related exception occurred."
def __init__(self, msg=None):
if msg is None:
msg = self.msg
super(SecureMessageException, self).__init__(msg)
class SharedKeyNotFound(SecureMessageException):
"""No shared key was found and no other external authentication mechanism
is available.
"""
msg = "Shared Key for [%s] Not Found. (%s)"
def __init__(self, name, errmsg):
super(SharedKeyNotFound, self).__init__(self.msg % (name, errmsg))
class InvalidMetadata(SecureMessageException):
"""The metadata is invalid."""
msg = "Invalid metadata: %s"
def __init__(self, err):
super(InvalidMetadata, self).__init__(self.msg % err)
class InvalidSignature(SecureMessageException):
"""Signature validation failed."""
msg = "Failed to validate signature (source=%s, destination=%s)"
def __init__(self, src, dst):
super(InvalidSignature, self).__init__(self.msg % (src, dst))
class UnknownDestinationName(SecureMessageException):
"""The Destination name is unknown to us."""
msg = "Invalid destination name (%s)"
def __init__(self, name):
super(UnknownDestinationName, self).__init__(self.msg % name)
class InvalidEncryptedTicket(SecureMessageException):
"""The Encrypted Ticket could not be successfully handled."""
msg = "Invalid Ticket (source=%s, destination=%s)"
def __init__(self, src, dst):
super(InvalidEncryptedTicket, self).__init__(self.msg % (src, dst))
class InvalidExpiredTicket(SecureMessageException):
"""The ticket received is already expired."""
msg = "Expired ticket (source=%s, destination=%s)"
def __init__(self, src, dst):
super(InvalidExpiredTicket, self).__init__(self.msg % (src, dst))
class CommunicationError(SecureMessageException):
"""The Communication with the KDS failed."""
msg = "Communication Error (target=%s): %s"
def __init__(self, target, errmsg):
super(CommunicationError, self).__init__(self.msg % (target, errmsg))
class InvalidArgument(SecureMessageException):
"""Bad initialization argument."""
msg = "Invalid argument: %s"
def __init__(self, errmsg):
super(InvalidArgument, self).__init__(self.msg % errmsg)
Ticket = collections.namedtuple('Ticket', ['skey', 'ekey', 'esek'])
class KeyStore(object):
"""A storage class for Signing and Encryption Keys.
This class creates an object that holds Generic Keys like Signing
Keys, Encryption Keys, Encrypted SEK Tickets ...
"""
def __init__(self):
self._kvps = dict()
def _get_key_name(self, source, target, ktype):
return (source, target, ktype)
def _put(self, src, dst, ktype, expiration, data):
name = self._get_key_name(src, dst, ktype)
self._kvps[name] = (expiration, data)
def _get(self, src, dst, ktype):
name = self._get_key_name(src, dst, ktype)
if name in self._kvps:
expiration, data = self._kvps[name]
if expiration > time.time():
return data
else:
del self._kvps[name]
return None
def clear(self):
"""Wipes the store clear of all data."""
self._kvps.clear()
def put_ticket(self, source, target, skey, ekey, esek, expiration):
"""Puts a sek pair in the cache.
:param source: Client name
:param target: Target name
:param skey: The Signing Key
:param ekey: The Encription Key
:param esek: The token encrypted with the target key
:param expiration: Expiration time in seconds since Epoch
"""
keys = Ticket(skey, ekey, esek)
self._put(source, target, 'ticket', expiration, keys)
def get_ticket(self, source, target):
"""Returns a Ticket (skey, ekey, esek) namedtuple for the
source/target pair.
"""
return self._get(source, target, 'ticket')
_KEY_STORE = KeyStore()
class _KDSClient(object):
USER_AGENT = 'oslo-incubator/rpc'
def __init__(self, endpoint=None, timeout=None):
"""A KDS Client class."""
self._endpoint = endpoint
if timeout is not None:
self.timeout = float(timeout)
else:
self.timeout = None
def _do_get(self, url, request):
req_kwargs = dict()
req_kwargs['headers'] = dict()
req_kwargs['headers']['User-Agent'] = self.USER_AGENT
req_kwargs['headers']['Content-Type'] = 'application/json'
req_kwargs['data'] = jsonutils.dumps({'request': request})
if self.timeout is not None:
req_kwargs['timeout'] = self.timeout
try:
resp = requests.get(url, **req_kwargs)
except requests.ConnectionError as e:
err = "Unable to establish connection. %s" % e
raise CommunicationError(url, err)
return resp
def _get_reply(self, url, resp):
if resp.text:
try:
body = jsonutils.loads(resp.text)
reply = body['reply']
except (KeyError, TypeError, ValueError):
msg = "Failed to decode reply: %s" % resp.text
raise CommunicationError(url, msg)
else:
msg = "No reply data was returned."
raise CommunicationError(url, msg)
return reply
def _get_ticket(self, request, url=None, redirects=10):
"""Send an HTTP request.
Wraps around 'requests' to handle redirects and common errors.
"""
if url is None:
if not self._endpoint:
raise CommunicationError(url, 'Endpoint not configured')
url = self._endpoint + '/kds/ticket'
while redirects:
resp = self._do_get(url, request)
if resp.status_code in (301, 302, 305):
# Redirected. Reissue the request to the new location.
url = resp.headers['location']
redirects -= 1
continue
elif resp.status_code != 200:
msg = "Request returned failure status: %s (%s)"
err = msg % (resp.status_code, resp.text)
raise CommunicationError(url, err)
return self._get_reply(url, resp)
raise CommunicationError(url, "Too many redirections, giving up!")
def get_ticket(self, source, target, crypto, key):
# prepare metadata
md = {'requestor': source,
'target': target,
'timestamp': time.time(),
'nonce': struct.unpack('Q', os.urandom(8))[0]}
metadata = base64.b64encode(jsonutils.dumps(md))
# sign metadata
signature = crypto.sign(key, metadata)
# HTTP request
reply = self._get_ticket({'metadata': metadata,
'signature': signature})
# verify reply
signature = crypto.sign(key, (reply['metadata'] + reply['ticket']))
if signature != reply['signature']:
raise InvalidEncryptedTicket(md['source'], md['destination'])
md = jsonutils.loads(base64.b64decode(reply['metadata']))
if ((md['source'] != source or
md['destination'] != target or
md['expiration'] < time.time())):
raise InvalidEncryptedTicket(md['source'], md['destination'])
# return ticket data
tkt = jsonutils.loads(crypto.decrypt(key, reply['ticket']))
return tkt, md['expiration']
# we need to keep a global nonce, as this value should never repeat non
# matter how many SecureMessage objects we create
_NONCE = None
def _get_nonce():
"""We keep a single counter per instance, as it is so huge we can't
possibly cycle through within 1/100 of a second anyway.
"""
global _NONCE
# Lazy initialize, for now get a random value, multiply by 2^32 and
# use it as the nonce base. The counter itself will rotate after
# 2^32 increments.
if _NONCE is None:
_NONCE = [struct.unpack('I', os.urandom(4))[0], 0]
# Increment counter and wrap at 2^32
_NONCE[1] += 1
if _NONCE[1] > 0xffffffff:
_NONCE[1] = 0
# Return base + counter
return long((_NONCE[0] * 0xffffffff)) + _NONCE[1]
class SecureMessage(object):
"""A Secure Message object.
This class creates a signing/encryption facility for RPC messages.
It encapsulates all the necessary crypto primitives to insulate
regular code from the intricacies of message authentication, validation
and optionally encryption.
:param topic: The topic name of the queue
:param host: The server name, together with the topic it forms a unique
name that is used to source signing keys, and verify
incoming messages.
:param conf: a ConfigOpts object
:param key: (optional) explicitly pass in endpoint private key.
If not provided it will be sourced from the service config
:param key_store: (optional) Storage class for local caching
:param encrypt: (defaults to False) Whether to encrypt messages
:param enctype: (defaults to AES) Cipher to use
:param hashtype: (defaults to SHA256) Hash function to use for signatures
"""
def __init__(self, topic, host, conf, key=None, key_store=None,
encrypt=None, enctype='AES', hashtype='SHA256'):
conf.register_group(secure_message_group)
conf.register_opts(secure_message_opts, group='secure_messages')
self._name = '%s.%s' % (topic, host)
self._key = key
self._conf = conf.secure_messages
self._encrypt = self._conf.encrypt if (encrypt is None) else encrypt
self._crypto = cryptoutils.SymmetricCrypto(enctype, hashtype)
self._hkdf = cryptoutils.HKDF(hashtype)
self._kds = _KDSClient(self._conf.kds_endpoint)
if self._key is None:
self._key = self._init_key(topic, self._name)
if self._key is None:
err = "Secret Key (or key file) is missing or malformed"
raise SharedKeyNotFound(self._name, err)
self._key_store = key_store or _KEY_STORE
def _init_key(self, topic, name):
keys = None
if self._conf.secret_keys_file:
with open(self._conf.secret_keys_file, 'r') as f:
keys = f.readlines()
elif self._conf.secret_key:
keys = self._conf.secret_key
if keys is None:
return None
for k in keys:
if k[0] == '#':
continue
if ':' not in k:
break
svc, key = k.split(':', 1)
if svc == topic or svc == name:
return base64.b64decode(key)
return None
def _split_key(self, key, size):
sig_key = key[:size]
enc_key = key[size:]
return sig_key, enc_key
def _decode_esek(self, key, source, target, timestamp, esek):
"""This function decrypts the esek buffer passed in and returns a
KeyStore to be used to check and decrypt the received message.
:param key: The key to use to decrypt the ticket (esek)
:param source: The name of the source service
:param traget: The name of the target service
:param timestamp: The incoming message timestamp
:param esek: a base64 encoded encrypted block containing a JSON string
"""
rkey = None
try:
s = self._crypto.decrypt(key, esek)
j = jsonutils.loads(s)
rkey = base64.b64decode(j['key'])
expiration = j['timestamp'] + j['ttl']
if j['timestamp'] > timestamp or timestamp > expiration:
raise InvalidExpiredTicket(source, target)
except Exception:
raise InvalidEncryptedTicket(source, target)
info = '%s,%s,%s' % (source, target, str(j['timestamp']))
sek = self._hkdf.expand(rkey, info, len(key) * 2)
return self._split_key(sek, len(key))
def _get_ticket(self, target):
"""This function will check if we already have a SEK for the specified
target in the cache, or will go and try to fetch a new SEK from the key
server.
:param target: The name of the target service
"""
ticket = self._key_store.get_ticket(self._name, target)
if ticket is not None:
return ticket
tkt, expiration = self._kds.get_ticket(self._name, target,
self._crypto, self._key)
self._key_store.put_ticket(self._name, target,
base64.b64decode(tkt['skey']),
base64.b64decode(tkt['ekey']),
tkt['esek'], expiration)
return self._key_store.get_ticket(self._name, target)
def encode(self, version, target, json_msg):
"""This is the main encoding function.
It takes a target and a message and returns a tuple consisting of a
JSON serialized metadata object, a JSON serialized (and optionally
encrypted) message, and a signature.
:param version: the current envelope version
:param target: The name of the target service (usually with hostname)
:param json_msg: a serialized json message object
"""
ticket = self._get_ticket(target)
metadata = jsonutils.dumps({'source': self._name,
'destination': target,
'timestamp': time.time(),
'nonce': _get_nonce(),
'esek': ticket.esek,
'encryption': self._encrypt})
message = json_msg
if self._encrypt:
message = self._crypto.encrypt(ticket.ekey, message)
signature = self._crypto.sign(ticket.skey,
version + metadata + message)
return (metadata, message, signature)
def decode(self, version, metadata, message, signature):
"""This is the main decoding function.
It takes a version, metadata, message and signature strings and
returns a tuple with a (decrypted) message and metadata or raises
an exception in case of error.
:param version: the current envelope version
:param metadata: a JSON serialized object with metadata for validation
:param message: a JSON serialized (base64 encoded encrypted) message
:param signature: a base64 encoded signature
"""
md = jsonutils.loads(metadata)
check_args = ('source', 'destination', 'timestamp',
'nonce', 'esek', 'encryption')
for arg in check_args:
if arg not in md:
raise InvalidMetadata('Missing metadata "%s"' % arg)
if md['destination'] != self._name:
# TODO(simo) handle group keys by checking target
raise UnknownDestinationName(md['destination'])
try:
skey, ekey = self._decode_esek(self._key,
md['source'], md['destination'],
md['timestamp'], md['esek'])
except InvalidExpiredTicket:
raise
except Exception:
raise InvalidMetadata('Failed to decode ESEK for %s/%s' % (
md['source'], md['destination']))
sig = self._crypto.sign(skey, version + metadata + message)
if sig != signature:
raise InvalidSignature(md['source'], md['destination'])
if md['encryption'] is True:
msg = self._crypto.decrypt(ekey, message)
else:
msg = message
return (md, msg)

View File

@ -1,52 +0,0 @@
# Copyright 2013 IBM Corp.
#
# Licensed under the Apache License, Version 2.0 (the "License"); you may
# not use this file except in compliance with the License. You may obtain
# a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
# License for the specific language governing permissions and limitations
# under the License.
"""Provides the definition of an RPC serialization handler"""
import abc
class Serializer(object):
"""Generic (de-)serialization definition base class."""
__metaclass__ = abc.ABCMeta
@abc.abstractmethod
def serialize_entity(self, context, entity):
"""Serialize something to primitive form.
:param context: Security context
:param entity: Entity to be serialized
:returns: Serialized form of entity
"""
pass
@abc.abstractmethod
def deserialize_entity(self, context, entity):
"""Deserialize something from primitive form.
:param context: Security context
:param entity: Primitive to be deserialized
:returns: Deserialized form of entity
"""
pass
class NoOpSerializer(Serializer):
"""A serializer that does nothing."""
def serialize_entity(self, context, entity):
return entity
def deserialize_entity(self, context, entity):
return entity

View File

@ -1,78 +0,0 @@
# vim: tabstop=4 shiftwidth=4 softtabstop=4
# Copyright 2010 United States Government as represented by the
# Administrator of the National Aeronautics and Space Administration.
# All Rights Reserved.
# Copyright 2011 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.
from trove.openstack.common.gettextutils import _ # noqa
from trove.openstack.common import log as logging
from trove.openstack.common import rpc
from trove.openstack.common.rpc import dispatcher as rpc_dispatcher
from trove.openstack.common import service
LOG = logging.getLogger(__name__)
class Service(service.Service):
"""Service object for binaries running on hosts.
A service enables rpc by listening to queues based on topic and host.
"""
def __init__(self, host, topic, manager=None, serializer=None):
super(Service, self).__init__()
self.host = host
self.topic = topic
self.serializer = serializer
if manager is None:
self.manager = self
else:
self.manager = manager
def start(self):
super(Service, self).start()
self.conn = rpc.create_connection(new=True)
LOG.debug(_("Creating Consumer connection for Service %s") %
self.topic)
dispatcher = rpc_dispatcher.RpcDispatcher([self.manager],
self.serializer)
# Share this same connection for these Consumers
self.conn.create_consumer(self.topic, dispatcher, fanout=False)
node_topic = '%s.%s' % (self.topic, self.host)
self.conn.create_consumer(node_topic, dispatcher, fanout=False)
self.conn.create_consumer(self.topic, dispatcher, fanout=True)
# Hook to allow the manager to do other initializations after
# the rpc connection is created.
if callable(getattr(self.manager, 'initialize_service_hook', None)):
self.manager.initialize_service_hook(self)
# Consume from all consumers in a thread
self.conn.consume_in_thread()
def stop(self):
# Try to shut the connection down, but if we get any sort of
# errors, go ahead and ignore them.. as we're shutting down anyway
try:
self.conn.close()
except Exception:
pass
super(Service, self).stop()

View File

@ -1,40 +0,0 @@
# vim: tabstop=4 shiftwidth=4 softtabstop=4
# 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 trove.openstack.common import log as logging
from trove.openstack.common import rpc
from trove.openstack.common.rpc import impl_zmq
CONF = cfg.CONF
CONF.register_opts(rpc.rpc_opts)
CONF.register_opts(impl_zmq.zmq_opts)
def main():
CONF(sys.argv[1:], project='oslo')
logging.setup("oslo")
with contextlib.closing(impl_zmq.ZmqProxy(CONF)) as reactor:
reactor.consume_in_thread()
reactor.wait()

View File

@ -39,13 +39,11 @@ from oslo.config import cfg
from trove.openstack.common import eventlet_backdoor
from trove.openstack.common.gettextutils import _LE, _LI, _LW
from trove.openstack.common import importutils
from trove.openstack.common import log as logging
from trove.openstack.common import systemd
from trove.openstack.common import threadgroup
rpc = importutils.try_import('trove.openstack.common.rpc')
CONF = cfg.CONF
LOG = logging.getLogger(__name__)
@ -180,12 +178,6 @@ class ServiceLauncher(Launcher):
status = exc.code
finally:
self.stop()
if rpc:
try:
rpc.cleanup()
except Exception:
# We're shutting down, so it doesn't matter at this point.
LOG.exception(_LE('Exception during rpc cleanup.'))
return status, signo

160
trove/rpc.py Normal file
View File

@ -0,0 +1,160 @@
# Copyright 2014 OpenStack Foundation
# 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.
# NOTE(esp): This code was taken from nova
__all__ = [
'init',
'cleanup',
'set_defaults',
'add_extra_exmods',
'clear_extra_exmods',
'get_allowed_exmods',
'RequestContextSerializer',
'get_client',
'get_server',
'get_notifier',
'TRANSPORT_ALIASES',
]
from oslo.config import cfg
from oslo import messaging
from trove.common.context import TroveContext
import trove.common.exception
from trove.openstack.common import jsonutils
CONF = cfg.CONF
TRANSPORT = None
NOTIFIER = None
ALLOWED_EXMODS = [
trove.common.exception.__name__,
]
EXTRA_EXMODS = []
# TODO(esp): Remove or update these paths
TRANSPORT_ALIASES = {
'trove.openstack.common.rpc.impl_kombu': 'rabbit',
'trove.openstack.common.rpc.impl_qpid': 'qpid',
'trove.openstack.common.rpc.impl_zmq': 'zmq',
'trove.rpc.impl_kombu': 'rabbit',
'trove.rpc.impl_qpid': 'qpid',
'trove.rpc.impl_zmq': 'zmq',
}
def init(conf):
global TRANSPORT, NOTIFIER
exmods = get_allowed_exmods()
TRANSPORT = messaging.get_transport(conf,
allowed_remote_exmods=exmods,
aliases=TRANSPORT_ALIASES)
#serializer = RequestContextSerializer(JsonPayloadSerializer())
# https://review.openstack.org/#/c/71532/1/nova/rpc.py
NOTIFIER = messaging.Notifier(TRANSPORT, serializer=None)
def cleanup():
global TRANSPORT, NOTIFIER
assert TRANSPORT is not None
assert NOTIFIER is not None
TRANSPORT.cleanup()
TRANSPORT = NOTIFIER = None
def set_defaults(control_exchange):
messaging.set_transport_defaults(control_exchange)
def add_extra_exmods(*args):
EXTRA_EXMODS.extend(args)
def clear_extra_exmods():
del EXTRA_EXMODS[:]
def get_allowed_exmods():
return ALLOWED_EXMODS + EXTRA_EXMODS
class JsonPayloadSerializer(messaging.NoOpSerializer):
@staticmethod
def serialize_entity(context, entity):
return jsonutils.to_primitive(entity, convert_instances=True)
class RequestContextSerializer(messaging.Serializer):
def __init__(self, base):
self._base = base
def serialize_entity(self, context, entity):
if not self._base:
return entity
return self._base.serialize_entity(context, entity)
def deserialize_entity(self, context, entity):
if not self._base:
return entity
return self._base.deserialize_entity(context, entity)
def serialize_context(self, context):
return context.to_dict()
def deserialize_context(self, context):
return TroveContext.from_dict(context)
def get_transport_url(url_str=None):
return messaging.TransportURL.parse(CONF, url_str, TRANSPORT_ALIASES)
def get_client(target, version_cap=None, serializer=None):
assert TRANSPORT is not None
serializer = RequestContextSerializer(serializer)
return messaging.RPCClient(TRANSPORT,
target,
version_cap=version_cap,
serializer=serializer)
def get_server(target, endpoints, serializer=None):
assert TRANSPORT is not None
# Thread module is not monkeypatched if remote debugging is enabled.
# Using eventlet executor without monkepatching thread module will
# lead to unpredictable results.
from trove.common import debug_utils
debug_utils.setup()
executor = "blocking" if debug_utils.enabled() else "eventlet"
serializer = RequestContextSerializer(serializer)
return messaging.get_rpc_server(TRANSPORT,
target,
endpoints,
executor=executor,
serializer=serializer)
def get_notifier(service=None, host=None, publisher_id=None):
assert NOTIFIER is not None
if not publisher_id:
publisher_id = "%s.%s" % (service, host or CONF.host)
return NOTIFIER.prepare(publisher_id=publisher_id)

View File

@ -18,27 +18,37 @@
Routes all the requests to the task manager.
"""
from oslo import messaging
from trove.common import cfg
from trove.common import exception
from trove.common import strategy
import trove.common.rpc.version as rpc_version
from trove.guestagent import models as agent_models
from trove.openstack.common.rpc import proxy
from trove import rpc
from trove.openstack.common import log as logging
CONF = cfg.CONF
LOG = logging.getLogger(__name__)
RPC_API_VERSION = "1.0"
class API(proxy.RpcProxy):
class API(object):
"""API for interacting with the task manager."""
def __init__(self, context):
self.context = context
super(API, self).__init__(self._get_routing_key(),
RPC_API_VERSION)
super(API, self).__init__()
target = messaging.Target(topic=CONF.taskmanager_queue,
version=rpc_version.RPC_API_VERSION)
self.version_cap = rpc_version.VERSION_ALIASES.get(
CONF.upgrade_levels.taskmanager)
self.client = self.get_client(target, self.version_cap)
def get_client(self, target, version_cap, serializer=None):
return rpc.get_client(target,
version_cap=version_cap,
serializer=serializer)
def _transform_obj(self, obj_ref):
# Turn the object into a dictionary and remove the mgr
@ -50,10 +60,6 @@ class API(proxy.RpcProxy):
return obj_dict
raise ValueError("Could not transform %s" % obj_ref)
def _get_routing_key(self):
"""Create the routing key for the taskmanager"""
return CONF.taskmanager_queue
def _delete_heartbeat(self, instance_id):
agent_heart_beat = agent_models.AgentHeartBeat()
try:
@ -65,56 +71,66 @@ class API(proxy.RpcProxy):
def resize_volume(self, new_size, instance_id):
LOG.debug("Making async call to resize volume for instance: %s"
% instance_id)
self.cast(self.context, self.make_msg("resize_volume",
new_size=new_size,
instance_id=instance_id))
cctxt = self.client.prepare(version=self.version_cap)
cctxt.cast(self.context, "resize_volume",
new_size=new_size,
instance_id=instance_id)
def resize_flavor(self, instance_id, old_flavor, new_flavor):
LOG.debug("Making async call to resize flavor for instance: %s" %
instance_id)
self.cast(self.context,
self.make_msg("resize_flavor",
instance_id=instance_id,
old_flavor=self._transform_obj(old_flavor),
new_flavor=self._transform_obj(new_flavor)))
cctxt = self.client.prepare(version=self.version_cap)
cctxt.cast(self.context, "resize_flavor",
instance_id=instance_id,
old_flavor=self._transform_obj(old_flavor),
new_flavor=self._transform_obj(new_flavor))
def reboot(self, instance_id):
LOG.debug("Making async call to reboot instance: %s" % instance_id)
self.cast(self.context,
self.make_msg("reboot", instance_id=instance_id))
cctxt = self.client.prepare(version=self.version_cap)
cctxt.cast(self.context, "reboot", instance_id=instance_id)
def restart(self, instance_id):
LOG.debug("Making async call to restart instance: %s" % instance_id)
self.cast(self.context,
self.make_msg("restart", instance_id=instance_id))
cctxt = self.client.prepare(version=self.version_cap)
cctxt.cast(self.context, "restart", instance_id=instance_id)
def detach_replica(self, instance_id):
LOG.debug("Making async call to detach replica: %s" % instance_id)
self.cast(self.context,
self.make_msg("detach_replica", instance_id=instance_id))
cctxt = self.client.prepare(version=self.version_cap)
cctxt.cast(self.context, "detach_replica", instance_id=instance_id)
def migrate(self, instance_id, host):
LOG.debug("Making async call to migrate instance: %s" % instance_id)
self.cast(self.context,
self.make_msg("migrate", instance_id=instance_id, host=host))
cctxt = self.client.prepare(version=self.version_cap)
cctxt.cast(self.context, "migrate", instance_id=instance_id, host=host)
def delete_instance(self, instance_id):
LOG.debug("Making async call to delete instance: %s" % instance_id)
self.cast(self.context,
self.make_msg("delete_instance", instance_id=instance_id))
self._delete_heartbeat(instance_id)
cctxt = self.client.prepare(version=self.version_cap)
cctxt.cast(self.context, "delete_instance", instance_id=instance_id)
def create_backup(self, backup_info, instance_id):
LOG.debug("Making async call to create a backup for instance: %s" %
instance_id)
self.cast(self.context, self.make_msg("create_backup",
backup_info=backup_info,
instance_id=instance_id))
cctxt = self.client.prepare(version=self.version_cap)
cctxt.cast(self.context, "create_backup",
backup_info=backup_info,
instance_id=instance_id)
def delete_backup(self, backup_id):
LOG.debug("Making async call to delete backup: %s" % backup_id)
self.cast(self.context, self.make_msg("delete_backup",
backup_id=backup_id))
cctxt = self.client.prepare(version=self.version_cap)
cctxt.cast(self.context, "delete_backup", backup_id=backup_id)
def create_instance(self, instance_id, name, flavor,
image_id, databases, users, datastore_manager,
@ -124,55 +140,57 @@ class API(proxy.RpcProxy):
cluster_config=None):
LOG.debug("Making async call to create instance %s " % instance_id)
self.cast(self.context,
self.make_msg("create_instance",
instance_id=instance_id, name=name,
flavor=self._transform_obj(flavor),
image_id=image_id,
databases=databases,
users=users,
datastore_manager=
datastore_manager,
packages=packages,
volume_size=volume_size,
backup_id=backup_id,
availability_zone=availability_zone,
root_password=root_password,
nics=nics,
overrides=overrides,
slave_of_id=slave_of_id,
cluster_config=cluster_config))
cctxt = self.client.prepare(version=self.version_cap)
cctxt.cast(self.context, "create_instance",
instance_id=instance_id, name=name,
flavor=self._transform_obj(flavor),
image_id=image_id,
databases=databases,
users=users,
datastore_manager=datastore_manager,
packages=packages,
volume_size=volume_size,
backup_id=backup_id,
availability_zone=availability_zone,
root_password=root_password,
nics=nics,
overrides=overrides,
slave_of_id=slave_of_id,
cluster_config=cluster_config)
def update_overrides(self, instance_id, overrides=None):
LOG.debug("Making async call to update datastore configurations for "
"instance %s" % instance_id)
self.cast(self.context,
self.make_msg("update_overrides",
instance_id=instance_id,
overrides=overrides))
cctxt = self.client.prepare(version=self.version_cap)
cctxt.cast(self.context, "update_overrides",
instance_id=instance_id,
overrides=overrides)
def unassign_configuration(self, instance_id, flavor, configuration_id):
LOG.debug("Making async call to remove datastore configurations for "
"instance %s" % instance_id)
self.cast(self.context,
self.make_msg("unassign_configuration",
instance_id=instance_id,
flavor=self._transform_obj(flavor),
configuration_id=configuration_id))
cctxt = self.client.prepare(version=self.version_cap)
cctxt.cast(self.context, "unassign_configuration",
instance_id=instance_id,
flavor=self._transform_obj(flavor),
configuration_id=configuration_id)
def create_cluster(self, cluster_id):
LOG.debug("Making async call to create cluster %s " % cluster_id)
self.cast(self.context,
self.make_msg("create_cluster",
cluster_id=cluster_id))
cctxt = self.client.prepare(version=self.version_cap)
cctxt.cast(self.context, "create_cluster",
cluster_id=cluster_id)
def delete_cluster(self, cluster_id):
LOG.debug("Making async call to delete cluster %s " % cluster_id)
self.cast(self.context,
self.make_msg("delete_cluster",
cluster_id=cluster_id))
cctxt = self.client.prepare(version=self.version_cap)
cctxt.cast(self.context, "delete_cluster",
cluster_id=cluster_id)
def load(context, manager=None):

View File

@ -12,12 +12,14 @@
# 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 import messaging
from trove.common.context import TroveContext
from oslo.utils import importutils
from trove.backup.models import Backup
import trove.common.cfg as cfg
import trove.common.rpc.version as rpc_version
from trove.common import exception
from trove.common import strategy
import trove.extensions.mgmt.instances.models as mgmtmodels
@ -27,12 +29,13 @@ from trove.taskmanager import models
from trove.taskmanager.models import FreshInstanceTasks
LOG = logging.getLogger(__name__)
RPC_API_VERSION = "1.0"
CONF = cfg.CONF
class Manager(periodic_task.PeriodicTasks):
target = messaging.Target(version=rpc_version.RPC_API_VERSION)
def __init__(self):
super(Manager, self).__init__()
self.admin_context = TroveContext(

View File

@ -65,9 +65,9 @@ from trove.instance.models import InstanceStatus
from trove.instance.models import InstanceServiceStatus
from trove.openstack.common import log as logging
from trove.common.i18n import _
from trove.openstack.common.notifier import api as notifier
from trove.quota.quota import run_with_quotas
import trove.common.remote as remote
from trove import rpc
LOG = logging.getLogger(__name__)
CONF = cfg.CONF
@ -143,8 +143,11 @@ class NotifyMixin(object):
payload.update(kwargs)
LOG.debug('Sending event: %(event_type)s, %(payload)s' %
{'event_type': event_type, 'payload': payload})
notifier.notify(self.context, publisher_id, event_type, 'INFO',
payload)
notifier = rpc.get_notifier(
service="taskmanager", publisher_id=publisher_id)
notifier.info(self.context, event_type, payload)
class ConfigurationMixin(object):

View File

@ -20,6 +20,7 @@ from proboscis import test
from novaclient.exceptions import BadRequest
from novaclient.v1_1.servers import Server
from oslo.messaging._drivers.common import RPCException
from trove.common.exception import PollTimeOut
from trove.common import template
@ -31,7 +32,6 @@ from trove.guestagent import api as guest
from trove.instance.models import DBInstance
from trove.instance.models import InstanceServiceStatus
from trove.instance.tasks import InstanceTasks
from trove.openstack.common.rpc.common import RPCException
from trove.taskmanager import models as models
from trove.tests.fakes import nova
from trove.tests.util import test_config

View File

@ -138,9 +138,6 @@ class FakeGuest(object):
if database['_name'] in self.dbs:
del self.dbs[database['_name']]
def delete_queue(self):
pass
def enable_root(self):
self.root_was_enabled = True
return self._create_user({

View File

@ -14,40 +14,54 @@
# under the License.
#
from collections import defaultdict
import eventlet
from trove.taskmanager import api
import trove.openstack.common.log as logging
from trove.taskmanager.api import API
from trove.taskmanager.manager import Manager
import trove.tests.util.usage as usage
from trove import rpc
LOG = logging.getLogger(__name__)
MESSAGE_QUEUE = defaultdict(list)
class FakeApi(api.API):
class FakeRpcClient(object):
def __init__(self, context):
self.context = context
def call(self, context, method_name, *args, **kwargs):
manager, method = self._get_tm_method(method_name)
return method(manager, context, *args, **kwargs)
def make_msg(self, method_name, *args, **kwargs):
return {"name": method_name, "args": args, "kwargs": kwargs}
def call(self, context, msg):
manager, method = self.get_tm_method(msg['name'])
return method(manager, context, *msg['args'], **msg['kwargs'])
def cast(self, context, msg):
manager, method = self.get_tm_method(msg['name'])
def cast(self, context, method_name, *args, **kwargs):
manager, method = self._get_tm_method(method_name)
def func():
method(manager, context, *msg['args'], **msg['kwargs'])
method(manager, context, *args, **kwargs)
eventlet.spawn_after(0.1, func)
def get_tm_method(self, method_name):
def _get_tm_method(self, method_name):
manager = Manager()
method = getattr(Manager, method_name)
return manager, method
def prepare(self, *args, **kwargs):
return self
class FakeNotifier:
def info(self, ctxt, event_type, payload):
usage.notify(event_type, payload)
def monkey_patch():
api.API = FakeApi
def fake_get_client(self, *args, **kwargs):
return FakeRpcClient()
def fake_load(context, manager=None):
return FakeApi(context)
api.load = fake_load
def fake_get_notifier(service=None, host=None, publisher_id=None):
return FakeNotifier()
API.get_client = fake_get_client
rpc.get_notifier = fake_get_notifier

View File

@ -39,6 +39,8 @@ BACKUP_DESC = 'Backup test'
BACKUP_FILENAME = '45a3d8cb-ade8-484c-a8a5-0c3c7286fb2f.xbstream.gz'
BACKUP_LOCATION = 'https://hpcs.com/tenant/database_backups/' + BACKUP_FILENAME
api.API.get_client = MagicMock()
class BackupCreateTest(testtools.TestCase):
def setUp(self):

View File

@ -32,6 +32,7 @@ from trove.guestagent.strategies.backup.base import BackupRunner
from trove.guestagent.strategies.backup.base import UnknownBackupType
from trove.guestagent.strategies.storage.base import Storage
conductor_api.API.get_client = Mock()
conductor_api.API.update_backup = Mock()

View File

@ -38,6 +38,7 @@ CONF = cfg.CONF
class ClusterTest(TestCase):
def setUp(self):
super(ClusterTest, self).setUp()
task_api.API.get_client = Mock()
self.cluster_id = str(uuid.uuid4())
self.cluster_name = "Cluster" + self.cluster_id
self.tenant_id = "23423432"

View File

@ -36,7 +36,7 @@ def mocked_conf(manager):
class NoopManager(object):
pass
RPC_API_VERSION = 1.0
class ConductorConfTests(testtools.TestCase):

View File

@ -11,30 +11,32 @@
# 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 eventlet import Timeout
import mock
import testtools
from testtools.matchers import KeysEqual, Is
from testtools.matchers import Is
import trove.common.context as context
from trove.common import exception
from trove.guestagent import api
import trove.openstack.common.rpc as rpc
import trove.common.rpc as trove_rpc
from trove import rpc
REPLICATION_SNAPSHOT = {'master': {'id': '123', 'host': '192.168.0.1',
'port': 3306},
'dataset': {},
'binlog_position': 'binpos'}
RPC_API_VERSION = '1.0'
def _mock_call_pwd_change(cmd, users=None):
def _mock_call_pwd_change(cmd, version=None, users=None):
if users == 'dummy':
return True
else:
raise BaseException("Test Failed")
def _mock_call(cmd, timerout, username=None, hostname=None,
def _mock_call(cmd, timeout, version=None, username=None, hostname=None,
database=None, databases=None):
#To check get_user, list_access, grant_access, revoke_access in cmd.
if cmd in ('get_user', 'list_access', 'grant_access', 'revoke_access'):
@ -47,11 +49,12 @@ class ApiTest(testtools.TestCase):
def setUp(self):
super(ApiTest, self).setUp()
self.context = context.TroveContext()
rpc.get_client = mock.Mock()
self.guest = api.API(self.context, 0)
self.guest._cast = _mock_call_pwd_change
self.guest._call = _mock_call
self.FAKE_ID = 'instance-id-x23d2d'
self.api = api.API(self.context, self.FAKE_ID)
self.api = api.API(self.context, "instance-id-x23d2d")
self._mock_rpc_client()
def test_change_passwords(self):
self.assertIsNone(self.guest.change_passwords("dummy"))
@ -70,306 +73,259 @@ class ApiTest(testtools.TestCase):
"dumdb"))
def test_get_routing_key(self):
self.assertEqual('guestagent.' + self.FAKE_ID,
self.assertEqual('guestagent.instance-id-x23d2d',
self.api._get_routing_key())
def test_delete_queue(self):
trove_rpc.delete_queue = mock.Mock()
# execute
self.api.delete_queue()
# verify
trove_rpc.delete_queue.assert_called_with(self.context, mock.ANY)
def test_create_user(self):
rpc.cast = mock.Mock()
exp_msg = RpcMsgMatcher('create_user', 'users')
self.api.create_user('test_user')
self._verify_rpc_cast(exp_msg, rpc.cast)
def test_rpc_cast_exception(self):
rpc.cast = mock.Mock(side_effect=IOError('host down'))
exp_msg = RpcMsgMatcher('create_user', 'users')
# execute
with testtools.ExpectedException(exception.GuestError, '.* host down'):
self.api.create_user('test_user')
# verify
self._verify_rpc_cast(exp_msg, rpc.cast)
self._verify_rpc_prepare_before_cast()
self._verify_cast('create_user', users='test_user')
def test_api_cast_exception(self):
self.call_context.cast.side_effect = IOError('host down')
self.assertRaises(exception.GuestError, self.api.create_user,
'test_user')
def test_api_call_exception(self):
self.call_context.call.side_effect = IOError('host_down')
self.assertRaises(exception.GuestError, self.api.list_users)
def test_api_call_timeout(self):
self.call_context.call.side_effect = Timeout()
self.assertRaises(exception.GuestTimeout, self.api.restart)
def test_list_users(self):
exp_resp = ['user1', 'user2', 'user3']
rpc.call = mock.Mock(return_value=exp_resp)
exp_msg = RpcMsgMatcher('list_users', 'limit', 'marker',
'include_marker')
# execute
act_resp = self.api.list_users()
# verify
self.assertThat(act_resp, Is(exp_resp))
self._verify_rpc_call(exp_msg, rpc.call)
self.call_context.call.return_value = exp_resp
def test_rpc_call_exception(self):
rpc.call = mock.Mock(side_effect=IOError('host_down'))
exp_msg = RpcMsgMatcher('list_users', 'limit', 'marker',
'include_marker')
# execute
with testtools.ExpectedException(exception.GuestError,
'An error occurred.*'):
self.api.list_users()
# verify
self._verify_rpc_call(exp_msg, rpc.call)
resp = self.api.list_users()
self._verify_rpc_prepare_before_call()
self._verify_call('list_users', limit=None, marker=None,
include_marker=False)
self.assertEqual(exp_resp, resp)
def test_delete_user(self):
rpc.cast = mock.Mock()
exp_msg = RpcMsgMatcher('delete_user', 'user')
# execute
self.api.delete_user('test_user')
# verify
self._verify_rpc_cast(exp_msg, rpc.cast)
self._verify_rpc_prepare_before_cast()
self._verify_cast('delete_user', user='test_user')
def test_create_database(self):
rpc.cast = mock.Mock()
exp_msg = RpcMsgMatcher('create_database', 'databases')
# execute
self.api.create_database(['db1', 'db2', 'db3'])
# verify
self._verify_rpc_cast(exp_msg, rpc.cast)
databases = ['db1', 'db2', 'db3']
self.api.create_database(databases)
self._verify_rpc_prepare_before_cast()
self.call_context.cast.assert_called_once_with(
self.context, "create_database", databases=databases)
def test_list_databases(self):
exp_resp = ['db1', 'db2', 'db3']
rpc.call = mock.Mock(return_value=exp_resp)
exp_msg = RpcMsgMatcher('list_databases', 'limit', 'marker',
'include_marker')
# execute
resp = self.api.list_databases(limit=1, marker=2,
include_marker=False)
# verify
self.assertThat(resp, Is(exp_resp))
self._verify_rpc_call(exp_msg, rpc.call)
self.call_context.call.return_value = exp_resp
resp = self.api.list_databases(
limit=1, marker=2, include_marker=False)
self._verify_rpc_prepare_before_call()
self._verify_call("list_databases", limit=1, marker=2,
include_marker=False)
self.assertEqual(exp_resp, resp)
def test_delete_database(self):
rpc.cast = mock.Mock()
exp_msg = RpcMsgMatcher('delete_database', 'database')
# execute
self.api.delete_database('test_database_name')
# verify
self._verify_rpc_cast(exp_msg, rpc.cast)
self._verify_rpc_prepare_before_cast()
self._verify_cast("delete_database", database='test_database_name')
def test_enable_root(self):
rpc.call = mock.Mock(return_value=True)
exp_msg = RpcMsgMatcher('enable_root')
# execute
self.assertThat(self.api.enable_root(), Is(True))
# verify
self._verify_rpc_call(exp_msg, rpc.call)
self.call_context.call.return_value = True
resp = self.api.enable_root()
self._verify_rpc_prepare_before_call()
self._verify_call('enable_root')
self.assertThat(resp, Is(True))
def test_disable_root(self):
rpc.call = mock.Mock(return_value=True)
exp_msg = RpcMsgMatcher('disable_root')
# execute
self.assertThat(self.api.disable_root(), Is(True))
# verify
self._verify_rpc_call(exp_msg, rpc.call)
self.call_context.call.return_value = True
resp = self.api.disable_root()
self._verify_rpc_prepare_before_call()
self._verify_call('disable_root')
self.assertThat(resp, Is(True))
def test_is_root_enabled(self):
rpc.call = mock.Mock(return_value=False)
exp_msg = RpcMsgMatcher('is_root_enabled')
# execute
self.assertThat(self.api.is_root_enabled(), Is(False))
# verify
self._verify_rpc_call(exp_msg, rpc.call)
self.call_context.call.return_value = False
resp = self.api.is_root_enabled()
self._verify_rpc_prepare_before_call()
self._verify_call('is_root_enabled')
self.assertThat(resp, Is(False))
def test_get_hwinfo(self):
rpc.call = mock.Mock(return_value='[blah]')
exp_msg = RpcMsgMatcher('get_hwinfo')
# execute
self.assertThat(self.api.get_hwinfo(), Is('[blah]'))
# verify
self._verify_rpc_call(exp_msg, rpc.call)
self.call_context.call.return_value = '[blah]'
resp = self.api.get_hwinfo()
self._verify_rpc_prepare_before_call()
self._verify_call('get_hwinfo')
self.assertThat(resp, Is('[blah]'))
def test_get_diagnostics(self):
rpc.call = mock.Mock(spec=rpc, return_value='[all good]')
exp_msg = RpcMsgMatcher('get_diagnostics')
# execute
self.assertThat(self.api.get_diagnostics(), Is('[all good]'))
# verify
self._verify_rpc_call(exp_msg, rpc.call)
self.call_context.call.return_value = '[all good]'
resp = self.api.get_diagnostics()
self._verify_rpc_prepare_before_call()
self._verify_call('get_diagnostics')
self.assertThat(resp, Is('[all good]'))
def test_restart(self):
rpc.call = mock.Mock()
exp_msg = RpcMsgMatcher('restart')
# execute
self.api.restart()
# verify
self._verify_rpc_call(exp_msg, rpc.call)
self._verify_rpc_prepare_before_call()
self._verify_call('restart')
def test_start_db_with_conf_changes(self):
rpc.call = mock.Mock()
exp_msg = RpcMsgMatcher('start_db_with_conf_changes',
'config_contents')
# execute
self.api.start_db_with_conf_changes(None)
# verify
self._verify_rpc_call(exp_msg, rpc.call)
self._verify_rpc_prepare_before_call()
self._verify_call('start_db_with_conf_changes', config_contents=None)
def test_stop_db(self):
rpc.call = mock.Mock()
exp_msg = RpcMsgMatcher('stop_db', 'do_not_start_on_reboot')
# execute
self.api.stop_db(do_not_start_on_reboot=False)
# verify
self._verify_rpc_call(exp_msg, rpc.call)
self._verify_rpc_prepare_before_call()
self._verify_call('stop_db', do_not_start_on_reboot=False)
def test_get_volume_info(self):
fake_resp = {'fake': 'resp'}
rpc.call = mock.Mock(return_value=fake_resp)
exp_msg = RpcMsgMatcher('get_filesystem_stats', 'fs_path')
# execute
self.assertThat(self.api.get_volume_info(), Is(fake_resp))
# verify
self._verify_rpc_call(exp_msg, rpc.call)
exp_resp = {'fake': 'resp'}
self.call_context.call.return_value = exp_resp
resp = self.api.get_volume_info()
self._verify_rpc_prepare_before_call()
self._verify_call('get_filesystem_stats', fs_path=None)
self.assertThat(resp, Is(exp_resp))
def test_update_guest(self):
rpc.call = mock.Mock()
exp_msg = RpcMsgMatcher('update_guest')
# execute
self.api.update_guest()
# verify
self._verify_rpc_call(exp_msg, rpc.call)
self._verify_rpc_prepare_before_call()
self._verify_call('update_guest')
def test_create_backup(self):
rpc.cast = mock.Mock()
exp_msg = RpcMsgMatcher('create_backup', 'backup_info')
# execute
self.api.create_backup({'id': '123'})
# verify
self._verify_rpc_cast(exp_msg, rpc.cast)
self._verify_rpc_prepare_before_cast()
self._verify_cast('create_backup', backup_info={'id': '123'})
def test_update_overrides(self):
rpc.cast = mock.Mock()
exp_msg = RpcMsgMatcher('update_overrides', 'overrides', 'remove')
# execute
self.api.update_overrides('123')
# verify
self._verify_rpc_cast(exp_msg, rpc.cast)
self._verify_rpc_prepare_before_cast()
self._verify_cast('update_overrides', overrides='123', remove=False)
def test_apply_overrides(self):
rpc.cast = mock.Mock()
exp_msg = RpcMsgMatcher('apply_overrides', 'overrides')
# execute
self.api.apply_overrides('123')
# verify
self._verify_rpc_cast(exp_msg, rpc.cast)
self._verify_rpc_prepare_before_cast()
self._verify_cast('apply_overrides', overrides='123')
def test_get_replication_snapshot(self):
exp_resp = REPLICATION_SNAPSHOT
rpc.call = mock.Mock(return_value=exp_resp)
exp_msg = RpcMsgMatcher('get_replication_snapshot', 'snapshot_info',
'replica_source_config')
# execute
self.api.get_replication_snapshot({})
# verify
self._verify_rpc_call(exp_msg, rpc.call)
self._verify_rpc_prepare_before_call()
self._verify_call('get_replication_snapshot', snapshot_info={},
replica_source_config=None)
def test_attach_replication_slave(self):
rpc.cast = mock.Mock()
exp_msg = RpcMsgMatcher('attach_replication_slave',
'snapshot', 'slave_config')
# execute
self.api.attach_replication_slave(REPLICATION_SNAPSHOT)
# verify
self._verify_rpc_cast(exp_msg, rpc.cast)
self._verify_rpc_prepare_before_cast()
self._verify_cast('attach_replication_slave',
snapshot=REPLICATION_SNAPSHOT, slave_config=None)
def test_detach_replica(self):
rpc.call = mock.Mock()
exp_msg = RpcMsgMatcher('detach_replica')
# execute
self.api.detach_replica()
# verify
self._verify_rpc_call(exp_msg, rpc.call)
self._verify_rpc_prepare_before_call()
self._verify_call('detach_replica')
def test_demote_replication_master(self):
rpc.call = mock.Mock()
exp_msg = RpcMsgMatcher('demote_replication_master')
# execute
self.api.demote_replication_master()
# verify
self._verify_rpc_call(exp_msg, rpc.call)
def _verify_rpc_connection_and_cast(self, rpc, mock_conn, exp_msg):
rpc.create_connection.assert_called_with(new=True)
mock_conn.create_consumer.assert_called_with(
self.api._get_routing_key(), None, fanout=False)
rpc.cast.assert_called_with(mock.ANY, mock.ANY, exp_msg)
self._verify_rpc_prepare_before_call()
self._verify_call('demote_replication_master')
def test_prepare(self):
mock_conn = mock.Mock()
rpc.create_connection = mock.Mock(return_value=mock_conn)
rpc.cast = mock.Mock()
exp_msg = RpcMsgMatcher('prepare', 'memory_mb', 'packages',
'databases', 'users', 'device_path',
'mount_point', 'backup_info',
'config_contents', 'root_password',
'overrides', 'cluster_config')
# execute
self.api._create_guest_queue = mock.Mock()
self.api.prepare('2048', 'package1', 'db1', 'user1', '/dev/vdt',
'/mnt/opt', 'bkup-1232', 'cont', '1-2-3-4',
'override', {"id": "2-3-4-5"})
# verify
self._verify_rpc_connection_and_cast(rpc, mock_conn, exp_msg)
'/mnt/opt', None, 'cont', '1-2-3-4',
'override', {'id': '2-3-4-5'})
self._verify_rpc_prepare_before_cast()
self._verify_cast(
'prepare', packages=['package1'], databases='db1',
memory_mb='2048', users='user1', device_path='/dev/vdt',
mount_point='/mnt/opt', backup_info=None,
config_contents='cont', root_password='1-2-3-4',
overrides='override', cluster_config={'id': '2-3-4-5'})
def test_prepare_with_backup(self):
mock_conn = mock.Mock()
rpc.create_connection = mock.Mock(return_value=mock_conn)
rpc.cast = mock.Mock()
exp_msg = RpcMsgMatcher('prepare', 'memory_mb', 'packages',
'databases', 'users', 'device_path',
'mount_point', 'backup_info',
'config_contents', 'root_password',
'overrides', 'cluster_config')
bkup = {'id': 'backup_id_123'}
# execute
self.api._create_guest_queue = mock.Mock()
backup = {'id': 'backup_id_123'}
self.api.prepare('2048', 'package1', 'db1', 'user1', '/dev/vdt',
'/mnt/opt', bkup, 'cont', '1-2-3-4',
'/mnt/opt', backup, 'cont', '1-2-3-4',
'overrides', {"id": "2-3-4-5"})
# verify
self._verify_rpc_connection_and_cast(rpc, mock_conn, exp_msg)
self._verify_rpc_prepare_before_cast()
self._verify_cast(
'prepare', packages=['package1'], databases='db1',
memory_mb='2048', users='user1', device_path='/dev/vdt',
mount_point='/mnt/opt', backup_info=backup,
config_contents='cont', root_password='1-2-3-4',
overrides='overrides', cluster_config={'id': '2-3-4-5'})
def test_upgrade(self):
instance_version = "v1.0.1"
strategy = "pip"
location = "http://swift/trove-guestagent-v1.0.1.tar.gz"
mock_conn = mock.Mock()
rpc.create_connection = mock.Mock(return_value=mock_conn)
rpc.cast = mock.Mock()
exp_msg = RpcMsgMatcher(
'upgrade', 'instance_version', 'location', 'metadata')
# execute
self.api.upgrade(instance_version, strategy, location)
self.api.upgrade(instance_version, location)
# verify
self._verify_rpc_cast(exp_msg, rpc.cast)
self._verify_rpc_prepare_before_cast()
self._verify_cast(
'upgrade', instance_version=instance_version,
location=location, metadata=None)
def test_rpc_cast_with_consumer_exception(self):
mock_conn = mock.Mock()
rpc.create_connection = mock.Mock(side_effect=IOError('host down'))
rpc.cast = mock.Mock()
# execute
with testtools.ExpectedException(exception.GuestError, '.* host down'):
self.api.prepare('2048', 'package1', 'db1', 'user1', '/dev/vdt',
'/mnt/opt')
# verify
rpc.create_connection.assert_called_with(new=True)
self.assertThat(mock_conn.call_count, Is(0))
self.assertThat(rpc.cast.call_count, Is(0))
def _verify_rpc_prepare_before_call(self):
self.api.client.prepare.assert_called_once_with(
version=RPC_API_VERSION, timeout=mock.ANY)
def _verify_rpc_call(self, exp_msg, mock_call=None):
mock_call.assert_called_with(self.context, mock.ANY, exp_msg,
mock.ANY)
def _verify_rpc_prepare_before_cast(self):
self.api.client.prepare.assert_called_once_with(
version=RPC_API_VERSION)
def _verify_rpc_cast(self, exp_msg, mock_cast=None):
mock_cast.assert_called_with(mock.ANY,
mock.ANY, exp_msg)
def _verify_cast(self, *args, **kwargs):
self.call_context.cast.assert_called_once_with(self.context, *args,
**kwargs)
def _verify_call(self, *args, **kwargs):
self.call_context.call.assert_called_once_with(self.context, *args,
**kwargs)
def _mock_rpc_client(self):
self.call_context = mock.Mock()
self.api.client.prepare = mock.Mock(return_value=self.call_context)
self.call_context.call = mock.Mock()
self.call_context.cast = mock.Mock()
class ApiStrategyTest(testtools.TestCase):
@ -381,44 +337,3 @@ class ApiStrategyTest(testtools.TestCase):
client = guest_client(mock.Mock(), mock.Mock(), 'mongodb')
self.assertFalse(hasattr(client, 'add_config_servers2'))
self.assertTrue(callable(client.add_config_servers))
class CastWithConsumerTest(testtools.TestCase):
def setUp(self):
super(CastWithConsumerTest, self).setUp()
self.context = context.TroveContext()
self.api = api.API(self.context, 'instance-id-x23d2d')
def test_cast_with_consumer(self):
mock_conn = mock.Mock()
rpc.create_connection = mock.Mock(return_value=mock_conn)
rpc.cast = mock.Mock()
# execute
self.api._cast_with_consumer('fake_method_name', fake_param=1)
# verify
rpc.create_connection.assert_called_with(new=True)
mock_conn.create_consumer.assert_called_with(mock.ANY, None,
fanout=False)
rpc.cast.assert_called_with(self.context, mock.ANY, mock.ANY)
class RpcMsgMatcher(object):
def __init__(self, method, *args_dict):
self.wanted_method = method
self.wanted_dict = KeysEqual('version', 'method', 'args', 'namespace')
args_dict = args_dict or [{}]
self.args_dict = KeysEqual(*args_dict)
def __eq__(self, arg):
if self.wanted_method != arg['method']:
raise Exception("Method does not match: %s != %s" %
(self.wanted_method, arg['method']))
#return False
if self.wanted_dict.match(arg) or self.args_dict.match(arg['args']):
raise Exception("Args do not match: %s != %s" %
(self.args_dict, arg['args']))
#return False
return True
def __repr__(self):
return "<Dict: %s>" % self.wanted_dict

View File

@ -64,6 +64,7 @@ FAKE_USER = [{"_name": "random", "_password": "guesswhat",
"_databases": [FAKE_DB]}]
conductor_api.API.get_client = Mock()
conductor_api.API.heartbeat = Mock()

View File

@ -32,6 +32,7 @@ from trove.tests.fakes import nova
from trove.tests.unittests.util import util
CONF = cfg.CONF
task_api.API.get_client = Mock()
class SimpleInstanceTest(TestCase):

View File

@ -30,9 +30,9 @@ from trove.instance.models import DBInstance
from trove.instance.models import InstanceServiceStatus
from trove.instance.tasks import InstanceTasks
import trove.extensions.mgmt.instances.models as mgmtmodels
from trove.openstack.common.notifier import api as notifier
from trove.common import remote
from trove.tests.unittests.util import util
from trove import rpc
CONF = cfg.CONF
@ -369,22 +369,23 @@ class TestMgmtInstanceTasks(MockMgmtInstanceTest):
flavor = MagicMock(spec=Flavor)
flavor.name = 'db.small'
notifier = MagicMock()
rpc.get_notifier = MagicMock(return_value=notifier)
with patch.object(mgmtmodels, 'load_mgmt_instances',
return_value=[mgmt_instance]):
with patch.object(self.flavor_mgr, 'get', return_value=flavor):
self.assertThat(self.context.auth_token,
Is('some_secret_password'))
with patch.object(notifier, 'notify', return_value=None):
with patch.object(notifier, 'info', return_value=None):
# invocation
mgmtmodels.publish_exist_events(
mgmtmodels.NovaNotificationTransformer(
context=self.context),
self.context)
# assertion
notifier.notify.assert_any_call(self.context,
'test_host',
'trove.instance.exists',
'INFO',
ANY)
notifier.info.assert_any_call(self.context,
'trove.instance.exists',
ANY)
self.assertThat(self.context.auth_token, Is(None))
self.addCleanup(self.do_cleanup, instance, service_status)

View File

@ -22,6 +22,7 @@ from trove.taskmanager import api as task_api
class TestAPI(TestCase):
def test_load_api(self):
task_api.API.get_client = Mock()
context = Mock()
manager = 'mongodb'

View File

@ -43,6 +43,7 @@ from trove.instance.models import DBInstance
from trove.instance.tasks import InstanceTasks
from trove.tests.unittests.util import util
from trove.common import utils
from trove import rpc
from swiftclient.client import ClientException
from tempfile import NamedTemporaryFile
import os
@ -467,6 +468,8 @@ class BuiltInstanceTasksTest(testtools.TestCase):
super(BuiltInstanceTasksTest, self).setUp()
self.new_flavor = {'id': 8, 'ram': 768, 'name': 'bigger_flavor'}
stub_nova_server = MagicMock()
rpc.get_notifier = MagicMock()
rpc.get_client = MagicMock()
db_instance = DBInstance(InstanceTasks.NONE,
id=INST_ID,
name='resize-inst-name',

View File

@ -70,11 +70,10 @@ class FakeVerifier(object):
pass
def notify(context, message):
def notify(event_type, payload):
"""Simple test notify function which saves the messages to global list."""
LOG.debug('Received Usage Notification: %s' % message)
payload = message.get('payload', None)
payload['event_type'] = message['event_type']
LOG.debug('Received Usage Notification: %s' % event_type)
payload['event_type'] = event_type
resource_id = payload['instance_id']
global MESSAGE_QUEUE
MESSAGE_QUEUE[resource_id].append(payload)