This is an automated email from the ASF dual-hosted git repository.
blerer pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git
The following commit(s) were added to refs/heads/trunk by this push:
new a209e78 Fix unguarded casts to NetworkTopologyStrategy
a209e78 is described below
commit a209e78dae8674ef131bab1b2dc06f09ae15d21d
Author: Bereng <[email protected]>
AuthorDate: Thu May 7 16:53:02 2020 +0200
Fix unguarded casts to NetworkTopologyStrategy
patch by Berenguer Blasi; reviewed by Benjamin Lerer for CASSANDRA-15791
---
.../org/apache/cassandra/db/ConsistencyLevel.java | 20 +++++++++++++++-----
.../service/DatacenterSyncWriteResponseHandler.java | 16 +++++++++++-----
2 files changed, 26 insertions(+), 10 deletions(-)
diff --git a/src/java/org/apache/cassandra/db/ConsistencyLevel.java
b/src/java/org/apache/cassandra/db/ConsistencyLevel.java
index 91e83a7..e3da5b3 100644
--- a/src/java/org/apache/cassandra/db/ConsistencyLevel.java
+++ b/src/java/org/apache/cassandra/db/ConsistencyLevel.java
@@ -100,11 +100,21 @@ public enum ConsistencyLevel
public static ObjectIntHashMap<String> eachQuorumForRead(Keyspace keyspace)
{
- NetworkTopologyStrategy strategy = (NetworkTopologyStrategy)
keyspace.getReplicationStrategy();
- ObjectIntHashMap<String> perDc = new
ObjectIntHashMap<>(((strategy.getDatacenters().size() + 1) * 4) / 3);
- for (String dc : strategy.getDatacenters())
- perDc.put(dc, ConsistencyLevel.localQuorumFor(keyspace, dc));
- return perDc;
+ AbstractReplicationStrategy strategy =
keyspace.getReplicationStrategy();
+ if (strategy instanceof NetworkTopologyStrategy)
+ {
+ NetworkTopologyStrategy npStrategy = (NetworkTopologyStrategy)
strategy;
+ ObjectIntHashMap<String> perDc = new
ObjectIntHashMap<>(((npStrategy.getDatacenters().size() + 1) * 4) / 3);
+ for (String dc : npStrategy.getDatacenters())
+ perDc.put(dc, ConsistencyLevel.localQuorumFor(keyspace, dc));
+ return perDc;
+ }
+ else
+ {
+ ObjectIntHashMap<String> perDc = new ObjectIntHashMap<>(1);
+ perDc.put(DatabaseDescriptor.getLocalDataCenter(),
quorumFor(keyspace));
+ return perDc;
+ }
}
public static ObjectIntHashMap<String> eachQuorumForWrite(Keyspace
keyspace, Endpoints<?> pendingWithDown)
diff --git
a/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java
b/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java
index 1f536c7..389dcd5 100644
---
a/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java
+++
b/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java
@@ -49,12 +49,18 @@ public class DatacenterSyncWriteResponseHandler<T> extends
AbstractWriteResponse
super(replicaPlan, callback, writeType, queryStartNanoTime);
assert replicaPlan.consistencyLevel() == ConsistencyLevel.EACH_QUORUM;
- NetworkTopologyStrategy strategy = (NetworkTopologyStrategy)
replicaPlan.keyspace().getReplicationStrategy();
-
- for (String dc : strategy.getDatacenters())
+ if (replicaPlan.keyspace().getReplicationStrategy() instanceof
NetworkTopologyStrategy)
+ {
+ NetworkTopologyStrategy strategy = (NetworkTopologyStrategy)
replicaPlan.keyspace().getReplicationStrategy();
+ for (String dc : strategy.getDatacenters())
+ {
+ int rf = strategy.getReplicationFactor(dc).allReplicas;
+ responses.put(dc, new AtomicInteger((rf / 2) + 1));
+ }
+ }
+ else
{
- int rf = strategy.getReplicationFactor(dc).allReplicas;
- responses.put(dc, new AtomicInteger((rf / 2) + 1));
+ responses.put(DatabaseDescriptor.getLocalDataCenter(), new
AtomicInteger(ConsistencyLevel.quorumFor(replicaPlan.keyspace())));
}
// During bootstrap, we have to include the pending endpoints or we
may fail the consistency level
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]