KafkaIndexTask: allow pause to break out of retry loop (#3401)

This commit is contained in:
David Lim 2016-09-06 22:29:37 -06:00 committed by Fangjin Yang
parent 8d2ae144a8
commit 146a17de48
2 changed files with 57 additions and 27 deletions

View File

@ -29,7 +29,6 @@ import com.google.common.base.Function;
import com.google.common.base.Joiner;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import com.google.common.base.Supplier;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
@ -40,7 +39,6 @@ import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import com.google.common.primitives.Ints;
import com.metamx.common.ISE;
import com.metamx.common.RetryUtils;
import com.metamx.common.guava.Sequence;
import com.metamx.common.logger.Logger;
import com.metamx.common.parsers.ParseException;
@ -97,7 +95,6 @@ import java.util.Map;
import java.util.Properties;
import java.util.Random;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.Condition;
@ -121,6 +118,7 @@ public class KafkaIndexTask extends AbstractTask implements ChatHandler
private static final String TYPE = "index_kafka";
private static final Random RANDOM = new Random();
private static final long POLL_TIMEOUT = 100;
private static final long POLL_RETRY_MS = 30000;
private static final String METADATA_NEXT_PARTITIONS = "nextPartitions";
private final DataSchema dataSchema;
@ -161,6 +159,8 @@ public class KafkaIndexTask extends AbstractTask implements ChatHandler
private final Lock pauseLock = new ReentrantLock();
private final Condition hasPaused = pauseLock.newCondition();
private final Condition shouldResume = pauseLock.newCondition();
private final Lock pollRetryLock = new ReentrantLock();
private final Condition isAwaitingRetry = pollRetryLock.newCondition();
private volatile boolean pauseRequested = false;
private volatile long pauseMillis = 0;
@ -342,6 +342,7 @@ public class KafkaIndexTask extends AbstractTask implements ChatHandler
// Main loop.
// Could eventually support leader/follower mode (for keeping replicas more in sync)
boolean stillReading = !assignment.isEmpty();
status = Status.READING;
try {
while (stillReading) {
if (possiblyPause(assignment)) {
@ -363,30 +364,23 @@ public class KafkaIndexTask extends AbstractTask implements ChatHandler
// The retrying business is because the KafkaConsumer throws OffsetOutOfRangeException if the seeked-to
// offset is not present in the topic-partition. This can happen if we're asking a task to read from data
// that has not been written yet (which is totally legitimate). So let's wait for it to show up.
final ConsumerRecords<byte[], byte[]> records = RetryUtils.retry(
new Callable<ConsumerRecords<byte[], byte[]>>()
{
@Override
public ConsumerRecords<byte[], byte[]> call() throws Exception
{
try {
return consumer.poll(POLL_TIMEOUT);
}
finally {
status = Status.READING;
}
}
},
new Predicate<Throwable>()
{
@Override
public boolean apply(Throwable input)
{
return input instanceof OffsetOutOfRangeException;
}
},
Integer.MAX_VALUE
);
ConsumerRecords<byte[], byte[]> records = ConsumerRecords.empty();
try {
records = consumer.poll(POLL_TIMEOUT);
}
catch (OffsetOutOfRangeException e) {
log.warn("OffsetOutOfRangeException with message [%s], retrying in %dms", e.getMessage(), POLL_RETRY_MS);
pollRetryLock.lockInterruptibly();
try {
long nanos = TimeUnit.MILLISECONDS.toNanos(POLL_RETRY_MS);
while (nanos > 0L && !pauseRequested) {
nanos = isAwaitingRetry.awaitNanos(nanos);
}
}
finally {
pollRetryLock.unlock();
}
}
for (ConsumerRecord<byte[], byte[]> record : records) {
if (log.isTraceEnabled()) {
@ -689,6 +683,14 @@ public class KafkaIndexTask extends AbstractTask implements ChatHandler
pauseMillis = timeout <= 0 ? PAUSE_FOREVER : timeout;
pauseRequested = true;
pollRetryLock.lockInterruptibly();
try {
isAwaitingRetry.signalAll();
}
finally {
pollRetryLock.unlock();
}
if (isPaused()) {
shouldResume.signalAll(); // kick the monitor so it re-awaits with the new pauseMillis
}

View File

@ -1200,6 +1200,34 @@ public class KafkaIndexTaskTest
Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc3));
}
@Test(timeout = 30_000L)
public void testRunWithOffsetOutOfRangeExceptionAndPause() throws Exception
{
final KafkaIndexTask task = createTask(
null,
new KafkaIOConfig(
"sequence0",
new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)),
new KafkaPartitions("topic0", ImmutableMap.of(0, 5L)),
kafkaServer.consumerProperties(),
true,
false,
null
),
null
);
runTask(task);
while (!task.getStatus().equals(KafkaIndexTask.Status.READING)) {
Thread.sleep(2000);
}
task.pause(0);
Assert.assertEquals(KafkaIndexTask.Status.PAUSED, task.getStatus());
}
private ListenableFuture<TaskStatus> runTask(final Task task)
{
try {