Switch to oslo.utils
Change-Id: I262e98d8d03acbe2ff8fee5d607279a669a89e34
This commit is contained in:
		@@ -1,12 +1,8 @@
 | 
			
		||||
[DEFAULT]
 | 
			
		||||
 | 
			
		||||
# The list of modules to copy from oslo-incubator.git
 | 
			
		||||
module=excutils
 | 
			
		||||
module=gettextutils
 | 
			
		||||
module=importutils
 | 
			
		||||
module=jsonutils
 | 
			
		||||
module=network_utils
 | 
			
		||||
module=timeutils
 | 
			
		||||
module=middleware/base
 | 
			
		||||
module=middleware/__init__
 | 
			
		||||
module=context
 | 
			
		||||
 
 | 
			
		||||
@@ -25,7 +25,6 @@ import six
 | 
			
		||||
from oslo import messaging
 | 
			
		||||
from oslo.messaging import _utils as utils
 | 
			
		||||
from oslo.messaging.openstack.common.gettextutils import _
 | 
			
		||||
from oslo.messaging.openstack.common import importutils
 | 
			
		||||
from oslo.messaging.openstack.common import jsonutils
 | 
			
		||||
 | 
			
		||||
LOG = logging.getLogger(__name__)
 | 
			
		||||
@@ -232,7 +231,8 @@ def deserialize_remote_exception(data, allowed_remote_exmods):
 | 
			
		||||
        return messaging.RemoteError(name, failure.get('message'), trace)
 | 
			
		||||
 | 
			
		||||
    try:
 | 
			
		||||
        mod = importutils.import_module(module)
 | 
			
		||||
        __import__(module)
 | 
			
		||||
        mod = sys.modules[module]
 | 
			
		||||
        klass = getattr(mod, name)
 | 
			
		||||
        if not issubclass(klass, Exception):
 | 
			
		||||
            raise TypeError("Can only deserialize Exceptions")
 | 
			
		||||
 
 | 
			
		||||
@@ -27,9 +27,9 @@ from oslo.messaging._drivers import amqpdriver
 | 
			
		||||
from oslo.messaging._drivers import common as rpc_common
 | 
			
		||||
from oslo.messaging import exceptions
 | 
			
		||||
from oslo.messaging.openstack.common.gettextutils import _
 | 
			
		||||
from oslo.messaging.openstack.common import importutils
 | 
			
		||||
from oslo.messaging.openstack.common import jsonutils
 | 
			
		||||
from oslo.messaging.openstack.common import network_utils
 | 
			
		||||
from oslo.utils import importutils
 | 
			
		||||
from oslo.utils import netutils
 | 
			
		||||
 | 
			
		||||
qpid_codec = importutils.try_import("qpid.codec010")
 | 
			
		||||
qpid_messaging = importutils.try_import("qpid.messaging")
 | 
			
		||||
@@ -475,7 +475,7 @@ class Connection(object):
 | 
			
		||||
        else:
 | 
			
		||||
            # Old configuration format
 | 
			
		||||
            for adr in self.conf.qpid_hosts:
 | 
			
		||||
                hostname, port = network_utils.parse_host_port(
 | 
			
		||||
                hostname, port = netutils.parse_host_port(
 | 
			
		||||
                    adr, default_port=5672)
 | 
			
		||||
 | 
			
		||||
                params = {
 | 
			
		||||
 
 | 
			
		||||
@@ -33,7 +33,7 @@ from oslo.messaging._drivers import amqpdriver
 | 
			
		||||
from oslo.messaging._drivers import common as rpc_common
 | 
			
		||||
from oslo.messaging import exceptions
 | 
			
		||||
from oslo.messaging.openstack.common.gettextutils import _
 | 
			
		||||
from oslo.messaging.openstack.common import network_utils
 | 
			
		||||
from oslo.utils import netutils
 | 
			
		||||
 | 
			
		||||
rabbit_opts = [
 | 
			
		||||
    cfg.StrOpt('kombu_ssl_version',
 | 
			
		||||
@@ -462,7 +462,7 @@ class Connection(object):
 | 
			
		||||
        else:
 | 
			
		||||
            # Old configuration format
 | 
			
		||||
            for adr in self.conf.rabbit_hosts:
 | 
			
		||||
                hostname, port = network_utils.parse_host_port(
 | 
			
		||||
                hostname, port = netutils.parse_host_port(
 | 
			
		||||
                    adr, default_port=self.conf.rabbit_port)
 | 
			
		||||
 | 
			
		||||
                params = {
 | 
			
		||||
 
 | 
			
		||||
@@ -32,10 +32,11 @@ from oslo.config import cfg
 | 
			
		||||
from oslo.messaging._drivers import base
 | 
			
		||||
from oslo.messaging._drivers import common as rpc_common
 | 
			
		||||
from oslo.messaging._executors import impl_eventlet  # FIXME(markmc)
 | 
			
		||||
from oslo.messaging.openstack.common import excutils
 | 
			
		||||
from oslo.messaging.openstack.common.gettextutils import _
 | 
			
		||||
from oslo.messaging.openstack.common import importutils
 | 
			
		||||
from oslo.messaging.openstack.common import jsonutils
 | 
			
		||||
from oslo.utils import excutils
 | 
			
		||||
from oslo.utils import importutils
 | 
			
		||||
 | 
			
		||||
 | 
			
		||||
zmq = importutils.try_import('eventlet.green.zmq')
 | 
			
		||||
 | 
			
		||||
 
 | 
			
		||||
@@ -18,7 +18,7 @@ return keys for direct exchanges, per (approximate) AMQP parlance.
 | 
			
		||||
 | 
			
		||||
from oslo.config import cfg
 | 
			
		||||
from oslo.messaging._drivers import matchmaker as mm_common
 | 
			
		||||
from oslo.messaging.openstack.common import importutils
 | 
			
		||||
from oslo.utils import importutils
 | 
			
		||||
 | 
			
		||||
redis = importutils.try_import('redis')
 | 
			
		||||
 | 
			
		||||
 
 | 
			
		||||
@@ -21,7 +21,7 @@ import greenlet
 | 
			
		||||
 | 
			
		||||
from oslo.config import cfg
 | 
			
		||||
from oslo.messaging._executors import base
 | 
			
		||||
from oslo.messaging.openstack.common import excutils
 | 
			
		||||
from oslo.utils import excutils
 | 
			
		||||
 | 
			
		||||
_eventlet_opts = [
 | 
			
		||||
    cfg.IntOpt('rpc_thread_pool_size',
 | 
			
		||||
 
 | 
			
		||||
@@ -23,8 +23,8 @@ import six
 | 
			
		||||
from stevedore import named
 | 
			
		||||
 | 
			
		||||
from oslo.config import cfg
 | 
			
		||||
from oslo.messaging.openstack.common import timeutils
 | 
			
		||||
from oslo.messaging import serializer as msg_serializer
 | 
			
		||||
from oslo.utils import timeutils
 | 
			
		||||
 | 
			
		||||
_notifier_opts = [
 | 
			
		||||
    cfg.MultiStrOpt('notification_driver',
 | 
			
		||||
 
 | 
			
		||||
@@ -1,99 +0,0 @@
 | 
			
		||||
# Copyright 2011 OpenStack Foundation.
 | 
			
		||||
# 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.
 | 
			
		||||
 | 
			
		||||
"""
 | 
			
		||||
Exception related utilities.
 | 
			
		||||
"""
 | 
			
		||||
 | 
			
		||||
import logging
 | 
			
		||||
import sys
 | 
			
		||||
import time
 | 
			
		||||
import traceback
 | 
			
		||||
 | 
			
		||||
import six
 | 
			
		||||
 | 
			
		||||
from oslo.messaging.openstack.common.gettextutils import _  # noqa
 | 
			
		||||
 | 
			
		||||
 | 
			
		||||
class save_and_reraise_exception(object):
 | 
			
		||||
    """Save current exception, run some code and then re-raise.
 | 
			
		||||
 | 
			
		||||
    In some cases the exception context can be cleared, resulting in None
 | 
			
		||||
    being attempted to be re-raised after an exception handler is run. This
 | 
			
		||||
    can happen when eventlet switches greenthreads or when running an
 | 
			
		||||
    exception handler, code raises and catches an exception. In both
 | 
			
		||||
    cases the exception context will be cleared.
 | 
			
		||||
 | 
			
		||||
    To work around this, we save the exception state, run handler code, and
 | 
			
		||||
    then re-raise the original exception. If another exception occurs, the
 | 
			
		||||
    saved exception is logged and the new exception is re-raised.
 | 
			
		||||
 | 
			
		||||
    In some cases the caller may not want to re-raise the exception, and
 | 
			
		||||
    for those circumstances this context provides a reraise flag that
 | 
			
		||||
    can be used to suppress the exception.  For example::
 | 
			
		||||
 | 
			
		||||
      except Exception:
 | 
			
		||||
          with save_and_reraise_exception() as ctxt:
 | 
			
		||||
              decide_if_need_reraise()
 | 
			
		||||
              if not should_be_reraised:
 | 
			
		||||
                  ctxt.reraise = False
 | 
			
		||||
    """
 | 
			
		||||
    def __init__(self):
 | 
			
		||||
        self.reraise = True
 | 
			
		||||
 | 
			
		||||
    def __enter__(self):
 | 
			
		||||
        self.type_, self.value, self.tb, = sys.exc_info()
 | 
			
		||||
        return self
 | 
			
		||||
 | 
			
		||||
    def __exit__(self, exc_type, exc_val, exc_tb):
 | 
			
		||||
        if exc_type is not None:
 | 
			
		||||
            logging.error(_('Original exception being dropped: %s'),
 | 
			
		||||
                          traceback.format_exception(self.type_,
 | 
			
		||||
                                                     self.value,
 | 
			
		||||
                                                     self.tb))
 | 
			
		||||
            return False
 | 
			
		||||
        if self.reraise:
 | 
			
		||||
            six.reraise(self.type_, self.value, self.tb)
 | 
			
		||||
 | 
			
		||||
 | 
			
		||||
def forever_retry_uncaught_exceptions(infunc):
 | 
			
		||||
    def inner_func(*args, **kwargs):
 | 
			
		||||
        last_log_time = 0
 | 
			
		||||
        last_exc_message = None
 | 
			
		||||
        exc_count = 0
 | 
			
		||||
        while True:
 | 
			
		||||
            try:
 | 
			
		||||
                return infunc(*args, **kwargs)
 | 
			
		||||
            except Exception as exc:
 | 
			
		||||
                this_exc_message = six.u(str(exc))
 | 
			
		||||
                if this_exc_message == last_exc_message:
 | 
			
		||||
                    exc_count += 1
 | 
			
		||||
                else:
 | 
			
		||||
                    exc_count = 1
 | 
			
		||||
                # Do not log any more frequently than once a minute unless
 | 
			
		||||
                # the exception message changes
 | 
			
		||||
                cur_time = int(time.time())
 | 
			
		||||
                if (cur_time - last_log_time > 60 or
 | 
			
		||||
                        this_exc_message != last_exc_message):
 | 
			
		||||
                    logging.exception(
 | 
			
		||||
                        _('Unexpected exception occurred %d time(s)... '
 | 
			
		||||
                          'retrying.') % exc_count)
 | 
			
		||||
                    last_log_time = cur_time
 | 
			
		||||
                    last_exc_message = this_exc_message
 | 
			
		||||
                    exc_count = 0
 | 
			
		||||
                # This should be a very rare event. In case it isn't, do
 | 
			
		||||
                # a sleep.
 | 
			
		||||
                time.sleep(1)
 | 
			
		||||
    return inner_func
 | 
			
		||||
@@ -1,98 +0,0 @@
 | 
			
		||||
# Copyright 2012 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.
 | 
			
		||||
 | 
			
		||||
"""
 | 
			
		||||
Network-related utilities and helper functions.
 | 
			
		||||
"""
 | 
			
		||||
 | 
			
		||||
from six.moves.urllib import parse
 | 
			
		||||
 | 
			
		||||
 | 
			
		||||
def parse_host_port(address, default_port=None):
 | 
			
		||||
    """Interpret a string as a host:port pair.
 | 
			
		||||
 | 
			
		||||
    An IPv6 address MUST be escaped if accompanied by a port,
 | 
			
		||||
    because otherwise ambiguity ensues: 2001:db8:85a3::8a2e:370:7334
 | 
			
		||||
    means both [2001:db8:85a3::8a2e:370:7334] and
 | 
			
		||||
    [2001:db8:85a3::8a2e:370]:7334.
 | 
			
		||||
 | 
			
		||||
    >>> parse_host_port('server01:80')
 | 
			
		||||
    ('server01', 80)
 | 
			
		||||
    >>> parse_host_port('server01')
 | 
			
		||||
    ('server01', None)
 | 
			
		||||
    >>> parse_host_port('server01', default_port=1234)
 | 
			
		||||
    ('server01', 1234)
 | 
			
		||||
    >>> parse_host_port('[::1]:80')
 | 
			
		||||
    ('::1', 80)
 | 
			
		||||
    >>> parse_host_port('[::1]')
 | 
			
		||||
    ('::1', None)
 | 
			
		||||
    >>> parse_host_port('[::1]', default_port=1234)
 | 
			
		||||
    ('::1', 1234)
 | 
			
		||||
    >>> parse_host_port('2001:db8:85a3::8a2e:370:7334', default_port=1234)
 | 
			
		||||
    ('2001:db8:85a3::8a2e:370:7334', 1234)
 | 
			
		||||
 | 
			
		||||
    """
 | 
			
		||||
    if address[0] == '[':
 | 
			
		||||
        # Escaped ipv6
 | 
			
		||||
        _host, _port = address[1:].split(']')
 | 
			
		||||
        host = _host
 | 
			
		||||
        if ':' in _port:
 | 
			
		||||
            port = _port.split(':')[1]
 | 
			
		||||
        else:
 | 
			
		||||
            port = default_port
 | 
			
		||||
    else:
 | 
			
		||||
        if address.count(':') == 1:
 | 
			
		||||
            host, port = address.split(':')
 | 
			
		||||
        else:
 | 
			
		||||
            # 0 means ipv4, >1 means ipv6.
 | 
			
		||||
            # We prohibit unescaped ipv6 addresses with port.
 | 
			
		||||
            host = address
 | 
			
		||||
            port = default_port
 | 
			
		||||
 | 
			
		||||
    return (host, None if port is None else int(port))
 | 
			
		||||
 | 
			
		||||
 | 
			
		||||
class ModifiedSplitResult(parse.SplitResult):
 | 
			
		||||
    """Split results class for urlsplit."""
 | 
			
		||||
 | 
			
		||||
    # NOTE(dims): The functions below are needed for Python 2.6.x.
 | 
			
		||||
    # We can remove these when we drop support for 2.6.x.
 | 
			
		||||
    @property
 | 
			
		||||
    def hostname(self):
 | 
			
		||||
        netloc = self.netloc.split('@', 1)[-1]
 | 
			
		||||
        host, port = parse_host_port(netloc)
 | 
			
		||||
        return host
 | 
			
		||||
 | 
			
		||||
    @property
 | 
			
		||||
    def port(self):
 | 
			
		||||
        netloc = self.netloc.split('@', 1)[-1]
 | 
			
		||||
        host, port = parse_host_port(netloc)
 | 
			
		||||
        return port
 | 
			
		||||
 | 
			
		||||
 | 
			
		||||
def urlsplit(url, scheme='', allow_fragments=True):
 | 
			
		||||
    """Parse a URL using urlparse.urlsplit(), splitting query and fragments.
 | 
			
		||||
    This function papers over Python issue9374 when needed.
 | 
			
		||||
 | 
			
		||||
    The parameters are the same as urlparse.urlsplit.
 | 
			
		||||
    """
 | 
			
		||||
    scheme, netloc, path, query, fragment = parse.urlsplit(
 | 
			
		||||
        url, scheme, allow_fragments)
 | 
			
		||||
    if allow_fragments and '#' in path:
 | 
			
		||||
        path, fragment = path.split('#', 1)
 | 
			
		||||
    if '?' in path:
 | 
			
		||||
        path, query = path.split('?', 1)
 | 
			
		||||
    return ModifiedSplitResult(scheme, netloc,
 | 
			
		||||
                               path, query, fragment)
 | 
			
		||||
@@ -1,9 +1,7 @@
 | 
			
		||||
oslo.config>=1.2.1
 | 
			
		||||
oslo.utils>=0.2.0
 | 
			
		||||
stevedore>=0.14
 | 
			
		||||
 | 
			
		||||
# for timeutils
 | 
			
		||||
iso8601>=0.1.9
 | 
			
		||||
 | 
			
		||||
# for jsonutils
 | 
			
		||||
six>=1.7.0
 | 
			
		||||
 | 
			
		||||
 
 | 
			
		||||
@@ -1,9 +1,7 @@
 | 
			
		||||
oslo.config>=1.4.0.0a3
 | 
			
		||||
oslo.utils>=0.2.0
 | 
			
		||||
stevedore>=0.14
 | 
			
		||||
 | 
			
		||||
# for timeutils
 | 
			
		||||
iso8601>=0.1.9
 | 
			
		||||
 | 
			
		||||
# for jsonutils
 | 
			
		||||
six>=1.7.0
 | 
			
		||||
 | 
			
		||||
 
 | 
			
		||||
@@ -20,7 +20,7 @@ import testscenarios
 | 
			
		||||
 | 
			
		||||
from oslo import messaging
 | 
			
		||||
from oslo.messaging.notify import dispatcher as notify_dispatcher
 | 
			
		||||
from oslo.messaging.openstack.common import timeutils
 | 
			
		||||
from oslo.utils import timeutils
 | 
			
		||||
from tests import utils as test_utils
 | 
			
		||||
 | 
			
		||||
load_tests = testscenarios.load_tests_apply_scenarios
 | 
			
		||||
 
 | 
			
		||||
@@ -27,7 +27,7 @@ import testscenarios
 | 
			
		||||
import testtools
 | 
			
		||||
 | 
			
		||||
from oslo import messaging
 | 
			
		||||
from oslo.messaging.openstack.common import timeutils
 | 
			
		||||
from oslo.utils import timeutils
 | 
			
		||||
from tests.notify import test_notifier
 | 
			
		||||
from tests import utils as test_utils
 | 
			
		||||
 | 
			
		||||
@@ -63,7 +63,7 @@ class TestLogNotifier(test_utils.BaseTestCase):
 | 
			
		||||
        self.addCleanup(messaging.notify._impl_test.reset)
 | 
			
		||||
        self.config(notification_driver=['test'])
 | 
			
		||||
 | 
			
		||||
    @mock.patch('oslo.messaging.openstack.common.timeutils.utcnow')
 | 
			
		||||
    @mock.patch('oslo.utils.timeutils.utcnow')
 | 
			
		||||
    def test_logger(self, mock_utcnow):
 | 
			
		||||
        with mock.patch('oslo.messaging.transport.get_transport',
 | 
			
		||||
                        return_value=test_notifier._FakeTransport(self.conf)):
 | 
			
		||||
@@ -106,7 +106,7 @@ class TestLogNotifier(test_utils.BaseTestCase):
 | 
			
		||||
 | 
			
		||||
    @testtools.skipUnless(hasattr(logging.config, 'dictConfig'),
 | 
			
		||||
                          "Need logging.config.dictConfig (Python >= 2.7)")
 | 
			
		||||
    @mock.patch('oslo.messaging.openstack.common.timeutils.utcnow')
 | 
			
		||||
    @mock.patch('oslo.utils.timeutils.utcnow')
 | 
			
		||||
    def test_logging_conf(self, mock_utcnow):
 | 
			
		||||
        with mock.patch('oslo.messaging.transport.get_transport',
 | 
			
		||||
                        return_value=test_notifier._FakeTransport(self.conf)):
 | 
			
		||||
 
 | 
			
		||||
@@ -31,8 +31,8 @@ from oslo.messaging.notify import _impl_messaging
 | 
			
		||||
from oslo.messaging.notify import _impl_test
 | 
			
		||||
from oslo.messaging.notify import notifier as msg_notifier
 | 
			
		||||
from oslo.messaging.openstack.common import jsonutils
 | 
			
		||||
from oslo.messaging.openstack.common import timeutils
 | 
			
		||||
from oslo.messaging import serializer as msg_serializer
 | 
			
		||||
from oslo.utils import timeutils
 | 
			
		||||
from tests import utils as test_utils
 | 
			
		||||
 | 
			
		||||
load_tests = testscenarios.load_tests_apply_scenarios
 | 
			
		||||
@@ -147,7 +147,7 @@ class TestMessagingNotifier(test_utils.BaseTestCase):
 | 
			
		||||
        self.stubs.Set(_impl_messaging, 'LOG', self.logger)
 | 
			
		||||
        self.stubs.Set(msg_notifier, '_LOG', self.logger)
 | 
			
		||||
 | 
			
		||||
    @mock.patch('oslo.messaging.openstack.common.timeutils.utcnow')
 | 
			
		||||
    @mock.patch('oslo.utils.timeutils.utcnow')
 | 
			
		||||
    def test_notifier(self, mock_utcnow):
 | 
			
		||||
        drivers = []
 | 
			
		||||
        if self.v1:
 | 
			
		||||
@@ -223,7 +223,7 @@ class TestSerializer(test_utils.BaseTestCase):
 | 
			
		||||
        super(TestSerializer, self).setUp()
 | 
			
		||||
        self.addCleanup(_impl_test.reset)
 | 
			
		||||
 | 
			
		||||
    @mock.patch('oslo.messaging.openstack.common.timeutils.utcnow')
 | 
			
		||||
    @mock.patch('oslo.utils.timeutils.utcnow')
 | 
			
		||||
    def test_serializer(self, mock_utcnow):
 | 
			
		||||
        transport = _FakeTransport(self.conf)
 | 
			
		||||
 | 
			
		||||
@@ -266,7 +266,7 @@ class TestSerializer(test_utils.BaseTestCase):
 | 
			
		||||
 | 
			
		||||
class TestLogNotifier(test_utils.BaseTestCase):
 | 
			
		||||
 | 
			
		||||
    @mock.patch('oslo.messaging.openstack.common.timeutils.utcnow')
 | 
			
		||||
    @mock.patch('oslo.utils.timeutils.utcnow')
 | 
			
		||||
    def test_notifier(self, mock_utcnow):
 | 
			
		||||
        self.config(notification_driver=['log'])
 | 
			
		||||
 | 
			
		||||
 
 | 
			
		||||
		Reference in New Issue
	
	Block a user