Move the cached-based watcher implementation into its own class

There's a lot of methods implementing the common watcher tools that are
inherited from all drivers whereas there is no need for that. The mixin
is actually not correctly designed, as it mixes the hooks and the cache
code.

This splits out the common implementation into its common class that can
be inherited to implement this model of driver.

The Kazoo and Zake drivers are also merged to simplify code, as the base
class was not used.

Change-Id: I1acbf1983165d00e68b96cbca2bc5c1e2acadfa8
This commit is contained in:
Julien Danjou
2016-11-10 17:31:35 +01:00
parent ca432a0de5
commit 0b1723ed77
10 changed files with 119 additions and 233 deletions

View File

@@ -215,7 +215,6 @@ class Heart(object):
return self._finished.is_set() return self._finished.is_set()
@six.add_metaclass(abc.ABCMeta)
class CoordinationDriver(object): class CoordinationDriver(object):
requires_beating = False requires_beating = False
@@ -233,20 +232,19 @@ class CoordinationDriver(object):
""" """
def __init__(self): def __init__(self):
super(CoordinationDriver, self).__init__()
self._started = False self._started = False
self._hooks_join_group = collections.defaultdict(Hooks) self._hooks_join_group = collections.defaultdict(Hooks)
self._hooks_leave_group = collections.defaultdict(Hooks) self._hooks_leave_group = collections.defaultdict(Hooks)
self._hooks_elected_leader = collections.defaultdict(Hooks) self._hooks_elected_leader = collections.defaultdict(Hooks)
# A cache for group members
self._group_members = collections.defaultdict(set)
self.requires_beating = ( self.requires_beating = (
CoordinationDriver.heartbeat != self.__class__.heartbeat CoordinationDriver.heartbeat != self.__class__.heartbeat
) )
self.heart = Heart(self) self.heart = Heart(self)
def _has_hooks_for_group(self, group_id): def _has_hooks_for_group(self, group_id):
return (len(self._hooks_join_group[group_id]) + return (group_id in self._hooks_join_group or
len(self._hooks_leave_group[group_id])) group_id in self._hooks_leave_group)
@staticmethod @staticmethod
def run_watchers(timeout=None): def run_watchers(timeout=None):
@@ -262,7 +260,6 @@ class CoordinationDriver(object):
"""Try to leader elect this coordinator & activate hooks on success.""" """Try to leader elect this coordinator & activate hooks on success."""
raise tooz.NotImplemented raise tooz.NotImplemented
@abc.abstractmethod
def watch_join_group(self, group_id, callback): def watch_join_group(self, group_id, callback):
"""Call a function when group_id sees a new member joined. """Call a function when group_id sees a new member joined.
@@ -275,7 +272,6 @@ class CoordinationDriver(object):
""" """
self._hooks_join_group[group_id].append(callback) self._hooks_join_group[group_id].append(callback)
@abc.abstractmethod
def unwatch_join_group(self, group_id, callback): def unwatch_join_group(self, group_id, callback):
"""Stop executing a function when a group_id sees a new member joined. """Stop executing a function when a group_id sees a new member joined.
@@ -284,15 +280,17 @@ class CoordinationDriver(object):
this group this group
""" """
try: try:
# Check if group_id is in hooks to avoid creating a default empty
# entry in hooks list.
if group_id not in self._hooks_join_group:
raise ValueError
self._hooks_join_group[group_id].remove(callback) self._hooks_join_group[group_id].remove(callback)
except ValueError: except ValueError:
raise WatchCallbackNotFound(group_id, callback) raise WatchCallbackNotFound(group_id, callback)
if (not self._has_hooks_for_group(group_id) and if not self._hooks_join_group[group_id]:
group_id in self._group_members): del self._hooks_join_group[group_id]
del self._group_members[group_id]
@abc.abstractmethod
def watch_leave_group(self, group_id, callback): def watch_leave_group(self, group_id, callback):
"""Call a function when group_id sees a new member leaving. """Call a function when group_id sees a new member leaving.
@@ -306,7 +304,6 @@ class CoordinationDriver(object):
""" """
self._hooks_leave_group[group_id].append(callback) self._hooks_leave_group[group_id].append(callback)
@abc.abstractmethod
def unwatch_leave_group(self, group_id, callback): def unwatch_leave_group(self, group_id, callback):
"""Stop executing a function when a group_id sees a new member leaving. """Stop executing a function when a group_id sees a new member leaving.
@@ -315,18 +312,17 @@ class CoordinationDriver(object):
this group this group
""" """
try: try:
# Check if group_id is in hooks to avoid creating a default empty
# entry in hooks list.
if group_id not in self._hooks_leave_group:
raise ValueError
self._hooks_leave_group[group_id].remove(callback) self._hooks_leave_group[group_id].remove(callback)
except ValueError: except ValueError:
raise WatchCallbackNotFound(group_id, callback) raise WatchCallbackNotFound(group_id, callback)
if (not self._has_hooks_for_group(group_id) and
group_id in self._group_members):
del self._group_members[group_id]
if not self._hooks_leave_group[group_id]: if not self._hooks_leave_group[group_id]:
del self._hooks_leave_group[group_id] del self._hooks_leave_group[group_id]
@abc.abstractmethod
def watch_elected_as_leader(self, group_id, callback): def watch_elected_as_leader(self, group_id, callback):
"""Call a function when member gets elected as leader. """Call a function when member gets elected as leader.
@@ -340,7 +336,6 @@ class CoordinationDriver(object):
""" """
self._hooks_elected_leader[group_id].append(callback) self._hooks_elected_leader[group_id].append(callback)
@abc.abstractmethod
def unwatch_elected_as_leader(self, group_id, callback): def unwatch_elected_as_leader(self, group_id, callback):
"""Call a function when member gets elected as leader. """Call a function when member gets elected as leader.
@@ -606,8 +601,36 @@ class CoordAsyncResult(object):
"""Returns True if the task is done, False otherwise.""" """Returns True if the task is done, False otherwise."""
class _RunWatchersMixin(object): class CoordinationDriverCachedRunWatchers(CoordinationDriver):
"""Mixin to share the *mostly* common ``run_watchers`` implementation.""" """Coordination driver with a `run_watchers` implementation.
This implementation of `run_watchers` is based on a cache of the group
members between each run of `run_watchers` that is being updated between
each run.
"""
def __init__(self):
super(CoordinationDriverCachedRunWatchers, self).__init__()
# A cache for group members
self._group_members = collections.defaultdict(set)
self._joined_groups = set()
def unwatch_join_group(self, group_id, callback):
super(CoordinationDriverCachedRunWatchers, self).unwatch_join_group(
group_id, callback)
if (not self._has_hooks_for_group(group_id) and
group_id in self._group_members):
del self._group_members[group_id]
def unwatch_leave_group(self, group_id, callback):
super(CoordinationDriverCachedRunWatchers, self).unwatch_leave_group(
group_id, callback)
if (not self._has_hooks_for_group(group_id) and
group_id in self._group_members):
del self._group_members[group_id]
def run_watchers(self, timeout=None): def run_watchers(self, timeout=None):
with timeutils.StopWatch(duration=timeout) as w: with timeutils.StopWatch(duration=timeout) as w:

View File

@@ -19,7 +19,6 @@ from __future__ import absolute_import
import consul import consul
from oslo_utils import encodeutils from oslo_utils import encodeutils
import tooz
from tooz import _retry from tooz import _retry
from tooz import coordination from tooz import coordination
from tooz import locking from tooz import locking
@@ -160,27 +159,3 @@ class ConsulDriver(coordination.CoordinationDriver):
for arg in args: for arg in args:
pieces.append(encodeutils.safe_decode(arg)) pieces.append(encodeutils.safe_decode(arg))
return u"/".join(pieces) return u"/".join(pieces)
@staticmethod
def watch_join_group(group_id, callback):
raise tooz.NotImplemented
@staticmethod
def unwatch_join_group(group_id, callback):
raise tooz.NotImplemented
@staticmethod
def watch_leave_group(group_id, callback):
raise tooz.NotImplemented
@staticmethod
def unwatch_leave_group(group_id, callback):
raise tooz.NotImplemented
@staticmethod
def watch_elected_as_leader(group_id, callback):
raise tooz.NotImplemented
@staticmethod
def unwatch_elected_as_leader(group_id, callback):
raise tooz.NotImplemented

View File

@@ -21,7 +21,6 @@ from oslo_utils import timeutils
import requests import requests
import six import six
import tooz
from tooz import coordination from tooz import coordination
from tooz import locking from tooz import locking
from tooz import utils from tooz import utils
@@ -227,27 +226,3 @@ class EtcdDriver(coordination.CoordinationDriver):
for lock in self._acquired_locks: for lock in self._acquired_locks:
lock.heartbeat() lock.heartbeat()
return self.lock_timeout return self.lock_timeout
@staticmethod
def watch_join_group(group_id, callback):
raise tooz.NotImplemented
@staticmethod
def unwatch_join_group(group_id, callback):
raise tooz.NotImplemented
@staticmethod
def watch_leave_group(group_id, callback):
raise tooz.NotImplemented
@staticmethod
def unwatch_leave_group(group_id, callback):
raise tooz.NotImplemented
@staticmethod
def watch_elected_as_leader(group_id, callback):
raise tooz.NotImplemented
@staticmethod
def unwatch_elected_as_leader(group_id, callback):
raise tooz.NotImplemented

View File

@@ -163,8 +163,7 @@ class FileLock(locking.Lock):
LOG.warning("Unreleased lock %s garbage collected", self.name) LOG.warning("Unreleased lock %s garbage collected", self.name)
class FileDriver(coordination._RunWatchersMixin, class FileDriver(coordination.CoordinationDriverCachedRunWatchers):
coordination.CoordinationDriver):
"""A file based driver. """A file based driver.
This driver uses files and directories (and associated file locks) to This driver uses files and directories (and associated file locks) to
@@ -493,16 +492,10 @@ class FileDriver(coordination._RunWatchersMixin,
self._init_watch_group(group_id) self._init_watch_group(group_id)
return super(FileDriver, self).watch_join_group(group_id, callback) return super(FileDriver, self).watch_join_group(group_id, callback)
def unwatch_join_group(self, group_id, callback):
return super(FileDriver, self).unwatch_join_group(group_id, callback)
def watch_leave_group(self, group_id, callback): def watch_leave_group(self, group_id, callback):
self._init_watch_group(group_id) self._init_watch_group(group_id)
return super(FileDriver, self).watch_leave_group(group_id, callback) return super(FileDriver, self).watch_leave_group(group_id, callback)
def unwatch_leave_group(self, group_id, callback):
return super(FileDriver, self).unwatch_leave_group(group_id, callback)
@staticmethod @staticmethod
def watch_elected_as_leader(group_id, callback): def watch_elected_as_leader(group_id, callback):
raise tooz.NotImplemented raise tooz.NotImplemented

View File

@@ -222,6 +222,16 @@ class IPCDriver(coordination.CoordinationDriver):
return IPCFutureResult(self._executor.submit(_delete_group)) return IPCFutureResult(self._executor.submit(_delete_group))
def watch_join_group(self, group_id, callback):
# Check the group exist
self.get_members(group_id).get()
super(IPCDriver, self).watch_join_group(group_id, callback)
def watch_leave_group(self, group_id, callback):
# Check the group exist
self.get_members(group_id).get()
super(IPCDriver, self).watch_leave_group(group_id, callback)
def _get_groups_handler(self): def _get_groups_handler(self):
with self._lock: with self._lock:
return self._read_group_list() return self._read_group_list()
@@ -234,30 +244,6 @@ class IPCDriver(coordination.CoordinationDriver):
def get_lock(name): def get_lock(name):
return IPCLock(name) return IPCLock(name)
@staticmethod
def watch_join_group(group_id, callback):
raise tooz.NotImplemented
@staticmethod
def unwatch_join_group(group_id, callback):
raise tooz.NotImplemented
@staticmethod
def watch_leave_group(group_id, callback):
raise tooz.NotImplemented
@staticmethod
def unwatch_leave_group(group_id, callback):
raise tooz.NotImplemented
@staticmethod
def watch_elected_as_leader(group_id, callback):
raise tooz.NotImplemented
@staticmethod
def unwatch_elected_as_leader(group_id, callback):
raise tooz.NotImplemented
class IPCFutureResult(coordination.CoordAsyncResult): class IPCFutureResult(coordination.CoordAsyncResult):
"""IPC asynchronous result that references a future.""" """IPC asynchronous result that references a future."""

View File

@@ -14,7 +14,6 @@
# License for the specific language governing permissions and limitations # License for the specific language governing permissions and limitations
# under the License. # under the License.
import collections
import errno import errno
import logging import logging
import socket import socket
@@ -170,8 +169,7 @@ class MemcachedLock(locking.Lock):
return self in self.coord._acquired_locks return self in self.coord._acquired_locks
class MemcachedDriver(coordination._RunWatchersMixin, class MemcachedDriver(coordination.CoordinationDriverCachedRunWatchers):
coordination.CoordinationDriver):
"""A `memcached`_ based driver. """A `memcached`_ based driver.
This driver users `memcached`_ concepts to provide the coordination driver This driver users `memcached`_ concepts to provide the coordination driver
@@ -222,7 +220,6 @@ class MemcachedDriver(coordination._RunWatchersMixin,
options = utils.collapse(options) options = utils.collapse(options)
self._options = options self._options = options
self._member_id = member_id self._member_id = member_id
self._joined_groups = set()
self._executor = utils.ProxyExecutor.build("Memcached", options) self._executor = utils.ProxyExecutor.build("Memcached", options)
self.host = (parsed_url.hostname or "localhost", self.host = (parsed_url.hostname or "localhost",
parsed_url.port or 11211) parsed_url.port or 11211)
@@ -268,7 +265,6 @@ class MemcachedDriver(coordination._RunWatchersMixin,
# Run heartbeat here because pymemcache use a lazy connection # Run heartbeat here because pymemcache use a lazy connection
# method and only connect once you do an operation. # method and only connect once you do an operation.
self.heartbeat() self.heartbeat()
self._group_members = collections.defaultdict(set)
self._executor.start() self._executor.start()
@_translate_failures @_translate_failures
@@ -495,27 +491,11 @@ class MemcachedDriver(coordination._RunWatchersMixin,
return super(MemcachedDriver, self).watch_join_group( return super(MemcachedDriver, self).watch_join_group(
group_id, callback) group_id, callback)
def unwatch_join_group(self, group_id, callback):
return super(MemcachedDriver, self).unwatch_join_group(
group_id, callback)
def watch_leave_group(self, group_id, callback): def watch_leave_group(self, group_id, callback):
self._init_watch_group(group_id) self._init_watch_group(group_id)
return super(MemcachedDriver, self).watch_leave_group( return super(MemcachedDriver, self).watch_leave_group(
group_id, callback) group_id, callback)
def unwatch_leave_group(self, group_id, callback):
return super(MemcachedDriver, self).unwatch_leave_group(
group_id, callback)
def watch_elected_as_leader(self, group_id, callback):
return super(MemcachedDriver, self).watch_elected_as_leader(
group_id, callback)
def unwatch_elected_as_leader(self, group_id, callback):
return super(MemcachedDriver, self).unwatch_elected_as_leader(
group_id, callback)
def get_lock(self, name): def get_lock(self, name):
return MemcachedLock(self, name, self.lock_timeout) return MemcachedLock(self, name, self.lock_timeout)

View File

@@ -106,8 +106,7 @@ class RedisLock(locking.Lock):
return self in self._coord._acquired_locks return self in self._coord._acquired_locks
class RedisDriver(coordination._RunWatchersMixin, class RedisDriver(coordination.CoordinationDriverCachedRunWatchers):
coordination.CoordinationDriver):
"""Redis provides a few nice benefits that act as a poormans zookeeper. """Redis provides a few nice benefits that act as a poormans zookeeper.
It **is** fully functional and implements all of the coordination It **is** fully functional and implements all of the coordination
@@ -730,24 +729,10 @@ return 1
self._init_watch_group(group_id) self._init_watch_group(group_id)
return super(RedisDriver, self).watch_join_group(group_id, callback) return super(RedisDriver, self).watch_join_group(group_id, callback)
def unwatch_join_group(self, group_id, callback):
return super(RedisDriver, self).unwatch_join_group(group_id, callback)
def watch_leave_group(self, group_id, callback): def watch_leave_group(self, group_id, callback):
self._init_watch_group(group_id) self._init_watch_group(group_id)
return super(RedisDriver, self).watch_leave_group(group_id, callback) return super(RedisDriver, self).watch_leave_group(group_id, callback)
def unwatch_leave_group(self, group_id, callback):
return super(RedisDriver, self).unwatch_leave_group(group_id, callback)
def watch_elected_as_leader(self, group_id, callback):
return super(RedisDriver, self).watch_elected_as_leader(
group_id, callback)
def unwatch_elected_as_leader(self, group_id, callback):
return super(RedisDriver, self).unwatch_elected_as_leader(
group_id, callback)
def _get_leader_lock(self, group_id): def _get_leader_lock(self, group_id):
name = self._encode_group_leader(group_id) name = self._encode_group_leader(group_id)
return self.get_lock(name) return self.get_lock(name)

View File

@@ -75,22 +75,73 @@ class ZooKeeperLock(locking.Lock):
return self._lock.is_acquired return self._lock.is_acquired
class BaseZooKeeperDriver(coordination.CoordinationDriver): class KazooDriver(coordination.CoordinationDriverCachedRunWatchers):
"""Initialize the zookeeper driver. """This driver uses the `kazoo`_ client against real `zookeeper`_ servers.
:param timeout: connection timeout to wait when first connecting to the It **is** fully functional and implements all of the coordination
zookeeper server driver API(s). It stores data into `zookeeper`_ using znodes
and `msgpack`_ encoded values.
To configure the client to your liking a subset of the options defined at
http://kazoo.readthedocs.org/en/latest/api/client.html
will be extracted from the coordinator url (or any provided options),
so that a specific coordinator can be created that will work for you.
Currently the following options will be proxied to the contained client:
================ =============================== ====================
Name Source Default
================ =============================== ====================
hosts url netloc + 'hosts' option key localhost:2181
timeout 'timeout' options key 10.0 (kazoo default)
connection_retry 'connection_retry' options key None
command_retry 'command_retry' options key None
randomize_hosts 'randomize_hosts' options key True
================ =============================== ====================
.. _kazoo: http://kazoo.readthedocs.org/
.. _zookeeper: http://zookeeper.apache.org/
.. _msgpack: http://msgpack.org/
""" """
#: Default namespace when none is provided. #: Default namespace when none is provided.
TOOZ_NAMESPACE = b"tooz" TOOZ_NAMESPACE = b"tooz"
HANDLERS = {
'threading': threading_handler.SequentialThreadingHandler,
}
if eventlet_handler:
HANDLERS['eventlet'] = eventlet_handler.SequentialEventletHandler
"""
Restricted immutable dict of handler 'kinds' -> handler classes that
this driver can accept via 'handler' option key (the expected value for
this option is one of the keys in this dictionary).
"""
CHARACTERISTICS = (
coordination.Characteristics.NON_TIMEOUT_BASED,
coordination.Characteristics.DISTRIBUTED_ACROSS_THREADS,
coordination.Characteristics.DISTRIBUTED_ACROSS_PROCESSES,
coordination.Characteristics.DISTRIBUTED_ACROSS_HOSTS,
# Writes *always* go through a single leader process, but it may
# take a while for those writes to propagate to followers (and =
# during this time clients can read older values)...
coordination.Characteristics.SEQUENTIAL,
)
"""
Tuple of :py:class:`~tooz.coordination.Characteristics` introspectable
enum member(s) that can be used to interogate how this driver works.
"""
def __init__(self, member_id, parsed_url, options): def __init__(self, member_id, parsed_url, options):
super(BaseZooKeeperDriver, self).__init__() super(KazooDriver, self).__init__()
options = utils.collapse(options, exclude=['hosts']) options = utils.collapse(options, exclude=['hosts'])
self._options = options
self._member_id = member_id self._member_id = member_id
self.timeout = int(options.get('timeout', '10')) self.timeout = int(options.get('timeout', '10'))
self._namespace = options.get('namespace', self.TOOZ_NAMESPACE) self._namespace = options.get('namespace', self.TOOZ_NAMESPACE)
self._coord = self._make_client(parsed_url, options)
self._timeout_exception = self._coord.handler.timeout_exception
def _start(self): def _start(self):
try: try:
@@ -107,7 +158,6 @@ class BaseZooKeeperDriver(coordination.CoordinationDriver):
coordination.raise_with_cause(coordination.ToozError, coordination.raise_with_cause(coordination.ToozError,
"Operational error: %s" % e_msg, "Operational error: %s" % e_msg,
cause=e) cause=e)
self._group_members = collections.defaultdict(set)
self._watchers = collections.deque() self._watchers = collections.deque()
self._leader_locks = {} self._leader_locks = {}
@@ -397,70 +447,6 @@ class BaseZooKeeperDriver(coordination.CoordinationDriver):
cleaned_args.append(arg) cleaned_args.append(arg)
return paths.join(*cleaned_args) return paths.join(*cleaned_args)
class KazooDriver(BaseZooKeeperDriver):
"""This driver uses the `kazoo`_ client against real `zookeeper`_ servers.
It **is** fully functional and implements all of the coordination
driver API(s). It stores data into `zookeeper`_ using znodes
and `msgpack`_ encoded values.
To configure the client to your liking a subset of the options defined at
http://kazoo.readthedocs.org/en/latest/api/client.html
will be extracted from the coordinator url (or any provided options),
so that a specific coordinator can be created that will work for you.
Currently the following options will be proxied to the contained client:
================ =============================== ====================
Name Source Default
================ =============================== ====================
hosts url netloc + 'hosts' option key localhost:2181
timeout 'timeout' options key 10.0 (kazoo default)
connection_retry 'connection_retry' options key None
command_retry 'command_retry' options key None
randomize_hosts 'randomize_hosts' options key True
================ =============================== ====================
.. _kazoo: http://kazoo.readthedocs.org/
.. _zookeeper: http://zookeeper.apache.org/
.. _msgpack: http://msgpack.org/
"""
HANDLERS = {
'threading': threading_handler.SequentialThreadingHandler,
}
if eventlet_handler:
HANDLERS['eventlet'] = eventlet_handler.SequentialEventletHandler
"""
Restricted immutable dict of handler 'kinds' -> handler classes that
this driver can accept via 'handler' option key (the expected value for
this option is one of the keys in this dictionary).
"""
CHARACTERISTICS = (
coordination.Characteristics.NON_TIMEOUT_BASED,
coordination.Characteristics.DISTRIBUTED_ACROSS_THREADS,
coordination.Characteristics.DISTRIBUTED_ACROSS_PROCESSES,
coordination.Characteristics.DISTRIBUTED_ACROSS_HOSTS,
# Writes *always* go through a single leader process, but it may
# take a while for those writes to propagate to followers (and =
# during this time clients can read older values)...
coordination.Characteristics.SEQUENTIAL,
)
"""
Tuple of :py:class:`~tooz.coordination.Characteristics` introspectable
enum member(s) that can be used to interogate how this driver works.
"""
def __init__(self, member_id, parsed_url, options):
super(KazooDriver, self).__init__(member_id, parsed_url, options)
self._coord = self._make_client(parsed_url, self._options)
self._member_id = member_id
self._timeout_exception = self._coord.handler.timeout_exception
def _make_client(self, parsed_url, options): def _make_client(self, parsed_url, options):
# Creates a kazoo client, # Creates a kazoo client,
# See: https://github.com/python-zk/kazoo/blob/2.2.1/kazoo/client.py # See: https://github.com/python-zk/kazoo/blob/2.2.1/kazoo/client.py
@@ -537,8 +523,7 @@ class KazooDriver(BaseZooKeeperDriver):
# Add the hook before starting watching to avoid race conditions # Add the hook before starting watching to avoid race conditions
# as the watching executor can be in a thread # as the watching executor can be in a thread
super(BaseZooKeeperDriver, self).watch_join_group( super(KazooDriver, self).watch_join_group(group_id, callback)
group_id, callback)
if not already_being_watched: if not already_being_watched:
try: try:
@@ -548,10 +533,6 @@ class KazooDriver(BaseZooKeeperDriver):
self.unwatch_join_group(group_id, callback) self.unwatch_join_group(group_id, callback)
raise raise
def unwatch_join_group(self, group_id, callback):
return super(BaseZooKeeperDriver, self).unwatch_join_group(
group_id, callback)
def watch_leave_group(self, group_id, callback): def watch_leave_group(self, group_id, callback):
# Check if we already have hooks for this group_id, if not, start # Check if we already have hooks for this group_id, if not, start
# watching it. # watching it.
@@ -559,8 +540,7 @@ class KazooDriver(BaseZooKeeperDriver):
# Add the hook before starting watching to avoid race conditions # Add the hook before starting watching to avoid race conditions
# as the watching executor can be in a thread # as the watching executor can be in a thread
super(BaseZooKeeperDriver, self).watch_leave_group( super(KazooDriver, self).watch_leave_group(group_id, callback)
group_id, callback)
if not already_being_watched: if not already_being_watched:
try: try:
@@ -570,18 +550,6 @@ class KazooDriver(BaseZooKeeperDriver):
self.unwatch_leave_group(group_id, callback) self.unwatch_leave_group(group_id, callback)
raise raise
def unwatch_leave_group(self, group_id, callback):
return super(BaseZooKeeperDriver, self).unwatch_leave_group(
group_id, callback)
def watch_elected_as_leader(self, group_id, callback):
return super(BaseZooKeeperDriver, self).watch_elected_as_leader(
group_id, callback)
def unwatch_elected_as_leader(self, group_id, callback):
return super(BaseZooKeeperDriver, self).unwatch_elected_as_leader(
group_id, callback)
def stand_down_group_leader(self, group_id): def stand_down_group_leader(self, group_id):
if group_id in self._leader_locks: if group_id in self._leader_locks:
self._leader_locks[group_id].release() self._leader_locks[group_id].release()

View File

@@ -552,7 +552,7 @@ class TestAPI(tests.TestCaseSkipNotImplemented):
# client now no longer believes its part of the group. # client now no longer believes its part of the group.
if (hasattr(self._coord, '_joined_groups') if (hasattr(self._coord, '_joined_groups')
and (self._coord.run_watchers and (self._coord.run_watchers
== tooz.coordination._RunWatchersMixin.run_watchers)): == tooz.coordination.CoordinationDriverCachedRunWatchers.run_watchers)): # noqa
self.assertIn(self.group_id, self._coord._joined_groups) self.assertIn(self.group_id, self._coord._joined_groups)
self._coord.run_watchers() self._coord.run_watchers()
self.assertNotIn(self.group_id, self._coord._joined_groups) self.assertNotIn(self.group_id, self._coord._joined_groups)

View File

@@ -70,8 +70,9 @@ class TestMemcacheDriverFailures(testcase.TestCase):
mock_client.set.side_effect = socket.timeout('timed-out') mock_client.set.side_effect = socket.timeout('timed-out')
self.assertRaises(coordination.ToozConnectionError, coord.heartbeat) self.assertRaises(coordination.ToozConnectionError, coord.heartbeat)
@mock.patch('tooz.coordination._RunWatchersMixin.run_watchers', @mock.patch(
autospec=True) 'tooz.coordination.CoordinationDriverCachedRunWatchers.run_watchers',
autospec=True)
@mock.patch('pymemcache.client.PooledClient') @mock.patch('pymemcache.client.PooledClient')
def test_client_run_watchers_mixin(self, mock_client_cls, def test_client_run_watchers_mixin(self, mock_client_cls,
mock_run_watchers): mock_run_watchers):