Skip to content
Open
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

import java.util.*;
import java.util.Map.Entry;
import java.util.concurrent.atomic.AtomicReference;

import org.apache.cassandra.locator.ReplicaCollection.Builder.Conflict;
import org.slf4j.Logger;
Expand All @@ -33,6 +34,7 @@
import org.apache.cassandra.service.ClientWarn;
import org.apache.cassandra.service.StorageService;
import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.NoSpamLogger;
import org.apache.cassandra.utils.Pair;

import com.google.common.collect.ImmutableMultimap;
Expand Down Expand Up @@ -61,6 +63,10 @@ public class NetworkTopologyStrategy extends AbstractReplicationStrategy
private final Map<String, ReplicationFactor> datacenters;
private final ReplicationFactor aggregateRf;
private static final Logger logger = LoggerFactory.getLogger(NetworkTopologyStrategy.class);
// a tracker for the last datacenter string that was seen--if it changes, we log the new datacenters. This
// way we don't lose the important information(changes), but we also don't fill the log with a billion
// datacenter messages
private static final AtomicReference<String> PAST_DC_STRING = new AtomicReference<>(null);

public NetworkTopologyStrategy(String keyspaceName, TokenMetadata tokenMetadata, IEndpointSnitch snitch, Map<String, String> configOptions) throws ConfigurationException
{
Expand All @@ -86,9 +92,22 @@ public NetworkTopologyStrategy(String keyspaceName, TokenMetadata tokenMetadata,

datacenters = Collections.unmodifiableMap(newDatacenters);
aggregateRf = ReplicationFactor.withTransient(replicas, trans);
logger.info("Configured datacenter replicas are {}", FBUtilities.toString(datacenters));
verifyDatacenterChanges(datacenters);
}

private void verifyDatacenterChanges(Map<String, ReplicationFactor> datacenters)
{
// this is guaranteed to capture any meaningful datacenter changes, and has negligiable performance impact.
// however, the lack of a CAS loop means that we may also get some extra log messages in the event that a
// topology changes is rapidly (and contentiously) discovered on multiple threads simultaneously.
String dcStr = FBUtilities.toString(datacenters);
String lastDcStr = PAST_DC_STRING.get();
if(!dcStr.equals(lastDcStr))
{
PAST_DC_STRING.compareAndSet(lastDcStr,dcStr);
logger.info("Configured datacenter replicas are {}", dcStr);
}
}
/**
* Endpoint adder applying the replication rules for a given DC.
*/
Expand Down