diff --git a/cassandra/connection.py b/cassandra/connection.py index 08501d0a2b..f7eac53c46 100644 --- a/cassandra/connection.py +++ b/cassandra/connection.py @@ -1821,15 +1821,15 @@ def __init__(self, connection, owner): self._exception = Exception("Failed to send heartbeat because connection 'in_flight' exceeds threshold") self._event.set() - def wait(self, timeout): + def wait(self, timeout, original_timeout): self._event.wait(timeout) if self._event.is_set(): if self._exception: raise self._exception else: - raise OperationTimedOut("Connection heartbeat timeout after %s seconds" % (timeout,), + raise OperationTimedOut("Connection heartbeat timeout after %s seconds" % (original_timeout,), self.connection.endpoint, - timeout=timeout, + timeout=original_timeout, in_flight=self.connection.in_flight) def _options_callback(self, response): @@ -1888,13 +1888,13 @@ def run(self): self._raise_if_stopped() # Wait max `self._timeout` seconds for all HeartbeatFutures to complete - timeout = self._timeout + timeout_left = self._timeout start_time = time.time() for f in futures: self._raise_if_stopped() connection = f.connection try: - f.wait(timeout) + f.wait(timeout_left, self._timeout) # TODO: move this, along with connection locks in pool, down into Connection with connection.lock: connection.in_flight -= 1 @@ -1904,7 +1904,7 @@ def run(self): id(connection), connection.endpoint) failed_connections.append((f.connection, f.owner, e)) - timeout = self._timeout - (time.time() - start_time) + timeout_left = self._timeout - (time.time() - start_time) for connection, owner, exc in failed_connections: self._raise_if_stopped() diff --git a/cassandra/policies.py b/cassandra/policies.py index ceb5ebdc45..2cb2625baa 100644 --- a/cassandra/policies.py +++ b/cassandra/policies.py @@ -773,7 +773,7 @@ class ConstantReconnectionPolicy(ReconnectionPolicy): in-between each reconnection attempt. """ - def __init__(self, delay, max_attempts=64): + def __init__(self, delay, max_attempts=None): """ `delay` should be a floating point number of seconds to wait in-between each attempt. @@ -807,10 +807,7 @@ class ExponentialReconnectionPolicy(ReconnectionPolicy): trying to reconnect at exactly the same time. """ - # TODO: max_attempts is 64 to preserve legacy default behavior - # consider changing to None in major release to prevent the policy - # giving up forever - def __init__(self, base_delay, max_delay, max_attempts=64): + def __init__(self, base_delay, max_delay, max_attempts=None): """ `base_delay` and `max_delay` should be in floating point units of seconds.