add control and status endpoints to KafkaIndexTask (#2730)

This commit is contained in:
David Lim 2016-04-21 16:34:59 -06:00 committed by Gian Merlino
parent 984a518c9f
commit 7641f2628f
4 changed files with 685 additions and 158 deletions

View File

@ -29,12 +29,14 @@ import java.util.Map;
public class KafkaIOConfig implements IOConfig
{
private static final boolean DEFAULT_USE_TRANSACTION = true;
private static final boolean DEFAULT_PAUSE_AFTER_READ = false;
private final String baseSequenceName;
private final KafkaPartitions startPartitions;
private final KafkaPartitions endPartitions;
private final Map<String, String> consumerProperties;
private final boolean useTransaction;
private final boolean pauseAfterRead;
@JsonCreator
public KafkaIOConfig(
@ -42,7 +44,8 @@ public class KafkaIOConfig implements IOConfig
@JsonProperty("startPartitions") KafkaPartitions startPartitions,
@JsonProperty("endPartitions") KafkaPartitions endPartitions,
@JsonProperty("consumerProperties") Map<String, String> consumerProperties,
@JsonProperty("useTransaction") Boolean useTransaction
@JsonProperty("useTransaction") Boolean useTransaction,
@JsonProperty("pauseAfterRead") Boolean pauseAfterRead
)
{
this.baseSequenceName = Preconditions.checkNotNull(baseSequenceName, "baseSequenceName");
@ -50,6 +53,7 @@ public class KafkaIOConfig implements IOConfig
this.endPartitions = Preconditions.checkNotNull(endPartitions, "endPartitions");
this.consumerProperties = Preconditions.checkNotNull(consumerProperties, "consumerProperties");
this.useTransaction = useTransaction != null ? useTransaction : DEFAULT_USE_TRANSACTION;
this.pauseAfterRead = pauseAfterRead != null ? pauseAfterRead : DEFAULT_PAUSE_AFTER_READ;
Preconditions.checkArgument(
startPartitions.getTopic().equals(endPartitions.getTopic()),
@ -101,6 +105,12 @@ public class KafkaIOConfig implements IOConfig
return useTransaction;
}
@JsonProperty
public boolean isPauseAfterRead()
{
return pauseAfterRead;
}
@Override
public String toString()
{
@ -110,6 +120,7 @@ public class KafkaIOConfig implements IOConfig
", endPartitions=" + endPartitions +
", consumerProperties=" + consumerProperties +
", useTransaction=" + useTransaction +
", pauseAfterRead=" + pauseAfterRead +
'}';
}
}

View File

@ -19,15 +19,19 @@
package io.druid.indexing.kafka;
import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.annotations.VisibleForTesting;
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;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
@ -66,6 +70,8 @@ import io.druid.segment.realtime.appenderator.FiniteAppenderatorDriver;
import io.druid.segment.realtime.appenderator.SegmentIdentifier;
import io.druid.segment.realtime.appenderator.SegmentsAndMetadata;
import io.druid.segment.realtime.appenderator.TransactionalSegmentPublisher;
import io.druid.segment.realtime.firehose.ChatHandler;
import io.druid.segment.realtime.firehose.ChatHandlerProvider;
import io.druid.timeline.DataSegment;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.ConsumerRecords;
@ -73,7 +79,17 @@ import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.clients.consumer.OffsetOutOfRangeException;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.serialization.ByteArrayDeserializer;
import org.joda.time.DateTime;
import javax.ws.rs.Consumes;
import javax.ws.rs.DefaultValue;
import javax.ws.rs.GET;
import javax.ws.rs.POST;
import javax.ws.rs.Path;
import javax.ws.rs.Produces;
import javax.ws.rs.QueryParam;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import java.io.File;
import java.io.IOException;
import java.nio.ByteBuffer;
@ -82,9 +98,25 @@ 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;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
public class KafkaIndexTask extends AbstractTask
public class KafkaIndexTask extends AbstractTask implements ChatHandler
{
public static final long PAUSE_FOREVER = -1L;
public enum Status
{
NOT_STARTED,
STARTING,
READING,
PAUSED,
PUBLISHING
}
private static final Logger log = new Logger(KafkaIndexTask.class);
private static final String TYPE = "index_kafka";
private static final Random RANDOM = new Random();
@ -95,13 +127,42 @@ public class KafkaIndexTask extends AbstractTask
private final InputRowParser<ByteBuffer> parser;
private final KafkaTuningConfig tuningConfig;
private final KafkaIOConfig ioConfig;
private final Optional<ChatHandlerProvider> chatHandlerProvider;
private final Map<Integer, Long> endOffsets = new ConcurrentHashMap<>();
private final Map<Integer, Long> nextOffsets = new ConcurrentHashMap<>();
private ObjectMapper mapper;
private volatile Appenderator appenderator = null;
private volatile FireDepartmentMetrics fireDepartmentMetrics = null;
private volatile boolean startedReading = false;
private volatile boolean stopping = false;
private volatile boolean publishing = false;
private volatile DateTime startTime;
private volatile Status status = Status.NOT_STARTED; // this is only ever set by the task runner thread (runThread)
private volatile Thread runThread = null;
private volatile boolean stopRequested = false;
private volatile boolean publishOnStop = false;
// The pause lock and associated conditions are to support coordination between the Jetty threads and the main
// ingestion loop. The goal is to provide callers of the API a guarantee that if pause() returns successfully
// the ingestion loop has been stopped at the returned offsets and will not ingest any more data until resumed. The
// fields are used as follows (every step requires acquiring [pauseLock]):
// Pausing:
// - In pause(), [pauseRequested] is set to true and then execution waits for [status] to change to PAUSED, with the
// condition checked when [hasPaused] is signalled.
// - In possiblyPause() called from the main loop, if [pauseRequested] is true, [status] is set to PAUSED,
// [hasPaused] is signalled, and execution pauses until [pauseRequested] becomes false, either by being set or by
// the [pauseMillis] timeout elapsing. [pauseRequested] is checked when [shouldResume] is signalled.
// Resuming:
// - In resume(), [pauseRequested] is set to false, [shouldResume] is signalled, and execution waits for [status] to
// change to something other than PAUSED, with the condition checked when [shouldResume] is signalled.
// - In possiblyPause(), when [shouldResume] is signalled, if [pauseRequested] has become false the pause loop ends,
// [status] is changed to STARTING and [shouldResume] is signalled.
private final Lock pauseLock = new ReentrantLock();
private final Condition hasPaused = pauseLock.newCondition();
private final Condition shouldResume = pauseLock.newCondition();
private volatile boolean pauseRequested = false;
private volatile long pauseMillis = 0;
@JsonCreator
public KafkaIndexTask(
@ -110,7 +171,8 @@ public class KafkaIndexTask extends AbstractTask
@JsonProperty("dataSchema") DataSchema dataSchema,
@JsonProperty("tuningConfig") KafkaTuningConfig tuningConfig,
@JsonProperty("ioConfig") KafkaIOConfig ioConfig,
@JsonProperty("context") Map<String, Object> context
@JsonProperty("context") Map<String, Object> context,
@JacksonInject ChatHandlerProvider chatHandlerProvider
)
{
super(
@ -125,6 +187,9 @@ public class KafkaIndexTask extends AbstractTask
this.parser = Preconditions.checkNotNull((InputRowParser<ByteBuffer>) dataSchema.getParser(), "parser");
this.tuningConfig = Preconditions.checkNotNull(tuningConfig, "tuningConfig");
this.ioConfig = Preconditions.checkNotNull(ioConfig, "ioConfig");
this.chatHandlerProvider = Optional.fromNullable(chatHandlerProvider);
this.endOffsets.putAll(ioConfig.getEndPartitions().getPartitionOffsetMap());
}
private static String makeTaskId(String dataSource, int randomBits)
@ -166,19 +231,20 @@ public class KafkaIndexTask extends AbstractTask
return ioConfig;
}
/**
* Public for tests.
*/
@JsonIgnore
public boolean hasStartedReading()
{
return startedReading;
}
@Override
public TaskStatus run(final TaskToolbox toolbox) throws Exception
{
log.info("Starting up!");
startTime = DateTime.now();
mapper = toolbox.getObjectMapper();
status = Status.STARTING;
if (chatHandlerProvider.isPresent()) {
log.info("Found chat handler of class[%s]", chatHandlerProvider.get().getClass().getName());
chatHandlerProvider.get().register(getId(), this);
} else {
log.warn("No chat handler detected");
}
runThread = Thread.currentThread();
@ -207,7 +273,6 @@ public class KafkaIndexTask extends AbstractTask
// Start up, set up initial offsets.
final Object restoredMetadata = driver.startJob();
final Map<Integer, Long> nextOffsets = Maps.newHashMap();
if (restoredMetadata == null) {
nextOffsets.putAll(ioConfig.getStartPartitions().getPartitionOffsetMap());
} else {
@ -272,39 +337,26 @@ public class KafkaIndexTask extends AbstractTask
}
};
// Initialize consumer assignment.
final Set<Integer> assignment = Sets.newHashSet();
for (Map.Entry<Integer, Long> entry : nextOffsets.entrySet()) {
final long endOffset = ioConfig.getEndPartitions().getPartitionOffsetMap().get(entry.getKey());
if (entry.getValue() < endOffset) {
assignment.add(entry.getKey());
} else if (entry.getValue() == endOffset) {
log.info("Finished reading partition[%d].", entry.getKey());
} else {
throw new ISE(
"WTF?! Cannot start from offset[%,d] > endOffset[%,d]",
entry.getValue(),
endOffset
);
}
}
assignPartitions(consumer, topic, assignment);
// Seek to starting offsets.
for (final int partition : assignment) {
final long offset = nextOffsets.get(partition);
log.info("Seeking partition[%d] to offset[%,d].", partition, offset);
consumer.seek(new TopicPartition(topic, partition), offset);
}
Set<Integer> assignment = assignPartitionsAndSeekToNext(consumer, topic);
// Main loop.
// Could eventually support early termination (triggered by a supervisor)
// Could eventually support leader/follower mode (for keeping replicas more in sync)
boolean stillReading = !assignment.isEmpty();
try {
while (stillReading) {
if (stopping) {
log.info("Stopping early.");
if (possiblyPause(assignment)) {
// The partition assignments may have changed while paused by a call to setEndOffsets() so reassign
// partitions upon resuming. This is safe even if the end offsets have not been modified.
assignment = assignPartitionsAndSeekToNext(consumer, topic);
if (assignment.isEmpty()) {
log.info("All partitions have been fully read");
publishOnStop = true;
stopRequested = true;
}
}
if (stopRequested) {
break;
}
@ -321,7 +373,7 @@ public class KafkaIndexTask extends AbstractTask
return consumer.poll(POLL_TIMEOUT);
}
finally {
startedReading = true;
status = Status.READING;
}
}
},
@ -346,7 +398,7 @@ public class KafkaIndexTask extends AbstractTask
);
}
if (record.offset() < ioConfig.getEndPartitions().getPartitionOffsetMap().get(record.partition())) {
if (record.offset() < endOffsets.get(record.partition())) {
if (record.offset() != nextOffsets.get(record.partition())) {
throw new ISE(
"WTF?! Got offset[%,d] after offset[%,d] in partition[%d].",
@ -388,37 +440,39 @@ public class KafkaIndexTask extends AbstractTask
}
}
final long nextOffset = record.offset() + 1;
final long endOffset = ioConfig.getEndPartitions().getPartitionOffsetMap().get(record.partition());
nextOffsets.put(record.partition(), record.offset() + 1);
}
nextOffsets.put(record.partition(), nextOffset);
if (nextOffset == endOffset && assignment.remove(record.partition())) {
if (nextOffsets.get(record.partition()).equals(endOffsets.get(record.partition()))
&& assignment.remove(record.partition())) {
log.info("Finished reading topic[%s], partition[%,d].", record.topic(), record.partition());
assignPartitions(consumer, topic, assignment);
stillReading = !assignment.isEmpty();
stillReading = ioConfig.isPauseAfterRead() || !assignment.isEmpty();
}
}
}
}
finally {
driver.persist(committerSupplier.get()); // persist pending data
}
// Persist pending data.
final Committer finalCommitter = committerSupplier.get();
driver.persist(finalCommitter);
publishing = true;
if (stopping) {
// Stopped gracefully. Exit code shouldn't matter, so fail to be on the safe side.
return TaskStatus.failure(getId());
if (stopRequested && !publishOnStop) {
throw new InterruptedException("Stopping without publishing");
}
status = Status.PUBLISHING;
final TransactionalSegmentPublisher publisher = new TransactionalSegmentPublisher()
{
@Override
public boolean publishSegments(Set<DataSegment> segments, Object commitMetadata) throws IOException
{
final KafkaPartitions finalPartitions = toolbox.getObjectMapper().convertValue(
((Map) commitMetadata).get(METADATA_NEXT_PARTITIONS),
KafkaPartitions.class
);
// Sanity check, we should only be publishing things that match our desired end state.
if (!ioConfig.getEndPartitions().equals(((Map) commitMetadata).get(METADATA_NEXT_PARTITIONS))) {
if (!endOffsets.equals(finalPartitions.getPartitionOffsetMap())) {
throw new ISE("WTF?! Driver attempted to publish invalid metadata[%s].", commitMetadata);
}
@ -428,7 +482,7 @@ public class KafkaIndexTask extends AbstractTask
action = new SegmentInsertAction(
segments,
new KafkaDataSourceMetadata(ioConfig.getStartPartitions()),
new KafkaDataSourceMetadata(ioConfig.getEndPartitions())
new KafkaDataSourceMetadata(finalPartitions)
);
} else {
action = new SegmentInsertAction(segments, null, null);
@ -463,6 +517,15 @@ public class KafkaIndexTask extends AbstractTask
);
}
}
catch (InterruptedException e) {
// if we were interrupted because we were asked to stop, handle the exception and return success, else rethrow
if (!stopRequested) {
Thread.currentThread().interrupt();
throw e;
}
log.info("The task was asked to stop before completing");
}
return success();
}
@ -473,14 +536,15 @@ public class KafkaIndexTask extends AbstractTask
return true;
}
@POST
@Path("/stop")
@Override
public void stopGracefully()
{
log.info("Stopping gracefully.");
stopping = true;
if (publishing && runThread.isAlive()) {
log.info("stopGracefully: Run thread started publishing, interrupting it.");
stopRequested = true;
if (runThread.isAlive()) {
log.info("Interrupting run thread (status: [%s])", status);
runThread.interrupt();
}
}
@ -503,12 +567,183 @@ public class KafkaIndexTask extends AbstractTask
};
}
@GET
@Path("/status")
@Produces(MediaType.APPLICATION_JSON)
public Status getStatus()
{
return status;
}
@GET
@Path("/offsets/current")
@Produces(MediaType.APPLICATION_JSON)
public Map<Integer, Long> getCurrentOffsets()
{
return nextOffsets;
}
@GET
@Path("/offsets/end")
@Produces(MediaType.APPLICATION_JSON)
public Map<Integer, Long> getEndOffsets()
{
return endOffsets;
}
@POST
@Path("/offsets/end")
@Consumes(MediaType.APPLICATION_JSON)
@Produces(MediaType.APPLICATION_JSON)
public Response setEndOffsets(
Map<Integer, Long> offsets,
@QueryParam("resume") @DefaultValue("false") final boolean resume
) throws InterruptedException
{
if (offsets == null) {
return Response.status(Response.Status.BAD_REQUEST)
.entity("Request body must contain a map of { partition:endOffset }")
.build();
} else if (!endOffsets.keySet().containsAll(offsets.keySet())) {
return Response.status(Response.Status.BAD_REQUEST)
.entity(
String.format(
"Request contains partitions not being handled by this task, my partitions: %s",
endOffsets.keySet()
)
)
.build();
}
pauseLock.lockInterruptibly();
try {
if (!isPaused()) {
return Response.status(Response.Status.BAD_REQUEST)
.entity("Task must be paused before changing the end offsets")
.build();
}
for (Map.Entry<Integer, Long> entry : offsets.entrySet()) {
if (entry.getValue().compareTo(nextOffsets.get(entry.getKey())) < 0) {
return Response.status(Response.Status.BAD_REQUEST)
.entity(
String.format(
"End offset must be >= current offset for partition [%s] (current: %s)",
entry.getKey(),
nextOffsets.get(entry.getKey())
)
)
.build();
}
}
endOffsets.putAll(offsets);
log.info("endOffsets changed to %s", endOffsets);
}
finally {
pauseLock.unlock();
}
if (resume) {
resume();
}
return Response.ok(endOffsets).build();
}
/**
* Signals the ingestion loop to pause.
*
* @param timeout how long to pause for before resuming in milliseconds, <= 0 means indefinitely
*
* @return one of the following Responses: 400 Bad Request if the task has started publishing; 202 Accepted if the
* method has timed out and returned before the task has paused; 200 OK with a map of the current partition offsets
* in the response body if the task successfully paused
*/
@POST
@Path("/pause")
@Produces(MediaType.APPLICATION_JSON)
public Response pause(@QueryParam("timeout") @DefaultValue("0") final long timeout)
throws InterruptedException
{
if (!(status == Status.PAUSED || status == Status.READING)) {
return Response.status(Response.Status.BAD_REQUEST)
.entity(String.format("Can't pause, task is not in a pausable state (state: [%s])", status))
.build();
}
pauseLock.lockInterruptibly();
try {
pauseMillis = timeout <= 0 ? PAUSE_FOREVER : timeout;
pauseRequested = true;
if (isPaused()) {
shouldResume.signalAll(); // kick the monitor so it re-awaits with the new pauseMillis
}
long nanos = TimeUnit.SECONDS.toNanos(2);
while (!isPaused()) {
if (nanos <= 0L) {
return Response.status(Response.Status.ACCEPTED)
.entity("Request accepted but task has not yet paused")
.build();
}
nanos = hasPaused.awaitNanos(nanos);
}
}
finally {
pauseLock.unlock();
}
try {
return Response.ok().entity(mapper.writeValueAsString(getCurrentOffsets())).build();
}
catch (JsonProcessingException e) {
throw Throwables.propagate(e);
}
}
@POST
@Path("/resume")
public void resume() throws InterruptedException
{
pauseLock.lockInterruptibly();
try {
pauseRequested = false;
shouldResume.signalAll();
long nanos = TimeUnit.SECONDS.toNanos(5);
while (isPaused()) {
if (nanos <= 0L) {
throw new RuntimeException("Resume command was not accepted within 5 seconds");
}
nanos = shouldResume.awaitNanos(nanos);
}
}
finally {
pauseLock.unlock();
}
}
@GET
@Path("/time/start")
@Produces(MediaType.APPLICATION_JSON)
public DateTime getStartTime()
{
return startTime;
}
@VisibleForTesting
public FireDepartmentMetrics getFireDepartmentMetrics()
FireDepartmentMetrics getFireDepartmentMetrics()
{
return fireDepartmentMetrics;
}
private boolean isPaused()
{
return status == Status.PAUSED;
}
private Appenderator newAppenderator(FireDepartmentMetrics metrics, TaskToolbox toolbox)
{
return Appenderators.createRealtime(
@ -590,4 +825,95 @@ public class KafkaIndexTask extends AbstractTask
)
);
}
private Set<Integer> assignPartitionsAndSeekToNext(KafkaConsumer consumer, String topic)
{
// Initialize consumer assignment.
final Set<Integer> assignment = Sets.newHashSet();
for (Map.Entry<Integer, Long> entry : nextOffsets.entrySet()) {
final long endOffset = endOffsets.get(entry.getKey());
if (entry.getValue() < endOffset) {
assignment.add(entry.getKey());
} else if (entry.getValue() == endOffset) {
log.info("Finished reading partition[%d].", entry.getKey());
} else {
throw new ISE(
"WTF?! Cannot start from offset[%,d] > endOffset[%,d]",
entry.getValue(),
endOffset
);
}
}
assignPartitions(consumer, topic, assignment);
// Seek to starting offsets.
for (final int partition : assignment) {
final long offset = nextOffsets.get(partition);
log.info("Seeking partition[%d] to offset[%,d].", partition, offset);
consumer.seek(new TopicPartition(topic, partition), offset);
}
return assignment;
}
/**
* Checks if the pauseRequested flag was set and if so blocks:
* a) if pauseMillis == PAUSE_FOREVER, until pauseRequested is cleared
* b) if pauseMillis != PAUSE_FOREVER, until pauseMillis elapses -or- pauseRequested is cleared
* <p>
* If pauseMillis is changed while paused, the new pause timeout will be applied. This allows adjustment of the
* pause timeout (making a timed pause into an indefinite pause and vice versa is valid) without having to resume
* and ensures that the loop continues to stay paused without ingesting any new events. You will need to signal
* shouldResume after adjusting pauseMillis for the new value to take effect.
* <p>
* Sets paused = true and signals paused so callers can be notified when the pause command has been accepted.
* <p>
* Additionally, pauses if all partitions assignments have been read and pauseAfterRead flag is set.
*
* @return true if a pause request was handled, false otherwise
*/
private boolean possiblyPause(Set<Integer> assignment) throws InterruptedException
{
pauseLock.lockInterruptibly();
try {
if (ioConfig.isPauseAfterRead() && assignment.isEmpty()) {
pauseMillis = PAUSE_FOREVER;
pauseRequested = true;
}
if (pauseRequested) {
status = Status.PAUSED;
long nanos = 0;
hasPaused.signalAll();
while (pauseRequested) {
if (pauseMillis == PAUSE_FOREVER) {
log.info("Pausing ingestion until resumed");
shouldResume.await();
} else {
if (pauseMillis > 0) {
log.info("Pausing ingestion for [%,d] ms", pauseMillis);
nanos = TimeUnit.MILLISECONDS.toNanos(pauseMillis);
pauseMillis = 0;
}
if (nanos <= 0L) {
pauseRequested = false; // timeout elapsed
}
nanos = shouldResume.awaitNanos(nanos);
}
}
status = Status.READING;
shouldResume.signalAll();
log.info("Ingestion loop resumed");
return true;
}
}
finally {
pauseLock.unlock();
}
return false;
}
}

View File

@ -19,6 +19,8 @@
package io.druid.indexing.kafka;
import com.fasterxml.jackson.core.JsonParser;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Charsets;
@ -137,6 +139,7 @@ import java.util.concurrent.Callable;
import java.util.concurrent.Executor;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
@RunWith(Parameterized.class)
public class KafkaIndexTaskTest
@ -159,6 +162,7 @@ public class KafkaIndexTaskTest
private final List<Task> runningTasks = Lists.newArrayList();
private static final Logger log = new Logger(KafkaIndexTaskTest.class);
private static final ObjectMapper objectMapper = new DefaultObjectMapper();
private static final DataSchema DATA_SCHEMA;
@ -175,7 +179,6 @@ public class KafkaIndexTaskTest
);
static {
ObjectMapper objectMapper = new DefaultObjectMapper();
DATA_SCHEMA = new DataSchema(
"test_ds",
objectMapper.convertValue(
@ -298,7 +301,8 @@ public class KafkaIndexTaskTest
new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)),
new KafkaPartitions("topic0", ImmutableMap.of(0, 5L)),
kafkaServer.consumerProperties(),
true
true,
false
),
null
);
@ -337,7 +341,8 @@ public class KafkaIndexTaskTest
new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)),
new KafkaPartitions("topic0", ImmutableMap.of(0, 5L)),
kafkaServer.consumerProperties(),
true
true,
false
),
null
);
@ -345,7 +350,7 @@ public class KafkaIndexTaskTest
final ListenableFuture<TaskStatus> future = runTask(task);
// Wait for the task to start reading
while (!task.hasStartedReading()) {
while (task.getStatus() != KafkaIndexTask.Status.READING) {
Thread.sleep(10);
}
@ -395,7 +400,8 @@ public class KafkaIndexTaskTest
new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)),
new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)),
kafkaServer.consumerProperties(),
true
true,
false
),
null
);
@ -433,7 +439,8 @@ public class KafkaIndexTaskTest
new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)),
new KafkaPartitions("topic0", ImmutableMap.of(0, 5L)),
kafkaServer.consumerProperties(),
true
true,
false
),
null
);
@ -482,7 +489,8 @@ public class KafkaIndexTaskTest
new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)),
new KafkaPartitions("topic0", ImmutableMap.of(0, 5L)),
kafkaServer.consumerProperties(),
true
true,
false
),
null
);
@ -530,7 +538,8 @@ public class KafkaIndexTaskTest
new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)),
new KafkaPartitions("topic0", ImmutableMap.of(0, 7L)),
kafkaServer.consumerProperties(),
true
true,
false
),
null
);
@ -560,7 +569,8 @@ public class KafkaIndexTaskTest
new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)),
new KafkaPartitions("topic0", ImmutableMap.of(0, 5L)),
kafkaServer.consumerProperties(),
true
true,
false
),
null
);
@ -571,7 +581,8 @@ public class KafkaIndexTaskTest
new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)),
new KafkaPartitions("topic0", ImmutableMap.of(0, 5L)),
kafkaServer.consumerProperties(),
true
true,
false
),
null
);
@ -622,7 +633,8 @@ public class KafkaIndexTaskTest
new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)),
new KafkaPartitions("topic0", ImmutableMap.of(0, 5L)),
kafkaServer.consumerProperties(),
true
true,
false
),
null
);
@ -633,7 +645,8 @@ public class KafkaIndexTaskTest
new KafkaPartitions("topic0", ImmutableMap.of(0, 3L)),
new KafkaPartitions("topic0", ImmutableMap.of(0, 7L)),
kafkaServer.consumerProperties(),
true
true,
false
),
null
);
@ -685,6 +698,7 @@ public class KafkaIndexTaskTest
new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)),
new KafkaPartitions("topic0", ImmutableMap.of(0, 5L)),
kafkaServer.consumerProperties(),
false,
false
),
null
@ -696,6 +710,7 @@ public class KafkaIndexTaskTest
new KafkaPartitions("topic0", ImmutableMap.of(0, 3L)),
new KafkaPartitions("topic0", ImmutableMap.of(0, 7L)),
kafkaServer.consumerProperties(),
false,
false
),
null
@ -753,7 +768,8 @@ public class KafkaIndexTaskTest
new KafkaPartitions("topic0", ImmutableMap.of(0, 2L, 1, 0L)),
new KafkaPartitions("topic0", ImmutableMap.of(0, 5L, 1, 2L)),
kafkaServer.consumerProperties(),
true
true,
false
),
null
);
@ -807,7 +823,8 @@ public class KafkaIndexTaskTest
new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)),
new KafkaPartitions("topic0", ImmutableMap.of(0, 5L)),
kafkaServer.consumerProperties(),
true
true,
false
),
null
);
@ -818,7 +835,8 @@ public class KafkaIndexTaskTest
new KafkaPartitions("topic0", ImmutableMap.of(1, 0L)),
new KafkaPartitions("topic0", ImmutableMap.of(1, 1L)),
kafkaServer.consumerProperties(),
true
true,
false
),
null
);
@ -871,7 +889,8 @@ public class KafkaIndexTaskTest
new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)),
new KafkaPartitions("topic0", ImmutableMap.of(0, 5L)),
kafkaServer.consumerProperties(),
true
true,
false
),
null
);
@ -891,9 +910,9 @@ public class KafkaIndexTaskTest
Assert.assertEquals(2, countEvents(task1));
// Stop gracefully
// Stop without publishing segment
task1.stopGracefully();
Assert.assertEquals(TaskStatus.Status.FAILED, future1.get().getStatusCode());
Assert.assertEquals(TaskStatus.Status.SUCCESS, future1.get().getStatusCode());
// Start a new task
final KafkaIndexTask task2 = createTask(
@ -903,7 +922,8 @@ public class KafkaIndexTaskTest
new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)),
new KafkaPartitions("topic0", ImmutableMap.of(0, 5L)),
kafkaServer.consumerProperties(),
true
true,
false
),
null
);
@ -942,6 +962,172 @@ public class KafkaIndexTaskTest
Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2));
}
@Test(timeout = 60_000L)
public void testRunWithPauseAndResume() 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
);
final ListenableFuture<TaskStatus> future = runTask(task);
// Insert some data, but not enough for the task to finish
try (final KafkaProducer<byte[], byte[]> kafkaProducer = kafkaServer.newProducer()) {
for (ProducerRecord<byte[], byte[]> record : Iterables.limit(RECORDS, 4)) {
kafkaProducer.send(record).get();
}
}
while (countEvents(task) != 2) {
Thread.sleep(25);
}
Assert.assertEquals(2, countEvents(task));
Assert.assertEquals(KafkaIndexTask.Status.READING, task.getStatus());
Map<Integer, Long> currentOffsets = objectMapper.readValue(
task.pause(0).getEntity().toString(),
new TypeReference<Map<Integer, Long>>()
{
}
);
Assert.assertEquals(KafkaIndexTask.Status.PAUSED, task.getStatus());
// Insert remaining data
try (final KafkaProducer<byte[], byte[]> kafkaProducer = kafkaServer.newProducer()) {
for (ProducerRecord<byte[], byte[]> record : Iterables.skip(RECORDS, 4)) {
kafkaProducer.send(record).get();
}
}
try {
future.get(10, TimeUnit.SECONDS);
Assert.fail("Task completed when it should have been paused");
}
catch (TimeoutException e) {
// carry on..
}
Assert.assertEquals(currentOffsets, task.getCurrentOffsets());
task.resume();
Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode());
Assert.assertEquals(task.getEndOffsets(), task.getCurrentOffsets());
// Check metrics
Assert.assertEquals(3, task.getFireDepartmentMetrics().processed());
Assert.assertEquals(0, task.getFireDepartmentMetrics().unparseable());
Assert.assertEquals(0, task.getFireDepartmentMetrics().thrownAway());
// Check published metadata
SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KafkaDataSourceMetadata(new KafkaPartitions("topic0", ImmutableMap.of(0, 5L))),
metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
);
// Check segments in deep storage
Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1));
Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2));
}
@Test(timeout = 60_000L)
public void testRunAndPauseAfterReadWithModifiedEndOffsets() throws Exception
{
final KafkaIndexTask task = createTask(
null,
new KafkaIOConfig(
"sequence0",
new KafkaPartitions("topic0", ImmutableMap.of(0, 1L)),
new KafkaPartitions("topic0", ImmutableMap.of(0, 3L)),
kafkaServer.consumerProperties(),
true,
true
),
null
);
final ListenableFuture<TaskStatus> future = runTask(task);
try (final KafkaProducer<byte[], byte[]> kafkaProducer = kafkaServer.newProducer()) {
for (ProducerRecord<byte[], byte[]> record : RECORDS) {
kafkaProducer.send(record).get();
}
}
while (task.getStatus() != KafkaIndexTask.Status.PAUSED) {
Thread.sleep(25);
}
// reached the end of the assigned offsets and paused instead of publishing
Assert.assertEquals(task.getEndOffsets(), task.getCurrentOffsets());
Assert.assertEquals(KafkaIndexTask.Status.PAUSED, task.getStatus());
Assert.assertEquals(ImmutableMap.of(0, 3L), task.getEndOffsets());
Map<Integer, Long> newEndOffsets = ImmutableMap.of(0, 4L);
task.setEndOffsets(newEndOffsets, false);
Assert.assertEquals(newEndOffsets, task.getEndOffsets());
Assert.assertEquals(KafkaIndexTask.Status.PAUSED, task.getStatus());
task.resume();
while (task.getStatus() != KafkaIndexTask.Status.PAUSED) {
Thread.sleep(25);
}
// reached the end of the updated offsets and paused
Assert.assertEquals(newEndOffsets, task.getCurrentOffsets());
Assert.assertEquals(KafkaIndexTask.Status.PAUSED, task.getStatus());
// try again but with resume flag == true
newEndOffsets = ImmutableMap.of(0, 6L);
task.setEndOffsets(newEndOffsets, true);
Assert.assertEquals(newEndOffsets, task.getEndOffsets());
Assert.assertNotEquals(KafkaIndexTask.Status.PAUSED, task.getStatus());
while (task.getStatus() != KafkaIndexTask.Status.PAUSED) {
Thread.sleep(25);
}
Assert.assertEquals(newEndOffsets, task.getCurrentOffsets());
Assert.assertEquals(KafkaIndexTask.Status.PAUSED, task.getStatus());
task.resume();
Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode());
// Check metrics
Assert.assertEquals(4, task.getFireDepartmentMetrics().processed());
Assert.assertEquals(1, task.getFireDepartmentMetrics().unparseable());
Assert.assertEquals(0, task.getFireDepartmentMetrics().thrownAway());
// Check published metadata
SegmentDescriptor desc1 = SD(task, "2009/P1D", 0);
SegmentDescriptor desc2 = SD(task, "2010/P1D", 0);
SegmentDescriptor desc3 = SD(task, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors());
Assert.assertEquals(
new KafkaDataSourceMetadata(new KafkaPartitions("topic0", ImmutableMap.of(0, 6L))),
metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
);
// Check segments in deep storage
Assert.assertEquals(ImmutableList.of("b"), readSegmentDim1(desc1));
Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc2));
Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc3));
}
private ListenableFuture<TaskStatus> runTask(final Task task)
{
try {
@ -1015,6 +1201,7 @@ public class KafkaIndexTaskTest
DATA_SCHEMA,
tuningConfig,
ioConfig,
null,
null
);
}

View File

@ -30,6 +30,8 @@ import io.druid.segment.IndexIO;
import io.druid.segment.IndexMerger;
import io.druid.segment.IndexMergerV9;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.realtime.firehose.ChatHandlerProvider;
import io.druid.segment.realtime.firehose.NoopChatHandlerProvider;
import java.util.List;
import java.util.concurrent.TimeUnit;
@ -70,6 +72,7 @@ public class TestUtils
.addValue(IndexIO.class, indexIO)
.addValue(IndexMerger.class, indexMerger)
.addValue(ObjectMapper.class, jsonMapper)
.addValue(ChatHandlerProvider.class, new NoopChatHandlerProvider())
);
}