package org.apache.cassandra.dht;
import java.net.InetAddress;
import java.util.*;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.ArrayListMultimap;
import com.google.common.collect.HashMultimap;
import com.google.common.collect.Multimap;
import com.google.common.collect.Sets;
import org.apache.commons.lang3.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.cassandra.db.Keyspace;
import org.apache.cassandra.db.SystemKeyspace;
import org.apache.cassandra.gms.EndpointState;
import org.apache.cassandra.gms.Gossiper;
import org.apache.cassandra.gms.IFailureDetector;
import org.apache.cassandra.locator.AbstractReplicationStrategy;
import org.apache.cassandra.locator.IEndpointSnitch;
import org.apache.cassandra.locator.TokenMetadata;
import org.apache.cassandra.service.StorageService;
import org.apache.cassandra.streaming.StreamPlan;
import org.apache.cassandra.streaming.StreamResultFuture;
import org.apache.cassandra.utils.FBUtilities;
public class RangeStreamer
{
private static final Logger logger = LoggerFactory.getLogger(RangeStreamer.class);
private final Collection<Token> tokens;
private final TokenMetadata metadata;
private final InetAddress address;
private final String description;
private final Multimap<String, Map.Entry<InetAddress, Collection<Range<Token>>>> toFetch = HashMultimap.create();
private final Set<ISourceFilter> sourceFilters = new HashSet<>();
private final StreamPlan streamPlan;
private final boolean useStrictConsistency;
private final IEndpointSnitch snitch;
private final StreamStateStore stateStore;
public static interface ISourceFilter
{
public boolean shouldInclude(InetAddress endpoint);
}
public static class FailureDetectorSourceFilter implements ISourceFilter
{
private final IFailureDetector fd;
public FailureDetectorSourceFilter(IFailureDetector fd)
{
this.fd = fd;
}
public boolean shouldInclude(InetAddress endpoint)
{
return fd.isAlive(endpoint);
}
}
public static class SingleDatacenterFilter implements ISourceFilter
{
private final String sourceDc;
private final IEndpointSnitch snitch;
public SingleDatacenterFilter(IEndpointSnitch snitch, String sourceDc)
{
this.sourceDc = sourceDc;
this.snitch = snitch;
}
public boolean shouldInclude(InetAddress endpoint)
{
return snitch.getDatacenter(endpoint).equals(sourceDc);
}
}
public static class ExcludeLocalNodeFilter implements ISourceFilter
{
public boolean shouldInclude(InetAddress endpoint)
{
return !FBUtilities.getBroadcastAddress().equals(endpoint);
}
}
public static class WhitelistedSourcesFilter implements ISourceFilter
{
private final Set<InetAddress> whitelistedSources;
public WhitelistedSourcesFilter(Set<InetAddress> whitelistedSources)
{
this.whitelistedSources = whitelistedSources;
}
public boolean shouldInclude(InetAddress endpoint)
{
return whitelistedSources.contains(endpoint);
}
}
public RangeStreamer(TokenMetadata metadata,
Collection<Token> tokens,
InetAddress address,
String description,
boolean useStrictConsistency,
IEndpointSnitch snitch,
StreamStateStore stateStore,
boolean connectSequentially)
{
this.metadata = metadata;
this.tokens = tokens;
this.address = address;
this.description = description;
this.streamPlan = new StreamPlan(description, true, connectSequentially);
this.useStrictConsistency = useStrictConsistency;
this.snitch = snitch;
this.stateStore = stateStore;
streamPlan.listeners(this.stateStore);
}
public void addSourceFilter(ISourceFilter filter)
{
sourceFilters.add(filter);
}
public void addRanges(String keyspaceName, Collection<Range<Token>> ranges)
{
Multimap<Range<Token>, InetAddress> rangesForKeyspace = useStrictSourcesForRanges(keyspaceName)
? getAllRangesWithStrictSourcesFor(keyspaceName, ranges) : getAllRangesWithSourcesFor(keyspaceName, ranges);
if (logger.isTraceEnabled())
{
for (Map.Entry<Range<Token>, InetAddress> entry : rangesForKeyspace.entries())
logger.trace("{}: range {} exists on {}", description, entry.getKey(), entry.getValue());
}
for (Map.Entry<InetAddress, Collection<Range<Token>>> entry : getRangeFetchMap(rangesForKeyspace, sourceFilters, keyspaceName, useStrictConsistency).asMap().entrySet())
{
if (logger.isTraceEnabled())
{
for (Range<Token> r : entry.getValue())
logger.trace("{}: range {} from source {} for keyspace {}", description, r, entry.getKey(), keyspaceName);
}
toFetch.put(keyspaceName, entry);
}
}
private boolean useStrictSourcesForRanges(String keyspaceName)
{
AbstractReplicationStrategy strat = Keyspace.open(keyspaceName).getReplicationStrategy();
return useStrictConsistency
&& tokens != null
&& metadata.getAllEndpoints().size() != strat.getReplicationFactor();
}
private Multimap<Range<Token>, InetAddress> getAllRangesWithSourcesFor(String keyspaceName, Collection<Range<Token>> desiredRanges)
{
AbstractReplicationStrategy strat = Keyspace.open(keyspaceName).getReplicationStrategy();
Multimap<Range<Token>, InetAddress> rangeAddresses = strat.getRangeAddresses(metadata.cloneOnlyTokenMap());
Multimap<Range<Token>, InetAddress> rangeSources = ArrayListMultimap.create();
for (Range<Token> desiredRange : desiredRanges)
{
for (Range<Token> range : rangeAddresses.keySet())
{
if (range.contains(desiredRange))
{
List<InetAddress> preferred = snitch.getSortedListByProximity(address, rangeAddresses.get(range));
rangeSources.putAll(desiredRange, preferred);
break;
}
}
if (!rangeSources.keySet().contains(desiredRange))
throw new IllegalStateException("No sources found for " + desiredRange);
}
return rangeSources;
}
private Multimap<Range<Token>, InetAddress> getAllRangesWithStrictSourcesFor(String keyspace, Collection<Range<Token>> desiredRanges)
{
assert tokens != null;
AbstractReplicationStrategy strat = Keyspace.open(keyspace).getReplicationStrategy();
TokenMetadata metadataClone = metadata.cloneOnlyTokenMap();
Multimap<Range<Token>, InetAddress> addressRanges = strat.getRangeAddresses(metadataClone);
metadataClone.updateNormalTokens(tokens, address);
Multimap<Range<Token>, InetAddress> pendingRangeAddresses = strat.getRangeAddresses(metadataClone);
Multimap<Range<Token>, InetAddress> rangeSources = ArrayListMultimap.create();
for (Range<Token> desiredRange : desiredRanges)
{
for (Map.Entry<Range<Token>, Collection<InetAddress>> preEntry : addressRanges.asMap().entrySet())
{
if (preEntry.getKey().contains(desiredRange))
{
Set<InetAddress> oldEndpoints = Sets.newHashSet(preEntry.getValue());
Set<InetAddress> newEndpoints = Sets.newHashSet(pendingRangeAddresses.get(desiredRange));
if (oldEndpoints.size() == strat.getReplicationFactor())
{
oldEndpoints.removeAll(newEndpoints);
assert oldEndpoints.size() == 1 : "Expected 1 endpoint but found " + oldEndpoints.size();
}
rangeSources.put(desiredRange, oldEndpoints.iterator().next());
}
}
Collection<InetAddress> addressList = rangeSources.get(desiredRange);
if (addressList == null || addressList.isEmpty())
throw new IllegalStateException("No sources found for " + desiredRange);
if (addressList.size() > 1)
throw new IllegalStateException("Multiple endpoints found for " + desiredRange);
InetAddress sourceIp = addressList.iterator().next();
EndpointState sourceState = Gossiper.instance.getEndpointStateForEndpoint(sourceIp);
if (Gossiper.instance.isEnabled() && (sourceState == null || !sourceState.isAlive()))
throw new RuntimeException("A node required to move the data consistently is down (" + sourceIp + "). " +
"If you wish to move the data from a potentially inconsistent replica, restart the node with -Dcassandra.consistent.rangemovement=false");
}
return rangeSources;
}
private static Multimap<InetAddress, Range<Token>> getRangeFetchMap(Multimap<Range<Token>, InetAddress> rangesWithSources,
Collection<ISourceFilter> sourceFilters, String keyspace,
boolean useStrictConsistency)
{
Multimap<InetAddress, Range<Token>> rangeFetchMapMap = HashMultimap.create();
for (Range<Token> range : rangesWithSources.keySet())
{
boolean foundSource = false;
outer:
for (InetAddress address : rangesWithSources.get(range))
{
for (ISourceFilter filter : sourceFilters)
{
if (!filter.shouldInclude(address))
continue outer;
}
if (address.equals(FBUtilities.getBroadcastAddress()))
{
foundSource = true;
continue;
}
rangeFetchMapMap.put(address, range);
foundSource = true;
break;
}
if (!foundSource)
{
AbstractReplicationStrategy strat = Keyspace.open(keyspace).getReplicationStrategy();
if (strat != null && strat.getReplicationFactor() == 1)
{
if (useStrictConsistency)
throw new IllegalStateException("Unable to find sufficient sources for streaming range " + range + " in keyspace " + keyspace + " with RF=1. " +
"Ensure this keyspace contains replicas in the source datacenter.");
else
logger.warn("Unable to find sufficient sources for streaming range {} in keyspace {} with RF=1. " +
"Keyspace might be missing data.", range, keyspace);
}
else
throw new IllegalStateException("Unable to find sufficient sources for streaming range " + range + " in keyspace " + keyspace);
}
}
return rangeFetchMapMap;
}
public static Multimap<InetAddress, Range<Token>> getWorkMap(Multimap<Range<Token>, InetAddress> rangesWithSourceTarget, String keyspace,
IFailureDetector fd, boolean useStrictConsistency)
{
return getRangeFetchMap(rangesWithSourceTarget, Collections.<ISourceFilter>singleton(new FailureDetectorSourceFilter(fd)), keyspace, useStrictConsistency);
}
@VisibleForTesting
Multimap<String, Map.Entry<InetAddress, Collection<Range<Token>>>> toFetch()
{
return toFetch;
}
public StreamResultFuture fetchAsync()
{
for (Map.Entry<String, Map.Entry<InetAddress, Collection<Range<Token>>>> entry : toFetch.entries())
{
String keyspace = entry.getKey();
InetAddress source = entry.getValue().getKey();
InetAddress preferred = SystemKeyspace.getPreferredIP(source);
Collection<Range<Token>> ranges = entry.getValue().getValue();
Set<Range<Token>> availableRanges = stateStore.getAvailableRanges(keyspace, StorageService.instance.getTokenMetadata().partitioner);
if (ranges.removeAll(availableRanges))
{
logger.info("Some ranges of {} are already available. Skipping streaming those ranges.", availableRanges);
}
if (logger.isTraceEnabled())
logger.trace("{}ing from {} ranges {}", description, source, StringUtils.join(ranges, ", "));
streamPlan.requestRanges(source, preferred, keyspace, ranges);
}
return streamPlan.execute();
}
}