From 04866ab9c31f499b612aac17d25584672f832758 Mon Sep 17 00:00:00 2001 From: Julien Danjou Date: Fri, 2 Dec 2016 13:43:33 +0100 Subject: [PATCH] Move ToozError to root module Change-Id: I73b1c5a37b75991f79d8a5fb383c605689503333 --- doc/source/developers.rst | 2 +- tooz/__init__.py | 19 +++++++++++- tooz/coordination.py | 53 +++++++++++++-------------------- tooz/drivers/etcd.py | 3 +- tooz/drivers/file.py | 4 +-- tooz/drivers/ipc.py | 2 +- tooz/drivers/memcached.py | 3 +- tooz/drivers/mysql.py | 4 +-- tooz/drivers/pgsql.py | 2 +- tooz/drivers/redis.py | 24 +++++++-------- tooz/drivers/zookeeper.py | 29 +++++++++--------- tooz/tests/drivers/test_file.py | 3 +- tooz/tests/test_coordination.py | 3 +- tooz/tests/test_mysql.py | 3 +- tooz/tests/test_postgresql.py | 7 +++-- tooz/tests/test_utils.py | 6 ++-- tooz/utils.py | 19 ++++++------ 17 files changed, 100 insertions(+), 86 deletions(-) diff --git a/doc/source/developers.rst b/doc/source/developers.rst index 25df237f..c6baf889 100644 --- a/doc/source/developers.rst +++ b/doc/source/developers.rst @@ -71,7 +71,7 @@ Zookeeper Exceptions ---------- -.. autoclass:: tooz.coordination.ToozError +.. autoclass:: tooz.ToozError .. autoclass:: tooz.coordination.ToozConnectionError .. autoclass:: tooz.coordination.OperationTimedOut .. autoclass:: tooz.coordination.GroupNotCreated diff --git a/tooz/__init__.py b/tooz/__init__.py index 1de10ab1..f2657597 100644 --- a/tooz/__init__.py +++ b/tooz/__init__.py @@ -15,5 +15,22 @@ # under the License. -class NotImplemented(NotImplementedError): +class ToozError(Exception): + """Exception raised when an internal error occurs. + + Raised for instance in case of server internal error. + + :ivar cause: the cause of the exception being raised, when not none this + will itself be an exception instance, this is useful for + creating a chain of exceptions for versions of python where + this is not yet implemented/supported natively. + + """ + + def __init__(self, message, cause=None): + super(ToozError, self).__init__(message) + self.cause = cause + + +class NotImplemented(NotImplementedError, ToozError): pass diff --git a/tooz/coordination.py b/tooz/coordination.py index 1b20eec4..7990bf72 100755 --- a/tooz/coordination.py +++ b/tooz/coordination.py @@ -21,6 +21,7 @@ import enum import logging import threading +from debtcollector import moves from oslo_utils import excutils from oslo_utils import netutils from oslo_utils import timeutils @@ -385,7 +386,7 @@ class CoordinationDriver(object): is initiated. """ if self._started: - raise ToozError( + raise tooz.ToozError( "Can not start a driver which has not been stopped") self._start() if self.requires_beating and start_heart: @@ -404,7 +405,8 @@ class CoordinationDriver(object): disappear from all joined groups. """ if not self._started: - raise ToozError("Can not stop a driver which has not been started") + raise tooz.ToozError( + "Can not stop a driver which has not been started") if self.heart.is_alive(): self.heart.stop() self.heart.wait() @@ -415,7 +417,7 @@ class CoordinationDriver(object): for fut in leaving: try: fut.get() - except ToozError: + except tooz.ToozError: # Whatever happens, ignore. Maybe we got booted out/never # existed in the first place, or something is down, but we just # want to call _stop after whatever happens to not leak any @@ -734,40 +736,28 @@ def get_coordinator(backend_url, member_id, return d -class ToozError(Exception): - """Exception raised when an internal error occurs. - - Raised for instance in case of server internal error. - - :ivar cause: the cause of the exception being raised, when not none this - will itself be an exception instance, this is useful for - creating a chain of exceptions for versions of python where - this is not yet implemented/supported natively. - - """ - - def __init__(self, message, cause=None): - super(ToozError, self).__init__(message) - self.cause = cause +@moves.moved_class(tooz.ToozError, "ToozError", "tooz.coordination") +class ToozError(tooz.ToozError): + pass -class ToozDriverChosenPoorly(ToozError): +class ToozDriverChosenPoorly(tooz.ToozError): """Raised when a driver does not match desired characteristics.""" -class ToozConnectionError(ToozError): +class ToozConnectionError(tooz.ToozError): """Exception raised when the client cannot connect to the server.""" -class OperationTimedOut(ToozError): +class OperationTimedOut(tooz.ToozError): """Exception raised when an operation times out.""" -class LockAcquireFailed(ToozError): +class LockAcquireFailed(tooz.ToozError): """Exception raised when a lock acquire fails in a context manager.""" -class GroupNotCreated(ToozError): +class GroupNotCreated(tooz.ToozError): """Exception raised when the caller request an nonexistent group.""" def __init__(self, group_id): self.group_id = group_id @@ -775,7 +765,7 @@ class GroupNotCreated(ToozError): "Group %s does not exist" % group_id) -class GroupAlreadyExist(ToozError): +class GroupAlreadyExist(tooz.ToozError): """Exception raised trying to create an already existing group.""" def __init__(self, group_id): self.group_id = group_id @@ -783,7 +773,7 @@ class GroupAlreadyExist(ToozError): "Group %s already exists" % group_id) -class MemberAlreadyExist(ToozError): +class MemberAlreadyExist(tooz.ToozError): """Exception raised trying to join a group already joined.""" def __init__(self, group_id, member_id): self.group_id = group_id @@ -793,7 +783,7 @@ class MemberAlreadyExist(ToozError): (member_id, group_id)) -class MemberNotJoined(ToozError): +class MemberNotJoined(tooz.ToozError): """Exception raised trying to access a member not in a group.""" def __init__(self, group_id, member_id): self.group_id = group_id @@ -802,14 +792,14 @@ class MemberNotJoined(ToozError): (member_id, group_id)) -class GroupNotEmpty(ToozError): +class GroupNotEmpty(tooz.ToozError): "Exception raised when the caller try to delete a group with members." def __init__(self, group_id): self.group_id = group_id super(GroupNotEmpty, self).__init__("Group %s is not empty" % group_id) -class WatchCallbackNotFound(ToozError): +class WatchCallbackNotFound(tooz.ToozError): """Exception raised when unwatching a group. Raised when the caller tries to unwatch a group with a callback that @@ -824,7 +814,7 @@ class WatchCallbackNotFound(ToozError): (callback.__name__, group_id)) -class SerializationError(ToozError): +class SerializationError(tooz.ToozError): "Exception raised when serialization or deserialization breaks." @@ -837,8 +827,7 @@ def raise_with_cause(exc_cls, message, *args, **kwargs): :pep:`3134`) we should try to raise the desired exception with the given *cause*. - :param exc_cls: the :py:class:`~tooz.coordination.ToozError` class - to raise. + :param exc_cls: the :py:class:`~tooz.ToozError` class to raise. :param message: the text/str message that will be passed to the exceptions constructor as its first positional argument. @@ -847,6 +836,6 @@ def raise_with_cause(exc_cls, message, *args, **kwargs): :param kwargs: any additional keyword arguments to pass to the exceptions constructor. """ - if not issubclass(exc_cls, ToozError): + if not issubclass(exc_cls, tooz.ToozError): raise ValueError("Subclass of tooz error is required") excutils.raise_with_cause(exc_cls, message, *args, **kwargs) diff --git a/tooz/drivers/etcd.py b/tooz/drivers/etcd.py index ee508600..f1cc9f8b 100644 --- a/tooz/drivers/etcd.py +++ b/tooz/drivers/etcd.py @@ -21,6 +21,7 @@ from oslo_utils import timeutils import requests import six +import tooz from tooz import coordination from tooz import locking from tooz import utils @@ -38,7 +39,7 @@ def _translate_failures(func): return func(*args, **kwargs) except ValueError as e: # Typically json decoding failed for some reason. - coordination.raise_with_cause(coordination.ToozError, + coordination.raise_with_cause(tooz.ToozError, encodeutils.exception_to_unicode(e), cause=e) except requests.exceptions.RequestException as e: diff --git a/tooz/drivers/file.py b/tooz/drivers/file.py index 32f5c8a6..393bef7f 100644 --- a/tooz/drivers/file.py +++ b/tooz/drivers/file.py @@ -55,7 +55,7 @@ def _translate_failures(): try: yield except (EnvironmentError, voluptuous.Invalid) as e: - coordination.raise_with_cause(coordination.ToozError, + coordination.raise_with_cause(tooz.ToozError, encodeutils.exception_to_unicode(e), cause=e) @@ -445,7 +445,7 @@ class FileDriver(coordination.CoordinationDriverCachedRunWatchers): if len(entries) > 1: raise coordination.GroupNotEmpty(group_id) elif len(entries) == 1 and entries != ['.metadata']: - raise coordination.ToozError( + raise tooz.ToozError( "Unexpected path '%s' found in" " group directory '%s' (expected to only find" " a '.metadata' path)" % (entries[0], group_dir)) diff --git a/tooz/drivers/ipc.py b/tooz/drivers/ipc.py index 0dcd3aaf..73dcbb08 100644 --- a/tooz/drivers/ipc.py +++ b/tooz/drivers/ipc.py @@ -196,7 +196,7 @@ class IPCDriver(coordination.CoordinationDriver): def _write_group_list(self, group_list): data = msgpack.dumps(list(group_list)) if len(data) >= self._SEGMENT_SIZE - 2: - raise coordination.ToozError("Group list is too big") + raise tooz.ToozError("Group list is too big") self._group_list.write(struct.pack('H', len(data))) self._group_list.write(data, offset=2) diff --git a/tooz/drivers/memcached.py b/tooz/drivers/memcached.py index 239d4eb2..57f3d9e4 100644 --- a/tooz/drivers/memcached.py +++ b/tooz/drivers/memcached.py @@ -23,6 +23,7 @@ from oslo_utils import encodeutils from pymemcache import client as pymemcache_client import six +import tooz from tooz import _retry from tooz import coordination from tooz import locking @@ -58,7 +59,7 @@ def _translate_failures(func): coordination.raise_with_cause(coordination.ToozConnectionError, msg, cause=e) except pymemcache_client.MemcacheError as e: - coordination.raise_with_cause(coordination.ToozError, + coordination.raise_with_cause(tooz.ToozError, encodeutils.exception_to_unicode(e), cause=e) diff --git a/tooz/drivers/mysql.py b/tooz/drivers/mysql.py index 8889708e..3d1e2047 100644 --- a/tooz/drivers/mysql.py +++ b/tooz/drivers/mysql.py @@ -64,7 +64,7 @@ class MySQLLock(locking.Lock): return True except pymysql.MySQLError as e: coordination.raise_with_cause( - coordination.ToozError, + tooz.ToozError, encodeutils.exception_to_unicode(e), cause=e) @@ -84,7 +84,7 @@ class MySQLLock(locking.Lock): self.acquired = False return True except pymysql.MySQLError as e: - coordination.raise_with_cause(coordination.ToozError, + coordination.raise_with_cause(tooz.ToozError, encodeutils.exception_to_unicode(e), cause=e) diff --git a/tooz/drivers/pgsql.py b/tooz/drivers/pgsql.py index cb0b9b83..08902c63 100644 --- a/tooz/drivers/pgsql.py +++ b/tooz/drivers/pgsql.py @@ -85,7 +85,7 @@ def _translating_cursor(conn): with conn.cursor() as cur: yield cur except psycopg2.Error as e: - coordination.raise_with_cause(coordination.ToozError, + coordination.raise_with_cause(tooz.ToozError, _format_exception(e), cause=e) diff --git a/tooz/drivers/redis.py b/tooz/drivers/redis.py index 01b7dbe8..8ccd2b44 100644 --- a/tooz/drivers/redis.py +++ b/tooz/drivers/redis.py @@ -49,7 +49,7 @@ def _translate_failures(): encodeutils.exception_to_unicode(e), cause=e) except exceptions.RedisError as e: - coordination.raise_with_cause(coordination.ToozError, + coordination.raise_with_cause(tooz.ToozError, encodeutils.exception_to_unicode(e), cause=e) @@ -501,7 +501,7 @@ return 1 for lock in self._acquired_locks.copy(): try: lock.heartbeat() - except coordination.ToozError: + except tooz.ToozError: LOG.warning("Unable to heartbeat lock '%s'", lock, exc_info=True) return min(self.lock_timeout, self.membership_timeout) @@ -511,7 +511,7 @@ return 1 lock = self._acquired_locks.pop() try: lock.release() - except coordination.ToozError: + except tooz.ToozError: LOG.warning("Unable to release lock '%s'", lock, exc_info=True) self._executor.stop() if self._client is not None: @@ -522,7 +522,7 @@ return 1 # exist in the first place, which is fine/expected/desired... with _translate_failures(): self._client.delete(beat_id) - except coordination.ToozError: + except tooz.ToozError: LOG.warning("Unable to delete heartbeat key '%s'", beat_id, exc_info=True) self._client = None @@ -532,14 +532,14 @@ return 1 def _submit(self, cb, *args, **kwargs): if not self._started: - raise coordination.ToozError("Redis driver has not been started") + raise tooz.ToozError("Redis driver has not been started") return self._executor.submit(cb, *args, **kwargs) def _get_script(self, script_key): try: return self._scripts[script_key] except KeyError: - raise coordination.ToozError("Redis driver has not been started") + raise tooz.ToozError("Redis driver has not been started") def create_group(self, group_id): script = self._get_script('create_group') @@ -696,13 +696,13 @@ return 1 if result == -3: raise coordination.GroupNotEmpty(group_id) if result == -4: - raise coordination.ToozError("Unable to remove '%s' key" - " from set located at '%s'" - % (args[0], keys[-1])) + raise tooz.ToozError("Unable to remove '%s' key" + " from set located at '%s'" + % (args[0], keys[-1])) if result != 1: - raise coordination.ToozError("Internal error, unable" - " to complete group '%s' removal" - % (group_id)) + raise tooz.ToozError("Internal error, unable" + " to complete group '%s' removal" + % (group_id)) return RedisFutureResult(self._submit(_delete_group, script)) diff --git a/tooz/drivers/zookeeper.py b/tooz/drivers/zookeeper.py index de66220b..006bef67 100644 --- a/tooz/drivers/zookeeper.py +++ b/tooz/drivers/zookeeper.py @@ -30,6 +30,7 @@ from oslo_utils import strutils import six from six.moves import filter as compat_filter +import tooz from tooz import coordination from tooz import locking from tooz import utils @@ -54,7 +55,7 @@ class ZooKeeperLock(locking.Lock): exceptions.NoNodeError): return False except exceptions.KazooException as e: - coordination.raise_with_cause(coordination.ToozError, + coordination.raise_with_cause(tooz.ToozError, "operation error: %s" % (e), cause=e) @@ -154,7 +155,7 @@ class KazooDriver(coordination.CoordinationDriverCachedRunWatchers): self._coord.ensure_path(self._paths_join("/", self._namespace)) except exceptions.KazooException as e: e_msg = encodeutils.exception_to_unicode(e) - coordination.raise_with_cause(coordination.ToozError, + coordination.raise_with_cause(tooz.ToozError, "Operational error: %s" % e_msg, cause=e) self._watchers = collections.deque() @@ -182,12 +183,12 @@ class KazooDriver(coordination.CoordinationDriverCachedRunWatchers): except exceptions.NodeExistsError: raise coordination.GroupAlreadyExist(group_id) except exceptions.NoNodeError as e: - coordination.raise_with_cause(coordination.ToozError, + coordination.raise_with_cause(tooz.ToozError, "Tooz namespace '%s' has not" " been created" % self._namespace, cause=e) except exceptions.ZookeeperError as e: - coordination.raise_with_cause(coordination.ToozError, + coordination.raise_with_cause(tooz.ToozError, encodeutils.exception_to_unicode(e), cause=e) @@ -212,7 +213,7 @@ class KazooDriver(coordination.CoordinationDriverCachedRunWatchers): except exceptions.NotEmptyError: raise coordination.GroupNotEmpty(group_id) except exceptions.ZookeeperError as e: - coordination.raise_with_cause(coordination.ToozError, + coordination.raise_with_cause(tooz.ToozError, encodeutils.exception_to_unicode(e), cause=e) @@ -237,7 +238,7 @@ class KazooDriver(coordination.CoordinationDriverCachedRunWatchers): except exceptions.NoNodeError: raise coordination.GroupNotCreated(group_id) except exceptions.ZookeeperError as e: - coordination.raise_with_cause(coordination.ToozError, + coordination.raise_with_cause(tooz.ToozError, encodeutils.exception_to_unicode(e), cause=e) @@ -263,7 +264,7 @@ class KazooDriver(coordination.CoordinationDriverCachedRunWatchers): except exceptions.NoNodeError: raise coordination.MemberNotJoined(group_id, member_id) except exceptions.ZookeeperError as e: - coordination.raise_with_cause(coordination.ToozError, + coordination.raise_with_cause(tooz.ToozError, encodeutils.exception_to_unicode(e), cause=e) @@ -281,7 +282,7 @@ class KazooDriver(coordination.CoordinationDriverCachedRunWatchers): except exceptions.NoNodeError: pass except exceptions.ZookeeperError as e: - coordination.raise_with_cause(coordination.ToozError, + coordination.raise_with_cause(tooz.ToozError, encodeutils.exception_to_unicode(e), cause=e) return self.timeout @@ -305,7 +306,7 @@ class KazooDriver(coordination.CoordinationDriverCachedRunWatchers): except exceptions.NoNodeError: raise coordination.GroupNotCreated(group_id) except exceptions.ZookeeperError as e: - coordination.raise_with_cause(coordination.ToozError, + coordination.raise_with_cause(tooz.ToozError, encodeutils.exception_to_unicode(e), cause=e) else: @@ -330,7 +331,7 @@ class KazooDriver(coordination.CoordinationDriverCachedRunWatchers): except exceptions.NoNodeError: raise coordination.MemberNotJoined(group_id, member_id) except exceptions.ZookeeperError as e: - coordination.raise_with_cause(coordination.ToozError, + coordination.raise_with_cause(tooz.ToozError, encodeutils.exception_to_unicode(e), cause=e) @@ -355,7 +356,7 @@ class KazooDriver(coordination.CoordinationDriverCachedRunWatchers): except exceptions.NoNodeError: raise coordination.MemberNotJoined(group_id, member_id) except exceptions.ZookeeperError as e: - coordination.raise_with_cause(coordination.ToozError, + coordination.raise_with_cause(tooz.ToozError, encodeutils.exception_to_unicode(e), cause=e) else: @@ -383,7 +384,7 @@ class KazooDriver(coordination.CoordinationDriverCachedRunWatchers): except exceptions.NoNodeError: raise coordination.MemberNotJoined(group_id, member_id) except exceptions.ZookeeperError as e: - coordination.raise_with_cause(coordination.ToozError, + coordination.raise_with_cause(tooz.ToozError, encodeutils.exception_to_unicode(e), cause=e) else: @@ -410,12 +411,12 @@ class KazooDriver(coordination.CoordinationDriverCachedRunWatchers): encodeutils.exception_to_unicode(e), cause=e) except exceptions.NoNodeError as e: - coordination.raise_with_cause(coordination.ToozError, + coordination.raise_with_cause(tooz.ToozError, "Tooz namespace '%s' has not" " been created" % self._namespace, cause=e) except exceptions.ZookeeperError as e: - coordination.raise_with_cause(coordination.ToozError, + coordination.raise_with_cause(tooz.ToozError, encodeutils.exception_to_unicode(e), cause=e) else: diff --git a/tooz/tests/drivers/test_file.py b/tooz/tests/drivers/test_file.py index f392f188..149c371a 100644 --- a/tooz/tests/drivers/test_file.py +++ b/tooz/tests/drivers/test_file.py @@ -21,6 +21,7 @@ import fixtures import mock from testtools import testcase +import tooz from tooz import coordination from tooz import tests @@ -52,7 +53,7 @@ class TestFileDriver(testcase.TestCase): pass os.unlink(os.path.join(file_path, 'groups', safe_group_id, '.metadata')) - self.assertRaises(coordination.ToozError, + self.assertRaises(tooz.ToozError, coord.delete_group(b"my_group").get) @mock.patch('os.path.normpath', lambda x: x.replace('/', '\\')) diff --git a/tooz/tests/test_coordination.py b/tooz/tests/test_coordination.py index c073161b..6f5979f8 100644 --- a/tooz/tests/test_coordination.py +++ b/tooz/tests/test_coordination.py @@ -25,6 +25,7 @@ from six.moves.urllib import parse from testtools import matchers from testtools import testcase +import tooz import tooz.coordination from tooz import tests @@ -78,7 +79,7 @@ class TestAPI(tests.TestCaseSkipNotImplemented): def test_stop_first(self): c = tooz.coordination.get_coordinator(self.url, self.member_id) - self.assertRaises(tooz.coordination.ToozError, + self.assertRaises(tooz.ToozError, c.stop) def test_create_group(self): diff --git a/tooz/tests/test_mysql.py b/tooz/tests/test_mysql.py index 01cc93fd..291f44ad 100644 --- a/tooz/tests/test_mysql.py +++ b/tooz/tests/test_mysql.py @@ -18,6 +18,7 @@ from oslo_utils import encodeutils from testtools import testcase +import tooz from tooz import coordination from tooz import tests @@ -29,7 +30,7 @@ class TestMySQLDriver(testcase.TestCase): def _safe_stop(coord): try: coord.stop() - except coordination.ToozError as e: + except tooz.ToozError as e: message = encodeutils.exception_to_unicode(e) if (message != 'Can not stop a driver which has not' ' been started'): diff --git a/tooz/tests/test_postgresql.py b/tooz/tests/test_postgresql.py index 24d86ea1..8556d94a 100644 --- a/tooz/tests/test_postgresql.py +++ b/tooz/tests/test_postgresql.py @@ -24,6 +24,7 @@ from oslo_utils import encodeutils import testtools from testtools import testcase +import tooz from tooz import coordination from tooz import tests @@ -47,7 +48,7 @@ class TestPostgreSQLFailures(testcase.TestCase): def _safe_stop(coord): try: coord.stop() - except coordination.ToozError as e: + except tooz.ToozError as e: # TODO(harlowja): make this better, so that we don't have to # do string checking... message = encodeutils.exception_to_unicode(e) @@ -88,7 +89,7 @@ class TestPostgreSQLFailures(testcase.TestCase): c = self._create_coordinator() c.start() test_lock = c.get_lock(b'test-lock') - self.assertRaises(coordination.ToozError, test_lock.acquire) + self.assertRaises(tooz.ToozError, test_lock.acquire) @mock.patch("tooz.drivers.pgsql.psycopg2.connect") def test_failure_release_lock(self, psycopg2_connector): @@ -110,4 +111,4 @@ class TestPostgreSQLFailures(testcase.TestCase): c.start() test_lock = c.get_lock(b'test-lock') self.assertTrue(test_lock.acquire()) - self.assertRaises(coordination.ToozError, test_lock.release) + self.assertRaises(tooz.ToozError, test_lock.release) diff --git a/tooz/tests/test_utils.py b/tooz/tests/test_utils.py index 6145f4ca..39f83fb0 100644 --- a/tooz/tests/test_utils.py +++ b/tooz/tests/test_utils.py @@ -22,7 +22,7 @@ import futurist import six from testtools import testcase -from tooz import coordination +import tooz from tooz import utils @@ -54,13 +54,13 @@ class TestProxyExecutor(testcase.TestCase): def test_fetch_unknown_executor(self): options = {'executor': 'huh'} - self.assertRaises(coordination.ToozError, + self.assertRaises(tooz.ToozError, utils.ProxyExecutor.build, 'test', options) def test_no_submit_stopped(self): executor = utils.ProxyExecutor.build("test", {}) - self.assertRaises(coordination.ToozError, + self.assertRaises(tooz.ToozError, executor.submit, lambda: None) diff --git a/tooz/utils.py b/tooz/utils.py index c166d93f..88367c68 100644 --- a/tooz/utils.py +++ b/tooz/utils.py @@ -25,6 +25,7 @@ from oslo_serialization import msgpackutils from oslo_utils import encodeutils import six +import tooz from tooz import coordination @@ -82,10 +83,10 @@ class ProxyExecutor(object): default_executor_fact = cls.KIND_TO_FACTORY[executor_kind] except KeyError: executors_known = sorted(list(cls.KIND_TO_FACTORY)) - raise coordination.ToozError("Unknown executor" - " '%s' provided, accepted values" - " are %s" % (executor_kind, - executors_known)) + raise tooz.ToozError("Unknown executor" + " '%s' provided, accepted values" + " are %s" % (executor_kind, + executors_known)) return cls(driver_name, default_executor_fact) def start(self): @@ -103,14 +104,14 @@ class ProxyExecutor(object): def submit(self, cb, *args, **kwargs): if not self.started: - raise coordination.ToozError("%s driver asynchronous executor" - " has not been started" - % self.driver_name) + raise tooz.ToozError("%s driver asynchronous executor" + " has not been started" + % self.driver_name) try: return self.executor.submit(cb, *args, **kwargs) except RuntimeError: - raise coordination.ToozError("%s driver asynchronous executor has" - " been shutdown" % self.driver_name) + raise tooz.ToozError("%s driver asynchronous executor has" + " been shutdown" % self.driver_name) def safe_abs_path(rooted_at, *pieces):