diff --git a/docs/content/development/extensions-core/kafka-ingestion.md b/docs/content/development/extensions-core/kafka-ingestion.md index 9cd16e914a4..e0289f388b8 100644 --- a/docs/content/development/extensions-core/kafka-ingestion.md +++ b/docs/content/development/extensions-core/kafka-ingestion.md @@ -105,10 +105,10 @@ A sample supervisor spec is shown below: |--------|-----------|---------| |`type`|The supervisor type, this should always be `kafka`.|yes| |`dataSchema`|The schema that will be used by the Kafka indexing task during ingestion, see [Ingestion Spec](../../ingestion/index.html).|yes| -|`tuningConfig`|A KafkaTuningConfig that will be provided to indexing tasks, see below.|no| -|`ioConfig`|A KafkaSupervisorIOConfig to configure the supervisor, see below.|yes| +|`tuningConfig`|A KafkaSupervisorTuningConfig to configure the supervisor and indexing tasks, see below.|no| +|`ioConfig`|A KafkaSupervisorIOConfig to configure the supervisor and indexing tasks, see below.|yes| -### KafkaTuningConfig +### KafkaSupervisorTuningConfig The tuningConfig is optional and default parameters will be used if no tuningConfig is specified. @@ -123,6 +123,10 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |`buildV9Directly`|Boolean|Whether to build a v9 index directly instead of first building a v8 index and then converting it to v9 format.|no (default == false)| |`reportParseExceptions`|Boolean|If true, exceptions encountered during parsing will be thrown and will halt ingestion; if false, unparseable rows and fields will be skipped.|no (default == false)| |`handoffConditionTimeout`|Long|Milliseconds to wait for segment handoff. It must be >= 0, where 0 means to wait forever.|no (default == 0)| +|`workerThreads`|Integer|The number of threads that will be used by the supervisor for asynchronous operations.|no (default == min(10, taskCount))| +|`chatThreads`|Integer|The number of threads that will be used for communicating with indexing tasks.|no (default == min(10, taskCount * replicas))| +|`chatRetries`|Integer|The number of times HTTP requests to indexing tasks will be retried before considering tasks unresponsive.|no (default == 8)| +|`httpTimeout`|ISO8601 Period|How long to wait for a HTTP response from an indexing task.|no (default == PT10S)| #### IndexSpec diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java index e2bf2b99b42..07893c01477 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java @@ -25,6 +25,10 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Charsets; import com.google.common.base.Optional; import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableMap; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; import com.metamx.common.IAE; import com.metamx.common.ISE; import com.metamx.emitter.EmittingLogger; @@ -32,6 +36,7 @@ import com.metamx.http.client.HttpClient; import com.metamx.http.client.Request; import com.metamx.http.client.response.FullResponseHandler; import com.metamx.http.client.response.FullResponseHolder; +import io.druid.concurrent.Execs; import io.druid.indexing.common.RetryPolicy; import io.druid.indexing.common.RetryPolicyConfig; import io.druid.indexing.common.RetryPolicyFactory; @@ -51,12 +56,14 @@ import java.io.IOException; import java.net.Socket; import java.net.URI; import java.util.Map; +import java.util.concurrent.Callable; public class KafkaIndexTaskClient { public class NoTaskLocationException extends RuntimeException { - public NoTaskLocationException(String message) { + public NoTaskLocationException(String message) + { super(message); } } @@ -76,33 +83,89 @@ public class KafkaIndexTaskClient private final HttpClient httpClient; private final ObjectMapper jsonMapper; private final TaskInfoProvider taskInfoProvider; + private final Duration httpTimeout; private final RetryPolicyFactory retryPolicyFactory; + private final ListeningExecutorService executorService; + private final long numRetries; - public KafkaIndexTaskClient(HttpClient httpClient, ObjectMapper jsonMapper, TaskInfoProvider taskInfoProvider) + public KafkaIndexTaskClient( + HttpClient httpClient, + ObjectMapper jsonMapper, + TaskInfoProvider taskInfoProvider, + String dataSource, + int numThreads, + Duration httpTimeout, + long numRetries + ) { this.httpClient = httpClient; this.jsonMapper = jsonMapper; this.taskInfoProvider = taskInfoProvider; + this.httpTimeout = httpTimeout; + this.numRetries = numRetries; this.retryPolicyFactory = createRetryPolicyFactory(); + + this.executorService = MoreExecutors.listeningDecorator( + Execs.multiThreaded( + numThreads, + String.format( + "KafkaIndexTaskClient-%s-%%d", + dataSource + ) + ) + ); } - public void stop(String id, boolean publish) + public void close() { - submitRequest(id, HttpMethod.POST, "stop", publish ? "publish=true" : null, true); + executorService.shutdownNow(); } - public void resume(String id) + public boolean stop(final String id, final boolean publish) { - submitRequest(id, HttpMethod.POST, "resume", null, true); + log.debug("Stop task[%s] publish[%s]", id, publish); + + try { + final FullResponseHolder response = submitRequest( + id, HttpMethod.POST, "stop", publish ? "publish=true" : null, true + ); + return response.getStatus().getCode() / 100 == 2; + } + catch (NoTaskLocationException e) { + return false; + } + catch (TaskNotRunnableException e) { + log.info("Task [%s] couldn't be stopped because it is no longer running", id); + return true; + } + catch (Exception e) { + log.warn(e, "Exception while stopping task [%s]", id); + return false; + } } - public Map pause(String id) + public boolean resume(final String id) + { + log.debug("Resume task[%s]", id); + + try { + final FullResponseHolder response = submitRequest(id, HttpMethod.POST, "resume", null, true); + return response.getStatus().getCode() / 100 == 2; + } + catch (NoTaskLocationException e) { + return false; + } + } + + public Map pause(final String id) { return pause(id, 0); } - public Map pause(String id, long timeout) + public Map pause(final String id, final long timeout) { + log.debug("Pause task[%s] timeout[%d]", id, timeout); + try { final FullResponseHolder response = submitRequest( id, @@ -128,72 +191,99 @@ public class KafkaIndexTaskClient } else { final long sleepTime = delay.getMillis(); log.info( - "Still waiting for task [%s] to pause; will try again in [%s]", id, new Duration(sleepTime).toString() + "Still waiting for task [%s] to pause; will try again in [%s]", + id, + new Duration(sleepTime).toString() ); Thread.sleep(sleepTime); } } } + catch (NoTaskLocationException e) { + return ImmutableMap.of(); + } catch (IOException | InterruptedException e) { throw Throwables.propagate(e); } } - public KafkaIndexTask.Status getStatus(String id) + public KafkaIndexTask.Status getStatus(final String id) { + log.debug("GetStatus task[%s]", id); + try { final FullResponseHolder response = submitRequest(id, HttpMethod.GET, "status", null, true); return jsonMapper.readValue(response.getContent(), KafkaIndexTask.Status.class); } + catch (NoTaskLocationException e) { + return KafkaIndexTask.Status.NOT_STARTED; + } catch (IOException e) { throw Throwables.propagate(e); } } - public DateTime getStartTime(String id) + public DateTime getStartTime(final String id) { + log.debug("GetStartTime task[%s]", id); + try { final FullResponseHolder response = submitRequest(id, HttpMethod.GET, "time/start", null, true); return response.getContent() == null || response.getContent().isEmpty() ? null : jsonMapper.readValue(response.getContent(), DateTime.class); } + catch (NoTaskLocationException e) { + return null; + } catch (IOException e) { throw Throwables.propagate(e); } } - public Map getCurrentOffsets(String id, boolean retry) + public Map getCurrentOffsets(final String id, final boolean retry) { + log.debug("GetCurrentOffsets task[%s] retry[%s]", id, retry); + try { final FullResponseHolder response = submitRequest(id, HttpMethod.GET, "offsets/current", null, retry); return jsonMapper.readValue(response.getContent(), new TypeReference>() {}); } + catch (NoTaskLocationException e) { + return ImmutableMap.of(); + } catch (IOException e) { throw Throwables.propagate(e); } } - public Map getEndOffsets(String id) + public Map getEndOffsets(final String id) { + log.debug("GetEndOffsets task[%s]", id); + try { final FullResponseHolder response = submitRequest(id, HttpMethod.GET, "offsets/end", null, true); return jsonMapper.readValue(response.getContent(), new TypeReference>() {}); } + catch (NoTaskLocationException e) { + return ImmutableMap.of(); + } catch (IOException e) { throw Throwables.propagate(e); } } - public void setEndOffsets(String id, Map endOffsets) + public boolean setEndOffsets(final String id, final Map endOffsets) { - setEndOffsets(id, endOffsets, false); + return setEndOffsets(id, endOffsets, false); } - public void setEndOffsets(String id, Map endOffsets, boolean resume) + public boolean setEndOffsets(final String id, final Map endOffsets, final boolean resume) { + log.debug("SetEndOffsets task[%s] endOffsets[%s] resume[%s]", id, endOffsets, resume); + try { - submitRequest( + final FullResponseHolder response = submitRequest( id, HttpMethod.POST, "offsets/end", @@ -201,24 +291,151 @@ public class KafkaIndexTaskClient jsonMapper.writeValueAsBytes(endOffsets), true ); + return response.getStatus().getCode() / 100 == 2; + } + catch (NoTaskLocationException e) { + return false; } catch (IOException e) { throw Throwables.propagate(e); } } + public ListenableFuture stopAsync(final String id, final boolean publish) + { + return executorService.submit( + new Callable() + { + @Override + public Boolean call() throws Exception + { + return stop(id, publish); + } + } + ); + } + + public ListenableFuture resumeAsync(final String id) + { + return executorService.submit( + new Callable() + { + @Override + public Boolean call() throws Exception + { + return resume(id); + } + } + ); + } + + public ListenableFuture> pauseAsync(final String id) + { + return pauseAsync(id, 0); + } + + public ListenableFuture> pauseAsync(final String id, final long timeout) + { + return executorService.submit( + new Callable>() + { + @Override + public Map call() throws Exception + { + return pause(id, timeout); + } + } + ); + } + + public ListenableFuture getStatusAsync(final String id) + { + return executorService.submit( + new Callable() + { + @Override + public KafkaIndexTask.Status call() throws Exception + { + return getStatus(id); + } + } + ); + } + + public ListenableFuture getStartTimeAsync(final String id) + { + return executorService.submit( + new Callable() + { + @Override + public DateTime call() throws Exception + { + return getStartTime(id); + } + } + ); + } + + public ListenableFuture> getCurrentOffsetsAsync(final String id, final boolean retry) + { + return executorService.submit( + new Callable>() + { + @Override + public Map call() throws Exception + { + return getCurrentOffsets(id, retry); + } + } + ); + } + + public ListenableFuture> getEndOffsetsAsync(final String id) + { + return executorService.submit( + new Callable>() + { + @Override + public Map call() throws Exception + { + return getEndOffsets(id); + } + } + ); + } + + public ListenableFuture setEndOffsetsAsync(final String id, final Map endOffsets) + { + return setEndOffsetsAsync(id, endOffsets, false); + } + + public ListenableFuture setEndOffsetsAsync( + final String id, final Map endOffsets, final boolean resume + ) + { + return executorService.submit( + new Callable() + { + @Override + public Boolean call() throws Exception + { + return setEndOffsets(id, endOffsets, resume); + } + } + ); + } + @VisibleForTesting RetryPolicyFactory createRetryPolicyFactory() { - // Retries for about a minute before giving up; this should be long enough to handle any temporary unresponsiveness - // such as network issues, if a task is still in the process of starting up, or if the task is in the middle of - // persisting to disk and doesn't respond immediately. - + // Retries [numRetries] times before giving up; this should be set long enough to handle any temporary + // unresponsiveness such as network issues, if a task is still in the process of starting up, or if the task is in + // the middle of persisting to disk and doesn't respond immediately. return new RetryPolicyFactory( new RetryPolicyConfig() .setMinWait(Period.seconds(2)) - .setMaxWait(Period.seconds(8)) - .setMaxRetryCount(8) + .setMaxWait(Period.seconds(10)) + .setMaxRetryCount(numRetries) ); } @@ -246,6 +463,8 @@ public class KafkaIndexTaskClient while (true) { FullResponseHolder response = null; Request request = null; + TaskLocation location = TaskLocation.unknown(); + String path = String.format("%s/%s/%s", BASE_PATH, id, pathSuffix); Optional status = taskInfoProvider.getTaskStatus(id); if (!status.isPresent() || !status.get().isRunnable()) { @@ -253,9 +472,8 @@ public class KafkaIndexTaskClient } try { - TaskLocation location = taskInfoProvider.getTaskLocation(id); + location = taskInfoProvider.getTaskLocation(id); if (location.equals(TaskLocation.unknown())) { - log.info("No TaskLocation available for task [%s], this task may not have been assigned to a worker yet", id); throw new NoTaskLocationException(String.format("No TaskLocation available for task [%s]", id)); } @@ -264,15 +482,7 @@ public class KafkaIndexTaskClient checkConnection(location.getHost(), location.getPort()); try { - URI serviceUri = new URI( - "http", - null, - location.getHost(), - location.getPort(), - String.format("%s/%s/%s", BASE_PATH, id, pathSuffix), - query, - null - ); + URI serviceUri = new URI("http", null, location.getHost(), location.getPort(), path, query, null); request = new Request(method, serviceUri.toURL()); // used to validate that we are talking to the correct worker @@ -282,7 +492,8 @@ public class KafkaIndexTaskClient request.setContent(MediaType.APPLICATION_JSON, content); } - response = httpClient.go(request, new FullResponseHandler(Charsets.UTF_8)).get(); + log.debug("HTTP %s: %s", method.getName(), serviceUri.toString()); + response = httpClient.go(request, new FullResponseHandler(Charsets.UTF_8), httpTimeout).get(); } catch (Exception e) { Throwables.propagateIfInstanceOf(e.getCause(), IOException.class); @@ -324,17 +535,26 @@ public class KafkaIndexTaskClient delay = retryPolicy.getAndIncrementRetryDelay(); } - if (!retry || delay == null) { + String urlForLog = (request != null + ? request.getUrl().toString() + : String.format("http://%s:%d%s", location.getHost(), location.getPort(), path)); + if (!retry) { + // if retry=false, we probably aren't too concerned if the operation doesn't succeed (i.e. the request was + // for informational purposes only) so don't log a scary stack trace + log.info("submitRequest failed for [%s], with message [%s]", urlForLog, e.getMessage()); + Throwables.propagate(e); + } else if (delay == null) { + log.warn(e, "Retries exhausted for [%s], last exception:", urlForLog); Throwables.propagate(e); } else { try { final long sleepTime = delay.getMillis(); log.debug( - "Bad response HTTP [%d] from %s; will try again in [%s] (body: [%s])", - (response != null ? response.getStatus().getCode() : 0), - (request != null ? request.getUrl() : "-"), + "Bad response HTTP [%s] from [%s]; will try again in [%s] (body/exception: [%s])", + (response != null ? response.getStatus().getCode() : "no response"), + urlForLog, new Duration(sleepTime).toString(), - (response != null ? response.getContent() : "[empty]") + (response != null ? response.getContent() : e.getMessage()) ); Thread.sleep(sleepTime); } @@ -343,6 +563,15 @@ public class KafkaIndexTaskClient } } } + catch (NoTaskLocationException e) { + log.info("No TaskLocation available for task [%s], this task may not have been assigned to a worker yet or " + + "may have already completed", id); + throw e; + } + catch (Exception e) { + log.warn(e, "Exception while sending request"); + throw e; + } } } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClientFactory.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClientFactory.java index 2f6ffcec4ad..ee602e50dbf 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClientFactory.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClientFactory.java @@ -25,6 +25,7 @@ import com.metamx.http.client.HttpClient; import io.druid.guice.annotations.Global; import io.druid.guice.annotations.Json; import io.druid.indexing.common.TaskInfoProvider; +import org.joda.time.Duration; public class KafkaIndexTaskClientFactory { @@ -38,8 +39,22 @@ public class KafkaIndexTaskClientFactory this.mapper = mapper; } - public KafkaIndexTaskClient build(TaskInfoProvider taskInfoProvider) + public KafkaIndexTaskClient build( + TaskInfoProvider taskInfoProvider, + String dataSource, + int numThreads, + Duration httpTimeout, + long numRetries + ) { - return new KafkaIndexTaskClient(httpClient, mapper, taskInfoProvider); + return new KafkaIndexTaskClient( + httpClient, + mapper, + taskInfoProvider, + dataSource, + numThreads, + httpTimeout, + numRetries + ); } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskModule.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskModule.java index f43c8b6861d..b0af2fdc8a5 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskModule.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskModule.java @@ -25,6 +25,7 @@ import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.common.collect.ImmutableList; import com.google.inject.Binder; import io.druid.indexing.kafka.supervisor.KafkaSupervisorSpec; +import io.druid.indexing.kafka.supervisor.KafkaSupervisorTuningConfig; import io.druid.initialization.DruidModule; import java.util.List; @@ -40,7 +41,7 @@ public class KafkaIndexTaskModule implements DruidModule new NamedType(KafkaIndexTask.class, "index_kafka"), new NamedType(KafkaDataSourceMetadata.class, "kafka"), new NamedType(KafkaIOConfig.class, "kafka"), - new NamedType(KafkaTuningConfig.class, "kafka"), + new NamedType(KafkaSupervisorTuningConfig.class, "kafka"), new NamedType(KafkaSupervisorSpec.class, "kafka") ) ); diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java index 8a60822fa0f..1b16158cc13 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java @@ -76,6 +76,21 @@ public class KafkaTuningConfig implements TuningConfig, AppenderatorConfig : handoffConditionTimeout; } + public static KafkaTuningConfig copyOf(KafkaTuningConfig config) + { + return new KafkaTuningConfig( + config.maxRowsInMemory, + config.maxRowsPerSegment, + config.intermediatePersistPeriod, + config.basePersistDirectory, + config.maxPendingPersists, + config.indexSpec, + config.buildV9Directly, + config.reportParseExceptions, + config.handoffConditionTimeout + ); + } + @JsonProperty public int getMaxRowsInMemory() { @@ -159,4 +174,78 @@ public class KafkaTuningConfig implements TuningConfig, AppenderatorConfig handoffConditionTimeout ); } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + KafkaTuningConfig that = (KafkaTuningConfig) o; + + if (maxRowsInMemory != that.maxRowsInMemory) { + return false; + } + if (maxRowsPerSegment != that.maxRowsPerSegment) { + return false; + } + if (maxPendingPersists != that.maxPendingPersists) { + return false; + } + if (buildV9Directly != that.buildV9Directly) { + return false; + } + if (reportParseExceptions != that.reportParseExceptions) { + return false; + } + if (handoffConditionTimeout != that.handoffConditionTimeout) { + return false; + } + if (intermediatePersistPeriod != null + ? !intermediatePersistPeriod.equals(that.intermediatePersistPeriod) + : that.intermediatePersistPeriod != null) { + return false; + } + if (basePersistDirectory != null + ? !basePersistDirectory.equals(that.basePersistDirectory) + : that.basePersistDirectory != null) { + return false; + } + return !(indexSpec != null ? !indexSpec.equals(that.indexSpec) : that.indexSpec != null); + } + + @Override + public int hashCode() + { + int result = maxRowsInMemory; + result = 31 * result + maxRowsPerSegment; + result = 31 * result + (intermediatePersistPeriod != null ? intermediatePersistPeriod.hashCode() : 0); + result = 31 * result + (basePersistDirectory != null ? basePersistDirectory.hashCode() : 0); + result = 31 * result + maxPendingPersists; + result = 31 * result + (indexSpec != null ? indexSpec.hashCode() : 0); + result = 31 * result + (buildV9Directly ? 1 : 0); + result = 31 * result + (reportParseExceptions ? 1 : 0); + result = 31 * result + (int) (handoffConditionTimeout ^ (handoffConditionTimeout >>> 32)); + return result; + } + + @Override + public String toString() + { + return "KafkaTuningConfig{" + + "maxRowsInMemory=" + maxRowsInMemory + + ", maxRowsPerSegment=" + maxRowsPerSegment + + ", intermediatePersistPeriod=" + intermediatePersistPeriod + + ", basePersistDirectory=" + basePersistDirectory + + ", maxPendingPersists=" + maxPendingPersists + + ", indexSpec=" + indexSpec + + ", buildV9Directly=" + buildV9Directly + + ", reportParseExceptions=" + reportParseExceptions + + ", handoffConditionTimeout=" + handoffConditionTimeout + + '}'; + } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java index 0f4791c708d..b61b6834984 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -23,16 +23,21 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.MapperFeature; 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.Throwables; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.primitives.Ints; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import com.metamx.common.ISE; import com.metamx.emitter.EmittingLogger; @@ -48,6 +53,7 @@ import io.druid.indexing.kafka.KafkaIndexTask; import io.druid.indexing.kafka.KafkaIndexTaskClient; import io.druid.indexing.kafka.KafkaIndexTaskClientFactory; import io.druid.indexing.kafka.KafkaPartitions; +import io.druid.indexing.kafka.KafkaTuningConfig; import io.druid.indexing.overlord.DataSourceMetadata; import io.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import io.druid.indexing.overlord.TaskMaster; @@ -66,6 +72,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.joda.time.DateTime; +import javax.annotation.Nullable; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -73,6 +80,9 @@ import java.util.Map; import java.util.Properties; import java.util.Random; import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.ScheduledExecutorService; @@ -91,7 +101,8 @@ public class KafkaSupervisor implements Supervisor private static final EmittingLogger log = new EmittingLogger(KafkaSupervisor.class); private static final Random RANDOM = new Random(); private static final long MAX_RUN_FREQUENCY_MILLIS = 1000; // prevent us from running too often in response to events - private static final int SHUTDOWN_TIMEOUT_MILLIS = 30000; + private static final int SHUTDOWN_TIMEOUT_MILLIS = 15000; + private static final long NOT_SET = -1; // Internal data structures // -------------------------------------------------------- @@ -111,13 +122,13 @@ public class KafkaSupervisor implements Supervisor // this task group has completed successfully, at which point this will be destroyed and a new task group will be // created with new starting offsets. This allows us to create replacement tasks for failed tasks that process the // same offsets, even if the values in [partitionGroups] has been changed. - final Map partitionOffsets; + final ImmutableMap partitionOffsets; - final Map tasks = new HashMap<>(); + final ConcurrentHashMap tasks = new ConcurrentHashMap<>(); final Optional minimumMessageTime; DateTime completionTimeout; // is set after signalTasksToFinish(); if not done by timeout, take corrective action - public TaskGroup(Map partitionOffsets, Optional minimumMessageTime) + public TaskGroup(ImmutableMap partitionOffsets, Optional minimumMessageTime) { this.partitionOffsets = partitionOffsets; this.minimumMessageTime = minimumMessageTime; @@ -131,25 +142,25 @@ public class KafkaSupervisor implements Supervisor } // Map<{group ID}, {actively reading task group}>; see documentation for TaskGroup class - private final HashMap taskGroups = new HashMap<>(); + private final ConcurrentHashMap taskGroups = new ConcurrentHashMap<>(); // After telling a taskGroup to stop reading and begin publishing a segment, it is moved from [taskGroups] to here so // we can monitor its status while we queue new tasks to read the next range of offsets. This is a list since we could // have multiple sets of tasks publishing at once if time-to-publish > taskDuration. // Map<{group ID}, List<{pending completion task groups}>> - private final HashMap> pendingCompletionTaskGroups = new HashMap<>(); + private final ConcurrentHashMap> pendingCompletionTaskGroups = new ConcurrentHashMap<>(); - // The starting offset for a new partition in [partitionGroups] is initially set to null. When a new task group - // is created and is assigned partitions, if the offset in [partitionGroups] is null it will take the starting + // The starting offset for a new partition in [partitionGroups] is initially set to NOT_SET. When a new task group + // is created and is assigned partitions, if the offset in [partitionGroups] is NOT_SET it will take the starting // offset value from the metadata store, and if it can't find it there, from Kafka. Once a task begins // publishing, the offset in partitionGroups will be updated to the ending offset of the publishing-but-not-yet- // completed task, which will cause the next set of tasks to begin reading from where the previous task left - // off. If that previous task now fails, we will set the offset in [partitionGroups] back to null which will + // off. If that previous task now fails, we will set the offset in [partitionGroups] back to NOT_SET which will // cause successive tasks to again grab their starting offset from metadata store. This mechanism allows us to // start up successive tasks without waiting for the previous tasks to succeed and still be able to handle task // failures during publishing. // Map<{group ID}, Map<{partition ID}, {startingOffset}>> - private Map> partitionGroups = new HashMap<>(); + private final ConcurrentHashMap> partitionGroups = new ConcurrentHashMap<>(); // -------------------------------------------------------- private final TaskStorage taskStorage; @@ -160,11 +171,14 @@ public class KafkaSupervisor implements Supervisor private final KafkaSupervisorSpec spec; private final String dataSource; private final KafkaSupervisorIOConfig ioConfig; + private final KafkaSupervisorTuningConfig tuningConfig; + private final KafkaTuningConfig taskTuningConfig; private final String supervisorId; private final TaskInfoProvider taskInfoProvider; private final ExecutorService exec; private final ScheduledExecutorService scheduledExec; + private final ListeningExecutorService workerExec; private final BlockingQueue notices = new LinkedBlockingDeque<>(); private final Object stopLock = new Object(); private final Object stateChangeLock = new Object(); @@ -194,10 +208,18 @@ public class KafkaSupervisor implements Supervisor this.dataSource = spec.getDataSchema().getDataSource(); this.ioConfig = spec.getIoConfig(); + this.tuningConfig = spec.getTuningConfig(); + this.taskTuningConfig = KafkaTuningConfig.copyOf(this.tuningConfig); this.supervisorId = String.format("KafkaSupervisor-%s", dataSource); - this.exec = Execs.singleThreaded(supervisorId + "-%d"); + this.exec = Execs.singleThreaded(supervisorId); this.scheduledExec = Execs.scheduledSingleThreaded(supervisorId + "-Scheduler-%d"); + int workerThreads = (this.tuningConfig.getWorkerThreads() != null + ? this.tuningConfig.getWorkerThreads() + : Math.min(10, this.ioConfig.getTaskCount())); + this.workerExec = MoreExecutors.listeningDecorator(Execs.multiThreaded(workerThreads, supervisorId + "-Worker-%d")); + log.info("Created worker pool with [%d] threads for dataSource [%s]", workerThreads, this.dataSource); + this.taskInfoProvider = new TaskInfoProvider() { @Override @@ -234,7 +256,23 @@ public class KafkaSupervisor implements Supervisor } }; - this.taskClient = taskClientFactory.build(taskInfoProvider); + int chatThreads = (this.tuningConfig.getChatThreads() != null + ? this.tuningConfig.getChatThreads() + : Math.min(10, this.ioConfig.getTaskCount() * this.ioConfig.getReplicas())); + this.taskClient = taskClientFactory.build( + taskInfoProvider, + dataSource, + chatThreads, + this.tuningConfig.getHttpTimeout(), + this.tuningConfig.getChatRetries() + ); + log.info( + "Created taskClient with dataSource[%s] chatThreads[%d] httpTimeout[%s] chatRetries[%d]", + dataSource, + chatThreads, + this.tuningConfig.getHttpTimeout(), + this.tuningConfig.getChatRetries() + ); } @Override @@ -289,7 +327,12 @@ public class KafkaSupervisor implements Supervisor ); started = true; - log.info("Started KafkaSupervisor[%s], first run in [%s]", dataSource, ioConfig.getStartDelay()); + log.info( + "Started KafkaSupervisor[%s], first run in [%s], with spec: [%s]", + dataSource, + ioConfig.getStartDelay(), + spec.toString() + ); } } @@ -335,6 +378,8 @@ public class KafkaSupervisor implements Supervisor } log.info("Shutdown notice handled"); + taskClient.close(); + workerExec.shutdownNow(); exec.shutdownNow(); started = false; @@ -392,13 +437,13 @@ public class KafkaSupervisor implements Supervisor private interface Notice { - void handle(); + void handle() throws ExecutionException, InterruptedException; } private class RunNotice implements Notice { @Override - public void handle() + public void handle() throws ExecutionException, InterruptedException { long nowTime = System.currentTimeMillis(); if (nowTime - lastRunTime < MAX_RUN_FREQUENCY_MILLIS) { @@ -413,23 +458,9 @@ public class KafkaSupervisor implements Supervisor private class GracefulShutdownNotice extends ShutdownNotice { @Override - public void handle() + public void handle() throws InterruptedException, ExecutionException { - // Prepare for shutdown by 1) killing all tasks that haven't been assigned to a worker yet, and 2) causing all - // running tasks to begin publishing by setting their startTime to a very long time ago so that the logic in - // checkTaskDuration() will be triggered. This is better than just telling these tasks to publish whatever they - // have, as replicas that are supposed to publish the same segment may not have read the same set of offsets. - for (TaskGroup taskGroup : taskGroups.values()) { - for (Map.Entry entry : taskGroup.tasks.entrySet()) { - if (taskInfoProvider.getTaskLocation(entry.getKey()).equals(TaskLocation.unknown())) { - killTask(entry.getKey()); - } else { - entry.getValue().startTime = new DateTime(0); - } - } - } - - checkTaskDuration(); + gracefulShutdownInternal(); super.handle(); } } @@ -437,7 +468,7 @@ public class KafkaSupervisor implements Supervisor private class ShutdownNotice implements Notice { @Override - public void handle() + public void handle() throws InterruptedException, ExecutionException { consumer.close(); @@ -449,7 +480,27 @@ public class KafkaSupervisor implements Supervisor } @VisibleForTesting - void runInternal() + void gracefulShutdownInternal() throws ExecutionException, InterruptedException + { + // Prepare for shutdown by 1) killing all tasks that haven't been assigned to a worker yet, and 2) causing all + // running tasks to begin publishing by setting their startTime to a very long time ago so that the logic in + // checkTaskDuration() will be triggered. This is better than just telling these tasks to publish whatever they + // have, as replicas that are supposed to publish the same segment may not have read the same set of offsets. + for (TaskGroup taskGroup : taskGroups.values()) { + for (Map.Entry entry : taskGroup.tasks.entrySet()) { + if (taskInfoProvider.getTaskLocation(entry.getKey()).equals(TaskLocation.unknown())) { + killTask(entry.getKey()); + } else { + entry.getValue().startTime = new DateTime(0); + } + } + } + + checkTaskDuration(); + } + + @VisibleForTesting + void runInternal() throws ExecutionException, InterruptedException { possiblyRegisterListener(); updatePartitionDataFromKafka(); @@ -484,7 +535,7 @@ public class KafkaSupervisor implements Supervisor String dataSchema, tuningConfig; try { dataSchema = sortingMapper.writeValueAsString(spec.getDataSchema()); - tuningConfig = sortingMapper.writeValueAsString(spec.getTuningConfig()); + tuningConfig = sortingMapper.writeValueAsString(taskTuningConfig); } catch (JsonProcessingException e) { throw Throwables.propagate(e); @@ -547,118 +598,166 @@ public class KafkaSupervisor implements Supervisor for (int partition = 0; partition < numPartitions; partition++) { int taskGroupId = getTaskGroupIdForPartition(partition); - if (partitionGroups.get(taskGroupId) == null) { - partitionGroups.put(taskGroupId, new HashMap()); - } + partitionGroups.putIfAbsent(taskGroupId, new ConcurrentHashMap()); - Map partitionMap = partitionGroups.get(taskGroupId); - if (!partitionMap.containsKey(partition)) { + ConcurrentHashMap partitionMap = partitionGroups.get(taskGroupId); + + // The starting offset for a new partition in [partitionGroups] is initially set to NOT_SET; when a new task group + // is created and is assigned partitions, if the offset in [partitionGroups] is NOT_SET it will take the starting + // offset value from the metadata store, and if it can't find it there, from Kafka. Once a task begins + // publishing, the offset in partitionGroups will be updated to the ending offset of the publishing-but-not-yet- + // completed task, which will cause the next set of tasks to begin reading from where the previous task left + // off. If that previous task now fails, we will set the offset in [partitionGroups] back to NOT_SET which will + // cause successive tasks to again grab their starting offset from metadata store. This mechanism allows us to + // start up successive tasks without waiting for the previous tasks to succeed and still be able to handle task + // failures during publishing. + if (partitionMap.putIfAbsent(partition, NOT_SET) == null) { log.info( - "New partition [%d] discovered for topic [%s], adding to task group [%d]", + "New partition [%d] discovered for topic [%s], added to task group [%d]", partition, ioConfig.getTopic(), taskGroupId ); - - // The starting offset for a new partition in [partitionGroups] is initially set to null; when a new task group - // is created and is assigned partitions, if the offset in [partitionGroups] is null it will take the starting - // offset value from the metadata store, and if it can't find it there, from Kafka. Once a task begins - // publishing, the offset in partitionGroups will be updated to the ending offset of the publishing-but-not-yet- - // completed task, which will cause the next set of tasks to begin reading from where the previous task left - // off. If that previous task now fails, we will set the offset in [partitionGroups] back to null which will - // cause successive tasks to again grab their starting offset from metadata store. This mechanism allows us to - // start up successive tasks without waiting for the previous tasks to succeed and still be able to handle task - // failures during publishing. - partitionMap.put(partition, null); } } } - private void discoverTasks() + private void discoverTasks() throws ExecutionException, InterruptedException { int taskCount = 0; + List futureTaskIds = Lists.newArrayList(); + List> futures = Lists.newArrayList(); List tasks = taskStorage.getActiveTasks(); for (Task task : tasks) { - if (task instanceof KafkaIndexTask && dataSource.equals(task.getDataSource())) { - taskCount++; - KafkaIndexTask kafkaTask = (KafkaIndexTask) task; - String taskId = task.getId(); + if (!(task instanceof KafkaIndexTask) || !dataSource.equals(task.getDataSource())) { + continue; + } - // Determine which task group this task belongs to based on one of the partitions handled by this task. If we - // later determine that this task is actively reading, we will make sure that it matches our current partition - // allocation (getTaskGroupIdForPartition(partition) should return the same value for every partition being read - // by this task) and kill it if it is not compatible. If the task is instead found to be in the publishing - // state, we will permit it to complete even if it doesn't match our current partition allocation to support - // seamless schema migration. + taskCount++; + final KafkaIndexTask kafkaTask = (KafkaIndexTask) task; + final String taskId = task.getId(); - Iterator it = kafkaTask.getIOConfig().getStartPartitions().getPartitionOffsetMap().keySet().iterator(); - Integer taskGroupId = (it.hasNext() ? getTaskGroupIdForPartition(it.next()) : null); + // Determine which task group this task belongs to based on one of the partitions handled by this task. If we + // later determine that this task is actively reading, we will make sure that it matches our current partition + // allocation (getTaskGroupIdForPartition(partition) should return the same value for every partition being read + // by this task) and kill it if it is not compatible. If the task is instead found to be in the publishing + // state, we will permit it to complete even if it doesn't match our current partition allocation to support + // seamless schema migration. - if (taskGroupId != null) { - // check to see if we already know about this task, either in [taskGroups] or in [pendingCompletionTaskGroups] - // and if not add it to taskGroups or pendingCompletionTaskGroups (if status = PUBLISHING) - TaskGroup taskGroup = taskGroups.get(taskGroupId); - if (!isTaskInPendingCompletionGroups(taskId) && (taskGroup == null || !taskGroup.tasks.containsKey(taskId))) { - Optional status = getTaskStatus(taskId); - if (status.isPresent() && status.get() == KafkaIndexTask.Status.PUBLISHING) { - addDiscoveredTaskToPendingCompletionTaskGroups( - taskGroupId, - taskId, - kafkaTask.getIOConfig() - .getStartPartitions() - .getPartitionOffsetMap() - ); + Iterator it = kafkaTask.getIOConfig().getStartPartitions().getPartitionOffsetMap().keySet().iterator(); + final Integer taskGroupId = (it.hasNext() ? getTaskGroupIdForPartition(it.next()) : null); - // update partitionGroups with the publishing task's offsets (if they are greater than what is existing) - // so that the next tasks will start reading from where this task left off - Map publishingTaskCurrentOffsets = getCurrentOffsets(taskId, true); - for (Map.Entry entry : publishingTaskCurrentOffsets.entrySet()) { - Integer partition = entry.getKey(); - Long offset = entry.getValue(); - Map partitionOffsets = partitionGroups.get(getTaskGroupIdForPartition(partition)); - if (partitionOffsets.get(partition) == null || partitionOffsets.get(partition) < offset) { - partitionOffsets.put(partition, offset); - } - } + if (taskGroupId != null) { + // check to see if we already know about this task, either in [taskGroups] or in [pendingCompletionTaskGroups] + // and if not add it to taskGroups or pendingCompletionTaskGroups (if status = PUBLISHING) + TaskGroup taskGroup = taskGroups.get(taskGroupId); + if (!isTaskInPendingCompletionGroups(taskId) && (taskGroup == null || !taskGroup.tasks.containsKey(taskId))) { - } else { - for (Integer partition : kafkaTask.getIOConfig().getStartPartitions().getPartitionOffsetMap().keySet()) { - if (!taskGroupId.equals(getTaskGroupIdForPartition(partition))) { - log.warn("Stopping task [%s] which does not match the expected partition allocation", taskId); - stopTask(taskId, false); - taskGroupId = null; - break; - } - } + futureTaskIds.add(taskId); + futures.add( + Futures.transform( + taskClient.getStatusAsync(taskId), new Function() + { + @Override + public Boolean apply(KafkaIndexTask.Status status) + { + if (status == KafkaIndexTask.Status.PUBLISHING) { + addDiscoveredTaskToPendingCompletionTaskGroups( + taskGroupId, + taskId, + kafkaTask.getIOConfig() + .getStartPartitions() + .getPartitionOffsetMap() + ); - if (taskGroupId == null) { - continue; - } + // update partitionGroups with the publishing task's offsets (if they are greater than what is + // existing) so that the next tasks will start reading from where this task left off + Map publishingTaskCurrentOffsets = taskClient.getCurrentOffsets(taskId, true); - if (!taskGroups.containsKey(taskGroupId)) { - log.debug("Creating new task group [%d]", taskGroupId); - taskGroups.put( - taskGroupId, - new TaskGroup( - kafkaTask.getIOConfig().getStartPartitions().getPartitionOffsetMap(), - kafkaTask.getIOConfig().getMinimumMessageTime() - ) - ); - } + for (Map.Entry entry : publishingTaskCurrentOffsets.entrySet()) { + Integer partition = entry.getKey(); + Long offset = entry.getValue(); + ConcurrentHashMap partitionOffsets = partitionGroups.get( + getTaskGroupIdForPartition(partition) + ); - if (!isTaskCurrent(taskGroupId, taskId)) { - log.info("Stopping task [%s] which does not match the expected parameters and ingestion spec", taskId); - stopTask(taskId, false); - } else { - taskGroups.get(taskGroupId).tasks.put(taskId, new TaskData()); - } - } - } + boolean succeeded; + do { + succeeded = true; + Long previousOffset = partitionOffsets.putIfAbsent(partition, offset); + if (previousOffset != null && previousOffset < offset) { + succeeded = partitionOffsets.replace(partition, previousOffset, offset); + } + } while (!succeeded); + } + + } else { + for (Integer partition : kafkaTask.getIOConfig() + .getStartPartitions() + .getPartitionOffsetMap() + .keySet()) { + if (!taskGroupId.equals(getTaskGroupIdForPartition(partition))) { + log.warn( + "Stopping task [%s] which does not match the expected partition allocation", + taskId + ); + try { + stopTask(taskId, false).get(); + } + catch (InterruptedException | ExecutionException e) { + log.warn(e, "Exception while stopping task"); + } + return false; + } + } + + if (taskGroups.putIfAbsent( + taskGroupId, + new TaskGroup( + ImmutableMap.copyOf( + kafkaTask.getIOConfig() + .getStartPartitions() + .getPartitionOffsetMap() + ), kafkaTask.getIOConfig().getMinimumMessageTime() + ) + ) == null) { + log.debug("Created new task group [%d]", taskGroupId); + } + + if (!isTaskCurrent(taskGroupId, taskId)) { + log.info( + "Stopping task [%s] which does not match the expected parameters and ingestion spec", + taskId + ); + try { + stopTask(taskId, false).get(); + } + catch (InterruptedException | ExecutionException e) { + log.warn(e, "Exception while stopping task"); + } + return false; + } else { + taskGroups.get(taskGroupId).tasks.putIfAbsent(taskId, new TaskData()); + } + } + return true; + } + }, workerExec + ) + ); } } } + List results = Futures.successfulAsList(futures).get(); + for (int i = 0; i < results.size(); i++) { + if (results.get(i) == null) { + String taskId = futureTaskIds.get(i); + log.warn("Task [%s] failed to return status, killing task", taskId); + killTask(taskId); + } + } log.debug("Found [%d] Kafka indexing tasks for dataSource [%s]", taskCount, dataSource); } @@ -668,16 +767,13 @@ public class KafkaSupervisor implements Supervisor Map startingPartitions ) { - if (!pendingCompletionTaskGroups.containsKey(groupId)) { - pendingCompletionTaskGroups.put(groupId, Lists.newArrayList()); - } + pendingCompletionTaskGroups.putIfAbsent(groupId, Lists.newCopyOnWriteArrayList()); - List taskGroupList = pendingCompletionTaskGroups.get(groupId); + CopyOnWriteArrayList taskGroupList = pendingCompletionTaskGroups.get(groupId); for (TaskGroup taskGroup : taskGroupList) { if (taskGroup.partitionOffsets.equals(startingPartitions)) { - if (!taskGroup.tasks.containsKey(taskId)) { - log.info("Adding discovered task [%s] to existing pending task group", taskId); - taskGroup.tasks.put(taskId, new TaskData()); + if (taskGroup.tasks.putIfAbsent(taskId, new TaskData()) == null) { + log.info("Added discovered task [%s] to existing pending task group", taskId); } return; } @@ -687,7 +783,7 @@ public class KafkaSupervisor implements Supervisor // reading the minimumMessageTime from the publishing task and setting it here is not necessary as this task cannot // change to a state where it will read any more events - TaskGroup newTaskGroup = new TaskGroup(startingPartitions, Optional.absent()); + TaskGroup newTaskGroup = new TaskGroup(ImmutableMap.copyOf(startingPartitions), Optional.absent()); newTaskGroup.tasks.put(taskId, new TaskData()); newTaskGroup.completionTimeout = DateTime.now().plus(ioConfig.getCompletionTimeout()); @@ -695,33 +791,47 @@ public class KafkaSupervisor implements Supervisor taskGroupList.add(newTaskGroup); } - private void updateTaskStatus() + private void updateTaskStatus() throws ExecutionException, InterruptedException { + final List> futures = Lists.newArrayList(); + final List futureTaskIds = Lists.newArrayList(); + // update status (and startTime if unknown) of current tasks in taskGroups for (TaskGroup group : taskGroups.values()) { for (Map.Entry entry : group.tasks.entrySet()) { - String taskId = entry.getKey(); - TaskData taskData = entry.getValue(); + final String taskId = entry.getKey(); + final TaskData taskData = entry.getValue(); if (taskData.startTime == null) { - try { - taskData.startTime = getTaskStartTime(taskId); - if (taskData.startTime != null) { - long millisRemaining = ioConfig.getTaskDuration().getMillis() - (System.currentTimeMillis() - - taskData.startTime.getMillis()); - if (millisRemaining > 0) { - scheduledExec.schedule( - buildRunTask(), - millisRemaining + MAX_RUN_FREQUENCY_MILLIS, - TimeUnit.MILLISECONDS - ); - } - } - } - catch (Exception e) { - log.warn(e, "Task [%s] failed to return start time, killing task", taskId); - killTask(taskId); - } + futureTaskIds.add(taskId); + futures.add( + Futures.transform( + taskClient.getStartTimeAsync(taskId), new Function() + { + @Nullable + @Override + public Boolean apply(@Nullable DateTime startTime) + { + if (startTime == null) { + return false; + } + + taskData.startTime = startTime; + long millisRemaining = ioConfig.getTaskDuration().getMillis() - (System.currentTimeMillis() + - taskData.startTime.getMillis()); + if (millisRemaining > 0) { + scheduledExec.schedule( + buildRunTask(), + millisRemaining + MAX_RUN_FREQUENCY_MILLIS, + TimeUnit.MILLISECONDS + ); + } + + return true; + } + }, workerExec + ) + ); } taskData.status = taskStorage.getStatus(taskId).get(); @@ -736,15 +846,27 @@ public class KafkaSupervisor implements Supervisor } } } + + List results = Futures.successfulAsList(futures).get(); + for (int i = 0; i < results.size(); i++) { + // false means the task hasn't started running yet and that's okay; null means it should be running but the HTTP + // request threw an exception so kill the task + if (results.get(i) == null) { + String taskId = futureTaskIds.get(i); + log.warn("Task [%s] failed to return start time, killing task", taskId); + killTask(taskId); + } + } } - private void checkTaskDuration() + private void checkTaskDuration() throws InterruptedException, ExecutionException { - Iterator> i = taskGroups.entrySet().iterator(); - while (i.hasNext()) { - Map.Entry groupEntry = i.next(); - Integer groupId = groupEntry.getKey(); - TaskGroup group = groupEntry.getValue(); + final List>> futures = Lists.newArrayList(); + final List futureGroupIds = Lists.newArrayList(); + + for (Map.Entry entry : taskGroups.entrySet()) { + Integer groupId = entry.getKey(); + TaskGroup group = entry.getValue(); // find the longest running task from this group DateTime earliestTaskStart = DateTime.now(); @@ -757,34 +879,48 @@ public class KafkaSupervisor implements Supervisor // if this task has run longer than the configured duration, signal all tasks in the group to persist if (earliestTaskStart.plus(ioConfig.getTaskDuration()).isBeforeNow()) { log.info("Task group [%d] has run for [%s]", groupId, ioConfig.getTaskDuration()); - Map endOffsets = signalTasksToFinish(groupId); + futureGroupIds.add(groupId); + futures.add(signalTasksToFinish(groupId)); + } + } + List> results = Futures.successfulAsList(futures).get(); + for (int j = 0; j < results.size(); j++) { + Integer groupId = futureGroupIds.get(j); + TaskGroup group = taskGroups.get(groupId); + Map endOffsets = results.get(j); + + if (endOffsets != null) { // set a timeout and put this group in pendingCompletionTaskGroups so that it can be monitored for completion group.completionTimeout = DateTime.now().plus(ioConfig.getCompletionTimeout()); - if (!pendingCompletionTaskGroups.containsKey(groupId)) { - pendingCompletionTaskGroups.put(groupId, Lists.newArrayList()); - } + pendingCompletionTaskGroups.putIfAbsent(groupId, Lists.newCopyOnWriteArrayList()); pendingCompletionTaskGroups.get(groupId).add(group); - // if we know what the endOffsets are going to be from talking to the tasks, set them as the next startOffsets - if (endOffsets != null) { - for (Map.Entry entry : endOffsets.entrySet()) { - partitionGroups.get(groupId).put(entry.getKey(), entry.getValue()); - } + // set endOffsets as the next startOffsets + for (Map.Entry entry : endOffsets.entrySet()) { + partitionGroups.get(groupId).put(entry.getKey(), entry.getValue()); + } + } else { + log.warn( + "All tasks in group [%s] failed to transition to publishing state, killing tasks [%s]", + groupId, + group.tasks.keySet() + ); + for (String id : group.tasks.keySet()) { + killTask(id); } - - // remove this task group from the list of current task groups now that it has been handled - i.remove(); } + + // remove this task group from the list of current task groups now that it has been handled + taskGroups.remove(groupId); } } - private Map signalTasksToFinish(int groupId) + private ListenableFuture> signalTasksToFinish(final int groupId) { - TaskGroup taskGroup = taskGroups.get(groupId); + final TaskGroup taskGroup = taskGroups.get(groupId); - // 1) Pause running tasks and build a map of the highest offset read by any task in the group for each partition - Map endOffsets = new HashMap<>(); + // 1) Check if any task completed (in which case we're done) and kill unassigned tasks Iterator> i = taskGroup.tasks.entrySet().iterator(); while (i.hasNext()) { Map.Entry taskEntry = i.next(); @@ -796,8 +932,17 @@ public class KafkaSupervisor implements Supervisor // This will cause us to create a new set of tasks next cycle that will start from the offsets in // metadata store (which will have advanced if we succeeded in publishing and will remain the same if publishing // failed and we need to re-ingest) - stopTasksInGroup(taskGroup); - return null; + return Futures.transform( + stopTasksInGroup(taskGroup), new Function>() + { + @Nullable + @Override + public Map apply(@Nullable Void input) + { + return null; + } + }, workerExec + ); } if (task.status.isRunnable()) { @@ -805,52 +950,84 @@ public class KafkaSupervisor implements Supervisor log.info("Killing task [%s] which hasn't been assigned to a worker", taskId); killTask(taskId); i.remove(); - } else { - Map currentOffsets; - try { - currentOffsets = taskClient.pause(taskId); // pause task and get offsets - } - catch (Exception e) { - log.warn(e, "Task [%s] failed to respond to [pause] in a timely manner, killing task", taskId); - killTask(taskId); - i.remove(); - continue; - } + } + } + } - for (Map.Entry offset : currentOffsets.entrySet()) { - if (!endOffsets.containsKey(offset.getKey()) - || endOffsets.get(offset.getKey()).compareTo(offset.getValue()) < 0) { - endOffsets.put(offset.getKey(), offset.getValue()); + // 2) Pause running tasks + final List>> pauseFutures = Lists.newArrayList(); + final List pauseTaskIds = ImmutableList.copyOf(taskGroup.tasks.keySet()); + for (final String taskId : pauseTaskIds) { + pauseFutures.add(taskClient.pauseAsync(taskId)); + } + + return Futures.transform( + Futures.successfulAsList(pauseFutures), new Function>, Map>() + { + @Nullable + @Override + public Map apply(List> input) + { + // 3) Build a map of the highest offset read by any task in the group for each partition + final Map endOffsets = new HashMap<>(); + for (int i = 0; i < input.size(); i++) { + Map result = input.get(i); + + if (result == null || result.isEmpty()) { // kill tasks that didn't return a value + String taskId = pauseTaskIds.get(i); + log.warn("Task [%s] failed to respond to [pause] in a timely manner, killing task", taskId); + killTask(taskId); + taskGroup.tasks.remove(taskId); + + } else { // otherwise build a map of the highest offsets seen + for (Map.Entry offset : result.entrySet()) { + if (!endOffsets.containsKey(offset.getKey()) + || endOffsets.get(offset.getKey()).compareTo(offset.getValue()) < 0) { + endOffsets.put(offset.getKey(), offset.getValue()); + } + } + } } + + // 4) Set the end offsets for each task to the values from step 3 and resume the tasks. All the tasks should + // finish reading and start publishing within a short period, depending on how in sync the tasks were. + final List> setEndOffsetFutures = Lists.newArrayList(); + final List setEndOffsetTaskIds = ImmutableList.copyOf(taskGroup.tasks.keySet()); + + if (setEndOffsetTaskIds.isEmpty()) { + log.info("All tasks in taskGroup [%d] have failed, tasks will be re-created", groupId); + return null; + } + + log.info("Setting endOffsets for tasks in taskGroup [%d] to %s and resuming", groupId, endOffsets); + for (final String taskId : setEndOffsetTaskIds) { + setEndOffsetFutures.add(taskClient.setEndOffsetsAsync(taskId, endOffsets, true)); + } + + try { + List results = Futures.successfulAsList(setEndOffsetFutures).get(); + for (int i = 0; i < results.size(); i++) { + if (results.get(i) == null || !results.get(i)) { + String taskId = setEndOffsetTaskIds.get(i); + log.warn("Task [%s] failed to respond to [set end offsets] in a timely manner, killing task", taskId); + killTask(taskId); + taskGroup.tasks.remove(taskId); + } + } + } + catch (Exception e) { + Throwables.propagate(e); + } + + if (taskGroup.tasks.isEmpty()) { + log.info("All tasks in taskGroup [%d] have failed, tasks will be re-created", groupId); + return null; + } + + return endOffsets; } - } - } - } - - // 2) Set the end offsets for each task to the values from step 1 and resume the tasks. All the tasks should - // finish reading and start publishing within a short period of time, depending on how in sync the tasks were. - log.info("Setting endOffsets for tasks in taskGroup [%d] to %s and resuming", groupId, endOffsets); - i = taskGroup.tasks.entrySet().iterator(); - while (i.hasNext()) { - Map.Entry taskEntry = i.next(); - String taskId = taskEntry.getKey(); - TaskData task = taskEntry.getValue(); - - if (task.status.isRunnable()) { - try { - taskClient.setEndOffsets(taskId, endOffsets, true); - } - catch (Exception e) { - log.warn(e, "Task [%s] failed to respond to [set end offsets] in a timely manner, killing task", taskId); - killTask(taskId); - i.remove(); - } - } - } - - // 3) Return the ending offsets so we can start the next set of tasks from where these tasks ended while the current - // set of tasks are publishing. - return endOffsets; + }, workerExec + ); } /** @@ -861,22 +1038,25 @@ public class KafkaSupervisor implements Supervisor * starting offset for subsequent task groups is no longer valid, and subsequent tasks would fail as soon as they * attempted to publish because of the contiguous range consistency check. */ - private void checkPendingCompletionTasks() + private void checkPendingCompletionTasks() throws ExecutionException, InterruptedException { - for (Map.Entry> pendingGroupList : pendingCompletionTaskGroups.entrySet()) { + List> futures = Lists.newArrayList(); + + for (Map.Entry> pendingGroupList : pendingCompletionTaskGroups.entrySet()) { boolean stopTasksInTaskGroup = false; Integer groupId = pendingGroupList.getKey(); - Iterator iTaskGroup = pendingGroupList.getValue().iterator(); - while (iTaskGroup.hasNext()) { + CopyOnWriteArrayList taskGroupList = pendingGroupList.getValue(); + List toRemove = Lists.newArrayList(); + + for (TaskGroup group : taskGroupList) { boolean foundSuccess = false, entireTaskGroupFailed = false; - TaskGroup group = iTaskGroup.next(); if (stopTasksInTaskGroup) { // One of the earlier groups that was handling the same partition set timed out before the segments were // published so stop any additional groups handling the same partition set that are pending completion. - stopTasksInGroup(group); - iTaskGroup.remove(); + futures.add(stopTasksInGroup(group)); + toRemove.add(group); continue; } @@ -897,9 +1077,9 @@ public class KafkaSupervisor implements Supervisor // If one of the pending completion tasks was successful, stop the rest of the tasks in the group as // we no longer need them to publish their segment. log.info("Task [%s] completed successfully, stopping tasks %s", task.getKey(), group.tasks.keySet()); - stopTasksInGroup(group); + futures.add(stopTasksInGroup(group)); foundSuccess = true; - iTaskGroup.remove(); // remove the TaskGroup from the list of pending completion task groups + toRemove.add(group); // remove the TaskGroup from the list of pending completion task groups break; // skip iterating the rest of the tasks in this group as they've all been stopped now } } @@ -919,22 +1099,27 @@ public class KafkaSupervisor implements Supervisor partitionGroups.remove(groupId); // stop all the tasks in this pending completion group - stopTasksInGroup(group); + futures.add(stopTasksInGroup(group)); // set a flag so the other pending completion groups for this set of partitions will also stop stopTasksInTaskGroup = true; // stop all the tasks in the currently reading task group and remove the bad task group - stopTasksInGroup(taskGroups.remove(groupId)); + futures.add(stopTasksInGroup(taskGroups.remove(groupId))); - iTaskGroup.remove(); + toRemove.add(group); } } + + taskGroupList.removeAll(toRemove); } + + Futures.successfulAsList(futures).get(); // wait for all task shutdowns to complete before returning } - private void checkCurrentTaskState() + private void checkCurrentTaskState() throws ExecutionException, InterruptedException { + List> futures = Lists.newArrayList(); Iterator> iTaskGroups = taskGroups.entrySet().iterator(); while (iTaskGroups.hasNext()) { Map.Entry taskGroupEntry = iTaskGroups.next(); @@ -958,7 +1143,7 @@ public class KafkaSupervisor implements Supervisor // stop and remove bad tasks from the task group if (!isTaskCurrent(groupId, taskId)) { log.info("Stopping task [%s] which does not match the expected offset range and ingestion spec", taskId); - stopTask(taskId, false); + futures.add(stopTask(taskId, false)); iTasks.remove(); continue; } @@ -972,13 +1157,15 @@ public class KafkaSupervisor implements Supervisor // check for successful tasks, and if we find one, stop all tasks in the group and remove the group so it can // be recreated with the next set of offsets if (taskData.status.isSuccess()) { - stopTasksInGroup(taskGroup); + futures.add(stopTasksInGroup(taskGroup)); iTaskGroups.remove(); break; } } log.debug("Task group [%d] post-pruning: %s", groupId, taskGroup.tasks.keySet()); } + + Futures.successfulAsList(futures).get(); // wait for all task shutdowns to complete before returning } void createNewTasks() @@ -1048,7 +1235,7 @@ public class KafkaSupervisor implements Supervisor taskId, new TaskResource(sequenceName, 1), spec.getDataSchema(), - spec.getTuningConfig(), + taskTuningConfig, kafkaIOConfig, ImmutableMap.of(), null @@ -1068,23 +1255,23 @@ public class KafkaSupervisor implements Supervisor } } - private Map generateStartingOffsetsForPartitionGroup(int groupId) + private ImmutableMap generateStartingOffsetsForPartitionGroup(int groupId) { - Map startingOffsets = new HashMap<>(); + ImmutableMap.Builder builder = ImmutableMap.builder(); for (Map.Entry entry : partitionGroups.get(groupId).entrySet()) { Integer partition = entry.getKey(); Long offset = entry.getValue(); - if (offset != null) { + if (offset != null && offset != NOT_SET) { // if we are given a startingOffset (set by a previous task group which is pending completion) then use it - startingOffsets.put(partition, offset); + builder.put(partition, offset); } else { // if we don't have a startingOffset (first run or we had some previous failures and reset the offsets) then // get the offset from metadata storage (if available) or Kafka (otherwise) - startingOffsets.put(partition, getOffsetFromStorageForPartition(partition)); + builder.put(partition, getOffsetFromStorageForPartition(partition)); } } - return startingOffsets; + return builder.build(); } /** @@ -1174,32 +1361,49 @@ public class KafkaSupervisor implements Supervisor return generateSequenceName(taskGroupId).equals(taskSequenceName); } - private void stopTasksInGroup(TaskGroup taskGroup) + private ListenableFuture stopTasksInGroup(TaskGroup taskGroup) { if (taskGroup == null) { - return; + return Futures.immediateFuture(null); } + final List> futures = Lists.newArrayList(); for (Map.Entry entry : taskGroup.tasks.entrySet()) { if (!entry.getValue().status.isComplete()) { - stopTask(entry.getKey(), false); + futures.add(stopTask(entry.getKey(), false)); } } + + return Futures.transform( + Futures.successfulAsList(futures), new Function, Void>() + { + @Nullable + @Override + public Void apply(@Nullable List input) + { + return null; + } + }, workerExec + ); } - private void stopTask(final String id, final boolean publish) + private ListenableFuture stopTask(final String id, final boolean publish) { - if (!taskInfoProvider.getTaskLocation(id).equals(TaskLocation.unknown())) { - try { - taskClient.stop(id, publish); - } - catch (Exception e) { - log.warn(e, "Task [%s] failed to stop in a timely manner, killing task", id); - killTask(id); - } - } else { - killTask(id); - } + return Futures.transform( + taskClient.stopAsync(id, publish), new Function() + { + @Nullable + @Override + public Void apply(@Nullable Boolean result) + { + if (result == null || !result) { + log.info("Task [%s] failed to stop in a timely manner, killing task", id); + killTask(id); + } + return null; + } + }, workerExec + ); } private void killTask(final String id) @@ -1212,47 +1416,6 @@ public class KafkaSupervisor implements Supervisor } } - private DateTime getTaskStartTime(final String id) - { - if (!taskInfoProvider.getTaskLocation(id).equals(TaskLocation.unknown())) { - DateTime startTime = taskClient.getStartTime(id); - log.debug("Received start time of [%s] from task [%s]", startTime, id); - return startTime; - } - - return null; - } - - private Optional getTaskStatus(final String id) - { - if (!taskInfoProvider.getTaskLocation(id).equals(TaskLocation.unknown())) { - try { - return Optional.of(taskClient.getStatus(id)); - } - catch (Exception e) { - log.warn(e, "Failed to get status for task [%s]", id); - } - } - - return Optional.absent(); - } - - private Map getCurrentOffsets(final String id, final boolean retry) - { - if (!taskInfoProvider.getTaskLocation(id).equals(TaskLocation.unknown())) { - try { - return taskClient.getCurrentOffsets(id, retry); - } - catch (Exception e) { - // this happens regularly if generateReport() is frequently hit and a task is in transition and isn't fatal so - // downgrade to info without stack trace - log.info("Failed to get current offsets for task [%s]", id); - } - } - - return ImmutableMap.of(); - } - private int getTaskGroupIdForPartition(int partition) { return partition % ioConfig.getTaskCount(); @@ -1286,6 +1449,9 @@ public class KafkaSupervisor implements Supervisor ioConfig.getTaskDuration().getMillis() / 1000 ); + List taskReports = Lists.newArrayList(); + List>> futures = Lists.newArrayList(); + try { for (TaskGroup taskGroup : taskGroups.values()) { for (Map.Entry entry : taskGroup.tasks.entrySet()) { @@ -1294,19 +1460,24 @@ public class KafkaSupervisor implements Supervisor Long remainingSeconds = null; if (startTime != null) { remainingSeconds = Math.max( - 0, - ioConfig.getTaskDuration().getMillis() - (DateTime.now().getMillis() - startTime - .getMillis()) + 0, ioConfig.getTaskDuration().getMillis() - (DateTime.now().getMillis() - startTime.getMillis()) ) / 1000; } - report.addActiveTask( - taskId, - (includeOffsets ? taskGroup.partitionOffsets : null), - (includeOffsets ? getCurrentOffsets(taskId, false) : null), - startTime, - remainingSeconds + taskReports.add( + new TaskReportData( + taskId, + (includeOffsets ? taskGroup.partitionOffsets : null), + null, + startTime, + remainingSeconds, + TaskReportData.TaskType.ACTIVE + ) ); + + if (includeOffsets) { + futures.add(taskClient.getCurrentOffsetsAsync(taskId, false)); + } } } @@ -1321,16 +1492,32 @@ public class KafkaSupervisor implements Supervisor / 1000; } - report.addPublishingTask( - taskId, - (includeOffsets ? taskGroup.partitionOffsets : null), - (includeOffsets ? getCurrentOffsets(taskId, false) : null), - startTime, - remainingSeconds + taskReports.add( + new TaskReportData( + taskId, + (includeOffsets ? taskGroup.partitionOffsets : null), + null, + startTime, + remainingSeconds, + TaskReportData.TaskType.PUBLISHING + ) ); + + if (includeOffsets) { + futures.add(taskClient.getCurrentOffsetsAsync(taskId, false)); + } } } } + + List> results = Futures.successfulAsList(futures).get(); + for (int i = 0; i < taskReports.size(); i++) { + TaskReportData reportData = taskReports.get(i); + if (includeOffsets) { + reportData.setCurrentOffsets(results.get(i)); + } + report.addTask(reportData); + } } catch (Exception e) { log.warn(e, "Failed to generate status report"); diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java index 5833cd6482d..516d9493c1e 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java @@ -136,6 +136,23 @@ public class KafkaSupervisorIOConfig return lateMessageRejectionPeriod; } + @Override + public String toString() + { + return "KafkaSupervisorIOConfig{" + + "topic='" + topic + '\'' + + ", replicas=" + replicas + + ", taskCount=" + taskCount + + ", taskDuration=" + taskDuration + + ", consumerProperties=" + consumerProperties + + ", startDelay=" + startDelay + + ", period=" + period + + ", useEarliestOffset=" + useEarliestOffset + + ", completionTimeout=" + completionTimeout + + ", lateMessageRejectionPeriod=" + lateMessageRejectionPeriod + + '}'; + } + private static Duration defaultDuration(final Period period, final String theDefault) { return (period == null ? new Period(theDefault) : period).toStandardDuration(); diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorReport.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorReport.java index cf08bd3103e..980ad3753c8 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorReport.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorReport.java @@ -21,80 +21,14 @@ package io.druid.indexing.kafka.supervisor; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Lists; +import com.metamx.common.IAE; import io.druid.indexing.overlord.supervisor.SupervisorReport; import org.joda.time.DateTime; import java.util.List; -import java.util.Map; public class KafkaSupervisorReport extends SupervisorReport { - public class TaskReportData - { - private final String id; - private final Map startingOffsets; - private final Map currentOffsets; - private final DateTime startTime; - private final Long remainingSeconds; - - public TaskReportData( - String id, - Map startingOffsets, - Map currentOffsets, - DateTime startTime, - Long remainingSeconds - ) - { - this.id = id; - this.startingOffsets = startingOffsets; - this.currentOffsets = currentOffsets; - this.startTime = startTime; - this.remainingSeconds = remainingSeconds; - } - - @JsonProperty - public String getId() - { - return id; - } - - @JsonProperty - public Map getStartingOffsets() - { - return startingOffsets; - } - - @JsonProperty - public Map getCurrentOffsets() - { - return currentOffsets; - } - - @JsonProperty - public DateTime getStartTime() - { - return startTime; - } - - @JsonProperty - public Long getRemainingSeconds() - { - return remainingSeconds; - } - - @Override - public String toString() - { - return "{" + - "id='" + id + '\'' + - (startingOffsets != null ? ", startingOffsets=" + startingOffsets : "") + - (currentOffsets != null ? ", currentOffsets=" + currentOffsets : "") + - ", startTime=" + startTime + - ", remainingSeconds=" + remainingSeconds + - '}'; - } - } - public class KafkaSupervisorReportPayload { private final String dataSource; @@ -200,26 +134,15 @@ public class KafkaSupervisorReport extends SupervisorReport return payload; } - public void addActiveTask( - String id, - Map startingOffsets, - Map currentOffsets, - DateTime startTime, - Long remainingSeconds - ) + public void addTask(TaskReportData data) { - payload.activeTasks.add(new TaskReportData(id, startingOffsets, currentOffsets, startTime, remainingSeconds)); - } - - public void addPublishingTask( - String id, - Map startingOffsets, - Map currentOffsets, - DateTime startTime, - Long remainingSeconds - ) - { - payload.publishingTasks.add(new TaskReportData(id, startingOffsets, currentOffsets, startTime, remainingSeconds)); + if (data.getType().equals(TaskReportData.TaskType.ACTIVE)) { + payload.activeTasks.add(data); + } else if (data.getType().equals(TaskReportData.TaskType.PUBLISHING)) { + payload.publishingTasks.add(data); + } else { + throw new IAE("Unknown task type [%s]", data.getType().name()); + } } @Override diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java index 016d924337a..06ace520e9f 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java @@ -25,9 +25,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import io.druid.guice.annotations.Json; -import io.druid.indexing.kafka.KafkaIndexTaskClient; import io.druid.indexing.kafka.KafkaIndexTaskClientFactory; -import io.druid.indexing.kafka.KafkaTuningConfig; import io.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import io.druid.indexing.overlord.TaskMaster; import io.druid.indexing.overlord.TaskStorage; @@ -38,7 +36,7 @@ import io.druid.segment.indexing.DataSchema; public class KafkaSupervisorSpec implements SupervisorSpec { private final DataSchema dataSchema; - private final KafkaTuningConfig tuningConfig; + private final KafkaSupervisorTuningConfig tuningConfig; private final KafkaSupervisorIOConfig ioConfig; private final TaskStorage taskStorage; @@ -50,7 +48,7 @@ public class KafkaSupervisorSpec implements SupervisorSpec @JsonCreator public KafkaSupervisorSpec( @JsonProperty("dataSchema") DataSchema dataSchema, - @JsonProperty("tuningConfig") KafkaTuningConfig tuningConfig, + @JsonProperty("tuningConfig") KafkaSupervisorTuningConfig tuningConfig, @JsonProperty("ioConfig") KafkaSupervisorIOConfig ioConfig, @JacksonInject TaskStorage taskStorage, @JacksonInject TaskMaster taskMaster, @@ -62,7 +60,21 @@ public class KafkaSupervisorSpec implements SupervisorSpec this.dataSchema = Preconditions.checkNotNull(dataSchema, "dataSchema"); this.tuningConfig = tuningConfig != null ? tuningConfig - : new KafkaTuningConfig(null, null, null, null, null, null, null, null, null); + : new KafkaSupervisorTuningConfig( + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + ); this.ioConfig = Preconditions.checkNotNull(ioConfig, "ioConfig"); this.taskStorage = taskStorage; @@ -79,7 +91,7 @@ public class KafkaSupervisorSpec implements SupervisorSpec } @JsonProperty - public KafkaTuningConfig getTuningConfig() + public KafkaSupervisorTuningConfig getTuningConfig() { return tuningConfig; } @@ -108,4 +120,14 @@ public class KafkaSupervisorSpec implements SupervisorSpec this ); } + + @Override + public String toString() + { + return "KafkaSupervisorSpec{" + + "dataSchema=" + dataSchema + + ", tuningConfig=" + tuningConfig + + ", ioConfig=" + ioConfig + + '}'; + } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java new file mode 100644 index 00000000000..88feaf8b5a8 --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java @@ -0,0 +1,119 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.kafka.supervisor; + +import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.indexing.kafka.KafkaTuningConfig; +import io.druid.segment.IndexSpec; +import org.joda.time.Duration; +import org.joda.time.Period; + +import java.io.File; + +public class KafkaSupervisorTuningConfig extends KafkaTuningConfig +{ + private final Integer workerThreads; + private final Integer chatThreads; + private final Long chatRetries; + private final Duration httpTimeout; + + public KafkaSupervisorTuningConfig( + @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory, + @JsonProperty("maxRowsPerSegment") Integer maxRowsPerSegment, + @JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod, + @JsonProperty("basePersistDirectory") File basePersistDirectory, + @JsonProperty("maxPendingPersists") Integer maxPendingPersists, + @JsonProperty("indexSpec") IndexSpec indexSpec, + @JsonProperty("buildV9Directly") Boolean buildV9Directly, + @JsonProperty("reportParseExceptions") Boolean reportParseExceptions, + @JsonProperty("handoffConditionTimeout") Long handoffConditionTimeout, + @JsonProperty("workerThreads") Integer workerThreads, + @JsonProperty("chatThreads") Integer chatThreads, + @JsonProperty("chatRetries") Long chatRetries, + @JsonProperty("httpTimeout") Period httpTimeout + ) + { + super( + maxRowsInMemory, + maxRowsPerSegment, + intermediatePersistPeriod, + basePersistDirectory, + maxPendingPersists, + indexSpec, + buildV9Directly, + reportParseExceptions, + handoffConditionTimeout + ); + + this.workerThreads = workerThreads; + this.chatThreads = chatThreads; + this.chatRetries = (chatRetries != null ? chatRetries : 8); + this.httpTimeout = defaultDuration(httpTimeout, "PT10S"); + } + + @JsonProperty + public Integer getWorkerThreads() + { + return workerThreads; + } + + @JsonProperty + public Integer getChatThreads() + { + return chatThreads; + } + + @JsonProperty + public Long getChatRetries() + { + return chatRetries; + } + + @JsonProperty + public Duration getHttpTimeout() + { + return httpTimeout; + } + + @Override + public String toString() + { + return "KafkaSupervisorTuningConfig{" + + "maxRowsInMemory=" + getMaxRowsInMemory() + + ", maxRowsPerSegment=" + getMaxRowsPerSegment() + + ", intermediatePersistPeriod=" + getIntermediatePersistPeriod() + + ", basePersistDirectory=" + getBasePersistDirectory() + + ", maxPendingPersists=" + getMaxPendingPersists() + + ", indexSpec=" + getIndexSpec() + + ", buildV9Directly=" + getBuildV9Directly() + + ", reportParseExceptions=" + isReportParseExceptions() + + ", handoffConditionTimeout=" + getHandoffConditionTimeout() + + ", workerThreads=" + workerThreads + + ", chatThreads=" + chatThreads + + ", chatRetries=" + chatRetries + + ", httpTimeout=" + httpTimeout + + '}'; + } + + private static Duration defaultDuration(final Period period, final String theDefault) + { + return (period == null ? new Period(theDefault) : period).toStandardDuration(); + } +} diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/TaskReportData.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/TaskReportData.java new file mode 100644 index 00000000000..dcb4fd474d5 --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/TaskReportData.java @@ -0,0 +1,110 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.kafka.supervisor; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.joda.time.DateTime; + +import java.util.Map; + +public class TaskReportData +{ + public enum TaskType + { + ACTIVE, PUBLISHING, UNKNOWN + } + + private final String id; + private final Map startingOffsets; + private final DateTime startTime; + private final Long remainingSeconds; + private final TaskType type; + private Map currentOffsets; + + public TaskReportData( + String id, + Map startingOffsets, + Map currentOffsets, + DateTime startTime, + Long remainingSeconds, + TaskType type + ) + { + this.id = id; + this.startingOffsets = startingOffsets; + this.currentOffsets = currentOffsets; + this.startTime = startTime; + this.remainingSeconds = remainingSeconds; + this.type = type; + } + + @JsonProperty + public String getId() + { + return id; + } + + @JsonProperty + public Map getStartingOffsets() + { + return startingOffsets; + } + + @JsonProperty + public Map getCurrentOffsets() + { + return currentOffsets; + } + + public void setCurrentOffsets(Map currentOffsets) + { + this.currentOffsets = currentOffsets; + } + + @JsonProperty + public DateTime getStartTime() + { + return startTime; + } + + @JsonProperty + public Long getRemainingSeconds() + { + return remainingSeconds; + } + + @JsonProperty + public TaskType getType() + { + return type; + } + + @Override + public String toString() + { + return "{" + + "id='" + id + '\'' + + (startingOffsets != null ? ", startingOffsets=" + startingOffsets : "") + + (currentOffsets != null ? ", currentOffsets=" + currentOffsets : "") + + ", startTime=" + startTime + + ", remainingSeconds=" + remainingSeconds + + '}'; + } +} diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java index 1cc9b93a609..32734e96ece 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java @@ -21,89 +21,127 @@ package io.druid.indexing.kafka; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; import com.metamx.common.IAE; import com.metamx.http.client.HttpClient; import com.metamx.http.client.Request; import com.metamx.http.client.response.FullResponseHandler; import com.metamx.http.client.response.FullResponseHolder; -import io.druid.indexing.common.RetryPolicyConfig; -import io.druid.indexing.common.RetryPolicyFactory; import io.druid.indexing.common.TaskInfoProvider; import io.druid.indexing.common.TaskLocation; import io.druid.indexing.common.TaskStatus; import io.druid.jackson.DefaultObjectMapper; import org.easymock.Capture; -import org.easymock.EasyMockRunner; +import org.easymock.CaptureType; import org.easymock.EasyMockSupport; -import org.easymock.Mock; import org.jboss.netty.handler.codec.http.HttpHeaders; import org.jboss.netty.handler.codec.http.HttpMethod; import org.jboss.netty.handler.codec.http.HttpResponse; import org.jboss.netty.handler.codec.http.HttpResponseStatus; import org.joda.time.DateTime; -import org.joda.time.Period; +import org.joda.time.Duration; +import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import java.io.IOException; import java.net.URL; +import java.util.List; import java.util.Map; import static org.easymock.EasyMock.anyObject; import static org.easymock.EasyMock.capture; +import static org.easymock.EasyMock.eq; import static org.easymock.EasyMock.expect; import static org.easymock.EasyMock.reset; -@RunWith(EasyMockRunner.class) +@RunWith(Parameterized.class) public class KafkaIndexTaskClientTest extends EasyMockSupport { private static final ObjectMapper objectMapper = new DefaultObjectMapper(); private static final String TEST_ID = "test-id"; + private static final List TEST_IDS = Lists.newArrayList("test-id1", "test-id2", "test-id3", "test-id4"); private static final String TEST_HOST = "test-host"; private static final int TEST_PORT = 1234; + private static final String TEST_DATASOURCE = "test-datasource"; + private static final Duration TEST_HTTP_TIMEOUT = new Duration(5000); + private static final long TEST_NUM_RETRIES = 0; + private static final String URL_FORMATTER = "http://%s:%d/druid/worker/v1/chat/%s/%s"; - @Mock + private int numThreads; private HttpClient httpClient; - - @Mock private TaskInfoProvider taskInfoProvider; - - @Mock private FullResponseHolder responseHolder; - - @Mock private HttpResponse response; - - @Mock private HttpHeaders headers; - private KafkaIndexTaskClient client; + @Parameterized.Parameters(name = "numThreads = {0}") + public static Iterable constructorFeeder() + { + return ImmutableList.of(new Object[]{1}, new Object[]{8}); + } + + public KafkaIndexTaskClientTest(int numThreads) + { + this.numThreads = numThreads; + } + @Before public void setUp() throws Exception { + httpClient = createMock(HttpClient.class); + taskInfoProvider = createMock(TaskInfoProvider.class); + responseHolder = createMock(FullResponseHolder.class); + response = createMock(HttpResponse.class); + headers = createMock(HttpHeaders.class); + client = new TestableKafkaIndexTaskClient(httpClient, objectMapper, taskInfoProvider); expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(new TaskLocation(TEST_HOST, TEST_PORT)).anyTimes(); expect(taskInfoProvider.getTaskStatus(TEST_ID)).andReturn(Optional.of(TaskStatus.running(TEST_ID))).anyTimes(); + + for (int i = 0; i < TEST_IDS.size(); i++) { + expect(taskInfoProvider.getTaskLocation(TEST_IDS.get(i))).andReturn(new TaskLocation(TEST_HOST, TEST_PORT)) + .anyTimes(); + expect(taskInfoProvider.getTaskStatus(TEST_IDS.get(i))).andReturn(Optional.of(TaskStatus.running(TEST_IDS.get(i)))) + .anyTimes(); + } } - @Test(expected = KafkaIndexTaskClient.NoTaskLocationException.class) - public void testNoTaskLocationException() throws Exception + @After + public void tearDown() throws Exception + { + client.close(); + } + + @Test + public void testNoTaskLocation() throws Exception { reset(taskInfoProvider); expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(TaskLocation.unknown()).anyTimes(); expect(taskInfoProvider.getTaskStatus(TEST_ID)).andReturn(Optional.of(TaskStatus.running(TEST_ID))).anyTimes(); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.BAD_REQUEST).times(2); - expect(responseHolder.getContent()).andReturn(""); - expect(httpClient.go(anyObject(Request.class), anyObject(FullResponseHandler.class))).andReturn( - Futures.immediateFuture(responseHolder) - ); replayAll(); - client.getCurrentOffsets(TEST_ID, true); + + Assert.assertEquals(false, client.stop(TEST_ID, true)); + Assert.assertEquals(false, client.resume(TEST_ID)); + Assert.assertEquals(ImmutableMap.of(), client.pause(TEST_ID)); + Assert.assertEquals(ImmutableMap.of(), client.pause(TEST_ID, 10)); + Assert.assertEquals(KafkaIndexTask.Status.NOT_STARTED, client.getStatus(TEST_ID)); + Assert.assertEquals(null, client.getStartTime(TEST_ID)); + Assert.assertEquals(ImmutableMap.of(), client.getCurrentOffsets(TEST_ID, true)); + Assert.assertEquals(ImmutableMap.of(), client.getEndOffsets(TEST_ID)); + Assert.assertEquals(false, client.setEndOffsets(TEST_ID, ImmutableMap.of())); + Assert.assertEquals(false, client.setEndOffsets(TEST_ID, ImmutableMap.of(), true)); + + verifyAll(); } @Test(expected = KafkaIndexTaskClient.TaskNotRunnableException.class) @@ -122,7 +160,13 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport public void testInternalServerError() throws Exception { expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.INTERNAL_SERVER_ERROR).times(2); - expect(httpClient.go(anyObject(Request.class), anyObject(FullResponseHandler.class))).andReturn( + expect( + httpClient.go( + anyObject(Request.class), + anyObject(FullResponseHandler.class), + eq(TEST_HTTP_TIMEOUT) + ) + ).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -136,7 +180,13 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport { expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.BAD_REQUEST).times(2); expect(responseHolder.getContent()).andReturn(""); - expect(httpClient.go(anyObject(Request.class), anyObject(FullResponseHandler.class))).andReturn( + expect( + httpClient.go( + anyObject(Request.class), + anyObject(FullResponseHandler.class), + eq(TEST_HTTP_TIMEOUT) + ) + ).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -155,7 +205,13 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport .andReturn("{}"); expect(response.headers()).andReturn(headers); expect(headers.get("X-Druid-Task-Id")).andReturn("a-different-task-id"); - expect(httpClient.go(anyObject(Request.class), anyObject(FullResponseHandler.class))).andReturn( + expect( + httpClient.go( + anyObject(Request.class), + anyObject(FullResponseHandler.class), + eq(TEST_HTTP_TIMEOUT) + ) + ).andReturn( Futures.immediateFuture(responseHolder) ).times(2); replayAll(); @@ -172,7 +228,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport Capture captured = Capture.newInstance(); expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}"); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class))).andReturn( + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -196,58 +252,72 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport @Test public void testGetCurrentOffsetsWithRetry() throws Exception { - client = new RetryingTestableKafkaIndexTaskClient(httpClient, objectMapper, taskInfoProvider); + client = new TestableKafkaIndexTaskClient(httpClient, objectMapper, taskInfoProvider, 3); - Capture captured = Capture.newInstance(); - Capture captured2 = Capture.newInstance(); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(2) - .andReturn(HttpResponseStatus.OK).times(2); - expect(responseHolder.getContent()).andReturn("") + Capture captured = Capture.newInstance(CaptureType.ALL); + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(6) + .andReturn(HttpResponseStatus.OK).times(1); + expect(responseHolder.getContent()).andReturn("").times(2) .andReturn("{\"0\":1, \"1\":10}"); - expect(responseHolder.getResponse()).andReturn(response); - expect(response.headers()).andReturn(headers); - expect(headers.get("X-Druid-Task-Id")).andReturn(TEST_ID); + expect(responseHolder.getResponse()).andReturn(response).times(2); + expect(response.headers()).andReturn(headers).times(2); + expect(headers.get("X-Druid-Task-Id")).andReturn(TEST_ID).times(2); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class))).andReturn( + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) - ); - expect(httpClient.go(capture(captured2), anyObject(FullResponseHandler.class))).andReturn( - Futures.immediateFuture(responseHolder) - ); + ).times(3); replayAll(); Map results = client.getCurrentOffsets(TEST_ID, true); verifyAll(); - Request request = captured.getValue(); - Assert.assertEquals(HttpMethod.GET, request.getMethod()); - Assert.assertEquals( - new URL("http://test-host:1234/druid/worker/v1/chat/test-id/offsets/current"), - request.getUrl() - ); - Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); - - request = captured2.getValue(); - Assert.assertEquals(HttpMethod.GET, request.getMethod()); - Assert.assertEquals( - new URL("http://test-host:1234/druid/worker/v1/chat/test-id/offsets/current"), - request.getUrl() - ); - Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); + Assert.assertEquals(3, captured.getValues().size()); + for (Request request : captured.getValues()) { + Assert.assertEquals(HttpMethod.GET, request.getMethod()); + Assert.assertEquals( + new URL("http://test-host:1234/druid/worker/v1/chat/test-id/offsets/current"), + request.getUrl() + ); + Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); + } Assert.assertEquals(2, results.size()); Assert.assertEquals(1, (long) results.get(0)); Assert.assertEquals(10, (long) results.get(1)); } + @Test(expected = RuntimeException.class) + public void testGetCurrentOffsetsWithExhaustedRetries() throws Exception + { + client = new TestableKafkaIndexTaskClient(httpClient, objectMapper, taskInfoProvider, 2); + + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).anyTimes(); + expect(responseHolder.getContent()).andReturn("").anyTimes(); + expect(responseHolder.getResponse()).andReturn(response).anyTimes(); + expect(response.headers()).andReturn(headers).anyTimes(); + expect(headers.get("X-Druid-Task-Id")).andReturn(TEST_ID).anyTimes(); + + expect( + httpClient.go( + anyObject(Request.class), + anyObject(FullResponseHandler.class), + eq(TEST_HTTP_TIMEOUT) + ) + ).andReturn(Futures.immediateFuture(responseHolder)).anyTimes(); + replayAll(); + + client.getCurrentOffsets(TEST_ID, true); + verifyAll(); + } + @Test public void testGetEndOffsets() throws Exception { Capture captured = Capture.newInstance(); expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}"); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class))).andReturn( + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -271,7 +341,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport @Test public void testGetStartTime() throws Exception { - client = new RetryingTestableKafkaIndexTaskClient(httpClient, objectMapper, taskInfoProvider); + client = new TestableKafkaIndexTaskClient(httpClient, objectMapper, taskInfoProvider, 2); DateTime now = DateTime.now(); Capture captured = Capture.newInstance(); @@ -281,7 +351,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport expect(response.headers()).andReturn(headers); expect(headers.get("X-Druid-Task-Id")).andReturn(null); expect(responseHolder.getContent()).andReturn(String.valueOf(now.getMillis())).anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class))).andReturn( + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) ).times(2); replayAll(); @@ -308,7 +378,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport Capture captured = Capture.newInstance(); expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); expect(responseHolder.getContent()).andReturn(String.format("\"%s\"", status.toString())).anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class))).andReturn( + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -333,7 +403,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport Capture captured = Capture.newInstance(); expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).times(2); expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}").anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class))).andReturn( + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -360,7 +430,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport Capture captured = Capture.newInstance(); expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).times(2); expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}").anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class))).andReturn( + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -391,13 +461,13 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport .andReturn(HttpResponseStatus.OK).times(2); expect(responseHolder.getContent()).andReturn("\"PAUSED\"") .andReturn("{\"0\":1, \"1\":10}").anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class))).andReturn( + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) ); - expect(httpClient.go(capture(captured2), anyObject(FullResponseHandler.class))).andReturn( + expect(httpClient.go(capture(captured2), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) ); - expect(httpClient.go(capture(captured3), anyObject(FullResponseHandler.class))).andReturn( + expect(httpClient.go(capture(captured3), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) ); @@ -437,8 +507,8 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport public void testResume() throws Exception { Capture captured = Capture.newInstance(); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class))).andReturn( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -461,8 +531,8 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport Map endOffsets = ImmutableMap.of(0, 15L, 1, 120L); Capture captured = Capture.newInstance(); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class))).andReturn( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -486,8 +556,8 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport Map endOffsets = ImmutableMap.of(0, 15L, 1, 120L); Capture captured = Capture.newInstance(); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class))).andReturn( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -509,8 +579,8 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport public void testStop() throws Exception { Capture captured = Capture.newInstance(); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class))).andReturn( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -531,8 +601,8 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport public void testStopAndPublish() throws Exception { Capture captured = Capture.newInstance(); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class))).andReturn( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -549,6 +619,345 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); } + @Test + public void testStopAsync() throws Exception + { + final int numRequests = TEST_IDS.size(); + Capture captured = Capture.newInstance(CaptureType.ALL); + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + Futures.immediateFuture(responseHolder) + ).times(numRequests); + replayAll(); + + List expectedUrls = Lists.newArrayList(); + List> futures = Lists.newArrayList(); + for (int i = 0; i < numRequests; i++) { + expectedUrls.add(new URL(String.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "stop"))); + futures.add(client.stopAsync(TEST_IDS.get(i), false)); + } + + List responses = Futures.allAsList(futures).get(); + + verifyAll(); + List requests = captured.getValues(); + + Assert.assertEquals(numRequests, requests.size()); + Assert.assertEquals(numRequests, responses.size()); + for (int i = 0; i < numRequests; i++) { + Assert.assertEquals(HttpMethod.POST, requests.get(i).getMethod()); + Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); + Assert.assertTrue(responses.get(i)); + } + } + + @Test + public void testResumeAsync() throws Exception + { + final int numRequests = TEST_IDS.size(); + Capture captured = Capture.newInstance(CaptureType.ALL); + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + Futures.immediateFuture(responseHolder) + ).times(numRequests); + replayAll(); + + List expectedUrls = Lists.newArrayList(); + List> futures = Lists.newArrayList(); + for (int i = 0; i < numRequests; i++) { + expectedUrls.add(new URL(String.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "resume"))); + futures.add(client.resumeAsync(TEST_IDS.get(i))); + } + + List responses = Futures.allAsList(futures).get(); + + verifyAll(); + List requests = captured.getValues(); + + Assert.assertEquals(numRequests, requests.size()); + Assert.assertEquals(numRequests, responses.size()); + for (int i = 0; i < numRequests; i++) { + Assert.assertEquals(HttpMethod.POST, requests.get(i).getMethod()); + Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); + Assert.assertTrue(responses.get(i)); + } + } + + @Test + public void testPauseAsync() throws Exception + { + final int numRequests = TEST_IDS.size(); + Capture captured = Capture.newInstance(CaptureType.ALL); + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + Futures.immediateFuture(responseHolder) + ).times(numRequests); + replayAll(); + + List expectedUrls = Lists.newArrayList(); + List>> futures = Lists.newArrayList(); + for (int i = 0; i < numRequests; i++) { + expectedUrls.add(new URL(String.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "pause"))); + futures.add(client.pauseAsync(TEST_IDS.get(i))); + } + + List> responses = Futures.allAsList(futures).get(); + + verifyAll(); + List requests = captured.getValues(); + + Assert.assertEquals(numRequests, requests.size()); + Assert.assertEquals(numRequests, responses.size()); + for (int i = 0; i < numRequests; i++) { + Assert.assertEquals(HttpMethod.POST, requests.get(i).getMethod()); + Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); + Assert.assertEquals(Maps.newLinkedHashMap(ImmutableMap.of(0, 1L)), responses.get(i)); + } + } + + @Test + public void testPauseAsyncWithTimeout() throws Exception + { + final int numRequests = TEST_IDS.size(); + Capture captured = Capture.newInstance(CaptureType.ALL); + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + Futures.immediateFuture(responseHolder) + ).times(numRequests); + replayAll(); + + List expectedUrls = Lists.newArrayList(); + List>> futures = Lists.newArrayList(); + for (int i = 0; i < numRequests; i++) { + expectedUrls.add(new URL(String.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "pause?timeout=9"))); + futures.add(client.pauseAsync(TEST_IDS.get(i), 9)); + } + + List> responses = Futures.allAsList(futures).get(); + + verifyAll(); + List requests = captured.getValues(); + + Assert.assertEquals(numRequests, requests.size()); + Assert.assertEquals(numRequests, responses.size()); + for (int i = 0; i < numRequests; i++) { + Assert.assertEquals(HttpMethod.POST, requests.get(i).getMethod()); + Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); + Assert.assertEquals(Maps.newLinkedHashMap(ImmutableMap.of(0, 1L)), responses.get(i)); + } + } + + @Test + public void testGetStatusAsync() throws Exception + { + final int numRequests = TEST_IDS.size(); + Capture captured = Capture.newInstance(CaptureType.ALL); + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(responseHolder.getContent()).andReturn("\"READING\"").anyTimes(); + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + Futures.immediateFuture(responseHolder) + ).times(numRequests); + replayAll(); + + List expectedUrls = Lists.newArrayList(); + List> futures = Lists.newArrayList(); + for (int i = 0; i < numRequests; i++) { + expectedUrls.add(new URL(String.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "status"))); + futures.add(client.getStatusAsync(TEST_IDS.get(i))); + } + + List responses = Futures.allAsList(futures).get(); + + verifyAll(); + List requests = captured.getValues(); + + Assert.assertEquals(numRequests, requests.size()); + Assert.assertEquals(numRequests, responses.size()); + for (int i = 0; i < numRequests; i++) { + Assert.assertEquals(HttpMethod.GET, requests.get(i).getMethod()); + Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); + Assert.assertEquals(KafkaIndexTask.Status.READING, responses.get(i)); + } + } + + @Test + public void testGetStartTimeAsync() throws Exception + { + final DateTime now = DateTime.now(); + final int numRequests = TEST_IDS.size(); + Capture captured = Capture.newInstance(CaptureType.ALL); + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(responseHolder.getContent()).andReturn(String.valueOf(now.getMillis())).anyTimes(); + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + Futures.immediateFuture(responseHolder) + ).times(numRequests); + replayAll(); + + List expectedUrls = Lists.newArrayList(); + List> futures = Lists.newArrayList(); + for (int i = 0; i < numRequests; i++) { + expectedUrls.add(new URL(String.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "time/start"))); + futures.add(client.getStartTimeAsync(TEST_IDS.get(i))); + } + + List responses = Futures.allAsList(futures).get(); + + verifyAll(); + List requests = captured.getValues(); + + Assert.assertEquals(numRequests, requests.size()); + Assert.assertEquals(numRequests, responses.size()); + for (int i = 0; i < numRequests; i++) { + Assert.assertEquals(HttpMethod.GET, requests.get(i).getMethod()); + Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); + Assert.assertEquals(now, responses.get(i)); + } + } + + @Test + public void testGetCurrentOffsetsAsync() throws Exception + { + final int numRequests = TEST_IDS.size(); + Capture captured = Capture.newInstance(CaptureType.ALL); + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + Futures.immediateFuture(responseHolder) + ).times(numRequests); + replayAll(); + + List expectedUrls = Lists.newArrayList(); + List>> futures = Lists.newArrayList(); + for (int i = 0; i < numRequests; i++) { + expectedUrls.add(new URL(String.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "offsets/current"))); + futures.add(client.getCurrentOffsetsAsync(TEST_IDS.get(i), false)); + } + + List> responses = Futures.allAsList(futures).get(); + + verifyAll(); + List requests = captured.getValues(); + + Assert.assertEquals(numRequests, requests.size()); + Assert.assertEquals(numRequests, responses.size()); + for (int i = 0; i < numRequests; i++) { + Assert.assertEquals(HttpMethod.GET, requests.get(i).getMethod()); + Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); + Assert.assertEquals(Maps.newLinkedHashMap(ImmutableMap.of(0, 1L)), responses.get(i)); + } + } + + @Test + public void testGetEndOffsetsAsync() throws Exception + { + final int numRequests = TEST_IDS.size(); + Capture captured = Capture.newInstance(CaptureType.ALL); + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + Futures.immediateFuture(responseHolder) + ).times(numRequests); + replayAll(); + + List expectedUrls = Lists.newArrayList(); + List>> futures = Lists.newArrayList(); + for (int i = 0; i < numRequests; i++) { + expectedUrls.add(new URL(String.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "offsets/end"))); + futures.add(client.getEndOffsetsAsync(TEST_IDS.get(i))); + } + + List> responses = Futures.allAsList(futures).get(); + + verifyAll(); + List requests = captured.getValues(); + + Assert.assertEquals(numRequests, requests.size()); + Assert.assertEquals(numRequests, responses.size()); + for (int i = 0; i < numRequests; i++) { + Assert.assertEquals(HttpMethod.GET, requests.get(i).getMethod()); + Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); + Assert.assertEquals(Maps.newLinkedHashMap(ImmutableMap.of(0, 1L)), responses.get(i)); + } + } + + @Test + public void testSetEndOffsetsAsync() throws Exception + { + final Map endOffsets = ImmutableMap.of(0, 15L, 1, 120L); + final int numRequests = TEST_IDS.size(); + Capture captured = Capture.newInstance(CaptureType.ALL); + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + Futures.immediateFuture(responseHolder) + ).times(numRequests); + replayAll(); + + List expectedUrls = Lists.newArrayList(); + List> futures = Lists.newArrayList(); + for (int i = 0; i < numRequests; i++) { + expectedUrls.add(new URL(String.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "offsets/end"))); + futures.add(client.setEndOffsetsAsync(TEST_IDS.get(i), endOffsets)); + } + + List responses = Futures.allAsList(futures).get(); + + verifyAll(); + List requests = captured.getValues(); + + Assert.assertEquals(numRequests, requests.size()); + Assert.assertEquals(numRequests, responses.size()); + for (int i = 0; i < numRequests; i++) { + Assert.assertEquals(HttpMethod.POST, requests.get(i).getMethod()); + Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); + Assert.assertTrue(responses.get(i)); + } + } + + @Test + public void testSetEndOffsetsAsyncWithResume() throws Exception + { + final Map endOffsets = ImmutableMap.of(0, 15L, 1, 120L); + final int numRequests = TEST_IDS.size(); + Capture captured = Capture.newInstance(CaptureType.ALL); + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + Futures.immediateFuture(responseHolder) + ).times(numRequests); + replayAll(); + + List expectedUrls = Lists.newArrayList(); + List> futures = Lists.newArrayList(); + for (int i = 0; i < numRequests; i++) { + expectedUrls.add( + new URL( + String.format( + URL_FORMATTER, + TEST_HOST, + TEST_PORT, + TEST_IDS.get(i), + "offsets/end?resume=true" + ) + ) + ); + futures.add(client.setEndOffsetsAsync(TEST_IDS.get(i), endOffsets, true)); + } + + List responses = Futures.allAsList(futures).get(); + + verifyAll(); + List requests = captured.getValues(); + + Assert.assertEquals(numRequests, requests.size()); + Assert.assertEquals(numRequests, responses.size()); + for (int i = 0; i < numRequests; i++) { + Assert.assertEquals(HttpMethod.POST, requests.get(i).getMethod()); + Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); + Assert.assertTrue(responses.get(i)); + } + } + private class TestableKafkaIndexTaskClient extends KafkaIndexTaskClient { public TestableKafkaIndexTaskClient( @@ -557,42 +966,20 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport TaskInfoProvider taskInfoProvider ) { - super(httpClient, jsonMapper, taskInfoProvider); + this(httpClient, jsonMapper, taskInfoProvider, TEST_NUM_RETRIES); } - @Override - RetryPolicyFactory createRetryPolicyFactory() + public TestableKafkaIndexTaskClient( + HttpClient httpClient, + ObjectMapper jsonMapper, + TaskInfoProvider taskInfoProvider, + long numRetries + ) { - return new RetryPolicyFactory( - new RetryPolicyConfig() - .setMinWait(new Period("PT1S")) - .setMaxRetryCount(0) - ); + super(httpClient, jsonMapper, taskInfoProvider, TEST_DATASOURCE, numThreads, TEST_HTTP_TIMEOUT, numRetries); } @Override void checkConnection(String host, int port) throws IOException { } } - - private class RetryingTestableKafkaIndexTaskClient extends TestableKafkaIndexTaskClient - { - public RetryingTestableKafkaIndexTaskClient( - HttpClient httpClient, - ObjectMapper jsonMapper, - TaskInfoProvider taskInfoProvider - ) - { - super(httpClient, jsonMapper, taskInfoProvider); - } - - @Override - RetryPolicyFactory createRetryPolicyFactory() - { - return new RetryPolicyFactory( - new RetryPolicyConfig() - .setMinWait(new Period("PT1S")) - .setMaxRetryCount(1) - ); - } - } } diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java index a77cf44a299..611006225ca 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java @@ -24,7 +24,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import io.druid.jackson.DefaultObjectMapper; import io.druid.segment.IndexSpec; import io.druid.segment.indexing.TuningConfig; -import org.joda.time.DateTime; import org.joda.time.Period; import org.junit.Assert; import org.junit.Test; @@ -101,4 +100,21 @@ public class KafkaTuningConfigTest Assert.assertEquals(true, config.isReportParseExceptions()); Assert.assertEquals(100, config.getHandoffConditionTimeout()); } + + @Test + public void testCopyOf() throws Exception + { + KafkaTuningConfig original = new KafkaTuningConfig(1, 2, new Period("PT3S"), new File("/tmp/xxx"), 4, new IndexSpec(), true, true, 5L); + KafkaTuningConfig copy = KafkaTuningConfig.copyOf(original); + + Assert.assertEquals(1, copy.getMaxRowsInMemory()); + Assert.assertEquals(2, copy.getMaxRowsPerSegment()); + Assert.assertEquals(new Period("PT3S"), copy.getIntermediatePersistPeriod()); + Assert.assertEquals(new File("/tmp/xxx"), copy.getBasePersistDirectory()); + Assert.assertEquals(4, copy.getMaxPendingPersists()); + Assert.assertEquals(new IndexSpec(), copy.getIndexSpec()); + Assert.assertEquals(true, copy.getBuildV9Directly()); + Assert.assertEquals(true, copy.isReportParseExceptions()); + Assert.assertEquals(5L, copy.getHandoffConditionTimeout()); + } } diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index abe0332bac9..93850ac2de8 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -24,6 +24,7 @@ import com.google.common.base.Charsets; import com.google.common.base.Optional; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.metamx.common.Granularity; import com.metamx.common.ISE; @@ -70,10 +71,9 @@ import org.apache.kafka.clients.producer.ProducerRecord; import org.easymock.Capture; import org.easymock.CaptureType; import org.easymock.EasyMock; -import org.easymock.EasyMockRunner; import org.easymock.EasyMockSupport; -import org.easymock.Mock; import org.joda.time.DateTime; +import org.joda.time.Duration; import org.joda.time.Interval; import org.joda.time.Period; import org.junit.After; @@ -83,6 +83,7 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import java.io.File; import java.util.ArrayList; @@ -95,50 +96,61 @@ import java.util.concurrent.Executor; import static org.easymock.EasyMock.anyObject; import static org.easymock.EasyMock.anyString; import static org.easymock.EasyMock.capture; +import static org.easymock.EasyMock.eq; import static org.easymock.EasyMock.expect; import static org.easymock.EasyMock.expectLastCall; import static org.easymock.EasyMock.replay; import static org.easymock.EasyMock.reset; -@RunWith(EasyMockRunner.class) +@RunWith(Parameterized.class) public class KafkaSupervisorTest extends EasyMockSupport { private static final ObjectMapper objectMapper = new DefaultObjectMapper(); private static final String KAFKA_TOPIC = "testTopic"; private static final String DATASOURCE = "testDS"; private static final int NUM_PARTITIONS = 3; + private static final int TEST_CHAT_THREADS = 3; + private static final long TEST_CHAT_RETRIES = 9L; + private static final Period TEST_HTTP_TIMEOUT = new Period("PT10S"); + private int numThreads; private TestingCluster zkServer; private TestBroker kafkaServer; private KafkaSupervisor supervisor; private String kafkaHost; private DataSchema dataSchema; - private KafkaTuningConfig tuningConfig; - - @Mock + private KafkaSupervisorTuningConfig tuningConfig; private TaskStorage taskStorage; - - @Mock private TaskMaster taskMaster; - - @Mock private TaskRunner taskRunner; - - @Mock private IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; - - @Mock private KafkaIndexTaskClient taskClient; - - @Mock private TaskQueue taskQueue; @Rule public final TemporaryFolder tempFolder = new TemporaryFolder(); + @Parameterized.Parameters(name = "numThreads = {0}") + public static Iterable constructorFeeder() + { + return ImmutableList.of(new Object[]{1}, new Object[]{8}); + } + + public KafkaSupervisorTest(int numThreads) + { + this.numThreads = numThreads; + } + @Before public void setUp() throws Exception { + taskStorage = createMock(TaskStorage.class); + taskMaster = createMock(TaskMaster.class); + taskRunner = createMock(TaskRunner.class); + indexerMetadataStorageCoordinator = createMock(IndexerMetadataStorageCoordinator.class); + taskClient = createMock(KafkaIndexTaskClient.class); + taskQueue = createMock(TaskQueue.class); + zkServer = new TestingCluster(1); zkServer.start(); @@ -152,7 +164,7 @@ public class KafkaSupervisorTest extends EasyMockSupport kafkaHost = String.format("localhost:%d", kafkaServer.getPort()); dataSchema = getDataSchema(DATASOURCE); - tuningConfig = new KafkaTuningConfig( + tuningConfig = new KafkaSupervisorTuningConfig( 1000, 50000, new Period("P1Y"), @@ -161,7 +173,11 @@ public class KafkaSupervisorTest extends EasyMockSupport null, true, false, - null + null, + numThreads, + TEST_CHAT_THREADS, + TEST_CHAT_RETRIES, + TEST_HTTP_TIMEOUT ); } @@ -202,7 +218,7 @@ public class KafkaSupervisorTest extends EasyMockSupport KafkaIndexTask task = captured.getValue(); Assert.assertEquals(dataSchema, task.getDataSchema()); - Assert.assertEquals(tuningConfig, task.getTuningConfig()); + Assert.assertEquals(KafkaTuningConfig.copyOf(tuningConfig), task.getTuningConfig()); KafkaIOConfig taskConfig = task.getIOConfig(); Assert.assertEquals(kafkaHost, taskConfig.getConsumerProperties().get("bootstrap.servers")); @@ -485,9 +501,9 @@ public class KafkaSupervisorTest extends EasyMockSupport expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); - expect(taskClient.getStatus(anyString())).andThrow(taskClient.new NoTaskLocationException("test-id")).anyTimes(); - expect(taskClient.getStartTime(anyString())).andThrow(taskClient.new NoTaskLocationException("test-id")) - .anyTimes(); + expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.NOT_STARTED)) + .anyTimes(); + expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTime.now())).anyTimes(); expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null @@ -568,9 +584,9 @@ public class KafkaSupervisorTest extends EasyMockSupport expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); expect(taskStorage.getTask("id4")).andReturn(Optional.of(id3)).anyTimes(); expect(taskStorage.getTask("id5")).andReturn(Optional.of(id3)).anyTimes(); - expect(taskClient.getStatus(anyString())).andThrow(taskClient.new NoTaskLocationException("test-id")).anyTimes(); - expect(taskClient.getStartTime(anyString())).andThrow(taskClient.new NoTaskLocationException("test-id")) - .anyTimes(); + expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.NOT_STARTED)) + .anyTimes(); + expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTime.now())).anyTimes(); expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null @@ -598,9 +614,9 @@ public class KafkaSupervisorTest extends EasyMockSupport expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - expect(taskClient.getStatus(anyString())).andThrow(taskClient.new NoTaskLocationException("test-id")).anyTimes(); - expect(taskClient.getStartTime(anyString())).andThrow(taskClient.new NoTaskLocationException("test-id")) - .anyTimes(); + expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.NOT_STARTED)) + .anyTimes(); + expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTime.now())).anyTimes(); expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null @@ -681,8 +697,8 @@ public class KafkaSupervisorTest extends EasyMockSupport expect(taskStorage.getActiveTasks()).andReturn(existingTasks).anyTimes(); expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); - expect(taskClient.getStartTime(anyString())).andThrow(taskClient.new NoTaskLocationException("test-id")) - .anyTimes(); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING)); + expect(taskClient.getStartTimeAsync("id1")).andReturn(Futures.immediateFuture(now)).anyTimes(); expect(taskQueue.add(capture(captured))).andReturn(true); expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( @@ -705,14 +721,18 @@ public class KafkaSupervisorTest extends EasyMockSupport KafkaIndexTask iHaveFailed = (KafkaIndexTask) existingTasks.get(0); reset(taskStorage); reset(taskQueue); + reset(taskClient); expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(captured.getValue())).anyTimes(); expect(taskStorage.getStatus(iHaveFailed.getId())).andReturn(Optional.of(TaskStatus.failure(iHaveFailed.getId()))); expect(taskStorage.getStatus(runningTaskId)).andReturn(Optional.of(TaskStatus.running(runningTaskId))).anyTimes(); expect(taskStorage.getTask(iHaveFailed.getId())).andReturn(Optional.of((Task) iHaveFailed)).anyTimes(); expect(taskStorage.getTask(runningTaskId)).andReturn(Optional.of(captured.getValue())).anyTimes(); + expect(taskClient.getStatusAsync(runningTaskId)).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING)); + expect(taskClient.getStartTimeAsync(runningTaskId)).andReturn(Futures.immediateFuture(now)).anyTimes(); expect(taskQueue.add(capture(aNewTaskCapture))).andReturn(true); replay(taskStorage); replay(taskQueue); + replay(taskClient); supervisor.runInternal(); verifyAll(); @@ -739,9 +759,9 @@ public class KafkaSupervisorTest extends EasyMockSupport expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - expect(taskClient.getStatus(anyString())).andThrow(taskClient.new NoTaskLocationException("test-id")).anyTimes(); - expect(taskClient.getStartTime(anyString())).andThrow(taskClient.new NoTaskLocationException("test-id")) - .anyTimes(); + expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.NOT_STARTED)) + .anyTimes(); + expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTime.now())).anyTimes(); expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null @@ -776,6 +796,7 @@ public class KafkaSupervisorTest extends EasyMockSupport KafkaIndexTask iAmSuccess = (KafkaIndexTask) tasks.get(0); reset(taskStorage); reset(taskQueue); + reset(taskClient); expect(taskStorage.getActiveTasks()).andReturn(imStillRunning).anyTimes(); for (Task task : imStillRunning) { expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); @@ -783,10 +804,11 @@ public class KafkaSupervisorTest extends EasyMockSupport } expect(taskStorage.getStatus(iAmSuccess.getId())).andReturn(Optional.of(TaskStatus.success(iAmSuccess.getId()))); expect(taskStorage.getTask(iAmSuccess.getId())).andReturn(Optional.of((Task) iAmSuccess)).anyTimes(); - taskQueue.shutdown(capture(shutdownTaskIdCapture)); expect(taskQueue.add(capture(newTasksCapture))).andReturn(true).times(2); + expect(taskClient.stopAsync(capture(shutdownTaskIdCapture), eq(false))).andReturn(Futures.immediateFuture(true)); replay(taskStorage); replay(taskQueue); + replay(taskClient); supervisor.runInternal(); verifyAll(); @@ -835,24 +857,25 @@ public class KafkaSupervisorTest extends EasyMockSupport expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); } expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); - expect(taskClient.getStatus(anyString())) - .andReturn(KafkaIndexTask.Status.READING) + expect(taskClient.getStatusAsync(anyString())) + .andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING)) .anyTimes(); - expect(taskClient.getStartTime(EasyMock.contains("sequenceName-0"))) - .andReturn(DateTime.now().minusMinutes(2)) - .andReturn(DateTime.now()); - expect(taskClient.getStartTime(EasyMock.contains("sequenceName-1"))) - .andReturn(DateTime.now()) + expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) + .andReturn(Futures.immediateFuture(DateTime.now().minusMinutes(2))) + .andReturn(Futures.immediateFuture(DateTime.now())); + expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) + .andReturn(Futures.immediateFuture(DateTime.now())) .times(2); - expect(taskClient.pause(EasyMock.contains("sequenceName-0"))) - .andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)) - .andReturn(ImmutableMap.of(0, 10L, 1, 15L, 2, 35L)); - taskClient.setEndOffsets( - EasyMock.contains("sequenceName-0"), - EasyMock.eq(ImmutableMap.of(0, 10L, 1, 20L, 2, 35L)), - EasyMock.eq(true) - ); - expectLastCall().times(2); + expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) + .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))) + .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 15L, 2, 35L))); + expect( + taskClient.setEndOffsetsAsync( + EasyMock.contains("sequenceName-0"), + EasyMock.eq(ImmutableMap.of(0, 10L, 1, 20L, 2, 35L)), + EasyMock.eq(true) + ) + ).andReturn(Futures.immediateFuture(true)).times(2); expect(taskQueue.add(capture(captured))).andReturn(true).times(2); replay(taskStorage, taskRunner, taskClient, taskQueue); @@ -863,7 +886,7 @@ public class KafkaSupervisorTest extends EasyMockSupport for (Task task : captured.getValues()) { KafkaIndexTask kafkaIndexTask = (KafkaIndexTask) task; Assert.assertEquals(dataSchema, kafkaIndexTask.getDataSchema()); - Assert.assertEquals(tuningConfig, kafkaIndexTask.getTuningConfig()); + Assert.assertEquals(KafkaTuningConfig.copyOf(tuningConfig), kafkaIndexTask.getTuningConfig()); KafkaIOConfig taskConfig = kafkaIndexTask.getIOConfig(); Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName()); @@ -909,8 +932,9 @@ public class KafkaSupervisorTest extends EasyMockSupport null ) ).anyTimes(); - expect(taskClient.getStatus("id1")).andReturn(KafkaIndexTask.Status.PUBLISHING); - expect(taskClient.getCurrentOffsets("id1", false)).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.PUBLISHING)); + expect(taskClient.getCurrentOffsetsAsync("id1", false)) + .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))); expect(taskClient.getCurrentOffsets("id1", true)).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); expect(taskQueue.add(capture(captured))).andReturn(true); @@ -936,7 +960,7 @@ public class KafkaSupervisorTest extends EasyMockSupport Assert.assertEquals(0, payload.getActiveTasks().size()); Assert.assertEquals(1, payload.getPublishingTasks().size()); - KafkaSupervisorReport.TaskReportData publishingReport = payload.getPublishingTasks().get(0); + TaskReportData publishingReport = payload.getPublishingTasks().get(0); Assert.assertEquals("id1", publishingReport.getId()); Assert.assertEquals(ImmutableMap.of(0, 0L, 1, 0L, 2, 0L), publishingReport.getStartingOffsets()); @@ -944,7 +968,7 @@ public class KafkaSupervisorTest extends EasyMockSupport KafkaIndexTask capturedTask = captured.getValue(); Assert.assertEquals(dataSchema, capturedTask.getDataSchema()); - Assert.assertEquals(tuningConfig, capturedTask.getTuningConfig()); + Assert.assertEquals(KafkaTuningConfig.copyOf(tuningConfig), capturedTask.getTuningConfig()); KafkaIOConfig capturedTaskConfig = capturedTask.getIOConfig(); Assert.assertEquals(kafkaHost, capturedTaskConfig.getConsumerProperties().get("bootstrap.servers")); @@ -997,8 +1021,9 @@ public class KafkaSupervisorTest extends EasyMockSupport null ) ).anyTimes(); - expect(taskClient.getStatus("id1")).andReturn(KafkaIndexTask.Status.PUBLISHING); - expect(taskClient.getCurrentOffsets("id1", false)).andReturn(ImmutableMap.of(0, 10L, 2, 30L)); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.PUBLISHING)); + expect(taskClient.getCurrentOffsetsAsync("id1", false)) + .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 2, 30L))); expect(taskClient.getCurrentOffsets("id1", true)).andReturn(ImmutableMap.of(0, 10L, 2, 30L)); expect(taskQueue.add(capture(captured))).andReturn(true); @@ -1024,7 +1049,7 @@ public class KafkaSupervisorTest extends EasyMockSupport Assert.assertEquals(0, payload.getActiveTasks().size()); Assert.assertEquals(1, payload.getPublishingTasks().size()); - KafkaSupervisorReport.TaskReportData publishingReport = payload.getPublishingTasks().get(0); + TaskReportData publishingReport = payload.getPublishingTasks().get(0); Assert.assertEquals("id1", publishingReport.getId()); Assert.assertEquals(ImmutableMap.of(0, 0L, 2, 0L), publishingReport.getStartingOffsets()); @@ -1032,7 +1057,7 @@ public class KafkaSupervisorTest extends EasyMockSupport KafkaIndexTask capturedTask = captured.getValue(); Assert.assertEquals(dataSchema, capturedTask.getDataSchema()); - Assert.assertEquals(tuningConfig, capturedTask.getTuningConfig()); + Assert.assertEquals(KafkaTuningConfig.copyOf(tuningConfig), capturedTask.getTuningConfig()); KafkaIOConfig capturedTaskConfig = capturedTask.getIOConfig(); Assert.assertEquals(kafkaHost, capturedTaskConfig.getConsumerProperties().get("bootstrap.servers")); @@ -1098,12 +1123,14 @@ public class KafkaSupervisorTest extends EasyMockSupport null ) ).anyTimes(); - expect(taskClient.getStatus("id1")).andReturn(KafkaIndexTask.Status.PUBLISHING); - expect(taskClient.getStatus("id2")).andReturn(KafkaIndexTask.Status.READING); - expect(taskClient.getStartTime("id2")).andReturn(startTime); - expect(taskClient.getCurrentOffsets("id1", false)).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.PUBLISHING)); + expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING)); + expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); + expect(taskClient.getCurrentOffsetsAsync("id1", false)) + .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))); expect(taskClient.getCurrentOffsets("id1", true)).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); - expect(taskClient.getCurrentOffsets("id2", false)).andReturn(ImmutableMap.of(0, 40L, 1, 50L, 2, 60L)); + expect(taskClient.getCurrentOffsetsAsync("id2", false)) + .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 40L, 1, 50L, 2, 60L))); taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); replayAll(); @@ -1127,8 +1154,8 @@ public class KafkaSupervisorTest extends EasyMockSupport Assert.assertEquals(1, payload.getActiveTasks().size()); Assert.assertEquals(1, payload.getPublishingTasks().size()); - KafkaSupervisorReport.TaskReportData activeReport = payload.getActiveTasks().get(0); - KafkaSupervisorReport.TaskReportData publishingReport = payload.getPublishingTasks().get(0); + TaskReportData activeReport = payload.getActiveTasks().get(0); + TaskReportData publishingReport = payload.getPublishingTasks().get(0); Assert.assertEquals("id2", activeReport.getId()); Assert.assertEquals(startTime, activeReport.getStartTime()); @@ -1140,6 +1167,191 @@ public class KafkaSupervisorTest extends EasyMockSupport Assert.assertEquals(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L), publishingReport.getCurrentOffsets()); } + @Test + public void testKillUnresponsiveTasksWhileGettingStartTime() throws Exception + { + supervisor = getSupervisor(2, 2, true, "PT1H", null); + addSomeEvents(1); + + Capture captured = Capture.newInstance(CaptureType.ALL); + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + new KafkaDataSourceMetadata( + null + ) + ).anyTimes(); + expect(taskQueue.add(capture(captured))).andReturn(true).times(4); + taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + + List tasks = captured.getValues(); + + reset(taskStorage, taskClient, taskQueue); + expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); + for (Task task : tasks) { + expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); + expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); + expect(taskClient.getStatusAsync(task.getId())) + .andReturn(Futures.immediateFuture(KafkaIndexTask.Status.NOT_STARTED)); + expect(taskClient.getStartTimeAsync(task.getId())) + .andReturn(Futures.immediateFailedFuture(new RuntimeException())); + taskQueue.shutdown(task.getId()); + } + replay(taskStorage, taskClient, taskQueue); + + supervisor.runInternal(); + verifyAll(); + } + + @Test + public void testKillUnresponsiveTasksWhilePausing() throws Exception + { + final TaskLocation location = new TaskLocation("testHost", 1234); + + supervisor = getSupervisor(2, 2, true, "PT1M", null); + addSomeEvents(100); + + Capture captured = Capture.newInstance(CaptureType.ALL); + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + new KafkaDataSourceMetadata( + null + ) + ).anyTimes(); + expect(taskQueue.add(capture(captured))).andReturn(true).times(4); + taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + + List tasks = captured.getValues(); + Collection workItems = new ArrayList<>(); + for (Task task : tasks) { + workItems.add(new TestTaskRunnerWorkItem(task.getId(), null, location)); + } + + reset(taskStorage, taskRunner, taskClient, taskQueue); + captured = Capture.newInstance(CaptureType.ALL); + expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); + for (Task task : tasks) { + expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); + expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); + } + expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + expect(taskClient.getStatusAsync(anyString())) + .andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING)) + .anyTimes(); + expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) + .andReturn(Futures.immediateFuture(DateTime.now().minusMinutes(2))) + .andReturn(Futures.immediateFuture(DateTime.now())); + expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) + .andReturn(Futures.immediateFuture(DateTime.now())) + .times(2); + expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) + .andReturn(Futures.>immediateFailedFuture(new RuntimeException())).times(2); + taskQueue.shutdown(EasyMock.contains("sequenceName-0")); + expectLastCall().times(2); + expect(taskQueue.add(capture(captured))).andReturn(true).times(2); + + replay(taskStorage, taskRunner, taskClient, taskQueue); + + supervisor.runInternal(); + verifyAll(); + + for (Task task : captured.getValues()) { + KafkaIOConfig taskConfig = ((KafkaIndexTask) task).getIOConfig(); + Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(0)); + Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(2)); + } + } + + @Test + public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception + { + final TaskLocation location = new TaskLocation("testHost", 1234); + + supervisor = getSupervisor(2, 2, true, "PT1M", null); + addSomeEvents(100); + + Capture captured = Capture.newInstance(CaptureType.ALL); + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + new KafkaDataSourceMetadata( + null + ) + ).anyTimes(); + expect(taskQueue.add(capture(captured))).andReturn(true).times(4); + taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + + List tasks = captured.getValues(); + Collection workItems = new ArrayList<>(); + for (Task task : tasks) { + workItems.add(new TestTaskRunnerWorkItem(task.getId(), null, location)); + } + + reset(taskStorage, taskRunner, taskClient, taskQueue); + captured = Capture.newInstance(CaptureType.ALL); + expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); + for (Task task : tasks) { + expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); + expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); + } + expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + expect(taskClient.getStatusAsync(anyString())) + .andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING)) + .anyTimes(); + expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) + .andReturn(Futures.immediateFuture(DateTime.now().minusMinutes(2))) + .andReturn(Futures.immediateFuture(DateTime.now())); + expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) + .andReturn(Futures.immediateFuture(DateTime.now())) + .times(2); + expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) + .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))) + .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 15L, 2, 35L))); + expect( + taskClient.setEndOffsetsAsync( + EasyMock.contains("sequenceName-0"), + EasyMock.eq(ImmutableMap.of(0, 10L, 1, 20L, 2, 35L)), + EasyMock.eq(true) + ) + ).andReturn(Futures.immediateFailedFuture(new RuntimeException())).times(2); + taskQueue.shutdown(EasyMock.contains("sequenceName-0")); + expectLastCall().times(2); + expect(taskQueue.add(capture(captured))).andReturn(true).times(2); + + replay(taskStorage, taskRunner, taskClient, taskQueue); + + supervisor.runInternal(); + verifyAll(); + + for (Task task : captured.getValues()) { + KafkaIOConfig taskConfig = ((KafkaIndexTask) task).getIOConfig(); + Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(0)); + Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(2)); + } + } + @Test(expected = IllegalStateException.class) public void testStopNotStarted() throws Exception { @@ -1151,6 +1363,7 @@ public class KafkaSupervisorTest extends EasyMockSupport public void testStop() throws Exception { expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + taskClient.close(); taskRunner.unregisterListener(String.format("KafkaSupervisor-%s", DATASOURCE)); replayAll(); @@ -1161,6 +1374,91 @@ public class KafkaSupervisorTest extends EasyMockSupport verifyAll(); } + @Test + public void testStopGracefully() throws Exception + { + final TaskLocation location1 = new TaskLocation("testHost", 1234); + final TaskLocation location2 = new TaskLocation("testHost2", 145); + final DateTime startTime = new DateTime(); + + supervisor = getSupervisor(2, 1, true, "PT1H", null); + addSomeEvents(1); + + Task id1 = createKafkaIndexTask( + "id1", + DATASOURCE, + "sequenceName-0", + new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), + new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), + null + ); + + Task id2 = createKafkaIndexTask( + "id2", + DATASOURCE, + "sequenceName-0", + new KafkaPartitions("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), + new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), + null + ); + + Task id3 = createKafkaIndexTask( + "id3", + DATASOURCE, + "sequenceName-0", + new KafkaPartitions("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), + new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), + null + ); + + Collection workItems = new ArrayList<>(); + workItems.add(new TestTaskRunnerWorkItem(id1.getId(), null, location1)); + workItems.add(new TestTaskRunnerWorkItem(id2.getId(), null, location2)); + + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2, id3)).anyTimes(); + expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); + expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); + expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); + expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); + expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + new KafkaDataSourceMetadata( + null + ) + ).anyTimes(); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.PUBLISHING)); + expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING)); + expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING)); + expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); + expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); + expect(taskClient.getCurrentOffsets("id1", true)).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); + + taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + + reset(taskRunner, taskClient, taskQueue); + expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + expect(taskClient.pauseAsync("id2")) + .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 15L, 1, 25L, 2, 30L))); + expect(taskClient.setEndOffsetsAsync("id2", ImmutableMap.of(0, 15L, 1, 25L, 2, 30L), true)) + .andReturn(Futures.immediateFuture(true)); + taskQueue.shutdown("id3"); + expectLastCall().times(2); + + replay(taskRunner, taskClient, taskQueue); + + supervisor.gracefulShutdownInternal(); + verifyAll(); + } + private void addSomeEvents(int numEventsPerPartition) throws Exception { try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { @@ -1203,8 +1501,17 @@ public class KafkaSupervisorTest extends EasyMockSupport KafkaIndexTaskClientFactory taskClientFactory = new KafkaIndexTaskClientFactory(null, null) { @Override - public KafkaIndexTaskClient build(TaskInfoProvider taskLocationProvider) + public KafkaIndexTaskClient build( + TaskInfoProvider taskInfoProvider, + String dataSource, + int numThreads, + Duration httpTimeout, + long numRetries + ) { + Assert.assertEquals(TEST_CHAT_THREADS, numThreads); + Assert.assertEquals(TEST_HTTP_TIMEOUT.toStandardDuration(), httpTimeout); + Assert.assertEquals(TEST_CHAT_RETRIES, numRetries); return taskClient; } }; diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfigTest.java new file mode 100644 index 00000000000..6b6ba4b9d80 --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfigTest.java @@ -0,0 +1,117 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.kafka.supervisor; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.druid.indexing.kafka.KafkaIndexTaskModule; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.segment.IndexSpec; +import io.druid.segment.indexing.TuningConfig; +import org.joda.time.Duration; +import org.joda.time.Period; +import org.junit.Assert; +import org.junit.Test; + +import java.io.File; + +public class KafkaSupervisorTuningConfigTest +{ + private final ObjectMapper mapper; + + public KafkaSupervisorTuningConfigTest() + { + mapper = new DefaultObjectMapper(); + mapper.registerModules((Iterable) new KafkaIndexTaskModule().getJacksonModules()); + } + + @Test + public void testSerdeWithDefaults() throws Exception + { + String jsonStr = "{\"type\": \"kafka\"}"; + + KafkaSupervisorTuningConfig config = (KafkaSupervisorTuningConfig) mapper.readValue( + mapper.writeValueAsString( + mapper.readValue( + jsonStr, + TuningConfig.class + ) + ), + TuningConfig.class + ); + + Assert.assertNotNull(config.getBasePersistDirectory()); + Assert.assertEquals(75000, config.getMaxRowsInMemory()); + Assert.assertEquals(5_000_000, config.getMaxRowsPerSegment()); + Assert.assertEquals(new Period("PT10M"), config.getIntermediatePersistPeriod()); + Assert.assertEquals(0, config.getMaxPendingPersists()); + Assert.assertEquals(new IndexSpec(), config.getIndexSpec()); + Assert.assertEquals(false, config.getBuildV9Directly()); + Assert.assertEquals(false, config.isReportParseExceptions()); + Assert.assertEquals(0, config.getHandoffConditionTimeout()); + Assert.assertNull(config.getWorkerThreads()); + Assert.assertNull(config.getChatThreads()); + Assert.assertEquals(8L, (long) config.getChatRetries()); + Assert.assertEquals(Duration.standardSeconds(10), config.getHttpTimeout()); + } + + @Test + public void testSerdeWithNonDefaults() throws Exception + { + String jsonStr = "{\n" + + " \"type\": \"kafka\",\n" + + " \"basePersistDirectory\": \"/tmp/xxx\",\n" + + " \"maxRowsInMemory\": 100,\n" + + " \"maxRowsPerSegment\": 100,\n" + + " \"intermediatePersistPeriod\": \"PT1H\",\n" + + " \"maxPendingPersists\": 100,\n" + + " \"buildV9Directly\": true,\n" + + " \"reportParseExceptions\": true,\n" + + " \"handoffConditionTimeout\": 100,\n" + + " \"workerThreads\": 12,\n" + + " \"chatThreads\": 13,\n" + + " \"chatRetries\": 14,\n" + + " \"httpTimeout\": \"PT15S\"\n" + + "}"; + + KafkaSupervisorTuningConfig config = (KafkaSupervisorTuningConfig) mapper.readValue( + mapper.writeValueAsString( + mapper.readValue( + jsonStr, + TuningConfig.class + ) + ), + TuningConfig.class + ); + + Assert.assertEquals(new File("/tmp/xxx"), config.getBasePersistDirectory()); + Assert.assertEquals(100, config.getMaxRowsInMemory()); + Assert.assertEquals(100, config.getMaxRowsPerSegment()); + Assert.assertEquals(new Period("PT1H"), config.getIntermediatePersistPeriod()); + Assert.assertEquals(100, config.getMaxPendingPersists()); + Assert.assertEquals(true, config.getBuildV9Directly()); + Assert.assertEquals(true, config.isReportParseExceptions()); + Assert.assertEquals(100, config.getHandoffConditionTimeout()); + Assert.assertEquals(12, (int) config.getWorkerThreads()); + Assert.assertEquals(13, (int) config.getChatThreads()); + Assert.assertEquals(14L, (long) config.getChatRetries()); + Assert.assertEquals(Duration.standardSeconds(15), config.getHttpTimeout()); + } +} diff --git a/integration-tests/docker/Dockerfile b/integration-tests/docker/Dockerfile index fd4a9fafa78..2b81c5ad39b 100644 --- a/integration-tests/docker/Dockerfile +++ b/integration-tests/docker/Dockerfile @@ -23,8 +23,8 @@ RUN wget -q -O - http://www.us.apache.org/dist/zookeeper/zookeeper-3.4.6/zookeep && ln -s /usr/local/zookeeper-3.4.6 /usr/local/zookeeper # Kafka -RUN wget -q -O - http://www.us.apache.org/dist/kafka/0.8.2.0/kafka_2.10-0.8.2.0.tgz | tar -xzf - -C /usr/local \ - && ln -s /usr/local/kafka_2.10-0.8.2.0 /usr/local/kafka +RUN wget -q -O - http://www.us.apache.org/dist/kafka/0.9.0.1/kafka_2.10-0.9.0.1.tgz | tar -xzf - -C /usr/local \ + && ln -s /usr/local/kafka_2.10-0.9.0.1 /usr/local/kafka # Druid system user RUN adduser --system --group --no-create-home druid \ diff --git a/integration-tests/pom.xml b/integration-tests/pom.xml index 70c3d5eee23..5f9f4f794b1 100644 --- a/integration-tests/pom.xml +++ b/integration-tests/pom.xml @@ -65,6 +65,11 @@ mysql-metadata-storage ${project.parent.version} + + io.druid.extensions + druid-kafka-indexing-service + ${project.parent.version} + io.druid druid-services @@ -109,6 +114,11 @@ + + org.apache.kafka + kafka-clients + 0.9.0.1 + diff --git a/integration-tests/src/main/java/io/druid/testing/clients/OverlordResourceTestClient.java b/integration-tests/src/main/java/io/druid/testing/clients/OverlordResourceTestClient.java index 022fa1a456b..ed96d7347b6 100644 --- a/integration-tests/src/main/java/io/druid/testing/clients/OverlordResourceTestClient.java +++ b/integration-tests/src/main/java/io/druid/testing/clients/OverlordResourceTestClient.java @@ -117,7 +117,8 @@ public class OverlordResourceTestClient public TaskStatus.Status getTaskStatus(String taskID) { try { - StatusResponseHolder response = makeRequest( HttpMethod.GET, + StatusResponseHolder response = makeRequest( + HttpMethod.GET, String.format( "%stask/%s/status", getIndexerURL(), @@ -158,7 +159,8 @@ public class OverlordResourceTestClient private List getTasks(String identifier) { try { - StatusResponseHolder response = makeRequest( HttpMethod.GET, + StatusResponseHolder response = makeRequest( + HttpMethod.GET, String.format("%s%s", getIndexerURL(), identifier) ); LOG.info("Tasks %s response %s", identifier, response.getContent()); @@ -176,10 +178,12 @@ public class OverlordResourceTestClient public Map shutDownTask(String taskID) { try { - StatusResponseHolder response = makeRequest( HttpMethod.POST, - String.format("%stask/%s/shutdown", getIndexerURL(), - URLEncoder.encode(taskID, "UTF-8") - ) + StatusResponseHolder response = makeRequest( + HttpMethod.POST, + String.format( + "%stask/%s/shutdown", getIndexerURL(), + URLEncoder.encode(taskID, "UTF-8") + ) ); LOG.info("Shutdown Task %s response %s", taskID, response.getContent()); return jsonMapper.readValue( @@ -195,7 +199,7 @@ public class OverlordResourceTestClient public void waitUntilTaskCompletes(final String taskID) { - waitUntilTaskCompletes(taskID, 60000, 10); + waitUntilTaskCompletes(taskID, 60000, 10); } public void waitUntilTaskCompletes(final String taskID, final int millisEach, final int numTimes) @@ -220,6 +224,61 @@ public class OverlordResourceTestClient ); } + public String submitSupervisor(String spec) + { + try { + StatusResponseHolder response = httpClient.go( + new Request(HttpMethod.POST, new URL(getIndexerURL() + "supervisor")) + .setContent( + "application/json", + spec.getBytes() + ), + responseHandler + ).get(); + if (!response.getStatus().equals(HttpResponseStatus.OK)) { + throw new ISE( + "Error while submitting supervisor to overlord, response [%s %s]", + response.getStatus(), + response.getContent() + ); + } + Map responseData = jsonMapper.readValue( + response.getContent(), new TypeReference>() + { + } + ); + String id = responseData.get("id"); + LOG.info("Submitted supervisor with id[%s]", id); + return id; + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + public void shutdownSupervisor(String id) + { + try { + StatusResponseHolder response = httpClient.go( + new Request( + HttpMethod.POST, new URL(String.format("%ssupervisor/%s/shutdown", getIndexerURL(), id)) + ), + responseHandler + ).get(); + if (!response.getStatus().equals(HttpResponseStatus.OK)) { + throw new ISE( + "Error while shutting down supervisor, response [%s %s]", + response.getStatus(), + response.getContent() + ); + } + LOG.info("Shutdown supervisor with id[%s]", id); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + private StatusResponseHolder makeRequest(HttpMethod method, String url) { try { diff --git a/integration-tests/src/test/java/io/druid/tests/indexer/ITKafkaIndexingServiceTest.java b/integration-tests/src/test/java/io/druid/tests/indexer/ITKafkaIndexingServiceTest.java new file mode 100644 index 00000000000..6e8afce7651 --- /dev/null +++ b/integration-tests/src/test/java/io/druid/tests/indexer/ITKafkaIndexingServiceTest.java @@ -0,0 +1,276 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.tests.indexer; + +import com.google.common.base.Throwables; +import com.google.inject.Inject; +import com.metamx.common.ISE; +import com.metamx.common.logger.Logger; +import io.druid.testing.IntegrationTestingConfig; +import io.druid.testing.guice.DruidTestModuleFactory; +import io.druid.testing.utils.RetryUtil; +import io.druid.testing.utils.TestQueryHelper; +import kafka.admin.AdminUtils; +import kafka.common.TopicExistsException; +import kafka.utils.ZKStringSerializer$; +import org.I0Itec.zkclient.ZkClient; +import org.apache.commons.io.IOUtils; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.StringSerializer; +import org.joda.time.DateTime; +import org.joda.time.DateTimeZone; +import org.joda.time.format.DateTimeFormat; +import org.joda.time.format.DateTimeFormatter; +import org.testng.annotations.AfterClass; +import org.testng.annotations.Guice; +import org.testng.annotations.Test; + +import java.io.IOException; +import java.io.InputStream; +import java.util.Properties; +import java.util.concurrent.Callable; + +/* + * This is a test for the Kafka indexing service. + */ +@Guice(moduleFactory = DruidTestModuleFactory.class) +public class ITKafkaIndexingServiceTest extends AbstractIndexerTest +{ + private static final Logger LOG = new Logger(ITKafkaIndexingServiceTest.class); + private static final int DELAY_BETWEEN_EVENTS_SECS = 5; + private static final String INDEXER_FILE = "/indexer/kafka_supervisor_spec.json"; + private static final String QUERIES_FILE = "/indexer/kafka_index_queries.json"; + private static final String DATASOURCE = "kafka_indexing_service_test"; + private static final String TOPIC_NAME = "kafka_indexing_service_topic"; + private static final int MINUTES_TO_SEND = 4; + + // We'll fill in the current time and numbers for added, deleted and changed + // before sending the event. + final String event_template = + "{\"timestamp\": \"%s\"," + + "\"page\": \"Gypsy Danger\"," + + "\"language\" : \"en\"," + + "\"user\" : \"nuclear\"," + + "\"unpatrolled\" : \"true\"," + + "\"newPage\" : \"true\"," + + "\"robot\": \"false\"," + + "\"anonymous\": \"false\"," + + "\"namespace\":\"article\"," + + "\"continent\":\"North America\"," + + "\"country\":\"United States\"," + + "\"region\":\"Bay Area\"," + + "\"city\":\"San Francisco\"," + + "\"added\":%d," + + "\"deleted\":%d," + + "\"delta\":%d}"; + + private String supervisorId; + private ZkClient zkClient; + private Boolean segmentsExist; // to tell if we should remove segments during teardown + + // format for the querying interval + private final DateTimeFormatter INTERVAL_FMT = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:'00Z'"); + // format for the expected timestamp in a query response + private final DateTimeFormatter TIMESTAMP_FMT = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss'.000Z'"); + private DateTime dtFirst; // timestamp of 1st event + private DateTime dtLast; // timestamp of last event + + @Inject + private TestQueryHelper queryHelper; + @Inject + private IntegrationTestingConfig config; + + @Test + public void testKafka() + { + LOG.info("Starting test: ITKafkaIndexingServiceTest"); + + // create topic + try { + int sessionTimeoutMs = 10000; + int connectionTimeoutMs = 10000; + String zkHosts = config.getZookeeperHosts(); + zkClient = new ZkClient( + zkHosts, sessionTimeoutMs, connectionTimeoutMs, + ZKStringSerializer$.MODULE$ + ); + int numPartitions = 4; + int replicationFactor = 1; + Properties topicConfig = new Properties(); + AdminUtils.createTopic(zkClient, TOPIC_NAME, numPartitions, replicationFactor, topicConfig); + } + catch (TopicExistsException e) { + // it's ok if the topic already exists + } + catch (Exception e) { + throw new ISE(e, "could not create kafka topic"); + } + + String spec; + try { + LOG.info("supervisorSpec name: [%s]", INDEXER_FILE); + spec = getTaskAsString(INDEXER_FILE) + .replaceAll("%%DATASOURCE%%", DATASOURCE) + .replaceAll("%%TOPIC%%", TOPIC_NAME) + .replaceAll("%%KAFKA_BROKER%%", config.getKafkaHost()); + LOG.info("supervisorSpec: [%s]\n", spec); + } + catch (Exception e) { + LOG.error("could not read file [%s]", INDEXER_FILE); + throw new ISE(e, "could not read file [%s]", INDEXER_FILE); + } + + // start supervisor + supervisorId = indexer.submitSupervisor(spec); + LOG.info("Submitted supervisor"); + + // set up kafka producer + Properties properties = new Properties(); + properties.put("bootstrap.servers", config.getKafkaHost()); + LOG.info("Kafka bootstrap.servers: [%s]", config.getKafkaHost()); + properties.put("acks", "all"); + properties.put("retries", "3"); + + KafkaProducer producer = new KafkaProducer<>( + properties, + new StringSerializer(), + new StringSerializer() + ); + + DateTimeZone zone = DateTimeZone.forID("UTC"); + // format for putting into events + DateTimeFormatter event_fmt = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss'Z'"); + + DateTime dt = new DateTime(zone); // timestamp to put on events + dtFirst = dt; // timestamp of 1st event + dtLast = dt; // timestamp of last event + // stop sending events when time passes this + DateTime dtStop = dtFirst.plusMinutes(MINUTES_TO_SEND).plusSeconds(30); + + // these are used to compute the expected aggregations + int added = 0; + int num_events = 0; + + // send data to kafka + while (dt.compareTo(dtStop) < 0) { // as long as we're within the time span + num_events++; + added += num_events; + // construct the event to send + String event = String.format(event_template, event_fmt.print(dt), num_events, 0, num_events); + LOG.info("sending event: [%s]", event); + try { + producer.send(new ProducerRecord(TOPIC_NAME, event)).get(); + } + catch (Exception ioe) { + throw Throwables.propagate(ioe); + } + + try { + Thread.sleep(DELAY_BETWEEN_EVENTS_SECS * 1000); + } + catch (InterruptedException ex) { /* nothing */ } + dtLast = dt; + dt = new DateTime(zone); + } + + producer.close(); + + InputStream is = ITKafkaIndexingServiceTest.class.getResourceAsStream(QUERIES_FILE); + if (null == is) { + throw new ISE("could not open query file: %s", QUERIES_FILE); + } + + // put the timestamps into the query structure + String query_response_template; + try { + query_response_template = IOUtils.toString(is, "UTF-8"); + } + catch (IOException e) { + throw new ISE(e, "could not read query file: %s", QUERIES_FILE); + } + + String queryStr = query_response_template + .replaceAll("%%DATASOURCE%%", DATASOURCE) + .replace("%%TIMEBOUNDARY_RESPONSE_TIMESTAMP%%", TIMESTAMP_FMT.print(dtFirst)) + .replace("%%TIMEBOUNDARY_RESPONSE_MAXTIME%%", TIMESTAMP_FMT.print(dtLast)) + .replace("%%TIMEBOUNDARY_RESPONSE_MINTIME%%", TIMESTAMP_FMT.print(dtFirst)) + .replace("%%TIMESERIES_QUERY_START%%", INTERVAL_FMT.print(dtFirst)) + .replace("%%TIMESERIES_QUERY_END%%", INTERVAL_FMT.print(dtFirst.plusMinutes(MINUTES_TO_SEND + 2))) + .replace("%%TIMESERIES_RESPONSE_TIMESTAMP%%", TIMESTAMP_FMT.print(dtFirst)) + .replace("%%TIMESERIES_ADDED%%", Integer.toString(added)) + .replace("%%TIMESERIES_NUMEVENTS%%", Integer.toString(num_events)); + + // this query will probably be answered from the indexing tasks but possibly from 2 historical segments / 2 indexing + try { + this.queryHelper.testQueriesFromString(queryStr, 2); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + + indexer.shutdownSupervisor(supervisorId); + + // wait for segments to be handed off + try { + RetryUtil.retryUntil( + new Callable() + { + @Override + public Boolean call() throws Exception + { + return coordinator.areSegmentsLoaded(DATASOURCE); + } + }, + true, + 30000, + 10, + "Real-time generated segments loaded" + ); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + LOG.info("segments are present"); + segmentsExist = true; + + // this query will be answered by at least 1 historical segment, most likely 2, and possibly up to all 4 + try { + this.queryHelper.testQueriesFromString(queryStr, 2); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + @AfterClass + public void afterClass() throws Exception + { + LOG.info("teardown"); + + // delete kafka topic + AdminUtils.deleteTopic(zkClient, TOPIC_NAME); + + // remove segments + if (segmentsExist) { + unloadAndKillData(DATASOURCE); + } + } +} diff --git a/integration-tests/src/test/java/io/druid/tests/indexer/ITKafkaTest.java b/integration-tests/src/test/java/io/druid/tests/indexer/ITKafkaTest.java index 5081f48621d..eb7f44a43df 100644 --- a/integration-tests/src/test/java/io/druid/tests/indexer/ITKafkaTest.java +++ b/integration-tests/src/test/java/io/druid/tests/indexer/ITKafkaTest.java @@ -125,24 +125,20 @@ public class ITKafkaTest extends AbstractIndexerTest throw new ISE(e, "could not create kafka topic"); } - String indexerSpec = ""; + String indexerSpec; // replace temp strings in indexer file try { LOG.info("indexerFile name: [%s]", INDEXER_FILE); - indexerSpec = getTaskAsString(INDEXER_FILE); - indexerSpec = indexerSpec.replaceAll("%%TOPIC%%", TOPIC_NAME); - indexerSpec = indexerSpec.replaceAll("%%ZOOKEEPER_SERVER%%", config.getZookeeperHosts()); - indexerSpec = indexerSpec.replaceAll("%%GROUP_ID%%", Long.toString(System.currentTimeMillis())); - indexerSpec = indexerSpec.replaceAll( - "%%SHUTOFFTIME%%", - new DateTime( - System.currentTimeMillis() + TimeUnit.MINUTES.toMillis( - 2 - * MINUTES_TO_SEND - ) - ).toString() - ); + indexerSpec = getTaskAsString(INDEXER_FILE) + .replaceAll("%%DATASOURCE%%", DATASOURCE) + .replaceAll("%%TOPIC%%", TOPIC_NAME) + .replaceAll("%%ZOOKEEPER_SERVER%%", config.getZookeeperHosts()) + .replaceAll("%%GROUP_ID%%", Long.toString(System.currentTimeMillis())) + .replaceAll( + "%%SHUTOFFTIME%%", + new DateTime(System.currentTimeMillis() + TimeUnit.MINUTES.toMillis(2 * MINUTES_TO_SEND)).toString() + ); LOG.info("indexerFile: [%s]\n", indexerSpec); } catch (Exception e) { @@ -217,26 +213,29 @@ public class ITKafkaTest extends AbstractIndexerTest try { query_response_template = IOUtils.toString(is, "UTF-8"); - } catch (IOException e) { - throw new ISE(e, "could not read query file: %s", QUERIES_FILE); + } + catch (IOException e) { + throw new ISE(e, "could not read query file: %s", QUERIES_FILE); } String queryStr = query_response_template - // time boundary - .replace("%%TIMEBOUNDARY_RESPONSE_TIMESTAMP%%", TIMESTAMP_FMT.print(dtFirst)) - .replace("%%TIMEBOUNDARY_RESPONSE_MAXTIME%%", TIMESTAMP_FMT.print(dtLast)) - .replace("%%TIMEBOUNDARY_RESPONSE_MINTIME%%", TIMESTAMP_FMT.print(dtFirst)) - // time series - .replace("%%TIMESERIES_QUERY_START%%", INTERVAL_FMT.print(dtFirst)) - .replace("%%TIMESERIES_QUERY_END%%", INTERVAL_FMT.print(dtFirst.plusMinutes(MINUTES_TO_SEND + 2))) - .replace("%%TIMESERIES_RESPONSE_TIMESTAMP%%", TIMESTAMP_FMT.print(dtFirst)) - .replace("%%TIMESERIES_ADDED%%", Integer.toString(added)) - .replace("%%TIMESERIES_NUMEVENTS%%", Integer.toString(num_events)); + .replaceAll("%%DATASOURCE%%", DATASOURCE) + // time boundary + .replace("%%TIMEBOUNDARY_RESPONSE_TIMESTAMP%%", TIMESTAMP_FMT.print(dtFirst)) + .replace("%%TIMEBOUNDARY_RESPONSE_MAXTIME%%", TIMESTAMP_FMT.print(dtLast)) + .replace("%%TIMEBOUNDARY_RESPONSE_MINTIME%%", TIMESTAMP_FMT.print(dtFirst)) + // time series + .replace("%%TIMESERIES_QUERY_START%%", INTERVAL_FMT.print(dtFirst)) + .replace("%%TIMESERIES_QUERY_END%%", INTERVAL_FMT.print(dtFirst.plusMinutes(MINUTES_TO_SEND + 2))) + .replace("%%TIMESERIES_RESPONSE_TIMESTAMP%%", TIMESTAMP_FMT.print(dtFirst)) + .replace("%%TIMESERIES_ADDED%%", Integer.toString(added)) + .replace("%%TIMESERIES_NUMEVENTS%%", Integer.toString(num_events)); // this query will probably be answered from the realtime task try { this.queryHelper.testQueriesFromString(queryStr, 2); - } catch (Exception e) { + } + catch (Exception e) { throw Throwables.propagate(e); } @@ -285,7 +284,7 @@ public class ITKafkaTest extends AbstractIndexerTest // remove segments if (segmentsExist) { - unloadAndKillData(DATASOURCE); + unloadAndKillData(DATASOURCE); } } } diff --git a/integration-tests/src/test/resources/indexer/kafka_index_queries.json b/integration-tests/src/test/resources/indexer/kafka_index_queries.json index 46c2bf0b56f..73876c53595 100644 --- a/integration-tests/src/test/resources/indexer/kafka_index_queries.json +++ b/integration-tests/src/test/resources/indexer/kafka_index_queries.json @@ -3,7 +3,7 @@ "description": "timeBoundary", "query": { "queryType":"timeBoundary", - "dataSource":"kafka_test" + "dataSource":"%%DATASOURCE%%" }, "expectedResults":[ { @@ -19,7 +19,7 @@ "description": "timeseries", "query": { "queryType": "timeseries", - "dataSource": "kafka_test", + "dataSource": "%%DATASOURCE%%", "intervals": [ "%%TIMESERIES_QUERY_START%%/%%TIMESERIES_QUERY_END%%" ], "granularity": "all", "aggregations": [ diff --git a/integration-tests/src/test/resources/indexer/kafka_index_task.json b/integration-tests/src/test/resources/indexer/kafka_index_task.json index 626a9403a60..282133af53f 100644 --- a/integration-tests/src/test/resources/indexer/kafka_index_task.json +++ b/integration-tests/src/test/resources/indexer/kafka_index_task.json @@ -2,7 +2,7 @@ "type" : "index_realtime", "spec" : { "dataSchema": { - "dataSource": "kafka_test", + "dataSource": "%%DATASOURCE%%", "parser" : { "type" : "string", "parseSpec" : { diff --git a/integration-tests/src/test/resources/indexer/kafka_supervisor_spec.json b/integration-tests/src/test/resources/indexer/kafka_supervisor_spec.json new file mode 100644 index 00000000000..ceec1befb0e --- /dev/null +++ b/integration-tests/src/test/resources/indexer/kafka_supervisor_spec.json @@ -0,0 +1,63 @@ +{ + "type": "kafka", + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "parser": { + "type": "string", + "parseSpec": { + "format": "json", + "timestampSpec": { + "column": "timestamp", + "format": "auto" + }, + "dimensionsSpec": { + "dimensions": ["page", "language", "user", "unpatrolled", "newPage", "robot", "anonymous", "namespace", "continent", "country", "region", "city"], + "dimensionExclusions": [], + "spatialDimensions": [] + } + } + }, + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + } + ], + "granularitySpec": { + "type": "uniform", + "segmentGranularity": "MINUTE", + "queryGranularity": "NONE" + } + }, + "tuningConfig": { + "type": "kafka", + "intermediatePersistPeriod": "PT30S", + "maxRowsPerSegment": 5000000, + "maxRowsInMemory": 500000 + }, + "ioConfig": { + "topic": "%%TOPIC%%", + "consumerProperties": { + "bootstrap.servers": "%%KAFKA_BROKER%%" + }, + "taskCount": 2, + "replicas": 1, + "taskDuration": "PT2M", + "useEarliestOffset": true + } +}