diff --git a/common/src/main/java/io/druid/metadata/MetadataStorageConnector.java b/common/src/main/java/io/druid/metadata/MetadataStorageConnector.java index 3d72fe3496cc..6a6a701963fb 100644 --- a/common/src/main/java/io/druid/metadata/MetadataStorageConnector.java +++ b/common/src/main/java/io/druid/metadata/MetadataStorageConnector.java @@ -51,4 +51,6 @@ byte[] lookup( void createTaskTables(); void createAuditTable(); + + void createSupervisorsTable(); } diff --git a/common/src/main/java/io/druid/metadata/MetadataStorageTablesConfig.java b/common/src/main/java/io/druid/metadata/MetadataStorageTablesConfig.java index 20c870a8e070..01acc55ab00a 100644 --- a/common/src/main/java/io/druid/metadata/MetadataStorageTablesConfig.java +++ b/common/src/main/java/io/druid/metadata/MetadataStorageTablesConfig.java @@ -31,7 +31,7 @@ public class MetadataStorageTablesConfig { public static MetadataStorageTablesConfig fromBase(String base) { - return new MetadataStorageTablesConfig(base, null, null, null, null, null, null, null, null, null); + return new MetadataStorageTablesConfig(base, null, null, null, null, null, null, null, null, null, null); } public static final String TASK_ENTRY_TYPE = "task"; @@ -72,6 +72,9 @@ public static MetadataStorageTablesConfig fromBase(String base) @JsonProperty("audit") private final String auditTable; + @JsonProperty("supervisors") + private final String supervisorTable; + @JsonCreator public MetadataStorageTablesConfig( @JsonProperty("base") String base, @@ -83,7 +86,8 @@ public MetadataStorageTablesConfig( @JsonProperty("tasks") String tasksTable, @JsonProperty("taskLog") String taskLogTable, @JsonProperty("taskLock") String taskLockTable, - @JsonProperty("audit") String auditTable + @JsonProperty("audit") String auditTable, + @JsonProperty("supervisors") String supervisorTable ) { this.base = (base == null) ? DEFAULT_BASE : base; @@ -100,7 +104,7 @@ public MetadataStorageTablesConfig( logTables.put(TASK_ENTRY_TYPE, this.taskLogTable); lockTables.put(TASK_ENTRY_TYPE, this.taskLockTable); this.auditTable = makeTableName(auditTable, "audit"); - + this.supervisorTable = makeTableName(supervisorTable, "supervisors"); } private String makeTableName(String explicitTableName, String defaultSuffix) @@ -170,4 +174,8 @@ public String getAuditTable() return auditTable; } + public String getSupervisorTable() + { + return supervisorTable; + } } diff --git a/docs/content/configuration/index.md b/docs/content/configuration/index.md index f86ec183b864..237463608937 100644 --- a/docs/content/configuration/index.md +++ b/docs/content/configuration/index.md @@ -184,6 +184,7 @@ These properties specify the jdbc connection and other configuration around the |`druid.metadata.storage.tables.tasks`|Used by the indexing service to store tasks.|druid_tasks| |`druid.metadata.storage.tables.taskLog`|Used by the indexing service to store task logs.|druid_taskLog| |`druid.metadata.storage.tables.taskLock`|Used by the indexing service to store task locks.|druid_taskLock| +|`druid.metadata.storage.tables.supervisors`|Used by the indexing service to store supervisor configurations.|druid_supervisors| |`druid.metadata.storage.tables.audit`|The table to use for audit history of configuration changes e.g. Coordinator rules.|druid_audit| ### Deep Storage diff --git a/docs/content/development/extensions-core/kafka-ingestion.md b/docs/content/development/extensions-core/kafka-ingestion.md new file mode 100644 index 000000000000..5b8e69551063 --- /dev/null +++ b/docs/content/development/extensions-core/kafka-ingestion.md @@ -0,0 +1,248 @@ +--- +layout: doc_page +--- + +# Kafka Indexing Service + +The Kafka indexing service enables the configuration of *supervisors* on the Overlord, which facilitate ingestion from +Kafka by managing the creation and lifetime of Kafka indexing tasks. These indexing tasks read events using Kafka's own +partition and offset mechanism and are therefore able to provide guarantees of exactly-once ingestion. They are also +able to read non-recent events from Kafka and are not subject to the window period considerations imposed on other +ingestion mechanisms. The supervisor oversees the state of the indexing tasks to coordinate handoffs, manage failures, +and ensure that the scalability and replication requirements are maintained. + +This service is provided in the `kafka-indexing-service` core extension (see +[Including Extensions](../../operations/including-extensions.html)). Please note that the Kafka indexing service is +currently designated as an *experimental feature* and is subject to the usual +[experimental caveats](../experimental.html). + +## Submitting a Supervisor Spec + +The Kafka indexing service requires that the `kafka-indexing-service` extension be loaded on both the overlord and the +middle managers. A supervisor for a dataSource is started by submitting a supervisor spec via HTTP POST to +`http://:/druid/indexer/v1/supervisor`, for example: + +``` +curl -X POST -H 'Content-Type: application/json' -d @supervisor-spec.json http://localhost:8090/druid/indexer/v1/supervisor +``` + +A sample supervisor spec is shown below: + +```json +{ + "type": "kafka", + "dataSchema": { + "dataSource": "metrics-kafka", + "parser": { + "type": "string", + "parseSpec": { + "format": "json", + "timestampSpec": { + "column": "timestamp", + "format": "auto" + }, + "dimensionsSpec": { + "dimensions": [], + "dimensionExclusions": [ + "timestamp", + "value" + ] + } + } + }, + "metricsSpec": [ + { + "name": "count", + "type": "count" + }, + { + "name": "value_sum", + "fieldName": "value", + "type": "doubleSum" + }, + { + "name": "value_min", + "fieldName": "value", + "type": "doubleMin" + }, + { + "name": "value_max", + "fieldName": "value", + "type": "doubleMax" + } + ], + "granularitySpec": { + "type": "uniform", + "segmentGranularity": "HOUR", + "queryGranularity": "NONE" + } + }, + "tuningConfig": { + "type": "kafka", + "maxRowsPerSegment": 5000000 + }, + "ioConfig": { + "topic": "metrics", + "consumerProperties": { + "bootstrap.servers": "localhost:9092" + }, + "taskCount": 1, + "replicas": 1, + "taskDuration": "PT1H" + } +} +``` + +## Supervisor Configuration + +|Field|Description|Required| +|--------|-----------|---------| +|`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| + +### KafkaTuningConfig + +The tuningConfig is optional and default parameters will be used if no tuningConfig is specified. + +|Field|Type|Description|Required| +|-----|----|-----------|--------| +|`type`|String|The indexing task type, this should always be `kafka`.|yes| +|`maxRowsInMemory`|Integer|The number of rows to aggregate before persisting. This number is the post-aggregation rows, so it is not equivalent to the number of input events, but the number of aggregated rows that those events result in. This is used to manage the required JVM heap size. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 75000)| +|`maxRowsPerSegment`|Integer|The number of rows to aggregate into a segment; this number is post-aggregation rows.|no (default == 5000000)| +|`intermediatePersistPeriod`|ISO8601 Period|The period that determines the rate at which intermediate persists occur.|no (default == PT10M)| +|`maxPendingPersists`|Integer|Maximum number of persists that can be pending but not started. If this limit would be exceeded by a new intermediate persist, ingestion will block until the currently-running persist finishes. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 0, meaning one persist can be running concurrently with ingestion, and none can be queued up)| +|`indexSpec`|Object|Tune how data is indexed, see 'IndexSpec' below for more details.|no| +|`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)| + +#### IndexSpec + +|Field|Type|Description|Required| +|-----|----|-----------|--------| +|`bitmap`|String|The type of bitmap index to create. Choose from `roaring` or `concise`.|no (default == `concise`)| +|`dimensionCompression`|String|Compression format for dimension columns. Choose from `LZ4`, `LZF`, or `uncompressed`.|no (default == `LZ4`)| +|`metricCompression`|String|Compression format for metric columns. Choose from `LZ4`, `LZF`, or `uncompressed`.|no (default == `LZ4`)| + +### KafkaSupervisorIOConfig + +|Field|Type|Description|Required| +|-----|----|-----------|--------| +|`topic`|String|The Kafka topic to read from. This must be a specific topic as topic patterns are not supported.|yes| +|`consumerProperties`|Map|A map of properties to be passed to the Kafka consumer. This must contain a property `bootstrap.servers` with a list of Kafka brokers in the form: `:,:,...`.|yes| +|`replicas`|Integer|The number of replica sets, where 1 means a single set of tasks (no replication). Replica tasks will always be assigned to different workers to provide resiliency against node failure.|no (default == 1)| +|`taskCount`|Integer|The maximum number of *reading* tasks in a *replica set*. This means that the maximum number of reading tasks will be `taskCount * replicas` and the total number of tasks (*reading* + *publishing*) will be higher than this. See 'Capacity Planning' below for more details. The number of reading tasks will be less than `taskCount` if `taskCount > {numKafkaPartitions}`.|no (default == 1)| +|`taskDuration`|ISO8601 Period|The length of time before tasks stop reading and begin publishing their segment. Note that segments are only pushed to deep storage and loadable by historical nodes when the indexing task completes.|no (default == PT1H)| +|`startDelay`|ISO8601 Period|The period to wait before the supervisor starts managing tasks.|no (default == PT5S)| +|`period`|ISO8601 Period|How often the supervisor will execute its management logic. Note that the supervisor will also run in response to certain events (such as tasks succeeding, failing, and reaching their taskDuration) so this value specifies the maximum time between iterations.|no (default == PT30S)| +|`useEarliestOffset`|Boolean|If a supervisor is managing a dataSource for the first time, it will obtain a set of starting offsets from Kafka. This flag determines whether it retrieves the earliest or latest offsets in Kafka. Under normal circumstances, subsequent tasks will start from where the previous segments ended so this flag will only be used on first run.|no (default == false)| +|`completionTimeout`|ISO8601 Period|The length of time to wait before declaring a publishing task as failed and terminating it. If this is set too low, your tasks may never publish. The publishing clock for a task begins roughly after `taskDuration` elapses.|no (default == PT30M)| + +## Supervisor API + +The following endpoints are available on the Overlord: + +#### Create Supervisor +``` +POST /druid/indexer/v1/supervisor +``` +Use `Content-Type: application/json` and provide a supervisor spec in the request body. + +#### Shutdown Supervisor +``` +POST /druid/indexer/v1/supervisor//shutdown +``` +Note that this will cause all indexing tasks managed by this supervisor to immediately stop and begin publishing their segments. + +#### Get Supervisor IDs +``` +GET /druid/indexer/v1/supervisor +``` +Returns a list of the currently active supervisors. + +#### Get Supervisor Spec +``` +GET /druid/indexer/v1/supervisor/ +``` +Returns the current spec for the supervisor with the provided ID. + +#### Get Supervisor Status Report +``` +GET /druid/indexer/v1/supervisor//status +``` +Returns a snapshot report of the current state of the tasks managed by the given supervisor. + +#### Get All Supervisor History +``` +GET /druid/indexer/v1/supervisor/history +``` +Returns an audit history of specs for all supervisors (current and past). + +#### Get Supervisor History +``` +GET /druid/indexer/v1/supervisor//history +``` +Returns an audit history of specs for the supervisor with the provided ID. + +## Capacity Planning + +Kafka indexing tasks run on middle managers and are thus limited by the resources available in the middle manager +cluster. In particular, you should make sure that you have sufficient worker capacity (configured using the +`druid.worker.capacity` property) to handle the configuration in the supervisor spec. Note that worker capacity is +shared across all types of indexing tasks, so you should plan your worker capacity to handle your total indexing load +(e.g. batch processing, realtime tasks, merging tasks, etc.). If your workers run out of capacity, Kafka indexing tasks +will queue and wait for the next available worker. This may cause queries to return partial results but will not result +in data loss (assuming the tasks run before Kafka purges those offsets). + +A running task will normally be in one of two states: *reading* or *publishing*. A task will remain in reading state for +`taskDuration`, at which point it will transition to publishing state. A task will remain in publishing state for as long +as it takes to generate segments, push segments to deep storage, and have them be loaded and served by a historical node +(or until `completionTimeout` elapses). + +The number of reading tasks is controlled by `replicas` and `taskCount`. In general, there will be `replicas * taskCount` +reading tasks, the exception being if taskCount > {numKafkaPartitions} in which case {numKafkaPartitions} tasks will +be used instead. When `taskDuration` elapses, these tasks will transition to publishing state and `replicas * taskCount` +new reading tasks will be created. Therefore to allow for reading tasks and publishing tasks to run concurrently, there +should be a minimum capacity of: + +``` +workerCapacity = 2 * replicas * taskCount +``` + +This value is for the ideal situation in which there is at most one set of tasks publishing while another set is reading. +In some circumstances, it is possible to have multiple sets of tasks publishing simultaneously. This would happen if the +time-to-publish (generate segment, push to deep storage, loaded on historical) > `taskDuration`. This is a valid +scenario (correctness-wise) but requires additional worker capacity to support. In general, it is a good idea to have +`taskDuration` be large enough that the previous set of tasks finishes publishing before the current set begins. + +## Supervisor Persistence + +When a supervisor spec is submitted via the `POST /druid/indexer/v1/supervisor` endpoint, it is persisted in the +configured metadata database. There can only be a single supervisor per dataSource, and submitting a second spec for +the same dataSource will fail with a `409 Conflict` if one already exists. + +When an overlord gains leadership, either by being started or as a result of another overlord failing, it will spawn +a supervisor for each supervisor spec in the metadata database. The supervisor will then discover running Kafka indexing +tasks and will attempt to adopt them if they are compatible with the supervisor's configuration. If they are not +compatible because they have a different ingestion spec or partition allocation, the tasks will be killed and the +supervisor will create a new set of tasks. In this way, the supervisors are persistent across overlord restarts and +fail-overs. + +A supervisor is stopped via the `POST /druid/indexer/v1/supervisor//shutdown` endpoint. This places a +tombstone marker in the database (to prevent the supervisor from being reloaded on a restart) and then gracefully +shuts down the currently running supervisor. When a supervisor is shut down in this way, it will instruct its +managed tasks to stop reading and begin publishing their segments immediately. The call to the shutdown endpoint will +return after all tasks have been signalled to stop but before the tasks finish publishing their segments. + +### Schema/Configuration Changes + +Following from the previous section, schema and configuration changes are managed by first shutting down the supervisor +with a call to the `POST /druid/indexer/v1/supervisor//shutdown` endpoint, waiting for the running tasks +to complete, and then submitting the updated schema via the `POST /druid/indexer/v1/supervisor` create supervisor +endpoint. The updated supervisor will begin reading from the offsets where the previous supervisor ended and no data +will be lost. If the updated schema is posted before the previously running tasks have completed, the supervisor will +detect that the tasks are no longer compatible with the new schema and will issue a shutdown command to the tasks which +may result in the current segments not being published. If this happens, the tasks based on the updated schema will +begin reading from the same starting offsets as the previous aborted tasks and no data will be lost. diff --git a/docs/content/toc.md b/docs/content/toc.md index 3c3a2d2e7ab7..9d77466232a3 100644 --- a/docs/content/toc.md +++ b/docs/content/toc.md @@ -96,7 +96,9 @@ * [Approximate Histograms and Quantiles](../development/extensions-core/approximate-histograms.html) * [Datasketches](../development/extensions-core/datasketches-aggregators.html) * [Geographic Queries](../development/geo.html) - * [Router](../development/router.html) + * [Router](../development/router.html) + * [Kafka Indexing Service](../development/extensions-core/kafka-ingestion.html) + ## Misc * [Papers & Talks](../misc/papers-and-talks.html) 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 new file mode 100644 index 000000000000..ad090889cdd1 --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java @@ -0,0 +1,326 @@ +/* + * 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; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Charsets; +import com.google.common.base.Throwables; +import com.google.inject.Inject; +import com.metamx.common.IAE; +import com.metamx.common.ISE; +import com.metamx.emitter.EmittingLogger; +import com.metamx.http.client.HttpClient; +import com.metamx.http.client.Request; +import com.metamx.http.client.response.StatusResponseHandler; +import com.metamx.http.client.response.StatusResponseHolder; +import io.druid.guice.annotations.Global; +import io.druid.indexing.common.RetryPolicy; +import io.druid.indexing.common.RetryPolicyConfig; +import io.druid.indexing.common.RetryPolicyFactory; +import io.druid.indexing.common.TaskLocation; +import org.jboss.netty.channel.ChannelException; +import org.jboss.netty.handler.codec.http.HttpMethod; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; +import org.joda.time.DateTime; +import org.joda.time.Duration; +import org.joda.time.Period; + +import javax.ws.rs.core.MediaType; +import java.io.IOException; +import java.net.Socket; +import java.net.URI; +import java.util.Map; + +public class KafkaIndexTaskClient +{ + private static final EmittingLogger log = new EmittingLogger(KafkaIndexTaskClient.class); + private static final String BASE_PATH = "/druid/worker/v1/chat"; + + private final HttpClient httpClient; + private final ObjectMapper jsonMapper; + private final RetryPolicyFactory retryPolicyFactory; + + @Inject + public KafkaIndexTaskClient(@Global HttpClient httpClient, ObjectMapper jsonMapper) + { + this.httpClient = httpClient; + this.jsonMapper = jsonMapper; + this.retryPolicyFactory = new RetryPolicyFactory( + new RetryPolicyConfig().setMinWait(Period.seconds(2)) + .setMaxWait(Period.seconds(8)) + .setMaxRetryCount(5) + ); + } + + public void stop(TaskLocation location, String id, boolean publish) + { + try { + final URI serviceUri = new URI( + "http", + null, + location.getHost(), + location.getPort(), + String.format("%s/%s/%s", BASE_PATH, id, "stop"), + publish ? "publish=true" : null, + null + ); + submitRequest(new Request(HttpMethod.POST, serviceUri.toURL()), true); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + public void resume(TaskLocation location, String id) + { + try { + final URI serviceUri = new URI( + "http", + null, + location.getHost(), + location.getPort(), + String.format("%s/%s/%s", BASE_PATH, id, "resume"), + null, + null + ); + submitRequest(new Request(HttpMethod.POST, serviceUri.toURL()), true); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + public Map pause(TaskLocation location, String id) + { + return pause(location, id, 0); + } + + public Map pause(TaskLocation location, String id, long timeout) + { + try { + final URI serviceUri = new URI( + "http", + null, + location.getHost(), + location.getPort(), + String.format("%s/%s/%s", BASE_PATH, id, "pause"), + timeout > 0 ? String.format("timeout=%d", timeout) : null, + null + ); + final StatusResponseHolder response = submitRequest(new Request(HttpMethod.POST, serviceUri.toURL()), true); + + if (response.getStatus().equals(HttpResponseStatus.OK)) { + return jsonMapper.readValue(response.getContent(), new TypeReference>() {}); + } + + final RetryPolicy retryPolicy = retryPolicyFactory.makeRetryPolicy(); + while (true) { + if (getStatus(location, id) == KafkaIndexTask.Status.PAUSED) { + return getCurrentOffsets(location, id, true); + } + + final Duration delay = retryPolicy.getAndIncrementRetryDelay(); + if (delay == null) { + throw new ISE("Task [%s] failed to pause, aborting", id); + } 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() + ); + Thread.sleep(sleepTime); + } + } + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + public KafkaIndexTask.Status getStatus(TaskLocation location, String id) + { + try { + final URI serviceUri = new URI( + "http", + null, + location.getHost(), + location.getPort(), + String.format("%s/%s/%s", BASE_PATH, id, "status"), + null, + null + ); + final StatusResponseHolder response = submitRequest(new Request(HttpMethod.GET, serviceUri.toURL()), true); + + return jsonMapper.readValue(response.getContent(), KafkaIndexTask.Status.class); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + public DateTime getStartTime(TaskLocation location, String id) + { + try { + final URI serviceUri = new URI( + "http", + null, + location.getHost(), + location.getPort(), + String.format("%s/%s/%s", BASE_PATH, id, "time/start"), + null, + null + ); + final StatusResponseHolder response = submitRequest(new Request(HttpMethod.GET, serviceUri.toURL()), true); + + return response.getContent() == null || response.getContent().isEmpty() + ? null + : jsonMapper.readValue(response.getContent(), DateTime.class); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + public Map getCurrentOffsets(TaskLocation location, String id, boolean retry) + { + try { + final URI serviceUri = new URI( + "http", + null, + location.getHost(), + location.getPort(), + String.format("%s/%s/%s", BASE_PATH, id, "offsets/current"), + null, + null + ); + final StatusResponseHolder response = submitRequest(new Request(HttpMethod.GET, serviceUri.toURL()), retry); + + return jsonMapper.readValue(response.getContent(), new TypeReference>() {}); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + public Map getEndOffsets(TaskLocation location, String id) + { + try { + final URI serviceUri = new URI( + "http", + null, + location.getHost(), + location.getPort(), + String.format("%s/%s/%s", BASE_PATH, id, "offsets/end"), + null, + null + ); + final StatusResponseHolder response = submitRequest(new Request(HttpMethod.GET, serviceUri.toURL()), true); + + return jsonMapper.readValue(response.getContent(), new TypeReference>() {}); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + public void setEndOffsets(TaskLocation location, String id, Map endOffsets) + { + setEndOffsets(location, id, endOffsets, false); + } + + public void setEndOffsets(TaskLocation location, String id, Map endOffsets, boolean resume) + { + try { + final URI serviceUri = new URI( + "http", + null, + location.getHost(), + location.getPort(), + String.format("%s/%s/%s", BASE_PATH, id, "offsets/end"), + resume ? "resume=true" : null, + null + ); + submitRequest( + new Request(HttpMethod.POST, serviceUri.toURL()).setContent( + MediaType.APPLICATION_JSON, + jsonMapper.writeValueAsBytes(endOffsets) + ), true + ); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + private StatusResponseHolder submitRequest(Request request, boolean retry) + { + final RetryPolicy retryPolicy = retryPolicyFactory.makeRetryPolicy(); + while (true) { + StatusResponseHolder response = null; + + try { + // Netty throws some annoying exceptions if a connection can't be opened, which happens relatively frequently + // for tasks that happen to still be starting up, so test the connection first to keep the logs clean. + new Socket(request.getUrl().getHost(), request.getUrl().getPort()).close(); + + try { + response = httpClient.go(request, new StatusResponseHandler(Charsets.UTF_8)).get(); + } + catch (Exception e) { + Throwables.propagateIfInstanceOf(e.getCause(), IOException.class); + Throwables.propagateIfInstanceOf(e.getCause(), ChannelException.class); + throw Throwables.propagate(e); + } + + int responseCode = response.getStatus().getCode(); + if (responseCode / 100 == 2) { + return response; + } else if (responseCode == 400) { // don't bother retrying if it's a bad request + throw new IAE("Received 400 Bad Request with body: %s", response.getContent()); + } else { + throw new IOException(String.format("Received status [%d] with: %s", responseCode, response.getContent())); + } + } + catch (IOException | ChannelException e) { + final Duration delay = retryPolicy.getAndIncrementRetryDelay(); + if (!retry || delay == null) { + 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.getUrl(), + new Duration(sleepTime).toString(), + (response != null ? response.getContent() : "[empty]") + ); + Thread.sleep(sleepTime); + } + catch (InterruptedException e2) { + Throwables.propagate(e2); + } + } + } + } + } +} 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 cce67287be50..f43c8b6861d4 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 @@ -24,6 +24,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.initialization.DruidModule; import java.util.List; @@ -39,7 +40,8 @@ public List getJacksonModules() new NamedType(KafkaIndexTask.class, "index_kafka"), new NamedType(KafkaDataSourceMetadata.class, "kafka"), new NamedType(KafkaIOConfig.class, "kafka"), - new NamedType(KafkaTuningConfig.class, "kafka") + new NamedType(KafkaTuningConfig.class, "kafka"), + new NamedType(KafkaSupervisorSpec.class, "kafka") ) ); } 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 new file mode 100644 index 000000000000..076b8e2cca02 --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -0,0 +1,1284 @@ +/* + * 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.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.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.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.MoreExecutors; +import com.metamx.common.ISE; +import com.metamx.emitter.EmittingLogger; +import io.druid.concurrent.Execs; +import io.druid.indexing.common.TaskLocation; +import io.druid.indexing.common.TaskStatus; +import io.druid.indexing.common.task.Task; +import io.druid.indexing.common.task.TaskResource; +import io.druid.indexing.kafka.KafkaDataSourceMetadata; +import io.druid.indexing.kafka.KafkaIOConfig; +import io.druid.indexing.kafka.KafkaIndexTask; +import io.druid.indexing.kafka.KafkaIndexTaskClient; +import io.druid.indexing.kafka.KafkaPartitions; +import io.druid.indexing.overlord.DataSourceMetadata; +import io.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import io.druid.indexing.overlord.TaskMaster; +import io.druid.indexing.overlord.TaskQueue; +import io.druid.indexing.overlord.TaskRunner; +import io.druid.indexing.overlord.TaskRunnerListener; +import io.druid.indexing.overlord.TaskRunnerWorkItem; +import io.druid.indexing.overlord.TaskStorage; +import io.druid.indexing.overlord.supervisor.Supervisor; +import io.druid.indexing.overlord.supervisor.SupervisorReport; +import io.druid.metadata.EntryExistsException; +import org.apache.commons.codec.digest.DigestUtils; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.joda.time.DateTime; + +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Random; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +/** + * Supervisor responsible for managing the KafkaIndexTasks for a single dataSource. At a high level, the class accepts a + * {@link KafkaSupervisorSpec} which includes the Kafka topic and configuration as well as an ingestion spec which will + * be used to generate the indexing tasks. The run loop periodically refreshes its view of the Kafka topic's partitions + * and the list of running indexing tasks and ensures that all partitions are being read from and that there are enough + * tasks to satisfy the desired number of replicas. As tasks complete, new tasks are queued to process the next range of + * Kafka offsets. + */ +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 + + // Internal data structures + // -------------------------------------------------------- + + /** + * A TaskGroup is the main data structure used by KafkaSupervisor to organize and monitor Kafka partitions and + * indexing tasks. All the tasks in a TaskGroup should always be doing the same thing (reading the same partitions and + * starting from the same offset) and if [replicas] is configured to be 1, a TaskGroup will contain a single task (the + * exception being if the supervisor started up and discovered and adopted some already running tasks). At any given + * time, there should only be up to a maximum of [taskCount] actively-reading task groups (tracked in the [taskGroups] + * map) + zero or more pending-completion task groups (tracked in [pendingCompletionTaskGroups]). + */ + private class TaskGroup + { + // This specifies the partitions and starting offsets for this task group. It is set on group creation from the data + // in [partitionGroups] and never changes during the lifetime of this task group, which will live until a task in + // 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 Map tasks = new HashMap<>(); + DateTime completionTimeout; // is set after signalTasksToFinish(); if not done by timeout, take corrective action + + public TaskGroup(Map partitionOffsets) + { + this.partitionOffsets = partitionOffsets; + } + } + + private class TaskData + { + TaskStatus status; + DateTime startTime; + } + + // Map<{group ID}, {actively reading task group}>; see documentation for TaskGroup class + private final HashMap taskGroups = new HashMap<>(); + + // 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<>(); + + // 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. + // Map<{group ID}, Map<{partition ID}, {startingOffset}>> + private Map> partitionGroups = new HashMap<>(); + // -------------------------------------------------------- + + private final TaskStorage taskStorage; + private final TaskMaster taskMaster; + private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; + private final KafkaIndexTaskClient taskClient; + private final ObjectMapper sortingMapper; + private final KafkaSupervisorSpec spec; + private final String dataSource; + private final KafkaSupervisorIOConfig ioConfig; + private final String supervisorId; + + private final ExecutorService exec; + private final ScheduledExecutorService scheduledExec; + private final BlockingQueue notices = new LinkedBlockingDeque<>(); + private final Object stopLock = new Object(); + + private boolean listenerRegistered = false; + private long lastRunTime; + + private volatile DateTime firstRunTime; + private volatile KafkaConsumer consumer; + private volatile boolean started = false; + private volatile boolean stopped = false; + + public KafkaSupervisor( + TaskStorage taskStorage, + TaskMaster taskMaster, + IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, + KafkaIndexTaskClient taskClient, + ObjectMapper mapper, + KafkaSupervisorSpec spec + ) + { + this.taskStorage = taskStorage; + this.taskMaster = taskMaster; + this.indexerMetadataStorageCoordinator = indexerMetadataStorageCoordinator; + this.taskClient = taskClient; + this.sortingMapper = mapper.copy().configure(MapperFeature.SORT_PROPERTIES_ALPHABETICALLY, true); + this.spec = spec; + + this.dataSource = spec.getDataSchema().getDataSource(); + this.ioConfig = spec.getIoConfig(); + this.supervisorId = String.format("KafkaSupervisor-%s", dataSource); + this.exec = Execs.singleThreaded(supervisorId + "-%d"); + this.scheduledExec = Execs.scheduledSingleThreaded(supervisorId + "-Scheduler-%d"); + } + + @Override + public void start() + { + Preconditions.checkState(!started, "already started"); + Preconditions.checkState(!exec.isShutdown(), "already stopped"); + + try { + consumer = getKafkaConsumer(); + + exec.submit( + new Runnable() + { + @Override + public void run() + { + try { + while (!Thread.currentThread().isInterrupted()) { + final Notice notice = notices.take(); + + try { + notice.handle(); + } + catch (Exception e) { + log.makeAlert(e, "KafkaSupervisor[%s] failed to handle notice", dataSource) + .addData("noticeClass", notice.getClass().getSimpleName()) + .emit(); + } + } + } + catch (InterruptedException e) { + log.info("KafkaSupervisor[%s] interrupted, exiting", dataSource); + } + } + } + ); + } + catch (Exception e) { + log.makeAlert(e, "Exception starting KafkaSupervisor[%s]", dataSource) + .emit(); + throw Throwables.propagate(e); + } + + firstRunTime = DateTime.now().plus(ioConfig.getStartDelay()); + scheduledExec.scheduleAtFixedRate( + buildRunTask(), + ioConfig.getStartDelay().getMillis(), + Math.max(ioConfig.getPeriod().getMillis(), MAX_RUN_FREQUENCY_MILLIS), + TimeUnit.MILLISECONDS + ); + + started = true; + log.info("Started KafkaSupervisor[%s], first run in [%s]", dataSource, ioConfig.getStartDelay()); + } + + @Override + public void stop(boolean stopGracefully) + { + Preconditions.checkState(started, "not started"); + + log.info("Beginning shutdown of KafkaSupervisor[%s]", dataSource); + + try { + scheduledExec.shutdownNow(); // stop recurring executions + + Optional taskRunner = taskMaster.getTaskRunner(); + if (taskRunner.isPresent()) { + taskRunner.get().unregisterListener(supervisorId); + } + + // Stopping gracefully will synchronize the end offsets of the tasks and signal them to publish, and will block + // until the tasks have acknowledged or timed out. We want this behavior when we're explicitly shut down through + // the API, but if we shut down for other reasons (e.g. we lose leadership) we want to just stop and leave the + // tasks as they are. + if (stopGracefully) { + log.info("Stopping gracefully, signalling managed tasks to complete and publish"); + synchronized (stopLock) { + notices.add(new ShutdownNotice()); + while (!stopped) { + stopLock.wait(); + } + } + log.info("Shutdown notice handled"); + } + + exec.shutdownNow(); + consumer.close(); + started = false; + + log.info("KafkaSupervisor[%s] has stopped", dataSource); + } + catch (Exception e) { + log.makeAlert(e, "Exception stopping KafkaSupervisor[%s]", dataSource) + .emit(); + } + } + + @Override + public SupervisorReport getStatus() + { + return generateReport(true); + } + + public void possiblyRegisterListener() + { + // getTaskRunner() sometimes fails if the task queue is still being initialized so retry later until we succeed + + if (listenerRegistered) { + return; + } + + Optional taskRunner = taskMaster.getTaskRunner(); + if (taskRunner.isPresent()) { + taskRunner.get().registerListener( + new TaskRunnerListener() + { + @Override + public String getListenerId() + { + return supervisorId; + } + + @Override + public void locationChanged(final String taskId, final TaskLocation newLocation) + { + // do nothing + } + + @Override + public void statusChanged(String taskId, TaskStatus status) + { + notices.add(new RunNotice()); + } + }, MoreExecutors.sameThreadExecutor() + ); + + listenerRegistered = true; + } + } + + private interface Notice + { + void handle(); + } + + private class RunNotice implements Notice + { + @Override + public void handle() + { + long nowTime = System.currentTimeMillis(); + if (nowTime - lastRunTime < MAX_RUN_FREQUENCY_MILLIS) { + return; + } + lastRunTime = nowTime; + + runInternal(); + } + } + + private class ShutdownNotice implements Notice + { + @Override + public void handle() + { + // 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 (getTaskLocation(entry.getKey()) == null) { + killTask(entry.getKey()); + } else { + entry.getValue().startTime = new DateTime(0); + } + } + } + + checkTaskDuration(); + + synchronized (stopLock) { + stopped = true; + stopLock.notifyAll(); + } + } + } + + @VisibleForTesting + void runInternal() + { + possiblyRegisterListener(); + updatePartitionDataFromKafka(); + discoverTasks(); + updateTaskStatus(); + checkTaskDuration(); + checkPendingCompletionTasks(); + checkCurrentTaskState(); + createNewTasks(); + + if (log.isDebugEnabled()) { + log.debug(generateReport(true).toString()); + } else { + log.info(generateReport(false).toString()); + } + } + + @VisibleForTesting + String generateSequenceName(int groupId) + { + StringBuilder sb = new StringBuilder(); + Map startPartitions = taskGroups.get(groupId).partitionOffsets; + + for (Map.Entry entry : startPartitions.entrySet()) { + sb.append(String.format("+%d(%d)", entry.getKey(), entry.getValue())); + } + String partitionOffsetStr = sb.toString().substring(1); + + String dataSchema, tuningConfig; + try { + dataSchema = sortingMapper.writeValueAsString(spec.getDataSchema()); + tuningConfig = sortingMapper.writeValueAsString(spec.getTuningConfig()); + } + catch (JsonProcessingException e) { + throw Throwables.propagate(e); + } + + String hashCode = DigestUtils.sha1Hex(dataSchema + tuningConfig + partitionOffsetStr).substring(0, 15); + + return Joiner.on("_").join("index_kafka", dataSource, hashCode); + } + + private static String getRandomId() + { + final StringBuilder suffix = new StringBuilder(8); + for (int i = 0; i < Ints.BYTES * 2; ++i) { + suffix.append((char) ('a' + ((RANDOM.nextInt() >>> (i * 4)) & 0x0F))); + } + return suffix.toString(); + } + + private KafkaConsumer getKafkaConsumer() + { + final Properties props = new Properties(); + props.putAll(ioConfig.getConsumerProperties()); + + props.setProperty("enable.auto.commit", "false"); + props.setProperty("metadata.max.age.ms", "10000"); + props.setProperty("group.id", String.format("kafka-supervisor-%s", getRandomId())); + + ClassLoader currCtxCl = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(getClass().getClassLoader()); + return new KafkaConsumer<>(props, new ByteArrayDeserializer(), new ByteArrayDeserializer()); + } + finally { + Thread.currentThread().setContextClassLoader(currCtxCl); + } + } + + private void updatePartitionDataFromKafka() + { + Map> topics; + try { + topics = consumer.listTopics(); // updates the consumer's list of partitions from the brokers + } + catch (Exception e) { // calls to the consumer throw NPEs when the broker doesn't respond + log.warn( + e, + "Unable to get partition data from Kafka for brokers [%s], are the brokers up?", + ioConfig.getConsumerProperties().get(KafkaSupervisorIOConfig.BOOTSTRAP_SERVERS_KEY) + ); + return; + } + + List partitions = topics.get(ioConfig.getTopic()); + int numPartitions = (partitions != null ? partitions.size() : 0); + + log.debug("Found [%d] Kafka partitions for topic [%s]", numPartitions, ioConfig.getTopic()); + + for (int partition = 0; partition < numPartitions; partition++) { + int taskGroupId = getTaskGroupIdForPartition(partition); + + if (partitionGroups.get(taskGroupId) == null) { + partitionGroups.put(taskGroupId, new HashMap()); + } + + Map partitionMap = partitionGroups.get(taskGroupId); + if (!partitionMap.containsKey(partition)) { + log.info( + "New partition [%d] discovered for topic [%s], adding 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() + { + int taskCount = 0; + 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(); + + // determine which task group this task belongs to and do a consistency check on partitions + Integer taskGroupId = null; + for (Integer partition : kafkaTask.getIOConfig().getStartPartitions().getPartitionOffsetMap().keySet()) { + if (taskGroupId == null) { + taskGroupId = getTaskGroupIdForPartition(partition); + } else if (!taskGroupId.equals(getTaskGroupIdForPartition(partition))) { + log.warn("Stopping task [%s] which does not match the expected partition allocation", taskId); + stopTask(taskId, false); + taskGroupId = null; // prevents the next block of code from adding the item to taskGroups + break; + } + } + + 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() + ); + + // 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); + Map partitionOffsets = partitionGroups.get(taskGroupId); + + for (Map.Entry entry : publishingTaskCurrentOffsets.entrySet()) { + Integer partition = entry.getKey(); + Long offset = entry.getValue(); + if (partitionOffsets.get(partition) == null || partitionOffsets.get(partition) < offset) { + partitionOffsets.put(partition, offset); + } + } + + } else { + if (!taskGroups.containsKey(taskGroupId)) { + log.debug("Creating new task group [%d]", taskGroupId); + taskGroups.put( + taskGroupId, + new TaskGroup(kafkaTask.getIOConfig().getStartPartitions().getPartitionOffsetMap()) + ); + } + + taskGroups.get(taskGroupId).tasks.put(taskId, new TaskData()); + } + } + } + } + } + + log.debug("Found [%d] Kafka indexing tasks for dataSource [%s]", taskCount, dataSource); + } + + private void addDiscoveredTaskToPendingCompletionTaskGroups( + int groupId, + String taskId, + Map startingPartitions + ) + { + if (!pendingCompletionTaskGroups.containsKey(groupId)) { + pendingCompletionTaskGroups.put(groupId, Lists.newArrayList()); + } + + List 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()); + } + return; + } + } + + log.info("Creating new pending completion task group for discovered task [%s]", taskId); + TaskGroup newTaskGroup = new TaskGroup(startingPartitions); + newTaskGroup.tasks.put(taskId, new TaskData()); + newTaskGroup.completionTimeout = DateTime.now().plus(ioConfig.getCompletionTimeout()); + + taskGroupList.add(newTaskGroup); + } + + private void updateTaskStatus() + { + // 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(); + + 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); + } + } + + taskData.status = taskStorage.getStatus(taskId).get(); + } + } + + // update status of pending completion tasks in pendingCompletionTaskGroups + for (List taskGroups : pendingCompletionTaskGroups.values()) { + for (TaskGroup group : taskGroups) { + for (Map.Entry entry : group.tasks.entrySet()) { + entry.getValue().status = taskStorage.getStatus(entry.getKey()).get(); + } + } + } + } + + private void checkTaskDuration() + { + Iterator> i = taskGroups.entrySet().iterator(); + while (i.hasNext()) { + Map.Entry groupEntry = i.next(); + Integer groupId = groupEntry.getKey(); + TaskGroup group = groupEntry.getValue(); + + // find the longest running task from this group + DateTime earliestTaskStart = DateTime.now(); + for (TaskData taskData : group.tasks.values()) { + if (earliestTaskStart.isAfter(taskData.startTime)) { + earliestTaskStart = taskData.startTime; + } + } + + // 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); + + // 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.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()); + } + } + + // remove this task group from the list of current task groups now that it has been handled + i.remove(); + } + } + } + + private Map signalTasksToFinish(int groupId) + { + 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<>(); + Iterator> i = taskGroup.tasks.entrySet().iterator(); + while (i.hasNext()) { + Map.Entry taskEntry = i.next(); + String taskId = taskEntry.getKey(); + TaskData task = taskEntry.getValue(); + + if (task.status.isSuccess()) { + // If any task in this group has already completed, stop the rest of the tasks in the group and return. + // 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; + } + + if (task.status.isRunnable()) { + TaskLocation location = getTaskLocation(taskId); + if (location == null) { + log.info("Killing task [%s] which hasn't been assigned to a worker", taskId); + killTask(taskId); + } else { + Map currentOffsets; + try { + currentOffsets = taskClient.pause(location, 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) 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()) { + TaskLocation location = getTaskLocation(taskId); + if (location != null) { + try { + taskClient.setEndOffsets(location, 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; + } + + /** + * Monitors [pendingCompletionTaskGroups] for tasks that have completed. If any task in a task group has completed, we + * can safely stop the rest of the tasks in that group. If a task group has exceeded its publishing timeout, then + * we need to stop all tasks in not only that task group but also 1) any subsequent task group that is also pending + * completion and 2) the current task group that is running, because the assumption that we have handled up to the + * 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() + { + for (Map.Entry> pendingGroupList : pendingCompletionTaskGroups.entrySet()) { + + boolean stopTasksInTaskGroup = false; + Integer groupId = pendingGroupList.getKey(); + Iterator iTaskGroup = pendingGroupList.getValue().iterator(); + while (iTaskGroup.hasNext()) { + 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(); + continue; + } + + Iterator> iTask = group.tasks.entrySet().iterator(); + while (iTask.hasNext()) { + Map.Entry task = iTask.next(); + + if (task.getValue().status.isFailure()) { + iTask.remove(); // remove failed task + if (group.tasks.isEmpty()) { + // if all tasks in the group have failed, just nuke all task groups with this partition set and restart + entireTaskGroupFailed = true; + break; + } + } + + if (task.getValue().status.isSuccess()) { + // 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); + foundSuccess = true; + iTaskGroup.remove(); // 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 + } + } + + if ((!foundSuccess && group.completionTimeout.isBeforeNow()) || entireTaskGroupFailed) { + if (entireTaskGroupFailed) { + log.warn("All tasks in group [%d] failed to publish, killing all tasks for these partitions", groupId); + } else { + log.makeAlert( + "No task in [%s] succeeded before the completion timeout elapsed [%s]!", + group.tasks.keySet(), + ioConfig.getCompletionTimeout() + ).emit(); + } + + // reset partitions offsets for this task group so that they will be re-read from metadata storage + partitionGroups.remove(groupId); + + // stop all the tasks in this pending completion group + 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)); + + iTaskGroup.remove(); + } + } + } + } + + private void checkCurrentTaskState() + { + Iterator> iTaskGroups = taskGroups.entrySet().iterator(); + while (iTaskGroups.hasNext()) { + Map.Entry taskGroupEntry = iTaskGroups.next(); + Integer groupId = taskGroupEntry.getKey(); + TaskGroup taskGroup = taskGroupEntry.getValue(); + + // Iterate the list of known tasks in this group and: + // 1) Kill any tasks which are not "current" (have the partitions and starting offsets in [taskGroups] + // 2) Remove any tasks that have failed from the list + // 3) If any task completed successfully, stop all the tasks in this group and move to the next group + + log.debug("Task group [%d] pre-pruning: %s", groupId, taskGroup.tasks.keySet()); + + Iterator> iTasks = taskGroup.tasks.entrySet().iterator(); + while (iTasks.hasNext()) { + Map.Entry task = iTasks.next(); + String taskId = task.getKey(); + TaskData taskData = task.getValue(); + + // 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); + iTasks.remove(); + continue; + } + + // remove failed tasks + if (taskData.status.isFailure()) { + iTasks.remove(); + continue; + } + + // 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); + iTaskGroups.remove(); + break; + } + } + log.debug("Task group [%d] post-pruning: %s", groupId, taskGroup.tasks.keySet()); + } + } + + void createNewTasks() + { + // check that there is a current task group for each group of partitions in [partitionGroups] + for (Integer groupId : partitionGroups.keySet()) { + if (!taskGroups.containsKey(groupId)) { + log.info("Creating new task group [%d] for partitions %s", groupId, partitionGroups.get(groupId).keySet()); + taskGroups.put(groupId, new TaskGroup(generateStartingOffsetsForPartitionGroup(groupId))); + } + } + + // iterate through all the current task groups and make sure each one has the desired number of replica tasks + boolean createdTask = false; + for (Map.Entry entry : taskGroups.entrySet()) { + TaskGroup taskGroup = entry.getValue(); + Integer groupId = entry.getKey(); + + if (ioConfig.getReplicas() > taskGroup.tasks.size()) { + log.info( + "Number of tasks [%d] does not match configured numReplicas [%d] in task group [%d], creating more tasks", + taskGroup.tasks.size(), ioConfig.getReplicas(), groupId + ); + createKafkaTasksForGroup(groupId, ioConfig.getReplicas() - taskGroup.tasks.size()); + createdTask = true; + } + } + + if (createdTask && firstRunTime.isBeforeNow()) { + // Schedule a run event after a short delay to update our internal data structures with the new tasks that were + // just created. This is mainly for the benefit of the status API in situations where the run period is lengthy. + scheduledExec.schedule(buildRunTask(), 5000, TimeUnit.MILLISECONDS); + } + } + + private void createKafkaTasksForGroup(int groupId, int replicas) + { + Map startPartitions = taskGroups.get(groupId).partitionOffsets; + Map endPartitions = new HashMap<>(); + for (Integer partition : startPartitions.keySet()) { + endPartitions.put(partition, Long.MAX_VALUE); + } + + String sequenceName = generateSequenceName(groupId); + + Map consumerProperties = Maps.newHashMap(ioConfig.getConsumerProperties()); + + KafkaIOConfig kafkaIOConfig = new KafkaIOConfig( + sequenceName, + new KafkaPartitions(ioConfig.getTopic(), startPartitions), + new KafkaPartitions(ioConfig.getTopic(), endPartitions), + consumerProperties, + true, + false + ); + + for (int i = 0; i < replicas; i++) { + String taskId = Joiner.on("_").join(sequenceName, getRandomId()); + KafkaIndexTask indexTask = new KafkaIndexTask( + taskId, + new TaskResource(sequenceName, 1), + spec.getDataSchema(), + spec.getTuningConfig(), + kafkaIOConfig, + ImmutableMap.of(), + null + ); + + Optional taskQueue = taskMaster.getTaskQueue(); + if (taskQueue.isPresent()) { + try { + taskQueue.get().add(indexTask); + } + catch (EntryExistsException e) { + log.error("Tried to add task [%s] but it already exists", indexTask.getId()); + } + } else { + log.error("Failed to get task queue because I'm not the leader!"); + } + } + } + + private Map generateStartingOffsetsForPartitionGroup(int groupId) + { + Map startingOffsets = new HashMap<>(); + for (Map.Entry entry : partitionGroups.get(groupId).entrySet()) { + Integer partition = entry.getKey(); + Long offset = entry.getValue(); + + if (offset != null) { + // if we are given a startingOffset (set by a previous task group which is pending completion) then use it + startingOffsets.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)); + } + } + return startingOffsets; + } + + /** + * Queries the dataSource metadata table to see if there is a previous ending offset for this partition. If it doesn't + * find any data, it will retrieve the latest or earliest Kafka offset depending on the useEarliestOffset config. + */ + private long getOffsetFromStorageForPartition(int partition) + { + long offset; + Map metadataOffsets = getOffsetsFromMetadataStorage(); + if (metadataOffsets.get(partition) != null) { + offset = metadataOffsets.get(partition); + log.debug("Getting offset [%,d] from metadata storage for partition [%d]", offset, partition); + + long latestKafkaOffset = getOffsetFromKafkaForPartition(partition, false); + if (offset > latestKafkaOffset) { + throw new ISE( + "Offset in metadata storage [%,d] > latest Kafka offset [%,d] for partition [%d]. If your Kafka offsets have" + + " been reset, you will need to remove the entry for [%s] from the dataSource table.", + offset, + latestKafkaOffset, + partition, + dataSource + ); + } + + } else { + offset = getOffsetFromKafkaForPartition(partition, ioConfig.isUseEarliestOffset()); + log.debug("Getting offset [%,d] from Kafka for partition [%d]", offset, partition); + } + + return offset; + } + + private Map getOffsetsFromMetadataStorage() + { + DataSourceMetadata dataSourceMetadata = indexerMetadataStorageCoordinator.getDataSourceMetadata(dataSource); + if (dataSourceMetadata != null && dataSourceMetadata instanceof KafkaDataSourceMetadata) { + KafkaPartitions partitions = ((KafkaDataSourceMetadata) dataSourceMetadata).getKafkaPartitions(); + if (partitions != null) { + if (!ioConfig.getTopic().equals(partitions.getTopic())) { + log.warn( + "Topic in metadata storage [%s] doesn't match spec topic [%s], ignoring stored offsets", + partitions.getTopic(), + ioConfig.getTopic() + ); + return ImmutableMap.of(); + } else if (partitions.getPartitionOffsetMap() != null) { + return partitions.getPartitionOffsetMap(); + } + } + } + + return ImmutableMap.of(); + } + + private long getOffsetFromKafkaForPartition(int partition, boolean useEarliestOffset) + { + TopicPartition topicPartition = new TopicPartition(ioConfig.getTopic(), partition); + if (!consumer.assignment().contains(topicPartition)) { + consumer.assign(Lists.newArrayList(topicPartition)); + } + + if (useEarliestOffset) { + consumer.seekToBeginning(topicPartition); + } else { + consumer.seekToEnd(topicPartition); + } + + return consumer.position(topicPartition); + } + + /** + * Compares the sequence name from the task with one generated for the task's group ID and returns false if they do + * not match. The sequence name is generated from a hash of the dataSchema, tuningConfig, and starting offsets. + */ + private boolean isTaskCurrent(int taskGroupId, String taskId) + { + Optional taskOptional = taskStorage.getTask(taskId); + if (!taskOptional.isPresent() || !(taskOptional.get() instanceof KafkaIndexTask)) { + return false; + } + + String taskSequenceName = ((KafkaIndexTask) taskOptional.get()).getIOConfig().getBaseSequenceName(); + + return generateSequenceName(taskGroupId).equals(taskSequenceName); + } + + private void stopTasksInGroup(TaskGroup taskGroup) + { + if (taskGroup == null) { + return; + } + + for (Map.Entry entry : taskGroup.tasks.entrySet()) { + if (!entry.getValue().status.isComplete()) { + stopTask(entry.getKey(), false); + } + } + } + + private void stopTask(final String id, final boolean publish) + { + TaskLocation taskLocation = getTaskLocation(id); + if (taskLocation != null) { + try { + taskClient.stop(taskLocation, 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); + } + } + + private void killTask(final String id) + { + Optional taskQueue = taskMaster.getTaskQueue(); + if (taskQueue.isPresent()) { + taskQueue.get().shutdown(id); + } else { + log.error("Failed to get task queue because I'm not the leader!"); + } + } + + private DateTime getTaskStartTime(final String id) + { + DateTime startTime = null; + TaskLocation taskLocation = getTaskLocation(id); + if (taskLocation != null) { + startTime = taskClient.getStartTime(taskLocation, id); + } + + log.debug("Received start time of [%s] from task [%s]", startTime, id); + return startTime; + } + + private Optional getTaskStatus(final String id) + { + TaskLocation taskLocation = getTaskLocation(id); + if (taskLocation != null) { + try { + return Optional.of(taskClient.getStatus(taskLocation, 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) + { + TaskLocation taskLocation = getTaskLocation(id); + if (taskLocation != null) { + try { + return taskClient.getCurrentOffsets(taskLocation, 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 TaskLocation getTaskLocation(final String id) + { + Preconditions.checkNotNull(id, "id"); + Optional taskRunner = taskMaster.getTaskRunner(); + if (taskRunner.isPresent()) { + Optional item = Iterables.tryFind( + taskRunner.get().getRunningTasks(), new Predicate() + { + @Override + public boolean apply(TaskRunnerWorkItem taskRunnerWorkItem) + { + return id.equals(taskRunnerWorkItem.getTaskId()); + } + } + ); + + if (item.isPresent() && item.get().getLocation() != TaskLocation.unknown()) { + return item.get().getLocation(); + } + } else { + log.error("Failed to get task runner because I'm not the leader!"); + } + + return null; + } + + private int getTaskGroupIdForPartition(int partition) + { + return partition % ioConfig.getTaskCount(); + } + + private boolean isTaskInPendingCompletionGroups(String taskId) + { + for (List taskGroups : pendingCompletionTaskGroups.values()) { + for (TaskGroup taskGroup : taskGroups) { + if (taskGroup.tasks.containsKey(taskId)) { + return true; + } + } + } + return false; + } + + private KafkaSupervisorReport generateReport(boolean includeOffsets) + { + int numPartitions = 0; + for (Map partitionGroup : partitionGroups.values()) { + numPartitions += partitionGroup.size(); + } + + KafkaSupervisorReport report = new KafkaSupervisorReport( + dataSource, + DateTime.now(), + ioConfig.getTopic(), + numPartitions, + ioConfig.getReplicas(), + ioConfig.getTaskDuration().getMillis() / 1000 + ); + + try { + for (TaskGroup taskGroup : taskGroups.values()) { + for (Map.Entry entry : taskGroup.tasks.entrySet()) { + String taskId = entry.getKey(); + DateTime startTime = entry.getValue().startTime; + Long remainingSeconds = null; + if (startTime != null) { + remainingSeconds = Math.max( + 0, + ioConfig.getTaskDuration().getMillis() - (DateTime.now().getMillis() - startTime + .getMillis()) + ) / 1000; + } + + report.addActiveTask( + taskId, + (includeOffsets ? taskGroup.partitionOffsets : null), + (includeOffsets ? getCurrentOffsets(taskId, false) : null), + startTime, + remainingSeconds + ); + } + } + + for (List taskGroups : pendingCompletionTaskGroups.values()) { + for (TaskGroup taskGroup : taskGroups) { + for (Map.Entry entry : taskGroup.tasks.entrySet()) { + String taskId = entry.getKey(); + DateTime startTime = entry.getValue().startTime; + Long remainingSeconds = null; + if (taskGroup.completionTimeout != null) { + remainingSeconds = Math.max(0, taskGroup.completionTimeout.getMillis() - DateTime.now().getMillis()) + / 1000; + } + + report.addPublishingTask( + taskId, + (includeOffsets ? taskGroup.partitionOffsets : null), + (includeOffsets ? getCurrentOffsets(taskId, false) : null), + startTime, + remainingSeconds + ); + } + } + } + } + catch (Exception e) { + log.warn(e, "Failed to generate status report"); + } + + return report; + } + + private Runnable buildRunTask() + { + return new Runnable() + { + @Override + public void run() + { + notices.add(new RunNotice()); + } + }; + } +} 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 new file mode 100644 index 000000000000..b53427d1506b --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java @@ -0,0 +1,131 @@ +/* + * 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.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.joda.time.Duration; +import org.joda.time.Period; + +import java.util.Map; + +public class KafkaSupervisorIOConfig +{ + public static final String BOOTSTRAP_SERVERS_KEY = "bootstrap.servers"; + + private final String topic; + private final Integer replicas; + private final Integer taskCount; + private final Duration taskDuration; + private final Map consumerProperties; + private final Duration startDelay; + private final Duration period; + private final Boolean useEarliestOffset; + private final Duration completionTimeout; + + @JsonCreator + public KafkaSupervisorIOConfig( + @JsonProperty("topic") String topic, + @JsonProperty("replicas") Integer replicas, + @JsonProperty("taskCount") Integer taskCount, + @JsonProperty("taskDuration") Period taskDuration, + @JsonProperty("consumerProperties") Map consumerProperties, + @JsonProperty("startDelay") Period startDelay, + @JsonProperty("period") Period period, + @JsonProperty("useEarliestOffset") Boolean useEarliestOffset, + @JsonProperty("completionTimeout") Period completionTimeout + ) + { + this.topic = Preconditions.checkNotNull(topic, "topic"); + this.consumerProperties = Preconditions.checkNotNull(consumerProperties, "consumerProperties"); + Preconditions.checkNotNull( + consumerProperties.get(BOOTSTRAP_SERVERS_KEY), + String.format("consumerProperties must contain entry for [%s]", BOOTSTRAP_SERVERS_KEY) + ); + + this.replicas = (replicas != null ? replicas : 1); + this.taskCount = (taskCount != null ? taskCount : 1); + this.taskDuration = defaultDuration(taskDuration, "PT1H"); + this.startDelay = defaultDuration(startDelay, "PT5S"); + this.period = defaultDuration(period, "PT30S"); + this.useEarliestOffset = (useEarliestOffset != null ? useEarliestOffset : false); + this.completionTimeout = defaultDuration(completionTimeout, "PT30M"); + } + + @JsonProperty + public String getTopic() + { + return topic; + } + + @JsonProperty + public Integer getReplicas() + { + return replicas; + } + + @JsonProperty + public Integer getTaskCount() + { + return taskCount; + } + + @JsonProperty + public Duration getTaskDuration() + { + return taskDuration; + } + + @JsonProperty + public Map getConsumerProperties() + { + return consumerProperties; + } + + @JsonProperty + public Duration getStartDelay() + { + return startDelay; + } + + @JsonProperty + public Duration getPeriod() + { + return period; + } + + @JsonProperty + public Boolean isUseEarliestOffset() + { + return useEarliestOffset; + } + + @JsonProperty + public Duration getCompletionTimeout() + { + return completionTimeout; + } + + 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 new file mode 100644 index 000000000000..cf08bd3103e9 --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorReport.java @@ -0,0 +1,234 @@ +/* + * 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 com.google.common.collect.Lists; +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; + private final String topic; + private final Integer partitions; + private final Integer replicas; + private final Long durationSeconds; + private final List activeTasks; + private final List publishingTasks; + + public KafkaSupervisorReportPayload( + String dataSource, + String topic, + Integer partitions, + Integer replicas, + Long durationSeconds + ) + { + this.dataSource = dataSource; + this.topic = topic; + this.partitions = partitions; + this.replicas = replicas; + this.durationSeconds = durationSeconds; + this.activeTasks = Lists.newArrayList(); + this.publishingTasks = Lists.newArrayList(); + } + + @JsonProperty + public String getDataSource() + { + return dataSource; + } + + @JsonProperty + public String getTopic() + { + return topic; + } + + @JsonProperty + public Integer getPartitions() + { + return partitions; + } + + @JsonProperty + public Integer getReplicas() + { + return replicas; + } + + @JsonProperty + public Long getDurationSeconds() + { + return durationSeconds; + } + + @JsonProperty + public List getActiveTasks() + { + return activeTasks; + } + + @JsonProperty + public List getPublishingTasks() + { + return publishingTasks; + } + + @Override + public String toString() + { + return "{" + + "dataSource='" + dataSource + '\'' + + ", topic='" + topic + '\'' + + ", partitions=" + partitions + + ", replicas=" + replicas + + ", durationSeconds=" + durationSeconds + + ", active=" + activeTasks + + ", publishing=" + publishingTasks + + '}'; + } + } + + private final KafkaSupervisorReportPayload payload; + + public KafkaSupervisorReport( + String dataSource, + DateTime generationTime, + String topic, + Integer partitions, + Integer replicas, + Long durationSeconds + ) + { + super(dataSource, generationTime); + this.payload = new KafkaSupervisorReportPayload(dataSource, topic, partitions, replicas, durationSeconds); + } + + @Override + public Object getPayload() + { + return payload; + } + + public void addActiveTask( + String id, + Map startingOffsets, + Map currentOffsets, + DateTime startTime, + Long remainingSeconds + ) + { + 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)); + } + + @Override + public String toString() + { + return "{" + + "id='" + getId() + '\'' + + ", generationTime=" + getGenerationTime() + + ", payload=" + payload + + '}'; + } +} 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 new file mode 100644 index 000000000000..1b3868eea0d0 --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java @@ -0,0 +1,109 @@ +/* + * 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.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import io.druid.indexing.kafka.KafkaIndexTaskClient; +import io.druid.indexing.kafka.KafkaTuningConfig; +import io.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import io.druid.indexing.overlord.TaskMaster; +import io.druid.indexing.overlord.TaskStorage; +import io.druid.indexing.overlord.supervisor.Supervisor; +import io.druid.indexing.overlord.supervisor.SupervisorSpec; +import io.druid.segment.indexing.DataSchema; + +public class KafkaSupervisorSpec implements SupervisorSpec +{ + private final DataSchema dataSchema; + private final KafkaTuningConfig tuningConfig; + private final KafkaSupervisorIOConfig ioConfig; + + private final TaskStorage taskStorage; + private final TaskMaster taskMaster; + private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; + private final KafkaIndexTaskClient kafkaIndexTaskClient; + private final ObjectMapper mapper; + + @JsonCreator + public KafkaSupervisorSpec( + @JsonProperty("dataSchema") DataSchema dataSchema, + @JsonProperty("tuningConfig") KafkaTuningConfig tuningConfig, + @JsonProperty("ioConfig") KafkaSupervisorIOConfig ioConfig, + @JacksonInject TaskStorage taskStorage, + @JacksonInject TaskMaster taskMaster, + @JacksonInject IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, + @JacksonInject KafkaIndexTaskClient kafkaIndexTaskClient, + @JacksonInject ObjectMapper mapper + ) + { + this.dataSchema = Preconditions.checkNotNull(dataSchema, "dataSchema"); + this.tuningConfig = tuningConfig != null + ? tuningConfig + : new KafkaTuningConfig(null, null, null, null, null, null, null, null, null); + this.ioConfig = Preconditions.checkNotNull(ioConfig, "ioConfig"); + + this.taskStorage = taskStorage; + this.taskMaster = taskMaster; + this.indexerMetadataStorageCoordinator = indexerMetadataStorageCoordinator; + this.kafkaIndexTaskClient = kafkaIndexTaskClient; + this.mapper = mapper; + } + + @JsonProperty + public DataSchema getDataSchema() + { + return dataSchema; + } + + @JsonProperty + public KafkaTuningConfig getTuningConfig() + { + return tuningConfig; + } + + @JsonProperty + public KafkaSupervisorIOConfig getIoConfig() + { + return ioConfig; + } + + @Override + public String getId() + { + return dataSchema.getDataSource(); + } + + @Override + public Supervisor createSupervisor() + { + return new KafkaSupervisor( + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + kafkaIndexTaskClient, + mapper, + this + ); + } +} 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 new file mode 100644 index 000000000000..fd381c2cf7fe --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -0,0 +1,1088 @@ +/* + * 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.ObjectMapper; +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.ListenableFuture; +import com.metamx.common.Granularity; +import com.metamx.common.ISE; +import io.druid.data.input.impl.DimensionSchema; +import io.druid.data.input.impl.DimensionsSpec; +import io.druid.data.input.impl.JSONParseSpec; +import io.druid.data.input.impl.JSONPathFieldSpec; +import io.druid.data.input.impl.JSONPathSpec; +import io.druid.data.input.impl.StringDimensionSchema; +import io.druid.data.input.impl.StringInputRowParser; +import io.druid.data.input.impl.TimestampSpec; +import io.druid.granularity.QueryGranularity; +import io.druid.indexing.common.TaskLocation; +import io.druid.indexing.common.TaskStatus; +import io.druid.indexing.common.task.RealtimeIndexTask; +import io.druid.indexing.common.task.Task; +import io.druid.indexing.kafka.KafkaDataSourceMetadata; +import io.druid.indexing.kafka.KafkaIOConfig; +import io.druid.indexing.kafka.KafkaIndexTask; +import io.druid.indexing.kafka.KafkaIndexTaskClient; +import io.druid.indexing.kafka.KafkaPartitions; +import io.druid.indexing.kafka.KafkaTuningConfig; +import io.druid.indexing.kafka.test.TestBroker; +import io.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import io.druid.indexing.overlord.TaskMaster; +import io.druid.indexing.overlord.TaskQueue; +import io.druid.indexing.overlord.TaskRunner; +import io.druid.indexing.overlord.TaskRunnerListener; +import io.druid.indexing.overlord.TaskRunnerWorkItem; +import io.druid.indexing.overlord.TaskStorage; +import io.druid.indexing.overlord.supervisor.SupervisorReport; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.segment.indexing.DataSchema; +import io.druid.segment.indexing.RealtimeIOConfig; +import io.druid.segment.indexing.granularity.UniformGranularitySpec; +import io.druid.segment.realtime.FireDepartment; +import org.apache.curator.test.TestingCluster; +import org.apache.kafka.clients.producer.KafkaProducer; +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.Interval; +import org.joda.time.Period; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; + +import java.io.File; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +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.expect; +import static org.easymock.EasyMock.expectLastCall; +import static org.easymock.EasyMock.replay; +import static org.easymock.EasyMock.reset; + +@RunWith(EasyMockRunner.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 TestingCluster zkServer; + private TestBroker kafkaServer; + private KafkaSupervisor supervisor; + private String kafkaHost; + private DataSchema dataSchema; + private KafkaTuningConfig tuningConfig; + + @Mock + 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(); + + @Before + public void setUp() throws Exception + { + zkServer = new TestingCluster(1); + zkServer.start(); + + kafkaServer = new TestBroker( + zkServer.getConnectString(), + tempFolder.newFolder(), + 1, + ImmutableMap.of("num.partitions", String.valueOf(NUM_PARTITIONS)) + ); + kafkaServer.start(); + kafkaHost = String.format("localhost:%d", kafkaServer.getPort()); + + dataSchema = getDataSchema(DATASOURCE); + tuningConfig = new KafkaTuningConfig( + 1000, + 50000, + new Period("P1Y"), + new File("/test"), + null, + null, + true, + false, + null + ); + } + + @After + public void tearDown() throws Exception + { + kafkaServer.close(); + kafkaServer = null; + + zkServer.stop(); + zkServer = null; + + supervisor = null; + } + + @Test + public void testNoInitialState() throws Exception + { + supervisor = getSupervisor(1, 1, true, "PT1H"); + addSomeEvents(1); + + Capture captured = Capture.newInstance(); + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + new KafkaDataSourceMetadata( + null + ) + ).anyTimes(); + expect(taskQueue.add(capture(captured))).andReturn(true); + taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + + KafkaIndexTask task = captured.getValue(); + Assert.assertEquals(dataSchema, task.getDataSchema()); + Assert.assertEquals(tuningConfig, task.getTuningConfig()); + + KafkaIOConfig taskConfig = task.getIOConfig(); + Assert.assertEquals(kafkaHost, taskConfig.getConsumerProperties().get("bootstrap.servers")); + Assert.assertEquals("myCustomValue", taskConfig.getConsumerProperties().get("myCustomKey")); + Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName()); + Assert.assertTrue("isUseTransaction", taskConfig.isUseTransaction()); + Assert.assertFalse("pauseAfterRead", taskConfig.isPauseAfterRead()); + + Assert.assertEquals(KAFKA_TOPIC, taskConfig.getStartPartitions().getTopic()); + Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(0)); + Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(1)); + Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(2)); + + Assert.assertEquals(KAFKA_TOPIC, taskConfig.getEndPartitions().getTopic()); + Assert.assertEquals(Long.MAX_VALUE, (long) taskConfig.getEndPartitions().getPartitionOffsetMap().get(0)); + Assert.assertEquals(Long.MAX_VALUE, (long) taskConfig.getEndPartitions().getPartitionOffsetMap().get(1)); + Assert.assertEquals(Long.MAX_VALUE, (long) taskConfig.getEndPartitions().getPartitionOffsetMap().get(2)); + } + + @Test + public void testMultiTask() throws Exception + { + supervisor = getSupervisor(1, 2, true, "PT1H"); + addSomeEvents(1); + + Capture captured = Capture.newInstance(CaptureType.ALL); + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).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(2); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + + KafkaIndexTask task1 = captured.getValues().get(0); + Assert.assertEquals(2, task1.getIOConfig().getStartPartitions().getPartitionOffsetMap().size()); + Assert.assertEquals(2, task1.getIOConfig().getEndPartitions().getPartitionOffsetMap().size()); + Assert.assertEquals(0L, (long) task1.getIOConfig().getStartPartitions().getPartitionOffsetMap().get(0)); + Assert.assertEquals(Long.MAX_VALUE, (long) task1.getIOConfig().getEndPartitions().getPartitionOffsetMap().get(0)); + Assert.assertEquals(0L, (long) task1.getIOConfig().getStartPartitions().getPartitionOffsetMap().get(2)); + Assert.assertEquals(Long.MAX_VALUE, (long) task1.getIOConfig().getEndPartitions().getPartitionOffsetMap().get(2)); + + KafkaIndexTask task2 = captured.getValues().get(1); + Assert.assertEquals(1, task2.getIOConfig().getStartPartitions().getPartitionOffsetMap().size()); + Assert.assertEquals(1, task2.getIOConfig().getEndPartitions().getPartitionOffsetMap().size()); + Assert.assertEquals(0L, (long) task2.getIOConfig().getStartPartitions().getPartitionOffsetMap().get(1)); + Assert.assertEquals(Long.MAX_VALUE, (long) task2.getIOConfig().getEndPartitions().getPartitionOffsetMap().get(1)); + } + + @Test + public void testReplicas() throws Exception + { + supervisor = getSupervisor(2, 1, true, "PT1H"); + addSomeEvents(1); + + Capture captured = Capture.newInstance(CaptureType.ALL); + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).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(2); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + + KafkaIndexTask task1 = captured.getValues().get(0); + Assert.assertEquals(3, task1.getIOConfig().getStartPartitions().getPartitionOffsetMap().size()); + Assert.assertEquals(3, task1.getIOConfig().getEndPartitions().getPartitionOffsetMap().size()); + Assert.assertEquals(0L, (long) task1.getIOConfig().getStartPartitions().getPartitionOffsetMap().get(0)); + Assert.assertEquals(0L, (long) task1.getIOConfig().getStartPartitions().getPartitionOffsetMap().get(1)); + Assert.assertEquals(0L, (long) task1.getIOConfig().getStartPartitions().getPartitionOffsetMap().get(2)); + + KafkaIndexTask task2 = captured.getValues().get(1); + Assert.assertEquals(3, task2.getIOConfig().getStartPartitions().getPartitionOffsetMap().size()); + Assert.assertEquals(3, task2.getIOConfig().getEndPartitions().getPartitionOffsetMap().size()); + Assert.assertEquals(0L, (long) task2.getIOConfig().getStartPartitions().getPartitionOffsetMap().get(0)); + Assert.assertEquals(0L, (long) task2.getIOConfig().getStartPartitions().getPartitionOffsetMap().get(1)); + Assert.assertEquals(0L, (long) task2.getIOConfig().getStartPartitions().getPartitionOffsetMap().get(2)); + } + + @Test + /** + * Test generating the starting offsets from the partition high water marks in Kafka. + */ + public void testLatestOffset() throws Exception + { + supervisor = getSupervisor(1, 1, false, "PT1H"); + addSomeEvents(1100); + + Capture captured = Capture.newInstance(); + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + new KafkaDataSourceMetadata( + null + ) + ).anyTimes(); + expect(taskQueue.add(capture(captured))).andReturn(true); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + + KafkaIndexTask task = captured.getValue(); + Assert.assertEquals(1100L, (long) task.getIOConfig().getStartPartitions().getPartitionOffsetMap().get(0)); + Assert.assertEquals(1100L, (long) task.getIOConfig().getStartPartitions().getPartitionOffsetMap().get(1)); + Assert.assertEquals(1100L, (long) task.getIOConfig().getStartPartitions().getPartitionOffsetMap().get(2)); + } + + @Test + /** + * Test generating the starting offsets from the partition data stored in druid_dataSource which contains the + * offsets of the last built segments. + */ + public void testDatasourceMetadata() throws Exception + { + supervisor = getSupervisor(1, 1, true, "PT1H"); + addSomeEvents(100); + + Capture captured = Capture.newInstance(); + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + new KafkaDataSourceMetadata( + new KafkaPartitions(KAFKA_TOPIC, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)) + ) + ).anyTimes(); + expect(taskQueue.add(capture(captured))).andReturn(true); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + + KafkaIndexTask task = captured.getValue(); + KafkaIOConfig taskConfig = task.getIOConfig(); + Assert.assertEquals(String.format("sequenceName-0", DATASOURCE), taskConfig.getBaseSequenceName()); + Assert.assertEquals(10L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(0)); + Assert.assertEquals(20L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(1)); + Assert.assertEquals(30L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(2)); + } + + @Test(expected = ISE.class) + public void testBadMetadataOffsets() throws Exception + { + supervisor = getSupervisor(1, 1, true, "PT1H"); + addSomeEvents(1); + + expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + new KafkaDataSourceMetadata( + new KafkaPartitions(KAFKA_TOPIC, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)) + ) + ).anyTimes(); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + } + + @Test + public void testKillIncompatibleTasks() throws Exception + { + supervisor = getSupervisor(2, 1, true, "PT1H"); + addSomeEvents(1); + + Task id1 = createKafkaIndexTask( // unexpected # of partitions (kill) + "id1", + DATASOURCE, + "index_kafka_testDS__some_other_sequenceName", + new KafkaPartitions("topic", ImmutableMap.of(0, 0L)), + new KafkaPartitions("topic", ImmutableMap.of(0, 10L)) + ); + + Task id2 = createKafkaIndexTask( // correct number of partitions and ranges (don't kill) + "id2", + DATASOURCE, + "sequenceName-0", + new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), + new KafkaPartitions("topic", ImmutableMap.of(0, 333L, 1, 333L, 2, 333L)) + ); + + Task id3 = createKafkaIndexTask( // unexpected range on partition 2 (kill) + "id3", + DATASOURCE, + "index_kafka_testDS__some_other_sequenceName", + new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 1L)), + new KafkaPartitions("topic", ImmutableMap.of(0, 333L, 1, 333L, 2, 330L)) + ); + + Task id4 = createKafkaIndexTask( // different datasource (don't kill) + "id4", + "other-datasource", + "index_kafka_testDS_d927edff33c4b3f", + new KafkaPartitions("topic", ImmutableMap.of(0, 0L)), + new KafkaPartitions("topic", ImmutableMap.of(0, 10L)) + ); + + Task id5 = new RealtimeIndexTask( // non KafkaIndexTask (don't kill) + "id5", + null, + new FireDepartment( + dataSchema, + new RealtimeIOConfig(null, null, null), + null + ), + null + ); + + List existingTasks = ImmutableList.of(id1, id2, id3, id4, id5); + + 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(existingTasks).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(); + taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + taskQueue.shutdown("id1"); + taskQueue.shutdown("id3"); + + expect(taskQueue.add(anyObject(Task.class))).andReturn(true); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + } + + @Test + public void testKillBadPartitionAssignment() throws Exception + { + supervisor = getSupervisor(1, 2, true, "PT1H"); + addSomeEvents(1); + + Task id1 = createKafkaIndexTask( + "id1", + DATASOURCE, + "sequenceName-0", + new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 2, 0L)), + new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)) + ); + Task id2 = createKafkaIndexTask( + "id2", + DATASOURCE, + "sequenceName-1", + new KafkaPartitions("topic", ImmutableMap.of(1, 0L)), + new KafkaPartitions("topic", ImmutableMap.of(1, Long.MAX_VALUE)) + ); + Task id3 = createKafkaIndexTask( + "id3", + 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)) + ); + Task id4 = createKafkaIndexTask( + "id4", + DATASOURCE, + "sequenceName-0", + new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L)), + new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE)) + ); + Task id5 = createKafkaIndexTask( + "id5", + DATASOURCE, + "sequenceName-0", + new KafkaPartitions("topic", ImmutableMap.of(1, 0L, 2, 0L)), + new KafkaPartitions("topic", ImmutableMap.of(1, Long.MAX_VALUE, 2, Long.MAX_VALUE)) + ); + + List existingTasks = ImmutableList.of(id1, id2, id3, id4, id5); + + 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(existingTasks).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.getStatus("id4")).andReturn(Optional.of(TaskStatus.running("id4"))).anyTimes(); + expect(taskStorage.getStatus("id5")).andReturn(Optional.of(TaskStatus.running("id5"))).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(taskStorage.getTask("id4")).andReturn(Optional.of(id3)).anyTimes(); + expect(taskStorage.getTask("id5")).andReturn(Optional.of(id3)).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + new KafkaDataSourceMetadata( + null + ) + ).anyTimes(); + taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + taskQueue.shutdown("id3"); + taskQueue.shutdown("id4"); + taskQueue.shutdown("id5"); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + } + + @Test + public void testRequeueTaskWhenFailed() throws Exception + { + supervisor = getSupervisor(2, 2, true, "PT1H"); + 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(); + + // test that running the main loop again checks the status of the tasks that were created and does nothing if they + // are all still running + reset(taskStorage); + 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(); + } + replay(taskStorage); + + supervisor.runInternal(); + verifyAll(); + + // test that a task failing causes a new task to be re-queued with the same parameters + Capture aNewTaskCapture = Capture.newInstance(); + List imStillAlive = tasks.subList(0, 3); + KafkaIndexTask iHaveFailed = (KafkaIndexTask) tasks.get(3); + reset(taskStorage); + reset(taskQueue); + expect(taskStorage.getActiveTasks()).andReturn(imStillAlive).anyTimes(); + for (Task task : imStillAlive) { + expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); + expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); + } + expect(taskStorage.getStatus(iHaveFailed.getId())).andReturn(Optional.of(TaskStatus.failure(iHaveFailed.getId()))); + expect(taskStorage.getTask(iHaveFailed.getId())).andReturn(Optional.of((Task) iHaveFailed)).anyTimes(); + expect(taskQueue.add(capture(aNewTaskCapture))).andReturn(true); + replay(taskStorage); + replay(taskQueue); + + supervisor.runInternal(); + verifyAll(); + + Assert.assertNotEquals(iHaveFailed.getId(), aNewTaskCapture.getValue().getId()); + Assert.assertEquals( + iHaveFailed.getIOConfig().getBaseSequenceName(), + ((KafkaIndexTask) aNewTaskCapture.getValue()).getIOConfig().getBaseSequenceName() + ); + } + + @Test + public void testQueueNextTasksOnSuccess() throws Exception + { + supervisor = getSupervisor(2, 2, true, "PT1H"); + 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); + 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(); + } + replay(taskStorage); + + supervisor.runInternal(); + verifyAll(); + + // test that a task succeeding causes a new task to be re-queued with the next offset range and causes any replica + // tasks to be shutdown + Capture newTasksCapture = Capture.newInstance(CaptureType.ALL); + Capture shutdownTaskIdCapture = Capture.newInstance(); + List imStillRunning = tasks.subList(1, 4); + KafkaIndexTask iAmSuccess = (KafkaIndexTask) tasks.get(0); + reset(taskStorage); + reset(taskQueue); + expect(taskStorage.getActiveTasks()).andReturn(imStillRunning).anyTimes(); + for (Task task : imStillRunning) { + expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); + expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); + } + 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); + replay(taskStorage); + replay(taskQueue); + + supervisor.runInternal(); + verifyAll(); + + // make sure we killed the right task (sequenceName for replicas are the same) + Assert.assertTrue(shutdownTaskIdCapture.getValue().contains(iAmSuccess.getIOConfig().getBaseSequenceName())); + } + + @Test + public void testBeginPublishAndQueueNextTasks() throws Exception + { + final TaskLocation location = new TaskLocation("testHost", 1234); + + supervisor = getSupervisor(2, 2, true, "PT1M"); + 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.getStatus(anyObject(TaskLocation.class), anyString())) + .andReturn(KafkaIndexTask.Status.READING) + .anyTimes(); + expect(taskClient.getStartTime(EasyMock.eq(location), EasyMock.contains("sequenceName-0"))) + .andReturn(DateTime.now().minusMinutes(2)) + .andReturn(DateTime.now()); + expect(taskClient.getStartTime(EasyMock.eq(location), EasyMock.contains("sequenceName-1"))) + .andReturn(DateTime.now()) + .times(2); + expect(taskClient.pause(EasyMock.eq(location), 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.eq(location), + EasyMock.contains("sequenceName-0"), + EasyMock.eq(ImmutableMap.of(0, 10L, 1, 20L, 2, 35L)), + EasyMock.eq(true) + ); + 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()) { + KafkaIndexTask kafkaIndexTask = (KafkaIndexTask) task; + Assert.assertEquals(dataSchema, kafkaIndexTask.getDataSchema()); + Assert.assertEquals(tuningConfig, kafkaIndexTask.getTuningConfig()); + + KafkaIOConfig taskConfig = kafkaIndexTask.getIOConfig(); + Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName()); + Assert.assertTrue("isUseTransaction", taskConfig.isUseTransaction()); + Assert.assertFalse("pauseAfterRead", taskConfig.isPauseAfterRead()); + + Assert.assertEquals(KAFKA_TOPIC, taskConfig.getStartPartitions().getTopic()); + Assert.assertEquals(10L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(0)); + Assert.assertEquals(20L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(1)); + Assert.assertEquals(35L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(2)); + } + } + + @Test + public void testDiscoverExistingPublishingTask() throws Exception + { + final TaskLocation location = new TaskLocation("testHost", 1234); + + supervisor = getSupervisor(1, 1, true, "PT1H"); + addSomeEvents(1); + + Task task = 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)) + ); + + Collection workItems = new ArrayList<>(); + workItems.add(new TestTaskRunnerWorkItem(task.getId(), null, location)); + + Capture captured = Capture.newInstance(); + 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(task)).anyTimes(); + expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + expect(taskStorage.getTask("id1")).andReturn(Optional.of(task)).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + new KafkaDataSourceMetadata( + null + ) + ).anyTimes(); + expect(taskClient.getStatus(location, "id1")).andReturn(KafkaIndexTask.Status.PUBLISHING); + expect(taskClient.getCurrentOffsets(location, "id1", false)).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); + expect(taskClient.getCurrentOffsets(location, "id1", true)).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); + expect(taskQueue.add(capture(captured))).andReturn(true); + + taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + SupervisorReport report = supervisor.getStatus(); + verifyAll(); + + Assert.assertEquals(DATASOURCE, report.getId()); + Assert.assertTrue(report.getPayload() instanceof KafkaSupervisorReport.KafkaSupervisorReportPayload); + + KafkaSupervisorReport.KafkaSupervisorReportPayload payload = (KafkaSupervisorReport.KafkaSupervisorReportPayload) + report.getPayload(); + + Assert.assertEquals(DATASOURCE, payload.getDataSource()); + Assert.assertEquals(3600L, (long) payload.getDurationSeconds()); + Assert.assertEquals(NUM_PARTITIONS, (int) payload.getPartitions()); + Assert.assertEquals(1, (int) payload.getReplicas()); + Assert.assertEquals(KAFKA_TOPIC, payload.getTopic()); + Assert.assertEquals(0, payload.getActiveTasks().size()); + Assert.assertEquals(1, payload.getPublishingTasks().size()); + + KafkaSupervisorReport.TaskReportData publishingReport = payload.getPublishingTasks().get(0); + + Assert.assertEquals("id1", publishingReport.getId()); + Assert.assertEquals(ImmutableMap.of(0, 0L, 1, 0L, 2, 0L), publishingReport.getStartingOffsets()); + Assert.assertEquals(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L), publishingReport.getCurrentOffsets()); + + KafkaIndexTask capturedTask = captured.getValue(); + Assert.assertEquals(dataSchema, capturedTask.getDataSchema()); + Assert.assertEquals(tuningConfig, capturedTask.getTuningConfig()); + + KafkaIOConfig capturedTaskConfig = capturedTask.getIOConfig(); + Assert.assertEquals(kafkaHost, capturedTaskConfig.getConsumerProperties().get("bootstrap.servers")); + Assert.assertEquals("myCustomValue", capturedTaskConfig.getConsumerProperties().get("myCustomKey")); + Assert.assertEquals("sequenceName-0", capturedTaskConfig.getBaseSequenceName()); + Assert.assertTrue("isUseTransaction", capturedTaskConfig.isUseTransaction()); + Assert.assertFalse("pauseAfterRead", capturedTaskConfig.isPauseAfterRead()); + + // check that the new task was created with starting offsets matching where the publishing task finished + Assert.assertEquals(KAFKA_TOPIC, capturedTaskConfig.getStartPartitions().getTopic()); + Assert.assertEquals(10L, (long) capturedTaskConfig.getStartPartitions().getPartitionOffsetMap().get(0)); + Assert.assertEquals(20L, (long) capturedTaskConfig.getStartPartitions().getPartitionOffsetMap().get(1)); + Assert.assertEquals(30L, (long) capturedTaskConfig.getStartPartitions().getPartitionOffsetMap().get(2)); + + Assert.assertEquals(KAFKA_TOPIC, capturedTaskConfig.getEndPartitions().getTopic()); + Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionOffsetMap().get(0)); + Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionOffsetMap().get(1)); + Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionOffsetMap().get(2)); + } + + @Test + public void testDiscoverExistingPublishingAndReadingTask() throws Exception + { + final TaskLocation location1 = new TaskLocation("testHost", 1234); + final TaskLocation location2 = new TaskLocation("testHost2", 145); + final DateTime startTime = new DateTime(); + + supervisor = getSupervisor(1, 1, true, "PT1H"); + 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)) + ); + + 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)) + ); + + 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)).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.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); + expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + new KafkaDataSourceMetadata( + null + ) + ).anyTimes(); + expect(taskClient.getStatus(location1, "id1")).andReturn(KafkaIndexTask.Status.PUBLISHING); + expect(taskClient.getStatus(location2, "id2")).andReturn(KafkaIndexTask.Status.READING); + expect(taskClient.getStartTime(location2, "id2")).andReturn(startTime); + expect(taskClient.getCurrentOffsets(location1, "id1", false)).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); + expect(taskClient.getCurrentOffsets(location1, "id1", true)).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); + expect(taskClient.getCurrentOffsets(location2, "id2", false)).andReturn(ImmutableMap.of(0, 40L, 1, 50L, 2, 60L)); + + taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + SupervisorReport report = supervisor.getStatus(); + verifyAll(); + + Assert.assertEquals(DATASOURCE, report.getId()); + Assert.assertTrue(report.getPayload() instanceof KafkaSupervisorReport.KafkaSupervisorReportPayload); + + KafkaSupervisorReport.KafkaSupervisorReportPayload payload = (KafkaSupervisorReport.KafkaSupervisorReportPayload) + report.getPayload(); + + Assert.assertEquals(DATASOURCE, payload.getDataSource()); + Assert.assertEquals(3600L, (long) payload.getDurationSeconds()); + Assert.assertEquals(NUM_PARTITIONS, (int) payload.getPartitions()); + Assert.assertEquals(1, (int) payload.getReplicas()); + Assert.assertEquals(KAFKA_TOPIC, payload.getTopic()); + 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); + + Assert.assertEquals("id2", activeReport.getId()); + Assert.assertEquals(startTime, activeReport.getStartTime()); + Assert.assertEquals(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L), activeReport.getStartingOffsets()); + Assert.assertEquals(ImmutableMap.of(0, 40L, 1, 50L, 2, 60L), activeReport.getCurrentOffsets()); + + Assert.assertEquals("id1", publishingReport.getId()); + Assert.assertEquals(ImmutableMap.of(0, 0L, 1, 0L, 2, 0L), publishingReport.getStartingOffsets()); + Assert.assertEquals(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L), publishingReport.getCurrentOffsets()); + } + + @Test(expected = IllegalStateException.class) + public void testStopNotStarted() throws Exception + { + supervisor = getSupervisor(1, 1, true, "PT1H"); + supervisor.stop(false); + } + + @Test + public void testStop() throws Exception + { + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + taskRunner.unregisterListener(String.format("KafkaSupervisor-%s", DATASOURCE)); + replayAll(); + + supervisor = getSupervisor(1, 1, true, "PT1H"); + supervisor.start(); + supervisor.stop(false); + + verifyAll(); + } + + private void addSomeEvents(int numEventsPerPartition) throws Exception + { + try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { + for (int i = 0; i < NUM_PARTITIONS; i++) { + for (int j = 0; j < numEventsPerPartition; j++) { + kafkaProducer.send( + new ProducerRecord( + KAFKA_TOPIC, + i, + null, + String.format("event-%d", j).getBytes() + ) + ).get(); + } + } + } + } + + private KafkaSupervisor getSupervisor(int replicas, int taskCount, boolean useEarliestOffset, String duration) + { + KafkaSupervisorIOConfig kafkaSupervisorIOConfig = new KafkaSupervisorIOConfig( + KAFKA_TOPIC, + replicas, + taskCount, + new Period(duration), + ImmutableMap.of("myCustomKey", "myCustomValue", "bootstrap.servers", kafkaHost), + new Period("P1D"), + new Period("PT30S"), + useEarliestOffset, + new Period("PT30M") + ); + + return new TestableKafkaSupervisor( + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClient, + objectMapper, + new KafkaSupervisorSpec( + dataSchema, + tuningConfig, + kafkaSupervisorIOConfig, + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClient, + objectMapper + ) + ); + } + + private DataSchema getDataSchema(String dataSource) + { + List dimensions = new ArrayList<>(); + dimensions.add(StringDimensionSchema.create("dim1")); + dimensions.add(StringDimensionSchema.create("dim2")); + + return new DataSchema( + dataSource, + objectMapper.convertValue( + new StringInputRowParser( + new JSONParseSpec( + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec( + dimensions, + null, + null + ), + new JSONPathSpec(true, ImmutableList.of()), + ImmutableMap.of() + ), + Charsets.UTF_8.name() + ), + Map.class + ), + new AggregatorFactory[]{new CountAggregatorFactory("rows")}, + new UniformGranularitySpec( + Granularity.HOUR, + QueryGranularity.NONE, + ImmutableList.of() + ), + objectMapper + ); + } + + private KafkaIndexTask createKafkaIndexTask( + String id, + String dataSource, + String sequenceName, + KafkaPartitions startPartitions, + KafkaPartitions endPartitions + ) + { + return new KafkaIndexTask( + id, + null, + getDataSchema(dataSource), + tuningConfig, + new KafkaIOConfig( + sequenceName, + startPartitions, + endPartitions, + ImmutableMap.of(), + true, + false + ), + ImmutableMap.of(), + null + ); + } + + private class TestTaskRunnerWorkItem extends TaskRunnerWorkItem + { + + private TaskLocation location; + + public TestTaskRunnerWorkItem(String taskId, ListenableFuture result, TaskLocation location) + { + super(taskId, result); + this.location = location; + } + + @Override + public TaskLocation getLocation() + { + return location; + } + } + + private class TestableKafkaSupervisor extends KafkaSupervisor + { + public TestableKafkaSupervisor( + TaskStorage taskStorage, + TaskMaster taskMaster, + IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, + KafkaIndexTaskClient taskClient, + ObjectMapper mapper, + KafkaSupervisorSpec spec + ) + { + super(taskStorage, taskMaster, indexerMetadataStorageCoordinator, taskClient, mapper, spec); + } + + @Override + protected String generateSequenceName(int groupId) + { + return String.format("sequenceName-%d", groupId); + } + } +} diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/test/TestBroker.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/test/TestBroker.java index 956a1443f955..d879b4d4739e 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/test/TestBroker.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/test/TestBroker.java @@ -39,6 +39,8 @@ public class TestBroker implements Closeable { + private final static Random RANDOM = new Random(); + private final String zookeeperConnect; private final File directory; private final int id; @@ -102,6 +104,8 @@ public Map consumerProperties() props.put("bootstrap.servers", String.format("localhost:%d", getPort())); props.put("key.deserializer", ByteArrayDeserializer.class.getName()); props.put("value.deserializer", ByteArrayDeserializer.class.getName()); + props.put("group.id", String.valueOf(RANDOM.nextInt())); + props.put("auto.offset.reset", "earliest"); return props; } diff --git a/extensions-core/postgresql-metadata-storage/src/test/java/io/druid/metadata/storage/postgresql/PostgreSQLConnectorTest.java b/extensions-core/postgresql-metadata-storage/src/test/java/io/druid/metadata/storage/postgresql/PostgreSQLConnectorTest.java index 2fda23303814..d1f9b8cbdacd 100644 --- a/extensions-core/postgresql-metadata-storage/src/test/java/io/druid/metadata/storage/postgresql/PostgreSQLConnectorTest.java +++ b/extensions-core/postgresql-metadata-storage/src/test/java/io/druid/metadata/storage/postgresql/PostgreSQLConnectorTest.java @@ -46,6 +46,7 @@ public void testIsTransientException() throws Exception null, null, null, + null, null ) ) diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/updater/MetadataStorageUpdaterJobSpec.java b/indexing-hadoop/src/main/java/io/druid/indexer/updater/MetadataStorageUpdaterJobSpec.java index cb2e8f2a927d..813f9c7948f8 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/updater/MetadataStorageUpdaterJobSpec.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/updater/MetadataStorageUpdaterJobSpec.java @@ -96,6 +96,7 @@ public MetadataStorageTablesConfig getMetadataStorageTablesConfig() null, null, null, + null, null ); } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/RetryPolicyConfig.java b/indexing-service/src/main/java/io/druid/indexing/common/RetryPolicyConfig.java index 45b1d0df9886..1cbbb65f9d6d 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/RetryPolicyConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/RetryPolicyConfig.java @@ -40,7 +40,7 @@ public Period getMinWait() return minWait; } - RetryPolicyConfig setMinWait(Period minWait) + public RetryPolicyConfig setMinWait(Period minWait) { this.minWait = minWait; return this; @@ -51,7 +51,7 @@ public Period getMaxWait() return maxWait; } - RetryPolicyConfig setMaxWait(Period maxWait) + public RetryPolicyConfig setMaxWait(Period maxWait) { this.maxWait = maxWait; return this; @@ -62,7 +62,7 @@ public long getMaxRetryCount() return maxRetryCount; } - RetryPolicyConfig setMaxRetryCount(long maxRetryCount) + public RetryPolicyConfig setMaxRetryCount(long maxRetryCount) { this.maxRetryCount = maxRetryCount; return this; diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java index 3a4f4806e8ab..c5b49a927219 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java @@ -21,9 +21,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.core.JsonParser; -import com.fasterxml.jackson.core.JsonToken; -import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.CharMatcher; import com.google.common.base.Joiner; @@ -175,8 +172,15 @@ public List>> restore() return retVal; } + @Override public void registerListener(TaskRunnerListener listener, Executor executor) { + for (Pair pair : listeners) { + if (pair.lhs.getListenerId().equals(listener.getListenerId())) { + throw new ISE("Listener [%s] already registered", listener.getListenerId()); + } + } + final Pair listenerPair = Pair.of(listener, executor); synchronized (tasks) { @@ -185,6 +189,19 @@ public void registerListener(TaskRunnerListener listener, Executor executor) } listeners.add(listenerPair); + log.info("Registered listener [%s]", listener.getListenerId()); + } + } + + @Override + public void unregisterListener(String listenerId) + { + for (Pair pair : listeners) { + if (pair.lhs.getListenerId().equals(listenerId)) { + listeners.remove(pair); + log.info("Unregistered listener [%s]", listenerId); + return; + } } } @@ -401,6 +418,11 @@ public TaskStatus call() } TaskRunnerUtils.notifyLocationChanged(listeners, task.getId(), taskLocation); + TaskRunnerUtils.notifyStatusChanged( + listeners, + task.getId(), + TaskStatus.running(task.getId()) + ); log.info("Logging task %s output to: %s", task.getId(), logFile); boolean runFailed = true; @@ -419,13 +441,17 @@ public TaskStatus call() taskLogPusher.pushTaskLog(task.getId(), logFile); } + TaskStatus status; if (!runFailed) { // Process exited successfully - return jsonMapper.readValue(statusFile, TaskStatus.class); + status = jsonMapper.readValue(statusFile, TaskStatus.class); } else { // Process exited unsuccessfully - return TaskStatus.failure(task.getId()); + status = TaskStatus.failure(task.getId()); } + + TaskRunnerUtils.notifyStatusChanged(listeners, task.getId(), status); + return status; } catch (Throwable t) { throw closer.rethrow(t); diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java index fe4eba5772fe..be5a7951e21e 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java @@ -44,6 +44,7 @@ import com.google.common.util.concurrent.ListeningScheduledExecutorService; import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.SettableFuture; +import com.metamx.common.ISE; import com.metamx.common.Pair; import com.metamx.common.RE; import com.metamx.common.lifecycle.LifecycleStart; @@ -349,6 +350,12 @@ public List>> restore() @Override public void registerListener(TaskRunnerListener listener, Executor executor) { + for (Pair pair : listeners) { + if (pair.lhs.getListenerId().equals(listener.getListenerId())) { + throw new ISE("Listener [%s] already registered", listener.getListenerId()); + } + } + final Pair listenerPair = Pair.of(listener, executor); synchronized (statusLock) { @@ -360,10 +367,23 @@ public void registerListener(TaskRunnerListener listener, Executor executor) ); } + log.info("Registered listener [%s]", listener.getListenerId()); listeners.add(listenerPair); } } + @Override + public void unregisterListener(String listenerId) + { + for (Pair pair : listeners) { + if (pair.lhs.getListenerId().equals(listenerId)) { + listeners.remove(pair); + log.info("Unregistered listener [%s]", listenerId); + return; + } + } + } + @Override public Collection getWorkers() { @@ -797,6 +817,7 @@ private boolean announceTask( RemoteTaskRunnerWorkItem newWorkItem = workItem.withWorker(theZkWorker.getWorker(), null); runningTasks.put(task.getId(), newWorkItem); log.info("Task %s switched from pending to running (on [%s])", task.getId(), newWorkItem.getWorker().getHost()); + TaskRunnerUtils.notifyStatusChanged(listeners, task.getId(), TaskStatus.running(task.getId())); // Syncing state with Zookeeper - don't assign new tasks until the task we just assigned is actually running // on a worker - this avoids overflowing a worker with tasks @@ -926,7 +947,8 @@ public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) th taskRunnerWorkItem = runningTasks.remove(taskId); if (taskRunnerWorkItem != null) { log.info("Task[%s] just disappeared!", taskId); - taskRunnerWorkItem.setResult(TaskStatus.failure(taskRunnerWorkItem.getTaskId())); + taskRunnerWorkItem.setResult(TaskStatus.failure(taskId)); + TaskRunnerUtils.notifyStatusChanged(listeners, taskId, TaskStatus.failure(taskId)); } else { log.info("Task[%s] went bye bye.", taskId); } @@ -1048,7 +1070,8 @@ public void run() log.info("Failing task[%s]", assignedTask); RemoteTaskRunnerWorkItem taskRunnerWorkItem = runningTasks.remove(assignedTask); if (taskRunnerWorkItem != null) { - taskRunnerWorkItem.setResult(TaskStatus.failure(taskRunnerWorkItem.getTaskId())); + taskRunnerWorkItem.setResult(TaskStatus.failure(assignedTask)); + TaskRunnerUtils.notifyStatusChanged(listeners, assignedTask, TaskStatus.failure(assignedTask)); } else { log.warn("RemoteTaskRunner has no knowledge of task[%s]", assignedTask); } @@ -1119,6 +1142,7 @@ private void taskComplete( // Notify interested parties taskRunnerWorkItem.setResult(taskStatus); + TaskRunnerUtils.notifyStatusChanged(listeners, taskStatus.getId(), taskStatus); } @Override diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskMaster.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskMaster.java index a9792200d958..24764692ef28 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskMaster.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskMaster.java @@ -34,6 +34,7 @@ import io.druid.indexing.common.task.Task; import io.druid.indexing.overlord.autoscaling.ScalingStats; import io.druid.indexing.overlord.config.TaskQueueConfig; +import io.druid.indexing.overlord.supervisor.SupervisorManager; import io.druid.server.DruidNode; import io.druid.server.initialization.IndexerZkConfig; import org.apache.curator.framework.CuratorFramework; @@ -55,6 +56,7 @@ public class TaskMaster private final ReentrantLock giant = new ReentrantLock(); private final Condition mayBeStopped = giant.newCondition(); private final TaskActionClientFactory taskActionClientFactory; + private final SupervisorManager supervisorManager; private final AtomicReference leaderLifecycleRef = new AtomicReference<>(null); @@ -75,9 +77,11 @@ public TaskMaster( final TaskRunnerFactory runnerFactory, final CuratorFramework curator, final ServiceAnnouncer serviceAnnouncer, - final ServiceEmitter emitter + final ServiceEmitter emitter, + final SupervisorManager supervisorManager ) { + this.supervisorManager = supervisorManager; this.taskActionClientFactory = taskActionClientFactory; this.leaderSelector = new LeaderSelector( curator, @@ -112,8 +116,11 @@ public void takeLeadership(CuratorFramework client) throws Exception log.makeAlert("TaskMaster set a new Lifecycle without the old one being cleared! Race condition") .emit(); } + leaderLifecycle.addManagedInstance(taskRunner); leaderLifecycle.addManagedInstance(taskQueue); + leaderLifecycle.addManagedInstance(supervisorManager); + leaderLifecycle.addHandler( new Lifecycle.Handler() { @@ -280,4 +287,13 @@ public Optional getScalingStats() return Optional.absent(); } } + + public Optional getSupervisorManager() + { + if (leading) { + return Optional.of(supervisorManager); + } else { + return Optional.absent(); + } + } } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunner.java index 4de22ba1962d..3871fb8c87dd 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunner.java @@ -56,6 +56,8 @@ public interface TaskRunner */ void registerListener(TaskRunnerListener listener, Executor executor); + void unregisterListener(String listenerId); + /** * Run a task. The returned status should be some kind of completed status. * diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerListener.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerListener.java index 0c6a0de1fb1a..df79696e1394 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerListener.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerListener.java @@ -20,6 +20,7 @@ package io.druid.indexing.overlord; import io.druid.indexing.common.TaskLocation; +import io.druid.indexing.common.TaskStatus; import java.util.concurrent.Executor; @@ -28,9 +29,16 @@ */ public interface TaskRunnerListener { + String getListenerId(); + /** * Called when the location of a task has changed. The task may not actually be done starting up when * this notification arrives, so it may not be listening at this location yet. */ void locationChanged(String taskId, TaskLocation newLocation); + + /** + * Called when the status of a task has changed. + */ + void statusChanged(String taskId, TaskStatus status); } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerUtils.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerUtils.java index 6fdcd881319d..9135da5bfeb0 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerUtils.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerUtils.java @@ -22,6 +22,7 @@ import com.metamx.common.Pair; import com.metamx.emitter.EmittingLogger; import io.druid.indexing.common.TaskLocation; +import io.druid.indexing.common.TaskStatus; import java.util.concurrent.Executor; @@ -58,4 +59,34 @@ public void run() } } } + + public static void notifyStatusChanged( + final Iterable> listeners, + final String taskId, + final TaskStatus status + ) + { + log.info("Task [%s] status changed to [%s].", taskId, status.getStatusCode()); + for (final Pair listener : listeners) { + try { + listener.rhs.execute( + new Runnable() + { + @Override + public void run() + { + listener.lhs.statusChanged(taskId, status); + } + } + ); + } + catch (Exception e) { + log.makeAlert(e, "Unable to notify task listener") + .addData("taskId", taskId) + .addData("taskStatus", status.getStatusCode()) + .addData("listener", listener.toString()) + .emit(); + } + } + } } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java index 8d78f5bce2e0..d6e6e20f05f7 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java @@ -29,6 +29,7 @@ import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import com.google.inject.Inject; +import com.metamx.common.ISE; import com.metamx.common.Pair; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.emitter.EmittingLogger; @@ -108,17 +109,36 @@ public List>> restore() @Override public void registerListener(TaskRunnerListener listener, Executor executor) { + for (Pair pair : listeners) { + if (pair.lhs.getListenerId().equals(listener.getListenerId())) { + throw new ISE("Listener [%s] already registered", listener.getListenerId()); + } + } + final Pair listenerPair = Pair.of(listener, executor); // Location never changes for an existing task, so it's ok to add the listener first and then issue bootstrap // callbacks without any special synchronization. listeners.add(listenerPair); + log.info("Registered listener [%s]", listener.getListenerId()); for (ThreadPoolTaskRunnerWorkItem item : runningItems) { TaskRunnerUtils.notifyLocationChanged(ImmutableList.of(listenerPair), item.getTaskId(), item.getLocation()); } } + @Override + public void unregisterListener(String listenerId) + { + for (Pair pair : listeners) { + if (pair.lhs.getListenerId().equals(listenerId)) { + listeners.remove(pair); + log.info("Unregistered listener [%s]", listenerId); + return; + } + } + } + private static ListeningExecutorService buildExecutorService(int priority) { return MoreExecutors.listeningDecorator( @@ -169,6 +189,8 @@ public void stop() task.getId(), System.currentTimeMillis() - start ); + + TaskRunnerUtils.notifyStatusChanged(listeners, task.getId(), taskStatus); } catch (Exception e) { log.makeAlert(e, "Graceful task shutdown failed: %s", task.getDataSource()) @@ -177,9 +199,11 @@ public void stop() .emit(); log.warn(e, "Graceful shutdown of task[%s] aborted with exception.", task.getId()); error = true; + TaskRunnerUtils.notifyStatusChanged(listeners, task.getId(), TaskStatus.failure(task.getId())); } } else { graceful = false; + TaskRunnerUtils.notifyStatusChanged(listeners, task.getId(), TaskStatus.failure(task.getId())); } elapsed = System.currentTimeMillis() - start; @@ -408,6 +432,7 @@ public TaskStatus call() task.getId(), location ); + TaskRunnerUtils.notifyStatusChanged(listeners, task.getId(), TaskStatus.running(task.getId())); status = task.run(toolbox); } catch (InterruptedException e) { @@ -431,7 +456,9 @@ public TaskStatus call() throw t; } - return status.withDuration(System.currentTimeMillis() - startTime); + status = status.withDuration(System.currentTimeMillis() - startTime); + TaskRunnerUtils.notifyStatusChanged(listeners, task.getId(), status); + return status; } } } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorManager.java b/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorManager.java new file mode 100644 index 000000000000..96b27757878d --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorManager.java @@ -0,0 +1,139 @@ +/* + * 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.overlord.supervisor; + +import com.google.common.base.Optional; +import com.google.common.base.Preconditions; +import com.google.inject.Inject; +import com.metamx.common.Pair; +import com.metamx.common.lifecycle.LifecycleStart; +import com.metamx.common.lifecycle.LifecycleStop; +import com.metamx.emitter.EmittingLogger; +import io.druid.metadata.MetadataSupervisorManager; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * Manages the creation and lifetime of {@link Supervisor}. + */ +public class SupervisorManager +{ + private static final EmittingLogger log = new EmittingLogger(SupervisorManager.class); + + private final MetadataSupervisorManager metadataSupervisorManager; + private final Map> supervisors = new HashMap<>(); + + @Inject + public SupervisorManager(MetadataSupervisorManager metadataSupervisorManager) + { + this.metadataSupervisorManager = metadataSupervisorManager; + } + + public Set getSupervisorIds() + { + return supervisors.keySet(); + } + + public Optional getSupervisorSpec(String id) + { + return supervisors.get(id) == null + ? Optional.absent() + : Optional.fromNullable(supervisors.get(id).rhs); + } + + public boolean hasSupervisor(String id) + { + return supervisors.containsKey(id); + } + + public boolean createAndStartSupervisor(SupervisorSpec spec) + { + Preconditions.checkNotNull(spec, "spec"); + Preconditions.checkNotNull(spec.getId(), "spec.getId()"); + + return createAndStartSupervisorInternal(spec, true); + } + + public void stopAndRemoveSupervisor(String id) + { + Pair pair = supervisors.get(id); + if (pair != null) { + metadataSupervisorManager.insert(id, new NoopSupervisorSpec()); // where NoopSupervisorSpec is a tombstone + pair.lhs.stop(true); + supervisors.remove(id); + } + } + + @LifecycleStart + public void start() + { + log.info("Loading stored supervisors from database"); + + Map supervisors = metadataSupervisorManager.getLatest(); + for (String id : supervisors.keySet()) { + SupervisorSpec spec = supervisors.get(id); + if (!(spec instanceof NoopSupervisorSpec)) { + createAndStartSupervisorInternal(spec, false); + } + } + } + + @LifecycleStop + public void stop() + { + for (String id : supervisors.keySet()) { + supervisors.get(id).lhs.stop(false); + } + + supervisors.clear(); + log.info("SupervisorManager stopped."); + } + + public Map> getSupervisorHistory() + { + return metadataSupervisorManager.getAll(); + } + + public Optional getSupervisorStatus(String id) + { + return supervisors.get(id) == null + ? Optional.absent() + : Optional.fromNullable(supervisors.get(id).lhs.getStatus()); + } + + private boolean createAndStartSupervisorInternal(SupervisorSpec spec, boolean persistSpec) + { + String id = spec.getId(); + if (!supervisors.containsKey(id)) { + if (persistSpec) { + metadataSupervisorManager.insert(id, spec); + } + + supervisors.put(id, Pair.of(spec.createSupervisor(), spec)); + supervisors.get(id).lhs.start(); + return true; + } + + return false; + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorResource.java b/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorResource.java new file mode 100644 index 000000000000..a929d9590a6a --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorResource.java @@ -0,0 +1,230 @@ +/* + * 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.overlord.supervisor; + +import com.google.common.base.Function; +import com.google.common.base.Optional; +import com.google.common.collect.ImmutableMap; +import com.google.inject.Inject; +import io.druid.indexing.overlord.TaskMaster; + +import javax.ws.rs.Consumes; +import javax.ws.rs.GET; +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; +import javax.ws.rs.Produces; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; +import java.util.List; +import java.util.Map; + +/** + * Endpoints for submitting and starting a {@link SupervisorSpec}, getting running supervisors, stopping supervisors, + * and getting supervisor history. + */ +@Path("/druid/indexer/v1/supervisor") +public class SupervisorResource +{ + private final TaskMaster taskMaster; + + @Inject + public SupervisorResource(TaskMaster taskMaster) + { + this.taskMaster = taskMaster; + } + + @POST + @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) + public Response specPost(final SupervisorSpec spec) + { + return asLeaderWithSupervisorManager( + new Function() + { + @Override + public Response apply(SupervisorManager manager) + { + if (manager.hasSupervisor(spec.getId())) { + return Response.status(Response.Status.CONFLICT) + .entity( + ImmutableMap.of( + "error", + String.format("Supervisor already exists for [%s]", spec.getId()) + ) + ).build(); + } + + manager.createAndStartSupervisor(spec); + return Response.ok(ImmutableMap.of("id", spec.getId())).build(); + } + } + ); + } + + @GET + @Produces(MediaType.APPLICATION_JSON) + public Response specGetAll() + { + return asLeaderWithSupervisorManager( + new Function() + { + @Override + public Response apply(SupervisorManager manager) + { + return Response.ok(manager.getSupervisorIds()).build(); + } + } + ); + } + + @GET + @Path("/{id}") + @Produces(MediaType.APPLICATION_JSON) + public Response specGet(@PathParam("id") final String id) + { + return asLeaderWithSupervisorManager( + new Function() + { + @Override + public Response apply(SupervisorManager manager) + { + Optional spec = manager.getSupervisorSpec(id); + if (!spec.isPresent()) { + return Response.status(Response.Status.NOT_FOUND) + .entity(ImmutableMap.of("error", String.format("[%s] does not exist", id))) + .build(); + } + + return Response.ok(spec.get()).build(); + } + } + ); + } + + @GET + @Path("/{id}/status") + @Produces(MediaType.APPLICATION_JSON) + public Response specGetStatus(@PathParam("id") final String id) + { + return asLeaderWithSupervisorManager( + new Function() + { + @Override + public Response apply(SupervisorManager manager) + { + Optional spec = manager.getSupervisorStatus(id); + if (!spec.isPresent()) { + return Response.status(Response.Status.NOT_FOUND) + .entity(ImmutableMap.of("error", String.format("[%s] does not exist", id))) + .build(); + } + + return Response.ok(spec.get()).build(); + } + } + ); + } + + @POST + @Path("/{id}/shutdown") + @Produces(MediaType.APPLICATION_JSON) + public Response shutdown(@PathParam("id") final String id) + { + return asLeaderWithSupervisorManager( + new Function() + { + @Override + public Response apply(SupervisorManager manager) + { + if (!manager.hasSupervisor(id)) { + return Response.status(Response.Status.NOT_FOUND) + .entity(ImmutableMap.of("error", String.format("[%s] does not exist", id))) + .build(); + } + + manager.stopAndRemoveSupervisor(id); + return Response.ok(ImmutableMap.of("id", id)).build(); + } + } + ); + } + + @GET + @Path("/history") + @Produces(MediaType.APPLICATION_JSON) + public Response specGetAllHistory() + { + return asLeaderWithSupervisorManager( + new Function() + { + @Override + public Response apply(SupervisorManager manager) + { + return Response.ok(manager.getSupervisorHistory()).build(); + } + } + ); + } + + @GET + @Path("/{id}/history") + @Produces(MediaType.APPLICATION_JSON) + public Response specGetHistory(@PathParam("id") final String id) + { + return asLeaderWithSupervisorManager( + new Function() + { + @Override + public Response apply(SupervisorManager manager) + { + Map> history = manager.getSupervisorHistory(); + if (history.containsKey(id)) { + return Response.ok(history.get(id)).build(); + } else { + return Response.status(Response.Status.NOT_FOUND) + .entity( + ImmutableMap.of( + "error", + String.format( + "No history for [%s] (history available for %s)", + id, + history.keySet() + ) + ) + ) + .build(); + } + } + } + ); + } + + private Response asLeaderWithSupervisorManager(Function f) + { + Optional supervisorManager = taskMaster.getSupervisorManager(); + if (supervisorManager.isPresent()) { + return f.apply(supervisorManager.get()); + } else { + // Encourage client to try again soon, when we'll likely have a redirect set up + return Response.status(Response.Status.SERVICE_UNAVAILABLE).build(); + } + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskMonitor.java b/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskMonitor.java index a60af3f92a2c..7dabf9af9fc1 100644 --- a/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskMonitor.java +++ b/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskMonitor.java @@ -217,11 +217,23 @@ private void registerLocationListener() taskRunner.registerListener( new TaskRunnerListener() { + @Override + public String getListenerId() + { + return "WorkerTaskMonitor"; + } + @Override public void locationChanged(final String taskId, final TaskLocation newLocation) { notices.add(new LocationNotice(taskId, newLocation)); } + + @Override + public void statusChanged(final String taskId, final TaskStatus status) + { + // do nothing + } }, MoreExecutors.sameThreadExecutor() ); @@ -257,6 +269,7 @@ public void stop() throws InterruptedException try { started = false; + taskRunner.unregisterListener("WorkerTaskMonitor"); exec.shutdownNow(); pathChildrenCache.close(); taskRunner.stop(); diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java index 16df2895f323..0d41340199c2 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java @@ -50,6 +50,7 @@ import io.druid.indexing.overlord.TaskStorageQueryAdapter; import io.druid.indexing.overlord.autoscaling.ScalingStats; import io.druid.indexing.overlord.config.TaskQueueConfig; +import io.druid.indexing.overlord.supervisor.SupervisorManager; import io.druid.server.DruidNode; import io.druid.server.initialization.IndexerZkConfig; import io.druid.server.initialization.ZkPathsConfig; @@ -96,6 +97,7 @@ public class OverlordTest private CountDownLatch[] taskCompletionCountDownLatches; private CountDownLatch[] runTaskCountDownLatches; private HttpServletRequest req; + private SupervisorManager supervisorManager; private void setupServerAndCurator() throws Exception { @@ -121,6 +123,7 @@ private void tearDownServerAndCurator() public void setUp() throws Exception { req = EasyMock.createStrictMock(HttpServletRequest.class); + supervisorManager = EasyMock.createMock(SupervisorManager.class); taskLockbox = EasyMock.createStrictMock(TaskLockbox.class); taskLockbox.syncFromStorage(); EasyMock.expectLastCall().atLeastOnce(); @@ -179,7 +182,8 @@ public void announce(DruidNode node) announcementLatch.countDown(); } }, - serviceEmitter + serviceEmitter, + supervisorManager ); EmittingLogger.registerEmitter(serviceEmitter); } @@ -316,6 +320,13 @@ public void registerListener(TaskRunnerListener listener, Executor executor) throw new UnsupportedOperationException(); } + @Override + public void unregisterListener(String listenerId) + { + // Overlord doesn't call this method + throw new UnsupportedOperationException(); + } + @Override public void stop() { diff --git a/processing/src/test/java/io/druid/guice/MetadataStorageTablesConfigTest.java b/processing/src/test/java/io/druid/guice/MetadataStorageTablesConfigTest.java index 1786754d88a5..498a043e5da4 100644 --- a/processing/src/test/java/io/druid/guice/MetadataStorageTablesConfigTest.java +++ b/processing/src/test/java/io/druid/guice/MetadataStorageTablesConfigTest.java @@ -78,5 +78,7 @@ public ObjectMapper jsonMapper() props.getProperty("druid.metadata.storage.tables.taskLock"), config.getLockTable(MetadataStorageTablesConfig.TASK_ENTRY_TYPE) ); + Assert.assertEquals(props.getProperty("druid.metadata.storage.tables.dataSource"), config.getDataSourceTable()); + Assert.assertEquals(props.getProperty("druid.metadata.storage.tables.supervisors"), config.getSupervisorTable()); } } diff --git a/processing/src/test/resources/test.runtime.properties b/processing/src/test/resources/test.runtime.properties index c87aa9003cc6..d84cf995ee1f 100644 --- a/processing/src/test/resources/test.runtime.properties +++ b/processing/src/test/resources/test.runtime.properties @@ -6,3 +6,5 @@ druid.metadata.storage.tables.tasks=ddd_tasks druid.metadata.storage.tables.taskLog=eee_tasklog druid.metadata.storage.tables.taskLock=fff_tasklock druid.metadata.storage.tables.audit=ggg_audit +druid.metadata.storage.tables.dataSource=hhh_dataSource +druid.metadata.storage.tables.supervisors=iii_supervisors diff --git a/server/src/main/java/io/druid/guice/SQLMetadataStorageDruidModule.java b/server/src/main/java/io/druid/guice/SQLMetadataStorageDruidModule.java index cc86bb8bdeb4..eac245f33ca2 100644 --- a/server/src/main/java/io/druid/guice/SQLMetadataStorageDruidModule.java +++ b/server/src/main/java/io/druid/guice/SQLMetadataStorageDruidModule.java @@ -36,6 +36,7 @@ import io.druid.metadata.MetadataStorageActionHandlerFactory; import io.druid.metadata.MetadataStorageConnector; import io.druid.metadata.MetadataStorageProvider; +import io.druid.metadata.MetadataSupervisorManager; import io.druid.metadata.NoopMetadataStorageProvider; import io.druid.metadata.SQLMetadataConnector; import io.druid.metadata.SQLMetadataRuleManager; @@ -45,6 +46,7 @@ import io.druid.metadata.SQLMetadataSegmentPublisher; import io.druid.metadata.SQLMetadataSegmentPublisherProvider; import io.druid.metadata.SQLMetadataStorageActionHandlerFactory; +import io.druid.metadata.SQLMetadataSupervisorManager; import io.druid.server.audit.AuditManagerProvider; import io.druid.server.audit.SQLAuditManager; import io.druid.server.audit.SQLAuditManagerConfig; @@ -156,6 +158,13 @@ public void createBindingChoices(Binder binder, String defaultPropertyValue) Key.get(SQLAuditManagerProvider.class), defaultPropertyValue ); + PolyBind.createChoiceWithDefault( + binder, + PROPERTY, + Key.get(MetadataSupervisorManager.class), + Key.get(SQLMetadataSupervisorManager.class), + defaultPropertyValue + ); } @Override @@ -217,5 +226,10 @@ public void configure(Binder binder) .addBinding(type) .to(SQLAuditManagerProvider.class) .in(LazySingleton.class); + + PolyBind.optionBinder(binder, Key.get(MetadataSupervisorManager.class)) + .addBinding(type) + .to(SQLMetadataSupervisorManager.class) + .in(LazySingleton.class); } } diff --git a/server/src/main/java/io/druid/indexing/overlord/supervisor/NoopSupervisorSpec.java b/server/src/main/java/io/druid/indexing/overlord/supervisor/NoopSupervisorSpec.java new file mode 100644 index 000000000000..f3c7937c3dff --- /dev/null +++ b/server/src/main/java/io/druid/indexing/overlord/supervisor/NoopSupervisorSpec.java @@ -0,0 +1,51 @@ +/* + * 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.overlord.supervisor; + +/** + * Used as a tombstone marker in the supervisors metadata table to indicate that the supervisor has been removed. + */ +public class NoopSupervisorSpec implements SupervisorSpec +{ + @Override + public String getId() + { + return null; + } + + @Override + public Supervisor createSupervisor() + { + return new Supervisor() + { + @Override + public void start() {} + + @Override + public void stop(boolean stopGracefully) {} + + @Override + public SupervisorReport getStatus() + { + return null; + } + }; + } +} diff --git a/server/src/main/java/io/druid/indexing/overlord/supervisor/Supervisor.java b/server/src/main/java/io/druid/indexing/overlord/supervisor/Supervisor.java new file mode 100644 index 000000000000..c32804b343e0 --- /dev/null +++ b/server/src/main/java/io/druid/indexing/overlord/supervisor/Supervisor.java @@ -0,0 +1,35 @@ +/* + * 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.overlord.supervisor; + +public interface Supervisor +{ + void start(); + + /** + * @param stopGracefully If true, supervisor will cleanly shutdown managed tasks if possible (for example signalling + * them to publish their segments and exit). The implementation may block until the tasks have + * either acknowledged or completed. If false, supervisor will stop immediately and leave any + * running tasks as they are. + */ + void stop(boolean stopGracefully); + + SupervisorReport getStatus(); +} diff --git a/server/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorReport.java b/server/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorReport.java new file mode 100644 index 000000000000..292cad275bf7 --- /dev/null +++ b/server/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorReport.java @@ -0,0 +1,50 @@ +/* + * 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.overlord.supervisor; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.joda.time.DateTime; + +public abstract class SupervisorReport +{ + private String id; + private DateTime generationTime; + + public SupervisorReport(String id, DateTime generationTime) + { + this.id = id; + this.generationTime = generationTime; + } + + @JsonProperty + public String getId() + { + return id; + } + + @JsonProperty + public DateTime getGenerationTime() + { + return generationTime; + } + + @JsonProperty + public abstract Object getPayload(); +} diff --git a/server/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorSpec.java b/server/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorSpec.java new file mode 100644 index 000000000000..9bbf8f7b254e --- /dev/null +++ b/server/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorSpec.java @@ -0,0 +1,34 @@ +/* + * 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.overlord.supervisor; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "NoopSupervisorSpec", value = NoopSupervisorSpec.class) +}) +public interface SupervisorSpec +{ + String getId(); + + Supervisor createSupervisor(); +} diff --git a/server/src/main/java/io/druid/indexing/overlord/supervisor/VersionedSupervisorSpec.java b/server/src/main/java/io/druid/indexing/overlord/supervisor/VersionedSupervisorSpec.java new file mode 100644 index 000000000000..66c152debecb --- /dev/null +++ b/server/src/main/java/io/druid/indexing/overlord/supervisor/VersionedSupervisorSpec.java @@ -0,0 +1,29 @@ +package io.druid.indexing.overlord.supervisor; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +public class VersionedSupervisorSpec +{ + private final SupervisorSpec spec; + private final String version; + + @JsonCreator + public VersionedSupervisorSpec(@JsonProperty("spec") SupervisorSpec spec, @JsonProperty("version") String version) + { + this.spec = spec; + this.version = version; + } + + @JsonProperty + public SupervisorSpec getSpec() + { + return spec; + } + + @JsonProperty + public String getVersion() + { + return version; + } +} diff --git a/server/src/main/java/io/druid/metadata/MetadataSupervisorManager.java b/server/src/main/java/io/druid/metadata/MetadataSupervisorManager.java new file mode 100644 index 000000000000..a7e23170120d --- /dev/null +++ b/server/src/main/java/io/druid/metadata/MetadataSupervisorManager.java @@ -0,0 +1,37 @@ +/* + * 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.metadata; + +import io.druid.indexing.overlord.supervisor.SupervisorSpec; +import io.druid.indexing.overlord.supervisor.VersionedSupervisorSpec; + +import java.util.List; +import java.util.Map; + +public interface MetadataSupervisorManager +{ + void start(); + + void insert(String id, SupervisorSpec spec); + + Map> getAll(); + + Map getLatest(); +} diff --git a/server/src/main/java/io/druid/metadata/SQLMetadataConnector.java b/server/src/main/java/io/druid/metadata/SQLMetadataConnector.java index a926f163483b..07b123003fa6 100644 --- a/server/src/main/java/io/druid/metadata/SQLMetadataConnector.java +++ b/server/src/main/java/io/druid/metadata/SQLMetadataConnector.java @@ -366,6 +366,26 @@ tableName, getSerialType(), getPayloadType(), entryTypeName ); } + public void createSupervisorsTable(final String tableName) + { + createTable( + tableName, + ImmutableList.of( + String.format( + "CREATE TABLE %1$s (\n" + + " id VARCHAR(255) NOT NULL,\n" + + " spec_id VARCHAR(255) NOT NULL,\n" + + " version VARCHAR(255) NOT NULL,\n" + + " payload %2$s NOT NULL,\n" + + " PRIMARY KEY (id)\n" + + ")", + tableName, getPayloadType() + ), + String.format("CREATE INDEX idx_%1$s_spec_id ON %1$s(spec_id)", tableName) + ) + ); + } + @Override public Void insertOrUpdate( final String tableName, @@ -468,6 +488,14 @@ public void createTaskTables() } } + @Override + public void createSupervisorsTable() + { + if (config.get().isCreateTables()) { + createSupervisorsTable(tablesConfigSupplier.get().getSupervisorTable()); + } + } + @Override public byte[] lookup( final String tableName, diff --git a/server/src/main/java/io/druid/metadata/SQLMetadataSupervisorManager.java b/server/src/main/java/io/druid/metadata/SQLMetadataSupervisorManager.java new file mode 100644 index 000000000000..64690c1844ff --- /dev/null +++ b/server/src/main/java/io/druid/metadata/SQLMetadataSupervisorManager.java @@ -0,0 +1,250 @@ +/* + * 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.metadata; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Supplier; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.inject.Inject; +import com.metamx.common.Pair; +import com.metamx.common.lifecycle.LifecycleStart; +import io.druid.guice.ManageLifecycle; +import io.druid.guice.annotations.Json; +import io.druid.indexing.overlord.supervisor.SupervisorSpec; +import io.druid.indexing.overlord.supervisor.VersionedSupervisorSpec; +import org.joda.time.DateTime; +import org.skife.jdbi.v2.FoldController; +import org.skife.jdbi.v2.Folder3; +import org.skife.jdbi.v2.Handle; +import org.skife.jdbi.v2.IDBI; +import org.skife.jdbi.v2.StatementContext; +import org.skife.jdbi.v2.tweak.HandleCallback; +import org.skife.jdbi.v2.tweak.ResultSetMapper; + +import java.io.IOException; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.List; +import java.util.Map; + +@ManageLifecycle +public class SQLMetadataSupervisorManager implements MetadataSupervisorManager +{ + private final ObjectMapper jsonMapper; + private final SQLMetadataConnector connector; + private final Supplier dbTables; + private final IDBI dbi; + + @Inject + public SQLMetadataSupervisorManager( + @Json ObjectMapper jsonMapper, + SQLMetadataConnector connector, + Supplier dbTables + ) + { + this.jsonMapper = jsonMapper; + this.connector = connector; + this.dbTables = dbTables; + this.dbi = connector.getDBI(); + } + + @LifecycleStart + public void start() + { + connector.createSupervisorsTable(); + } + + @Override + public void insert(final String id, final SupervisorSpec spec) + { + dbi.withHandle( + new HandleCallback() + { + @Override + public Void withHandle(Handle handle) throws Exception + { + final String version = new DateTime().toString(); + handle.createStatement( + String.format( + "INSERT INTO %s (id, spec_id, version, payload) VALUES (:id, :spec_id, :version, :payload)", + getSupervisorsTable() + ) + ) + .bind("id", String.format("%s_%s", id, version)) + .bind("spec_id", id) + .bind("version", version) + .bind("payload", jsonMapper.writeValueAsBytes(spec)) + .execute(); + + return null; + } + } + ); + } + + @Override + public Map> getAll() + { + return ImmutableMap.copyOf( + dbi.withHandle( + new HandleCallback>>() + { + @Override + public Map> withHandle(Handle handle) throws Exception + { + return handle.createQuery( + String.format( + "SELECT spec_id, version, payload FROM %1$s ORDER BY version DESC", + getSupervisorsTable() + ) + ).map( + new ResultSetMapper>() + { + @Override + public Pair map(int index, ResultSet r, StatementContext ctx) + throws SQLException + { + try { + SupervisorSpec payload = jsonMapper.readValue( + r.getBytes("payload"), + new TypeReference() + { + } + ); + return Pair.of( + r.getString("spec_id"), + new VersionedSupervisorSpec(payload, r.getString("version")) + ); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + } + ).fold( + Maps.>newHashMap(), + new Folder3>, Pair>() + { + @Override + public Map> fold( + Map> retVal, + Pair pair, + FoldController foldController, + StatementContext statementContext + ) throws SQLException + { + try { + String specId = pair.lhs; + if (!retVal.containsKey(specId)) { + retVal.put(specId, Lists.newArrayList()); + } + + retVal.get(specId).add(pair.rhs); + return retVal; + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + } + ); + } + } + ) + ); + } + + @Override + public Map getLatest() + { + return ImmutableMap.copyOf( + dbi.withHandle( + new HandleCallback>() + { + @Override + public Map withHandle(Handle handle) throws Exception + { + return handle.createQuery( + String.format( + "SELECT r.spec_id, r.payload " + + "FROM %1$s r " + + "INNER JOIN(SELECT spec_id, max(version) as version FROM %1$s GROUP BY spec_id) latest " + + "ON r.spec_id = latest.spec_id and r.version = latest.version", + getSupervisorsTable() + ) + ).map( + new ResultSetMapper>() + { + @Override + public Pair map(int index, ResultSet r, StatementContext ctx) + throws SQLException + { + try { + return Pair.of( + r.getString("spec_id"), + jsonMapper.readValue( + r.getBytes("payload"), new TypeReference() + { + } + ) + ); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + } + ).fold( + Maps.newHashMap(), + new Folder3, Pair>() + { + @Override + public Map fold( + Map retVal, + Pair stringObjectMap, + FoldController foldController, + StatementContext statementContext + ) throws SQLException + { + try { + retVal.put(stringObjectMap.lhs, stringObjectMap.rhs); + return retVal; + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + } + ); + } + } + ) + ); + } + + private String getSupervisorsTable() + { + return dbTables.get().getSupervisorTable(); + } +} diff --git a/server/src/test/java/io/druid/metadata/SQLMetadataConnectorTest.java b/server/src/test/java/io/druid/metadata/SQLMetadataConnectorTest.java index 20dd731c3354..c257f0c67855 100644 --- a/server/src/test/java/io/druid/metadata/SQLMetadataConnectorTest.java +++ b/server/src/test/java/io/druid/metadata/SQLMetadataConnectorTest.java @@ -54,12 +54,14 @@ public void testCreateTables() throws Exception tables.add(tablesConfig.getLogTable(entryType)); tables.add(tablesConfig.getEntryTable(entryType)); tables.add(tablesConfig.getAuditTable()); + tables.add(tablesConfig.getSupervisorTable()); connector.createSegmentTable(); connector.createConfigTable(); connector.createRulesTable(); connector.createTaskTables(); connector.createAuditTable(); + connector.createSupervisorsTable(); connector.getDBI().withHandle( new HandleCallback() diff --git a/server/src/test/java/io/druid/metadata/SQLMetadataSupervisorManagerTest.java b/server/src/test/java/io/druid/metadata/SQLMetadataSupervisorManagerTest.java new file mode 100644 index 000000000000..0b3fee42d006 --- /dev/null +++ b/server/src/test/java/io/druid/metadata/SQLMetadataSupervisorManagerTest.java @@ -0,0 +1,136 @@ +/* + * 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.metadata; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Suppliers; +import com.google.common.collect.ImmutableMap; +import io.druid.indexing.overlord.supervisor.SupervisorSpec; +import io.druid.indexing.overlord.supervisor.VersionedSupervisorSpec; +import io.druid.jackson.DefaultObjectMapper; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.skife.jdbi.v2.Handle; +import org.skife.jdbi.v2.tweak.HandleCallback; + +import java.util.List; +import java.util.Map; + +public class SQLMetadataSupervisorManagerTest +{ + private static final ObjectMapper mapper = new DefaultObjectMapper(); + + private TestDerbyConnector connector; + private MetadataStorageTablesConfig tablesConfig; + private SQLMetadataSupervisorManager supervisorManager; + + @Rule + public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); + + @BeforeClass + public static void setupStatic() + { + mapper.registerSubtypes(TestSupervisorSpec.class); + } + + @Before + public void setUp() + { + connector = derbyConnectorRule.getConnector(); + tablesConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); + connector.createSupervisorsTable(); + + supervisorManager = new SQLMetadataSupervisorManager(mapper, connector, Suppliers.ofInstance(tablesConfig)); + } + + @Test + public void testInsertAndGet() + { + final String supervisor1 = "test-supervisor-1"; + final String supervisor2 = "test-supervisor-2"; + final Map data1rev1 = ImmutableMap.of("key1-1", "value1-1-1", "key1-2", "value1-2-1"); + final Map data1rev2 = ImmutableMap.of("key1-1", "value1-1-2", "key1-2", "value1-2-2"); + final Map data1rev3 = ImmutableMap.of("key1-1", "value1-1-3", "key1-2", "value1-2-3"); + final Map data2rev1 = ImmutableMap.of("key2-1", "value2-1-1", "key2-2", "value2-2-1"); + final Map data2rev2 = ImmutableMap.of("key2-3", "value2-3-2", "key2-4", "value2-4-2"); + + Assert.assertTrue(supervisorManager.getAll().isEmpty()); + + // add 2 supervisors, one revision each, and make sure the state is as expected + supervisorManager.insert(supervisor1, new TestSupervisorSpec(supervisor1, data1rev1)); + supervisorManager.insert(supervisor2, new TestSupervisorSpec(supervisor2, data2rev1)); + + Map> supervisorSpecs = supervisorManager.getAll(); + Map latestSpecs = supervisorManager.getLatest(); + Assert.assertEquals(2, supervisorSpecs.size()); + Assert.assertEquals(1, supervisorSpecs.get(supervisor1).size()); + Assert.assertEquals(1, supervisorSpecs.get(supervisor2).size()); + Assert.assertEquals(supervisor1, supervisorSpecs.get(supervisor1).get(0).getSpec().getId()); + Assert.assertEquals(supervisor2, supervisorSpecs.get(supervisor2).get(0).getSpec().getId()); + Assert.assertEquals(data1rev1, ((TestSupervisorSpec) supervisorSpecs.get(supervisor1).get(0).getSpec()).getData()); + Assert.assertEquals(data2rev1, ((TestSupervisorSpec) supervisorSpecs.get(supervisor2).get(0).getSpec()).getData()); + Assert.assertEquals(2, latestSpecs.size()); + Assert.assertEquals(data1rev1, ((TestSupervisorSpec) latestSpecs.get(supervisor1)).getData()); + Assert.assertEquals(data2rev1, ((TestSupervisorSpec) latestSpecs.get(supervisor2)).getData()); + + // add more revisions to the supervisors + supervisorManager.insert(supervisor1, new TestSupervisorSpec(supervisor1, data1rev2)); + supervisorManager.insert(supervisor1, new TestSupervisorSpec(supervisor1, data1rev3)); + supervisorManager.insert(supervisor2, new TestSupervisorSpec(supervisor2, data2rev2)); + + supervisorSpecs = supervisorManager.getAll(); + latestSpecs = supervisorManager.getLatest(); + Assert.assertEquals(2, supervisorSpecs.size()); + Assert.assertEquals(3, supervisorSpecs.get(supervisor1).size()); + Assert.assertEquals(2, supervisorSpecs.get(supervisor2).size()); + + // make sure getAll() returns each spec in descending order + Assert.assertEquals(data1rev3, ((TestSupervisorSpec) supervisorSpecs.get(supervisor1).get(0).getSpec()).getData()); + Assert.assertEquals(data1rev2, ((TestSupervisorSpec) supervisorSpecs.get(supervisor1).get(1).getSpec()).getData()); + Assert.assertEquals(data1rev1, ((TestSupervisorSpec) supervisorSpecs.get(supervisor1).get(2).getSpec()).getData()); + Assert.assertEquals(data2rev2, ((TestSupervisorSpec) supervisorSpecs.get(supervisor2).get(0).getSpec()).getData()); + Assert.assertEquals(data2rev1, ((TestSupervisorSpec) supervisorSpecs.get(supervisor2).get(1).getSpec()).getData()); + + // make sure getLatest() returns the last revision + Assert.assertEquals(data1rev3, ((TestSupervisorSpec) latestSpecs.get(supervisor1)).getData()); + Assert.assertEquals(data2rev2, ((TestSupervisorSpec) latestSpecs.get(supervisor2)).getData()); + } + + @After + public void cleanup() + { + connector.getDBI().withHandle( + new HandleCallback() + { + @Override + public Void withHandle(Handle handle) throws Exception + { + handle.createStatement(String.format("DROP TABLE %s", tablesConfig.getSupervisorTable())) + .execute(); + return null; + } + } + ); + } +} diff --git a/server/src/test/java/io/druid/metadata/TestSupervisorSpec.java b/server/src/test/java/io/druid/metadata/TestSupervisorSpec.java new file mode 100644 index 000000000000..7b0a38c9e5d8 --- /dev/null +++ b/server/src/test/java/io/druid/metadata/TestSupervisorSpec.java @@ -0,0 +1,57 @@ +/* + * 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.metadata; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.indexing.overlord.supervisor.Supervisor; +import io.druid.indexing.overlord.supervisor.SupervisorSpec; + +public class TestSupervisorSpec implements SupervisorSpec +{ + private String id; + private Object data; + + @JsonCreator + public TestSupervisorSpec(@JsonProperty("id") String id, @JsonProperty("data") Object data) + { + this.id = id; + this.data = data; + } + + @Override + @JsonProperty + public String getId() + { + return id; + } + + @Override + public Supervisor createSupervisor() + { + return null; + } + + @JsonProperty + public Object getData() + { + return data; + } +} diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java index 41efbdc23f56..3aea375a774c 100644 --- a/services/src/main/java/io/druid/cli/CliOverlord.java +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -71,6 +71,7 @@ import io.druid.indexing.overlord.http.OverlordRedirectInfo; import io.druid.indexing.overlord.http.OverlordResource; import io.druid.indexing.overlord.setup.WorkerBehaviorConfig; +import io.druid.indexing.overlord.supervisor.SupervisorResource; import io.druid.indexing.worker.config.WorkerConfig; import io.druid.segment.realtime.firehose.ChatHandlerProvider; import io.druid.server.audit.AuditManagerProvider; @@ -158,6 +159,7 @@ public void configure(Binder binder) binder.bind(JettyServerInitializer.class).toInstance(new OverlordJettyServerInitializer()); Jerseys.addResource(binder, OverlordResource.class); + Jerseys.addResource(binder, SupervisorResource.class); LifecycleModule.register(binder, Server.class); } diff --git a/services/src/main/java/io/druid/cli/CreateTables.java b/services/src/main/java/io/druid/cli/CreateTables.java index c63b262bb9e6..2c7920e6ff3d 100644 --- a/services/src/main/java/io/druid/cli/CreateTables.java +++ b/services/src/main/java/io/druid/cli/CreateTables.java @@ -115,5 +115,6 @@ public void run() dbConnector.createConfigTable(); dbConnector.createTaskTables(); dbConnector.createAuditTable(); + dbConnector.createSupervisorsTable(); } }