Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Supervisor for KafkaIndexTask #2656

Merged
merged 2 commits into from
May 5, 2016
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -51,4 +51,6 @@ byte[] lookup(
void createTaskTables();

void createAuditTable();

void createSupervisorsTable();
}
Original file line number Diff line number Diff line change
Expand Up @@ -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";
Expand Down Expand Up @@ -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,
Expand All @@ -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
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

needs doc update

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sounds good

)
{
this.base = (base == null) ? DEFAULT_BASE : base;
Expand All @@ -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)
Expand Down Expand Up @@ -170,4 +174,8 @@ public String getAuditTable()
return auditTable;
}

public String getSupervisorTable()
{
return supervisorTable;
}
}
1 change: 1 addition & 0 deletions docs/content/configuration/index.md
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
240 changes: 240 additions & 0 deletions docs/content/ingestion/kafka-ingestion.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,240 @@
---
layout: doc_page
---

# Kafka Ingestion

The recommended way of ingesting data from Kafka is to use the `kafka-indexing-service` core extension (see
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I like the optimism but this language is a bit strong for first release :)

The language here should be telling people that this is an experimental feature, API subject to change, etc.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

haha, sounds good

[Including Extensions](../operations/including-extensions.html)). 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. The supervisor oversees the state of the indexing tasks to
coordinate handoffs, manage failures, and ensure that the scalability and replication requirements are maintained.

## 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",
"kafkaBrokers": "<BROKER_1>:<PORT_1>,<BROKER_2>:<PORT_2>,...",
"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)|
|`basePersistDirectory`|String|The local directory used for persisted data.|no (default == Java temp directory)|
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

basePersistDirectory is ignored by tasks (it's overridden to the task dir) so we can take this out.

|`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|
|`kafkaBrokers`|String|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)|
|`consumerProperties`|Map<String, String>|A map of properties to be passed to the Kafka consumer. Note that 'bootstrap.servers' will be overridden with the value in `kafkaBrokers`.|no|
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

[minor] I think it would more user friendly to get bootstrap.servers from here rather than have a separate kafkaBrokers thing. (it better leverages users' existing knowledge of kafka and existing kafka consumer configs)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sounds good

|`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
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

what happens if you submit a new supervisor before the old tasks exit? (users might want to read about this in the docs)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Will add it to the docs. If the new supervisor has a different spec, it will kill the previous tasks, which may or may not have already published the segment it was working on.

endpoint. The updated supervisor will begin reading from the offsets where the previous supervisor ended and no data
will be lost.
1 change: 1 addition & 0 deletions docs/content/toc.md
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
* [Stream Ingestion](../ingestion/stream-ingestion.html)
* [Stream Push](../ingestion/stream-push.html)
* [Stream Pull](../ingestion/stream-pull.html)
* [Kafka Ingestion](../ingestion/kafka-ingestion.html)
* [Updating Existing Data](../ingestion/update-existing-data.html)
* [FAQ](../ingestion/faq.html)

Expand Down
Loading