Skip to content

Commit 5a4b45e

Browse files
author
Sylvain Lebresne
committed
Don't hop DCs for LOCAL_* CLs by default
JAVA-289 #fixes
1 parent 1239a04 commit 5a4b45e

File tree

4 files changed

+70
-1
lines changed

4 files changed

+70
-1
lines changed

driver-core/CHANGELOG.rst

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -7,6 +7,7 @@ CHANGELOG
77
- [new] Add method to check if a Cluster/Session instance has been closed already (JAVA-114)
88
- [new] Expose the name of the partitioner in use in the cluster metadata (JAVA-179)
99
- [new] Add new WhiteListPolicy to limit the nodes connected to a particular list
10+
- [improvement] Do not hop DC for LOCAL_* CL in DCAwareRoundRobinPolicy (JAVA-289)
1011

1112

1213
1.0.6:

driver-core/src/main/java/com/datastax/driver/core/ConsistencyLevel.java

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -61,4 +61,13 @@ static org.apache.cassandra.db.ConsistencyLevel toCassandraCL(ConsistencyLevel c
6161
}
6262
throw new AssertionError();
6363
}
64+
65+
/**
66+
* Whether or not the the consistency level applies to the local data-center only.
67+
*
68+
* @return whether this consistency level is {@code LOCAL_ONE} or {@code LOCAL_QUORUM}.
69+
*/
70+
public boolean isDCLocal() {
71+
return this == LOCAL_ONE || this == LOCAL_QUORUM;
72+
}
6473
}

driver-core/src/main/java/com/datastax/driver/core/policies/DCAwareRoundRobinPolicy.java

Lines changed: 47 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -49,6 +49,7 @@ public class DCAwareRoundRobinPolicy implements LoadBalancingPolicy {
4949
private final AtomicInteger index = new AtomicInteger();
5050
private final String localDc;
5151
private final int usedHostsPerRemoteDc;
52+
private final boolean dontHopForLocalCL;
5253

5354
/**
5455
* Creates a new datacenter aware round robin policy given the name of
@@ -74,6 +75,15 @@ public DCAwareRoundRobinPolicy(String localDc) {
7475
* <p>
7576
* The name of the local datacenter provided must be the local
7677
* datacenter name as known by Cassandra.
78+
* <p>
79+
* If {@code usedHostsPerRemoteDc > 0}, then if for a query no host
80+
* in the local datacenter can be reached and if the consistency
81+
* level of the query is not {@code LOCAL_ONE} or {@code LOCAL_QUORUM},
82+
* then up to {@code usedHostsPerRemoteDc} host per remote data-center
83+
* will be tried by the policy as a fallback. Please note that no
84+
* remote host will be used for {@code LOCAL_ONE} and {@code LOCAL_QUORUM}
85+
* since this would change the meaning of the consistency level (and
86+
* thus somewhat break the consistency contract).
7787
*
7888
* @param localDc the name of the local datacenter (as known by
7989
* Cassandra).
@@ -86,8 +96,41 @@ public DCAwareRoundRobinPolicy(String localDc) {
8696
* connections to them will be maintained).
8797
*/
8898
public DCAwareRoundRobinPolicy(String localDc, int usedHostsPerRemoteDc) {
99+
this(localDc, usedHostsPerRemoteDc, false);
100+
}
101+
102+
/**
103+
* Creates a new DCAwareRoundRobin policy given the name of the local
104+
* datacenter and that uses the provided number of host per remote
105+
* datacenter as failover for the local hosts.
106+
* <p>
107+
* This constructor is equivalent to {@link DCAwareRoundRobinPolicy(String, int)}
108+
* but allows to override the policy of never using remote data-center
109+
* nodes for {@code LOCAL_ONE} and {@code LOCAL_QUORUM} queries. It is
110+
* however inadvisable to do so in almost all cases, as this would
111+
* potentially break consistency guarantees and if you are fine with that,
112+
* it's probably better to use a weaker consitency like {@code ONE}, {@code
113+
* TWO} or {@code THREE}. As such, this constructor should generally
114+
* be avoided in favor of {@link DCAwareRoundRobinPolicy(String, int)}.
115+
* Use it only if you know and understand what you do.
116+
*
117+
* @param localDc the name of the local datacenter (as known by
118+
* Cassandra).
119+
* @param usedHostsPerRemoteDc the number of host per remote
120+
* datacenter that policies created by the returned factory should
121+
* consider. Created policies {@code distance} method will return a
122+
* {@code HostDistance.REMOTE} distance for only {@code
123+
* usedHostsPerRemoteDc} hosts per remote datacenter. Other hosts
124+
* of the remote datacenters will be ignored (and thus no
125+
* connections to them will be maintained).
126+
* @param allowRemoteDCsForLocalConsistencyLevel whether or not the
127+
* policy may return remote host when building query plan for query
128+
* having consitency {@code LOCAL_ONE} and {@code LOCAL_QUORUM}.
129+
*/
130+
public DCAwareRoundRobinPolicy(String localDc, int usedHostsPerRemoteDc, boolean allowRemoteDCsForLocalConsistencyLevel) {
89131
this.localDc = localDc;
90132
this.usedHostsPerRemoteDc = usedHostsPerRemoteDc;
133+
this.dontHopForLocalCL = !allowRemoteDCsForLocalConsistencyLevel;
91134
}
92135

93136
@Override
@@ -158,7 +201,7 @@ public HostDistance distance(Host host) {
158201
* try first for querying, which one to use as failover, etc...
159202
*/
160203
@Override
161-
public Iterator<Host> newQueryPlan(Query query) {
204+
public Iterator<Host> newQueryPlan(final Query query) {
162205

163206
CopyOnWriteArrayList<Host> localLiveHosts = perDcLiveHosts.get(localDc);
164207
final List<Host> hosts = localLiveHosts == null ? Collections.<Host>emptyList() : cloneList(localLiveHosts);
@@ -192,6 +235,9 @@ protected Host computeNext() {
192235
return currentDcHosts.get(c);
193236
}
194237

238+
if (dontHopForLocalCL && query.getConsistencyLevel().isDCLocal())
239+
return endOfData();
240+
195241
if (remoteDcs == null) {
196242
Set<String> copy = new HashSet<String>(perDcLiveHosts.keySet());
197243
copy.remove(localDc);

driver-core/src/main/java/com/datastax/driver/core/policies/RoundRobinPolicy.java

Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,8 @@
2323
import java.util.concurrent.atomic.AtomicInteger;
2424

2525
import com.google.common.collect.AbstractIterator;
26+
import org.slf4j.Logger;
27+
import org.slf4j.LoggerFactory;
2628

2729
import com.datastax.driver.core.Cluster;
2830
import com.datastax.driver.core.Host;
@@ -43,9 +45,13 @@
4345
*/
4446
public class RoundRobinPolicy implements LoadBalancingPolicy {
4547

48+
private static final Logger logger = LoggerFactory.getLogger(RoundRobinPolicy.class);
49+
4650
private final CopyOnWriteArrayList<Host> liveHosts = new CopyOnWriteArrayList<Host>();
4751
private final AtomicInteger index = new AtomicInteger();
4852

53+
private volatile boolean hasLoggedLocalCLUse;
54+
4955
/**
5056
* Creates a load balancing policy that picks host to query in a round robin
5157
* fashion (on all the hosts of the Cassandra cluster).
@@ -87,6 +93,13 @@ public HostDistance distance(Host host) {
8793
@Override
8894
public Iterator<Host> newQueryPlan(Query query) {
8995

96+
if (!hasLoggedLocalCLUse && query.getConsistencyLevel().isDCLocal()) {
97+
hasLoggedLocalCLUse = true;
98+
logger.warn("Detected request at Consistency Level {} but the non-DC aware RoundRobinPolicy is in use. "
99+
+ "It is strongly advised to use DCAwareRoundRobinPolicy if you have multiple DCs/use DC-aware consistency levels "
100+
+ "(note: this message will only be logged once)", query.getConsistencyLevel());
101+
}
102+
90103
// We clone liveHosts because we want a version of the list that
91104
// cannot change concurrently of the query plan iterator (this
92105
// would be racy). We use clone() as it don't involve a copy of the

0 commit comments

Comments
 (0)