refactor extensions into their own docs

This commit is contained in:
fjy 2016-03-22 13:54:49 -07:00
parent 00d7021291
commit 943cbe6e76
27 changed files with 954 additions and 883 deletions

View File

@ -1,7 +1,9 @@
--- ---
layout: doc_page layout: doc_page
--- ---
# Deep Storage # Deep Storage
Deep storage is where segments are stored. It is a storage mechanism that Druid does not provide. This deep storage infrastructure defines the level of durability of your data, as long as Druid nodes can see this storage infrastructure and get at the segments stored on it, you will not lose data no matter how many Druid nodes you lose. If segments disappear from this storage layer, then you will lose whatever data those segments represented. Deep storage is where segments are stored. It is a storage mechanism that Druid does not provide. This deep storage infrastructure defines the level of durability of your data, as long as Druid nodes can see this storage infrastructure and get at the segments stored on it, you will not lose data no matter how many Druid nodes you lose. If segments disappear from this storage layer, then you will lose whatever data those segments represented.
## Local Mount ## Local Mount
@ -21,24 +23,12 @@ If you are using the Hadoop indexer in local mode, then just give it a local fil
## S3-compatible ## S3-compatible
S3-compatible deep storage is basically either S3 or something like Google Storage which exposes the same API as S3. See [druid-s3-extensions extension documentation](../development/extensions-core/s3.html).
S3 configuration parameters are
|Property|Possible Values|Description|Default|
|--------|---------------|-----------|-------|
|`druid.s3.accessKey`||S3 access key.|Must be set.|
|`druid.s3.secretKey`||S3 secret key.|Must be set.|
|`druid.storage.bucket`||Bucket to store in.|Must be set.|
|`druid.storage.baseKey`||Base key prefix to use, i.e. what directory.|Must be set.|
## HDFS ## HDFS
In order to use hdfs for deep storage, you need to set the following configuration in your common configs. See [druid-hdfs-storage extension documentation](../development/extensions-core/hdfs.html).
|Property|Possible Values|Description|Default| ## Additional Deep Stores
|--------|---------------|-----------|-------|
|`druid.storage.type`|hdfs||Must be set.|
|`druid.storage.storageDirectory`||Directory for storing segments.|Must be set.|
If you are using the Hadoop indexer, set your output directory to be a location on Hadoop and it will work For additional deep stores, please see our [extensions list](../development/extensions.html).

View File

@ -1,128 +1,19 @@
--- ---
layout: doc_page layout: doc_page
--- ---
# Metadata Storage # Metadata Storage
The Metadata Storage is an external dependency of Druid. Druid uses it to store The Metadata Storage is an external dependency of Druid. Druid uses it to store
various metadata about the system, but not to store the actual data. There are various metadata about the system, but not to store the actual data. There are
a number of tables used for various purposes described below. a number of tables used for various purposes described below.
## Supported Metadata Storages Derby is the default metadata store for Druid, however, it is not suitable for production.
[MySQL](../development/extensions-core/mysql.html) and [PostgreSQL](../development/extensions-core/postgresql.html) are more production suitable metadata stores.
The following metadata storage engines are supported: <div class="note caution">
Derby is not suitable for production use as a metadata store. Use MySQL or PostgreSQL instead.
* Derby (default, but not suitable for production) </div>
* MySQL
* PostgreSQL
Even though Derby is the default, it works only if you have all Druid
processes running on the same host, and should be used only for experimentation.
For production, MySQL or PostgreSQL should be used.
To choose the metadata storage type, set `druid.metadata.storage.type` to
`mysql`, `postgres` or `derby`.
Set other `druid.metadata.storage` configuration
keywords as shown below to give Druid information about how to connect to
the database.
As discussed in [Including Extensions](../operations/including-extensions.html),
there are two ways for giving Druid the extension files it needs for the
database you are using.
The first is to put the extension files in the classpath. The second is to
put the extension files in a subdirectory of
`druid.extensions.directory` (by default `extensions` under the Druid working directory) and list the subdirectory name in
`druid.extensions.loadList`. The example properties below show the second
way.
## Setting up MySQL
1. Install MySQL
Use your favorite package manager to install mysql, e.g.:
- on Ubuntu/Debian using apt `apt-get install mysql-server`
- on OS X, using [Homebrew](http://brew.sh/) `brew install mysql`
Alternatively, download and follow installation instructions for MySQL
Community Server here:
[http://dev.mysql.com/downloads/mysql/](http://dev.mysql.com/downloads/mysql/)
2. Create a druid database and user
Connect to MySQL from the machine where it is installed.
```bash
> mysql -u root
```
Paste the following snippet into the mysql prompt:
```sql
-- create a druid database, make sure to use utf8 as encoding
CREATE DATABASE druid DEFAULT CHARACTER SET utf8;
-- create a druid user, and grant it all permission on the database we just created
GRANT ALL ON druid.* TO 'druid'@'localhost' IDENTIFIED BY 'diurd';
```
3. Configure your Druid metadata storage extension:
Add the following parameters to your Druid configuration, replacing `<host>`
with the location (host name and port) of the database.
```properties
druid.extensions.loadList=["mysql-metadata-storage"]
druid.metadata.storage.type=mysql
druid.metadata.storage.connector.connectURI=jdbc:mysql://<host>/druid
druid.metadata.storage.connector.user=druid
druid.metadata.storage.connector.password=diurd
```
Note: the metadata storage extension is not packaged within the main Druid tarball; it is
packaged in a separate tarball that can be downloaded from [here](http://druid.io/downloads.html).
You can also get it using [pull-deps](../pull-deps.html), or you can build
it from source code; see [Build from Source](../development/build.html).
## Setting up PostgreSQL
1. Install PostgreSQL
Use your favorite package manager to install PostgreSQL, e.g.:
- on Ubuntu/Debian using apt `apt-get install postgresql`
- on OS X, using [Homebrew](http://brew.sh/) `brew install postgresql`
2. Create a druid database and user
On the machine where PostgreSQL is installed, using an account with proper
postgresql permissions:
Create a druid user, enter `diurd` when prompted for the password.
```bash
createuser druid -P
```
Create a druid database owned by the user we just created
```bash
createdb druid -O druid
```
*Note:* On Ubuntu / Debian you may have to prefix the `createuser` and
`createdb` commands with `sudo -u postgres` in order to gain proper
permissions.
3. Configure your Druid metadata storage extension:
Add the following parameters to your Druid configuration, replacing `<host>`
with the location (host name and port) of the database.
```properties
druid.extensions.loadList=["postgresql-metadata-storage"]
druid.metadata.storage.type=postgresql
druid.metadata.storage.connector.connectURI=jdbc:postgresql://<host>/druid
druid.metadata.storage.connector.user=druid
druid.metadata.storage.connector.password=diurd
```
## Using derby ## Using derby
@ -132,6 +23,14 @@ way.
druid.metadata.storage.type=derby druid.metadata.storage.type=derby
druid.metadata.storage.connector.connectURI=jdbc:derby://localhost:1527//home/y/var/druid_state/derby;create=true druid.metadata.storage.connector.connectURI=jdbc:derby://localhost:1527//home/y/var/druid_state/derby;create=true
``` ```
## MySQL
See [mysql-metadata-storage extension documentation](../development/extensions-core/mysql.html).
## PostgreSQL
See [postgresql-metadata-storage](../development/extensions-core/postgresql.html).
## Metadata Storage Tables ## Metadata Storage Tables

View File

@ -2,7 +2,9 @@
layout: doc_page layout: doc_page
--- ---
### Approximate Histogram aggregator # Approximate Histogram aggregator
Make sure to [include](../../operations/including-extensions.html) `druid-histogram` as an extension.
This aggregator is based on This aggregator is based on
[http://jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf](http://jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf) [http://jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf](http://jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf)

View File

@ -0,0 +1,114 @@
---
layout: doc_page
---
# Avro
This extension enables Druid to ingest and understand the Apache Avro data format. Make sure to [include](../../operations/including-extensions.html) `druid-avro-extensions` as an extension.
### Avro Stream Parser
This is for streaming/realtime ingestion.
| Field | Type | Description | Required |
|-------|------|-------------|----------|
| type | String | This should say `avro_stream`. | no |
| avroBytesDecoder | JSON Object | Specifies how to decode bytes to Avro record. | yes |
| parseSpec | JSON Object | Specifies the timestamp and dimensions of the data. Should be a timeAndDims parseSpec. | yes |
For example, using Avro stream parser with schema repo Avro bytes decoder:
```json
"parser" : {
"type" : "avro_stream",
"avroBytesDecoder" : {
"type" : "schema_repo",
"subjectAndIdConverter" : {
"type" : "avro_1124",
"topic" : "${YOUR_TOPIC}"
},
"schemaRepository" : {
"type" : "avro_1124_rest_client",
"url" : "${YOUR_SCHEMA_REPO_END_POINT}",
}
},
"parseSpec" : {
"type": "timeAndDims",
"timestampSpec": <standard timestampSpec>,
"dimensionsSpec": <standard dimensionsSpec>
}
}
```
#### Avro Bytes Decoder
If `type` is not included, the avroBytesDecoder defaults to `schema_repo`.
##### SchemaRepo Based Avro Bytes Decoder
This Avro bytes decoder first extract `subject` and `id` from input message bytes, then use them to lookup the Avro schema with which to decode Avro record from bytes. Details can be found in [schema repo](https://github.com/schema-repo/schema-repo) and [AVRO-1124](https://issues.apache.org/jira/browse/AVRO-1124). You will need an http service like schema repo to hold the avro schema. Towards schema registration on the message producer side, you can refer to `io.druid.data.input.AvroStreamInputRowParserTest#testParse()`.
| Field | Type | Description | Required |
|-------|------|-------------|----------|
| type | String | This should say `schema_repo`. | no |
| subjectAndIdConverter | JSON Object | Specifies the how to extract subject and id from message bytes. | yes |
| schemaRepository | JSON Object | Specifies the how to lookup Avro schema from subject and id. | yes |
##### Avro-1124 Subject And Id Converter
| Field | Type | Description | Required |
|-------|------|-------------|----------|
| type | String | This should say `avro_1124`. | no |
| topic | String | Specifies the topic of your kafka stream. | yes |
##### Avro-1124 Schema Repository
| Field | Type | Description | Required |
|-------|------|-------------|----------|
| type | String | This should say `avro_1124_rest_client`. | no |
| url | String | Specifies the endpoint url of your Avro-1124 schema repository. | yes |
### Avro Hadoop Parser
This is for batch ingestion using the HadoopDruidIndexer. The `inputFormat` of `inputSpec` in `ioConfig` must be set to `"io.druid.data.input.avro.AvroValueInputFormat"`. You may want to set Avro reader's schema in `jobProperties` in `tuningConfig`, eg: `"avro.schema.path.input.value": "/path/to/your/schema.avsc"` or `"avro.schema.input.value": "your_schema_JSON_object"`, if reader's schema is not set, the schema in Avro object container file will be used, see [Avro specification](http://avro.apache.org/docs/1.7.7/spec.html#Schema+Resolution). Make sure to include "io.druid.extensions:druid-avro-extensions" as an extension.
| Field | Type | Description | Required |
|-------|------|-------------|----------|
| type | String | This should say `avro_hadoop`. | no |
| parseSpec | JSON Object | Specifies the timestamp and dimensions of the data. Should be a timeAndDims parseSpec. | yes |
| fromPigAvroStorage | Boolean | Specifies whether the data file is stored using AvroStorage. | no(default == false) |
For example, using Avro Hadoop parser with custom reader's schema file:
```json
{
"type" : "index_hadoop",
"spec" : {
"dataSchema" : {
"dataSource" : "",
"parser" : {
"type" : "avro_hadoop",
"parseSpec" : {
"type": "timeAndDims",
"timestampSpec": <standard timestampSpec>,
"dimensionsSpec": <standard dimensionsSpec>
}
}
},
"ioConfig" : {
"type" : "hadoop",
"inputSpec" : {
"type" : "static",
"inputFormat": "io.druid.data.input.avro.AvroValueInputFormat",
"paths" : ""
}
},
"tuningConfig" : {
"jobProperties" : {
"avro.schema.path.input.value" : "/path/to/my/schema.avsc",
}
}
}
}
```

View File

@ -8,7 +8,7 @@ Druid aggregators based on [datasketches](http://datasketches.github.io/) librar
At ingestion time, this aggregator creates the theta sketch objects which get stored in Druid segments. Logically speaking, a theta sketch object can be thought of as a Set data structure. At query time, sketches are read and aggregated (set unioned) together. In the end, by default, you receive the estimate of the number of unique entries in the sketch object. Also, you can use post aggregators to do union, intersection or difference on sketch columns in the same row. At ingestion time, this aggregator creates the theta sketch objects which get stored in Druid segments. Logically speaking, a theta sketch object can be thought of as a Set data structure. At query time, sketches are read and aggregated (set unioned) together. In the end, by default, you receive the estimate of the number of unique entries in the sketch object. Also, you can use post aggregators to do union, intersection or difference on sketch columns in the same row.
Note that you can use `thetaSketch` aggregator on columns which were not ingested using same, it will return estimated cardinality of the column. It is recommended to use it at ingestion time as well to make querying faster. Note that you can use `thetaSketch` aggregator on columns which were not ingested using same, it will return estimated cardinality of the column. It is recommended to use it at ingestion time as well to make querying faster.
To use the datasketch aggregators, make sure you include the extension in your config file: To use the datasketch aggregators, make sure you [include](../operations/including-extensions.html) the extension in your config file:
``` ```
druid.extensions.loadList=["druid-datasketches"] druid.extensions.loadList=["druid-datasketches"]
@ -20,26 +20,20 @@ druid.extensions.loadList=["druid-datasketches"]
{ {
"type" : "thetaSketch", "type" : "thetaSketch",
"name" : <output_name>, "name" : <output_name>,
"fieldName" : <metric_name>, "fieldName" : <metric_name>,
"isInputThetaSketch": false,
//following boolean field is optional. This should only be used at
//indexing time if your input data contains theta sketch objects.
//that would be the case if you use datasketches library outside of Druid,
//say with Pig/Hive, to produce the data that you are ingesting into Druid
"isInputThetaSketch": false
//following field is optional, default = 16384. must be a power of 2.
//Internally, size refers to the maximum number
//of entries sketch object will retain, higher size would mean higher
//accuracy but higher space needed to store those sketches.
//note that after you index with a particular size, druid will persist sketch in segments
//and you will use size greater or equal to that at query time.
//See [theta-size](http://datasketches.github.io/docs/ThetaSize.html) for details.
//In general, We recommend just sticking to default size, which has worked well.
"size": 16384 "size": 16384
} }
``` ```
|property|description|required?|
|--------|-----------|---------|
|type|This String should always be "thetaSketch"|yes|
|name|A String for the output (result) name of the calculation.|yes|
|fieldName|A String for the name of the aggregator used at ingestion time.|yes|
|isInputThetaSketch|This should only be used at indexing time if your input data contains theta sketch objects. This would be the case if you use datasketches library outside of Druid, say with Pig/Hive, to produce the data that you are ingesting into Druid |no, defaults to false|
|size|Must be a power of 2. Internally, size refers to the maximum number of entries sketch object will retain. Higher size means higher accuracy but more space to store sketches. Note that after you index with a particular size, druid will persist sketch in segments and you will use size greater or equal to that at query time. See [theta-size](http://datasketches.github.io/docs/ThetaSize.html) for details. In general, We recommend just sticking to default size. |no, defaults to 16384|
### Post Aggregators ### Post Aggregators
#### Sketch Estimator #### Sketch Estimator

View File

@ -0,0 +1,25 @@
---
layout: doc_page
---
# Druid examples
## TwitterSpritzerFirehose
This firehose connects directly to the twitter spritzer data stream.
Sample spec:
```json
"firehose" : {
"type" : "twitzer",
"maxEventCount": -1,
"maxRunMinutes": 0
}
```
|property|description|default|required?|
|--------|-----------|-------|---------|
|type|This should be "twitzer"|N/A|yes|
|maxEventCount|max events to receive, -1 is infinite, 0 means nothing is delivered; use this to prevent infinite space consumption or to prevent getting throttled at an inconvenient time.|N/A|yes|
|maxRunMinutes|maximum number of minutes to fetch Twitter events. Use this to prevent getting throttled at an inconvenient time. If zero or less, no time limit for run.|N/A|yes|

View File

@ -0,0 +1,18 @@
---
layout: doc_page
---
# HDFS
Make sure to [include](../../operations/including-extensions.html) `druid-hdfs-storage` as an extension.
## Deep Storage
### Configuration
|Property|Possible Values|Description|Default|
|--------|---------------|-----------|-------|
|`druid.storage.type`|hdfs||Must be set.|
|`druid.storage.storageDirectory`||Directory for storing segments.|Must be set.|
If you are using the Hadoop indexer, set your output directory to be a location on Hadoop and it will work

View File

@ -0,0 +1,34 @@
---
layout: doc_page
---
# Kafka Eight Firehose
Make sure to [include](../../operations/including-extensions.html) `druid-kafka-eight` as an extension.
This firehose acts as a Kafka 0.8.x consumer and ingests data from Kafka.
Sample spec:
```json
"firehose": {
"type": "kafka-0.8",
"consumerProps": {
"zookeeper.connect": "localhost:2181",
"zookeeper.connection.timeout.ms" : "15000",
"zookeeper.session.timeout.ms" : "15000",
"zookeeper.sync.time.ms" : "5000",
"group.id": "druid-example",
"fetch.message.max.bytes" : "1048586",
"auto.offset.reset": "largest",
"auto.commit.enable": "false"
},
"feed": "wikipedia"
}
```
|property|description|required?|
|--------|-----------|---------|
|type|This should be "kafka-0.8"|yes|
|consumerProps|The full list of consumer configs can be [here](https://kafka.apache.org/08/configuration.html).|yes|
|feed|Kafka maintains feeds of messages in categories called topics. This is the topic name.|yes|

View File

@ -0,0 +1,60 @@
---
layout: doc_page
---
# Kafka Namespaced Lookup
<div class="note caution">
Lookups are an <a href="../development/experimental.html">experimental</a> feature.
</div>
Make sure to [include](../../operations/including-extensions.html) `druid-namespace-lookup` and `druid-kafka-extraction-namespace` as an extension.
Note that this lookup does not employ a `pollPeriod`.
If you need updates to populate as promptly as possible, it is possible to plug into a kafka topic whose key is the old value and message is the desired new value (both in UTF-8). This requires the following extension: "io.druid.extensions:kafka-extraction-namespace"
```json
{
"type":"kafka",
"namespace":"testTopic",
"kafkaTopic":"testTopic"
}
```
|Parameter|Description|Required|Default|
|---------|-----------|--------|-------|
|`namespace`|The namespace to define|Yes||
|`kafkaTopic`|The kafka topic to read the data from|Yes||
## Kafka renames
The extension `kafka-extraction-namespace` enables reading from a kafka feed which has name/key pairs to allow renaming of dimension values. An example use case would be to rename an ID to a human readable format.
Currently the historical node caches the key/value pairs from the kafka feed in an ephemeral memory mapped DB via MapDB.
## Configuration
The following options are used to define the behavior and should be included wherever the extension is included (all query servicing nodes):
|Property|Description|Default|
|--------|-----------|-------|
|`druid.query.rename.kafka.properties`|A json map of kafka consumer properties. See below for special properties.|See below|
The following are the handling for kafka consumer properties in `druid.query.rename.kafka.properties`
|Property|Description|Default|
|--------|-----------|-------|
|`zookeeper.connect`|Zookeeper connection string|`localhost:2181/kafka`|
|`group.id`|Group ID, auto-assigned for publish-subscribe model and cannot be overridden|`UUID.randomUUID().toString()`|
|`auto.offset.reset`|Setting to get the entire kafka rename stream. Cannot be overridden|`smallest`|
## Testing the Kafka rename functionality
To test this setup, you can send key/value pairs to a kafka stream via the following producer console:
```
./bin/kafka-console-producer.sh --property parse.key=true --property key.separator="->" --broker-list localhost:9092 --topic testTopic
```
Renames can then be published as `OLD_VAL->NEW_VAL` followed by newline (enter or return)

View File

@ -0,0 +1,55 @@
---
layout: doc_page
---
# MySQL Metadata Store
Make sure to [include](../../operations/including-extensions.html) `mysql-metadata-storage` as an extension.
## Setting up MySQL
1. Install MySQL
Use your favorite package manager to install mysql, e.g.:
- on Ubuntu/Debian using apt `apt-get install mysql-server`
- on OS X, using [Homebrew](http://brew.sh/) `brew install mysql`
Alternatively, download and follow installation instructions for MySQL
Community Server here:
[http://dev.mysql.com/downloads/mysql/](http://dev.mysql.com/downloads/mysql/)
2. Create a druid database and user
Connect to MySQL from the machine where it is installed.
```bash
> mysql -u root
```
Paste the following snippet into the mysql prompt:
```sql
-- create a druid database, make sure to use utf8 as encoding
CREATE DATABASE druid DEFAULT CHARACTER SET utf8;
-- create a druid user, and grant it all permission on the database we just created
GRANT ALL ON druid.* TO 'druid'@'localhost' IDENTIFIED BY 'diurd';
```
3. Configure your Druid metadata storage extension:
Add the following parameters to your Druid configuration, replacing `<host>`
with the location (host name and port) of the database.
```properties
druid.extensions.loadList=["mysql-metadata-storage"]
druid.metadata.storage.type=mysql
druid.metadata.storage.connector.connectURI=jdbc:mysql://<host>/druid
druid.metadata.storage.connector.user=druid
druid.metadata.storage.connector.password=diurd
```
Note: the metadata storage extension is not packaged within the main Druid tarball; it is
packaged in a separate tarball that can be downloaded from [here](http://druid.io/downloads.html).
You can also get it using [pull-deps](../pull-deps.html), or you can build
it from source code; see [Build from Source](../development/build.html).

View File

@ -0,0 +1,498 @@
---
layout: doc_page
---
# Namespaced Lookup
<div class="note caution">
Lookups are an <a href="../development/experimental.html">experimental</a> feature.
</div>
Make sure to [include](../../operations/including-extensions.html) `druid-namespace-lookup` as an extension.
## Configuration
|Property|Description|Default|
|--------|-----------|-------|
|`druid.lookup.snapshotWorkingDir`| Working path used to store snapshot of current lookup configuration, leaving this property null will disable snapshot/bootstrap utility|null|
Namespaced lookups are appropriate for lookups which are not possible to pass at query time due to their size,
or are not desired to be passed at query time because the data is to reside in and be handled by the Druid servers.
Namespaced lookups can be specified as part of the runtime properties file. The property is a list of the namespaces
described as per the sections on this page. For example:
```json
druid.query.extraction.namespace.lookups=
[
{
"type": "uri",
"namespace": "some_uri_lookup",
"uri": "file:/tmp/prefix/",
"namespaceParseSpec": {
"format": "csv",
"columns": [
"key",
"value"
]
},
"pollPeriod": "PT5M"
},
{
"type": "jdbc",
"namespace": "some_jdbc_lookup",
"connectorConfig": {
"createTables": true,
"connectURI": "jdbc:mysql:\/\/localhost:3306\/druid",
"user": "druid",
"password": "diurd"
},
"table": "lookupTable",
"keyColumn": "mykeyColumn",
"valueColumn": "MyValueColumn",
"tsColumn": "timeColumn"
}
]
```
Proper functionality of Namespaced lookups requires the following extension to be loaded on the broker, peon, and historical nodes:
`druid-namespace-lookup`
## Cache Settings
Lookups are cached locally on historical nodes. The following are settings used by the nodes which service queries when
setting namespaces (broker, peon, historical)
|Property|Description|Default|
|--------|-----------|-------|
|`druid.query.extraction.namespace.cache.type`|Specifies the type of caching to be used by the namespaces. May be one of [`offHeap`, `onHeap`]. `offHeap` uses a temporary file for off-heap storage of the namespace (memory mapped files). `onHeap` stores all cache on the heap in standard java map types.|`onHeap`|
The cache is populated in different ways depending on the settings below. In general, most namespaces employ
a `pollPeriod` at the end of which time they poll the remote resource of interest for updates.
# Supported Lookups
For additional lookups, please see our [extensions list](../development/extensions.html).
## URI namespace update
The remapping values for each namespaced lookup can be specified by json as per
```json
{
"type":"uri",
"namespace":"some_lookup",
"uri": "s3://bucket/some/key/prefix/",
"namespaceParseSpec":{
"format":"csv",
"columns":["key","value"]
},
"pollPeriod":"PT5M",
"versionRegex": "renames-[0-9]*\\.gz"
}
```
|Property|Description|Required|Default|
|--------|-----------|--------|-------|
|`namespace`|The namespace to define|Yes||
|`pollPeriod`|Period between polling for updates|No|0 (only once)|
|`versionRegex`|Regex to help find newer versions of the namespace data|Yes||
|`namespaceParseSpec`|How to interpret the data at the URI|Yes||
The `pollPeriod` value specifies the period in ISO 8601 format between checks for updates. If the source of the lookup is capable of providing a timestamp, the lookup will only be updated if it has changed since the prior tick of `pollPeriod`. A value of 0, an absent parameter, or `null` all mean populate once and do not attempt to update. Whenever an update occurs, the updating system will look for a file with the most recent timestamp and assume that one with the most recent data.
The `versionRegex` value specifies a regex to use to determine if a filename in the parent path of the uri should be considered when trying to find the latest version. Omitting this setting or setting it equal to `null` will match to all files it can find (equivalent to using `".*"`). The search occurs in the most significant "directory" of the uri.
The `namespaceParseSpec` can be one of a number of values. Each of the examples below would rename foo to bar, baz to bat, and buck to truck. All parseSpec types assumes each input is delimited by a new line. See below for the types of parseSpec supported.
### csv lookupParseSpec
|Parameter|Description|Required|Default|
|---------|-----------|--------|-------|
|`columns`|The list of columns in the csv file|yes|`null`|
|`keyColumn`|The name of the column containing the key|no|The first column|
|`valueColumn`|The name of the column containing the value|no|The second column|
*example input*
```
bar,something,foo
bat,something2,baz
truck,something3,buck
```
*example namespaceParseSpec*
```json
"namespaceParseSpec": {
"format": "csv",
"columns": ["value","somethingElse","key"],
"keyColumn": "key",
"valueColumn": "value"
}
```
### tsv lookupParseSpec
|Parameter|Description|Required|Default|
|---------|-----------|--------|-------|
|`columns`|The list of columns in the csv file|yes|`null`|
|`keyColumn`|The name of the column containing the key|no|The first column|
|`valueColumn`|The name of the column containing the value|no|The second column|
|`delimiter`|The delimiter in the file|no|tab (`\t`)|
*example input*
```
bar|something,1|foo
bat|something,2|baz
truck|something,3|buck
```
*example namespaceParseSpec*
```json
"namespaceParseSpec": {
"format": "tsv",
"columns": ["value","somethingElse","key"],
"keyColumn": "key",
"valueColumn": "value",
"delimiter": "|"
}
```
### customJson lookupParseSpec
|Parameter|Description|Required|Default|
|---------|-----------|--------|-------|
|`keyFieldName`|The field name of the key|yes|null|
|`valueFieldName`|The field name of the value|yes|null|
*example input*
```json
{"key": "foo", "value": "bar", "somethingElse" : "something"}
{"key": "baz", "value": "bat", "somethingElse" : "something"}
{"key": "buck", "somethingElse": "something", "value": "truck"}
```
*example namespaceParseSpec*
```json
"namespaceParseSpec": {
"format": "customJson",
"keyFieldName": "key",
"valueFieldName": "value"
}
```
### simpleJson lookupParseSpec
The `simpleJson` lookupParseSpec does not take any parameters. It is simply a line delimited json file where the field is the key, and the field's value is the value.
*example input*
```json
{"foo": "bar"}
{"baz": "bat"}
{"buck": "truck"}
```
*example namespaceParseSpec*
```json
"namespaceParseSpec":{
"format": "simpleJson"
}
```
## JDBC namespaced lookup
The JDBC lookups will poll a database to populate its local cache. If the `tsColumn` is set it must be able to accept comparisons in the format `'2015-01-01 00:00:00'`. For example, the following must be valid sql for the table `SELECT * FROM some_lookup_table WHERE timestamp_column > '2015-01-01 00:00:00'`. If `tsColumn` is set, the caching service will attempt to only poll values that were written *after* the last sync. If `tsColumn` is not set, the entire table is pulled every time.
|Parameter|Description|Required|Default|
|---------|-----------|--------|-------|
|`namespace`|The namespace to define|Yes||
|`connectorConfig`|The connector config to use|Yes||
|`table`|The table which contains the key value pairs|Yes||
|`keyColumn`|The column in `table` which contains the keys|Yes||
|`valueColumn`|The column in `table` which contains the values|Yes||
|`tsColumn`| The column in `table` which contains when the key was updated|No|Not used|
|`pollPeriod`|How often to poll the DB|No|0 (only once)|
```json
{
"type":"jdbc",
"namespace":"some_lookup",
"connectorConfig":{
"createTables":true,
"connectURI":"jdbc:mysql://localhost:3306/druid",
"user":"druid",
"password":"diurd"
},
"table":"some_lookup_table",
"keyColumn":"the_old_dim_value",
"valueColumn":"the_new_dim_value",
"tsColumn":"timestamp_column",
"pollPeriod":600000
}
```
Dynamic configuration (EXPERIMENTAL)
------------------------------------
The following documents the behavior of the cluster-wide config which is accessible through the coordinator.
The configuration is propagated through the concept of "tier" of servers.
A "tier" is defined as a group of services which should receive a set of lookups.
For example, you might have all historicals be part of `__default`, and Peons be part of individual tiers for the datasources they are tasked with.
The tiers for lookups are completely independent of historical tiers.
These configs are accessed using JSON through the following URI template
```
http://<COORDINATOR_IP>:<PORT>/druid/coordinator/v1/lookups/{tier}/{id}
```
All URIs below are assumed to have `http://<COORDINATOR_IP>:<PORT>` prepended.
If you have NEVER configured lookups before, you MUST post an empty json object `{}` to `/druid/coordinator/v1/lookups` to initialize the configuration.
These endpoints will return one of the following results:
* 404 if the resource is not found
* 400 if there is a problem in the formatting of the request
* 202 if the request was accepted asynchronously (`POST` and `DELETE`)
* 200 if the request succeeded (`GET` only)
## Configuration propagation behavior
The configuration is propagated to the query serving nodes (broker / router / peon / historical) by the coordinator.
The query serving nodes have an internal API for managing `POST`/`GET`/`DELETE` of lookups.
The coordinator periodically checks the dynamic configuration for changes and, when it detects a change it does the following:
1. Post all lookups for a tier to all Druid nodes within that tier.
2. Delete lookups from a tier which were dropped between the prior configuration values and this one.
If there is no configuration change, the coordinator checks for any nodes which might be new since the last time it propagated lookups and adds all lookups for that node (assuming that node's tier has lookups).
If there are errors while trying to add or update configuration on a node, that node is temporarily skipped until the next management period. The next management period the update will attempt to be propagated again.
If there is an error while trying to delete a lookup from a node (or if a node is down when the coordinator is propagating the config), the delete is not attempted again. In such a case it is possible that a node has lookups that are no longer managed by the coordinator.
## Bulk update
Lookups can be updated in bulk by posting a JSON object to `/druid/coordinator/v1/lookups`. The format of the json object is as follows:
```json
{
"tierName": {
"lookupExtractorFactoryName": {
"someExtractorField": "someExtractorValue"
}
}
}
```
So a config might look something like:
```json
{
"__default": {
"country_code": {
"type": "simple_json",
"uri": "http://some.host.com/codes.json"
},
"site_id": {
"type": "confidential_jdbc",
"auth": "/etc/jdbc.internal",
"table": "sites",
"key": "site_id",
"value": "site_name"
},
"site_id_customer1": {
"type": "confidential_jdbc",
"auth": "/etc/jdbc.customer1",
"table": "sites",
"key": "site_id",
"value": "site_name"
},
"site_id_customer2": {
"type": "confidential_jdbc",
"auth": "/etc/jdbc.customer2",
"table": "sites",
"key": "site_id",
"value": "site_name"
}
},
"realtime_customer1": {
"country_code": {
"type": "simple_json",
"uri": "http://some.host.com/codes.json"
},
"site_id_customer1": {
"type": "confidential_jdbc",
"auth": "/etc/jdbc.customer1",
"table": "sites",
"key": "site_id",
"value": "site_name"
}
},
"realtime_customer2": {
"country_code": {
"type": "simple_json",
"uri": "http://some.host.com/codes.json"
},
"site_id_customer2": {
"type": "confidential_jdbc",
"auth": "/etc/jdbc.customer2",
"table": "sites",
"key": "site_id",
"value": "site_name"
}
}
}
```
All entries in the map will UPDATE existing entries. No entries will be deleted.
## Update Lookup
A `POST` to a particular lookup extractor factory via `/druid/coordinator/v1/lookups/{tier}/{id}` will update that specific extractor factory.
For example, a post to `/druid/coordinator/v1/lookups/realtime_customer1/site_id_customer1` might contain the following:
```json
{
"type": "confidential_jdbc",
"auth": "/etc/jdbc.customer1",
"table": "sites_updated",
"key": "site_id",
"value": "site_name"
}
```
This will replace the `site_id_customer1` lookup in the `realtime_customer1` with the definition above.
## Get Lookup
A `GET` to a particular lookup extractor factory is accomplished via `/druid/coordinator/v1/lookups/{tier}/{id}`
Using the prior example, a `GET` to `/druid/coordinator/v1/lookups/realtime_customer2/site_id_customer2` should return
```json
{
"type": "confidential_jdbc",
"auth": "/etc/jdbc.customer2",
"table": "sites",
"key": "site_id",
"value": "site_name"
}
```
## Delete Lookup
A `DELETE` to `/druid/coordinator/v1/lookups/{tier}/{id}` will remove that lookup from the cluster.
## List tier names
A `GET` to `/druid/coordinator/v1/lookups` will return a list of known tier names in the dynamic configuration.
To discover a list of tiers currently active in the cluster **instead of** ones known in the dynamic configuration, the parameter `discover=true` can be added as per `/druid/coordinator/v1/lookups?discover=true`.
## List lookup names
A `GET` to `/druid/coordinator/v1/lookups/{tier}` will return a list of known lookup names for that tier.
# Internal API
The Peon, Router, Broker, and Historical nodes all have the ability to consume lookup configuration.
There is an internal API these nodes use to list/load/drop their lookups starting at `/druid/listen/v1/lookups`.
These follow the same convention for return values as the cluster wide dynamic configuration.
Usage of these endpoints is quite advanced and not recommended for most users.
The endpoints are as follows:
## Get Lookups
A `GET` to the node at `/druid/listen/v1/lookups` will return a json map of all the lookups currently active on the node.
The return value will be a json map of the lookups to their extractor factories.
```json
{
"some_lookup_name": {
"type": "simple_json",
"uri": "http://some.host.com/codes.json"
}
}
```
## Get Lookup
A `GET` to the node at `/druid/listen/v1/lookups/some_lookup_name` will return the LookupExtractorFactory for the lookup identified by `some_lookup_name`.
The return value will be the json representation of the factory.
```json
{
"type": "simple_json",
"uri": "http://some.host.com/codes.json"
}
```
## Bulk Add or Update Lookups
A `POST` to the node at `/druid/listen/v1/lookups` of a JSON map of lookup names to LookupExtractorFactory will cause the service to add or update its lookups.
The return value will be a JSON map in the following format:
```json
{
"status": "accepted",
"failedUpdates": {}
}
```
If a lookup cannot be started, or is left in an undefined state, the lookup in error will be returned in the `failedUpdates` field as per:
```json
{
"status": "accepted",
"failedUpdates": {
"country_code": {
"type": "simple_json",
"uri": "http://some.host.com/codes.json"
}
}
}
```
The `failedUpdates` field of the return value should be checked if a user is wanting to assure that every update succeeded.
## Add or Update Lookup
A `POST` to the node at `/druid/listen/v1/lookups/some_lookup_name` will behave very similarly to a bulk update.
If `some_lookup_name` is desired to have the LookupExtractorFactory definition of
```json
{
"type": "simple_json",
"uri": "http://some.host.com/codes.json"
}
```
Then a post to `/druid/listen/v1/lookups/some_lookup_name` will behave the same as a `POST` to `/druid/listen/v1/lookups` of
```json
{
"some_lookup_name": {
"type": "simple_json",
"uri": "http://some.host.com/codes.json"
}
}
```
## Remove a Lookup
A `DELETE` to `/druid/listen/v1/lookups/some_lookup_name` will remove that lookup from the node. Success will reflect the ID.
# Configuration
See the [coordinator configuration guilde](../configuration/coordinator.html) for coordinator configuration
To configure a Broker / Router / Historical / Peon to announce itself as part of a lookup tier, use the `druid.zk.paths.lookupTier` property.
|Property | Description | Default |
|---------|-------------|---------|
|`druid.lookup.tierName`| The tier for **lookups** for this node. This is independent of other tiers.|`__default`|

View File

@ -0,0 +1,49 @@
---
layout: doc_page
---
# PostgreSQL Metadata Store
Make sure to [include](../../operations/including-extensions.html) `postgresql-metadata-storage` as an extension.
## Setting up PostgreSQL
1. Install PostgreSQL
Use your favorite package manager to install PostgreSQL, e.g.:
- on Ubuntu/Debian using apt `apt-get install postgresql`
- on OS X, using [Homebrew](http://brew.sh/) `brew install postgresql`
2. Create a druid database and user
On the machine where PostgreSQL is installed, using an account with proper
postgresql permissions:
Create a druid user, enter `diurd` when prompted for the password.
```bash
createuser druid -P
```
Create a druid database owned by the user we just created
```bash
createdb druid -O druid
```
*Note:* On Ubuntu / Debian you may have to prefix the `createuser` and
`createdb` commands with `sudo -u postgres` in order to gain proper
permissions.
3. Configure your Druid metadata storage extension:
Add the following parameters to your Druid configuration, replacing `<host>`
with the location (host name and port) of the database.
```properties
druid.extensions.loadList=["postgresql-metadata-storage"]
druid.metadata.storage.type=postgresql
druid.metadata.storage.connector.connectURI=jdbc:postgresql://<host>/druid
druid.metadata.storage.connector.user=druid
druid.metadata.storage.connector.password=diurd
```

View File

@ -0,0 +1,38 @@
---
layout: doc_page
---
# S3-compatible
Make sure to [include](../../operations/including-extensions.html) `druid-s3-extensions` as an extension.
## Deep Storage
S3-compatible deep storage is basically either S3 or something like Google Storage which exposes the same API as S3.
### Configuration
|Property|Possible Values|Description|Default|
|--------|---------------|-----------|-------|
|`druid.s3.accessKey`||S3 access key.|Must be set.|
|`druid.s3.secretKey`||S3 secret key.|Must be set.|
|`druid.storage.bucket`||Bucket to store in.|Must be set.|
|`druid.storage.baseKey`||Base key prefix to use, i.e. what directory.|Must be set.|
## StaticS3Firehose
This firehose ingests events from a predefined list of S3 objects.
Sample spec:
```json
"firehose" : {
"type" : "static-s3",
"uris": ["s3://foo/bar/file.gz", "s3://bar/foo/file2.gz"]
}
```
|property|description|default|required?|
|--------|-----------|-------|---------|
|type|This should be "static-s3"|N/A|yes|
|uris|JSON array of URIs where s3 files to be ingested are located.|N/A|yes|

View File

@ -21,16 +21,16 @@ Core extensions are maintained by Druid committers.
|Name|Description|Docs| |Name|Description|Docs|
|----|-----------|----| |----|-----------|----|
|druid-avro-extensions|Support for data in Apache Avro data format.|[link](../ingestion/index.html)| |druid-avro-extensions|Support for data in Apache Avro data format.|[link](../development/extensions-core/avro.html)|
|druid-datasketches|Support for approximate counts and set operations with [DataSketches](http://datasketches.github.io/).|[link](../development/datasketches-aggregators.html)| |druid-datasketches|Support for approximate counts and set operations with [DataSketches](http://datasketches.github.io/).|[link](../development/extensions-core/datasketches-aggregators.html)|
|druid-hdfs-storage|Support for data in Apache Avro data format.|[link](../ingestion/index.html)| |druid-hdfs-storage|HDFS deep storage.|[link](../development/extensions-core/hdfs.html)|
|druid-histogram|HDFS deep storage.|[link](../dependencies/deep-storage.html#hdfs)| |druid-histogram|Approximate histograms and quantiles aggregator.|[link](../development/extensions-core/approximate-histograms.html)|
|druid-kafka-eight|Kafka ingest firehose (high level consumer).|[link](../ingestion/firehose.html#kafkaeightfirehose)| |druid-kafka-eight|Kafka ingest firehose (high level consumer).|[link](../development/extensions-core/kafka-eight-firehose.html)|
|druid-kafka-extraction-namespace|Kafka namespaced lookup.|[link](../querying/lookups.html#kafka-namespaced-lookup)| |druid-kafka-extraction-namespace|Kafka-based namespaced lookup. Requires namespace lookup extension.|[link](../development/extensions-core/kafka-extraction-namespace.html)|
|druid-namespace-lookup|Namespaced lookups.|[link](../querying/lookups.html)| |druid-namespace-lookup|Required module for [lookups](../querying/lookups.html).|[link](../development/extensions-core/namespaced-lookup.html)|
|druid-s3-extensions|S3 deep storage.|[link](../dependencies/deep-storage.html#s3-compatible/)| |druid-s3-extensions|Interfacing with data in AWS S3, and using S3 as deep storage.|[link](../development/extensions-core/s3.html)|
|mysql-metadata-storage|MySQL metadata store.|[link](../dependencies/metadata-storage.html#setting-up-mysql)| |mysql-metadata-storage|MySQL metadata store.|[link](../development/extensions-core/mysql.html)|
|postgresql-metadata-storage|PostgreSQL metadata store.|[link](../dependencies/metadata-storage.html#setting-up-postgresql)| |postgresql-metadata-storage|PostgreSQL metadata store.|[link](../development/extensions-core/postgresql.html)|
# Community Extensions # Community Extensions
@ -40,13 +40,13 @@ If you'd like to take on maintenance for a community extension, please post on [
|Name|Description|Docs| |Name|Description|Docs|
|----|-----------|----| |----|-----------|----|
|druid-azure-extensions|Microsoft Azure deep storage.|[link](../development/community-extensions/azure.html)| |druid-azure-extensions|Microsoft Azure deep storage.|[link](../development/extensions-contrib/azure.html)|
|druid-cassandra-storage|Apache Cassandra deep storage.|[link](../development/community-extensions/cassandra.html)| |druid-cassandra-storage|Apache Cassandra deep storage.|[link](../development/extensions-contrib/cassandra.html)|
|druid-cloudfiles-extensions|Rackspace Cloudfiles deep storage and firehose.|[link](../development/community-extensions/cloudfiles.html)| |druid-cloudfiles-extensions|Rackspace Cloudfiles deep storage and firehose.|[link](../development/extensions-contrib/cloudfiles.html)|
|druid-kafka-eight-simpleConsumer|Kafka ingest firehose (low level consumer).|[link](../development/community-extensions/kafka-simple.html)| |druid-kafka-eight-simpleConsumer|Kafka ingest firehose (low level consumer).|[link](../development/extensions-contrib/kafka-simple.html)|
|druid-rabbitmq|RabbitMQ firehose.|[link](../development/community-extensions/rabbitmq.html)| |druid-rabbitmq|RabbitMQ firehose.|[link](../development/extensions-contrib/rabbitmq.html)|
|druid-rocketmq|RocketMQ firehose.|[link](../development/community-extensions/rocketmq.html)| |druid-rocketmq|RocketMQ firehose.|[link](../development/extensions-contrib/rocketmq.html)|
|graphite-emitter|Graphite metrics emitter|[link](../development/community-extensions/graphite.html)| |graphite-emitter|Graphite metrics emitter|[link](../development/extensions-contrib/graphite.html)|
## Promoting Community Extension to Core Extension ## Promoting Community Extension to Core Extension

View File

@ -7,6 +7,8 @@ Data Formats for Ingestion
Druid can ingest denormalized data in JSON, CSV, or a delimited form such as TSV, or any custom format. While most examples in the documentation use data in JSON format, it is not difficult to configure Druid to ingest any other delimited data. Druid can ingest denormalized data in JSON, CSV, or a delimited form such as TSV, or any custom format. While most examples in the documentation use data in JSON format, it is not difficult to configure Druid to ingest any other delimited data.
We welcome any contributions to new formats. We welcome any contributions to new formats.
For additional data formats, please see our [extensions list](../development/extensions.html).
## Formatting the Data ## Formatting the Data
The following are some samples of the data used in the [Wikipedia example](../tutorials/quickstart.html). The following are some samples of the data used in the [Wikipedia example](../tutorials/quickstart.html).

View File

@ -4,67 +4,17 @@ layout: doc_page
# Druid Firehoses # Druid Firehoses
Firehoses describe the data stream source. They are pluggable and thus the configuration schema can and will vary based on the `type` of the firehose. Firehoses are used in the [stream-pull](../ingestion/stream-pull.html) ingestion model. They are pluggable and thus the configuration schema can and will vary based on the `type` of the firehose.
| Field | Type | Description | Required | | Field | Type | Description | Required |
|-------|------|-------------|----------| |-------|------|-------------|----------|
| type | String | Specifies the type of firehose. Each value will have its own configuration schema, firehoses packaged with Druid are described below. | yes | | type | String | Specifies the type of firehose. Each value will have its own configuration schema, firehoses packaged with Druid are described below. | yes |
We describe the configuration of the [Kafka firehose example](../ingestion/stream-pull.html#realtime-specfile), but there are other types available in Druid (see below). ## Additional Firehoses
- `consumerProps` is a map of properties for the Kafka consumer. The JSON object is converted into a Properties object and passed along to the Kafka consumer.
- `feed` is the feed that the Kafka consumer should read from.
## Available Firehoses
There are several firehoses readily available in Druid, some are meant for examples, others can be used directly in a production environment. There are several firehoses readily available in Druid, some are meant for examples, others can be used directly in a production environment.
#### KafkaEightFirehose For additional firehoses, please see our [extensions list](../development/extensions.html).
Please note that the [druid-kafka-eight module](../operations/including-extensions.html) is required for this firehose. This firehose acts as a Kafka 0.8.x consumer and ingests data from Kafka.
Sample spec:
```json
"firehose": {
"type": "kafka-0.8",
"consumerProps": {
"zookeeper.connect": "localhost:2181",
"zookeeper.connection.timeout.ms" : "15000",
"zookeeper.session.timeout.ms" : "15000",
"zookeeper.sync.time.ms" : "5000",
"group.id": "druid-example",
"fetch.message.max.bytes" : "1048586",
"auto.offset.reset": "largest",
"auto.commit.enable": "false"
},
"feed": "wikipedia"
}
```
|property|description|required?|
|--------|-----------|---------|
|type|This should be "kafka-0.8"|yes|
|consumerProps|The full list of consumer configs can be [here](https://kafka.apache.org/08/configuration.html).|yes|
|feed|Kafka maintains feeds of messages in categories called topics. This is the topic name.|yes|
#### StaticS3Firehose
This firehose ingests events from a predefined list of S3 objects.
Sample spec:
```json
"firehose" : {
"type" : "static-s3",
"uris": ["s3://foo/bar/file.gz", "s3://bar/foo/file2.gz"]
}
```
|property|description|default|required?|
|--------|-----------|-------|---------|
|type|This should be "static-s3"|N/A|yes|
|uris|JSON array of URIs where s3 files to be ingested are located.|N/A|yes|
#### LocalFirehose #### LocalFirehose
@ -169,23 +119,3 @@ An example is shown below:
|type|This should be "timed"|yes| |type|This should be "timed"|yes|
|shutoffTime|time at which the firehose should shut down, in ISO8601 format|yes| |shutoffTime|time at which the firehose should shut down, in ISO8601 format|yes|
|delegate|firehose to use|yes| |delegate|firehose to use|yes|
#### TwitterSpritzerFirehose
This firehose connects directly to the twitter spritzer data stream.
Sample spec:
```json
"firehose" : {
"type" : "twitzer",
"maxEventCount": -1,
"maxRunMinutes": 0
}
```
|property|description|default|required?|
|--------|-----------|-------|---------|
|type|This should be "twitzer"|N/A|yes|
|maxEventCount|max events to receive, -1 is infinite, 0 means nothing is delivered; use this to prevent infinite space consumption or to prevent getting throttled at an inconvenient time.|N/A|yes|
|maxRunMinutes|maximum number of minutes to fetch Twitter events. Use this to prevent getting throttled at an inconvenient time. If zero or less, no time limit for run.|N/A|yes|

View File

@ -75,7 +75,7 @@ An example dataSchema is shown below:
## Parser ## Parser
If `type` is not included, the parser defaults to `string`. If `type` is not included, the parser defaults to `string`. For additional data formats, please see our [extensions list](../development/extensions.html).
### String Parser ### String Parser
@ -91,113 +91,6 @@ If `type` is not included, the parser defaults to `string`.
| type | String | This should say `protobuf`. | no | | type | String | This should say `protobuf`. | no |
| parseSpec | JSON Object | Specifies the timestamp and dimensions of the data. Should be a timeAndDims parseSpec. | yes | | parseSpec | JSON Object | Specifies the timestamp and dimensions of the data. Should be a timeAndDims parseSpec. | yes |
### Avro Stream Parser
This is for realtime ingestion. Make sure to include `druid-avro-extensions` as an extension.
| Field | Type | Description | Required |
|-------|------|-------------|----------|
| type | String | This should say `avro_stream`. | no |
| avroBytesDecoder | JSON Object | Specifies how to decode bytes to Avro record. | yes |
| parseSpec | JSON Object | Specifies the timestamp and dimensions of the data. Should be a timeAndDims parseSpec. | yes |
For example, using Avro stream parser with schema repo Avro bytes decoder:
```json
"parser" : {
"type" : "avro_stream",
"avroBytesDecoder" : {
"type" : "schema_repo",
"subjectAndIdConverter" : {
"type" : "avro_1124",
"topic" : "${YOUR_TOPIC}"
},
"schemaRepository" : {
"type" : "avro_1124_rest_client",
"url" : "${YOUR_SCHEMA_REPO_END_POINT}",
}
},
"parseSpec" : {
"type": "timeAndDims",
"timestampSpec": <standard timestampSpec>,
"dimensionsSpec": <standard dimensionsSpec>
}
}
```
#### Avro Bytes Decoder
If `type` is not included, the avroBytesDecoder defaults to `schema_repo`.
##### SchemaRepo Based Avro Bytes Decoder
This Avro bytes decoder first extract `subject` and `id` from input message bytes, then use them to lookup the Avro schema with which to decode Avro record from bytes. Details can be found in [schema repo](https://github.com/schema-repo/schema-repo) and [AVRO-1124](https://issues.apache.org/jira/browse/AVRO-1124). You will need an http service like schema repo to hold the avro schema. Towards schema registration on the message producer side, you can refer to `io.druid.data.input.AvroStreamInputRowParserTest#testParse()`.
| Field | Type | Description | Required |
|-------|------|-------------|----------|
| type | String | This should say `schema_repo`. | no |
| subjectAndIdConverter | JSON Object | Specifies the how to extract subject and id from message bytes. | yes |
| schemaRepository | JSON Object | Specifies the how to lookup Avro schema from subject and id. | yes |
##### Avro-1124 Subject And Id Converter
| Field | Type | Description | Required |
|-------|------|-------------|----------|
| type | String | This should say `avro_1124`. | no |
| topic | String | Specifies the topic of your kafka stream. | yes |
##### Avro-1124 Schema Repository
| Field | Type | Description | Required |
|-------|------|-------------|----------|
| type | String | This should say `avro_1124_rest_client`. | no |
| url | String | Specifies the endpoint url of your Avro-1124 schema repository. | yes |
### Avro Hadoop Parser
This is for batch ingestion using the HadoopDruidIndexer. The `inputFormat` of `inputSpec` in `ioConfig` must be set to `"io.druid.data.input.avro.AvroValueInputFormat"`. You may want to set Avro reader's schema in `jobProperties` in `tuningConfig`, eg: `"avro.schema.path.input.value": "/path/to/your/schema.avsc"` or `"avro.schema.input.value": "your_schema_JSON_object"`, if reader's schema is not set, the schema in Avro object container file will be used, see [Avro specification](http://avro.apache.org/docs/1.7.7/spec.html#Schema+Resolution). Make sure to include "io.druid.extensions:druid-avro-extensions" as an extension.
| Field | Type | Description | Required |
|-------|------|-------------|----------|
| type | String | This should say `avro_hadoop`. | no |
| parseSpec | JSON Object | Specifies the timestamp and dimensions of the data. Should be a timeAndDims parseSpec. | yes |
| fromPigAvroStorage | Boolean | Specifies whether the data file is stored using AvroStorage. | no(default == false) |
For example, using Avro Hadoop parser with custom reader's schema file:
```json
{
"type" : "index_hadoop",
"spec" : {
"dataSchema" : {
"dataSource" : "",
"parser" : {
"type" : "avro_hadoop",
"parseSpec" : {
"type": "timeAndDims",
"timestampSpec": <standard timestampSpec>,
"dimensionsSpec": <standard dimensionsSpec>
}
}
},
"ioConfig" : {
"type" : "hadoop",
"inputSpec" : {
"type" : "static",
"inputFormat": "io.druid.data.input.avro.AvroValueInputFormat",
"paths" : ""
}
},
"tuningConfig" : {
"jobProperties" : {
"avro.schema.path.input.value" : "/path/to/my/schema.avsc",
}
}
}
}
```
### ParseSpec ### ParseSpec
ParseSpecs serve two purposes: ParseSpecs serve two purposes:

View File

@ -1,6 +1,7 @@
--- ---
layout: doc_page layout: doc_page
--- ---
# Lookups # Lookups
<div class="note caution"> <div class="note caution">
@ -27,537 +28,6 @@ and such data belongs in the raw denormalized data for use in Druid.
Very small lookups (count of keys on the order of a few dozen to a few hundred) can be passed at query time as a "map" Very small lookups (count of keys on the order of a few dozen to a few hundred) can be passed at query time as a "map"
lookup as per [dimension specs](../querying/dimensionspecs.html). lookup as per [dimension specs](../querying/dimensionspecs.html).
## Configuration For lookups defined cluster-wide rather than embedded in the query, please look at the [namespaced lookup extension](../development/extensions-core/namespaced-lookup.html).
|Property|Description|Default| For other additional lookups, please see our [extensions list](../development/extensions.html).
|--------|-----------|-------|
|`druid.lookup.snapshotWorkingDir`| Working path used to store snapshot of current lookup configuration, leaving this property null will disable snapshot/bootstrap utility|null|
Namespaced lookups are appropriate for lookups which are not possible to pass at query time due to their size,
or are not desired to be passed at query time because the data is to reside in and be handled by the Druid servers.
Namespaced lookups can be specified as part of the runtime properties file. The property is a list of the namespaces
described as per the sections on this page. For example:
```json
druid.query.extraction.namespace.lookups=
[
{
"type": "uri",
"namespace": "some_uri_lookup",
"uri": "file:/tmp/prefix/",
"namespaceParseSpec": {
"format": "csv",
"columns": [
"key",
"value"
]
},
"pollPeriod": "PT5M"
},
{
"type": "jdbc",
"namespace": "some_jdbc_lookup",
"connectorConfig": {
"createTables": true,
"connectURI": "jdbc:mysql:\/\/localhost:3306\/druid",
"user": "druid",
"password": "diurd"
},
"table": "lookupTable",
"keyColumn": "mykeyColumn",
"valueColumn": "MyValueColumn",
"tsColumn": "timeColumn"
}
]
```
Proper functionality of Namespaced lookups requires the following extension to be loaded on the broker, peon, and historical nodes:
`druid-namespace-lookup`
## Cache Settings
Lookups are cached locally on historical nodes. The following are settings used by the nodes which service queries when
setting namespaces (broker, peon, historical)
|Property|Description|Default|
|--------|-----------|-------|
|`druid.query.extraction.namespace.cache.type`|Specifies the type of caching to be used by the namespaces. May be one of [`offHeap`, `onHeap`]. `offHeap` uses a temporary file for off-heap storage of the namespace (memory mapped files). `onHeap` stores all cache on the heap in standard java map types.|`onHeap`|
The cache is populated in different ways depending on the settings below. In general, most namespaces employ
a `pollPeriod` at the end of which time they poll the remote resource of interest for updates. A notable exception
is the Kafka namespace lookup, defined below.
## URI namespace update
The remapping values for each namespaced lookup can be specified by json as per
```json
{
"type":"uri",
"namespace":"some_lookup",
"uri": "s3://bucket/some/key/prefix/",
"namespaceParseSpec":{
"format":"csv",
"columns":["key","value"]
},
"pollPeriod":"PT5M",
"versionRegex": "renames-[0-9]*\\.gz"
}
```
|Property|Description|Required|Default|
|--------|-----------|--------|-------|
|`namespace`|The namespace to define|Yes||
|`pollPeriod`|Period between polling for updates|No|0 (only once)|
|`versionRegex`|Regex to help find newer versions of the namespace data|Yes||
|`namespaceParseSpec`|How to interpret the data at the URI|Yes||
The `pollPeriod` value specifies the period in ISO 8601 format between checks for updates. If the source of the lookup is capable of providing a timestamp, the lookup will only be updated if it has changed since the prior tick of `pollPeriod`. A value of 0, an absent parameter, or `null` all mean populate once and do not attempt to update. Whenever an update occurs, the updating system will look for a file with the most recent timestamp and assume that one with the most recent data.
The `versionRegex` value specifies a regex to use to determine if a filename in the parent path of the uri should be considered when trying to find the latest version. Omitting this setting or setting it equal to `null` will match to all files it can find (equivalent to using `".*"`). The search occurs in the most significant "directory" of the uri.
The `namespaceParseSpec` can be one of a number of values. Each of the examples below would rename foo to bar, baz to bat, and buck to truck. All parseSpec types assumes each input is delimited by a new line. See below for the types of parseSpec supported.
### csv lookupParseSpec
|Parameter|Description|Required|Default|
|---------|-----------|--------|-------|
|`columns`|The list of columns in the csv file|yes|`null`|
|`keyColumn`|The name of the column containing the key|no|The first column|
|`valueColumn`|The name of the column containing the value|no|The second column|
*example input*
```
bar,something,foo
bat,something2,baz
truck,something3,buck
```
*example namespaceParseSpec*
```json
"namespaceParseSpec": {
"format": "csv",
"columns": ["value","somethingElse","key"],
"keyColumn": "key",
"valueColumn": "value"
}
```
### tsv lookupParseSpec
|Parameter|Description|Required|Default|
|---------|-----------|--------|-------|
|`columns`|The list of columns in the csv file|yes|`null`|
|`keyColumn`|The name of the column containing the key|no|The first column|
|`valueColumn`|The name of the column containing the value|no|The second column|
|`delimiter`|The delimiter in the file|no|tab (`\t`)|
*example input*
```
bar|something,1|foo
bat|something,2|baz
truck|something,3|buck
```
*example namespaceParseSpec*
```json
"namespaceParseSpec": {
"format": "tsv",
"columns": ["value","somethingElse","key"],
"keyColumn": "key",
"valueColumn": "value",
"delimiter": "|"
}
```
### customJson lookupParseSpec
|Parameter|Description|Required|Default|
|---------|-----------|--------|-------|
|`keyFieldName`|The field name of the key|yes|null|
|`valueFieldName`|The field name of the value|yes|null|
*example input*
```json
{"key": "foo", "value": "bar", "somethingElse" : "something"}
{"key": "baz", "value": "bat", "somethingElse" : "something"}
{"key": "buck", "somethingElse": "something", "value": "truck"}
```
*example namespaceParseSpec*
```json
"namespaceParseSpec": {
"format": "customJson",
"keyFieldName": "key",
"valueFieldName": "value"
}
```
### simpleJson lookupParseSpec
The `simpleJson` lookupParseSpec does not take any parameters. It is simply a line delimited json file where the field is the key, and the field's value is the value.
*example input*
```json
{"foo": "bar"}
{"baz": "bat"}
{"buck": "truck"}
```
*example namespaceParseSpec*
```json
"namespaceParseSpec":{
"format": "simpleJson"
}
```
## JDBC namespaced lookup
The JDBC lookups will poll a database to populate its local cache. If the `tsColumn` is set it must be able to accept comparisons in the format `'2015-01-01 00:00:00'`. For example, the following must be valid sql for the table `SELECT * FROM some_lookup_table WHERE timestamp_column > '2015-01-01 00:00:00'`. If `tsColumn` is set, the caching service will attempt to only poll values that were written *after* the last sync. If `tsColumn` is not set, the entire table is pulled every time.
|Parameter|Description|Required|Default|
|---------|-----------|--------|-------|
|`namespace`|The namespace to define|Yes||
|`connectorConfig`|The connector config to use|Yes||
|`table`|The table which contains the key value pairs|Yes||
|`keyColumn`|The column in `table` which contains the keys|Yes||
|`valueColumn`|The column in `table` which contains the values|Yes||
|`tsColumn`| The column in `table` which contains when the key was updated|No|Not used|
|`pollPeriod`|How often to poll the DB|No|0 (only once)|
```json
{
"type":"jdbc",
"namespace":"some_lookup",
"connectorConfig":{
"createTables":true,
"connectURI":"jdbc:mysql://localhost:3306/druid",
"user":"druid",
"password":"diurd"
},
"table":"some_lookup_table",
"keyColumn":"the_old_dim_value",
"valueColumn":"the_new_dim_value",
"tsColumn":"timestamp_column",
"pollPeriod":600000
}
```
# Kafka namespaced lookup
If you need updates to populate as promptly as possible, it is possible to plug into a kafka topic whose key is the old value and message is the desired new value (both in UTF-8). This requires the following extension: "io.druid.extensions:kafka-extraction-namespace"
```json
{
"type":"kafka",
"namespace":"testTopic",
"kafkaTopic":"testTopic"
}
```
|Parameter|Description|Required|Default|
|---------|-----------|--------|-------|
|`namespace`|The namespace to define|Yes||
|`kafkaTopic`|The kafka topic to read the data from|Yes||
## Kafka renames
The extension `kafka-extraction-namespace` enables reading from a kafka feed which has name/key pairs to allow renaming of dimension values. An example use case would be to rename an ID to a human readable format.
Currently the historical node caches the key/value pairs from the kafka feed in an ephemeral memory mapped DB via MapDB.
## Configuration
The following options are used to define the behavior and should be included wherever the extension is included (all query servicing nodes):
|Property|Description|Default|
|--------|-----------|-------|
|`druid.query.rename.kafka.properties`|A json map of kafka consumer properties. See below for special properties.|See below|
The following are the handling for kafka consumer properties in `druid.query.rename.kafka.properties`
|Property|Description|Default|
|--------|-----------|-------|
|`zookeeper.connect`|Zookeeper connection string|`localhost:2181/kafka`|
|`group.id`|Group ID, auto-assigned for publish-subscribe model and cannot be overridden|`UUID.randomUUID().toString()`|
|`auto.offset.reset`|Setting to get the entire kafka rename stream. Cannot be overridden|`smallest`|
## Testing the Kafka rename functionality
To test this setup, you can send key/value pairs to a kafka stream via the following producer console:
```
./bin/kafka-console-producer.sh --property parse.key=true --property key.separator="->" --broker-list localhost:9092 --topic testTopic
```
Renames can then be published as `OLD_VAL->NEW_VAL` followed by newline (enter or return)
Dynamic configuration (EXPERIMENTAL)
------------------------------------
The following documents the behavior of the cluster-wide config which is accessible through the coordinator.
The configuration is propagated through the concept of "tier" of servers.
A "tier" is defined as a group of services which should receive a set of lookups.
For example, you might have all historicals be part of `__default`, and Peons be part of individual tiers for the datasources they are tasked with.
The tiers for lookups are completely independent of historical tiers.
These configs are accessed using JSON through the following URI template
```
http://<COORDINATOR_IP>:<PORT>/druid/coordinator/v1/lookups/{tier}/{id}
```
All URIs below are assumed to have `http://<COORDINATOR_IP>:<PORT>` prepended.
If you have NEVER configured lookups before, you MUST post an empty json object `{}` to `/druid/coordinator/v1/lookups` to initialize the configuration.
These endpoints will return one of the following results:
* 404 if the resource is not found
* 400 if there is a problem in the formatting of the request
* 202 if the request was accepted asynchronously (`POST` and `DELETE`)
* 200 if the request succeeded (`GET` only)
## Configuration propagation behavior
The configuration is propagated to the query serving nodes (broker / router / peon / historical) by the coordinator.
The query serving nodes have an internal API for managing `POST`/`GET`/`DELETE` of lookups.
The coordinator periodically checks the dynamic configuration for changes and, when it detects a change it does the following:
1. Post all lookups for a tier to all Druid nodes within that tier.
2. Delete lookups from a tier which were dropped between the prior configuration values and this one.
If there is no configuration change, the coordinator checks for any nodes which might be new since the last time it propagated lookups and adds all lookups for that node (assuming that node's tier has lookups).
If there are errors while trying to add or update configuration on a node, that node is temporarily skipped until the next management period. The next management period the update will attempt to be propagated again.
If there is an error while trying to delete a lookup from a node (or if a node is down when the coordinator is propagating the config), the delete is not attempted again. In such a case it is possible that a node has lookups that are no longer managed by the coordinator.
## Bulk update
Lookups can be updated in bulk by posting a JSON object to `/druid/coordinator/v1/lookups`. The format of the json object is as follows:
```json
{
"tierName": {
"lookupExtractorFactoryName": {
"someExtractorField": "someExtractorValue"
}
}
}
```
So a config might look something like:
```json
{
"__default": {
"country_code": {
"type": "simple_json",
"uri": "http://some.host.com/codes.json"
},
"site_id": {
"type": "confidential_jdbc",
"auth": "/etc/jdbc.internal",
"table": "sites",
"key": "site_id",
"value": "site_name"
},
"site_id_customer1": {
"type": "confidential_jdbc",
"auth": "/etc/jdbc.customer1",
"table": "sites",
"key": "site_id",
"value": "site_name"
},
"site_id_customer2": {
"type": "confidential_jdbc",
"auth": "/etc/jdbc.customer2",
"table": "sites",
"key": "site_id",
"value": "site_name"
}
},
"realtime_customer1": {
"country_code": {
"type": "simple_json",
"uri": "http://some.host.com/codes.json"
},
"site_id_customer1": {
"type": "confidential_jdbc",
"auth": "/etc/jdbc.customer1",
"table": "sites",
"key": "site_id",
"value": "site_name"
}
},
"realtime_customer2": {
"country_code": {
"type": "simple_json",
"uri": "http://some.host.com/codes.json"
},
"site_id_customer2": {
"type": "confidential_jdbc",
"auth": "/etc/jdbc.customer2",
"table": "sites",
"key": "site_id",
"value": "site_name"
}
}
}
```
All entries in the map will UPDATE existing entries. No entries will be deleted.
## Update Lookup
A `POST` to a particular lookup extractor factory via `/druid/coordinator/v1/lookups/{tier}/{id}` will update that specific extractor factory.
For example, a post to `/druid/coordinator/v1/lookups/realtime_customer1/site_id_customer1` might contain the following:
```json
{
"type": "confidential_jdbc",
"auth": "/etc/jdbc.customer1",
"table": "sites_updated",
"key": "site_id",
"value": "site_name"
}
```
This will replace the `site_id_customer1` lookup in the `realtime_customer1` with the definition above.
## Get Lookup
A `GET` to a particular lookup extractor factory is accomplished via `/druid/coordinator/v1/lookups/{tier}/{id}`
Using the prior example, a `GET` to `/druid/coordinator/v1/lookups/realtime_customer2/site_id_customer2` should return
```json
{
"type": "confidential_jdbc",
"auth": "/etc/jdbc.customer2",
"table": "sites",
"key": "site_id",
"value": "site_name"
}
```
## Delete Lookup
A `DELETE` to `/druid/coordinator/v1/lookups/{tier}/{id}` will remove that lookup from the cluster.
## List tier names
A `GET` to `/druid/coordinator/v1/lookups` will return a list of known tier names in the dynamic configuration.
To discover a list of tiers currently active in the cluster **instead of** ones known in the dynamic configuration, the parameter `discover=true` can be added as per `/druid/coordinator/v1/lookups?discover=true`.
## List lookup names
A `GET` to `/druid/coordinator/v1/lookups/{tier}` will return a list of known lookup names for that tier.
# Internal API
The Peon, Router, Broker, and Historical nodes all have the ability to consume lookup configuration.
There is an internal API these nodes use to list/load/drop their lookups starting at `/druid/listen/v1/lookups`.
These follow the same convention for return values as the cluster wide dynamic configuration.
Usage of these endpoints is quite advanced and not recommended for most users.
The endpoints are as follows:
## Get Lookups
A `GET` to the node at `/druid/listen/v1/lookups` will return a json map of all the lookups currently active on the node.
The return value will be a json map of the lookups to their extractor factories.
```json
{
"some_lookup_name": {
"type": "simple_json",
"uri": "http://some.host.com/codes.json"
}
}
```
## Get Lookup
A `GET` to the node at `/druid/listen/v1/lookups/some_lookup_name` will return the LookupExtractorFactory for the lookup identified by `some_lookup_name`.
The return value will be the json representation of the factory.
```json
{
"type": "simple_json",
"uri": "http://some.host.com/codes.json"
}
```
## Bulk Add or Update Lookups
A `POST` to the node at `/druid/listen/v1/lookups` of a JSON map of lookup names to LookupExtractorFactory will cause the service to add or update its lookups.
The return value will be a JSON map in the following format:
```json
{
"status": "accepted",
"failedUpdates": {}
}
```
If a lookup cannot be started, or is left in an undefined state, the lookup in error will be returned in the `failedUpdates` field as per:
```json
{
"status": "accepted",
"failedUpdates": {
"country_code": {
"type": "simple_json",
"uri": "http://some.host.com/codes.json"
}
}
}
```
The `failedUpdates` field of the return value should be checked if a user is wanting to assure that every update succeeded.
## Add or Update Lookup
A `POST` to the node at `/druid/listen/v1/lookups/some_lookup_name` will behave very similarly to a bulk update.
If `some_lookup_name` is desired to have the LookupExtractorFactory definition of
```json
{
"type": "simple_json",
"uri": "http://some.host.com/codes.json"
}
```
Then a post to `/druid/listen/v1/lookups/some_lookup_name` will behave the same as a `POST` to `/druid/listen/v1/lookups` of
```json
{
"some_lookup_name": {
"type": "simple_json",
"uri": "http://some.host.com/codes.json"
}
}
```
## Remove a Lookup
A `DELETE` to `/druid/listen/v1/lookups/some_lookup_name` will remove that lookup from the node. Success will reflect the ID.
# Configuration
See the [coordinator configuration guilde](../configuration/coordinator.html) for coordinator configuration
To configure a Broker / Router / Historical / Peon to announce itself as part of a lookup tier, use the `druid.zk.paths.lookupTier` property.
|Property | Description | Default |
|---------|-------------|---------|
|`druid.lookup.tierName`| The tier for **lookups** for this node. This is independent of other tiers.|`__default`|

View File

@ -93,9 +93,9 @@
* [Integration](../development/integrating-druid-with-other-technologies.html) * [Integration](../development/integrating-druid-with-other-technologies.html)
* Experimental Features * Experimental Features
* [Overview](../development/experimental.html) * [Overview](../development/experimental.html)
* [Approximate Histograms and Quantiles](../development/extensions-core/approximate-histograms.html)
* [Datasketches](../development/extensions-core/datasketches-aggregators.html)
* [Geographic Queries](../development/geo.html) * [Geographic Queries](../development/geo.html)
* [Approximate Histograms and Quantiles](../development/approximate-histograms.html)
* [Datasketches](../development/datasketches-aggregators.html)
* [Router](../development/router.html) * [Router](../development/router.html)
## Misc ## Misc

View File

@ -1,6 +1,6 @@
# Community Extensions # Community Extensions
Please contribute all community extensions in this directory and include a doc of how your extension can be used under /docs/content/development/community-extensions/. Please contribute all community extensions in this directory and include a doc of how your extension can be used under /docs/content/development/extensions-contrib/.
Please note that community extensions are maintained by their original contributors and are not packaged with the core Druid distribution. Please note that community extensions are maintained by their original contributors and are not packaged with the core Druid distribution.
If you'd like to take on maintenance for a community extension, please post on [druid-development group](https://groups.google.com/forum/#!forum/druid-development) to let us know! If you'd like to take on maintenance for a community extension, please post on [druid-development group](https://groups.google.com/forum/#!forum/druid-development) to let us know!