Merge pull request #1607 from druid-io/even-more-docs

Fixes and more docs across many areas
This commit is contained in:
Xavier Léauté 2015-08-13 11:53:53 -07:00
commit 24dc23967f
6 changed files with 329 additions and 208 deletions

View File

@ -6,7 +6,7 @@ Production Cluster Configuration
__This configuration is an example of what a production cluster could look like. Many other hardware combinations are possible! Cheaper hardware is absolutely possible.__
This production Druid cluster assumes that metadata storage and Zookeeper are already set up. The deep storage that is used for examples is S3 and memcached is used for a distributed cache.
This production Druid cluster assumes that metadata storage and Zookeeper are already set up. The deep storage that is used for examples is S3 and memcached is used as a distributed cache.
The nodes that respond to queries (Historical, Broker, and Middle manager 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 not well characterized yet and would depend on types of queries, query load, and the schema. Historical daemons should have a heap a size of at least 1GB per core for normal usage, but could be squeezed into a smaller heap for testing. Since in-memory caching is essential for good performance, even more RAM is better. Broker nodes will use RAM for caching, so they do more than just route queries. SSDs are highly recommended for Historical nodes not all data is loaded in available memory.
@ -328,3 +328,56 @@ druid.processing.numThreads=31
druid.server.http.numThreads=50
```
### Real-time Node
Run:
```
io.druid.cli.Main server realtime
```
Hardware (this is a little overkill):
```
r3.8xlarge (Cores: 32, Memory: 244 GB, SSD - this hardware is way overkill for the real-time node but we choose it for simplicity)
```
JVM Configuration:
```
-server
-Xmx13g
-Xms13g
-XX:NewSize=2g
-XX:MaxNewSize=2g
-XX:MaxDirectMemorySize=9g
-XX:+UseConcMarkSweepGC
-XX:+PrintGCDetails
-XX:+PrintGCTimeStamps
-XX:+HeapDumpOnOutOfMemoryError
-Duser.timezone=UTC
-Dfile.encoding=UTF-8
-Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager
-Djava.io.tmpdir=/mnt/tmp
-Dcom.sun.management.jmxremote.port=17071
-Dcom.sun.management.jmxremote.authenticate=false
-Dcom.sun.management.jmxremote.ssl=false
```
Runtime.properties:
```
druid.host=#{IP_ADDR}
druid.port=8080
druid.service=druid/prod/realtime
druid.processing.buffer.sizeBytes=1073741824
druid.processing.numThreads=7
druid.server.http.numThreads=50
druid.monitoring.monitors=["io.druid.segment.realtime.RealtimeMetricsMonitor", "com.metamx.metrics.JvmMonitor"]
```

View File

@ -24,14 +24,18 @@ This data set is composed of three distinct components. If you are acquainted wi
* **Timestamp column**: We treat timestamp separately because all of our queries
center around the time axis.
* **Dimension columns**: We have four dimensions of publisher, advertiser, gender, and country.
* **Dimension columns**: Dimensions are string attributes of an event, and the columns most commonly used in filtering the data.
We have four dimensions in our example data set: publisher, advertiser, gender, and country.
They each represent an axis of the data that weve chosen to slice across.
* **Metric columns**: These are clicks and price. These represent values, usually numeric,
which are derived from an aggregation operation such as count, sum, and mean.
* **Metric columns**: Metrics are columns used in aggregations and computations. In our example, the metrics are clicks and price.
Metrics are usually numeric values, and computations include operations such as count, sum, and mean.
Also known as measures in standard OLAP terminology.
Individually, the events are not very interesting, however, summarizations of this type of data can yield many useful insights.
## Roll-up
The individual events in our example data set are not very interesting because there may be trillions of such events.
However, summarizations of this type of data can yield many useful insights.
Druid summarizes this raw data at ingestion time using a process we refer to as "roll-up".
Roll-up is a first-level aggregation operation over a selected set of dimensions, equivalent to (in pseudocode):
@ -47,12 +51,10 @@ The compacted version of our original raw data looks something like this:
2011-01-01T02:00:00Z bieberfever.com google.com Male UK 3194 170 34.01
In practice, we see that rolling up data can dramatically reduce the size of data that needs to be stored (up to a factor of 100).
Druid will roll up data as it is ingested to minimize the amount of raw data that needs to be stored.
This storage reduction does come at a cost; as we roll up data, we lose the ability to query individual events. Phrased another way,
the rollup granularity is the minimum granularity you will be able to query data at. Hence, Druid ingestion specs define this granularity as the `queryGranularity` of the data.
The lowest `queryGranularity` is millisecond.
Druid is designed to perform single table operations and does not currently support joins.
Although many production setups instrument joins at the ETL level, data must be denormalized before it is loaded into Druid.
the rollup granularity is the minimum granularity you will be able to explore data at and events are floored to this granularity.
Hence, Druid ingestion specs define this granularity as the `queryGranularity` of the data. The lowest supported `queryGranularity` is millisecond.
## Sharding the Data
@ -75,7 +77,9 @@ Segments are self-contained containers for the time interval of data they hold.
contain data stored in compressed column orientations, along with the indexes for those columns. Druid queries only understand how to
scan segments.
Segments are uniquely identified by a datasource, interval, version, and an optional partition number.
Segments are uniquely identified by a datasource, interval, version, and an optional partition number.
Examining our example segments, the segments are named following this convention: `dataSource_interval_version_partitionNumber`
## Indexing the Data
@ -94,6 +98,13 @@ Druid has two means of ingestion, real-time and batch. Real-time ingestion in Dr
Batch ingestion provides exactly once guarantees and segments created via batch processing will accurately reflect the ingested data.
One common approach to operating Druid is to have a real-time pipeline for recent insights, and a batch pipeline for the accurate copy of the data.
## Querying the Data
Druid's native query language is JSON over HTTP, although the community has contributed query libraries in [numerous languages](../development/libraries.html), including SQL.
Druid is designed to perform single table operations and does not currently support joins.
Many production setups do joins at ETL because data must be denormalized before loading into Druid.
## The Druid Cluster
A Druid Cluster is composed of several different types of nodes. Each node is designed to do a small set of things very well.

View File

@ -34,17 +34,66 @@ The code to reproduce the results in the blog post are all open source. The blog
## Colocating Druid Processes for a POC
Not all Druid node processes need to run on separate machines. You can set up a small cluster with colocated processes to load several gigabytes of data.
Not all Druid node processes need to run on separate machines. You can set up a small cluster with colocated processes to load several gigabytes of data. Please note this cluster is not highly available.
It is recommended you follow the [example production configuration](../configuration/production-cluster.html) for an actual production setup.
1. node1: [Coordinator](../design/coordinator.html) + metadata store + zookeeper
2. node2: [Broker](../design/broker.html) + [Historical](../design/historical.html)
3. node3: [Overlord](../design/indexing-service.html)
The deep storage to use in this POC example can be S3 or HDFS.
1. node1: [Coordinator](../design/coordinator.html) + metadata store + zookeeper.
Example hardware: EC2 c3.2xlarge node (8 cores, Intel Xeon E5-2680 v2 @ 2.80GHz and 15GB of RAM).
See [here](../configuration/production-cluster.html) for the runtime.properties. Some example JVM configs for this hardware:
```
-server
-Xmx6g
-Xms6g
-XX:NewSize=512m
-XX:MaxNewSize=512m
-XX:+UseConcMarkSweepGC
```
2. node2: [Broker](../design/broker.html)
Example hardware: EC2 c3.2xlarge node (8 cores, Intel Xeon E5-2680 v2 @ 2.80GHz and 15GB of RAM).
[Example configs](https://github.com/druid-io/druid-benchmark/tree/master/config) (see broker-* files).
2. node3: [Historical](../design/historical.html).
Example hardware: EC2 m3.2xlarge instances (8 cores, Intel Xeon E5-2670 v2 @ 2.50GHz with 160GB SSD and 30GB of RAM)
[Example configs](https://github.com/druid-io/druid-benchmark/tree/master/config) (see compute-* files).
3. node4 (optional): [Real-time](../design/realtime.html) node or [Overlord](../design/indexing-service.html) (depending on how you choose to ingest data).
Example hardware: EC2 c3.2xlarge node (8 cores, Intel Xeon E5-2680 v2 @ 2.80GHz and 15GB of RAM).
For the real-time node, see [here](../configuration/production-cluster.html) for the runtime.properties. Use with the following JVM configs:
```
-server
-Xmx8g
-Xms8g
-XX:NewSize=1g
-XX:MaxNewSize=1g
-XX:+UseConcMarkSweepGC
```
For the overlord running in local mode to do all ingestion, see [here](../configuration/production-cluster.html) for the runtime.properties. Use with the following JVM configs:
```
-server
-Xmx2g
-Xms2g
-XX:NewSize=256m
-XX:MaxNewSize=256m
```
The size of the runner javaOpts can be bumped up:
```
druid.indexer.runner.javaOpts="-server -Xmx6g -Xms6g -XX:NewSize=256m -XX:MaxNewSize=256m"
```
The coordination pieces (coordinator, metadata store, ZK) can be colocated on the same node. These processes do not require many resources, even for reasonably large clusters.
The query pieces (broker + historical) can be colocated. You can add more of these nodes if your data doesn't fit on a single machine. Make sure to allocate enough heap/off-heap size to both processes.
You can add more historical nodes if your data doesn't fit on a single machine.
For small ingest workloads, you can run the overlord in local mode to load your data.

View File

@ -0,0 +1,188 @@
---
layout: doc_page
---
What to Do When You Have a Firewall
-----------------------------------
When you are behind a firewall, the Maven Druid dependencies will not be accessible, as well as the IRC wikipedia channels that feed realtime data into Druid. To workaround those two challenges, you will need to:
1. Make the Maven Druid dependencies available offline
2. Make the Wikipedia example GeoLite DB dependency available offline
## Making Maven Druid Dependencies Available Offline
1. Extract Druid to a machine that has internet access; e.g. `/Users/foo/druid-<version>`
2. Create a repository directory to download the dependencies to; e.g. `/Users/foo/druid-<version>\repo`
3. Create property `druid.extensions.localRepository=`*`path to repo directory`* in the *`Druid Directory`*`\config\_common/common.runtime.properties` file; e.g. `druid.extensions.localRepository=/Users/foo/druid-<version>/repo`
4. From within Druid directory, run the `pull-deps` command to download all Druid dependencies to the repository specified in the `common.runtime.properties` file:
```
java -classpath "config\_common;lib\*" io.druid.cli.Main tools pull-deps
```
5. Once all dependencies have been downloaded successfully, replicate the `repo` directory to the machine behind the firewall; e.g. `/opt/druid-<version>/repo`
6. Create property `druid.extensions.localRepository=`*`path to repo directory`* in the *`Druid Directory`*`/config/_common/common.runtime.properties` file; e.g. `druid.extensions.localRepository=/opt/druid-<version>/repo`
## Making the Wikipedia Example GeoLite DB Dependency Available Offline
1. Download GeoLite2 City DB from http://dev.maxmind.com/geoip/geoip2/geolite2/
2. Copy and extract the DB to *`java.io.tmpdir`*`/io.druid.segment.realtime.firehose.WikipediaIrcDecoder.GeoLite2-City.mmdb`; e.g. `/tmp/io.druid.segment.realtime.firehose.WikipediaIrcDecoder.GeoLite2-City.mmdb`
**Note**: depending on the machine's reboot policy, if the `java.io.tmpdir` resolves to the `/tmp` directory, you may have to create this file again in the `tmp` directory after a machine reboot
## Loading the Data into Druid directly from Kafka
As an alternative to reading the data from the IRC channels, which is a challenge to try to do it from behind a firewall, we will use Kafka to stream the data to Druid. To do so, we will need to:
1. Configure the Wikipedia example to read streaming data from Kafka
2. Set up and configure Kafka
#### Wikipedia Example Configuration
1. In your favorite editor, open the file `druid-<version>/examples/wikipedia/wikipedia_realtime.spec`
2. Backup the file, if necessary, then replace the file content with the following:
```json
[
{
"dataSchema": {
"dataSource": "wikipedia",
"parser": {
"type": "string",
"parseSpec": {
"format": "json",
"timestampSpec": {
"column": "timestamp",
"format": "auto"
},
"dimensionsSpec": {
"dimensions": [
"page",
"language",
"user",
"unpatrolled",
"newPage",
"robot",
"anonymous",
"namespace",
"continent",
"country",
"region",
"city"
],
"dimensionExclusions": [],
"spatialDimensions": []
}
}
},
"metricsSpec": [
{
"type": "count",
"name": "count"
},
{
"type": "doubleSum",
"name": "added",
"fieldName": "added"
},
{
"type": "doubleSum",
"name": "deleted",
"fieldName": "deleted"
},
{
"type": "doubleSum",
"name": "delta",
"fieldName": "delta"
}
],
"granularitySpec": {
"type": "uniform",
"segmentGranularity": "DAY",
"queryGranularity": "NONE"
}
},
"ioConfig": {
"type": "realtime",
"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"
},
"plumber": {
"type": "realtime"
}
},
"tuningConfig": {
"type": "realtime",
"maxRowsInMemory": 500000,
"intermediatePersistPeriod": "PT10m",
"windowPeriod": "PT10m",
"basePersistDirectory": "/tmp/realtime/basePersist",
"rejectionPolicy": {
"type": "messageTime"
}
}
}
]
```
3. Refer to the [Running Example Scripts](#running-example-scripts) section to start the example Druid Realtime node by issuing the following from within your Druid directory:
```bash
./run_example_server.sh
```
#### Kafka Setup and Configuration
1. Download Kafka
For this tutorial we will [download Kafka 0.8.2.1]
(https://www.apache.org/dyn/closer.cgi?path=/kafka/0.8.2.1/kafka_2.10-0.8.2.1.tgz)
```bash
tar -xzf kafka_2.10-0.8.2.1.tgz
cd kafka_2.10-0.8.2.1
```
2. Start Kafka
**First, launch ZooKeeper** (refer to the [Set up Zookeeper](#set-up-zookeeper) section for details), then start the Kafka server (in a separate console):
```bash
./bin/kafka-server-start.sh config/server.properties
```
3. Create a topic named `wikipedia`
```bash
./bin/kafka-topics.sh --create --zookeeper localhost:2181 --replication-factor 1 --partitions 1 --topic wikipedia
```
4. Launch a console producer for the topic `wikipedia`
```bash
./bin/kafka-console-producer.sh --broker-list localhost:9092 --topic wikipedia
```
5. Copy and paste the following data into the terminal where we launched the Kafka console producer in the previous step:
```json
{"timestamp": "2013-08-31T01:02:33Z", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 57, "deleted": 200, "delta": -143}
{"timestamp": "2013-08-31T03:32:45Z", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330}
{"timestamp": "2013-08-31T07:11:21Z", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111}
{"timestamp": "2013-08-31T11:58:39Z", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900}
{"timestamp": "2013-08-31T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9}
```
#### Finally
Now, that data has been fed into Druid, refer to the [Running Example Scripts](#running-example-scripts) section to query the real-time node by issuing the following from within the Druid directory:
```bash
./run_example_client.sh
```
The [Querying Druid](../querying/querying.md) section also has further querying examples.

View File

@ -5,6 +5,8 @@ layout: doc_page
# Tutorial: A First Look at Druid
Greetings! This tutorial will help clarify some core Druid concepts. We will use a real-time dataset and issue some basic Druid queries. If you are ready to explore Druid, and learn a thing or two, read on!
Note: If you are behind a corporate firewall, please see our additional [instructions](../tutorials/firewall.html) for running this tutorial.
About the data
--------------
@ -73,17 +75,21 @@ You should see a bunch of files:
* run_example_client.sh
* LICENSE, config, examples, lib directories
## External Dependencies
Druid requires 3 external dependencies.
* A "deep storage" that acts as a data repository. This is generally distributed storage like HDFS or S3. For prototyping or experimentation on a single machine, Druid can use the local filesystem.
* A "metadata storage" to hold configuration and metadata information. This is generally a small, shared database like MySQL or Postgres. For prototyping or experimentation on a single machine, Druid can use a local instance of [Apache Derby](http://db.apache.org/derby/).
* [Apache Zookeeper](http://zookeeper.apache.org/) for coordination among different pieces of the cluster.
This tutorial only requires Zookeeper be set up.
#### Set up Zookeeper
* Download zookeeper from [http://www.apache.org/dyn/closer.cgi/zookeeper/](http://www.apache.org/dyn/closer.cgi/zookeeper/)
* Download zookeeper from [http://www.apache.org/dyn/closer.cgi/zookeeper/](http://www.apache.org/dyn/closer.cgi/zookeeper/).
* Install zookeeper.
```bash
@ -98,7 +104,7 @@ cd ..
Running Example Scripts
-----------------------
Let's start doing stuff. You can start an example Druid [Realtime](../design/realtime.html) node by issuing:
Let's start by spinning up a server. You can start an example Druid [Realtime](../design/realtime.html) node by issuing:
```
./run_example_server.sh
@ -119,7 +125,7 @@ Note that the first time you start the example, it may take some extra time due
2015-02-17T21:46:37,009 INFO [ServerInventoryView-0] io.druid.client.BatchServerInventoryView - Inventory Initialized
```
The Druid real time-node ingests events in an in-memory buffer. Periodically, these events will be persisted to disk. If you are interested in the details of our real-time architecture and why we persist indexes to disk, we suggest you read our [White Paper](http://static.druid.io/docs/druid.pdf).
The Druid real-time node ingests events in an in-memory buffer. Periodically, these events will be persisted to disk. If you are interested in the details of our real-time architecture and why we persist indexes to disk, we suggest you read our [White Paper](http://static.druid.io/docs/druid.pdf).
To query the real-time node you've spun up, you can issue:
@ -295,191 +301,6 @@ To learn more about loading streaming data, see [Loading Streaming Data](../tuto
To learn more about loading batch data, see [Loading Batch Data](../tutorials/tutorial-loading-batch-data.html).
What to Do When You Have a Firewall
-----------------------------------
When you are behind a firewall, the Maven Druid dependencies will not be accessible, as well as the IRC wikipedia channels that feed realtime data into Druid. To workaround those two challenges, you will need to:
1. make the Maven Druid dependencies available offline
2. make the Wikipedia example GeoLite DB dependency available offline
3. use an alternative approach to load data into Druid
#### Making Maven Druid Dependencies Available Offline
1. Extract Druid to a machine that has internet access; e.g. `C:\druid-<version>`
2. Create a repository directory to download the dependencies to; e.g. `C:\druid-<version>\repo`
3. Create property `druid.extensions.localRepository=`*`path to repo directory`* in the *`Druid Directory`*`\config\_common/common.runtime.properties` file; e.g. `druid.extensions.localRepository=C\:/druid-<version>/repo`
4. From within Druid directory, run the `pull-deps` command to download all Druid dependencies to the repository specified in the `common.runtime.properties` file:
```
java -classpath "config\_common;lib\*" io.druid.cli.Main tools pull-deps
```
5. Once all dependencies have been downloaded successfully, replicate the `repo` directory to the machine behind the firewall; e.g. `/opt/druid-<version>/repo`
6. Create property `druid.extensions.localRepository=`*`path to repo directory`* in the *`Druid Directory`*`/config/_common/common.runtime.properties` file; e.g. `druid.extensions.localRepository=/opt/druid-<version>/repo`
#### Making the Wikipedia Example GeoLite DB Dependency Available Offline
1. Download GeoLite2 City DB from http://dev.maxmind.com/geoip/geoip2/geolite2/
2. Copy and extract the DB to *`java.io.tmpdir`*`/io.druid.segment.realtime.firehose.WikipediaIrcDecoder.GeoLite2-City.mmdb`; e.g. `/tmp/io.druid.segment.realtime.firehose.WikipediaIrcDecoder.GeoLite2-City.mmdb`
**Note**: depending on the machine's reboot policy, if the `java.io.tmpdir` resolves to the `/tmp` directory, you may have to create this file again in the `tmp` directory after a machine reboot
#### Loading the Data into Druid
As an alternative to reading the data from the IRC channels, which is a challenge to try to do it from behind a firewall, we will use Kafka to stream the data to Druid. To do so, we will need to:
1. configure the Wikipedia example to read streaming data from Kafka
2. set up and configure Kafka
###### Wikipedia Example Configuration
1. In your favorite editor, open the file `druid-<version>/examples/wikipedia/wikipedia_realtime.spec`
2. Backup the file, if necessary, then replace the file content with the following:
```json
[
{
"dataSchema": {
"dataSource": "wikipedia",
"parser": {
"type": "string",
"parseSpec": {
"format": "json",
"timestampSpec": {
"column": "timestamp",
"format": "auto"
},
"dimensionsSpec": {
"dimensions": [
"page",
"language",
"user",
"unpatrolled",
"newPage",
"robot",
"anonymous",
"namespace",
"continent",
"country",
"region",
"city"
],
"dimensionExclusions": [],
"spatialDimensions": []
}
}
},
"metricsSpec": [
{
"type": "count",
"name": "count"
},
{
"type": "doubleSum",
"name": "added",
"fieldName": "added"
},
{
"type": "doubleSum",
"name": "deleted",
"fieldName": "deleted"
},
{
"type": "doubleSum",
"name": "delta",
"fieldName": "delta"
}
],
"granularitySpec": {
"type": "uniform",
"segmentGranularity": "DAY",
"queryGranularity": "NONE"
}
},
"ioConfig": {
"type": "realtime",
"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"
},
"plumber": {
"type": "realtime"
}
},
"tuningConfig": {
"type": "realtime",
"maxRowsInMemory": 500000,
"intermediatePersistPeriod": "PT10m",
"windowPeriod": "PT10m",
"basePersistDirectory": "/tmp/realtime/basePersist",
"rejectionPolicy": {
"type": "messageTime"
}
}
}
]
```
3. Refer to the [Running Example Scripts](#running-example-scripts) section to start the example Druid Realtime node by issuing the following from within your Druid directory:
```bash
./run_example_server.sh
```
###### Kafka Setup and Configuration
1. Download Kafka
For this tutorial we will [download Kafka 0.8.2.1]
(https://www.apache.org/dyn/closer.cgi?path=/kafka/0.8.2.1/kafka_2.10-0.8.2.1.tgz)
```bash
tar -xzf kafka_2.10-0.8.2.1.tgz
cd kafka_2.10-0.8.2.1
```
2. Start Kafka
**First, launch ZooKeeper** (refer to the [Set up Zookeeper](#set-up-zookeeper) section for details), then start the Kafka server (in a separate console):
```bash
./bin/kafka-server-start.sh config/server.properties
```
3. Create a topic named `wikipedia`
```bash
./bin/kafka-topics.sh --create --zookeeper localhost:2181 --replication-factor 1 --partitions 1 --topic wikipedia
```
4. Launch a console producer for the topic `wikipedia`
```bash
./bin/kafka-console-producer.sh --broker-list localhost:9092 --topic wikipedia
```
5. Copy and paste the following data into the terminal where we launched the Kafka console producer in the previous step:
```json
{"timestamp": "2013-08-31T01:02:33Z", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 57, "deleted": 200, "delta": -143}
{"timestamp": "2013-08-31T03:32:45Z", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330}
{"timestamp": "2013-08-31T07:11:21Z", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111}
{"timestamp": "2013-08-31T11:58:39Z", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900}
{"timestamp": "2013-08-31T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9}
```
#### Finally
Now, that data has been fed into Druid, refer to the [Running Example Scripts](#running-example-scripts) section to query the real-time node by issuing the following from within the Druid directory:
```bash
./run_example_client.sh
```
The [Querying Druid](#querying-druid) section also has further querying examples.
Additional Information
----------------------

View File

@ -10,7 +10,6 @@ This tutorial will hopefully answer these questions!
In this tutorial, we will set up other types of Druid nodes and external dependencies for a fully functional Druid cluster. The architecture of Druid is very much like the [Megazord](http://www.youtube.com/watch?v=7mQuHh1X4H4) from the popular 90s show Mighty Morphin' Power Rangers. Each Druid node has a specific purpose and the nodes come together to form a fully functional system.
## Downloading Druid
If you followed the first tutorial, you should already have Druid downloaded. If not, let's go back and do that first.
You can download the latest version of druid [here](http://druid.io/downloads.html). You can also [Build From Source](../development/build.html) and grab the tarball from services/target/druid-<version>-bin.tar.gz.
@ -28,7 +27,7 @@ Druid requires 3 external dependencies.
* A "metadata storage" to hold configuration and metadata information. This is generally a small, shared database like MySQL or Postgres. For prototyping or experimentation on a single machine, Druid can use a local instance of [Apache Derby](http://db.apache.org/derby/).
* [Apache Zookeeper](http://zookeeper.apache.org/) for coordination among different pieces of the cluster.
For deep storage, we will use the local disk in this tutorial, but for production, HDFS and S3 are popular options. For the metadata storage, Derby is used, but for production Mysql or PostgreSQL etc should be used.
For deep storage, we will use the local disk in this tutorial, but for production, HDFS and S3 are popular options. For the metadata storage, Derby is used in this tutorial, but for production MySQL or PostgreSQL etc should be used.
#### Set up Zookeeper