mirror of https://github.com/apache/druid.git
Merge pull request #2398 from implydata/doc-render
Numerous fixes to enables docs to render correctly
This commit is contained in:
commit
8ac77a9644
|
@ -5,14 +5,13 @@ layout: doc_page
|
|||
# Caching
|
||||
|
||||
Caching can optionally be enabled on the broker, historical, and realtime
|
||||
nodes, as well as realtime index tasks. See [broker](broker.html#caching),
|
||||
processing. See [broker](broker.html#caching),
|
||||
[historical](historical.html#caching), and [realtime](realtime.html#caching)
|
||||
configuration options for how to enable it for individual node types.
|
||||
configuration options for how to enable it for different processes.
|
||||
|
||||
Druid uses a local in-memory cache by default, unless a diffrent type of cache is specified.
|
||||
Use the `druid.cache.type` configuration to set a different kind of cache.
|
||||
|
||||
|
||||
## Cache configuration
|
||||
|
||||
Cache settings are set globally, so the same configuration can be re-used
|
||||
|
|
|
@ -6,16 +6,7 @@ Logging
|
|||
|
||||
Druid nodes will emit logs that are useful for debugging to the console. Druid nodes also emit periodic metrics about their state. For more about metrics, see [Configuration](../configuration/index.html). Metric logs are printed to the console by default, and can be disabled with `-Ddruid.emitter.logging.logLevel=debug`.
|
||||
|
||||
Druid uses [log4j2](http://logging.apache.org/log4j/2.x/) for logging. Logging can be configured with a log4j2.xml file. Add the path to the directory containing the log4j2.xml file (eg a config dir) to your classpath if you want to override default Druid log configuration. Note that this directory should be earlier in the classpath than the druid jars. The easiest way to do this is to prefix the classpath with the config dir. For example, if the log4j2.xml file is in config/_common:
|
||||
|
||||
```bash
|
||||
java -Xmx512m -Duser.timezone=UTC -Dfile.encoding=UTF-8 \
|
||||
-Ddruid.realtime.specFile=examples/indexing/wikipedia.spec \
|
||||
-classpath "config/_common:config/realtime:lib/*" \
|
||||
io.druid.cli.Main server realtime
|
||||
```
|
||||
|
||||
Note the "-classpath" in this example has the config dir before the jars under lib/*.
|
||||
Druid uses [log4j2](http://logging.apache.org/log4j/2.x/) for logging. Logging can be configured with a log4j2.xml file. Add the path to the directory containing the log4j2.xml file (e.g. the _common/ dir) to your classpath if you want to override default Druid log configuration. Note that this directory should be earlier in the classpath than the druid jars. The easiest way to do this is to prefix the classpath with the config dir.
|
||||
|
||||
To enable java logging to go through log4j2, set the `-Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager` server parameter.
|
||||
|
||||
|
|
|
@ -4,18 +4,18 @@ layout: doc_page
|
|||
Production Cluster Configuration
|
||||
================================
|
||||
|
||||
```note-info
|
||||
<div class="note info">
|
||||
This configuration is an example of what a production cluster could look like. Many other hardware combinations are
|
||||
possible! Cheaper hardware is absolutely possible.
|
||||
```
|
||||
</div>
|
||||
|
||||
This production Druid cluster assumes that metadata storage and Zookeeper are already set up. The deep storage that is
|
||||
used for examples is [S3](https://aws.amazon.com/s3/) and [memcached](http://memcached.org/) is used for a distributed cache.
|
||||
|
||||
```note-info
|
||||
<div class="note info">
|
||||
The nodes in this example do not need to be on their own individual servers. Overlord and Coordinator nodes should be
|
||||
co-located on the same hardware.
|
||||
```
|
||||
</div>
|
||||
|
||||
The nodes that respond to queries (Historical, Broker, and MiddleManager nodes) will use as many cores as are available,
|
||||
depending on usage, so it is best to keep these on dedicated machines. The upper limit of effectively utilized cores is
|
||||
|
@ -35,9 +35,9 @@ We'll use [EC2](https://aws.amazon.com/ec2/) r3.8xlarge nodes for query facing n
|
|||
The following examples work relatively well in production, however, a more optimized tuning for the nodes we selected and
|
||||
more optimal hardware for a Druid cluster are both definitely possible.
|
||||
|
||||
```note-caution
|
||||
<div class="note caution">
|
||||
For high availability, there should be at least a redundant copy of every process running on separate hardware.
|
||||
```
|
||||
</div>
|
||||
|
||||
### Common Configuration (common.runtime.properties)
|
||||
|
||||
|
|
|
@ -3,11 +3,6 @@
|
|||
layout: doc_page
|
||||
---
|
||||
|
||||
```note-caution
|
||||
If you are doing stream-pull based ingestion, we suggest using [stream-pushed](../ingestion/stream-push.html) based ingestion instead and not
|
||||
using real-time nodes.
|
||||
```
|
||||
|
||||
Realtime Node Configuration
|
||||
==============================
|
||||
For general Realtime Node information, see [here](../design/realtime.html).
|
||||
|
|
|
@ -154,14 +154,16 @@ an issue).
|
|||
|
||||
The `payload` column stores a JSON blob that has all of the metadata for the segment (some of the data stored in this payload is redundant with some of the columns in the table, that is intentional). This looks something like
|
||||
|
||||
```
|
||||
```json
|
||||
{
|
||||
"dataSource":"wikipedia",
|
||||
"interval":"2012-05-23T00:00:00.000Z/2012-05-24T00:00:00.000Z",
|
||||
"version":"2012-05-24T00:10:00.046Z",
|
||||
"loadSpec":{"type":"s3_zip",
|
||||
"bucket":"bucket_for_segment",
|
||||
"key":"path/to/segment/on/s3"},
|
||||
"loadSpec":{
|
||||
"type":"s3_zip",
|
||||
"bucket":"bucket_for_segment",
|
||||
"key":"path/to/segment/on/s3"
|
||||
},
|
||||
"dimensions":"comma-delimited-list-of-dimension-names",
|
||||
"metrics":"comma-delimited-list-of-metric-names",
|
||||
"shardSpec":{"type":"none"},
|
||||
|
@ -197,13 +199,12 @@ The Audit table is used to store the audit history for configuration changes
|
|||
e.g rule changes done by [Coordinator](../design/coordinator.html) and other
|
||||
config changes.
|
||||
|
||||
+
|
||||
+##Accessed By: ##
|
||||
+
|
||||
+The Metadata Storage is accessed only by:
|
||||
+
|
||||
+1. Realtime Nodes
|
||||
2. Indexing Service Nodes (if any)
|
||||
+3. Coordinator Nodes
|
||||
+
|
||||
+Thus you need to give permissions (eg in AWS Security Groups) only for these machines to access the Metadata storage.
|
||||
##Accessed By: ##
|
||||
|
||||
The Metadata Storage is accessed only by:
|
||||
|
||||
1. Indexing Service Nodes (if any)
|
||||
2. Realtime Nodes (if any)
|
||||
3. Coordinator Nodes
|
||||
|
||||
Thus you need to give permissions (eg in AWS Security Groups) only for these machines to access the Metadata storage.
|
||||
|
|
|
@ -1,45 +0,0 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
Concepts and Terminology
|
||||
========================
|
||||
|
||||
The following definitions are given with respect to the Druid data store. They are intended to help you better understand the Druid documentation, where these terms and concepts occur.
|
||||
|
||||
More definitions are available on the [design page](../design/design.html).
|
||||
|
||||
* **Aggregation** The summarizing of data meeting certain specifications. Druid aggregates [timeseries data](#timeseries), which in effect compacts the data. Time intervals (set in configuration) are used to create buckets, while [timestamps](#timestamp) determine which buckets data aggregated in.
|
||||
|
||||
* **Aggregators** A mechanism for combining records during realtime incremental indexing, Hadoop batch indexing, and in queries.
|
||||
|
||||
* **Compute node** Obsolete name for a [Historical node](../design/historical.html).
|
||||
|
||||
* **DataSource** A table-like view of data; specified in [specFiles](#specfile) and in queries. A dataSource specifies the source of data being ingested and ultimately stored in [segments](#segment).
|
||||
|
||||
* **Dimensions** Aspects or categories of data, such as languages or locations. For example, with *language* and *country* as the type of dimension, values could be "English" or "Mandarin" for language, or "USA" or "China" for country. In Druid, dimensions can serve as filters for narrowing down hits (for example, language = "English" or country = "China").
|
||||
|
||||
* **Ephemeral Node** A Zookeeper node (or "znode") that exists for as long as the session that created the znode is active. More info [here](http://zookeeper.apache.org/doc/r3.2.1/zookeeperProgrammers.html#Ephemeral+Nodes). In a Druid cluster, ephemeral nodes are typically used for commands (such as assigning [segments](#segment) to certain nodes).
|
||||
|
||||
* **Granularity** The time interval corresponding to aggregation by time. Druid configuration settings specify the granularity of [timestamp](#timestamp) buckets in a [segment](#segment) (for example, by minute or by hour), as well as the granularity of the segment itself. The latter is essentially the overall range of absolute time covered by the segment. In queries, granularity settings control the summarization of findings.
|
||||
|
||||
* **Ingestion** The pulling and initial storing and processing of data. Druid supports realtime and batch ingestion of data, and applies indexing in both cases.
|
||||
|
||||
* **Master node** Obsolete name for a [Coordinator node](../design/coordinator.html).
|
||||
|
||||
* **Metrics** Countable data that can be aggregated. Metrics, for example, can be the number of visitors to a website, number of tweets per day, or average revenue.
|
||||
|
||||
* **Rollup** The aggregation of data that occurs at one or more stages, based on settings in a [configuration file](#specfile).
|
||||
|
||||
<a name="segment"></a>
|
||||
* **Segment** A collection of (internal) records that are stored and processed together. Druid chunks data into segments representing a time interval, and these are stored and manipulated in the cluster.
|
||||
|
||||
* **Shard** A sub-partition of the data, allowing multiple [segments](#segment) to represent the data in a certain time interval. Sharding occurs along time partitions to better handle amounts of data that exceed certain limits on segment size, although sharding along dimensions may also occur to optimize efficiency.
|
||||
|
||||
<a name="specfile"></a>
|
||||
* **specFile** The specification for services in JSON format; see [Realtime](../design/realtime.html) and [Batch-ingestion](../ingestion/batch-ingestion.html)
|
||||
|
||||
<a name="timeseries"></a>
|
||||
* **Timeseries Data** Data points which are ordered in time. The closing value of a financial index or the number of tweets per hour with a certain hashtag are examples of timeseries data.
|
||||
|
||||
<a name="timestamp"></a>
|
||||
* **Timestamp** An absolute position on a timeline, given in a standard alpha-numerical format such as with UTC time. [Timeseries data](#timeseries) points can be ordered by timestamp, and in Druid, they are.
|
|
@ -31,7 +31,6 @@ Each of the systems, or components, described below also has a dedicated page wi
|
|||
The node types that currently exist are:
|
||||
|
||||
* [**Historical**](../design/historical.html) nodes are the workhorses that handle storage and querying on "historical" data (non-realtime). Historical nodes download segments from deep storage, respond to the queries from broker nodes about these segments, and return results to the broker nodes. They announce themselves and the segments they are serving in Zookeeper, and also use Zookeeper to monitor for signals to load or drop new segments.
|
||||
* [**Realtime**](../design/realtime.html) nodes ingest data in real time. They are in charge of listening to a stream of incoming data and making it available immediately inside the Druid system. Real-time nodes respond to query requests from Broker nodes, returning query results to those nodes. Aged data is pushed from Realtime nodes to deep storage. Realtime nodes monitor ZooKeeper to discover segments that they've pushed to deep storage have been loaded by Historicals—if so, they drop those segments.
|
||||
* [**Coordinator**](../design/coordinator.html) nodes monitor the grouping of historical nodes to ensure that data is available, replicated and in a generally "optimal" configuration. They do this by reading segment metadata information from metadata storage to determine what segments should be loaded in the cluster, using Zookeeper to determine what Historical nodes exist, and creating Zookeeper entries to tell Historical nodes to load and drop new segments.
|
||||
* [**Broker**](../design/broker.html) nodes receive queries from external clients and forward those queries to Realtime and Historical nodes. When Broker nodes receive results, they merge these results and return them to the caller. For knowing topology, Broker nodes use Zookeeper to determine what Realtime and Historical nodes exist.
|
||||
* [**Indexing Service**](../design/indexing-service.html) nodes form a cluster of workers to load batch and real-time data into the system as well as allow for alterations to the data stored in the system.
|
||||
|
|
|
@ -3,7 +3,8 @@ layout: doc_page
|
|||
---
|
||||
|
||||
## DataSketches aggregator
|
||||
Druid aggregators based on [datasketches]()http://datasketches.github.io/) library. Note that sketch algorithms are approxiate, see details in the "Accuracy" section of the datasketches doc.
|
||||
|
||||
Druid aggregators based on [datasketches](http://datasketches.github.io/) library. Note that sketch algorithms are approxiate, see details in the "Accuracy" section of the datasketches doc.
|
||||
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 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.
|
||||
|
||||
|
|
|
@ -9,7 +9,7 @@ This page discusses how we can integrate Druid with other technologies.
|
|||
|
||||
Event streams can be stored in a distributed message bus such as Kafka and further processed via a distributed stream
|
||||
processor system such as Storm, Samza, or Spark Streaming. Data processed by the stream processor can feed into Druid using
|
||||
the [Tranquility](https://github.com/druid-io/tranquility) library. Data can be
|
||||
the [Tranquility](https://github.com/druid-io/tranquility) library.
|
||||
|
||||
<img src="../../img/druid-production.png" width="800"/>
|
||||
|
||||
|
|
|
@ -39,26 +39,13 @@ Some great folks have written their own libraries to interact with Druid
|
|||
#### SQL
|
||||
|
||||
* [implydata/plyql](https://github.com/implydata/plyql) - A command line interface for issuing SQL queries to Druid via [plywood](https://github.com/implydata/plywood)
|
||||
* [srikalyc/Sql4D](https://github.com/srikalyc/Sql4D) - A SQL client for Druid. Used in production at Yahoo.
|
||||
|
||||
|
||||
Community Helper Libraries
|
||||
--------------------------
|
||||
|
||||
* [madvertise/druid-dumbo](https://github.com/madvertise/druid-dumbo) - Scripts to help generate batch configs for the ingestion of data into Druid
|
||||
* [housejester/druid-test-harness](https://github.com/housejester/druid-test-harness) - A set of scripts to simplify standing up some servers and seeing how things work
|
||||
* [mingfang/docker-druid](https://github.com/mingfang/docker-druid) - A Dockerfile to run the entire Druid cluster
|
||||
|
||||
Other Druid Distributions
|
||||
-------------------------
|
||||
|
||||
* [Imply Analytics Platform](http://imply.io/download) - The Imply Analytics platform repackages Druid, all its dependencies, and an UI and SQL layer.
|
||||
|
||||
Tools
|
||||
---
|
||||
|
||||
* [Insert Segments](../../operations/insert-segment-to-db.html) - A tool that can insert segments' metadata into Druid metadata storage.
|
||||
|
||||
UIs
|
||||
---
|
||||
|
||||
|
@ -68,13 +55,20 @@ UIs
|
|||
* [Metabase](https://github.com/metabase/metabase) - Simple dashboards, charts and query tool for your Druid DB
|
||||
|
||||
Tools
|
||||
---
|
||||
-----
|
||||
|
||||
* [Insert Segments](../../operations/insert-segment-to-db.html) - A tool that can insert segments' metadata into Druid metadata storage.
|
||||
|
||||
Other Community Extensions
|
||||
Community Helper Libraries
|
||||
--------------------------
|
||||
|
||||
* [madvertise/druid-dumbo](https://github.com/madvertise/druid-dumbo) - Scripts to help generate batch configs for the ingestion of data into Druid
|
||||
* [housejester/druid-test-harness](https://github.com/housejester/druid-test-harness) - A set of scripts to simplify standing up some servers and seeing how things work
|
||||
* [mingfang/docker-druid](https://github.com/mingfang/docker-druid) - A Dockerfile to run the entire Druid cluster
|
||||
|
||||
Community Extensions
|
||||
--------------------
|
||||
|
||||
These are extensions from the community. (If you would like yours listed please speak up!)
|
||||
|
||||
* [acesinc/druid-cors-filter-extension](https://github.com/acesinc/druid-cors-filter-extension) - An extension to enable CORS headers in http requests.
|
||||
|
|
|
@ -70,7 +70,8 @@ of a Druid [overlord](../design/indexing-service.html). A sample task is shown b
|
|||
"tuningConfig" : {
|
||||
"type": "hadoop"
|
||||
}
|
||||
}
|
||||
},
|
||||
"hadoopDependencyCoordinates": <my_hadoop_version>
|
||||
}
|
||||
```
|
||||
|
||||
|
@ -261,7 +262,7 @@ with Amazon-specific features such as S3 encryption and consistent views. If you
|
|||
features, you will need to make the Amazon EMR Hadoop JARs available to Druid through one of the
|
||||
mechanisms described in the [Using other Hadoop distributions](#using-other-hadoop-distributions) section.
|
||||
|
||||
### Using other Hadoop distributions
|
||||
## Using other Hadoop distributions
|
||||
|
||||
Druid works out of the box with many Hadoop distributions.
|
||||
|
||||
|
|
|
@ -10,6 +10,13 @@ To run:
|
|||
java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/*:<hadoop_config_dir> io.druid.cli.Main index hadoop <spec_file>
|
||||
```
|
||||
|
||||
## Options
|
||||
|
||||
- "--coordinate" - provide a version of Hadoop to use. This property will override the default Hadoop coordinates. Once specified, Druid will look for those Hadoop dependencies from the location specified by `druid.extensions.hadoopDependenciesDir`.
|
||||
- "--no-default-hadoop" - don't pull down the default hadoop version
|
||||
|
||||
## Spec file
|
||||
|
||||
The spec file needs to contain a JSON object where the contents are the same as the "spec" field in the Hadoop index task.
|
||||
In addition, the following fields need to be added to the ioConfig:
|
||||
|
||||
|
|
|
@ -66,6 +66,8 @@ All forms of Druid ingestion require some form of schema object. The format of t
|
|||
}
|
||||
```
|
||||
|
||||
If you have nested JSON, [Druid can automatically flatten it for you](flatten-json.html).
|
||||
|
||||
### CSV
|
||||
|
||||
Since the CSV data cannot contain the column names (no header is allowed), these must be added before that data can be processed:
|
||||
|
|
|
@ -93,7 +93,7 @@ If `type` is not included, the parser defaults to `string`.
|
|||
|
||||
### Avro Stream Parser
|
||||
|
||||
This is for realtime ingestion.
|
||||
This is for realtime ingestion. Make sure to include "io.druid.extensions:druid-avro-extensions" as an extension.
|
||||
|
||||
| Field | Type | Description | Required |
|
||||
|-------|------|-------------|----------|
|
||||
|
@ -102,6 +102,7 @@ This is for realtime ingestion.
|
|||
| parseSpec | JSON Object | Specifies the format of the data. | yes |
|
||||
|
||||
For example, using Avro stream parser with schema repo Avro bytes decoder:
|
||||
|
||||
```json
|
||||
"parser" : {
|
||||
"type" : "avro_stream",
|
||||
|
@ -116,11 +117,7 @@ For example, using Avro stream parser with schema repo Avro bytes decoder:
|
|||
"url" : "${YOUR_SCHEMA_REPO_END_POINT}",
|
||||
}
|
||||
},
|
||||
"parsSpec" : {
|
||||
"format" : "timeAndDims",
|
||||
"timestampSpec" : {},
|
||||
"dimensionsSpec" : {}
|
||||
}
|
||||
"parseSpec" : <standard_druid_parseSpec>
|
||||
}
|
||||
```
|
||||
|
||||
|
@ -155,7 +152,7 @@ This Avro bytes decoder first extract `subject` and `id` from input message byte
|
|||
|
||||
### 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).
|
||||
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 |
|
||||
|-------|------|-------------|----------|
|
||||
|
@ -164,20 +161,16 @@ This is for batch ingestion using the HadoopDruidIndexer. The `inputFormat` of `
|
|||
| 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",
|
||||
"hadoopDependencyCoordinates" : ["io.druid.extensions:druid-avro-extensions"],
|
||||
"spec" : {
|
||||
"dataSchema" : {
|
||||
"dataSource" : "",
|
||||
"parser" : {
|
||||
"type" : "avro_hadoop",
|
||||
"parsSpec" : {
|
||||
"format" : "timeAndDims",
|
||||
"timestampSpec" : {},
|
||||
"dimensionsSpec" : {}
|
||||
}
|
||||
"parseSpec" : <standard_druid_parseSpec>
|
||||
}
|
||||
},
|
||||
"ioConfig" : {
|
||||
|
|
|
@ -4,7 +4,7 @@ layout: doc_page
|
|||
|
||||
# Loading streams
|
||||
|
||||
Streams can be ingested in Druid using either [Tranquility](https://github.com/druid-io/tranquility (a Druid-aware
|
||||
Streams can be ingested in Druid using either [Tranquility](https://github.com/druid-io/tranquility) (a Druid-aware
|
||||
client) and the [indexing service](../design/indexing-service.html) or through standalone [Realtime nodes](../design/realtime.html).
|
||||
The first approach will be more complex to set up, but also offers scalability and high availability characteristics that advanced production
|
||||
setups may require. The second approach has some known [limitations](../ingestion/stream-pull.html#limitations).
|
||||
|
|
|
@ -25,7 +25,6 @@ For Real-time Node Configuration, see [Realtime Configuration](../configuration/
|
|||
|
||||
For writing your own plugins to the real-time node, see [Firehose](../ingestion/firehose.html).
|
||||
|
||||
<a id="realtime-specfile"></a>
|
||||
## Realtime "specFile"
|
||||
|
||||
The property `druid.realtime.specFile` has the path of a file (absolute or relative path and file name) with realtime specifications in it. This "specFile" should be a JSON Array of JSON objects like the following:
|
||||
|
@ -166,7 +165,7 @@ The following policies are available:
|
|||
* `none` – All events are accepted. Never hands off data unless shutdown() is called on the configured firehose.
|
||||
|
||||
|
||||
####<a id="sharding"></a> Sharding
|
||||
#### Sharding
|
||||
|
||||
Druid uses shards, or segments with partition numbers, to more efficiently handle large amounts of incoming data. In Druid, shards represent the segments that together cover a time interval based on the value of `segmentGranularity`. If, for example, `segmentGranularity` is set to "hour", then a number of shards may be used to store the data for that hour. Sharding along dimensions may also occur to optimize efficiency.
|
||||
|
||||
|
|
|
@ -8,10 +8,10 @@ Druid can connect to any streaming data source through
|
|||
[Tranquility](https://github.com/druid-io/tranquility/blob/master/README.md), a package for pushing
|
||||
streams to Druid in real-time. Druid does not come bundled with Tranquility, and you will have to download the distribution.
|
||||
|
||||
```note-info
|
||||
<div class="note info">
|
||||
If you've never loaded streaming data into Druid, we recommend trying out the
|
||||
[stream loading tutorial](../tutorials/tutorial-streams.html) first and then coming back to this page.
|
||||
```
|
||||
<a href="../tutorials/tutorial-streams.html">stream loading tutorial</a> first and then coming back to this page.
|
||||
</div>
|
||||
|
||||
Note that with all streaming ingestion options, you must ensure that incoming data is recent
|
||||
enough (within a [configurable windowPeriod](#segmentgranularity-and-windowperiod) of the current
|
||||
|
|
|
@ -25,21 +25,12 @@ To let Druid load your extensions, follow the steps below
|
|||
|
||||
Example:
|
||||
|
||||
Suppose you specify `druid.extensions.directory=/usr/local/druid/extensions`, and want Druid to load normal extensions ```druid-examples```, ```druid-kafka-eight``` and ```mysql-metadata-storage```.
|
||||
Suppose you specify `druid.extensions.directory=/usr/local/druid/extensions`, and want Druid to load normal extensions ```druid-kafka-eight``` and ```mysql-metadata-storage```.
|
||||
|
||||
Then under ```extensions```, it should look like this,
|
||||
|
||||
```
|
||||
extensions/
|
||||
├── druid-examples
|
||||
│ ├── commons-beanutils-1.8.3.jar
|
||||
│ ├── commons-digester-1.8.jar
|
||||
│ ├── commons-logging-1.1.1.jar
|
||||
│ ├── commons-validator-1.4.0.jar
|
||||
│ ├── druid-examples-0.8.0-rc1.jar
|
||||
│ ├── twitter4j-async-3.0.3.jar
|
||||
│ ├── twitter4j-core-3.0.3.jar
|
||||
│ └── twitter4j-stream-3.0.3.jar
|
||||
├── druid-kafka-eight
|
||||
│ ├── druid-kafka-eight-0.7.3.jar
|
||||
│ ├── jline-0.9.94.jar
|
||||
|
@ -61,7 +52,7 @@ extensions/
|
|||
└── mysql-metadata-storage-0.8.0-rc1.jar
|
||||
```
|
||||
|
||||
As you can see, under ```extensions``` there are three sub-directories ```druid-examples```, ```druid-kafka-eight``` and ```mysql-metadata-storage```, each sub-directory denotes an extension that Druid might load.
|
||||
As you can see, under ```extensions``` there are two sub-directories ```druid-kafka-eight``` and ```mysql-metadata-storage```, each sub-directory denotes an extension that Druid might load.
|
||||
|
||||
3) To have Druid load a specific list of extensions present under the root extension directory, set `druid.extensions.loadList` to the list of extensions to load. Using the example above, if you want Druid to load ```druid-kafka-eight``` and ```mysql-metadata-storage```, you can specify `druid.extensions.loadList=["druid-kafka-eight", "mysql-metadata-storage"]`.
|
||||
|
||||
|
|
|
@ -114,9 +114,14 @@ your functionality as a native Java aggregator.
|
|||
The javascript aggregator is recommended for rapidly prototyping features. This aggregator will be much slower in production
|
||||
use than a native Java aggregator.
|
||||
|
||||
## Approximate Aggregations
|
||||
|
||||
### Cardinality aggregator
|
||||
|
||||
Computes the cardinality of a set of Druid dimensions, using HyperLogLog to estimate the cardinality.
|
||||
Computes the cardinality of a set of Druid dimensions, using HyperLogLog to estimate the cardinality. Please note that this
|
||||
aggregator will be much slower than indexing a column with the hyperUnique aggregator. This aggregator also runs over a dimension column, which
|
||||
means the string dimension cannot be removed from the dataset to improve rollup. In general, we strongly recommend using the hyperUnique aggregator
|
||||
instead of the cardinality aggregator if you do not care about the individual values of a dimension.
|
||||
|
||||
```json
|
||||
{
|
||||
|
@ -181,10 +186,6 @@ Determine the number of distinct people (i.e. combinations of first and last nam
|
|||
}
|
||||
```
|
||||
|
||||
## Complex Aggregations
|
||||
|
||||
Druid supports complex aggregations such as various types of approximate sketches.
|
||||
|
||||
### HyperUnique aggregator
|
||||
|
||||
Uses [HyperLogLog](http://algo.inria.fr/flajolet/Publications/FlFuGaMe07.pdf) to compute the estimated cardinality of a dimension that has been aggregated as a "hyperUnique" metric at indexing time.
|
||||
|
@ -193,6 +194,8 @@ Uses [HyperLogLog](http://algo.inria.fr/flajolet/Publications/FlFuGaMe07.pdf) to
|
|||
{ "type" : "hyperUnique", "name" : <output_name>, "fieldName" : <metric_name> }
|
||||
```
|
||||
|
||||
For more approximate aggregators, please see [theta sketches](../development/datasketches-aggregators.html).
|
||||
|
||||
## Miscellaneous Aggregations
|
||||
|
||||
### Filtered Aggregator
|
||||
|
|
|
@ -361,11 +361,13 @@ Then groupBy/topN processing pipeline "explodes" all multi-valued dimensions res
|
|||
In addition to "query filter" which efficiently selects the rows to be processed, you can use the filtering dimension spec to filter for specific values within the values of a multi-valued dimension. These dimensionSpecs take a delegate DimensionSpec and a filtering criteria. From the "exploded" rows, only rows matching the given filtering criteria are returned in the query result.
|
||||
|
||||
The following filtered dimension spec acts as a whitelist or blacklist for values as per the "isWhitelist" attribute value.
|
||||
|
||||
```json
|
||||
{ "type" : "listFiltered", "delegate" : <dimensionSpec>, "values": <array of strings>, "isWhitelist": <optional attribute for true/false, default is true> }
|
||||
```
|
||||
|
||||
Following filtered dimension spec retains only the values matching regex. Note that `listFiltered` is faster than this and one should use that for whitelist or blacklist usecase.
|
||||
|
||||
```json
|
||||
{ "type" : "regexFiltered", "delegate" : <dimensionSpec>, "pattern": <java regex pattern> }
|
||||
```
|
||||
|
|
|
@ -5,7 +5,7 @@ layout: doc_page
|
|||
|
||||
Druid has limited support for joins through [query-time lookups](../querying/lookups.html). The common use case of
|
||||
query-time lookups is to replace one dimension value that (e.g. a String ID) with another value (e.g. a human-readable
|
||||
String value).
|
||||
String value). This is similar a star-schema join.
|
||||
|
||||
Druid does not yet have full support for joins. Although Druid’s storage format would allow for the implementation
|
||||
of joins (there is no loss of fidelity for columns included as dimensions), full support for joins have not yet been implemented yet
|
||||
|
|
|
@ -103,7 +103,7 @@ It can be used in a sample calculation as so:
|
|||
}]
|
||||
```
|
||||
|
||||
#### Example Usage
|
||||
## Example Usage
|
||||
|
||||
In this example, let’s calculate a simple percentage using post aggregators. Let’s imagine our data set has a metric called "total".
|
||||
|
||||
|
|
|
@ -199,13 +199,13 @@ In `conf/_common/common.runtime.properties`, replace
|
|||
- `druid.metadata.storage.connector.connectURI`
|
||||
- `druid.metadata.storage.connector.host`
|
||||
|
||||
```note-caution
|
||||
<div class="note caution">
|
||||
In production, we recommend running 2 servers, each running a Druid Coordinator
|
||||
and a Druid Overlord. We also recommend running a ZooKeeper cluster on its own dedicated hardware,
|
||||
as well as replicated [metadata
|
||||
storage](http://druid.io/docs/latest/dependencies/metadata-storage.html) such as MySQL or
|
||||
PostgreSQL, on its own dedicated hardware.
|
||||
```
|
||||
</div>
|
||||
|
||||
## Tune Druid processes that serve queries
|
||||
|
||||
|
@ -227,9 +227,9 @@ hardware. The most commonly adjusted configurations are:
|
|||
- `druid.server.maxSize` and `druid.segmentCache.locations` on Historical Nodes
|
||||
- `druid.worker.capacity` on MiddleManagers
|
||||
|
||||
```note
|
||||
<div class="note info">
|
||||
Keep -XX:MaxDirectMemory >= numThreads*sizeBytes, otherwise Druid will fail to start up..
|
||||
```
|
||||
</div>
|
||||
|
||||
Please see the Druid [configuration documentation](../configuration/index.html) for a full description of all
|
||||
possible configuration options.
|
||||
|
@ -251,9 +251,9 @@ hardware. The most commonly adjusted configurations are:
|
|||
- `druid.query.groupBy.maxIntermediateRows`
|
||||
- `druid.query.groupBy.maxResults`
|
||||
|
||||
```note-caution
|
||||
Keep -XX:MaxDirectMemory >= numThreads*sizeBytes, otherwise Druid will fail to start up..
|
||||
```
|
||||
<div class="note caution">
|
||||
Keep -XX:MaxDirectMemory >= numThreads*sizeBytes, otherwise Druid will fail to start up.
|
||||
</div>
|
||||
|
||||
Please see the Druid [configuration documentation](../configuration/index.html) for a full description of all
|
||||
possible configuration options.
|
||||
|
@ -276,11 +276,12 @@ tar -xzf zookeeper-3.4.6.tar.gz
|
|||
cd zookeeper-3.4.6
|
||||
cp conf/zoo_sample.cfg conf/zoo.cfg
|
||||
./bin/zkServer.sh start
|
||||
|
||||
```note-caution
|
||||
In production, we also recommend running a ZooKeeper cluster on its own dedicated hardware.
|
||||
```
|
||||
|
||||
<div class="note caution">
|
||||
In production, we also recommend running a ZooKeeper cluster on its own dedicated hardware.
|
||||
</div>
|
||||
|
||||
On your coordination server, *cd* into the distribution and start up the coordination services (you should do this in different windows or pipe the log to a file):
|
||||
|
||||
```bash
|
||||
|
@ -304,11 +305,11 @@ java `cat conf/druid/middleManager/jvm.config | xargs` -cp conf/druid/_common:co
|
|||
|
||||
You can add more servers with Druid Historicals and MiddleManagers as needed.
|
||||
|
||||
```note-info
|
||||
<div class="note info">
|
||||
For clusters with complex resource allocation needs, you can break apart Historicals and MiddleManagers and scale the components individually.
|
||||
This also allows you take advantage of Druid's built-in MiddleManager
|
||||
autoscaling facility.
|
||||
```
|
||||
</div>
|
||||
|
||||
If you are doing push-based stream ingestion with Kafka or over HTTP, you can also start Tranquility server on the same
|
||||
hardware that holds MiddleManagers and Historicals. For large scale production, MiddleManagers and Tranquility server
|
||||
|
@ -326,7 +327,7 @@ bin/tranquility <server or kafka> -configFile <path_to_druid_distro>/conf/tranqu
|
|||
|
||||
Copy the Druid distribution and your edited configurations to your servers set aside for the Druid Brokers.
|
||||
|
||||
On each one, *cd* into the distribution and run this command to start a Broker (you want to pipe the output to a log file):
|
||||
On each one, *cd* into the distribution and run this command to start a Broker (you may want to pipe the output to a log file):
|
||||
|
||||
```bash
|
||||
java `cat conf/druid/broker/jvm.config | xargs` -cp conf/druid/_common:conf/druid/broker:lib/* io.druid.cli.Main server broker
|
||||
|
|
|
@ -9,15 +9,15 @@ layout: doc_page
|
|||
Druid supports streaming (real-time) and file-based (batch) ingestion methods. The most
|
||||
popular configurations are:
|
||||
|
||||
- [Files](batch-ingestion.html) - Load data from HDFS, S3, local files, or any supported Hadoop
|
||||
- [Files](../ingestion/batch-ingestion.html) - Load data from HDFS, S3, local files, or any supported Hadoop
|
||||
filesystem in batches. We recommend this method if your dataset is already in flat files.
|
||||
|
||||
- [Stream push](stream-ingestion.html#stream-push) - Push a data stream into Druid in real-time
|
||||
- [Stream push](../ingestion/stream-ingestion.html#stream-push) - Push a data stream into Druid in real-time
|
||||
using [Tranquility](http://github.com/druid-io/tranquility), a client library for sending streams
|
||||
to Druid. We recommend this method if your dataset originates in a streaming system like Kafka,
|
||||
Storm, Spark Streaming, or your own system.
|
||||
|
||||
- [Stream pull](stream-ingestion.html#stream-pull) - Pull a data stream directly from an external
|
||||
- [Stream pull](../ingestion/stream-ingestion.html#stream-pull) - Pull a data stream directly from an external
|
||||
data source into Druid using Realtime Nodes.
|
||||
|
||||
## Getting started
|
||||
|
|
|
@ -65,8 +65,9 @@ With Zookeeper running, return to the druid-0.9.0 directory. In that directory,
|
|||
bin/init
|
||||
```
|
||||
|
||||
Next, you can start up the Druid processes in different terminal windows. This tutorial runs every Druid process on the same system. In production,
|
||||
many of these Druid processes can be colocated even in a distributed cluster.
|
||||
This will setup up some directories for you. Next, you can start up the Druid processes in different terminal windows.
|
||||
This tutorial runs every Druid process on the same system. In a large distributed production cluster,
|
||||
many of these Druid processes can still be co-located together.
|
||||
|
||||
```bash
|
||||
java `cat conf-quickstart/druid/historical/jvm.config | xargs` -cp conf-quickstart/druid/_common:conf-quickstart/druid/historical:lib/* io.druid.cli.Main server historical
|
||||
|
@ -87,11 +88,11 @@ Once every service has started, you are now ready to load data.
|
|||
|
||||
We've included a sample of Wikipedia edits from September 12, 2015 to get you started.
|
||||
|
||||
```note-info
|
||||
This section shows you how to load data in batches, but you can skip ahead to learn how to [load
|
||||
streams in real-time](quickstart.html#load-streaming-data). Druid's streaming ingestion can load data
|
||||
<div class="note info">
|
||||
This section shows you how to load data in batches, but you can skip ahead to learn how to <a href="quickstart.html#load-streaming-data">load
|
||||
streams in real-time</a>. Druid's streaming ingestion can load data
|
||||
with virtually no delay between events occurring and being available for queries.
|
||||
```
|
||||
</div>
|
||||
|
||||
The [dimensions](https://en.wikipedia.org/wiki/Dimension_%28data_warehouse%29) (attributes you can
|
||||
filter and split on) in the Wikipedia dataset, other than time, are:
|
||||
|
@ -132,7 +133,7 @@ curl -X 'POST' -H 'Content-Type:application/json' -d @quickstart/wikiticker-inde
|
|||
|
||||
Which will print the ID of the task if the submission was successful:
|
||||
|
||||
```base
|
||||
```bash
|
||||
{"task":"index_hadoop_wikipedia_2013-10-09T21:30:32.802Z"}
|
||||
```
|
||||
|
||||
|
@ -171,11 +172,11 @@ directly to Druid.
|
|||
bin/tranquility server -configFile <path_to_druid_distro>/conf-quickstart/tranquility/server.json
|
||||
```
|
||||
|
||||
```note-info
|
||||
<div class="note info">
|
||||
This section shows you how to load data using Tranquility Server, but Druid also supports a wide
|
||||
variety of [other streaming ingestion options](ingestion-streams.html#stream-push), including from
|
||||
variety of <a href="ingestion-streams.html#stream-push">other streaming ingestion options</a>, including from
|
||||
popular streaming systems like Kafka, Storm, Samza, and Spark Streaming.
|
||||
```
|
||||
</div>
|
||||
|
||||
The [dimensions](https://en.wikipedia.org/wiki/Dimension_%28data_warehouse%29) (attributes you can
|
||||
filter and split on) for this datasource are flexible. It's configured for *schemaless dimensions*,
|
||||
|
|
|
@ -2,12 +2,25 @@
|
|||
layout: doc_page
|
||||
---
|
||||
|
||||
## Load your own batch data
|
||||
# Tutorial: Load your own batch data
|
||||
|
||||
Before you get started with loading your own batch data, you should have first completed the [quickstart](quickstart.html).
|
||||
## Getting started
|
||||
|
||||
You can easily load any timestamped dataset into Druid. For Druid batch loads, the most important
|
||||
questions are:
|
||||
This tutorial shows you how to load your own data files into Druid.
|
||||
|
||||
For this tutorial, we'll assume you've already downloaded Druid as described in
|
||||
the [single-machine quickstart](quickstart.html) and have it running on your local machine. You
|
||||
don't need to have loaded any data yet.
|
||||
|
||||
Once that's complete, you can load your own dataset by writing a custom ingestion spec.
|
||||
|
||||
## Writing an ingestion spec
|
||||
|
||||
When loading files into Druid, you will use Druid's [batch loading](ingestion-batch.html) process.
|
||||
There's an example batch ingestion spec in `quickstart/wikiticker-index.json` that you can modify
|
||||
for your own needs.
|
||||
|
||||
The most important questions are:
|
||||
|
||||
* What should the dataset be called? This is the "dataSource" field of the "dataSchema".
|
||||
* Where is the dataset located? The file paths belong in the "paths" of the "inputSpec". If you
|
||||
|
@ -17,10 +30,8 @@ want to load multiple files, you can provide them as a comma-separated string.
|
|||
* Which fields should be treated as metrics? This belongs in the "metricsSpec".
|
||||
* What time ranges (intervals) are being loaded? This belongs in the "intervals" of the "granularitySpec".
|
||||
|
||||
```note-info
|
||||
If your data does not have a natural sense of time, you can tag each row with the current time.
|
||||
You can also tag all rows with a fixed timestamp, like "2000-01-01T00:00:00.000Z".
|
||||
```
|
||||
|
||||
Let's use this pageviews dataset as an example. Druid supports TSV, CSV, and JSON out of the box.
|
||||
Note that nested JSON objects are not supported, so if you do use JSON, you should provide a file
|
||||
|
@ -90,19 +101,39 @@ And modify it by altering these sections:
|
|||
}
|
||||
```
|
||||
|
||||
Finally, fire off the task and indexing will proceed!
|
||||
## Running the task
|
||||
|
||||
To actually run this task, first make sure that the indexing task can read *pageviews.json*:
|
||||
|
||||
- If you're running locally (no configuration for connecting to Hadoop; this is the default) then
|
||||
place it in the root of the Druid distribution.
|
||||
- If you configured Druid to connect to a Hadoop cluster, upload
|
||||
the pageviews.json file to HDFS. You may need to adjust the `paths` in the ingestion spec.
|
||||
|
||||
To kick off the indexing process, POST your indexing task to the Druid Overlord. In a standard Druid
|
||||
install, the URL is `http://OVERLORD_IP:8090/druid/indexer/v1/task`.
|
||||
|
||||
```bash
|
||||
curl -X 'POST' -H 'Content-Type:application/json' -d @quickstart/pageviews-index.json localhost:8090/druid/indexer/v1/task
|
||||
curl -X 'POST' -H 'Content-Type:application/json' -d @my-index-task.json OVERLORD_IP:8090/druid/indexer/v1/task
|
||||
```
|
||||
|
||||
If you're running everything on a single machine, you can use localhost:
|
||||
|
||||
```bash
|
||||
curl -X 'POST' -H 'Content-Type:application/json' -d @my-index-task.json localhost:8090/druid/indexer/v1/task
|
||||
```
|
||||
|
||||
If anything goes wrong with this task (e.g. it finishes with status FAILED), you can troubleshoot
|
||||
by visiting the "Task log" on the [overlord console](http://localhost:8090/console.html).
|
||||
|
||||
```note-info
|
||||
Druid supports a wide variety of data formats, ingestion options, and configurations not
|
||||
discussed here. For a full explanation of all available features, see the ingestion sections of the Druid
|
||||
documentation.
|
||||
```
|
||||
## Querying your data
|
||||
|
||||
Your data should become fully available within a minute or two. You can monitor this process on
|
||||
your Coordinator console at [http://localhost:8081/#/](http://localhost:8081/#/).
|
||||
|
||||
Once your data is fully available, you can query it using any of the
|
||||
[supported query methods](../querying/querying.html).
|
||||
|
||||
## Further reading
|
||||
|
||||
For more information on loading batch data, please see [the batch ingestion documentation](../ingestion/batch-ingestion.html).
|
||||
|
|
|
@ -12,12 +12,12 @@ For this tutorial, we'll assume you've already downloaded Druid and Tranquility
|
|||
the [single-machine quickstart](quickstart.html) and have it running on your local machine. You
|
||||
don't need to have loaded any data yet.
|
||||
|
||||
```note-info
|
||||
<div class="note info">
|
||||
This tutorial will show you how to load data from Kafka into Druid, but Druid additionally supports
|
||||
a wide variety of batch and streaming loading methods. See the *[Loading files](../ingestion/batch-ingestion.html)*
|
||||
and *[Loading streams](../ingestion/stream-ingestion.html)* pages for more information about other options,
|
||||
a wide variety of batch and streaming loading methods. See the <a href="../ingestion/batch-ingestion.html">Loading files</a>
|
||||
and <a href="../ingestion/stream-ingestion.html">Loading streams</a> pages for more information about other options,
|
||||
including from Hadoop, HTTP, Storm, Samza, Spark Streaming, and your own JVM apps.
|
||||
```
|
||||
</div>
|
||||
|
||||
## Start Kafka
|
||||
|
||||
|
|
|
@ -2,7 +2,7 @@
|
|||
layout: doc_page
|
||||
---
|
||||
|
||||
## Load your own streaming data
|
||||
# Tutorial: Load your own streaming data
|
||||
|
||||
## Getting started
|
||||
|
||||
|
@ -20,12 +20,12 @@ When loading streams into Druid, we recommend using the [stream push](../ingesti
|
|||
process. In this tutorial we'll be using [Tranquility Server](../ingestion/stream-ingestion.html#server) to push
|
||||
data into Druid over HTTP.
|
||||
|
||||
```note-info
|
||||
<div class="note info">
|
||||
This tutorial will show you how to push streams to Druid using HTTP, but Druid additionally supports
|
||||
a wide variety of batch and streaming loading methods. See the *[Loading files](batch-ingestion.html)*
|
||||
and *[Loading streams](stream-ingestion.html)* pages for more information about other options,
|
||||
a wide variety of batch and streaming loading methods. See the <a href="../ingestion/batch-ingestion.html">Loading files</a>
|
||||
and <a href="../ingestion/stream-ingestion.html">Loading streams</a> pages for more information about other options,
|
||||
including from Hadoop, Kafka, Storm, Samza, Spark Streaming, and your own JVM apps.
|
||||
```
|
||||
</div>
|
||||
|
||||
You can prepare for loading a new dataset over HTTP by writing a custom Tranquility Server
|
||||
configuration. The bundled configuration is in `conf-quickstart/tranquility/server.json`, which
|
||||
|
|
Loading…
Reference in New Issue