mirror of https://github.com/apache/druid.git
KafkaIndexTask: allow pause to break out of retry loop (#3401)
This commit is contained in:
parent
8d2ae144a8
commit
146a17de48
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
|
|
Loading…
Reference in New Issue