Supervisor for KafkaIndexTask (#2656)

* supervisor for kafka indexing tasks

* cr changes
This commit is contained in:
David Lim 2016-05-05 00:13:13 -06:00 committed by Gian Merlino
parent 2a769a9fb7
commit b489f63698
43 changed files with 4716 additions and 18 deletions

View File

@ -51,4 +51,6 @@ public interface MetadataStorageConnector
void createTaskTables();
void createAuditTable();
void createSupervisorsTable();
}

View File

@ -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 class MetadataStorageTablesConfig
@JsonProperty("audit")
private final String auditTable;
@JsonProperty("supervisors")
private final String supervisorTable;
@JsonCreator
public MetadataStorageTablesConfig(
@JsonProperty("base") String base,
@ -83,7 +86,8 @@ public class 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 class 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 class MetadataStorageTablesConfig
return auditTable;
}
public String getSupervisorTable()
{
return supervisorTable;
}
}

View File

@ -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

View File

@ -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://<OVERLORD_IP>:<OVERLORD_PORT>/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<String, String>|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: `<BROKER_1>:<PORT_1>,<BROKER_2>:<PORT_2>,...`.|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/<supervisorId>/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/<supervisorId>
```
Returns the current spec for the supervisor with the provided ID.
#### Get Supervisor Status Report
```
GET /druid/indexer/v1/supervisor/<supervisorId>/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/<supervisorId>/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/<supervisorId>/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/<supervisorId>/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.

View File

@ -97,6 +97,8 @@
* [Datasketches](../development/extensions-core/datasketches-aggregators.html)
* [Geographic Queries](../development/geo.html)
* [Router](../development/router.html)
* [Kafka Indexing Service](../development/extensions-core/kafka-ingestion.html)
## Misc
* [Papers & Talks](../misc/papers-and-talks.html)

View File

@ -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<Integer, Long> pause(TaskLocation location, String id)
{
return pause(location, id, 0);
}
public Map<Integer, Long> 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<Map<Integer, Long>>() {});
}
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<Integer, Long> 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<Map<Integer, Long>>() {});
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
public Map<Integer, Long> 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<Map<Integer, Long>>() {});
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
public void setEndOffsets(TaskLocation location, String id, Map<Integer, Long> endOffsets)
{
setEndOffsets(location, id, endOffsets, false);
}
public void setEndOffsets(TaskLocation location, String id, Map<Integer, Long> 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);
}
}
}
}
}
}

View File

@ -24,6 +24,7 @@ import com.fasterxml.jackson.databind.jsontype.NamedType;
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 class KafkaIndexTaskModule implements DruidModule
new NamedType(KafkaIndexTask.class, "index_kafka"),
new NamedType(KafkaDataSourceMetadata.class, "kafka"),
new NamedType(KafkaIOConfig.class, "kafka"),
new NamedType(KafkaTuningConfig.class, "kafka")
new NamedType(KafkaTuningConfig.class, "kafka"),
new NamedType(KafkaSupervisorSpec.class, "kafka")
)
);
}

View File

@ -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<String, String> 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<String, String> 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<String, String> 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();
}
}

View File

@ -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<Integer, Long> startingOffsets;
private final Map<Integer, Long> currentOffsets;
private final DateTime startTime;
private final Long remainingSeconds;
public TaskReportData(
String id,
Map<Integer, Long> startingOffsets,
Map<Integer, Long> 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<Integer, Long> getStartingOffsets()
{
return startingOffsets;
}
@JsonProperty
public Map<Integer, Long> 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<TaskReportData> activeTasks;
private final List<TaskReportData> 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<TaskReportData> getActiveTasks()
{
return activeTasks;
}
@JsonProperty
public List<TaskReportData> 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<Integer, Long> startingOffsets,
Map<Integer, Long> currentOffsets,
DateTime startTime,
Long remainingSeconds
)
{
payload.activeTasks.add(new TaskReportData(id, startingOffsets, currentOffsets, startTime, remainingSeconds));
}
public void addPublishingTask(
String id,
Map<Integer, Long> startingOffsets,
Map<Integer, Long> 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 +
'}';
}
}

View File

@ -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
);
}
}

View File

@ -39,6 +39,8 @@ import java.util.Random;
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 class TestBroker implements Closeable
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;
}

View File

@ -46,6 +46,7 @@ public class PostgreSQLConnectorTest
null,
null,
null,
null,
null
)
)

View File

@ -96,6 +96,7 @@ public class MetadataStorageUpdaterJobSpec implements Supplier<MetadataStorageCo
null,
null,
null,
null,
null
);
}

View File

@ -40,7 +40,7 @@ public class RetryPolicyConfig
return minWait;
}
RetryPolicyConfig setMinWait(Period minWait)
public RetryPolicyConfig setMinWait(Period minWait)
{
this.minWait = minWait;
return this;
@ -51,7 +51,7 @@ public class RetryPolicyConfig
return maxWait;
}
RetryPolicyConfig setMaxWait(Period maxWait)
public RetryPolicyConfig setMaxWait(Period maxWait)
{
this.maxWait = maxWait;
return this;
@ -62,7 +62,7 @@ public class RetryPolicyConfig
return maxRetryCount;
}
RetryPolicyConfig setMaxRetryCount(long maxRetryCount)
public RetryPolicyConfig setMaxRetryCount(long maxRetryCount)
{
this.maxRetryCount = maxRetryCount;
return this;

View File

@ -21,9 +21,6 @@ package io.druid.indexing.overlord;
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 class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
return retVal;
}
@Override
public void registerListener(TaskRunnerListener listener, Executor executor)
{
for (Pair<TaskRunnerListener, Executor> pair : listeners) {
if (pair.lhs.getListenerId().equals(listener.getListenerId())) {
throw new ISE("Listener [%s] already registered", listener.getListenerId());
}
}
final Pair<TaskRunnerListener, Executor> listenerPair = Pair.of(listener, executor);
synchronized (tasks) {
@ -185,6 +189,19 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
}
listeners.add(listenerPair);
log.info("Registered listener [%s]", listener.getListenerId());
}
}
@Override
public void unregisterListener(String listenerId)
{
for (Pair<TaskRunnerListener, Executor> pair : listeners) {
if (pair.lhs.getListenerId().equals(listenerId)) {
listeners.remove(pair);
log.info("Unregistered listener [%s]", listenerId);
return;
}
}
}
@ -401,6 +418,11 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
}
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 class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
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);

View File

@ -44,6 +44,7 @@ import com.google.common.util.concurrent.ListenableScheduledFuture;
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 class RemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
@Override
public void registerListener(TaskRunnerListener listener, Executor executor)
{
for (Pair<TaskRunnerListener, Executor> pair : listeners) {
if (pair.lhs.getListenerId().equals(listener.getListenerId())) {
throw new ISE("Listener [%s] already registered", listener.getListenerId());
}
}
final Pair<TaskRunnerListener, Executor> listenerPair = Pair.of(listener, executor);
synchronized (statusLock) {
@ -360,10 +367,23 @@ public class RemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
);
}
log.info("Registered listener [%s]", listener.getListenerId());
listeners.add(listenerPair);
}
}
@Override
public void unregisterListener(String listenerId)
{
for (Pair<TaskRunnerListener, Executor> pair : listeners) {
if (pair.lhs.getListenerId().equals(listenerId)) {
listeners.remove(pair);
log.info("Unregistered listener [%s]", listenerId);
return;
}
}
}
@Override
public Collection<ImmutableWorkerInfo> getWorkers()
{
@ -797,6 +817,7 @@ public class RemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
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 class RemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
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 class RemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
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 @@ public class RemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
// Notify interested parties
taskRunnerWorkItem.setResult(taskStatus);
TaskRunnerUtils.notifyStatusChanged(listeners, taskStatus.getId(), taskStatus);
}
@Override

View File

@ -34,6 +34,7 @@ import io.druid.indexing.common.actions.TaskActionClientFactory;
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<Lifecycle> leaderLifecycleRef = new AtomicReference<>(null);
@ -75,9 +77,11 @@ public class 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 class TaskMaster
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 class TaskMaster
return Optional.absent();
}
}
public Optional<SupervisorManager> getSupervisorManager()
{
if (leading) {
return Optional.of(supervisorManager);
} else {
return Optional.absent();
}
}
}

View File

@ -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.
*

View File

@ -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 @@ import java.util.concurrent.Executor;
*/
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);
}

View File

@ -22,6 +22,7 @@ package io.druid.indexing.overlord;
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 class TaskRunnerUtils
}
}
}
public static void notifyStatusChanged(
final Iterable<Pair<TaskRunnerListener, Executor>> listeners,
final String taskId,
final TaskStatus status
)
{
log.info("Task [%s] status changed to [%s].", taskId, status.getStatusCode());
for (final Pair<TaskRunnerListener, Executor> 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();
}
}
}
}

View File

@ -29,6 +29,7 @@ import com.google.common.util.concurrent.ListenableFuture;
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 class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker
@Override
public void registerListener(TaskRunnerListener listener, Executor executor)
{
for (Pair<TaskRunnerListener, Executor> pair : listeners) {
if (pair.lhs.getListenerId().equals(listener.getListenerId())) {
throw new ISE("Listener [%s] already registered", listener.getListenerId());
}
}
final Pair<TaskRunnerListener, Executor> 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<TaskRunnerListener, Executor> 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 class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker
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 class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker
.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 class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker
task.getId(),
location
);
TaskRunnerUtils.notifyStatusChanged(listeners, task.getId(), TaskStatus.running(task.getId()));
status = task.run(toolbox);
}
catch (InterruptedException e) {
@ -431,7 +456,9 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker
throw t;
}
return status.withDuration(System.currentTimeMillis() - startTime);
status = status.withDuration(System.currentTimeMillis() - startTime);
TaskRunnerUtils.notifyStatusChanged(listeners, task.getId(), status);
return status;
}
}
}

View File

@ -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<String, Pair<Supervisor, SupervisorSpec>> supervisors = new HashMap<>();
@Inject
public SupervisorManager(MetadataSupervisorManager metadataSupervisorManager)
{
this.metadataSupervisorManager = metadataSupervisorManager;
}
public Set<String> getSupervisorIds()
{
return supervisors.keySet();
}
public Optional<SupervisorSpec> getSupervisorSpec(String id)
{
return supervisors.get(id) == null
? Optional.<SupervisorSpec>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<Supervisor, SupervisorSpec> 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<String, SupervisorSpec> 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<String, List<VersionedSupervisorSpec>> getSupervisorHistory()
{
return metadataSupervisorManager.getAll();
}
public Optional<SupervisorReport> getSupervisorStatus(String id)
{
return supervisors.get(id) == null
? Optional.<SupervisorReport>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;
}
}

View File

@ -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<SupervisorManager, Response>()
{
@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<SupervisorManager, Response>()
{
@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<SupervisorManager, Response>()
{
@Override
public Response apply(SupervisorManager manager)
{
Optional<SupervisorSpec> 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<SupervisorManager, Response>()
{
@Override
public Response apply(SupervisorManager manager)
{
Optional<SupervisorReport> 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<SupervisorManager, Response>()
{
@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<SupervisorManager, Response>()
{
@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<SupervisorManager, Response>()
{
@Override
public Response apply(SupervisorManager manager)
{
Map<String, List<VersionedSupervisorSpec>> 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<SupervisorManager, Response> f)
{
Optional<SupervisorManager> 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();
}
}
}

View File

@ -217,11 +217,23 @@ public class WorkerTaskMonitor
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 class WorkerTaskMonitor
try {
started = false;
taskRunner.unregisterListener("WorkerTaskMonitor");
exec.shutdownNow();
pathChildrenCache.close();
taskRunner.stop();

View File

@ -50,6 +50,7 @@ import io.druid.indexing.overlord.TaskStorage;
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 @@ public class OverlordTest
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 class OverlordTest
announcementLatch.countDown();
}
},
serviceEmitter
serviceEmitter,
supervisorManager
);
EmittingLogger.registerEmitter(serviceEmitter);
}
@ -316,6 +320,13 @@ public class OverlordTest
throw new UnsupportedOperationException();
}
@Override
public void unregisterListener(String listenerId)
{
// Overlord doesn't call this method
throw new UnsupportedOperationException();
}
@Override
public void stop()
{

View File

@ -78,5 +78,7 @@ public class MetadataStorageTablesConfigTest
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());
}
}

View File

@ -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

View File

@ -36,6 +36,7 @@ import io.druid.metadata.MetadataSegmentPublisherProvider;
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.SQLMetadataSegmentManagerProvider;
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 class SQLMetadataStorageDruidModule implements Module
Key.get(SQLAuditManagerProvider.class),
defaultPropertyValue
);
PolyBind.createChoiceWithDefault(
binder,
PROPERTY,
Key.get(MetadataSupervisorManager.class),
Key.get(SQLMetadataSupervisorManager.class),
defaultPropertyValue
);
}
@Override
@ -217,5 +226,10 @@ public class SQLMetadataStorageDruidModule implements Module
.addBinding(type)
.to(SQLAuditManagerProvider.class)
.in(LazySingleton.class);
PolyBind.optionBinder(binder, Key.get(MetadataSupervisorManager.class))
.addBinding(type)
.to(SQLMetadataSupervisorManager.class)
.in(LazySingleton.class);
}
}

View File

@ -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;
}
};
}
}

View File

@ -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();
}

View File

@ -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();
}

View File

@ -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();
}

View File

@ -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;
}
}

View File

@ -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<String, List<VersionedSupervisorSpec>> getAll();
Map<String, SupervisorSpec> getLatest();
}

View File

@ -366,6 +366,26 @@ public abstract class SQLMetadataConnector implements MetadataStorageConnector
);
}
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 abstract class SQLMetadataConnector implements MetadataStorageConnector
}
}
@Override
public void createSupervisorsTable()
{
if (config.get().isCreateTables()) {
createSupervisorsTable(tablesConfigSupplier.get().getSupervisorTable());
}
}
@Override
public byte[] lookup(
final String tableName,

View File

@ -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<MetadataStorageTablesConfig> dbTables;
private final IDBI dbi;
@Inject
public SQLMetadataSupervisorManager(
@Json ObjectMapper jsonMapper,
SQLMetadataConnector connector,
Supplier<MetadataStorageTablesConfig> 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<Void>()
{
@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<String, List<VersionedSupervisorSpec>> getAll()
{
return ImmutableMap.copyOf(
dbi.withHandle(
new HandleCallback<Map<String, List<VersionedSupervisorSpec>>>()
{
@Override
public Map<String, List<VersionedSupervisorSpec>> 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<Pair<String, VersionedSupervisorSpec>>()
{
@Override
public Pair<String, VersionedSupervisorSpec> map(int index, ResultSet r, StatementContext ctx)
throws SQLException
{
try {
SupervisorSpec payload = jsonMapper.readValue(
r.getBytes("payload"),
new TypeReference<SupervisorSpec>()
{
}
);
return Pair.of(
r.getString("spec_id"),
new VersionedSupervisorSpec(payload, r.getString("version"))
);
}
catch (IOException e) {
throw Throwables.propagate(e);
}
}
}
).fold(
Maps.<String, List<VersionedSupervisorSpec>>newHashMap(),
new Folder3<Map<String, List<VersionedSupervisorSpec>>, Pair<String, VersionedSupervisorSpec>>()
{
@Override
public Map<String, List<VersionedSupervisorSpec>> fold(
Map<String, List<VersionedSupervisorSpec>> retVal,
Pair<String, VersionedSupervisorSpec> pair,
FoldController foldController,
StatementContext statementContext
) throws SQLException
{
try {
String specId = pair.lhs;
if (!retVal.containsKey(specId)) {
retVal.put(specId, Lists.<VersionedSupervisorSpec>newArrayList());
}
retVal.get(specId).add(pair.rhs);
return retVal;
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
}
);
}
}
)
);
}
@Override
public Map<String, SupervisorSpec> getLatest()
{
return ImmutableMap.copyOf(
dbi.withHandle(
new HandleCallback<Map<String, SupervisorSpec>>()
{
@Override
public Map<String, SupervisorSpec> 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<Pair<String, SupervisorSpec>>()
{
@Override
public Pair<String, SupervisorSpec> map(int index, ResultSet r, StatementContext ctx)
throws SQLException
{
try {
return Pair.of(
r.getString("spec_id"),
jsonMapper.<SupervisorSpec>readValue(
r.getBytes("payload"), new TypeReference<SupervisorSpec>()
{
}
)
);
}
catch (IOException e) {
throw Throwables.propagate(e);
}
}
}
).fold(
Maps.<String, SupervisorSpec>newHashMap(),
new Folder3<Map<String, SupervisorSpec>, Pair<String, SupervisorSpec>>()
{
@Override
public Map<String, SupervisorSpec> fold(
Map<String, SupervisorSpec> retVal,
Pair<String, SupervisorSpec> 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();
}
}

View File

@ -54,12 +54,14 @@ public class SQLMetadataConnectorTest
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<Void>()

View File

@ -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<String, String> data1rev1 = ImmutableMap.of("key1-1", "value1-1-1", "key1-2", "value1-2-1");
final Map<String, String> data1rev2 = ImmutableMap.of("key1-1", "value1-1-2", "key1-2", "value1-2-2");
final Map<String, String> data1rev3 = ImmutableMap.of("key1-1", "value1-1-3", "key1-2", "value1-2-3");
final Map<String, String> data2rev1 = ImmutableMap.of("key2-1", "value2-1-1", "key2-2", "value2-2-1");
final Map<String, String> 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<String, List<VersionedSupervisorSpec>> supervisorSpecs = supervisorManager.getAll();
Map<String, SupervisorSpec> 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<Void>()
{
@Override
public Void withHandle(Handle handle) throws Exception
{
handle.createStatement(String.format("DROP TABLE %s", tablesConfig.getSupervisorTable()))
.execute();
return null;
}
}
);
}
}

View File

@ -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;
}
}

View File

@ -71,6 +71,7 @@ import io.druid.indexing.overlord.config.TaskQueueConfig;
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 class CliOverlord extends ServerRunnable
binder.bind(JettyServerInitializer.class).toInstance(new OverlordJettyServerInitializer());
Jerseys.addResource(binder, OverlordResource.class);
Jerseys.addResource(binder, SupervisorResource.class);
LifecycleModule.register(binder, Server.class);
}

View File

@ -115,5 +115,6 @@ public class CreateTables extends GuiceRunnable
dbConnector.createConfigTable();
dbConnector.createTaskTables();
dbConnector.createAuditTable();
dbConnector.createSupervisorsTable();
}
}