jonmeredith commented on a change in pull request #1195:
URL: https://github.com/apache/cassandra/pull/1195#discussion_r779918762
##########
File path:
src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
##########
@@ -457,4 +435,54 @@ protected void validateExpectedOptions() throws
ConfigurationException
throw new ConfigurationException(String.format("Unrecognized
strategy option {%s} passed to %s for keyspace %s", key,
getClass().getSimpleName(), keyspaceName));
}
}
+
+ static class ReplicaCache<K, V>
+ {
+ private final AtomicReference<ReplicaHolder<K, V>> cachedReplicas =
new AtomicReference<>(new ReplicaHolder<>(0, ImmutableMap.of()));
+
+ V get(long ringVersion, K keyToken)
+ {
+ maybeClear(ringVersion);
+ ReplicaHolder<K, V> replicaHolder = cachedReplicas.get();
+ if (ringVersion != replicaHolder.ringVersion)
+ return null;
+
+ return replicaHolder.replicas.get(keyToken);
+ }
+
+ void put(long ringVersion, K keyToken, V endpoints)
+ {
+ maybeClear(ringVersion);
+ ReplicaHolder<K, V> current = cachedReplicas.get();
+ // if we have the same ringVersion, but already know about the
keyToken the endpoints should be the same
+ if (current.ringVersion == ringVersion &&
!current.replicas.containsKey(keyToken))
+ {
+ ReplicaHolder<K, V> next = current.withReplica(keyToken,
endpoints);
+ cachedReplicas.compareAndSet(current, next);
+ }
+ }
+
+ private void maybeClear(long ringVersion)
+ {
+ if (ringVersion > cachedReplicas.get().ringVersion)
+ cachedReplicas.set(new ReplicaHolder<>(ringVersion,
ImmutableMap.of()));
+ }
+ }
+
+ static class ReplicaHolder<K, V>
+ {
+ private final long ringVersion;
+ private final ImmutableMap<K, V> replicas;
+
+ ReplicaHolder(long ringVersion, ImmutableMap<K, V> replicas)
+ {
+ this.ringVersion = ringVersion;
+ this.replicas = replicas;
+ }
+
+ ReplicaHolder<K, V> withReplica(K key, V value)
+ {
+ return new ReplicaHolder<>(ringVersion, ImmutableMap.<K,
V>builder().putAll(replicas).put(key, value).build());
Review comment:
Might be more efficient to initialize with an expected size. Looks like
the default is 4 which we would exceed.
`ImmutableMap.<>builderWithExpectedSize(replicas.size()+1).putAll(replicas).put(key,
value).build()`
##########
File path:
src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
##########
@@ -457,4 +435,54 @@ protected void validateExpectedOptions() throws
ConfigurationException
throw new ConfigurationException(String.format("Unrecognized
strategy option {%s} passed to %s for keyspace %s", key,
getClass().getSimpleName(), keyspaceName));
}
}
+
+ static class ReplicaCache<K, V>
+ {
+ private final AtomicReference<ReplicaHolder<K, V>> cachedReplicas =
new AtomicReference<>(new ReplicaHolder<>(0, ImmutableMap.of()));
+
+ V get(long ringVersion, K keyToken)
+ {
+ maybeClear(ringVersion);
+ ReplicaHolder<K, V> replicaHolder = cachedReplicas.get();
+ if (ringVersion != replicaHolder.ringVersion)
+ return null;
+
+ return replicaHolder.replicas.get(keyToken);
+ }
+
+ void put(long ringVersion, K keyToken, V endpoints)
+ {
+ maybeClear(ringVersion);
+ ReplicaHolder<K, V> current = cachedReplicas.get();
+ // if we have the same ringVersion, but already know about the
keyToken the endpoints should be the same
+ if (current.ringVersion == ringVersion &&
!current.replicas.containsKey(keyToken))
+ {
+ ReplicaHolder<K, V> next = current.withReplica(keyToken,
endpoints);
+ cachedReplicas.compareAndSet(current, next);
+ }
Review comment:
Why use an `ImmutableMap` here with the cost of rebuilding it for all
tokens once invalidated? The semantic is allowing writes at most once for a
key, so why not stay with the lock-free `NonBlockingHashMap` and just use
`putIfAbsent`?
```
if (current.ringVersion == ringVersion)
{
current.replicas.putIfAbsent(keyToken, endpoints);
}
```
I ended up trying it out and pushed a version to
https://github.com/jonmeredith/cassandra/tree/marcuse/16673 that makes the
change and only bothers updating the hash if the ring version was current at
the time of receiving it.
##########
File path: src/java/org/apache/cassandra/locator/TokenMetadata.java
##########
@@ -129,12 +129,18 @@ public TokenMetadata(IEndpointSnitch snitch)
}
private TokenMetadata(BiMultiValMap<Token, InetAddressAndPort>
tokenToEndpointMap, BiMap<InetAddressAndPort, UUID> endpointsMap, Topology
topology, IPartitioner partitioner)
+ {
+ this(tokenToEndpointMap, endpointsMap, topology, partitioner, 0);
+ }
+
+ private TokenMetadata(BiMultiValMap<Token, InetAddressAndPort>
tokenToEndpointMap, BiMap<InetAddressAndPort, UUID> endpointsMap, Topology
topology, IPartitioner partitioner, long ringVersion)
{
this.tokenToEndpointMap = tokenToEndpointMap;
this.topology = topology;
this.partitioner = partitioner;
endpointToHostIdMap = endpointsMap;
sortedTokens = sortTokens();
+ this.ringVersion = ringVersion;
Review comment:
`cloneWithNewPartitioner` should probably copy the ringVersion too.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]