// rate limit is in bytes per second. Uses Double.MAX_VALUE if disabled (set to 0 in cassandra.yaml).
// max rate is scaled by the number of nodes in the cluster (CASSANDRA-5272).
int throttleInKB = DatabaseDescriptor.getHintedHandoffThrottleInKB()
/ (StorageService.instance.getTokenMetadata().getAllEndpoints().size() - 1);
RateLimiter rateLimiter = RateLimiter.create(throttleInKB == 0 ? Double.MAX_VALUE : throttleInKB * 1024);
boolean finished = false;
delivery:
while (true)
{
long now = System.currentTimeMillis();
QueryFilter filter = QueryFilter.getSliceFilter(epkey,
SystemKeyspace.HINTS_CF,
startColumn,
Composites.EMPTY,
false,
pageSize,
now);
ColumnFamily hintsPage = ColumnFamilyStore.removeDeleted(hintStore.getColumnFamily(filter), (int) (now / 1000));
if (pagingFinished(hintsPage, startColumn))
{
logger.info("Finished hinted handoff of {} rows to endpoint {}", rowsReplayed, endpoint);
finished = true;
break;
}
// check if node is still alive and we should continue delivery process
if (!FailureDetector.instance.isAlive(endpoint))
{
logger.info("Endpoint {} died during hint delivery; aborting ({} delivered)", endpoint, rowsReplayed);
break;
}
List<WriteResponseHandler> responseHandlers = Lists.newArrayList();
for (final Cell hint : hintsPage)
{
// check if hints delivery has been paused during the process
if (hintedHandOffPaused)
{
logger.debug("Hints delivery process is paused, aborting");
break delivery;
}
// Skip tombstones:
// if we iterate quickly enough, it's possible that we could request a new page in the same millisecond
// in which the local deletion timestamp was generated on the last column in the old page, in which
// case the hint will have no columns (since it's deleted) but will still be included in the resultset
// since (even with gcgs=0) it's still a "relevant" tombstone.
if (!hint.isLive())
continue;
startColumn = hint.name();
int version = Int32Type.instance.compose(hint.name().get(1));
DataInputStream in = new DataInputStream(ByteBufferUtil.inputStream(hint.value()));
Mutation mutation;
try
{
mutation = Mutation.serializer.deserialize(in, version);
}
catch (UnknownColumnFamilyException e)
{
logger.debug("Skipping delivery of hint for deleted columnfamily", e);
deleteHint(hostIdBytes, hint.name(), hint.timestamp());
continue;
}
catch (IOException e)
{
throw new AssertionError(e);
}
for (UUID cfId : mutation.getColumnFamilyIds())
{
if (hint.timestamp() <= SystemKeyspace.getTruncatedAt(cfId))
{
logger.debug("Skipping delivery of hint for truncated columnfamily {}", cfId);
mutation = mutation.without(cfId);
}
}
if (mutation.isEmpty())
{
deleteHint(hostIdBytes, hint.name(), hint.timestamp());
continue;
}
MessageOut<Mutation> message = mutation.createMessage();
rateLimiter.acquire(message.serializedSize(MessagingService.current_version));
Runnable callback = new Runnable()
{
public void run()
{
rowsReplayed.incrementAndGet();