package org.apache.cassandra.dht;
import java.math.BigInteger;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
public abstract class Splitter
{
private final IPartitioner partitioner;
protected Splitter(IPartitioner partitioner)
{
this.partitioner = partitioner;
}
protected abstract Token tokenForValue(BigInteger value);
protected abstract BigInteger valueForToken(Token token);
public List<Token> splitOwnedRanges(int parts, List<Range<Token>> localRanges, boolean dontSplitRanges)
{
if (localRanges.isEmpty() || parts == 1)
return Collections.singletonList(partitioner.getMaximumToken());
BigInteger totalTokens = BigInteger.ZERO;
for (Range<Token> r : localRanges)
{
BigInteger right = valueForToken(token(r.right));
totalTokens = totalTokens.add(right.subtract(valueForToken(r.left)));
}
BigInteger perPart = totalTokens.divide(BigInteger.valueOf(parts));
if (perPart.equals(BigInteger.ZERO))
return Collections.singletonList(partitioner.getMaximumToken());
if (dontSplitRanges)
return splitOwnedRangesNoPartialRanges(localRanges, perPart, parts);
List<Token> boundaries = new ArrayList<>();
BigInteger sum = BigInteger.ZERO;
for (Range<Token> r : localRanges)
{
Token right = token(r.right);
BigInteger currentRangeWidth = valueForToken(right).subtract(valueForToken(r.left)).abs();
BigInteger left = valueForToken(r.left);
while (sum.add(currentRangeWidth).compareTo(perPart) >= 0)
{
BigInteger withinRangeBoundary = perPart.subtract(sum);
left = left.add(withinRangeBoundary);
boundaries.add(tokenForValue(left));
currentRangeWidth = currentRangeWidth.subtract(withinRangeBoundary);
sum = BigInteger.ZERO;
}
sum = sum.add(currentRangeWidth);
}
boundaries.set(boundaries.size() - 1, partitioner.getMaximumToken());
assert boundaries.size() == parts : boundaries.size() + "!=" + parts + " " + boundaries + ":" + localRanges;
return boundaries;
}
private List<Token> splitOwnedRangesNoPartialRanges(List<Range<Token>> localRanges, BigInteger perPart, int parts)
{
List<Token> boundaries = new ArrayList<>(parts);
BigInteger sum = BigInteger.ZERO;
int i = 0;
final int rangesCount = localRanges.size();
while (boundaries.size() < parts - 1 && i < rangesCount - 1)
{
Range<Token> r = localRanges.get(i);
Range<Token> nextRange = localRanges.get(i + 1);
Token right = token(r.right);
Token nextRight = token(nextRange.right);
BigInteger currentRangeWidth = valueForToken(right).subtract(valueForToken(r.left));
BigInteger nextRangeWidth = valueForToken(nextRight).subtract(valueForToken(nextRange.left));
sum = sum.add(currentRangeWidth);
if (sum.compareTo(perPart) > 0 || sum.add(nextRangeWidth).compareTo(perPart) > 0)
{
BigInteger diffCurrent = sum.subtract(perPart).abs();
BigInteger diffNext = sum.add(nextRangeWidth).subtract(perPart).abs();
if (diffNext.compareTo(diffCurrent) >= 0)
{
sum = BigInteger.ZERO;
boundaries.add(right);
}
}
i++;
}
boundaries.add(partitioner.getMaximumToken());
return boundaries;
}
private Token token(Token t)
{
return t.equals(partitioner.getMinimumToken()) ? partitioner.getMaximumToken() : t;
}
}