cassandra.policies - Load balancing and Failure Handling Policies

Load Balancing

class cassandra.policies.HostDistance[source]

A measure of how “distant” a node is from the client, which may influence how the load balancer distributes requests and how many connections are opened to the node.

IGNORED = -1

A node with this distance should never be queried or have connections opened to it.

LOCAL = 0

Nodes with LOCAL distance will be preferred for operations under some load balancing policies (such as DCAwareRoundRobinPolicy) and will have a greater number of connections opened against them by default.

This distance is typically used for nodes within the same datacenter as the client.

REMOTE = 1

Nodes with REMOTE distance will be treated as a last resort by some load balancing policies (such as DCAwareRoundRobinPolicy) and will have a smaller number of connections opened against them by default.

This distance is typically used for nodes outside of the datacenter that the client is running in.

class cassandra.policies.LoadBalancingPolicy[source]

Load balancing policies are used to decide how to distribute requests among all possible coordinator nodes in the cluster.

In particular, they may focus on querying “near” nodes (those in a local datacenter) or on querying nodes who happen to be replicas for the requested data.

You may also use subclasses of LoadBalancingPolicy for custom behavior.

distance(host)[source]

Returns a measure of how remote a Host is in terms of the HostDistance enums.

populate(cluster, hosts)[source]

This method is called to initialize the load balancing policy with a set of Host instances before its first use. The cluster parameter is an instance of Cluster.

make_query_plan(working_keyspace=None, query=None)[source]

Given a Statement instance, return a iterable of Host instances which should be queried in that order. A generator may work well for custom implementations of this method.

Note that the query argument may be None when preparing statements.

working_keyspace should be the string name of the current keyspace, as set through Session.set_keyspace() or with a USE statement.

check_supported()[source]

This will be called after the cluster Metadata has been initialized. If the load balancing policy implementation cannot be supported for some reason (such as a missing C extension), this is the point at which it should raise an exception.

class cassandra.policies.RoundRobinPolicy[source]

A subclass of LoadBalancingPolicy which evenly distributes queries across all nodes in the cluster, regardless of what datacenter the nodes may be in.

This load balancing policy is used by default.

class cassandra.policies.DCAwareRoundRobinPolicy(local_dc='', used_hosts_per_remote_dc=0)[source]

Similar to RoundRobinPolicy, but prefers hosts in the local datacenter and only uses nodes in remote datacenters as a last resort.

The local_dc parameter should be the name of the datacenter (such as is reported by nodetool ring) that should be considered local. If not specified, the driver will choose a local_dc based on the first host among Cluster.contact_points having a valid DC. If relying on this mechanism, all specified contact points should be nodes in a single, local DC.

used_hosts_per_remote_dc controls how many nodes in each remote datacenter will have connections opened against them. In other words, used_hosts_per_remote_dc hosts will be considered REMOTE and the rest will be considered IGNORED. By default, all remote hosts are ignored.

class cassandra.policies.WhiteListRoundRobinPolicy(hosts)[source]

A subclass of RoundRobinPolicy which evenly distributes queries across all nodes in the cluster, regardless of what datacenter the nodes may be in, but only if that node exists in the list of allowed nodes

This policy is addresses the issue described in https://datastax-oss.atlassian.net/browse/JAVA-145 Where connection errors occur when connection attempts are made to private IP addresses remotely

The hosts parameter should be a sequence of hosts to permit connections to.

class cassandra.policies.TokenAwarePolicy(child_policy)[source]

A LoadBalancingPolicy wrapper that adds token awareness to a child policy.

This alters the child policy’s behavior so that it first attempts to send queries to LOCAL replicas (as determined by the child policy) based on the Statement‘s routing_key. Once those hosts are exhausted, the remaining hosts in the child policy’s query plan will be used.

If no routing_key is set on the query, the child policy’s query plan will be used as is.

Marking Hosts Up or Down

class cassandra.policies.ConvictionPolicy(host)[source]

A policy which decides when hosts should be considered down based on the types of failures and the number of failures.

If custom behavior is needed, this class may be subclassed.

host is an instance of Host.

add_failure(connection_exc)[source]

Implementations should return True if the host should be convicted, False otherwise.

reset()[source]

Implementations should clear out any convictions or state regarding the host.

class cassandra.policies.SimpleConvictionPolicy(host)[source]

The default implementation of ConvictionPolicy, which simply marks a host as down after the first failure of any kind.

host is an instance of Host.

Reconnecting to Dead Hosts

class cassandra.policies.ReconnectionPolicy[source]

This class and its subclasses govern how frequently an attempt is made to reconnect to nodes that are marked as dead.

If custom behavior is needed, this class may be subclassed.

new_schedule()[source]

This should return a finite or infinite iterable of delays (each as a floating point number of seconds) inbetween each failed reconnection attempt. Note that if the iterable is finite, reconnection attempts will cease once the iterable is exhausted.

class cassandra.policies.ConstantReconnectionPolicy(delay, max_attempts=64)[source]

A ReconnectionPolicy subclass which sleeps for a fixed delay inbetween each reconnection attempt.

delay should be a floating point number of seconds to wait inbetween each attempt.

max_attempts should be a total number of attempts to be made before giving up, or None to continue reconnection attempts forever. The default is 64.

class cassandra.policies.ExponentialReconnectionPolicy(base_delay, max_delay)[source]

A ReconnectionPolicy subclass which exponentially increases the length of the delay inbetween each reconnection attempt up to a set maximum delay.

base_delay and max_delay should be in floating point units of seconds.

Retrying Failed Operations

class cassandra.policies.WriteType[source]

For usage with RetryPolicy, this describe a type of write operation.

SIMPLE = 0

A write to a single partition key. Such writes are guaranteed to be atomic and isolated.

BATCH = 1

A write to multiple partition keys that used the distributed batch log to ensure atomicity.

UNLOGGED_BATCH = 2

A write to multiple partition keys that did not use the distributed batch log. Atomicity for such writes is not guaranteed.

COUNTER = 3

A counter write (for one or multiple partition keys). Such writes should not be replayed in order to avoid overcount.

BATCH_LOG = 4

The initial write to the distributed batch log that Cassandra performs internally before a BATCH write.

CAS = 5

A lighweight-transaction write, such as “DELETE ... IF EXISTS”.

class cassandra.policies.RetryPolicy[source]

A policy that describes whether to retry, rethrow, or ignore timeout and unavailable failures.

To specify a default retry policy, set the Cluster.default_retry_policy attribute to an instance of this class or one of its subclasses.

To specify a retry policy per query, set the Statement.retry_policy attribute to an instance of this class or one of its subclasses.

If custom behavior is needed for retrying certain operations, this class may be subclassed.

RETRY = 0

This should be returned from the below methods if the operation should be retried on the same connection.

RETHROW = 1

This should be returned from the below methods if the failure should be propagated and no more retries attempted.

IGNORE = 2

This should be returned from the below methods if the failure should be ignored but no more retries should be attempted.

on_read_timeout(query, consistency, required_responses, received_responses, data_retrieved, retry_num)[source]

This is called when a read operation times out from the coordinator’s perspective (i.e. a replica did not respond to the coordinator in time). It should return a tuple with two items: one of the class enums (such as RETRY) and a ConsistencyLevel to retry the operation at or None to keep the same consistency level.

query is the Statement that timed out.

consistency is the ConsistencyLevel that the operation was attempted at.

The required_responses and received_responses parameters describe how many replicas needed to respond to meet the requested consistency level and how many actually did respond before the coordinator timed out the request. data_retrieved is a boolean indicating whether any of those responses contained data (as opposed to just a digest).

retry_num counts how many times the operation has been retried, so the first time this method is called, retry_num will be 0.

By default, operations will be retried at most once, and only if a sufficient number of replicas responded (with data digests).

on_write_timeout(query, consistency, write_type, required_responses, received_responses, retry_num)[source]

This is called when a write operation times out from the coordinator’s perspective (i.e. a replica did not respond to the coordinator in time).

query is the Statement that timed out.

consistency is the ConsistencyLevel that the operation was attempted at.

write_type is one of the WriteType enums describing the type of write operation.

The required_responses and received_responses parameters describe how many replicas needed to acknowledge the write to meet the requested consistency level and how many replicas actually did acknowledge the write before the coordinator timed out the request.

retry_num counts how many times the operation has been retried, so the first time this method is called, retry_num will be 0.

By default, failed write operations will retried at most once, and they will only be retried if the write_type was BATCH_LOG.

on_unavailable(query, consistency, required_replicas, alive_replicas, retry_num)[source]

This is called when the coordinator node determines that a read or write operation cannot be successful because the number of live replicas are too low to meet the requested ConsistencyLevel. This means that the read or write operation was never forwared to any replicas.

query is the Statement that failed.

consistency is the ConsistencyLevel that the operation was attempted at.

required_replicas is the number of replicas that would have needed to acknowledge the operation to meet the requested consistency level. alive_replicas is the number of replicas that the coordinator considered alive at the time of the request.

retry_num counts how many times the operation has been retried, so the first time this method is called, retry_num will be 0.

By default, no retries will be attempted and the error will be re-raised.

class cassandra.policies.FallthroughRetryPolicy[source]

A retry policy that never retries and always propagates failures to the application.

class cassandra.policies.DowngradingConsistencyRetryPolicy[source]

A retry policy that sometimes retries with a lower consistency level than the one initially requested.

BEWARE: This policy may retry queries using a lower consistency level than the one initially requested. By doing so, it may break consistency guarantees. In other words, if you use this retry policy, there are cases (documented below) where a read at QUORUM may not see a preceding write at QUORUM. Do not use this policy unless you have understood the cases where this can happen and are ok with that. It is also recommended to subclass this class so that queries that required a consistency level downgrade can be recorded (so that repairs can be made later, etc).

This policy implements the same retries as RetryPolicy, but on top of that, it also retries in the following cases:

  • On a read timeout: if the number of replicas that responded is greater than one but lower than is required by the requested consistency level, the operation is retried at a lower consistency level.
  • On a write timeout: if the operation is an UNLOGGED_BATCH and at least one replica acknowledged the write, the operation is retried at a lower consistency level. Furthermore, for other write types, if at least one replica acknowledged the write, the timeout is ignored.
  • On an unavailable exception: if at least one replica is alive, the operation is retried at a lower consistency level.

The reasoning behind this retry policy is as follows: if, based on the information the Cassandra coordinator node returns, retrying the operation with the initially requested consistency has a chance to succeed, do it. Otherwise, if based on that information we know the initially requested consistency level cannot be achieved currently, then:

  • For writes, ignore the exception (thus silently failing the consistency requirement) if we know the write has been persisted on at least one replica.
  • For reads, try reading at a lower consistency level (thus silently failing the consistency requirement).

In other words, this policy implements the idea that if the requested consistency level cannot be achieved, the next best thing for writes is to make sure the data is persisted, and that reading something is better than reading nothing, even if there is a risk of reading stale data.