You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by jo...@apache.org on 2018/08/10 04:04:42 UTC

[incubator-druid] branch 0.12.3 updated: [Backport] New docs intro (#6146)

This is an automated email from the ASF dual-hosted git repository.

jonwei pushed a commit to branch 0.12.3
in repository https://gitbox.apache.org/repos/asf/incubator-druid.git


The following commit(s) were added to refs/heads/0.12.3 by this push:
     new d8ed1ea  [Backport] New docs intro (#6146)
d8ed1ea is described below

commit d8ed1ea4bc17f609efe57f107fb7e4b537ae1b81
Author: Jonathan Wei <jo...@users.noreply.github.com>
AuthorDate: Thu Aug 9 21:04:40 2018 -0700

    [Backport] New docs intro (#6146)
---
 docs/content/design/index.md       | 305 +++++++++++++++++++++----------------
 docs/content/ingestion/overview.md | 279 +++++++++++++++++++++++++++++++++
 docs/content/toc.md                |  10 +-
 docs/img/druid-architecture.png    | Bin 0 -> 151681 bytes
 docs/img/druid-timeline.png        | Bin 0 -> 36729 bytes
 5 files changed, 461 insertions(+), 133 deletions(-)

diff --git a/docs/content/design/index.md b/docs/content/design/index.md
index a39f9de..a0bed67 100644
--- a/docs/content/design/index.md
+++ b/docs/content/design/index.md
@@ -2,152 +2,193 @@
 layout: doc_page
 ---
 
-# Druid Concepts
+# What is Druid?<a id="what-is-druid"></a>
+
+Druid is a data store designed for high-performance slice-and-dice analytics
+("[OLAP](http://en.wikipedia.org/wiki/Online_analytical_processing)"-style) on large data sets. Druid is most often
+used as a data store for powering GUI analytical applications, or as a backend for highly-concurrent APIs that need
+fast aggregations. Common application areas for Druid include:
+
+- Clickstream analytics
+- Network flow analytics
+- Server metrics storage
+- Application performance metrics
+- Digital marketing analytics
+- Business intelligence / OLAP
+
+Druid's key features are:
+
+1. **Columnar storage format.** Druid uses column-oriented storage, meaning it only needs to load the exact columns
+needed for a particular query.  This gives a huge speed boost to queries that only hit a few columns. In addition, each
+column is stored optimized for its particular data type, which supports fast scans and aggregations.
+2. **Scalable distributed system.** Druid is typically deployed in clusters of tens to hundreds of servers, and can
+offer ingest rates of millions of records/sec, retention of trillions of records, and query latencies of sub-second to a
+few seconds.
+3. **Massively parallel processing.** Druid can process a query in parallel across the entire cluster.
+4. **Realtime or batch ingestion.** Druid can ingest data either realtime (ingested data is immediately available for
+querying) or in batches.
+5. **Self-healing, self-balancing, easy to operate.** As an operator, to scale the cluster out or in, simply add or
+remove servers and the cluster will rebalance itself automatically, in the background, without any downtime. If any
+Druid servers fail, the system will automatically route around the damage until those servers can be replaced. Druid
+is designed to run 24/7 with no need for planned downtimes for any reason, including configuration changes and software
+updates.
+6. **Cloud-native, fault-tolerant architecture that won't lose data.** Once Druid has ingested your data, a copy is
+stored safely in [deep storage](#deep-storage) (typically cloud storage, HDFS, or a shared filesystem). Your data can be
+recovered from deep storage even if every single Druid server fails. For more limited failures affecting just a few
+Druid servers, replication ensures that queries are still possible while the system recovers.
+7. **Indexes for quick filtering.** Druid uses [CONCISE](https://arxiv.org/pdf/1004.0403) or
+[Roaring](https://roaringbitmap.org/) compressed bitmap indexes to create indexes that power fast filtering and
+searching across multiple columns.
+8. **Approximate algorithms.** Druid includes algorithms for approximate count-distinct, approximate ranking, and
+computation of approximate histograms and quantiles. These algorithms offer bounded memory usage and are often
+substantially faster than exact computations. For situations where accuracy is more important than speed, Druid also
+offers exact count-distinct and exact ranking.
+9. **Automatic summarization at ingest time.** Druid optionally supports data summarization at ingestion time. This
+summarization partially pre-aggregates your data, and can lead to big costs savings and performance boosts.
+
+# When should I use Druid?<a id="when-to-use-druid"></a>
+
+Druid is likely a good choice if your use case fits a few of the following descriptors:
+
+- Insert rates are very high, but updates are less common.
+- Most of your queries are aggregation and reporting queries ("group by" queries). You may also have searching and
+scanning queries.
+- You are targeting query latencies of 100ms to a few seconds.
+- Your data has a time component (Druid includes optimizations and design choices specifically related to time).
+- You may have more than one table, but each query hits just one big distributed table. Queries may potentially hit more
+than one smaller "lookup" table.
+- You have high cardinality data columns (e.g. URLs, user IDs) and need fast counting and ranking over them.
+- You want to load data from Kafka, HDFS, flat files, or object storage like Amazon S3.
+
+Situations where you would likely _not_ want to use Druid include:
+
+- You need low-latency updates of _existing_ records using a primary key. Druid supports streaming inserts, but not streaming updates (updates are done using
+background batch jobs).
+- You are building an offline reporting system where query latency is not very important.
+- You want to do "big" joins (joining one big fact table to another big fact table).
+
+# Architecture
+
+Druid has a multi-process, distributed architecture that is designed to be cloud-friendly and easy to operate. Each
+Druid process type can be configured and scaled independently, giving you maximum flexibility over your cluster. This
+design also provides enhanced fault tolerance: an outage of one component will not immediately affect other components.
+
+Druid's process types are:
+
+* [**Historical**](../design/historical.html) processes are the workhorses that handle storage and querying on "historical" data
+(including any streaming data that has been in the system long enough to be committed). Historical processes
+download segments from deep storage and respond to queries about these segments. They don't accept writes.
+* [**MiddleManager**](../design/middlemanager.html) processes handle ingestion of new data into the cluster. They are responsible
+for reading from external data sources and publishing new Druid segments.
+* [**Broker**](../design/broker.html) processes receive queries from external clients and forward those queries to Historicals and
+MiddleManagers. When Brokers receive results from those subqueries, they merge those results and return them to the
+caller. End users typically query Brokers rather than querying Historicals or MiddleManagers directly.
+* [**Coordinator**](../design/coordinator.html) processes watch over the Historical processes. They are responsible for assigning
+segments to specific servers, and for ensuring segments are well-balanced across Historicals.
+* [**Overlord**](../design/overlord.html) processes watch over the MiddleManager processes and are the controllers of data ingestion
+into Druid. They are responsible for assigning ingestion tasks to MiddleManagers and for coordinating segment
+publishing.
+* [**Router**](../development/router.html) processes are _optional_ processes that provide a unified API gateway in front of Druid Brokers,
+Overlords, and Coordinators. They are optional since you can also simply contact the Druid Brokers, Overlords, and
+Coordinators directly.
+
+Druid processes can be deployed individually (one per physical server, virtual server, or container) or can be colocated
+on shared servers. One common colocation plan is a three-type plan:
+
+1. "Data" servers run Historical and MiddleManager processes.
+2. "Query" servers run Broker and (optionally) Router processes.
+3. "Master" servers run Coordinator and Overlord processes. They may run ZooKeeper as well.
+
+In addition to these process types, Druid also has three external dependencies. These are intended to be able to
+leverage existing infrastructure, where present.
+
+* [**Deep storage**](#deep-storage), shared file storage accessible by every Druid server. This is typically going to
+be a distributed object store like S3 or HDFS, or a network mounted filesystem. Druid uses this to store any data that
+has been ingested into the system.
+* [**Metadata store**](#metadata-storage), shared metadata storage. This is typically going to be a traditional RDBMS
+like PostgreSQL or MySQL.
+* [**ZooKeeper**](#zookeeper) is used for internal service discovery, coordination, and leader election.
+
+The idea behind this architecture is to make a Druid cluster simple to operate in production at scale. For example, the
+separation of deep storage and the metadata store from the rest of the cluster means that Druid processes are radically
+fault tolerant: even if every single Druid server fails, you can still relaunch your cluster from data stored in deep
+storage and the metadata store.
+
+The following diagram shows how queries and data flow through this architecture:
+
+<img src="../../img/druid-architecture.png" width="800"/>
+
+# Datasources and segments
+
+Druid data is stored in "datasources", which are similar to tables in a traditional RDBMS. Each datasource is
+partitioned by time and, optionally, further partitioned by other attributes. Each time range is called a "chunk" (for
+example, a single day, if your datasource is partitioned by day). Within a chunk, data is partitioned into one or more
+"segments". Each segment is a single file, typically comprising up to a few million rows of data. Since segments are
+organized into time chunks, it's sometimes helpful to think of segments as living on a timeline like the following:
+
+<img src="../../img/druid-timeline.png" width="800" />
+
+A datasource may have anywhere from just a few segments, up to hundreds of thousands and even millions of segments. Each
+segment starts life off being created on a MiddleManager, and at that point, is mutable and uncommitted. The segment
+building process includes the following steps, designed to produce a data file that is compact and supports fast
+queries:
+
+- Conversion to columnar format
+- Indexing with bitmap indexes
+- Compression using various algorithms
+    - Dictionary encoding with id storage minimization for String columns
+    - Bitmap compression for bitmap indexes
+    - Type-aware compression for all columns
 
-Druid is an open source data store designed for [OLAP](http://en.wikipedia.org/wiki/Online_analytical_processing) queries on event data.
-This page is meant to provide readers with a high level overview of how Druid stores data, and the architecture of a Druid cluster.
+Periodically, segments are committed and published. At this point, they are written to [deep storage](#deep-storage), 
+become immutable, and move from MiddleManagers to the Historical processes (see [Architecture](#architecture) above
+for details). An entry about the segment is also written to the [metadata store](#metadata-storage). This entry is a
+self-describing bit of metadata about the segment, including things like the schema of the segment, its size, and its
+location on deep storage. These entries are what the Coordinator uses to know what data *should* be available on the
+cluster.
 
-## The Data
+# Query processing
 
-To frame our discussion, let's begin with an example data set (from online advertising):
+Queries first enter the Broker, where the Broker will identify which segments have data that may pertain to that query.
+The list of segments is always pruned by time, and may also be pruned by other attributes depending on how your
+datasource is partitioned. The Broker will then identify which Historicals and MiddleManagers are serving those segments
+and send a rewritten subquery to each of those processes. The Historical/MiddleManager processes will take in the
+queries, process them and return results. The Broker receives results and merges them together to get the final answer,
+which it returns to the original caller.
 
-    timestamp             publisher          advertiser  gender  country  click  price
-    2011-01-01T01:01:35Z  bieberfever.com    google.com  Male    USA      0      0.65
-    2011-01-01T01:03:63Z  bieberfever.com    google.com  Male    USA      0      0.62
-    2011-01-01T01:04:51Z  bieberfever.com    google.com  Male    USA      1      0.45
-    2011-01-01T01:00:00Z  ultratrimfast.com  google.com  Female  UK       0      0.87
-    2011-01-01T02:00:00Z  ultratrimfast.com  google.com  Female  UK       0      0.99
-    2011-01-01T02:00:00Z  ultratrimfast.com  google.com  Female  UK       1      1.53
+Broker pruning is an important way that Druid limits the amount of data that must be scanned for each query, but it is
+not the only way. For filters at a more granular level than what the Broker can use for pruning, indexing structures
+inside each segment allow Druid to figure out which (if any) rows match the filter set before looking at any row of
+data. Once Druid knows which rows match a particular query, it only accesses the specific columns it needs for that
+query. Within those columns, Druid can skip from row to row, avoiding reading data that doesn't match the query filter.
 
-This data set is composed of three distinct components. If you are acquainted with OLAP terminology, the following concepts should be familiar.
+So Druid uses three different techniques to maximize query performance:
 
-* **Timestamp column**: We treat timestamp separately because all of our queries
- center around the time axis.
+- Pruning which segments are accessed for each query.
+- Within each segment, using indexes to identify which rows must be accessed.
+- Within each segment, only reading the specific rows and columns that are relevant to a particular query.
 
-* **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 we’ve chosen to slice across.
 
-* **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.
+# External Dependencies
 
-## Sharding the Data
+## Deep storage
 
-Druid shards are called `segments` and Druid always first shards data by time. In our compacted data set, we can create two segments, one for each hour of data.
+Druid uses deep storage only as a backup of your data and as a way to transfer data in the background between
+Druid processes. To respond to queries, Historical processes do not read from deep storage, but instead read pre-fetched
+segments from their local disks before any queries are served. This means that Druid never needs to access deep storage
+during a query, helping it offer the best query latencies possible. It also means that you must have enough disk space
+both in deep storage and across your Historical processes for the data you plan to load.
 
-For example:
+For more details, please see [Deep storage dependency](../dependencies/deep-storage.html).
 
-Segment `sampleData_2011-01-01T01:00:00:00Z_2011-01-01T02:00:00:00Z_v1_0` contains
+## Metadata storage
 
-     2011-01-01T01:00:00Z  ultratrimfast.com  google.com  Male   USA     1800        25     15.70
-     2011-01-01T01:00:00Z  bieberfever.com    google.com  Male   USA     2912        42     29.18
+The metadata storage holds various system metadata such as segment availability information and task information.
 
+For more details, please see [Metadata storage dependency](..dependencies/metadata-storage.html)
 
-Segment `sampleData_2011-01-01T02:00:00:00Z_2011-01-01T03:00:00:00Z_v1_0` contains
+## Zookeeper
 
-     2011-01-01T02:00:00Z  ultratrimfast.com  google.com  Male   UK      1953        17     17.31
-     2011-01-01T02:00:00Z  bieberfever.com    google.com  Male   UK      3194        170    34.01
-
-Segments are self-contained containers for the time interval of data they hold. Segments
-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.
-Examining our example segments, the segments are named following this convention: `dataSource_interval_version_partitionNumber`
-
-## 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):
-
-    GROUP BY timestamp, publisher, advertiser, gender, country
-      :: impressions = COUNT(1),  clicks = SUM(click),  revenue = SUM(price)
-
-The compacted version of our original raw data looks something like this:
-
-     timestamp             publisher          advertiser  gender country impressions clicks revenue
-     2011-01-01T01:00:00Z  ultratrimfast.com  google.com  Male   USA     1800        25     15.70
-     2011-01-01T01:00:00Z  bieberfever.com    google.com  Male   USA     2912        42     29.18
-     2011-01-01T02:00:00Z  ultratrimfast.com  google.com  Male   UK      1953        17     17.31
-     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 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.
-
-### Roll-up modes
-
-Druid supports two roll-up modes, i.e., _perfect roll-up_ and _best-effort roll-up_. In the perfect roll-up mode, Druid guarantees that input data are perfectly aggregated at ingestion time. Meanwhile, in the best-effort roll-up, input data might not be perfectly aggregated and thus there can be multiple segments holding the rows which should belong to the same segment with the perfect roll-up since they have the same dimension value and their timestamps fall into the same interval.
-
-The perfect roll-up mode encompasses an additional preprocessing step to determine intervals and shardSpecs before actual data ingestion if they are not specified in the ingestionSpec. This preprocessing step usually scans the entire input data which might increase the ingestion time. The [Hadoop indexing task](../ingestion/batch-ingestion.html) always runs with this perfect roll-up mode.
-
-On the contrary, the best-effort roll-up mode doesn't require any preprocessing step, but the size of ingested data might be larger than that of the perfect roll-up. All types of [streaming indexing (i.e., realtime index task, kafka indexing service, ...)](../ingestion/stream-ingestion.html) run with this mode.
-
-Finally, the [native index task](../ingestion/tasks.html) supports both modes and you can choose either one which fits to your application.
-
-## Indexing the Data
-
-Druid gets its speed in part from how it stores data. Borrowing ideas from search infrastructure,
-Druid creates immutable snapshots of data, stored in data structures highly optimized for analytic queries.
-
-Druid is a column store, which means each individual column is stored separately. Only the columns that pertain to a query are used
-in that query, and Druid is pretty good about only scanning exactly what it needs for a query.
-Different columns can also employ different compression methods. Different columns can also have different indexes associated with them.
-
-Druid indexes data on a per-shard (segment) level.
-
-## Loading the Data
-
-Druid has two means of ingestion, real-time and batch. Real-time ingestion in Druid is best effort. Exactly once semantics are not guaranteed with real-time ingestion in Druid, although we have it on our roadmap to support this.
-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.
-
-* **Historical Nodes** Historical nodes commonly form the backbone of a Druid cluster. Historical nodes download immutable segments locally and serve queries over those segments.
-The nodes have a shared nothing architecture and know how to load segments, drop segments, and serve queries on segments.
-
-* **Broker Nodes** Broker nodes are what clients and applications query to get data from Druid. Broker nodes are responsible for scattering queries and gathering and merging results.
-Broker nodes know what segments live where.
-
-* **Coordinator Nodes** Coordinator nodes manage segments on historical nodes in a cluster. Coordinator nodes tell historical nodes to load new segments, drop old segments, and move segments to load balance.
-
-* **Real-time Processing** Real-time processing in Druid can currently be done using standalone realtime nodes or using the indexing service. The real-time logic is common between these two services.
-Real-time processing involves ingesting data, indexing the data (creating segments), and handing segments off to historical nodes. Data is queryable as soon as it is
- ingested by the realtime processing logic. The hand-off process is also lossless; data remains queryable throughout the entire process.
-
-### External Dependencies
-
-Druid has a couple of external dependencies for cluster operations.
-
-* **Zookeeper** Druid relies on Zookeeper for intra-cluster communication.
-
-* **Metadata Storage** Druid relies on a metadata storage to store metadata about segments and configuration. Services that create segments write new entries to the metadata store
-  and the coordinator nodes monitor the metadata store to know when new data needs to be loaded or old data needs to be dropped. The metadata store is not
-  involved in the query path. MySQL and PostgreSQL are popular metadata stores for production, but Derby can be used for experimentation when you are running all druid nodes on a single machine.
-
-* **Deep Storage** Deep storage acts as a permanent backup of segments. Services that create segments upload segments to deep storage and historical nodes download
-segments from deep storage. Deep storage is not involved in the query path. S3 and HDFS are popular deep storages.
-
-### High Availability Characteristics
-
-Druid is designed to have no single point of failure. Different node types are able to fail without impacting the services of the other node types. To run a highly available Druid cluster, you should have at least 2 nodes of every node type running.
-
-### Comprehensive Architecture
-
-For a comprehensive look at Druid architecture, please read our [white paper](http://static.druid.io/docs/druid.pdf).
+Druid uses [ZooKeeper](http://zookeeper.apache.org/) (ZK) for management of current cluster state.
 
+For more details, please see [Zookeeper dependency](../dependencies/zookeeper.html).
diff --git a/docs/content/ingestion/overview.md b/docs/content/ingestion/overview.md
new file mode 100644
index 0000000..c7f0d67
--- /dev/null
+++ b/docs/content/ingestion/overview.md
@@ -0,0 +1,279 @@
+---
+layout: doc_page
+---
+
+# Ingestion
+
+## Overview
+
+### Datasources and segments
+
+Druid data is stored in "datasources", which are similar to tables in a traditional RDBMS. Each datasource is
+partitioned by time and, optionally, further partitioned by other attributes. Each time range is called a "chunk" (for
+example, a single day, if your datasource is partitioned by day). Within a chunk, data is partitioned into one or more
+"segments". Each segment is a single file, typically comprising up to a few million rows of data. Since segments are
+organized into time chunks, it's sometimes helpful to think of segments as living on a timeline like the following:
+
+<img src="../../img/druid-timeline.png" width="800" />
+
+A datasource may have anywhere from just a few segments, up to hundreds of thousands and even millions of segments. Each
+segment starts life off being created on a MiddleManager, and at that point, is mutable and uncommitted. The segment
+building process includes the following steps, designed to produce a data file that is compact and supports fast
+queries:
+
+- Conversion to columnar format
+- Indexing with bitmap indexes
+- Compression using various algorithms
+    - Dictionary encoding with id storage minimization for String columns
+    - Bitmap compression for bitmap indexes
+    - Type-aware compression for all columns
+
+Periodically, segments are published (committed). At this point, they are written to deep storage, become immutable, and
+move from MiddleManagers to the Historical processes. An entry about the segment is also written to the metadata store.
+This entry is a self-describing bit of metadata about the segment, including things like the schema of the segment, its
+size, and its location on deep storage. These entries are what the Coordinator uses to know what data *should* be
+available on the cluster.
+
+For details on the segment file format, please see [segment files](../design/segments.html).
+
+#### Segment identifiers
+
+Segments all have a four-part identifier with the following components:
+
+- Datasource name.
+- Time interval (for the time chunk containing the segment; this corresponds to the `segmentGranularity` specified
+at ingestion time).
+- Version number (generally an ISO8601 timestamp corresponding to when the segment set was first started).
+- Partition number (an integer, unique within a datasource+interval+version; may not necessarily be contiguous).
+
+For example, this is the identifier for a segment in datasource `clarity-cloud0`, time chunk
+`2018-05-21T16:00:00.000Z/2018-05-21T17:00:00.000Z`, version `2018-05-21T15:56:09.909Z`, and partition number 1:
+
+```
+clarity-cloud0_2018-05-21T16:00:00.000Z_2018-05-21T17:00:00.000Z_2018-05-21T15:56:09.909Z_1
+```
+
+Segments with partition number 0 (the first partition in a chunk) omit the partition number, like the following
+example, which is a segment in the same time chunk as the previous one, but with partition number 0 instead of 1:
+
+```
+clarity-cloud0_2018-05-21T16:00:00.000Z_2018-05-21T17:00:00.000Z_2018-05-21T15:56:09.909Z
+```
+
+#### Segment versioning
+
+You may be wondering what the "version number" described in the previous section is for. Or, you might not be, in which
+case good for you and you can skip this section!
+
+It's there to support batch-mode overwriting. In Druid, if all you ever do is append data, then there will be just a
+single version for each time chunk. But when you overwrite data, what happens behind the scenes is that a new set of
+segments is created with the same datasource, same time interval, but a higher version number. This is a signal to the
+rest of the Druid system that the older version should be removed from the cluster, and the new version should replace
+it.
+
+The switch appears to happen instantaneously to a user, because Druid handles this by first loading the new data (but
+not allowing it to be queried), and then, as soon as the new data is all loaded, switching all new queries to use those
+new segments. Then it drops the old segments a few minutes later.
+
+
+#### Segment states
+
+Segments can be either _available_ or _unavailable_, which refers to whether or not they are currently served by some
+Druid server process. They can also be _published_ or _unpublished_, which refers to whether or not they have been
+written to deep storage and the metadata store. And published segments can be either _used_ or _unused_, which refers to
+whether or not Druid considers them active segments that should be served.
+
+Putting these together, there are five basic states that a segment can be in:
+
+- **Published, available, and used:** These segments are published in deep storage and the metadata store, and they are
+served by Historical processes. They are the majority of active data in a Druid cluster (they include everything except
+in-flight realtime data).
+- **Published, available, and unused:** These segments are being served by Historicals, but won't be for very long. They 
+may be segments that have recently been overwritten (see [Segment versioning](#segment-versioning)) or dropped for
+other reasons (like drop rules, or being dropped manually).
+- **Published, unavailable, and used:** These segments are published in deep storage and the metadata store, and
+_should_ be served, but are not actually being served. If segments stay in this state for more than a few minutes, it's
+usually because something is wrong. Some of the more common causes include: failure of a large number of Historicals,
+Historicals being out of capacity to download more segments, and some issue with coordination that prevents the
+Coordinator from telling Historicals to load new segments.
+- **Published, unavailable, and unused:** These segments are published in deep storage and the metadata store, but
+are inactive (because they have been overwritten or dropped). They lie dormant, and can potentially be resurrected
+by manual action if needed (in particular: setting the "used" flag to true).
+- **Unpublished and available:** This is the state that segments are in while they are being built by Druid ingestion
+tasks. This includes all "realtime" data that has not been handed off to Historicals yet. Segments in this state may or
+may not be replicated. If all replicas are lost, then the segment must be rebuilt from scratch. This may or may not be
+possible. (It is possible with Kafka, and happens automatically; it is possible with S3/HDFS by restarting the job; and
+it is _not_ possible with Tranquility, so in that case, data will be lost.)
+
+The sixth state in this matrix, "unpublished and unavailable," isn't possible. If a segment isn't published and isn't
+being served then does it really exist?
+
+
+#### Indexing and handoff
+
+_Indexing_ is the mechanism by which new segments are created, and _handoff_ is the mechanism by which they are published
+and begin being served by Historical processes. The mechanism works like this on the indexing side:
+
+1. An _indexing task_ starts running and building a new segment. It must determine the identifier of the segment before
+it starts building it. For a task that is appending (like a Kafka task, or an index task in append mode) this will be
+done by calling an "allocate" API on the Overlord to potentially add a new partition to an existing set of segments. For
+a task that is overwriting (like a Hadoop task, or an index task _not_ in append mode) this is done by locking an
+interval and creating a new version number and new set of segments.
+2. If the indexing task is a realtime task (like a Kafka task) then the segment is immediately queryable at this point.
+It's available, but unpublished.
+3. When the indexing task has finished reading data for the segment, it pushes it to deep storage and then publishes it
+by writing a record into the metadata store.
+4. If the indexing task is a realtime task, at this point it waits for a Historical process to load the segment. If the
+indexing task is not a realtime task, it exits immediately.
+
+And like this on the Coordinator / Historical side:
+
+1. The Coordinator polls the metadata store periodically (by default, every 1 minute) for newly published segments.
+2. When the Coordinator finds a segment that is published and used, but unavailable, it chooses a Historical process
+to load that segment and instructs that Historical to do so.
+3. The Historical loads the segment and begins serving it.
+4. At this point, if the indexing task was waiting for handoff, it will exit.
+
+
+## Ingestion methods
+
+In most ingestion methods, this work is done by Druid
+MiddleManager nodes. One exception is Hadoop-based ingestion, where this work is instead done using a Hadoop MapReduce
+job on YARN (although MiddleManager nodes are still involved in starting and monitoring the Hadoop jobs).
+
+Once segments have been generated and stored in [deep storage](../dependencies/deep-storage.html), they will be loaded by Druid Historical nodes. Some Druid
+ingestion methods additionally support _real-time queries_, meaning you can query in-flight data on MiddleManager nodes
+before it is finished being converted and written to deep storage. In general, a small amount of data will be in-flight
+on MiddleManager nodes relative to the larger amount of historical data being served from Historical nodes.
+
+See the [Design](../design/index.html) page for more details on how Druid stores and manages your data.
+
+The table below lists Druid's most common data ingestion methods, along with comparisons to help you choose
+the best one for your situation.
+
+|Method|How it works|Can append and overwrite?|Can handle late data?|Exactly-once ingestion?|Real-time queries?|
+|------|------------|-------------------------|---------------------|-----------------------|------------------|
+|[Native batch](native_tasks.html)|Druid loads data directly from S3, HTTP, NFS, or other networked storage.|Append or overwrite|Yes|Yes|No|
+|[Hadoop](hadoop.html)|Druid launches Hadoop Map/Reduce jobs to load data files.|Append or overwrite|Yes|Yes|No|
+|[Kafka indexing service](../development/extensions-core/kafka-ingestion.html)|Druid reads directly from Kafka.|Append only|Yes|Yes|Yes|
+|[Tranquility](stream-push.html)|You use Tranquility, a client side library, to push individual records into Druid.|Append only|No - late data is dropped|No - may drop or duplicate data|Yes|
+
+## Partitioning
+
+Druid is a distributed data store, and it partitions your data in order to process it in parallel. Druid
+[datasources](../design/index.html) are always partitioned first by time based on the
+[segmentGranularity](../ingestion/index.html#granularityspec) parameter of your ingestion spec. Each of these time partitions is called
+a _time chunk_, and each time chunk contains one or more [segments](../design/segments.html). The segments within a
+particular time chunk may be partitioned further using options that vary based on the ingestion method you have chosen.
+
+ * With [Hadoop](hadoop.html) you can do hash- or range-based partitioning on one or more columns.
+ * With [Native batch](native_tasks.html) you can partition on a hash of all dimension columns. This is useful when
+ rollup is enabled, since it maximizes your space savings.
+ * With [Kafka indexing](../development/extensions-core/kafka-ingestion.html), partitioning is based on Kafka
+ partitions, and is not configurable through Druid. You can configure it on the Kafka side by using the partitioning
+ functionality of the Kafka producer.
+ * With [Tranquility](stream-push.html), partitioning is done by default on a hash of all dimension columns in order
+ to maximize rollup. You can also provide a custom Partitioner class; see the
+ [Tranquility documentation](https://github.com/druid-io/tranquility/blob/master/docs/overview.md#partitioning-and-replication)
+ for details.
+
+All Druid datasources are partitioned by time. Each data ingestion method must acquire a write lock on a particular
+time range when loading data, so no two methods can operate on the same time range of the same datasource at the same
+time. However, two data ingestion methods _can_ operate on different time ranges of the same datasource at the same
+time. For example, you can do a batch backfill from Hadoop while also doing a real-time load from Kafka, so long as
+the backfill data and the real-time data do not need to be written to the same time partitions. (If they do, the
+real-time load will take priority.)
+
+## Rollup
+
+Druid is able to summarize 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", where a set of "metrics" are aggregated.
+
+Suppose we have the following raw data, representing total packet/byte counts in particular seconds for traffic between a source and destination. The `srcIP` and `dstIP` fields are dimensions, while `packets` and `bytes` are metrics.
+
+```
+timestamp                 srcIP         dstIP          packets     bytes
+2018-01-01T01:01:35Z      1.1.1.1       2.2.2.2            100      1000
+2018-01-01T01:01:51Z      1.1.1.1       2.2.2.2            200      2000
+2018-01-01T01:01:59Z      1.1.1.1       2.2.2.2            300      3000
+2018-01-01T01:02:14Z      1.1.1.1       2.2.2.2            400      4000
+2018-01-01T01:02:29Z      1.1.1.1       2.2.2.2            500      5000
+2018-01-01T01:03:29Z      1.1.1.1       2.2.2.2            600      6000
+2018-01-02T21:33:14Z      7.7.7.7       8.8.8.8            100      1000
+2018-01-02T21:33:45Z      7.7.7.7       8.8.8.8            200      2000
+2018-01-02T21:35:45Z      7.7.7.7       8.8.8.8            300      3000
+```
+
+If we ingest this data into Druid with a `queryGranularity` of `minute` (which will floor timestamps to minutes), the roll-up operation is equivalent to the following pseudocode:
+
+```
+GROUP BY TRUNCATE(timestamp, MINUTE), srcIP, dstIP :: SUM(packets), SUM(bytes)
+```
+
+After the data above is aggregated during roll-up, the following rows will be ingested:
+
+```
+timestamp                 srcIP         dstIP          packets     bytes
+2018-01-01T01:01:00Z      1.1.1.1       2.2.2.2            600      6000
+2018-01-01T01:02:00Z      1.1.1.1       2.2.2.2            900      9000
+2018-01-01T01:03:00Z      1.1.1.1       2.2.2.2            600      6000
+2018-01-02T21:33:00Z      7.7.7.7       8.8.8.8            300      3000
+2018-01-02T21:35:00Z      7.7.7.7       8.8.8.8            300      3000
+```
+
+Druid can roll up data as it is ingested to minimize the amount of raw data that needs to be stored.
+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).
+This storage reduction does come at a cost: as we roll up data, we lose the ability to query individual events. 
+
+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.
+
+The following links may be helpful in further understanding dimensions and metrics:
+* https://en.wikipedia.org/wiki/Dimension_(data_warehouse)
+* https://en.wikipedia.org/wiki/Measure_(data_warehouse))
+
+### Roll-up modes
+
+Druid supports two roll-up modes, i.e., _perfect roll-up_ and _best-effort roll-up_. In the perfect roll-up mode, Druid guarantees that input data are perfectly aggregated at ingestion time. Meanwhile, in the best-effort roll-up, input data might not be perfectly aggregated and thus there can be multiple segments holding the rows which should belong to the same segment with the perfect roll-up since they have the same dimension value and their timestamps fall into the same interval.
+
+The perfect roll-up mode encompasses an additional preprocessing step to determine intervals and shardSpecs before actual data ingestion if they are not specified in the ingestionSpec. This preprocessing step usually scans the entire input data which might increase the ingestion time. The [Hadoop indexing task](../ingestion/hadoop.html) always runs with this perfect roll-up mode.
+
+On the contrary, the best-effort roll-up mode doesn't require any preprocessing step, but the size of ingested data might be larger than that of the perfect roll-up. All types of [streaming indexing (e.g., kafka indexing service)](../ingestion/stream-ingestion.html) run with this mode.
+
+Finally, the [native index task](../ingestion/native_tasks.html) supports both modes and you can choose either one which fits to your application.
+
+## Data maintenance
+
+### Inserts and overwrites
+
+Druid can insert new data to an existing datasource by appending new segments to existing segment sets. It can also add new data by merging an existing set of segments with new data and overwriting the original set. 
+
+Druid does not support single-record updates by primary key.
+
+Updates are described further at [update existing data](../ingestion/update-existing-data.html).
+
+### Compaction
+
+Compaction is a type of overwrite operation, which reads an existing set of segments, combines them into a new set with larger but fewer segments, and overwrites the original set with the new compacted set, without changing the data that is stored.
+
+For performance reasons, it is sometimes beneficial to compact a set of segments into a set of larger but fewer segments, as there is some per-segment processing and memory overhead in both the ingestion and querying paths.
+
+For compaction documentation, please see [tasks](../ingestion/tasks.html).
+
+### Retention and Tiering
+
+Druid supports retention rules, which are used to define intervals of time where data should be preserved, and intervals where data should be discarded.
+
+Druid also supports separating historical nodes into tiers, and the retention rules can be configured to assign data for specific intervals to specific tiers.
+
+These features are useful for performance/cost management; a common use case is separating historical nodes into a "hot" tier and a "cold" tier.
+
+For more information, please see [Load rules](../operations/rule-configuration.html).
+
+### Deletes
+
+Druid supports permanent deletion of segments that are in an "unused" state (see the [Segment states](#segment-states) section above).
+
+The Kill Task deletes unused segments within a specified interval from metadata storage and deep storage.
+
+For more information, please see [Kill Task](../ingestion/tasks.html#kill-task).
\ No newline at end of file
diff --git a/docs/content/toc.md b/docs/content/toc.md
index 6af5707..cfe8745 100644
--- a/docs/content/toc.md
+++ b/docs/content/toc.md
@@ -3,7 +3,14 @@ layout: toc
 ---
 
 ## Getting Started
-  * [Concepts](/docs/VERSION/design/)
+  * [Design](/docs/VERSION/design/index.html)
+    * [What is Druid?](/docs/VERSION/design/index.html#what-is-druid)
+    * [When should I use Druid](/docs/VERSION/design/index.html#when-to-use-druid)
+    * [Architecture](/docs/VERSION/design/index.html#architecture)
+    * [Datasources & Segments](/docs/VERSION/design/index.html#datasources-and-segments)
+    * [Query processing](/docs/VERSION/design/index.html#query-processing)
+    * [External dependencies](/docs/VERSION/design/index.html#external-dependencies)
+    * [Ingestion overview](/docs/VERSION/ingestion/overview.html)
   * [Quickstart](/docs/VERSION/tutorials/quickstart.html)
   * [Loading Data](/docs/VERSION/tutorials/ingestion.html)
     * [Loading from Files](/docs/VERSION/tutorials/tutorial-batch.html)
@@ -12,6 +19,7 @@ layout: toc
   * [Clustering](/docs/VERSION/tutorials/cluster.html)
 
 ## Data Ingestion
+  * [Ingestion overview](/docs/VERSION/ingestion/overview.html)
   * [Data Formats](/docs/VERSION/ingestion/data-formats.html)
   * [Data Schema](/docs/VERSION/ingestion/index.html)
   * [Schema Design](/docs/VERSION/ingestion/schema-design.html)
diff --git a/docs/img/druid-architecture.png b/docs/img/druid-architecture.png
new file mode 100644
index 0000000..cac289e
Binary files /dev/null and b/docs/img/druid-architecture.png differ
diff --git a/docs/img/druid-timeline.png b/docs/img/druid-timeline.png
new file mode 100644
index 0000000..fdeaf8c
Binary files /dev/null and b/docs/img/druid-timeline.png differ


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org