Add per-Host and ControlConnection reconnection handlers

This commit is contained in:
Tyler Hobbs
2013-03-27 15:30:32 -05:00
parent 84de3e6a76
commit 4c5601f6a2
3 changed files with 178 additions and 12 deletions

View File

@@ -7,9 +7,12 @@ from futures import ThreadPoolExecutor
from connection import Connection
from decoder import ConsistencyLevel, QueryMessage
from metadata import Metadata
from policies import RoundRobinPolicy, SimpleConvictionPolicy, ExponentialReconnectionPolicy, HostDistance
from policies import (RoundRobinPolicy, SimpleConvictionPolicy,
ExponentialReconnectionPolicy, HostDistance)
from query import SimpleStatement
from pool import ConnectionException, BusyConnectionException
from pool import (ConnectionException, BusyConnectionException,
AuthenticationException, _ReconnectionHandler,
_HostReconnectionHandler)
# TODO: we might want to make this configurable
MAX_SCHEMA_AGREEMENT_WAIT_MS = 10000
@@ -111,7 +114,7 @@ class Cluster(object):
auth_provider = None
load_balancing_policy = None
reconnecting_policy = None
reconnection_policy = ExponentialReconnectionPolicy(2 * 1000, 5 * 60 * 1000)
retry_policy = None
compression = None
@@ -213,10 +216,32 @@ class Cluster(object):
return session
def on_up(self, host):
pass
reconnector = host.get_and_set_reconnection_handler(None)
if reconnector:
reconnector.cancel()
# TODO prepareAllQueries(host)
self._control_connection.on_up(host)
for session in self.sessions:
session.on_up(host)
def on_down(self, host):
pass
self._control_connection.on_down(host)
for session in self.sessions:
session.on_down(host)
schedule = self.reconnection_policy.new_schedule()
reconnector = _HostReconnectionHandler(
host, self._connection_factory, self.scheduler, schedule,
callback=host.get_and_set_reconnection_handler,
callback_kwargs=dict(new_handler=None))
old_reconnector = host.get_and_set_reconnection_handler(reconnector)
if old_reconnector:
old_reconnector.cancel()
reconnector.start()
def on_add(self, host):
self.prepare_all_queries(host)
@@ -239,10 +264,36 @@ class Cluster(object):
if host and self.metdata.remove_host(host):
self.on_remove(host)
def ensure_core_connections(self):
for session in self.session:
for pool in session._pools.values():
pool.ensure_core_connections()
class NoHostAvailable(Exception):
pass
class _ControlReconnectionHandler(_ReconnectionHandler):
def __init__(self, control_connection, *args, **kwargs):
_ReconnectionHandler.__init__(self, *args, **kwargs)
self.control_connection = control_connection
def try_reconnect(self):
return self.control_connection._reconnect_internal()
def on_reconnection(self, connection):
self.control_connection._set_new_connection(connection)
def on_exception(self, exc, next_delay):
# TODO only overridden to add logging, so add logging
if isinstance(exc, AuthenticationException):
return False
else:
return True
class ControlConnection(object):
_SELECT_KEYSPACES = "SELECT * FROM system.schema_keyspaces"
@@ -261,8 +312,12 @@ class ControlConnection(object):
self._balancing_policy.populate(cluster, metadata.all_hosts())
self._reconnection_policy = ExponentialReconnectionPolicy(2 * 1000, 5 * 60 * 1000)
self._connection = None
self._lock = RLock()
self._reconnection_handler = None
self._reconnection_lock = RLock()
self._is_shutdown = False
def connect(self):
@@ -304,6 +359,31 @@ class ControlConnection(object):
self.refresh_schema()
return connection
def reconnect(self):
if self._is_shutdown:
return
try:
self._set_new_connection(self._reconnect_internal())
except NoHostAvailable:
schedule = self._reconnection_policy.new_schedule()
with self._reconnection_lock:
if self._reconnection_handler:
self._reconnection_handler.cancel()
self._reconnection_handler = _ControlReconnectionHandler(
self, self._cluster.scheduler, schedule,
callback=self._get_and_set_reconnection_handler,
callback_kwargs=dict(new_handler=None))
self._reconnection_handler.start()
def _get_and_set_reconnection_handler(self, new_handler):
with self._reconnection_lock:
if self._reconnection_handler:
return self._reconnection_handler
else:
self._reconnection_handler = new_handler
return None
def shutdown(self):
self._is_shutdown = True
with self._lock:

View File

@@ -1,4 +1,4 @@
from itertools import islice, cycle, groupby
from itertools import islice, cycle, groupby, repeat
from random import randint
from threading import RLock
@@ -131,8 +131,8 @@ class ConstantReconnectionPolicy(object):
self.delay = delay
def get_next_delay(self):
return self.delay
def new_schedule(self):
return repeat(self.delay)
class ExponentialReconnectionPolicy(object):
@@ -144,10 +144,11 @@ class ExponentialReconnectionPolicy(object):
if max_delay < base_delay:
raise ValueError("Max delay must be greater than base delay")
self._delay_generator = (min(base_delay * (i ** 2), max_delay) for i in range(64))
self.base_delay = base_delay
self.max_delay = max_delay
def get_next_delay(self):
return self._delay_generator.next()
def new_schedule(self):
return (min(self.base_delay * (i ** 2), self.max_delay) for i in xrange(64))
class WriteType(object):

View File

@@ -4,7 +4,7 @@
# - proper threadpool submissions
import time
from threading import RLock, Condition
from threading import Lock, RLock, Condition
from connection import MAX_STREAM_PER_CONNECTION
@@ -38,10 +38,21 @@ class Host(object):
self.address = inet_address
self.monitor = HealthMonitor(conviction_policy(self))
self._reconnection_handler = None
self._reconnection_lock = Lock()
def set_location_info(self, datacenter, rack):
self._datacenter = datacenter
self._rack = rack
def get_and_set_reconnection_handler(self, new_handler):
with self._reconnection_lock:
if self._reconnection_handler:
return self._reconnection_handler
else:
self._reconnection_handler = new_handler
return None
def __eq__(self, other):
if not isinstance(other, Host):
return False
@@ -52,6 +63,80 @@ class Host(object):
return self.address
class _ReconnectionHandler(object):
def __init__(self, scheduler, schedule, callback, *callback_args, **callback_kwargs):
self.scheduler = scheduler
self.schedule = schedule
self.callback = callback
self.callback_args = callback_args
self.callback_kwargs = callback_kwargs
self._cancelled
def start(self):
if self._cancelled:
return
first_delay = self.schedule.get_next_delay()
self.scheduler.schedule(first_delay, self.run)
# TODO cancel previous
def run(self):
if self._cancelled:
return
# TODO wait for readyForNext?
self.on_reconnection(self.try_reconnect())
self.callback(*self.callback_args, **self.callback_kwargs)
def cancel(self):
self._cancelled = True
def reschedule(self, delay):
if self._cancelled:
return
try:
self.scheduler.schedule(delay, self.run)
except Exception, exc:
next_delay = self.schedule.get_next_delay()
if self.on_exception(exc, next_delay):
self.reschedule(next_delay)
def try_reconnect(self):
raise NotImplemented()
def on_reconnection(self, connection):
raise NotImplemented()
def on_exception(self, exc, next_delay):
if isinstance(exc, AuthenticationException):
return False
else:
return True
class _HostReconnectionHandler(_ReconnectionHandler):
def __init__(self, host, connection_factory, *args, **kwargs):
_ReconnectionHandler.__init__(self, *args, **kwargs)
self.host = host
self.connection_factory = connection_factory
def try_reconnect(self):
return self.connection_factory.open(self.host)
def on_reconnection(self, connection):
self.host.monitor.reset()
def on_exception(self, exc, next_delay):
# TODO only overridden to add logging, so add logging
if isinstance(exc, AuthenticationException):
return False
else:
return True
class HealthMonitor(object):
is_up = True