You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by gi...@apache.org on 2019/01/31 03:41:13 UTC

[incubator-druid] branch master updated: Add master/data/query server concepts to docs/packaging (#6916)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 8213787  Add master/data/query server concepts to docs/packaging (#6916)
8213787 is described below

commit 82137874ea48a5dd2f6e2b6283da634ed8417c43
Author: Jonathan Wei <jo...@users.noreply.github.com>
AuthorDate: Wed Jan 30 19:41:07 2019 -0800

    Add master/data/query server concepts to docs/packaging (#6916)
    
    * Add master/data/query server concepts to docs/packaging
    
    * PR comments
    
    * TOC and markdown fix
    
    * Update image legend
    
    * PR comment
    
    * More PR comments
---
 docs/content/comparisons/druid-vs-redshift.md      |   2 +-
 docs/content/configuration/index.md                | 532 +++++++++++----------
 docs/content/configuration/realtime.md             |   2 +-
 .../content/dependencies/cassandra-deep-storage.md |   4 +-
 docs/content/dependencies/metadata-storage.md      |   3 +-
 docs/content/dependencies/zookeeper.md             |   6 +-
 docs/content/design/auth.md                        |   2 +-
 docs/content/design/broker.md                      |   6 +-
 docs/content/design/coordinator.md                 |  40 +-
 docs/content/design/historical.md                  |  14 +-
 docs/content/design/index.md                       | 105 ++--
 docs/content/design/indexing-service.md            |   4 +-
 docs/content/design/middlemanager.md               |   4 +-
 docs/content/design/overlord.md                    |  14 +-
 docs/content/design/peons.md                       |   2 +-
 docs/content/design/processes.md                   | 129 +++++
 .../extensions-contrib/materialized-view.md        |   4 +-
 .../extensions-core/druid-basic-security.md        |  18 +-
 .../development/extensions-core/druid-kerberos.md  |   4 +-
 .../development/extensions-core/kafka-ingestion.md |  16 +-
 .../extensions-core/kinesis-ingestion.md           |  16 +-
 .../extensions-core/lookups-cached-global.md       |   8 +-
 docs/content/development/extensions-core/s3.md     |   2 +-
 docs/content/development/javascript.md             |   2 +-
 docs/content/development/modules.md                |   8 +-
 docs/content/development/overview.md               |   2 +-
 docs/content/development/router.md                 |  56 +--
 docs/content/ingestion/faq.md                      |  10 +-
 docs/content/ingestion/hadoop.md                   |   4 +-
 docs/content/ingestion/index.md                    |   4 +-
 docs/content/ingestion/locking-and-priority.md     |   4 +-
 docs/content/ingestion/native_tasks.md             |   6 +-
 docs/content/ingestion/reports.md                  |   4 +-
 docs/content/ingestion/stream-pull.md              |   2 +-
 docs/content/ingestion/stream-push.md              |   6 +-
 docs/content/ingestion/tasks.md                    |   2 +-
 docs/content/ingestion/update-existing-data.md     |   4 +-
 docs/content/operations/api-reference.md           | 171 ++++---
 docs/content/operations/metrics.md                 |  10 +-
 docs/content/operations/other-hadoop.md            |   4 +-
 docs/content/operations/performance-faq.md         |   6 +-
 docs/content/operations/recommendations.md         |   2 +-
 docs/content/operations/rolling-updates.md         |   4 +-
 docs/content/operations/rule-configuration.md      |  14 +-
 docs/content/operations/segment-optimization.md    |   6 +-
 docs/content/querying/caching.md                   |   2 +-
 docs/content/querying/datasource.md                |   2 +-
 docs/content/querying/dimensionspecs.md            |   4 +-
 docs/content/querying/groupbyquery.md              |  36 +-
 docs/content/querying/lookups.md                   |  32 +-
 docs/content/querying/multitenancy.md              |   6 +-
 docs/content/querying/query-context.md             |  10 +-
 docs/content/querying/querying.md                  |   4 +-
 docs/content/querying/scan-query.md                |   2 +-
 docs/content/querying/searchquery.md               |   4 +-
 docs/content/querying/segmentmetadataquery.md      |   4 +-
 docs/content/querying/sql.md                       |  50 +-
 docs/content/querying/topnquery.md                 |   2 +-
 docs/content/toc.md                                |  11 +-
 docs/content/tutorials/cluster.md                  |  79 +--
 docs/content/tutorials/tutorial-batch.md           |  10 +-
 docs/content/tutorials/tutorial-compaction.md      |   6 +-
 docs/content/tutorials/tutorial-delete-data.md     |   2 +-
 docs/content/tutorials/tutorial-query.md           |   4 +-
 docs/img/druid-architecture.png                    | Bin 151681 -> 207086 bytes
 65 files changed, 853 insertions(+), 685 deletions(-)

diff --git a/docs/content/comparisons/druid-vs-redshift.md b/docs/content/comparisons/druid-vs-redshift.md
index de74a69..86e2076 100644
--- a/docs/content/comparisons/druid-vs-redshift.md
+++ b/docs/content/comparisons/druid-vs-redshift.md
@@ -42,7 +42,7 @@ Druid’s write semantics are not as fluid and does not support full joins (we s
 
 ### Data distribution model
 
-Druid’s data distribution is segment-based and leverages a highly available "deep" storage such as S3 or HDFS. Scaling up (or down) does not require massive copy actions or downtime; in fact, losing any number of historical nodes does not result in data loss because new historical nodes can always be brought up by reading data from "deep" storage.
+Druid’s data distribution is segment-based and leverages a highly available "deep" storage such as S3 or HDFS. Scaling up (or down) does not require massive copy actions or downtime; in fact, losing any number of Historical nodes does not result in data loss because new Historical nodes can always be brought up by reading data from "deep" storage.
 
 To contrast, ParAccel’s data distribution model is hash-based. Expanding the cluster requires re-hashing the data across the nodes, making it difficult to perform without taking downtime. Amazon’s Redshift works around this issue with a multi-step process:
 
diff --git a/docs/content/configuration/index.md b/docs/content/configuration/index.md
index 0b114a4..990f9ce 100644
--- a/docs/content/configuration/index.md
+++ b/docs/content/configuration/index.md
@@ -43,44 +43,48 @@ This page documents all of the configuration properties for each Druid service t
     * [Metadata Storage](#metadata-storage)
     * [Deep Storage](#deep-storage)
     * [Task Logging](#task-logging)
-    * [Indexing Service Discovery](#indexing-service-discovery)
+    * [Overlord Discovery](#overlord-discovery)
     * [Coordinator Discovery](#coordinator-discovery)
     * [Announcing Segments](#announcing-segments)
     * [JavaScript](#javascript)
     * [Double Column Storage](#double-column-storage)
-  * [Coordinator](#coordinator)
-    * [Static Configuration](#static-configuration)
-        * [Node Config](#coordinator-node-config)
-        * [Coordinator Operation](#coordinator-operation)
-        * [Segment Management](#segment-management)
-        * [Metadata Retrieval](#metadata-retrieval)
-    * [Dynamic Configuration](#dynamic-configuration)
-        * [Lookups](#lookups-dynamic-configuration)
-        * [Compaction](#compaction-dynamic-configuration)
-  * [Overlord](#overlord)
-    * [Node Config](#overlord-node-config)
-    * [Static Configuration](#overlord-static-configuration)
-    * [Dynamic Configuration](#overlord-dynamic-configuration)
-        * [Worker Select Strategy](#worker-select-strategy)
-        * [Autoscaler](#autoscaler)
-  * [MiddleManager & Peons](#middlemanager-and-peons)
-    * [Node Config](#middlemanager-node-config)
-    * [MiddleManager Configuration](#middlemanager-configuration)
-    * [Peon Processing](#peon-processing)
-    * [Peon Query Configuration](#peon-query-configuration)
-    * [Caching](#peon-caching)
-    * [Additional Peon Configuration](#additional-peon-configuration)
-  * [Broker](#broker)
-    * [Node Config](#broker-node-configs)
-    * [Query Configuration](#broker-query-configuration)
-    * [SQL](#sql)
-    * [Caching](#broker-caching)
-    * [Segment Discovery](#segment-discovery)
-  * [Historical](#historical)
-    * [Node Configuration](#historical-node-config)
-    * [General Configuration](#historical-general-configuration)
-    * [Query Configs](#historical-query-configs)
-    * [Caching](#historical-caching)
+  * [Master Server](#master-server)
+    * [Coordinator](#coordinator)
+        * [Static Configuration](#static-configuration)
+            * [Node Config](#coordinator-node-config)
+            * [Coordinator Operation](#coordinator-operation)
+            * [Segment Management](#segment-management)
+            * [Metadata Retrieval](#metadata-retrieval)
+        * [Dynamic Configuration](#dynamic-configuration)
+            * [Lookups](#lookups-dynamic-configuration)
+            * [Compaction](#compaction-dynamic-configuration)
+    * [Overlord](#overlord)
+        * [Static Configuration](#overlord-static-configuration)
+            * [Node Config](#overlord-node-config)
+            * [Overlord Operations](#overlord-operations)
+        * [Dynamic Configuration](#overlord-dynamic-configuration)
+            * [Worker Select Strategy](#worker-select-strategy)
+            * [Autoscaler](#autoscaler)
+  * [Data Server](#data-server)
+    * [MiddleManager & Peons](#middlemanager-and-peons)
+        * [Node Config](#middlemanager-node-config)
+        * [MiddleManager Configuration](#middlemanager-configuration)
+        * [Peon Processing](#peon-processing)
+        * [Peon Query Configuration](#peon-query-configuration)
+        * [Caching](#peon-caching)
+        * [Additional Peon Configuration](#additional-peon-configuration)
+    * [Historical](#historical)
+        * [Node Configuration](#historical-node-config)
+        * [General Configuration](#historical-general-configuration)
+        * [Query Configs](#historical-query-configs)
+        * [Caching](#historical-caching)
+  * [Query Server](#query-server)
+    * [Broker](#broker)
+        * [Node Config](#broker-node-configs)
+        * [Query Configuration](#broker-query-configuration)
+        * [SQL](#sql)
+        * [Caching](#broker-caching)
+        * [Segment Discovery](#segment-discovery)
   * [Caching](#cache-configuration)
   * [General Query Configuration](#general-query-configuration)
   * [Realtime nodes (Deprecated)](#realtime-nodes)
@@ -118,7 +122,7 @@ conf/tranquility:
 kafka.json  server.json
 ```
 
-Each directory has a `runtime.properties` file containing configuration properties for the specific Druid service correponding to the directory (e.g., `historical`).
+Each directory has a `runtime.properties` file containing configuration properties for the specific Druid process correponding to the directory (e.g., `historical`).
 
 The `jvm.config` files contain JVM flags such as heap sizing properties for each service.
 
@@ -185,8 +189,8 @@ Druid interacts with ZK through a set of standard path configurations. We recomm
 |`druid.zk.paths.propertiesPath`|Zookeeper properties path.|`${druid.zk.paths.base}/properties`|
 |`druid.zk.paths.announcementsPath`|Druid node announcement path.|`${druid.zk.paths.base}/announcements`|
 |`druid.zk.paths.liveSegmentsPath`|Current path for where Druid nodes announce their segments.|`${druid.zk.paths.base}/segments`|
-|`druid.zk.paths.loadQueuePath`|Entries here cause historical nodes to load and drop segments.|`${druid.zk.paths.base}/loadQueue`|
-|`druid.zk.paths.coordinatorPath`|Used by the coordinator for leader election.|`${druid.zk.paths.base}/coordinator`|
+|`druid.zk.paths.loadQueuePath`|Entries here cause Historical nodes to load and drop segments.|`${druid.zk.paths.base}/loadQueue`|
+|`druid.zk.paths.coordinatorPath`|Used by the Coordinator for leader election.|`${druid.zk.paths.base}/coordinator`|
 |`druid.zk.paths.servedSegmentsPath`|@Deprecated. Legacy path for where Druid nodes announce their segments.|`${druid.zk.paths.base}/servedSegments`|
 
 The indexing service also uses its own set of paths. These configs can be included in the common configuration.
@@ -195,7 +199,7 @@ The indexing service also uses its own set of paths. These configs can be includ
 |--------|-----------|-------|
 |`druid.zk.paths.indexer.base`|Base zookeeper path for |`${druid.zk.paths.base}/indexer`|
 |`druid.zk.paths.indexer.announcementsPath`|Middle managers announce themselves here.|`${druid.zk.paths.indexer.base}/announcements`|
-|`druid.zk.paths.indexer.tasksPath`|Used to assign tasks to middle managers.|`${druid.zk.paths.indexer.base}/tasks`|
+|`druid.zk.paths.indexer.tasksPath`|Used to assign tasks to MiddleManagers.|`${druid.zk.paths.indexer.base}/tasks`|
 |`druid.zk.paths.indexer.statusPath`|Parent path for announcement of task statuses.|`${druid.zk.paths.indexer.base}/status`|
 
 If `druid.zk.paths.base` and `druid.zk.paths.indexer.base` are both set, and none of the other `druid.zk.paths.*` or `druid.zk.paths.indexer.*` values are set, then the other properties will be evaluated relative to their respective `base`.
@@ -488,7 +492,7 @@ To use graphite as emitter set `druid.emitter=graphite`. For configuration detai
 
 ### Metadata Storage
 
-These properties specify the jdbc connection and other configuration around the metadata storage. The only processes that connect to the metadata storage with these properties are the [Coordinator](../design/coordinator.html), [Indexing service](../design/indexing-service.html) and [Realtime Nodes](../design/realtime.html).
+These properties specify the jdbc connection and other configuration around the metadata storage. The only processes that connect to the metadata storage with these properties are the [Coordinator](../design/coordinator.html), [Overlord](../design/overlord.html) and [Realtime Nodes](../design/realtime.html).
 
 |Property|Description|Default|
 |--------|-----------|-------|
@@ -540,7 +544,7 @@ This deep storage is used to interface with Amazon's S3. Note that the `druid-s3
 |`druid.storage.bucket`|S3 bucket name.|none|
 |`druid.storage.baseKey`|S3 object key prefix for storage.|none|
 |`druid.storage.disableAcl`|Boolean flag for ACL.|false|
-|`druid.storage.archiveBucket`|S3 bucket name for archiving when running the indexing-service *archive task*.|none|
+|`druid.storage.archiveBucket`|S3 bucket name for archiving when running the *archive task*.|none|
 |`druid.storage.archiveBaseKey`|S3 object key prefix for archiving.|none|
 |`druid.storage.useS3aSchema`|If true, use the "s3a" filesystem when using Hadoop-based ingestion. If false, the "s3n" filesystem will be used. Only affects Hadoop-based ingestion.|false|
 
@@ -575,9 +579,9 @@ Caution: Automatic log file deletion typically works based on log file modificat
 
 |Property|Description|Default|
 |--------|-----------|-------|
-|`druid.indexer.logs.kill.enabled`|Boolean value for whether to enable deletion of old task logs. If set to true, overlord will submit kill tasks periodically based on `druid.indexer.logs.kill.delay` specified, which will delete task logs from the log directory as well as tasks and tasklogs table entries in metadata storage except for tasks created in the last `druid.indexer.logs.kill.durationToRetain` period. |false|
+|`druid.indexer.logs.kill.enabled`|Boolean value for whether to enable deletion of old task logs. If set to true, Overlord will submit kill tasks periodically based on `druid.indexer.logs.kill.delay` specified, which will delete task logs from the log directory as well as tasks and tasklogs table entries in metadata storage except for tasks created in the last `druid.indexer.logs.kill.durationToRetain` period. |false|
 |`druid.indexer.logs.kill.durationToRetain`| Required if kill is enabled. In milliseconds, task logs and entries in task-related metadata storage tables to be retained created in last x milliseconds. |None|
-|`druid.indexer.logs.kill.initialDelay`| Optional. Number of milliseconds after overlord start when first auto kill is run. |random value less than 300000 (5 mins)|
+|`druid.indexer.logs.kill.initialDelay`| Optional. Number of milliseconds after Overlord start when first auto kill is run. |random value less than 300000 (5 mins)|
 |`druid.indexer.logs.kill.delay`|Optional. Number of milliseconds of delay between successive executions of auto kill run. |21600000 (6 hours)|
 
 #### File Task Logs
@@ -625,13 +629,13 @@ Store task logs in HDFS. Note that the `druid-hdfs-storage` extension must be lo
 |--------|-----------|-------|
 |`druid.indexer.logs.directory`|The directory to store logs.|none|
 
-### Indexing Service Discovery
+### Overlord Discovery
 
-This config is used to find the [Indexing Service](../design/indexing-service.html) using Curator service discovery. Only required if you are actually running an indexing service.
+This config is used to find the [Overlord](../design/overlord.html) using Curator service discovery. Only required if you are actually running an Overlord.
 
 |Property|Description|Default|
 |--------|-----------|-------|
-|`druid.selectors.indexing.serviceName`|The druid.service name of the indexing service Overlord node. To start the Overlord with a different name, set it with this property. |druid/overlord|
+|`druid.selectors.indexing.serviceName`|The druid.service name of the Overlord node. To start the Overlord with a different name, set it with this property. |druid/overlord|
 
 
 ### Coordinator Discovery
@@ -640,7 +644,7 @@ This config is used to find the [Coordinator](../design/coordinator.html) using
 
 |Property|Description|Default|
 |--------|-----------|-------|
-|`druid.selectors.coordinator.serviceName`|The druid.service name of the coordinator node. To start the Coordinator with a different name, set it with this property. |druid/coordinator|
+|`druid.selectors.coordinator.serviceName`|The druid.service name of the Coordinator node. To start the Coordinator with a different name, set it with this property. |druid/coordinator|
 
 
 ### Announcing Segments
@@ -655,8 +659,8 @@ In current Druid, multiple data segments may be announced under the same Znode.
 |--------|-----------|-------|
 |`druid.announcer.segmentsPerNode`|Each Znode contains info for up to this many segments.|50|
 |`druid.announcer.maxBytesPerNode`|Max byte size for Znode.|524288|
-|`druid.announcer.skipDimensionsAndMetrics`|Skip Dimensions and Metrics list from segment announcements. NOTE: Enabling this will also remove the dimensions and metrics list from coordinator and broker endpoints.|false|
-|`druid.announcer.skipLoadSpec`|Skip segment LoadSpec from segment announcements. NOTE: Enabling this will also remove the loadspec from coordinator and broker endpoints.|false|
+|`druid.announcer.skipDimensionsAndMetrics`|Skip Dimensions and Metrics list from segment announcements. NOTE: Enabling this will also remove the dimensions and metrics list from Coordinator and Broker endpoints.|false|
+|`druid.announcer.skipLoadSpec`|Skip segment LoadSpec from segment announcements. NOTE: Enabling this will also remove the loadspec from Coordinator and Broker endpoints.|false|
 
 ### JavaScript
 
@@ -685,15 +689,19 @@ Support for 64-bit floating point columns was released in Druid 0.11.0, so if yo
 |--------|-----------|-------|
 |`druid.indexing.doubleStorage`|Set to "float" to use 32-bit double representation for double columns.|double|
 
-## Coordinator
+## Master Server
+
+This section contains the configuration options for the processes that reside on Master servers (Coordinators and Overlords) in the suggested [three-server configuration](../design/processes.html#server-types).
+
+### Coordinator
 
 For general Coordinator Node information, see [here](../design/coordinator.html).
 
-### Static Configuration
+#### Static Configuration
 
-These coordinator static configurations can be defined in the `coordinator/runtime.properties` file.
+These Coordinator static configurations can be defined in the `coordinator/runtime.properties` file.
 
-#### Coordinator Node Config
+##### Coordinator Node Config
 
 |Property|Description|Default|
 |--------|-----------|-------|
@@ -703,51 +711,51 @@ These coordinator static configurations can be defined in the `coordinator/runti
 |`druid.tlsPort`|TLS port for HTTPS connector, if [druid.enableTlsPort](../operations/tls-support.html) is set then this config will be used. If `druid.host` contains port then that port will be ignored. This should be a non-negative Integer.|8281|
 |`druid.service`|The name of the service. This is used as a dimension when emitting metrics and alerts to differentiate between the various services|druid/coordinator|
 
-#### Coordinator Operation
+##### Coordinator Operation
 
 |Property|Description|Default|
 |--------|-----------|-------|
-|`druid.coordinator.period`|The run period for the coordinator. The coordinator’s operates by maintaining the current state of the world in memory and periodically looking at the set of segments available and segments being served to make decisions about whether any changes need to be made to the data topology. This property sets the delay between each of these runs.|PT60S|
+|`druid.coordinator.period`|The run period for the Coordinator. The Coordinator’s operates by maintaining the current state of the world in memory and periodically looking at the set of segments available and segments being served to make decisions about whether any changes need to be made to the data topology. This property sets the delay between each of these runs.|PT60S|
 |`druid.coordinator.period.indexingPeriod`|How often to send compact/merge/conversion tasks to the indexing service. It's recommended to be longer than `druid.manager.segments.pollDuration`|PT1800S (30 mins)|
 |`druid.coordinator.startDelay`|The operation of the Coordinator works on the assumption that it has an up-to-date view of the state of the world when it runs, the current ZK interaction code, however, is written in a way that doesn’t allow the Coordinator to know for a fact that it’s done loading the current state of the world. This delay is a hack to give it enough time to believe that it has all the data.|PT300S|
-|`druid.coordinator.merge.on`|Boolean flag for whether or not the coordinator should try and merge small segments into a more optimal segment size.|false|
-|`druid.coordinator.load.timeout`|The timeout duration for when the coordinator assigns a segment to a historical node.|PT15M|
-|`druid.coordinator.kill.pendingSegments.on`|Boolean flag for whether or not the coordinator clean up old entries in the `pendingSegments` table of metadata store. If set to true, coordinator will check the created time of most recently complete task. If it doesn't exist, it finds the created time of the earlist running/pending/waiting tasks. Once the created time is found, then for all dataSources not in the `killPendingSegmentsSkipList` (see [Dynamic configuration](#dynamic-configurati [...]
-|`druid.coordinator.kill.on`|Boolean flag for whether or not the coordinator should submit kill task for unused segments, that is, hard delete them from metadata store and deep storage. If set to true, then for all whitelisted dataSources (or optionally all), coordinator will submit tasks periodically based on `period` specified. These kill tasks will delete all segments except for the last `durationToRetain` period. Whitelist or All can be set via dynamic configuration `killAllDataSourc [...]
+|`druid.coordinator.merge.on`|Boolean flag for whether or not the Coordinator should try and merge small segments into a more optimal segment size.|false|
+|`druid.coordinator.load.timeout`|The timeout duration for when the Coordinator assigns a segment to a Historical node.|PT15M|
+|`druid.coordinator.kill.pendingSegments.on`|Boolean flag for whether or not the Coordinator clean up old entries in the `pendingSegments` table of metadata store. If set to true, Coordinator will check the created time of most recently complete task. If it doesn't exist, it finds the created time of the earlist running/pending/waiting tasks. Once the created time is found, then for all dataSources not in the `killPendingSegmentsSkipList` (see [Dynamic configuration](#dynamic-configurati [...]
+|`druid.coordinator.kill.on`|Boolean flag for whether or not the Coordinator should submit kill task for unused segments, that is, hard delete them from metadata store and deep storage. If set to true, then for all whitelisted dataSources (or optionally all), Coordinator will submit tasks periodically based on `period` specified. These kill tasks will delete all segments except for the last `durationToRetain` period. Whitelist or All can be set via dynamic configuration `killAllDataSourc [...]
 |`druid.coordinator.kill.period`|How often to send kill tasks to the indexing service. Value must be greater than `druid.coordinator.period.indexingPeriod`. Only applies if kill is turned on.|P1D (1 Day)|
 |`druid.coordinator.kill.durationToRetain`| Do not kill segments in last `durationToRetain`, must be greater or equal to 0. Only applies and MUST be specified if kill is turned on. Note that default value is invalid.|PT-1S (-1 seconds)|
 |`druid.coordinator.kill.maxSegments`|Kill at most n segments per kill task submission, must be greater than 0. Only applies and MUST be specified if kill is turned on. Note that default value is invalid.|0|
-|`druid.coordinator.balancer.strategy`|Specify the type of balancing strategy that the coordinator should use to distribute segments among the historicals. `cachingCost` is logically equivalent to `cost` but is more CPU-efficient on large clusters and will replace `cost` in the future versions, users are invited to try it. Use `diskNormalized` to distribute segments among nodes so that the disks fill up uniformly and use `random` to randomly pick nodes to distribute segments.|`cost`|
+|`druid.coordinator.balancer.strategy`|Specify the type of balancing strategy that the Coordinator should use to distribute segments among the Historicals. `cachingCost` is logically equivalent to `cost` but is more CPU-efficient on large clusters and will replace `cost` in the future versions, users are invited to try it. Use `diskNormalized` to distribute segments among nodes so that the disks fill up uniformly and use `random` to randomly pick nodes to distribute segments.|`cost`|
 |`druid.coordinator.loadqueuepeon.repeatDelay`|The start and repeat delay for the loadqueuepeon , which manages the load and drop of segments.|PT0.050S (50 ms)|
-|`druid.coordinator.asOverlord.enabled`|Boolean value for whether this coordinator node should act like an overlord as well. This configuration allows users to simplify a druid cluster by not having to deploy any standalone overlord nodes. If set to true, then overlord console is available at `http://coordinator-host:port/console.html` and be sure to set `druid.coordinator.asOverlord.overlordService` also. See next.|false|
+|`druid.coordinator.asOverlord.enabled`|Boolean value for whether this Coordinator node should act like an Overlord as well. This configuration allows users to simplify a druid cluster by not having to deploy any standalone Overlord nodes. If set to true, then Overlord console is available at `http://coordinator-host:port/console.html` and be sure to set `druid.coordinator.asOverlord.overlordService` also. See next.|false|
 |`druid.coordinator.asOverlord.overlordService`| Required, if `druid.coordinator.asOverlord.enabled` is `true`. This must be same value as `druid.service` on standalone Overlord nodes and `druid.selectors.indexing.serviceName` on Middle Managers.|NULL|
 
-#### Segment Management
+##### Segment Management
 |Property|Possible Values|Description|Default|
 |--------|---------------|-----------|-------|
 |`druid.serverview.type`|batch or http|Segment discovery method to use. "http" enables discovering segments using HTTP instead of zookeeper.|batch|
-|`druid.coordinator.loadqueuepeon.type`|curator or http|Whether to use "http" or "curator" implementation to assign segment loads/drops to historical|curator|
+|`druid.coordinator.loadqueuepeon.type`|curator or http|Whether to use "http" or "curator" implementation to assign segment loads/drops to Historical|curator|
 
-##### Additional config when "http" loadqueuepeon is used
+###### Additional config when "http" loadqueuepeon is used
 |Property|Description|Default|
 |--------|-----------|-------|
-|`druid.coordinator.loadqueuepeon.http.batchSize`|Number of segment load/drop requests to batch in one HTTP request. Note that it must be smaller than `druid.segmentCache.numLoadingThreads` config on historical node.|1|
+|`druid.coordinator.loadqueuepeon.http.batchSize`|Number of segment load/drop requests to batch in one HTTP request. Note that it must be smaller than `druid.segmentCache.numLoadingThreads` config on Historical node.|1|
 
-#### Metadata Retrieval
+##### Metadata Retrieval
 
 |Property|Description|Default|
 |--------|-----------|-------|
 |`druid.manager.config.pollDuration`|How often the manager polls the config table for updates.|PT1M|
-|`druid.manager.segments.pollDuration`|The duration between polls the Coordinator does for updates to the set of active segments. Generally defines the amount of lag time it can take for the coordinator to notice new segments.|PT1M|
-|`druid.manager.rules.pollDuration`|The duration between polls the Coordinator does for updates to the set of active rules. Generally defines the amount of lag time it can take for the coordinator to notice rules.|PT1M|
+|`druid.manager.segments.pollDuration`|The duration between polls the Coordinator does for updates to the set of active segments. Generally defines the amount of lag time it can take for the Coordinator to notice new segments.|PT1M|
+|`druid.manager.rules.pollDuration`|The duration between polls the Coordinator does for updates to the set of active rules. Generally defines the amount of lag time it can take for the Coordinator to notice rules.|PT1M|
 |`druid.manager.rules.defaultTier`|The default tier from which default rules will be loaded from.|_default|
 |`druid.manager.rules.alertThreshold`|The duration after a failed poll upon which an alert should be emitted.|PT10M|
 
-### Dynamic Configuration
+#### Dynamic Configuration
 
-The coordinator has dynamic configuration to change certain behaviour on the fly. The coordinator uses a JSON spec object from the Druid [metadata storage](../dependencies/metadata-storage.html) config table. This object is detailed below:
+The Coordinator has dynamic configuration to change certain behaviour on the fly. The Coordinator uses a JSON spec object from the Druid [metadata storage](../dependencies/metadata-storage.html) config table. This object is detailed below:
 
-It is recommended that you use the Coordinator Console to configure these parameters. However, if you need to do it via HTTP, the JSON object can be submitted to the coordinator via a POST request at:
+It is recommended that you use the Coordinator Console to configure these parameters. However, if you need to do it via HTTP, the JSON object can be submitted to the Coordinator via a POST request at:
 
 ```
 http://<COORDINATOR_IP>:<PORT>/druid/coordinator/v1/config
@@ -760,7 +768,7 @@ Optional Header Parameters for auditing the config change can also be specified.
 |`X-Druid-Author`| author making the config change|""|
 |`X-Druid-Comment`| comment describing the change being done|""|
 
-A sample coordinator dynamic config JSON object is shown below:
+A sample Coordinator dynamic config JSON object is shown below:
 
 ```json
 {
@@ -779,11 +787,11 @@ Issuing a GET request at the same URL will return the spec that is currently in
 
 |Property|Description|Default|
 |--------|-----------|-------|
-|`millisToWaitBeforeDeleting`|How long does the coordinator need to be active before it can start removing (marking unused) segments in metadata storage.|900000 (15 mins)|
+|`millisToWaitBeforeDeleting`|How long does the Coordinator need to be active before it can start removing (marking unused) segments in metadata storage.|900000 (15 mins)|
 |`mergeBytesLimit`|The maximum total uncompressed size in bytes of segments to merge.|524288000L|
 |`mergeSegmentsLimit`|The maximum number of segments that can be in a single [append task](../ingestion/tasks.html).|100|
 |`maxSegmentsToMove`|The maximum number of segments that can be moved at any given time.|5|
-|`replicantLifetime`|The maximum number of coordinator runs for a segment to be replicated before we start alerting.|15|
+|`replicantLifetime`|The maximum number of Coordinator runs for a segment to be replicated before we start alerting.|15|
 |`replicationThrottleLimit`|The maximum number of segments that can be replicated at one time.|10|
 |`balancerComputeThreads`|Thread pool size for computing moving cost of segments in segment balancing. Consider increasing this if you have a lot of segments and moving segment starts to get stuck.|1|
 |`emitBalancingStats`|Boolean flag for whether or not we should emit balancing stats. This is an expensive operation.|false|
@@ -792,21 +800,21 @@ Issuing a GET request at the same URL will return the spec that is currently in
 |`killPendingSegmentsSkipList`|List of dataSources for which pendingSegments are _NOT_ cleaned up if property `druid.coordinator.kill.pendingSegments.on` is true. This can be a list of comma-separated dataSources or a JSON array.|none|
 |`maxSegmentsInNodeLoadingQueue`|The maximum number of segments that could be queued for loading to any given server. This parameter could be used to speed up segments loading process, especially if there are "slow" nodes in the cluster (with low loading speed) or if too much segments scheduled to be replicated to some particular node (faster loading could be preferred to better segments distribution). Desired value depends on segments loading speed, acceptable replication time and numbe [...]
 
-To view the audit history of coordinator dynamic config issue a GET request to the URL -
+To view the audit history of Coordinator dynamic config issue a GET request to the URL -
 
 ```
 http://<COORDINATOR_IP>:<PORT>/druid/coordinator/v1/config/history?interval=<interval>
 ```
 
-default value of interval can be specified by setting `druid.audit.manager.auditHistoryMillis` (1 week if not configured) in coordinator runtime.properties
+default value of interval can be specified by setting `druid.audit.manager.auditHistoryMillis` (1 week if not configured) in Coordinator runtime.properties
 
-To view last <n> entries of the audit history of coordinator dynamic config issue a GET request to the URL -
+To view last <n> entries of the audit history of Coordinator dynamic config issue a GET request to the URL -
 
 ```
 http://<COORDINATOR_IP>:<PORT>/druid/coordinator/v1/config/history?count=<n>
 ```
 
-#### Lookups Dynamic Configuration (EXPERIMENTAL)<a id="lookups-dynamic-configuration"></a>
+##### Lookups Dynamic Configuration (EXPERIMENTAL)<a id="lookups-dynamic-configuration"></a>
 These configuration options control the behavior of the Lookup dynamic configuration described in the [lookups page](../querying/lookups.html)
 
 |Property|Description|Default|
@@ -818,9 +826,9 @@ These configuration options control the behavior of the Lookup dynamic configura
 |`druid.manager.lookups.threadPoolSize`|How many nodes can be managed concurrently (concurrent POST and DELETE requests). Requests this limit will wait in a queue until a slot becomes available.|10|
 |`druid.manager.lookups.period`|How many milliseconds between checks for configuration changes|30_000|
 
-#### Compaction Dynamic Configuration
+##### Compaction Dynamic Configuration
 
-Compaction configurations can also be set or updated dynamically without restarting coordinators. For segment compaction,
+Compaction configurations can also be set or updated dynamically without restarting Coordinators. For segment compaction,
 please see [Compacting Segments](../design/coordinator.html#compacting-segments).
 
 A description of the compaction config is:
@@ -853,7 +861,7 @@ If this is the case, the coordinator's automatic compaction might get stuck beca
 This kind of problem may happen especially in Kafka/Kinesis indexing systems which allow late data arrival.
 If you see this problem, it's recommended to set `skipOffsetFromLatest` to some large enough value to avoid such conflicts between compaction tasks and realtime tasks.
 
-##### Compaction TuningConfig
+###### Compaction TuningConfig
 
 |Property|Description|Required|
 |--------|-----------|--------|
@@ -863,15 +871,15 @@ If you see this problem, it's recommended to set `skipOffsetFromLatest` to some
 |`maxPendingPersists`|See [tuningConfig for indexTask](../ingestion/native_tasks.html#tuningconfig)|no (default = 0 (meaning one persist can be running concurrently with ingestion, and none can be queued up))|
 |`pushTimeout`|See [tuningConfig for indexTask](../ingestion/native_tasks.html#tuningconfig)|no (default = 0)|
 
-## Overlord
+### Overlord
 
-For general Overlord Node information, see [here](../design/indexing-service.html).
+For general Overlord Node information, see [here](../design/overlord.html).
 
-### Overlord Static Configuration
+#### Overlord Static Configuration
 
-These overlord static configurations can be defined in the `overlord/runtime.properties` file.
+These Overlord static configurations can be defined in the `overlord/runtime.properties` file.
 
-#### Overlord Node Configs
+##### Overlord Node Configs
 
 |Property|Description|Default|
 |--------|-----------|-------|
@@ -881,30 +889,30 @@ These overlord static configurations can be defined in the `overlord/runtime.pro
 |`druid.tlsPort`|TLS port for HTTPS connector, if [druid.enableTlsPort](../operations/tls-support.html) is set then this config will be used. If `druid.host` contains port then that port will be ignored. This should be a non-negative Integer.|8290|
 |`druid.service`|The name of the service. This is used as a dimension when emitting metrics and alerts to differentiate between the various services|druid/overlord|
 
-#### Overlord Operations
+##### Overlord Operations
 
 |Property|Description|Default|
 |--------|-----------|-------|
 |`druid.indexer.runner.type`|Choices "local" or "remote". Indicates whether tasks should be run locally or in a distributed environment. Experimental task runner "httpRemote" is also available which is same as "remote" but uses HTTP to interact with Middle Manaters instead of Zookeeper.|local|
-|`druid.indexer.storage.type`|Choices are "local" or "metadata". Indicates whether incoming tasks should be stored locally (in heap) or in metadata storage. Storing incoming tasks in metadata storage allows for tasks to be resumed if the overlord should fail.|local|
+|`druid.indexer.storage.type`|Choices are "local" or "metadata". Indicates whether incoming tasks should be stored locally (in heap) or in metadata storage. Storing incoming tasks in metadata storage allows for tasks to be resumed if the Overlord should fail.|local|
 |`druid.indexer.storage.recentlyFinishedThreshold`|A duration of time to store task results.|PT24H|
 |`druid.indexer.queue.maxSize`|Maximum number of active tasks at one time.|Integer.MAX_VALUE|
-|`druid.indexer.queue.startDelay`|Sleep this long before starting overlord queue management. This can be useful to give a cluster time to re-orient itself after e.g. a widespread network issue.|PT1M|
-|`druid.indexer.queue.restartDelay`|Sleep this long when overlord queue management throws an exception before trying again.|PT30S|
-|`druid.indexer.queue.storageSyncRate`|Sync overlord state this often with an underlying task persistence mechanism.|PT1M|
+|`druid.indexer.queue.startDelay`|Sleep this long before starting Overlord queue management. This can be useful to give a cluster time to re-orient itself after e.g. a widespread network issue.|PT1M|
+|`druid.indexer.queue.restartDelay`|Sleep this long when Overlord queue management throws an exception before trying again.|PT30S|
+|`druid.indexer.queue.storageSyncRate`|Sync Overlord state this often with an underlying task persistence mechanism.|PT1M|
 
-The following configs only apply if the overlord is running in remote mode. For a description of local vs. remote mode, please see (../design/indexing-service.html#overlord-node).
+The following configs only apply if the Overlord is running in remote mode. For a description of local vs. remote mode, please see (../design/overlord.html).
 
 |Property|Description|Default|
 |--------|-----------|-------|
-|`druid.indexer.runner.taskAssignmentTimeout`|How long to wait after a task as been assigned to a middle manager before throwing an error.|PT5M|
-|`druid.indexer.runner.minWorkerVersion`|The minimum middle manager version to send tasks to. |"0"|
-|`druid.indexer.runner.compressZnodes`|Indicates whether or not the overlord should expect middle managers to compress Znodes.|true|
+|`druid.indexer.runner.taskAssignmentTimeout`|How long to wait after a task as been assigned to a MiddleManager before throwing an error.|PT5M|
+|`druid.indexer.runner.minWorkerVersion`|The minimum MiddleManager version to send tasks to. |"0"|
+|`druid.indexer.runner.compressZnodes`|Indicates whether or not the Overlord should expect MiddleManagers to compress Znodes.|true|
 |`druid.indexer.runner.maxZnodeBytes`|The maximum size Znode in bytes that can be created in Zookeeper.|524288|
-|`druid.indexer.runner.taskCleanupTimeout`|How long to wait before failing a task after a middle manager is disconnected from Zookeeper.|PT15M|
-|`druid.indexer.runner.taskShutdownLinkTimeout`|How long to wait on a shutdown request to a middle manager before timing out|PT1M|
+|`druid.indexer.runner.taskCleanupTimeout`|How long to wait before failing a task after a MiddleManager is disconnected from Zookeeper.|PT15M|
+|`druid.indexer.runner.taskShutdownLinkTimeout`|How long to wait on a shutdown request to a MiddleManager before timing out|PT1M|
 |`druid.indexer.runner.pendingTasksRunnerNumThreads`|Number of threads to allocate pending-tasks to workers, must be at least 1.|1|
-|`druid.indexer.runner.maxRetriesBeforeBlacklist`|Number of consecutive times the middle manager can fail tasks,  before the worker is blacklisted, must be at least 1|5|
+|`druid.indexer.runner.maxRetriesBeforeBlacklist`|Number of consecutive times the MiddleManager can fail tasks,  before the worker is blacklisted, must be at least 1|5|
 |`druid.indexer.runner.workerBlackListBackoffTime`|How long to wait before a task is whitelisted again. This value should be greater that the value set for taskBlackListCleanupPeriod.|PT15M|
 |`druid.indexer.runner.workerBlackListCleanupPeriod`|A duration after which the cleanup thread will startup to clean blacklisted workers.|PT5M|
 |`druid.indexer.runner.maxPercentageBlacklistWorkers`|The maximum percentage of workers to blacklist, this must be between 0 and 100.|20|
@@ -915,21 +923,21 @@ There are additional configs for autoscaling (if it is enabled):
 |--------|-----------|-------|
 |`druid.indexer.autoscale.strategy`|Choices are "noop" or "ec2". Sets the strategy to run when autoscaling is required.|noop|
 |`druid.indexer.autoscale.doAutoscale`|If set to "true" autoscaling will be enabled.|false|
-|`druid.indexer.autoscale.provisionPeriod`|How often to check whether or not new middle managers should be added.|PT1M|
-|`druid.indexer.autoscale.terminatePeriod`|How often to check when middle managers should be removed.|PT5M|
+|`druid.indexer.autoscale.provisionPeriod`|How often to check whether or not new MiddleManagers should be added.|PT1M|
+|`druid.indexer.autoscale.terminatePeriod`|How often to check when MiddleManagers should be removed.|PT5M|
 |`druid.indexer.autoscale.originTime`|The starting reference timestamp that the terminate period increments upon.|2012-01-01T00:55:00.000Z|
 |`druid.indexer.autoscale.workerIdleTimeout`|How long can a worker be idle (not a run task) before it can be considered for termination.|PT90M|
-|`druid.indexer.autoscale.maxScalingDuration`|How long the overlord will wait around for a middle manager to show up before giving up.|PT15M|
+|`druid.indexer.autoscale.maxScalingDuration`|How long the Overlord will wait around for a MiddleManager to show up before giving up.|PT15M|
 |`druid.indexer.autoscale.numEventsToTrack`|The number of autoscaling related events (node creation and termination) to track.|10|
-|`druid.indexer.autoscale.pendingTaskTimeout`|How long a task can be in "pending" state before the overlord tries to scale up.|PT30S|
+|`druid.indexer.autoscale.pendingTaskTimeout`|How long a task can be in "pending" state before the Overlord tries to scale up.|PT30S|
 |`druid.indexer.autoscale.workerVersion`|If set, will only create nodes of set version during autoscaling. Overrides dynamic configuration. |null|
-|`druid.indexer.autoscale.workerPort`|The port that middle managers will run on.|8080|
+|`druid.indexer.autoscale.workerPort`|The port that MiddleManagers will run on.|8080|
 
-### Overlord Dynamic Configuration
+#### Overlord Dynamic Configuration
 
-The overlord can dynamically change worker behavior.
+The Overlord can dynamically change worker behavior.
 
-The JSON object can be submitted to the overlord via a POST request at:
+The JSON object can be submitted to the Overlord via a POST request at:
 
 ```
 http://<OVERLORD_IP>:<port>/druid/indexer/v1/worker
@@ -984,7 +992,7 @@ Issuing a GET request at the same URL will return the current worker config spec
 
 |Property|Description|Default|
 |--------|-----------|-------|
-|`selectStrategy`|How to assign tasks to middle managers. Choices are `fillCapacity`, `equalDistribution`, and `javascript`.|equalDistribution|
+|`selectStrategy`|How to assign tasks to MiddleManagers. Choices are `fillCapacity`, `equalDistribution`, and `javascript`.|equalDistribution|
 |`autoScaler`|Only used if autoscaling is enabled. See below.|null|
 
 To view the audit history of worker config issue a GET request to the URL -
@@ -993,7 +1001,7 @@ To view the audit history of worker config issue a GET request to the URL -
 http://<OVERLORD_IP>:<port>/druid/indexer/v1/worker/history?interval=<interval>
 ```
 
-default value of interval can be specified by setting `druid.audit.manager.auditHistoryMillis` (1 week if not configured) in overlord runtime.properties.
+default value of interval can be specified by setting `druid.audit.manager.auditHistoryMillis` (1 week if not configured) in Overlord runtime.properties.
 
 To view last <n> entries of the audit history of worker config issue a GET request to the URL -
 
@@ -1001,11 +1009,11 @@ To view last <n> entries of the audit history of worker config issue a GET reque
 http://<OVERLORD_IP>:<port>/druid/indexer/v1/worker/history?count=<n>
 ```
 
-#### Worker Select Strategy
+##### Worker Select Strategy
 
 Worker select strategies control how Druid assigns tasks to middleManagers.
 
-##### Equal Distribution
+###### Equal Distribution
 
 Tasks are assigned to the middleManager with the most available capacity at the time the task begins running. This is
 useful if you want work evenly distributed across your middleManagers.
@@ -1015,7 +1023,7 @@ useful if you want work evenly distributed across your middleManagers.
 |`type`|`equalDistribution`.|required; must be `equalDistribution`|
 |`affinityConfig`|[Affinity config](#affinity) object|null (no affinity)|
 
-##### Fill Capacity
+###### Fill Capacity
 
 Tasks are assigned to the worker with the most currently-running tasks at the time the task begins running. This is
 useful in situations where you are elastically auto-scaling middleManagers, since it will tend to pack some full and
@@ -1029,7 +1037,7 @@ middleManagers up to capacity simultaneously, rather than a single middleManager
 |`type`|`fillCapacity`.|required; must be `fillCapacity`|
 |`affinityConfig`|[Affinity config](#affinity) object|null (no affinity)|
 
-##### Javascript
+###### Javascript<a id="javascript-worker-select-strategy"></a>
 
 Allows defining arbitrary logic for selecting workers to run task using a JavaScript function.
 The function is passed remoteTaskRunnerConfig, map of workerId to available workers and task to be executed and returns the workerId on which the task should be run or null if the task cannot be run.
@@ -1055,7 +1063,7 @@ Example: a function that sends batch_index_task to workers 10.0.0.1 and 10.0.0.2
 JavaScript-based functionality is disabled by default. Please refer to the Druid <a href="../development/javascript.html">JavaScript programming guide</a> for guidelines about using Druid's JavaScript functionality, including instructions on how to enable it.
 </div>
 
-##### Affinity
+###### Affinity
 
 Affinity configs can be provided to the _equalDistribution_ and _fillCapacity_ strategies using the "affinityConfig"
 field. If not provided, the default is to not use affinity at all.
@@ -1065,7 +1073,7 @@ field. If not provided, the default is to not use affinity at all.
 |`affinity`|JSON object mapping a datasource String name to a list of indexing service middleManager host:port String values. Druid doesn't perform DNS resolution, so the 'host' value must match what is configured on the middleManager and what the middleManager announces itself as (examine the Overlord logs to see what your middleManager announces itself as).|{}|
 |`strong`|With weak affinity (the default), tasks for a dataSource may be assigned to other middleManagers if their affinity-mapped middleManagers are not able to run all pending tasks in the queue for that dataSource. With strong affinity, tasks for a dataSource will only ever be assigned to their affinity-mapped middleManagers, and will wait in the pending queue if necessary.|false|
 
-#### Autoscaler
+##### Autoscaler
 
 Amazon's EC2 is currently the only supported autoscaler.
 
@@ -1077,11 +1085,15 @@ Amazon's EC2 is currently the only supported autoscaler.
 |`nodeData`|A JSON object that describes how to launch new nodes.|none; required|
 |`userData`|A JSON object that describes how to configure new nodes. If you have set druid.indexer.autoscale.workerVersion, this must have a versionReplacementString. Otherwise, a versionReplacementString is not necessary.|none; optional|
 
-## MiddleManager and Peons
+## Data Server
+
+This section contains the configuration options for the processes that reside on Data servers (MiddleManagers/Peons and Historicals) in the suggested [three-server configuration](../design/processes.html#server-types).
+
+### MiddleManager and Peons
 
 These MiddleManager and Peon configurations can be defined in the `middleManager/runtime.properties` file.
 
-### MiddleManager Node Config
+#### MiddleManager Node Config
 
 |Property|Description|Default|
 |--------|-----------|-------|
@@ -1091,14 +1103,14 @@ These MiddleManager and Peon configurations can be defined in the `middleManager
 |`druid.tlsPort`|TLS port for HTTPS connector, if [druid.enableTlsPort](../operations/tls-support.html) is set then this config will be used. If `druid.host` contains port then that port will be ignored. This should be a non-negative Integer.|8291|
 |`druid.service`|The name of the service. This is used as a dimension when emitting metrics and alerts to differentiate between the various services|druid/middlemanager|
 
-### MiddleManager Configuration
+#### MiddleManager Configuration
 
-Middle managers pass their configurations down to their child peons. The middle manager requires the following configs:
+Middle managers pass their configurations down to their child peons. The MiddleManager requires the following configs:
 
 |Property|Description|Default|
 |--------|-----------|-------|
 |`druid.indexer.runner.allowedPrefixes`|Whitelist of prefixes for configs that can be passed down to child peons.|"com.metamx", "druid", "org.apache.druid", "user.timezone", "file.encoding", "java.io.tmpdir", "hadoop"|
-|`druid.indexer.runner.compressZnodes`|Indicates whether or not the middle managers should compress Znodes.|true|
+|`druid.indexer.runner.compressZnodes`|Indicates whether or not the MiddleManagers should compress Znodes.|true|
 |`druid.indexer.runner.classpath`|Java classpath for the peon.|System.getProperty("java.class.path")|
 |`druid.indexer.runner.javaCommand`|Command required to execute java.|java|
 |`druid.indexer.runner.javaOpts`|*DEPRECATED* A string of -X Java options to pass to the peon's JVM. Quotable parameters or parameters with spaces are encouraged to use javaOptsArray|""|
@@ -1108,10 +1120,10 @@ Middle managers pass their configurations down to their child peons. The middle
 |`druid.indexer.runner.endPort`|Ending port used for peon processes, should be greater than or equal to `druid.indexer.runner.startPort` and less than 65536.|65535|
 |`druid.indexer.runner.ports`|A json array of integers to specify ports that used for peon processes. If provided and non-empty, ports for peon processes will be chosen from these ports. And `druid.indexer.runner.startPort/druid.indexer.runner.endPort` will be completely ignored.|`[]`|
 |`druid.worker.ip`|The IP of the worker.|localhost|
-|`druid.worker.version`|Version identifier for the middle manager.|0|
-|`druid.worker.capacity`|Maximum number of tasks the middle manager can accept.|Number of available processors - 1|
+|`druid.worker.version`|Version identifier for the MiddleManager.|0|
+|`druid.worker.capacity`|Maximum number of tasks the MiddleManager can accept.|Number of available processors - 1|
 
-### Peon Processing
+#### Peon Processing
 
 Processing properties set on the Middlemanager will be passed through to Peons.
 
@@ -1119,7 +1131,7 @@ Processing properties set on the Middlemanager will be passed through to Peons.
 |--------|-----------|-------|
 |`druid.processing.buffer.sizeBytes`|This specifies a buffer size for the storage of intermediate results. The computation engine in both the Historical and Realtime nodes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed.|auto (max 1GB)|
 |`druid.processing.buffer.poolCacheMaxCount`|processing buffer pool caches the buffers for later use, this is the maximum count cache will grow to. note that pool can create more buffers than it can cache if necessary.|Integer.MAX_VALUE|
-|`druid.processing.formatString`|Realtime and historical nodes use this format string to name their processing threads.|processing-%s|
+|`druid.processing.formatString`|Realtime and Historical nodes use this format string to name their processing threads.|processing-%s|
 |`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. If you are using any queries that require merge buffers (currently, just groupBy v2) then you should have at least two of these.|`max(2, druid.processing.numThreads / 4)`|
 |`druid.processing.numThreads`|The number of processing threads to have available for parallel processing of segments. Our rule of thumb is `num_cores - 1`, which means that even under heavy load there will still be one core available to do background tasks like talking with ZooKeeper and pulling down segments. If only one core is available, this property defaults to the value `1`.|Number of cores - 1 (or 1)|
 |`druid.processing.columnCache.sizeBytes`|Maximum size in bytes for the dimension value lookup cache. Any value greater than `0` enables the cache. It is currently disabled by default. Enabling the lookup cache can significantly improve the performance of aggregators operating on dimension values, such as the JavaScript aggregator, or cardinality aggregator, but can slow things down if the cache hit rate is low (i.e. dimensions with few repeating values). Enabling it may also require add [...]
@@ -1131,11 +1143,11 @@ The amount of direct memory needed by Druid is at least
 ensure at least this amount of direct memory is available by providing `-XX:MaxDirectMemorySize=<VALUE>` in
 `druid.indexer.runner.javaOptsArray` as documented above.
 
-### Peon Query Configuration
+#### Peon Query Configuration
 
 See [general query configuration](#general-query-configuration).
 
-### Peon Caching
+#### Peon Caching
 
 You can optionally configure caching to be enabled on the peons by setting caching configs here.
 
@@ -1148,8 +1160,8 @@ You can optionally configure caching to be enabled on the peons by setting cachi
 See [cache configuration](#cache-configuration) for how to configure cache settings.
 
 
-### Additional Peon Configuration
-Although peons inherit the configurations of their parent middle managers, explicit child peon configs in middle manager can be set by prefixing them with:
+#### Additional Peon Configuration
+Although peons inherit the configurations of their parent MiddleManagers, explicit child peon configs in MiddleManager can be set by prefixing them with:
 
 ```
 druid.indexer.fork.property
@@ -1169,15 +1181,15 @@ Additional peon configs include:
 |`druid.indexer.task.restoreTasksOnRestart`|If true, middleManagers will attempt to stop tasks gracefully on shutdown and restore them on restart.|false|
 |`druid.indexer.server.maxChatRequests`|Maximum number of concurrent requests served by a task's chat handler. Set to 0 to disable limiting.|0|
 
-If the peon is running in remote mode, there must be an overlord up and running. Peons in remote mode can set the following configurations:
+If the peon is running in remote mode, there must be an Overlord up and running. Peons in remote mode can set the following configurations:
 
 |Property|Description|Default|
 |--------|-----------|-------|
-|`druid.peon.taskActionClient.retry.minWait`|The minimum retry time to communicate with overlord.|PT5S|
-|`druid.peon.taskActionClient.retry.maxWait`|The maximum retry time to communicate with overlord.|PT1M|
-|`druid.peon.taskActionClient.retry.maxRetryCount`|The maximum number of retries to communicate with overlord.|60|
+|`druid.peon.taskActionClient.retry.minWait`|The minimum retry time to communicate with Overlord.|PT5S|
+|`druid.peon.taskActionClient.retry.maxWait`|The maximum retry time to communicate with Overlord.|PT1M|
+|`druid.peon.taskActionClient.retry.maxRetryCount`|The maximum number of retries to communicate with Overlord.|60|
 
-#### SegmentWriteOutMediumFactory
+##### SegmentWriteOutMediumFactory
 
 When new segments are created, Druid temporarily stores some pre-processed data in some buffers. Currently two types of
 *medium* exist for those buffers: *temporary files* and *off-heap memory*.
@@ -1200,13 +1212,107 @@ then the value from the configuration below is used:
 |--------|-----------|-------|
 |`druid.peon.defaultSegmentWriteOutMediumFactory.type`|`tmpFile` or `offHeapMemory`, see explanation above|`tmpFile`|
 
-## Broker
+### Historical
+
+For general Historical Node information, see [here](../design/historical.html).
+
+These Historical configurations can be defined in the `historical/runtime.properties` file.
+
+#### Historical Node Configuration
+|Property|Description|Default|
+|--------|-----------|-------|
+|`druid.host`|The host for the current node. This is used to advertise the current processes location as reachable from another node and should generally be specified such that `http://${druid.host}/` could actually talk to this process|InetAddress.getLocalHost().getCanonicalHostName()|
+|`druid.bindOnHost`|Indicating whether the node's internal jetty server bind on `druid.host`. Default is false, which means binding to all interfaces.|false|
+|`druid.plaintextPort`|This is the port to actually listen on; unless port mapping is used, this will be the same port as is on `druid.host`|8083|
+|`druid.tlsPort`|TLS port for HTTPS connector, if [druid.enableTlsPort](../operations/tls-support.html) is set then this config will be used. If `druid.host` contains port then that port will be ignored. This should be a non-negative Integer.|8283|
+|`druid.service`|The name of the service. This is used as a dimension when emitting metrics and alerts to differentiate between the various services|druid/historical|
+
+
+#### Historical General Configuration
+
+|Property|Description|Default|
+|--------|-----------|-------|
+|`druid.server.maxSize`|The maximum number of bytes-worth of segments that the node wants assigned to it. This is not a limit that Historical nodes actually enforces, just a value published to the Coordinator node so it can plan accordingly.|0|
+|`druid.server.tier`| A string to name the distribution tier that the storage node belongs to. Many of the [rules Coordinator nodes use](../operations/rule-configuration.html) to manage segments can be keyed on tiers. |  `_default_tier` |
+|`druid.server.priority`|In a tiered architecture, the priority of the tier, thus allowing control over which nodes are queried. Higher numbers mean higher priority. The default (no priority) works for architecture with no cross replication (tiers that have no data-storage overlap). Data centers typically have equal priority. | 0 |
+
+#### Storing Segments
+
+|Property|Description|Default|
+|--------|-----------|-------|
+|`druid.segmentCache.locations`|Segments assigned to a Historical node are first stored on the local file system (in a disk cache) and then served by the Historical node. These locations define where that local cache resides. This value cannot be NULL or EMPTY. Here is an example `druid.segmentCache.locations=[{"path": "/mnt/druidSegments", "maxSize": 10000, "freeSpacePercent": 1.0}]`. "freeSpacePercent" is optional, if provided then enforces that much of free disk partition space while  [...]
+|`druid.segmentCache.deleteOnRemove`|Delete segment files from cache once a node is no longer serving a segment.|true|
+|`druid.segmentCache.dropSegmentDelayMillis`|How long a node delays before completely dropping segment.|30000 (30 seconds)|
+|`druid.segmentCache.infoDir`|Historical nodes keep track of the segments they are serving so that when the process is restarted they can reload the same segments without waiting for the Coordinator to reassign. This path defines where this metadata is kept. Directory will be created if needed.|${first_location}/info_dir|
+|`druid.segmentCache.announceIntervalMillis`|How frequently to announce segments while segments are loading from cache. Set this value to zero to wait for all segments to be loaded before announcing.|5000 (5 seconds)|
+|`druid.segmentCache.numLoadingThreads`|How many segments to drop or load concurrently from from deep storage.|10|
+|`druid.segmentCache.numBootstrapThreads`|How many segments to load concurrently from local storage at startup.|Same as numLoadingThreads|
+
+In `druid.segmentCache.locations`, *freeSpacePercent* was added because *maxSize* setting is only a theoretical limit and assumes that much space will always be available for storing segments. In case of any druid bug leading to unaccounted segment files left alone on disk or some other process writing stuff to disk, This check can start failing segment loading early before filling up the disk completely and leaving the host usable otherwise.
+
+#### Historical Query Configs
+
+##### Concurrent Requests
+
+Druid uses Jetty to serve HTTP requests.
+
+|Property|Description|Default|
+|--------|-----------|-------|
+|`druid.server.http.numThreads`|Number of threads for HTTP requests.|max(10, (Number of cores * 17) / 16 + 2) + 30|
+|`druid.server.http.queueSize`|Size of the worker queue used by Jetty server to temporarily store incoming client connections. If this value is set and a request is rejected by jetty because queue is full then client would observe request failure with TCP connection being closed immediately with a completely empty response from server.|Unbounded|
+|`druid.server.http.maxIdleTime`|The Jetty max idle time for a connection.|PT5M|
+|`druid.server.http.enableRequestLimit`|If enabled, no requests would be queued in jetty queue and "HTTP 429 Too Many Requests" error response would be sent. |false|
+|`druid.server.http.defaultQueryTimeout`|Query timeout in millis, beyond which unfinished queries will be cancelled|300000|
+|`druid.server.http.gracefulShutdownTimeout`|The maximum amount of time Jetty waits after receiving shutdown signal. After this timeout the threads will be forcefully shutdown. This allows any queries that are executing to complete.|`PT0S` (do not wait)|
+|`druid.server.http.unannouncePropagationDelay`|How long to wait for zookeeper unannouncements to propagate before shutting down Jetty. This is a minimum and `druid.server.http.gracefulShutdownTimeout` does not start counting down until after this period elapses.|`PT0S` (do not wait)|
+|`druid.server.http.maxQueryTimeout`|Maximum allowed value (in milliseconds) for `timeout` parameter. See [query-context](../querying/query-context.html) to know more about `timeout`. Query is rejected if the query context `timeout` is greater than this value. |Long.MAX_VALUE|
+|`druid.server.http.maxRequestHeaderSize`|Maximum size of a request header in bytes. Larger headers consume more memory and can make a server more vulnerable to denial of service attacks.|8 * 1024|
+
+##### Processing
+
+|Property|Description|Default|
+|--------|-----------|-------|
+|`druid.processing.buffer.sizeBytes`|This specifies a buffer size for the storage of intermediate results. The computation engine in both the Historical and Realtime nodes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed.|auto (max 1GB)|
+|`druid.processing.buffer.poolCacheMaxCount`|processing buffer pool caches the buffers for later use, this is the maximum count cache will grow to. note that pool can create more buffers than it can cache if necessary.|Integer.MAX_VALUE|
+|`druid.processing.formatString`|Realtime and Historical nodes use this format string to name their processing threads.|processing-%s|
+|`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. If you are using any queries that require merge buffers (currently, just groupBy v2) then you should have at least two of these.|`max(2, druid.processing.numThreads / 4)`|
+|`druid.processing.numThreads`|The number of processing threads to have available for parallel processing of segments. Our rule of thumb is `num_cores - 1`, which means that even under heavy load there will still be one core available to do background tasks like talking with ZooKeeper and pulling down segments. If only one core is available, this property defaults to the value `1`.|Number of cores - 1 (or 1)|
+|`druid.processing.columnCache.sizeBytes`|Maximum size in bytes for the dimension value lookup cache. Any value greater than `0` enables the cache. It is currently disabled by default. Enabling the lookup cache can significantly improve the performance of aggregators operating on dimension values, such as the JavaScript aggregator, or cardinality aggregator, but can slow things down if the cache hit rate is low (i.e. dimensions with few repeating values). Enabling it may also require add [...]
+|`druid.processing.fifo`|If the processing queue should treat tasks of equal priority in a FIFO manner|`false`|
+|`druid.processing.tmpDir`|Path where temporary files created while processing a query should be stored. If specified, this configuration takes priority over the default `java.io.tmpdir` path.|path represented by `java.io.tmpdir`|
+
+The amount of direct memory needed by Druid is at least
+`druid.processing.buffer.sizeBytes * (druid.processing.numMergeBuffers + druid.processing.numThreads + 1)`. You can
+ensure at least this amount of direct memory is available by providing `-XX:MaxDirectMemorySize=<VALUE>` at the command
+line.
+
+##### Historical Query Configuration
+
+See [general query configuration](#general-query-configuration).
+
+#### Historical Caching
+
+You can optionally only configure caching to be enabled on the Historical by setting caching configs here.
+
+|Property|Possible Values|Description|Default|
+|--------|---------------|-----------|-------|
+|`druid.historical.cache.useCache`|true, false|Enable the cache on the Historical.|false|
+|`druid.historical.cache.populateCache`|true, false|Populate the cache on the Historical.|false|
+|`druid.historical.cache.unCacheable`|All druid query types|All query types to not cache.|["groupBy", "select"]|
+
+See [cache configuration](#cache-configuration) for how to configure cache settings.
+
+## Query Server
+
+This section contains the configuration options for the processes that reside on Query servers (Brokers) in the suggested [three-server configuration](../design/processes.html#server-types).
+
+### Broker
 
-For general Broker Node information, see [here](../design/broker.html).
+For general Broker process information, see [here](../design/broker.html).
 
 These Broker configurations can be defined in the `broker/runtime.properties` file.
 
-### Broker Node Configs
+#### Broker Node Configs
 
 |Property|Description|Default|
 |--------|-----------|-------|
@@ -1216,17 +1322,17 @@ These Broker configurations can be defined in the `broker/runtime.properties` fi
 |`druid.tlsPort`|TLS port for HTTPS connector, if [druid.enableTlsPort](../operations/tls-support.html) is set then this config will be used. If `druid.host` contains port then that port will be ignored. This should be a non-negative Integer.|8282|
 |`druid.service`|The name of the service. This is used as a dimension when emitting metrics and alerts to differentiate between the various services|druid/broker|
 
-### Query Configuration
+#### Query Configuration
 
-#### Query Prioritization
+##### Query Prioritization
 
 |Property|Possible Values|Description|Default|
 |--------|---------------|-----------|-------|
-|`druid.broker.balancer.type`|`random`, `connectionCount`|Determines how the broker balances connections to historical nodes. `random` choose randomly, `connectionCount` picks the node with the fewest number of active connections to|`random`|
+|`druid.broker.balancer.type`|`random`, `connectionCount`|Determines how the broker balances connections to Historical nodes. `random` choose randomly, `connectionCount` picks the node with the fewest number of active connections to|`random`|
 |`druid.broker.select.tier`|`highestPriority`, `lowestPriority`, `custom`|If segments are cross-replicated across tiers in a cluster, you can tell the broker to prefer to select segments in a tier with a certain priority.|`highestPriority`|
 |`druid.broker.select.tier.custom.priorities`|`An array of integer priorities.`|Select servers in tiers with a custom priority list.|None|
 
-#### Server Configuration
+##### Server Configuration
 
 Druid uses Jetty to serve HTTP requests.
 
@@ -1237,26 +1343,26 @@ Druid uses Jetty to serve HTTP requests.
 |`druid.server.http.maxIdleTime`|The Jetty max idle time for a connection.|PT5M|
 |`druid.server.http.enableRequestLimit`|If enabled, no requests would be queued in jetty queue and "HTTP 429 Too Many Requests" error response would be sent. |false|
 |`druid.server.http.defaultQueryTimeout`|Query timeout in millis, beyond which unfinished queries will be cancelled|300000|
-|`druid.server.http.maxScatterGatherBytes`|Maximum number of bytes gathered from data nodes such as historicals and realtime processes to execute a query. Queries that exceed this limit will fail. This is an advance configuration that allows to protect in case broker is under heavy load and not utilizing the data gathered in memory fast enough and leading to OOMs. This limit can be further reduced at query time using `maxScatterGatherBytes` in the context. Note that having large limit is [...]
+|`druid.server.http.maxScatterGatherBytes`|Maximum number of bytes gathered from data nodes such as Historicals and realtime processes to execute a query. Queries that exceed this limit will fail. This is an advance configuration that allows to protect in case Broker is under heavy load and not utilizing the data gathered in memory fast enough and leading to OOMs. This limit can be further reduced at query time using `maxScatterGatherBytes` in the context. Note that having large limit is [...]
 |`druid.server.http.gracefulShutdownTimeout`|The maximum amount of time Jetty waits after receiving shutdown signal. After this timeout the threads will be forcefully shutdown. This allows any queries that are executing to complete.|`PT0S` (do not wait)|
 |`druid.server.http.unannouncePropagationDelay`|How long to wait for zookeeper unannouncements to propagate before shutting down Jetty. This is a minimum and `druid.server.http.gracefulShutdownTimeout` does not start counting down until after this period elapses.|`PT0S` (do not wait)|
 |`druid.server.http.maxQueryTimeout`|Maximum allowed value (in milliseconds) for `timeout` parameter. See [query-context](../querying/query-context.html) to know more about `timeout`. Query is rejected if the query context `timeout` is greater than this value. |Long.MAX_VALUE|
 |`druid.server.http.maxRequestHeaderSize`|Maximum size of a request header in bytes. Larger headers consume more memory and can make a server more vulnerable to denial of service attacks. |8 * 1024|
 
-#### Client Configuration
+##### Client Configuration
 
-Druid Brokers use an HTTP client to communicate with with data servers (historical servers and real-time tasks). This
+Druid Brokers use an HTTP client to communicate with with data servers (Historical servers and real-time tasks). This
 client has the following configuration options.
 
 |Property|Description|Default|
 |--------|-----------|-------|
-|`druid.broker.http.numConnections`|Size of connection pool for the Broker to connect to historical and real-time processes. If there are more queries than this number that all need to speak to the same node, then they will queue up.|20|
-|`druid.broker.http.compressionCodec`|Compression codec the Broker uses to communicate with historical and real-time processes. May be "gzip" or "identity".|gzip|
-|`druid.broker.http.readTimeout`|The timeout for data reads from historical servers and real-time tasks.|PT15M|
+|`druid.broker.http.numConnections`|Size of connection pool for the Broker to connect to Historical and real-time processes. If there are more queries than this number that all need to speak to the same node, then they will queue up.|20|
+|`druid.broker.http.compressionCodec`|Compression codec the Broker uses to communicate with Historical and real-time processes. May be "gzip" or "identity".|gzip|
+|`druid.broker.http.readTimeout`|The timeout for data reads from Historical servers and real-time tasks.|PT15M|
 |`druid.broker.http.unusedConnectionTimeout`|The timeout for idle connections in connection pool. This timeout should be less than `druid.broker.http.readTimeout`. Set this timeout = ~90% of `druid.broker.http.readTimeout`|`PT4M`|
 |`druid.broker.http.maxQueuedBytes`|Maximum number of bytes queued per query before exerting backpressure on the channel to the data server. Similar to `druid.server.http.maxScatterGatherBytes`, except unlike that configuration, this one will trigger backpressure rather than query failure. Zero means disabled. Can be overridden by the ["maxQueuedBytes" query context parameter](../querying/query-context.html).|0 (disabled)|
 
-#### Retry Policy
+##### Retry Policy
 
 Druid broker can optionally retry queries internally for transient errors.
 
@@ -1264,7 +1370,7 @@ Druid broker can optionally retry queries internally for transient errors.
 |--------|-----------|-------|
 |`druid.broker.retryPolicy.numTries`|Number of tries.|1|
 
-#### Processing
+##### Processing
 
 The broker uses processing configs for nested groupBy queries. And, if you use groupBy v1, long-interval queries (of any type) can be broken into shorter interval queries and processed in parallel inside this thread pool. For more details, see "chunkPeriod" in [Query Context](../querying/query-context.html) doc.
 
@@ -1272,7 +1378,7 @@ The broker uses processing configs for nested groupBy queries. And, if you use g
 |--------|-----------|-------|
 |`druid.processing.buffer.sizeBytes`|This specifies a buffer size for the storage of intermediate results. The computation engine in both the Historical and Realtime nodes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed.|auto (max 1GB)|
 |`druid.processing.buffer.poolCacheMaxCount`|processing buffer pool caches the buffers for later use, this is the maximum count cache will grow to. note that pool can create more buffers than it can cache if necessary.|Integer.MAX_VALUE|
-|`druid.processing.formatString`|Realtime and historical nodes use this format string to name their processing threads.|processing-%s|
+|`druid.processing.formatString`|Realtime and Historical nodes use this format string to name their processing threads.|processing-%s|
 |`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. If you are using any queries that require merge buffers (currently, just groupBy v2) then you should have at least two of these.|`max(2, druid.processing.numThreads / 4)`|
 |`druid.processing.numThreads`|The number of processing threads to have available for parallel processing of segments. Our rule of thumb is `num_cores - 1`, which means that even under heavy load there will still be one core available to do background tasks like talking with ZooKeeper and pulling down segments. If only one core is available, this property defaults to the value `1`.|Number of cores - 1 (or 1)|
 |`druid.processing.columnCache.sizeBytes`|Maximum size in bytes for the dimension value lookup cache. Any value greater than `0` enables the cache. It is currently disabled by default. Enabling the lookup cache can significantly improve the performance of aggregators operating on dimension values, such as the JavaScript aggregator, or cardinality aggregator, but can slow things down if the cache hit rate is low (i.e. dimensions with few repeating values). Enabling it may also require add [...]
@@ -1284,13 +1390,13 @@ The amount of direct memory needed by Druid is at least
 ensure at least this amount of direct memory is available by providing `-XX:MaxDirectMemorySize=<VALUE>` at the command
 line.
 
-#### Broker Query Configuration
+##### Broker Query Configuration
 
 See [general query configuration](#general-query-configuration).
 
-### SQL
+#### SQL
 
-The Druid SQL server is configured through the following properties on the broker.
+The Druid SQL server is configured through the following properties on the Broker.
 
 |Property|Description|Default|
 |--------|-----------|-------|
@@ -1309,136 +1415,46 @@ The Druid SQL server is configured through the following properties on the broke
 |`druid.sql.planner.selectThreshold`|Page size threshold for [Select queries](../querying/select-query.html). Select queries for larger resultsets will be issued back-to-back using pagination.|1000|
 |`druid.sql.planner.useApproximateCountDistinct`|Whether to use an approximate cardinalty algorithm for `COUNT(DISTINCT foo)`.|true|
 |`druid.sql.planner.useApproximateTopN`|Whether to use approximate [TopN queries](../querying/topnquery.html) when a SQL query could be expressed as such. If false, exact [GroupBy queries](../querying/groupbyquery.html) will be used instead.|true|
-|`druid.sql.planner.useFallback`|Whether to evaluate operations on the broker when they cannot be expressed as Druid queries. This option is not recommended for production since it can generate unscalable query plans. If false, SQL queries that cannot be translated to Druid queries will fail.|false|
+|`druid.sql.planner.useFallback`|Whether to evaluate operations on the Broker when they cannot be expressed as Druid queries. This option is not recommended for production since it can generate unscalable query plans. If false, SQL queries that cannot be translated to Druid queries will fail.|false|
 |`druid.sql.planner.requireTimeCondition`|Whether to require SQL to have filter conditions on __time column so that all generated native queries will have user specified intervals. If true, all queries wihout filter condition on __time column will fail|false|
 |`druid.sql.planner.sqlTimeZone`|Sets the default time zone for the server, which will affect how time functions and timestamp literals behave. Should be a time zone name like "America/Los_Angeles" or offset like "-08:00".|UTC|
 
-### Broker Caching
+#### Broker Caching
 
-You can optionally only configure caching to be enabled on the broker by setting caching configs here.
+You can optionally only configure caching to be enabled on the Broker by setting caching configs here.
 
 |Property|Possible Values|Description|Default|
 |--------|---------------|-----------|-------|
-|`druid.broker.cache.useCache`|true, false|Enable the cache on the broker.|false|
-|`druid.broker.cache.populateCache`|true, false|Populate the cache on the broker.|false|
-|`druid.broker.cache.useResultLevelCache`|true, false|Enable result level caching on the broker.|false|
-|`druid.broker.cache.populateResultLevelCache`|true, false|Populate the result level cache on the broker.|false|
+|`druid.broker.cache.useCache`|true, false|Enable the cache on the Broker.|false|
+|`druid.broker.cache.populateCache`|true, false|Populate the cache on the Broker.|false|
+|`druid.broker.cache.useResultLevelCache`|true, false|Enable result level caching on the Broker.|false|
+|`druid.broker.cache.populateResultLevelCache`|true, false|Populate the result level cache on the Broker.|false|
 |`druid.broker.cache.resultLevelCacheLimit`|positive integer|Maximum size of query response that can be cached.|`Integer.MAX_VALUE`|
 |`druid.broker.cache.unCacheable`|All druid query types|All query types to not cache.|`["groupBy", "select"]`|
-|`druid.broker.cache.cacheBulkMergeLimit`|positive integer or 0|Queries with more segments than this number will not attempt to fetch from cache at the broker level, leaving potential caching fetches (and cache result merging) to the historicals|`Integer.MAX_VALUE`|
+|`druid.broker.cache.cacheBulkMergeLimit`|positive integer or 0|Queries with more segments than this number will not attempt to fetch from cache at the broker level, leaving potential caching fetches (and cache result merging) to the Historicals|`Integer.MAX_VALUE`|
 
 See [cache configuration](#cache-configuration) for how to configure cache settings.
 
-### Segment Discovery
+#### Segment Discovery
 |Property|Possible Values|Description|Default|
 |--------|---------------|-----------|-------|
 |`druid.serverview.type`|batch or http|Segment discovery method to use. "http" enables discovering segments using HTTP instead of zookeeper.|batch|
-|`druid.broker.segment.watchedTiers`|List of strings|Broker watches the segment announcements from nodes serving segments to build cache of which node is serving which segments, this configuration allows to only consider segments being served from a whitelist of tiers. By default, Broker would consider all tiers. This can be used to partition your dataSources in specific historical tiers and configure brokers in partitions so that they are only queryable for specific dataSources.|none|
+|`druid.broker.segment.watchedTiers`|List of strings|Broker watches the segment announcements from nodes serving segments to build cache of which node is serving which segments, this configuration allows to only consider segments being served from a whitelist of tiers. By default, Broker would consider all tiers. This can be used to partition your dataSources in specific Historical tiers and configure brokers in partitions so that they are only queryable for specific dataSources.|none|
 |`druid.broker.segment.watchedDataSources`|List of strings|Broker watches the segment announcements from nodes serving segments to build cache of which node is serving which segments, this configuration allows to only consider segments being served from a whitelist of dataSources. By default, Broker would consider all datasources. This can be used to configure brokers in partitions so that they are only queryable for specific dataSources.|none|
 |`druid.broker.segment.awaitInitializationOnStart`|Boolean|Whether the the Broker will wait for its view of segments to fully initialize before starting up. If set to 'true', the Broker's HTTP server will not start up, and the Broker will not announce itself as available, until the server view is initialized. See also `druid.sql.planner.awaitInitializationOnStart`, a related setting.|true|
 
-## Historical
-
-For general Historical Node information, see [here](../design/historical.html).
-
-These Historical configurations can be defined in the `historical/runtime.properties` file.
-
-### Historical Node Configuration
-|Property|Description|Default|
-|--------|-----------|-------|
-|`druid.host`|The host for the current node. This is used to advertise the current processes location as reachable from another node and should generally be specified such that `http://${druid.host}/` could actually talk to this process|InetAddress.getLocalHost().getCanonicalHostName()|
-|`druid.bindOnHost`|Indicating whether the node's internal jetty server bind on `druid.host`. Default is false, which means binding to all interfaces.|false|
-|`druid.plaintextPort`|This is the port to actually listen on; unless port mapping is used, this will be the same port as is on `druid.host`|8083|
-|`druid.tlsPort`|TLS port for HTTPS connector, if [druid.enableTlsPort](../operations/tls-support.html) is set then this config will be used. If `druid.host` contains port then that port will be ignored. This should be a non-negative Integer.|8283|
-|`druid.service`|The name of the service. This is used as a dimension when emitting metrics and alerts to differentiate between the various services|druid/historical|
-
-
-### Historical General Configuration
-
-|Property|Description|Default|
-|--------|-----------|-------|
-|`druid.server.maxSize`|The maximum number of bytes-worth of segments that the node wants assigned to it. This is not a limit that Historical nodes actually enforces, just a value published to the Coordinator node so it can plan accordingly.|0|
-|`druid.server.tier`| A string to name the distribution tier that the storage node belongs to. Many of the [rules Coordinator nodes use](../operations/rule-configuration.html) to manage segments can be keyed on tiers. |  `_default_tier` |
-|`druid.server.priority`|In a tiered architecture, the priority of the tier, thus allowing control over which nodes are queried. Higher numbers mean higher priority. The default (no priority) works for architecture with no cross replication (tiers that have no data-storage overlap). Data centers typically have equal priority. | 0 |
-
-### Storing Segments
-
-|Property|Description|Default|
-|--------|-----------|-------|
-|`druid.segmentCache.locations`|Segments assigned to a Historical node are first stored on the local file system (in a disk cache) and then served by the Historical node. These locations define where that local cache resides. This value cannot be NULL or EMPTY. Here is an example `druid.segmentCache.locations=[{"path": "/mnt/druidSegments", "maxSize": 10000, "freeSpacePercent": 1.0}]`. "freeSpacePercent" is optional, if provided then enforces that much of free disk partition space while  [...]
-|`druid.segmentCache.deleteOnRemove`|Delete segment files from cache once a node is no longer serving a segment.|true|
-|`druid.segmentCache.dropSegmentDelayMillis`|How long a node delays before completely dropping segment.|30000 (30 seconds)|
-|`druid.segmentCache.infoDir`|Historical nodes keep track of the segments they are serving so that when the process is restarted they can reload the same segments without waiting for the Coordinator to reassign. This path defines where this metadata is kept. Directory will be created if needed.|${first_location}/info_dir|
-|`druid.segmentCache.announceIntervalMillis`|How frequently to announce segments while segments are loading from cache. Set this value to zero to wait for all segments to be loaded before announcing.|5000 (5 seconds)|
-|`druid.segmentCache.numLoadingThreads`|How many segments to drop or load concurrently from from deep storage.|10|
-|`druid.segmentCache.numBootstrapThreads`|How many segments to load concurrently from local storage at startup.|Same as numLoadingThreads|
-
-In `druid.segmentCache.locations`, *freeSpacePercent* was added because *maxSize* setting is only a theoretical limit and assumes that much space will always be available for storing segments. In case of any druid bug leading to unaccounted segment files left alone on disk or some other process writing stuff to disk, This check can start failing segment loading early before filling up the disk completely and leaving the host usable otherwise.
-
-### Historical Query Configs
-
-#### Concurrent Requests
-
-Druid uses Jetty to serve HTTP requests.
-
-|Property|Description|Default|
-|--------|-----------|-------|
-|`druid.server.http.numThreads`|Number of threads for HTTP requests.|max(10, (Number of cores * 17) / 16 + 2) + 30|
-|`druid.server.http.queueSize`|Size of the worker queue used by Jetty server to temporarily store incoming client connections. If this value is set and a request is rejected by jetty because queue is full then client would observe request failure with TCP connection being closed immediately with a completely empty response from server.|Unbounded|
-|`druid.server.http.maxIdleTime`|The Jetty max idle time for a connection.|PT5M|
-|`druid.server.http.enableRequestLimit`|If enabled, no requests would be queued in jetty queue and "HTTP 429 Too Many Requests" error response would be sent. |false|
-|`druid.server.http.defaultQueryTimeout`|Query timeout in millis, beyond which unfinished queries will be cancelled|300000|
-|`druid.server.http.gracefulShutdownTimeout`|The maximum amount of time Jetty waits after receiving shutdown signal. After this timeout the threads will be forcefully shutdown. This allows any queries that are executing to complete.|`PT0S` (do not wait)|
-|`druid.server.http.unannouncePropagationDelay`|How long to wait for zookeeper unannouncements to propagate before shutting down Jetty. This is a minimum and `druid.server.http.gracefulShutdownTimeout` does not start counting down until after this period elapses.|`PT0S` (do not wait)|
-|`druid.server.http.maxQueryTimeout`|Maximum allowed value (in milliseconds) for `timeout` parameter. See [query-context](../querying/query-context.html) to know more about `timeout`. Query is rejected if the query context `timeout` is greater than this value. |Long.MAX_VALUE|
-|`druid.server.http.maxRequestHeaderSize`|Maximum size of a request header in bytes. Larger headers consume more memory and can make a server more vulnerable to denial of service attacks.|8 * 1024|
-
-#### Processing
-
-|Property|Description|Default|
-|--------|-----------|-------|
-|`druid.processing.buffer.sizeBytes`|This specifies a buffer size for the storage of intermediate results. The computation engine in both the Historical and Realtime nodes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed.|auto (max 1GB)|
-|`druid.processing.buffer.poolCacheMaxCount`|processing buffer pool caches the buffers for later use, this is the maximum count cache will grow to. note that pool can create more buffers than it can cache if necessary.|Integer.MAX_VALUE|
-|`druid.processing.formatString`|Realtime and historical nodes use this format string to name their processing threads.|processing-%s|
-|`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. If you are using any queries that require merge buffers (currently, just groupBy v2) then you should have at least two of these.|`max(2, druid.processing.numThreads / 4)`|
-|`druid.processing.numThreads`|The number of processing threads to have available for parallel processing of segments. Our rule of thumb is `num_cores - 1`, which means that even under heavy load there will still be one core available to do background tasks like talking with ZooKeeper and pulling down segments. If only one core is available, this property defaults to the value `1`.|Number of cores - 1 (or 1)|
-|`druid.processing.columnCache.sizeBytes`|Maximum size in bytes for the dimension value lookup cache. Any value greater than `0` enables the cache. It is currently disabled by default. Enabling the lookup cache can significantly improve the performance of aggregators operating on dimension values, such as the JavaScript aggregator, or cardinality aggregator, but can slow things down if the cache hit rate is low (i.e. dimensions with few repeating values). Enabling it may also require add [...]
-|`druid.processing.fifo`|If the processing queue should treat tasks of equal priority in a FIFO manner|`false`|
-|`druid.processing.tmpDir`|Path where temporary files created while processing a query should be stored. If specified, this configuration takes priority over the default `java.io.tmpdir` path.|path represented by `java.io.tmpdir`|
-
-The amount of direct memory needed by Druid is at least
-`druid.processing.buffer.sizeBytes * (druid.processing.numMergeBuffers + druid.processing.numThreads + 1)`. You can
-ensure at least this amount of direct memory is available by providing `-XX:MaxDirectMemorySize=<VALUE>` at the command
-line.
-
-#### Historical Query Configuration
-
-See [general query configuration](#general-query-configuration).
-
-### Historical Caching
-
-You can optionally only configure caching to be enabled on the historical by setting caching configs here.
-
-|Property|Possible Values|Description|Default|
-|--------|---------------|-----------|-------|
-|`druid.historical.cache.useCache`|true, false|Enable the cache on the historical.|false|
-|`druid.historical.cache.populateCache`|true, false|Populate the cache on the historical.|false|
-|`druid.historical.cache.unCacheable`|All druid query types|All query types to not cache.|["groupBy", "select"]|
-
-See [cache configuration](#cache-configuration) for how to configure cache settings.
-
 ## Cache Configuration
 
-This section describes caching configuration that is common to broker, historical, and middleManager/peon nodes.
+This section describes caching configuration that is common to Broker, Historical, and MiddleManager/Peon processes.
  
-Caching can optionally be enabled on the broker, historical, and middleManager/peon nodes. See [broker](#broker-caching), 
-[historical](#historical-caching), and [peon](#peon-caching) configuration options for how to enable it for different processes.
+Caching can optionally be enabled on the Broker, Historical, and MiddleManager/Peon processses. See [Broker](#broker-caching), 
+[Historical](#Historical-caching), and [Peon](#peon-caching) 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 settings are set globally, so the same configuration can be re-used
-for both broker and historical nodes, when defined in the common properties file.
+for both Broker and Historical nodes, when defined in the common properties file.
 
 
 ### Cache Type
@@ -1527,12 +1543,12 @@ If there is an L1 miss and L2 hit, it will also populate L1.
 |`druid.cache.l2.type`|type of cache to use for L2 cache. See `druid.cache.type` configuration for valid types.|`caffeine`|
 |`druid.cache.l1.*`|Any property valid for the given type of L1 cache can be set using this prefix. For instance, if you are using a `caffeine` L1 cache, specify `druid.cache.l1.sizeInBytes` to set its size.|defaults are the same as for the given cache type.|
 |`druid.cache.l2.*`|Prefix for L2 cache settings, see description for L1.|defaults are the same as for the given cache type.|
-|`druid.cache.useL2`|A boolean indicating whether to query L2 cache, if it's a miss in L1. It makes sense to configure this to `false` on historical nodes, if L2 is a remote cache like `memcached`, and this cache also used on brokers, because in this case if a query reached historical it means that a broker didn't find corresponding results in the same remote cache, so a query to the remote cache from historical is guaranteed to be a miss.|`true`|
+|`druid.cache.useL2`|A boolean indicating whether to query L2 cache, if it's a miss in L1. It makes sense to configure this to `false` on Historical nodes, if L2 is a remote cache like `memcached`, and this cache also used on brokers, because in this case if a query reached Historical it means that a broker didn't find corresponding results in the same remote cache, so a query to the remote cache from Historical is guaranteed to be a miss.|`true`|
 |`druid.cache.populateL2`|A boolean indicating whether to put results into L2 cache.|`true`|
 
 ## General Query Configuration
 
-This section describes configurations that control behavior of Druid's query types, applicable to broker, historical, and middle manager nodes.
+This section describes configurations that control behavior of Druid's query types, applicable to Broker, Historical, and MiddleManager nodes.
 
 ### TopN Query config
 
@@ -1556,7 +1572,7 @@ This section describes configurations that control behavior of Druid's query typ
 
 ### GroupBy Query Config
 
-This section describes the configurations for groupBy queries. You can set the runtime properties in the `runtime.properties` file on broker, historical, and MiddleManager nodes. You can set the query context parameters through the [query context](../querying/query-context.html).
+This section describes the configurations for groupBy queries. You can set the runtime properties in the `runtime.properties` file on Broker, Historical, and MiddleManager nodes. You can set the query context parameters through the [query context](../querying/query-context.html).
   
 #### Configurations for groupBy v2
 
@@ -1616,7 +1632,7 @@ Supported query contexts:
 |`intermediateCombineDegree`|Overrides the value of `druid.query.groupBy.intermediateCombineDegree`|None|
 |`numParallelCombineThreads`|Overrides the value of `druid.query.groupBy.numParallelCombineThreads`|None|
 |`sortByDimsFirst`|Sort the results first by dimension values and then by timestamp.|false|
-|`forceLimitPushDown`|When all fields in the orderby are part of the grouping key, the broker will push limit application down to the historical nodes. When the sorting order uses fields that are not in the grouping key, applying this optimization can result in approximate results with unknown accuracy, so this optimization is disabled by default in that case. Enabling this context flag turns on limit push down for limit/orderbys that contain non-grouping key columns.|false|
+|`forceLimitPushDown`|When all fields in the orderby are part of the grouping key, the broker will push limit application down to the Historical nodes. When the sorting order uses fields that are not in the grouping key, applying this optimization can result in approximate results with unknown accuracy, so this optimization is disabled by default in that case. Enabling this context flag turns on limit push down for limit/orderbys that contain non-grouping key columns.|false|
 
 
 #### GroupBy v1 configurations
diff --git a/docs/content/configuration/realtime.md b/docs/content/configuration/realtime.md
index de636ec..396e1f7 100644
--- a/docs/content/configuration/realtime.md
+++ b/docs/content/configuration/realtime.md
@@ -61,7 +61,7 @@ The realtime node uses several of the global configs in [Configuration](../confi
 |Property|Description|Default|
 |--------|-----------|-------|
 |`druid.processing.buffer.sizeBytes`|This specifies a buffer size for the storage of intermediate results. The computation engine in both the Historical and Realtime nodes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed.|auto (max 1GB)|
-|`druid.processing.formatString`|Realtime and historical nodes use this format string to name their processing threads.|processing-%s|
+|`druid.processing.formatString`|Realtime and Historical nodes use this format string to name their processing threads.|processing-%s|
 |`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. If you are using any queries that require merge buffers (currently, just groupBy v2) then you should have at least two of these.|`max(2, druid.processing.numThreads / 4)`|
 |`druid.processing.numThreads`|The number of processing threads to have available for parallel processing of segments. Our rule of thumb is `num_cores - 1`, which means that even under heavy load there will still be one core available to do background tasks like talking with ZooKeeper and pulling down segments. If only one core is available, this property defaults to the value `1`.|Number of cores - 1 (or 1)|
 |`druid.processing.columnCache.sizeBytes`|Maximum size in bytes for the dimension value lookup cache. Any value greater than `0` enables the cache. It is currently disabled by default. Enabling the lookup cache can significantly improve the performance of aggregators operating on dimension values, such as the JavaScript aggregator, or cardinality aggregator, but can slow things down if the cache hit rate is low (i.e. dimensions with few repeating values). Enabling it may also require add [...]
diff --git a/docs/content/dependencies/cassandra-deep-storage.md b/docs/content/dependencies/cassandra-deep-storage.md
index 681d298..7eb4b1a 100644
--- a/docs/content/dependencies/cassandra-deep-storage.md
+++ b/docs/content/dependencies/cassandra-deep-storage.md
@@ -29,7 +29,7 @@ title: "Cassandra Deep Storage"
 Druid can use Cassandra as a deep storage mechanism. Segments and their metadata are stored in Cassandra in two tables:
 `index_storage` and `descriptor_storage`.  Underneath the hood, the Cassandra integration leverages Astyanax.  The
 index storage table is a [Chunked Object](https://github.com/Netflix/astyanax/wiki/Chunked-Object-Store) repository. It contains
-compressed segments for distribution to historical nodes.  Since segments can be large, the Chunked Object storage allows the integration to multi-thread
+compressed segments for distribution to Historical nodes.  Since segments can be large, the Chunked Object storage allows the integration to multi-thread
 the write to Cassandra, and spreads the data across all the nodes in a cluster.  The descriptor storage table is a normal C* table that
 stores the segment metadatak.
 
@@ -52,7 +52,7 @@ CREATE TABLE descriptor_storage(key varchar,
 First create the schema above. I use a new keyspace called `druid` for this purpose, which can be created using the
 [Cassandra CQL `CREATE KEYSPACE`](http://www.datastax.com/documentation/cql/3.1/cql/cql_reference/create_keyspace_r.html) command.
 
-Then, add the following to your historical and realtime runtime properties files to enable a Cassandra backend.
+Then, add the following to your Historical and realtime runtime properties files to enable a Cassandra backend.
 
 ```properties
 druid.extensions.loadList=["druid-cassandra-storage"]
diff --git a/docs/content/dependencies/metadata-storage.md b/docs/content/dependencies/metadata-storage.md
index f82ebfb..bfc77cb 100644
--- a/docs/content/dependencies/metadata-storage.md
+++ b/docs/content/dependencies/metadata-storage.md
@@ -122,8 +122,7 @@ parameters across the cluster at runtime.
 
 ### Task-related Tables
 
-There are also a number of tables created and used by the [Indexing
-Service](../design/indexing-service.html) in the course of its work.
+There are also a number of tables created and used by the [Overlord](../design/overlord.html) and [MiddleManager](../design/middlemanager.html) when managing tasks.
 
 ### Audit Table
 
diff --git a/docs/content/dependencies/zookeeper.md b/docs/content/dependencies/zookeeper.md
index b1ceeb5..d2b4cd0 100644
--- a/docs/content/dependencies/zookeeper.md
+++ b/docs/content/dependencies/zookeeper.md
@@ -29,8 +29,8 @@ Druid uses [ZooKeeper](http://zookeeper.apache.org/) (ZK) for management of curr
 1.  [Coordinator](../design/coordinator.html) leader election
 2.  Segment "publishing" protocol from [Historical](../design/historical.html) and [Realtime](../design/realtime.html)
 3.  Segment load/drop protocol between [Coordinator](../design/coordinator.html) and [Historical](../design/historical.html)
-4.  [Overlord](../design/indexing-service.html) leader election
-5.  [Indexing Service](../design/indexing-service.html) task management
+4.  [Overlord](../design/overlord.html) leader election
+5.  [Overlord](../design/overlord.html) and [MiddleManager](../design/middlemanager.html) task management
 
 ### Coordinator Leader Election
 
@@ -74,4 +74,4 @@ When the [Coordinator](../design/coordinator.html) decides that a [Historical](.
 ${druid.zk.paths.loadQueuePath}/_host_of_historical_node/_segment_identifier
 ```
 
-This node will contain a payload that indicates to the historical node what it should do with the given segment. When the historical node is done with the work, it will delete the znode in order to signify to the Coordinator that it is complete.
+This node will contain a payload that indicates to the Historical node what it should do with the given segment. When the Historical node is done with the work, it will delete the znode in order to signify to the Coordinator that it is complete.
diff --git a/docs/content/design/auth.md b/docs/content/design/auth.md
index c8b7a8c..95f8be6 100644
--- a/docs/content/design/auth.md
+++ b/docs/content/design/auth.md
@@ -80,7 +80,7 @@ druid.auth.authenticator.anonymous.authorizerName=myBasicAuthorizer
 ```
 
 ## Escalator
-The `druid.escalator.type` property determines what authentication scheme should be used for internal Druid cluster communications (such as when a broker node communicates with historical nodes for query processing).
+The `druid.escalator.type` property determines what authentication scheme should be used for internal Druid cluster communications (such as when a Broker process communicates with Historical processes for query processing).
 
 The Escalator chosen for this property must use an authentication scheme that is supported by an Authenticator in `druid.auth.authenticationChain`. Authenticator extension implementors must also provide a corresponding Escalator implementation if they intend to use a particular authentication scheme for internal Druid communications.
 
diff --git a/docs/content/design/broker.md b/docs/content/design/broker.md
index ff90ffe..1eb51cd 100644
--- a/docs/content/design/broker.md
+++ b/docs/content/design/broker.md
@@ -47,9 +47,9 @@ org.apache.druid.cli.Main server broker
 
 Most druid queries contain an interval object that indicates a span of time for which data is requested. Likewise, Druid [Segments](../design/segments.html) are partitioned to contain data for some interval of time and segments are distributed across a cluster. Consider a simple datasource with 7 segments where each segment contains data for a given day of the week. Any query issued to the datasource for more than one day of data will hit more than one segment. These segments will likely [...]
 
-To determine which nodes to forward queries to, the Broker node first builds a view of the world from information in Zookeeper. Zookeeper maintains information about [Historical](../design/historical.html) and [Realtime](../design/realtime.html) nodes and the segments they are serving. For every datasource in Zookeeper, the Broker node builds a timeline of segments and the nodes that serve them. When queries are received for a specific datasource and interval, the Broker node performs a  [...]
+To determine which nodes to forward queries to, the Broker node first builds a view of the world from information in Zookeeper. Zookeeper maintains information about [Historical](../design/historical.html) and [Realtime](../design/realtime.html) nodes and the segments they are serving. For every datasource in Zookeeper, the Broker node builds a timeline of segments and the nodes that serve them. When queries are received for a specific datasource and interval, the Broker node performs a  [...]
 
 ### Caching
 
-Broker nodes employ a cache with a LRU cache invalidation strategy. The broker cache stores per-segment results. The cache can be local to each broker node or shared across multiple nodes using an external distributed cache such as [memcached](http://memcached.org/). Each time a broker node receives a query, it first maps the query to a set of segments. A subset of these segment results may already exist in the cache and the results can be directly pulled from the cache. For any segment r [...]
-historical nodes. Once the historical nodes return their results, the broker will store those results in the cache. Real-time segments are never cached and hence requests for real-time data will always be forwarded to real-time nodes. Real-time data is perpetually changing and caching the results would be unreliable.
+Broker nodes employ a cache with a LRU cache invalidation strategy. The Broker cache stores per-segment results. The cache can be local to each Broker process or shared across multiple processes using an external distributed cache such as [memcached](http://memcached.org/). Each time a broker node receives a query, it first maps the query to a set of segments. A subset of these segment results may already exist in the cache and the results can be directly pulled from the cache. For any se [...]
+Historical nodes. Once the Historical processes return their results, the Broker will store those results in the cache. Real-time segments are never cached and hence requests for real-time data will always be forwarded to real-time nodes. Real-time data is perpetually changing and caching the results would be unreliable.
diff --git a/docs/content/design/coordinator.md b/docs/content/design/coordinator.md
index e8dea29..a0fdb6b 100644
--- a/docs/content/design/coordinator.md
+++ b/docs/content/design/coordinator.md
@@ -34,11 +34,11 @@ For a list of API endpoints supported by the Coordinator, see [Coordinator API](
 
 ### Overview
 
-The Druid coordinator node is primarily responsible for segment management and distribution. More specifically, the Druid coordinator node communicates to historical nodes to load or drop segments based on configurations. The Druid coordinator is responsible for loading new segments, dropping outdated segments, managing segment replication, and balancing segment load.
+The Druid Coordinator node is primarily responsible for segment management and distribution. More specifically, the Druid Coordinator node communicates to Historical nodes to load or drop segments based on configurations. The Druid Coordinator is responsible for loading new segments, dropping outdated segments, managing segment replication, and balancing segment load.
 
-The Druid coordinator runs periodically and the time between each run is a configurable parameter. Each time the Druid coordinator runs, it assesses the current state of the cluster before deciding on the appropriate actions to take. Similar to the broker and historical nodes, the Druid coordinator maintains a connection to a Zookeeper cluster for current cluster information. The coordinator also maintains a connection to a database containing information about available segments and rul [...]
+The Druid Coordinator runs periodically and the time between each run is a configurable parameter. Each time the Druid Coordinator runs, it assesses the current state of the cluster before deciding on the appropriate actions to take. Similar to the Broker and Historical processses, the Druid Coordinator maintains a connection to a Zookeeper cluster for current cluster information. The Coordinator also maintains a connection to a database containing information about available segments an [...]
 
-Before any unassigned segments are serviced by historical nodes, the available historical nodes for each tier are first sorted in terms of capacity, with least capacity servers having the highest priority. Unassigned segments are always assigned to the nodes with least capacity to maintain a level of balance between nodes. The coordinator does not directly communicate with a historical node when assigning it a new segment; instead the coordinator creates some temporary information about  [...]
+Before any unassigned segments are serviced by Historical nodes, the available Historical nodes for each tier are first sorted in terms of capacity, with least capacity servers having the highest priority. Unassigned segments are always assigned to the nodes with least capacity to maintain a level of balance between nodes. The Coordinator does not directly communicate with a historical node when assigning it a new segment; instead the Coordinator creates some temporary information about  [...]
 
 ### Running
 
@@ -52,23 +52,23 @@ Segments can be automatically loaded and dropped from the cluster based on a set
 
 ### Cleaning Up Segments
 
-Each run, the Druid coordinator compares the list of available database segments in the database with the current segments in the cluster. Segments that are not in the database but are still being served in the cluster are flagged and appended to a removal list. Segments that are overshadowed (their versions are too old and their data has been replaced by newer segments) are also dropped.
-Note that if all segments in database are deleted(or marked unused), then coordinator will not drop anything from the historicals. This is done to prevent a race condition in which the coordinator would drop all segments if it started running cleanup before it finished polling the database for available segments for the first time and believed that there were no segments.
+Each run, the Druid Coordinator compares the list of available database segments in the database with the current segments in the cluster. Segments that are not in the database but are still being served in the cluster are flagged and appended to a removal list. Segments that are overshadowed (their versions are too old and their data has been replaced by newer segments) are also dropped.
+Note that if all segments in database are deleted(or marked unused), then Coordinator will not drop anything from the Historicals. This is done to prevent a race condition in which the Coordinator would drop all segments if it started running cleanup before it finished polling the database for available segments for the first time and believed that there were no segments.
 
 ### Segment Availability
 
-If a historical node restarts or becomes unavailable for any reason, the Druid coordinator will notice a node has gone missing and treat all segments served by that node as being dropped. Given a sufficient period of time, the segments may be reassigned to other historical nodes in the cluster. However, each segment that is dropped is not immediately forgotten. Instead, there is a transitional data structure that stores all dropped segments with an associated lifetime. The lifetime repre [...]
+If a Historical node restarts or becomes unavailable for any reason, the Druid Coordinator will notice a node has gone missing and treat all segments served by that node as being dropped. Given a sufficient period of time, the segments may be reassigned to other Historical nodes in the cluster. However, each segment that is dropped is not immediately forgotten. Instead, there is a transitional data structure that stores all dropped segments with an associated lifetime. The lifetime repre [...]
 
 ### Balancing Segment Load
 
-To ensure an even distribution of segments across historical nodes in the cluster, the coordinator node will find the total size of all segments being served by every historical node each time the coordinator runs. For every historical node tier in the cluster, the coordinator node will determine the historical node with the highest utilization and the historical node with the lowest utilization. The percent difference in utilization between the two nodes is computed, and if the result e [...]
+To ensure an even distribution of segments across Historical nodes in the cluster, the Coordinator node will find the total size of all segments being served by every Historical node each time the Coordinator runs. For every Historical node tier in the cluster, the Coordinator node will determine the Historical node with the highest utilization and the Historical node with the lowest utilization. The percent difference in utilization between the two nodes is computed, and if the result e [...]
 
 ### Compacting Segments
 
-Each run, the Druid coordinator compacts small segments abutting each other. This is useful when you have a lot of small
+Each run, the Druid Coordinator compacts small segments abutting each other. This is useful when you have a lot of small
 segments which may degrade the query performance as well as increasing the disk space usage.
 
-The coordinator first finds the segments to compact together based on the [segment search policy](#segment-search-policy).
+The Coordinator first finds the segments to compact together based on the [segment search policy](#segment-search-policy).
 Once some segments are found, it launches a [compaction task](../ingestion/tasks.html#compaction-task) to compact those segments.
 The maximum number of running compaction tasks is `min(sum of worker capacity * slotRatio, maxSlots)`.
 Note that even though `min(sum of worker capacity * slotRatio, maxSlots)` = 0, at least one compaction task is always submitted
@@ -80,7 +80,7 @@ Compaction tasks might fail due to the following reasons.
 - If the input segments of a compaction task are removed or overshadowed before it starts, that compaction task fails immediately.
 - If a task of a higher priority acquires a lock for an interval overlapping with the interval of a compaction task, the compaction task fails.
 
-Once a compaction task fails, the coordinator simply finds the segments for the interval of the failed task again, and launches a new compaction task in the next run.
+Once a compaction task fails, the Coordinator simply finds the segments for the interval of the failed task again, and launches a new compaction task in the next run.
 
 ### Segment Search Policy
 
@@ -113,28 +113,28 @@ If it finds such segments, it simply skips them.
 
 ### The Coordinator Console
 
-The Druid coordinator exposes a web GUI for displaying cluster information and rule configuration. After the coordinator starts, the console can be accessed at:
+The Druid Coordinator exposes a web GUI for displaying cluster information and rule configuration. After the Coordinator starts, the console can be accessed at:
 
 ```
 http://<COORDINATOR_IP>:<COORDINATOR_PORT>
 ```
 
- There exists a full cluster view (which shows only the realtime and historical nodes), as well as views for individual historical nodes, datasources and segments themselves. Segment information can be displayed in raw JSON form or as part of a sortable and filterable table.
+ There exists a full cluster view (which shows only the realtime and Historical nodes), as well as views for individual Historical nodes, datasources and segments themselves. Segment information can be displayed in raw JSON form or as part of a sortable and filterable table.
 
-The coordinator console also exposes an interface to creating and editing rules. All valid datasources configured in the segment database, along with a default datasource, are available for configuration. Rules of different types can be added, deleted or edited.
+The Coordinator console also exposes an interface to creating and editing rules. All valid datasources configured in the segment database, along with a default datasource, are available for configuration. Rules of different types can be added, deleted or edited.
 
 ### FAQ
 
-1. **Do clients ever contact the coordinator node?**
+1. **Do clients ever contact the Coordinator node?**
 
-    The coordinator is not involved in a query.
+    The Coordinator is not involved in a query.
 
-    historical nodes never directly contact the coordinator node. The Druid coordinator tells the historical nodes to load/drop data via Zookeeper, but the historical nodes are completely unaware of the coordinator.
+    Historical nodes never directly contact the Coordinator node. The Druid Coordinator tells the Historical nodes to load/drop data via Zookeeper, but the Historical nodes are completely unaware of the Coordinator.
 
-    Brokers also never contact the coordinator. Brokers base their understanding of the data topology on metadata exposed by the historical nodes via ZK and are completely unaware of the coordinator.
+    Brokers also never contact the Coordinator. Brokers base their understanding of the data topology on metadata exposed by the Historical nodes via ZK and are completely unaware of the Coordinator.
 
-2. **Does it matter if the coordinator node starts up before or after other processes?**
+2. **Does it matter if the Coordinator node starts up before or after other processes?**
 
-    No. If the Druid coordinator is not started up, no new segments will be loaded in the cluster and outdated segments will not be dropped. However, the coordinator node can be started up at any time, and after a configurable delay, will start running coordinator tasks.
+    No. If the Druid Coordinator is not started up, no new segments will be loaded in the cluster and outdated segments will not be dropped. However, the Coordinator node can be started up at any time, and after a configurable delay, will start running Coordinator tasks.
 
-    This also means that if you have a working cluster and all of your coordinators die, the cluster will continue to function, it just won’t experience any changes to its data topology.
+    This also means that if you have a working cluster and all of your Coordinators die, the cluster will continue to function, it just won’t experience any changes to its data topology.
diff --git a/docs/content/design/historical.md b/docs/content/design/historical.md
index fa7e79f..5d4efb4 100644
--- a/docs/content/design/historical.md
+++ b/docs/content/design/historical.md
@@ -40,20 +40,20 @@ org.apache.druid.cli.Main server historical
 
 ### Loading and Serving Segments
 
-Each historical node maintains a constant connection to Zookeeper and watches a configurable set of Zookeeper paths for new segment information. Historical nodes do not communicate directly with each other or with the coordinator nodes but instead rely on Zookeeper for coordination.
+Each Historical node maintains a constant connection to Zookeeper and watches a configurable set of Zookeeper paths for new segment information. Historical nodes do not communicate directly with each other or with the Coordinator nodes but instead rely on Zookeeper for coordination.
 
-The [Coordinator](../design/coordinator.html) node is responsible for assigning new segments to historical nodes. Assignment is done by creating an ephemeral Zookeeper entry under a load queue path associated with a historical node. For more information on how the coordinator assigns segments to historical nodes, please see [Coordinator](../design/coordinator.html).
+The [Coordinator](../design/coordinator.html) node is responsible for assigning new segments to Historical nodes. Assignment is done by creating an ephemeral Zookeeper entry under a load queue path associated with a Historical node. For more information on how the Coordinator assigns segments to Historical nodes, please see [Coordinator](../design/coordinator.html).
 
-When a historical node notices a new load queue entry in its load queue path, it will first check a local disk directory (cache) for the information about segment. If no information about the segment exists in the cache, the historical node will download metadata about the new segment to serve from Zookeeper. This metadata includes specifications about where the segment is located in deep storage and about how to decompress and process the segment. For more information about segment meta [...]
+When a Historical node notices a new load queue entry in its load queue path, it will first check a local disk directory (cache) for the information about segment. If no information about the segment exists in the cache, the Historical node will download metadata about the new segment to serve from Zookeeper. This metadata includes specifications about where the segment is located in deep storage and about how to decompress and process the segment. For more information about segment meta [...]
 
 ### Loading and Serving Segments From Cache
 
-Recall that when a historical node notices a new segment entry in its load queue path, the historical node first checks a configurable cache directory on its local disk to see if the segment had been previously downloaded. If a local cache entry already exists, the historical node will directly read the segment binary files from disk and load the segment.
+Recall that when a Historical node notices a new segment entry in its load queue path, the Historical node first checks a configurable cache directory on its local disk to see if the segment had been previously downloaded. If a local cache entry already exists, the Historical node will directly read the segment binary files from disk and load the segment.
 
-The segment cache is also leveraged when a historical node is first started. On startup, a historical node will search through its cache directory and immediately load and serve all segments that are found. This feature allows historical nodes to be queried as soon they come online.
+The segment cache is also leveraged when a Historical node is first started. On startup, a Historical node will search through its cache directory and immediately load and serve all segments that are found. This feature allows Historical nodes to be queried as soon they come online.
 
 ### Querying Segments
 
-Please see [Querying](../querying/querying.html) for more information on querying historical nodes.
+Please see [Querying](../querying/querying.html) for more information on querying Historical nodes.
 
-A historical can be configured to log and report metrics for every query it services.
+A Historical can be configured to log and report metrics for every query it services.
diff --git a/docs/content/design/index.md b/docs/content/design/index.md
index 29c1aee..60a5a2b 100644
--- a/docs/content/design/index.md
+++ b/docs/content/design/index.md
@@ -93,48 +93,62 @@ Druid has a multi-process, distributed architecture that is designed to be cloud
 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
+## Processes and Servers
+
+Druid has several process types, briefly described below:
+
+* [**Coordinator**](../design/coordinator.html) processes manage data availability on the cluster.
+* [**Overlord**](../design/overlord.html) processes control the assignment of data ingestion workloads.
+* [**Broker**](../design/broker.html) processes handle queries from external clients.
+* [**Router**](../development/router.html) processes are optional processes that can route requests to Brokers, Coordinators, and Overlords.
+* [**Historical**](../design/historical.html) processes store queryable data.
+* [**MiddleManager**](../design/middlemanager.html) processes are responsible for ingesting data.
+
+Druid processes can be deployed any way you like, but for ease of deployment we suggest organizing them into three server types: Master, Query, and Data.
+
+* **Master**: Runs Coordinator and Overlord processes, manages data availability and ingestion.
+* **Query**: Runs Broker and optional Router processes, handles queries from external clients.
+* **Data**: Runs Historical and MiddleManager processes, executes ingestion workloads and stores all queryable data.
+
+For more details on process and server organization, please see [Druid Processses and Servers](../design/processes.html).
+
+### External dependencies
+
+In addition to its built-in 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
+#### 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.
+
+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 more details, please see [Deep storage dependency](../dependencies/deep-storage.html).
+
+#### Metadata storage
+The metadata storage holds various shared system metadata such as segment availability information and task information. This is typically going to be a traditional RDBMS
+like PostgreSQL or MySQL. 
+
+For more details, please see [Metadata storage dependency](../dependencies/metadata-storage.html)
+
+#### Zookeeper
+Used for internal service discovery, coordination, and leader election.
+
+For more details, please see [Zookeeper dependency](../dependencies/zookeeper.html).
 
 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:
+### Architecture diagram
+
+The following diagram shows how queries and data flow through this architecture, using the suggested Master/Query/Data server organization:
 
 <img src="../../img/druid-architecture.png" width="800"/>
 
@@ -186,29 +200,4 @@ So Druid uses three different techniques to maximize query performance:
 
 - 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.
-
-
-# External Dependencies
-
-## Deep storage
-
-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 more details, please see [Deep storage dependency](../dependencies/deep-storage.html).
-
-## Metadata storage
-
-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)
-
-## Zookeeper
-
-Druid uses [ZooKeeper](http://zookeeper.apache.org/) (ZK) for management of current cluster state.
-
-For more details, please see [Zookeeper dependency](../dependencies/zookeeper.html).
+- Within each segment, only reading the specific rows and columns that are relevant to a particular query.
\ No newline at end of file
diff --git a/docs/content/design/indexing-service.md b/docs/content/design/indexing-service.md
index f11ab5e..0d583b7 100644
--- a/docs/content/design/indexing-service.md
+++ b/docs/content/design/indexing-service.md
@@ -28,8 +28,8 @@ The indexing service is a highly-available, distributed service that runs indexi
 
 Indexing tasks [tasks](../ingestion/tasks.html) create (and sometimes destroy) Druid [segments](../design/segments.html). The indexing service has a master/slave like architecture.
 
-The indexing service is composed of three main components: a [Peon](../design/peons.html) component that can run a single task, a [Middle Manager](../design/middlemanager.html) component that manages peons, and an [Overlord](../design/overlord.html) component that manages task distribution to middle managers.
-Overlords and middle managers may run on the same node or across multiple nodes while middle managers and peons always run on the same node.
+The indexing service is composed of three main components: a [Peon](../design/peons.html) component that can run a single task, a [Middle Manager](../design/middlemanager.html) component that manages Peons, and an [Overlord](../design/overlord.html) component that manages task distribution to MiddleManagers.
+Overlords and MiddleManagers may run on the same node or across multiple nodes while MiddleManagers and Peons always run on the same node.
 
 Tasks are managed using API endpoints on the Overlord service. Please see [Overlord Task API](../operations/api-reference.html#overlord-tasks) for more information.
 
diff --git a/docs/content/design/middlemanager.md b/docs/content/design/middlemanager.md
index 8d0f47f..fa5b191 100644
--- a/docs/content/design/middlemanager.md
+++ b/docs/content/design/middlemanager.md
@@ -34,8 +34,8 @@ For a list of API endpoints supported by the MiddleManager, please see the [API
 
 ### Overview
 
-The middle manager node is a worker node that executes submitted tasks. Middle Managers forward tasks to peons that run in separate JVMs.
-The reason we have separate JVMs for tasks is for resource and log isolation. Each [Peon](../design/peons.html) is capable of running only one task at a time, however, a middle manager may have multiple peons.
+The MiddleManager node is a worker node that executes submitted tasks. Middle Managers forward tasks to Peons that run in separate JVMs.
+The reason we have separate JVMs for tasks is for resource and log isolation. Each [Peon](../design/peons.html) is capable of running only one task at a time, however, a MiddleManager may have multiple Peons.
 
 ### Running
 
diff --git a/docs/content/design/overlord.md b/docs/content/design/overlord.md
index d96c98d..fb5cf56 100644
--- a/docs/content/design/overlord.md
+++ b/docs/content/design/overlord.md
@@ -34,14 +34,14 @@ For a list of API endpoints supported by the Overlord, please see the [API refer
 
 ### Overview
 
-The overlord node is responsible for accepting tasks, coordinating task distribution, creating locks around tasks, and returning statuses to callers. Overlord can be configured to run in one of two modes - local or remote (local being default).
-In local mode overlord is also responsible for creating peons for executing tasks. When running the overlord in local mode, all middle manager and peon configurations must be provided as well.
-Local mode is typically used for simple workflows.  In remote mode, the overlord and middle manager are run in separate processes and you can run each on a different server.
+The Overlord node is responsible for accepting tasks, coordinating task distribution, creating locks around tasks, and returning statuses to callers. Overlord can be configured to run in one of two modes - local or remote (local being default).
+In local mode Overlord is also responsible for creating Peons for executing tasks. When running the Overlord in local mode, all MiddleManager and Peon configurations must be provided as well.
+Local mode is typically used for simple workflows.  In remote mode, the Overlord and MiddleManager are run in separate processes and you can run each on a different server.
 This mode is recommended if you intend to use the indexing service as the single endpoint for all Druid indexing.
 
 ### Overlord Console
 
-The overlord console can be used to view pending tasks, running tasks, available workers, and recent worker creation and termination. The console can be accessed at:
+The Overlord console can be used to view pending tasks, running tasks, available workers, and recent worker creation and termination. The console can be accessed at:
 
 ```
 http://<OVERLORD_IP>:<port>/console.html
@@ -49,7 +49,7 @@ http://<OVERLORD_IP>:<port>/console.html
 
 ### Blacklisted Workers
 
-If the workers fail tasks above a threshold, the overlord will blacklist these workers. No more than 20% of the nodes can be blacklisted. Blacklisted nodes will be periodically whitelisted.
+If the workers fail tasks above a threshold, the Overlord will blacklist these workers. No more than 20% of the nodes can be blacklisted. Blacklisted nodes will be periodically whitelisted.
 
 The following vairables can be used to set the threshold and blacklist timeouts.
 
@@ -62,6 +62,6 @@ druid.indexer.runner.maxPercentageBlacklistWorkers
 
 ### Autoscaling
 
-The Autoscaling mechanisms currently in place are tightly coupled with our deployment infrastructure but the framework should be in place for other implementations. We are highly open to new implementations or extensions of the existing mechanisms. In our own deployments, middle manager nodes are Amazon AWS EC2 nodes and they are provisioned to register themselves in a [galaxy](https://github.com/ning/galaxy) environment.
+The Autoscaling mechanisms currently in place are tightly coupled with our deployment infrastructure but the framework should be in place for other implementations. We are highly open to new implementations or extensions of the existing mechanisms. In our own deployments, MiddleManager nodes are Amazon AWS EC2 nodes and they are provisioned to register themselves in a [galaxy](https://github.com/ning/galaxy) environment.
 
-If autoscaling is enabled, new middle managers may be added when a task has been in pending state for too long. Middle managers may be terminated if they have not run any tasks for a period of time.
+If autoscaling is enabled, new MiddleManagers may be added when a task has been in pending state for too long. Middle managers may be terminated if they have not run any tasks for a period of time.
diff --git a/docs/content/design/peons.md b/docs/content/design/peons.md
index 7008a10..5af3fd8 100644
--- a/docs/content/design/peons.md
+++ b/docs/content/design/peons.md
@@ -37,7 +37,7 @@ Peons should rarely (if ever for testing purposes) be run on their own.
 
 ### Running
 
-The peon should very rarely ever be run independent of the middle manager unless for development purposes.
+The Peon should very rarely ever be run independent of the MiddleManager unless for development purposes.
 
 ```
 org.apache.druid.cli.Main internal peon <task_file> <status_file>
diff --git a/docs/content/design/processes.md b/docs/content/design/processes.md
new file mode 100644
index 0000000..87730ec
--- /dev/null
+++ b/docs/content/design/processes.md
@@ -0,0 +1,129 @@
+---
+layout: doc_page
+title: "Druid Processes and Servers"
+---
+
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one
+  ~ or more contributor license agreements.  See the NOTICE file
+  ~ distributed with this work for additional information
+  ~ regarding copyright ownership.  The ASF licenses this file
+  ~ to you under the Apache License, Version 2.0 (the
+  ~ "License"); you may not use this file except in compliance
+  ~ with the License.  You may obtain a copy of the License at
+  ~
+  ~   http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing,
+  ~ software distributed under the License is distributed on an
+  ~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+  ~ KIND, either express or implied.  See the License for the
+  ~ specific language governing permissions and limitations
+  ~ under the License.
+  -->
+
+# Druid Processes and Servers
+
+## Process Types
+
+Druid has several process types:
+
+* [Coordinator](../design/coordinator.html)
+* [Overlord](../design/overlord.html)
+* [Broker](../design/broker.html)
+* [Historical](../design/historical.html)
+* [MiddleManager](../design/middlemanager.html) and [Peons](../design/peons.html)
+* [Router (Optional)](../development/router.html) 
+
+## Server Types
+
+Druid processes can be deployed any way you like, but for ease of deployment we suggest organizing them into three server types:
+
+* **Master**
+* **Query**
+* **Data**
+
+<img src="../../img/druid-architecture.png" width="800"/>
+
+This section describes the Druid processes and the suggested Master/Query/Data server organization, as shown in the architecture diagram above.
+
+### Master server
+
+A Master server manages data ingestion and availability: it is responsible for starting new ingestion jobs and coordinating availability of data on the "Data servers" described below.
+
+Within a Master server, functionality is split between two processes, the Coordinator and Overlord.
+
+#### Coordinator process
+
+[**Coordinator**](../design/coordinator.html) processes watch over the Historical processes on the Data servers. They are responsible for assigning segments to specific servers, and for ensuring segments are well-balanced across Historicals.
+
+#### Overlord process
+
+[**Overlord**](../design/overlord.html) processes watch over the MiddleManager processes on the Data servers and are the controllers of data ingestion into Druid. They are responsible for assigning ingestion tasks to MiddleManagers and for coordinating segment publishing.
+
+### Query server
+
+A Query server provides the endpoints that users and client applications interact with, routing queries to Data servers or other Query servers (and optionally proxied Master server requests as well).
+
+Within a Query server, functionality is split between two processes, the Broker and Router.
+
+#### Broker process
+
+[**Broker**](../design/broker.html) processes receive queries from external clients and forward those queries to Data servers. 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 processes on Data servers directly.
+
+#### Router process (optional)
+
+[**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.
+
+### Data server
+
+A Data server executes ingestion jobs and stores queryable data.
+
+Within a Data server, functionality is split between two processes, the Historical and MiddleManager.
+
+### Historical process
+
+[**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.
+
+### Middle Manager process
+
+[**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.
+
+#### Peon processes
+
+[**Peon**](../design/peons.html) processes are task execution engines spawned by MiddleManagers. Each Peon runs a separate JVM and is responsible for executing a single task. Peons always run on the same host as the MiddleManager that spawned them.
+
+## Pros and cons of colocation
+
+Druid processes can be colocated based on the Master/Data/Query server organization as
+described above. This organization generally results in better utilization of
+hardware resources for most clusters.
+
+For very large scale clusters, however, it can be desirable to split the Druid processes
+such that they run on individual servers to avoid resource contention.
+
+This section describes guidelines and configuration parameters related to process colocation.
+
+### Coordinators and Overlords
+
+The workload on the Coordinator process tends to increase with the number of segments in the cluster. The Overlord's workload also increases based on the number of segments in the cluster, but to a lesser degree than the Coordinator.
+
+In clusters with very high segment counts, it can make sense to separate the Coordinator and Overlord processes to provide more resources for the Coordinator's segment balancing workload.
+
+#### Unified Process
+
+The Coordinator and Overlord processes can be run as a single combined process by setting the `druid.coordinator.asOverlord.enabled` property.
+
+Please see [Coordinator Configuration: Operation](../configuration/index.html#coordinator-operation) for details.
+
+### Historicals and MiddleManagers
+
+With higher levels of ingestion or query load, it can make sense to deploy the Historical and MiddleManager processes on separate nodes to to avoid CPU and memory contention. 
+
+The Historical also benefits from having free memory for memory mapped segments, which can be another reason to deploy the Historical and MiddleManager processes separately.
\ No newline at end of file
diff --git a/docs/content/development/extensions-contrib/materialized-view.md b/docs/content/development/extensions-contrib/materialized-view.md
index 01198a3..65fa689 100644
--- a/docs/content/development/extensions-contrib/materialized-view.md
+++ b/docs/content/development/extensions-contrib/materialized-view.md
@@ -24,7 +24,7 @@ title: "Materialized View"
 
 # Materialized View
 
-To use this feature, make sure to only load materialized-view-selection on broker and load materialized-view-maintenance on overlord. In addtion, this feature currently requires a hadoop cluster.
+To use this feature, make sure to only load `materialized-view-selection` on Broker and load `materialized-view-maintenance` on Overlord. In addtion, this feature currently requires a Hadoop cluster.
 
 This feature enables Druid to greatly improve the query performance, especially when the query dataSource has a very large number of dimensions but the query only required several dimensions. This feature includes two parts. One is `materialized-view-maintenance`, and the other is `materialized-view-selection`.
 
@@ -76,7 +76,7 @@ A sample derivativeDataSource supervisor spec is shown below:
 |tuningConfig	|TuningConfig must be HadoopTuningConfig. See [Hadoop tuning config](../../ingestion/hadoop.html#tuningconfig).|yes|
 |dataSource	|The name of this derived dataSource. 	|no(default=baseDataSource-hashCode of supervisor)|
 |hadoopDependencyCoordinates	|A JSON array of Hadoop dependency coordinates that Druid will 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|
-|classpathPrefix	|Classpath that will be pre-appended for the peon process.	|no|
+|classpathPrefix	|Classpath that will be pre-appended for the Peon process.	|no|
 |context	|See below.	|no|
 
 **Context**
diff --git a/docs/content/development/extensions-core/druid-basic-security.md b/docs/content/development/extensions-core/druid-basic-security.md
index c812929..890b273 100644
--- a/docs/content/development/extensions-core/druid-basic-security.md
+++ b/docs/content/development/extensions-core/druid-basic-security.md
@@ -43,10 +43,10 @@ These configuration properties should be added to the common runtime properties
 ### Properties
 |Property|Description|Default|required|
 |--------|-----------|-------|--------|
-|`druid.auth.basic.common.pollingPeriod`|Defines in milliseconds how often nodes should poll the coordinator for the current authenticator/authorizer database state.|60000|No|
+|`druid.auth.basic.common.pollingPeriod`|Defines in milliseconds how often nodes should poll the Coordinator for the current authenticator/authorizer database state.|60000|No|
 |`druid.auth.basic.common.maxRandomDelay`|Defines in milliseconds the amount of random delay to add to the pollingPeriod, to spread polling requests across time.|6000|No|
-|`druid.auth.basic.common.maxSyncRetries`|Determines how many times a service will retry if the authentication/authorization database state sync with the coordinator fails.|10|No|
-|`druid.auth.basic.common.cacheDirectory`|If defined, snapshots of the basic Authenticator and Authorizer database caches will be stored on disk in this directory. If this property is defined, when a service is starting, it will attempt to initialize its caches from these on-disk snapshots, if the service is unable to initialize its state by communicating with the coordinator.|null|No|
+|`druid.auth.basic.common.maxSyncRetries`|Determines how many times a service will retry if the authentication/authorization database state sync with the Coordinator fails.|10|No|
+|`druid.auth.basic.common.cacheDirectory`|If defined, snapshots of the basic Authenticator and Authorizer database caches will be stored on disk in this directory. If this property is defined, when a service is starting, it will attempt to initialize its caches from these on-disk snapshots, if the service is unable to initialize its state by communicating with the Coordinator.|null|No|
 
 
 ### Creating an Authenticator
@@ -75,7 +75,7 @@ The configuration examples in the rest of this document will use "MyBasicAuthent
 |--------|-----------|-------|--------|
 |`druid.auth.authenticator.MyBasicAuthenticator.initialAdminPassword`|Initial [Password Provider](../../operations/password-provider.html) for the automatically created default admin user. If no password is specified, the default admin user will not be created. If the default admin user already exists, setting this property will not affect its password.|null|No|
 |`druid.auth.authenticator.MyBasicAuthenticator.initialInternalClientPassword`|Initial [Password Provider](../../operations/password-provider.html) for the default internal system user, used for internal node communication. If no password is specified, the default internal system user will not be created. If the default internal system user already exists, setting this property will not affect its password.|null|No|
-|`druid.auth.authenticator.MyBasicAuthenticator.enableCacheNotifications`|If true, the coordinator will notify Druid nodes whenever a configuration change to this Authenticator occurs, allowing them to immediately update their state without waiting for polling.|true|No|
+|`druid.auth.authenticator.MyBasicAuthenticator.enableCacheNotifications`|If true, the Coordinator will notify Druid nodes whenever a configuration change to this Authenticator occurs, allowing them to immediately update their state without waiting for polling.|true|No|
 |`druid.auth.authenticator.MyBasicAuthenticator.cacheNotificationTimeout`|The timeout in milliseconds for the cache notifications.|5000|No|
 |`druid.auth.authenticator.MyBasicAuthenticator.credentialIterations`|Number of iterations to use for password hashing.|10000|No|
 |`druid.auth.authenticator.MyBasicAuthenticator.authorizerName`|Authorizer that requests should be directed to|N/A|Yes|
@@ -116,7 +116,7 @@ druid.auth.authorizer.<authorizerName>.<authorizerProperty>
 #### Properties
 |Property|Description|Default|required|
 |--------|-----------|-------|--------|
-|`druid.auth.authorizer.MyBasicAuthorizer.enableCacheNotifications`|If true, the coordinator will notify Druid nodes whenever a configuration change to this Authorizer occurs, allowing them to immediately update their state without waiting for polling.|true|No|
+|`druid.auth.authorizer.MyBasicAuthorizer.enableCacheNotifications`|If true, the Coordinator will notify Druid nodes whenever a configuration change to this Authorizer occurs, allowing them to immediately update their state without waiting for polling.|true|No|
 |`druid.auth.authorizer.MyBasicAuthorizer.cacheNotificationTimeout`|The timeout in milliseconds for the cache notifications.|5000|No|
 
 ## Usage
@@ -302,7 +302,7 @@ There is only one possible resource name for the "STATE" config resource type, "
 |`/druid-internal/v1/segments/`|historical|
 |`/druid-internal/v1/segments/`|peon|
 |`/druid-internal/v1/segments/`|realtime|
-|`/status`|all nodes|
+|`/status`|all process types|
 
 ### HTTP methods
 
@@ -312,10 +312,10 @@ GET requires READ permission, while POST and DELETE require WRITE permission.
 
 ## Configuration Propagation
 
-To prevent excessive load on the coordinator, the Authenticator and Authorizer user/role database state is cached on each Druid node.
+To prevent excessive load on the Coordinator, the Authenticator and Authorizer user/role database state is cached on each Druid node.
 
-Each node will periodically poll the coordinator for the latest database state, controlled by the `druid.auth.basic.common.pollingPeriod` and `druid.auth.basic.common.maxRandomDelay` properties.
+Each node will periodically poll the Coordinator for the latest database state, controlled by the `druid.auth.basic.common.pollingPeriod` and `druid.auth.basic.common.maxRandomDelay` properties.
 
-When a configuration update occurs, the coordinator can optionally notify each node with the updated database state. This behavior is controlled by the `enableCacheNotifications` and `cacheNotificationTimeout` properties on Authenticators and Authorizers.
+When a configuration update occurs, the Coordinator can optionally notify each node with the updated database state. This behavior is controlled by the `enableCacheNotifications` and `cacheNotificationTimeout` properties on Authenticators and Authorizers.
 
 Note that because of the caching, changes made to the user/role database may not be immediately reflected at each Druid node.
diff --git a/docs/content/development/extensions-core/druid-kerberos.md b/docs/content/development/extensions-core/druid-kerberos.md
index decd384..c97004a 100644
--- a/docs/content/development/extensions-core/druid-kerberos.md
+++ b/docs/content/development/extensions-core/druid-kerberos.md
@@ -98,14 +98,14 @@ Druid internal nodes communicate with each other using an escalated http Client.
     curl --negotiate -u:anyUser -b ~/cookies.txt -c ~/cookies.txt -X POST -H'Content-Type: application/json' <HTTP_END_POINT>
     ```
 
-    e.g to send a query from file `query.json` to druid broker use this command -
+    e.g to send a query from file `query.json` to the Druid Broker use this command -
 
     ```
     curl --negotiate -u:anyUser -b ~/cookies.txt -c ~/cookies.txt -X POST -H'Content-Type: application/json'  http://broker-host:port/druid/v2/?pretty -d @query.json
     ```
     Note: Above command will authenticate the user first time using SPNego negotiate mechanism and store the authentication cookie in file. For subsequent requests the cookie will be used for authentication.
 
-## Accessing coordinator or overlord console from web browser
+## Accessing Coordinator or Overlord console from web browser
 To access Coordinator/Overlord console from browser you will need to configure your browser for SPNego authentication as follows -
 
 1. Safari - No configurations required.
diff --git a/docs/content/development/extensions-core/kafka-ingestion.md b/docs/content/development/extensions-core/kafka-ingestion.md
index af391bb..4753585 100644
--- a/docs/content/development/extensions-core/kafka-ingestion.md
+++ b/docs/content/development/extensions-core/kafka-ingestion.md
@@ -44,8 +44,8 @@ version 0.10.x or better before using this service. Refer <a href="https://kafka
 
 ## Submitting a Supervisor Spec
 
-The Kafka indexing service requires that the `druid-kafka-indexing-service` extension be loaded on both the overlord and the
-middle managers. A supervisor for a dataSource is started by submitting a supervisor spec via HTTP POST to
+The Kafka indexing service requires that the `druid-kafka-indexing-service` extension be loaded on both the Overlord and the
+MiddleManagers. A supervisor for a dataSource is started by submitting a supervisor spec via HTTP POST to
 `http://<OVERLORD_IP>:<OVERLORD_PORT>/druid/indexer/v1/supervisor`, for example:
 
 ```
@@ -271,7 +271,7 @@ spec to the create api.
 
 ### Capacity Planning
 
-Kafka indexing tasks run on middle managers and are thus limited by the resources available in the middle manager
+Kafka indexing tasks run on MiddleManagers and are thus limited by the resources available in the MiddleManager
 cluster. In particular, you should make sure that you have sufficient worker capacity (configured using the
 `druid.worker.capacity` property) to handle the configuration in the supervisor spec. Note that worker capacity is
 shared across all types of indexing tasks, so you should plan your worker capacity to handle your total indexing load
@@ -281,7 +281,7 @@ in data loss (assuming the tasks run before Kafka purges those offsets).
 
 A running task will normally be in one of two states: *reading* or *publishing*. A task will remain in reading state for
 `taskDuration`, at which point it will transition to publishing state. A task will remain in publishing state for as long
-as it takes to generate segments, push segments to deep storage, and have them be loaded and served by a historical node
+as it takes to generate segments, push segments to deep storage, and have them be loaded and served by a Historical node
 (or until `completionTimeout` elapses).
 
 The number of reading tasks is controlled by `replicas` and `taskCount`. In general, there will be `replicas * taskCount`
@@ -296,7 +296,7 @@ workerCapacity = 2 * replicas * taskCount
 
 This value is for the ideal situation in which there is at most one set of tasks publishing while another set is reading.
 In some circumstances, it is possible to have multiple sets of tasks publishing simultaneously. This would happen if the
-time-to-publish (generate segment, push to deep storage, loaded on historical) > `taskDuration`. This is a valid
+time-to-publish (generate segment, push to deep storage, loaded on Historical) > `taskDuration`. This is a valid
 scenario (correctness-wise) but requires additional worker capacity to support. In general, it is a good idea to have
 `taskDuration` be large enough that the previous set of tasks finishes publishing before the current set begins.
 
@@ -306,11 +306,11 @@ When a supervisor spec is submitted via the `POST /druid/indexer/v1/supervisor`
 configured metadata database. There can only be a single supervisor per dataSource, and submitting a second spec for
 the same dataSource will overwrite the previous one.
 
-When an overlord gains leadership, either by being started or as a result of another overlord failing, it will spawn
+When an Overlord gains leadership, either by being started or as a result of another Overlord failing, it will spawn
 a supervisor for each supervisor spec in the metadata database. The supervisor will then discover running Kafka indexing
 tasks and will attempt to adopt them if they are compatible with the supervisor's configuration. If they are not
 compatible because they have a different ingestion spec or partition allocation, the tasks will be killed and the
-supervisor will create a new set of tasks. In this way, the supervisors are persistent across overlord restarts and
+supervisor will create a new set of tasks. In this way, the supervisors are persistent across Overlord restarts and
 fail-overs.
 
 A supervisor is stopped via the `POST /druid/indexer/v1/supervisor/<supervisorId>/terminate` endpoint. This places a
@@ -322,7 +322,7 @@ return after all tasks have been signalled to stop but before the tasks finish p
 ### Schema/Configuration Changes
 
 Schema and configuration changes are handled by submitting the new supervisor spec via the same
-`POST /druid/indexer/v1/supervisor` endpoint used to initially create the supervisor. The overlord will initiate a
+`POST /druid/indexer/v1/supervisor` endpoint used to initially create the supervisor. The Overlord will initiate a
 graceful shutdown of the existing supervisor which will cause the tasks being managed by that supervisor to stop reading
 and begin publishing their segments. A new supervisor will then be started which will create a new set of tasks that
 will start reading from the offsets where the previous now-publishing tasks left off, but using the updated schema.
diff --git a/docs/content/development/extensions-core/kinesis-ingestion.md b/docs/content/development/extensions-core/kinesis-ingestion.md
index 8b3583b..d0c29de 100644
--- a/docs/content/development/extensions-core/kinesis-ingestion.md
+++ b/docs/content/development/extensions-core/kinesis-ingestion.md
@@ -37,8 +37,8 @@ currently designated as an *experimental feature* and is subject to the usual
 
 ## Submitting a Supervisor Spec
 
-The Kinesis indexing service requires that the `druid-kinesis-indexing-service` extension be loaded on both the overlord
-and the middle managers. A supervisor for a dataSource is started by submitting a supervisor spec via HTTP POST to
+The Kinesis indexing service requires that the `druid-kinesis-indexing-service` extension be loaded on both the Overlord
+and the MiddleManagers. A supervisor for a dataSource is started by submitting a supervisor spec via HTTP POST to
 `http://<OVERLORD_IP>:<OVERLORD_PORT>/druid/indexer/v1/supervisor`, for example:
 
 ```
@@ -272,7 +272,7 @@ spec to the create api.
 
 ### Capacity Planning
 
-Kinesis indexing tasks run on middle managers and are thus limited by the resources available in the middle manager
+Kinesis indexing tasks run on MiddleManagers and are thus limited by the resources available in the MiddleManager
 cluster. In particular, you should make sure that you have sufficient worker capacity (configured using the
 `druid.worker.capacity` property) to handle the configuration in the supervisor spec. Note that worker capacity is
 shared across all types of indexing tasks, so you should plan your worker capacity to handle your total indexing load
@@ -282,7 +282,7 @@ in data loss (assuming the tasks run before Kinesis purges those sequence number
 
 A running task will normally be in one of two states: *reading* or *publishing*. A task will remain in reading state for
 `taskDuration`, at which point it will transition to publishing state. A task will remain in publishing state for as long
-as it takes to generate segments, push segments to deep storage, and have them be loaded and served by a historical node
+as it takes to generate segments, push segments to deep storage, and have them be loaded and served by a Historical node
 (or until `completionTimeout` elapses).
 
 The number of reading tasks is controlled by `replicas` and `taskCount`. In general, there will be `replicas * taskCount`
@@ -297,7 +297,7 @@ workerCapacity = 2 * replicas * taskCount
 
 This value is for the ideal situation in which there is at most one set of tasks publishing while another set is reading.
 In some circumstances, it is possible to have multiple sets of tasks publishing simultaneously. This would happen if the
-time-to-publish (generate segment, push to deep storage, loaded on historical) > `taskDuration`. This is a valid
+time-to-publish (generate segment, push to deep storage, loaded on Historical) > `taskDuration`. This is a valid
 scenario (correctness-wise) but requires additional worker capacity to support. In general, it is a good idea to have
 `taskDuration` be large enough that the previous set of tasks finishes publishing before the current set begins.
 
@@ -307,11 +307,11 @@ When a supervisor spec is submitted via the `POST /druid/indexer/v1/supervisor`
 configured metadata database. There can only be a single supervisor per dataSource, and submitting a second spec for
 the same dataSource will overwrite the previous one.
 
-When an overlord gains leadership, either by being started or as a result of another overlord failing, it will spawn
+When an Overlord gains leadership, either by being started or as a result of another Overlord failing, it will spawn
 a supervisor for each supervisor spec in the metadata database. The supervisor will then discover running Kinesis indexing
 tasks and will attempt to adopt them if they are compatible with the supervisor's configuration. If they are not
 compatible because they have a different ingestion spec or shard allocation, the tasks will be killed and the
-supervisor will create a new set of tasks. In this way, the supervisors are persistent across overlord restarts and
+supervisor will create a new set of tasks. In this way, the supervisors are persistent across Overlord restarts and
 fail-overs.
 
 A supervisor is stopped via the `POST /druid/indexer/v1/supervisor/<supervisorId>/terminate` endpoint. This places a
@@ -323,7 +323,7 @@ return after all tasks have been signalled to stop but before the tasks finish p
 ### Schema/Configuration Changes
 
 Schema and configuration changes are handled by submitting the new supervisor spec via the same
-`POST /druid/indexer/v1/supervisor` endpoint used to initially create the supervisor. The overlord will initiate a
+`POST /druid/indexer/v1/supervisor` endpoint used to initially create the supervisor. The Overlord will initiate a
 graceful shutdown of the existing supervisor which will cause the tasks being managed by that supervisor to stop reading
 and begin publishing their segments. A new supervisor will then be started which will create a new set of tasks that
 will start reading from the sequence numbers where the previous now-publishing tasks left off, but using the updated schema.
diff --git a/docs/content/development/extensions-core/lookups-cached-global.md b/docs/content/development/extensions-core/lookups-cached-global.md
index d0fbb42..31e6f2f 100644
--- a/docs/content/development/extensions-core/lookups-cached-global.md
+++ b/docs/content/development/extensions-core/lookups-cached-global.md
@@ -95,7 +95,7 @@ The parameters are as follows
 
 If `firstCacheTimeout` is set to a non-zero value, it should be less than `druid.manager.lookups.hostUpdateTimeout`. If `firstCacheTimeout` is NOT set, then management is essentially asynchronous and does not know if a lookup succeeded or failed in starting. In such a case logs from the lookup nodes should be monitored for repeated failures.
 
-Proper functionality of globally cached lookups requires the following extension to be loaded on the broker, peon, and historical nodes:
+Proper functionality of globally cached lookups requires the following extension to be loaded on the Broker, Peon, and Historical nodes:
 `druid-lookups-cached-global`
 
 ## Example configuration
@@ -131,7 +131,7 @@ In a simple case where only one [tier](../../querying/lookups.html#dynamic-confi
 }
 ```
 
-Where the coordinator endpoint `/druid/coordinator/v1/lookups/realtime_customer2/country_code` should return
+Where the Coordinator endpoint `/druid/coordinator/v1/lookups/realtime_customer2/country_code` should return
 
 ```json
 {
@@ -160,8 +160,8 @@ Where the coordinator endpoint `/druid/coordinator/v1/lookups/realtime_customer2
 
 ## Cache Settings
 
-Lookups are cached locally on historical nodes. The following are settings used by the nodes which service queries when 
-setting namespaces (broker, peon, historical)
+Lookups are cached locally on Historical nodes. The following are settings used by the nodes which service queries when 
+setting namespaces (Broker, Peon, Historical)
 
 |Property|Description|Default|
 |--------|-----------|-------|
diff --git a/docs/content/development/extensions-core/s3.md b/docs/content/development/extensions-core/s3.md
index c7ca62a..cead6cd 100644
--- a/docs/content/development/extensions-core/s3.md
+++ b/docs/content/development/extensions-core/s3.md
@@ -37,7 +37,7 @@ The AWS SDK requires that the target region be specified. Two ways of doing this
 As an example, to set the region to 'us-east-1' through system properties:
 
 - Add `-Daws.region=us-east-1` to the jvm.config file for all Druid services.
-- Add `-Daws.region=us-east-1` to `druid.indexer.runner.javaOpts` in middleManager/runtime.properties so that the property will be passed to peon (worker) processes.
+- Add `-Daws.region=us-east-1` to `druid.indexer.runner.javaOpts` in middleManager/runtime.properties so that the property will be passed to Peon (worker) processes.
 
 |Property|Description|Default|
 |--------|-----------|-------|
diff --git a/docs/content/development/javascript.md b/docs/content/development/javascript.md
index faa09bd..1bfbff6 100644
--- a/docs/content/development/javascript.md
+++ b/docs/content/development/javascript.md
@@ -36,7 +36,7 @@ JavaScript can be used to extend Druid in a variety of ways:
 - [Post-aggregators](../querying/post-aggregations.html#javascript-post-aggregator)
 - [Input parsers](../ingestion/data-formats.html#javascript)
 - [Router strategy](../development/router.html#javascript)
-- [Worker select strategy](../configuration/indexing-service.html#javascript)
+- [Worker select strategy](../configuration/index.html#javascript-worker-select-strategy)
 
 JavaScript can be injected dynamically at runtime, making it convenient to rapidly prototype new functionality
 without needing to write and deploy Druid extensions.
diff --git a/docs/content/development/modules.md b/docs/content/development/modules.md
index 6b7b0d2..c8d6ba9 100644
--- a/docs/content/development/modules.md
+++ b/docs/content/development/modules.md
@@ -109,8 +109,8 @@ In addition to DataSegmentPusher and DataSegmentPuller, you can also bind:
 **WARNING!** This is not a formal procedure, but a collection of hints to validate if your new deep storage implementation is able do push, pull and kill segments.
 
 It's recommended to use batch ingestion tasks to validate your implementation.
-The segment will be automatically rolled up to historical note after ~20 seconds. 
-In this way, you can validate both push (at realtime node) and pull (at historical node) segments.
+The segment will be automatically rolled up to Historical note after ~20 seconds. 
+In this way, you can validate both push (at realtime node) and pull (at Historical node) segments.
 
 * DataSegmentPusher
 
@@ -118,9 +118,9 @@ Wherever your data storage (cloud storage service, distributed file system, etc.
 
 * DataSegmentPuller
 
-After ~20 secs your ingestion task ends, you should be able to see your historical node trying to load the new segment.
+After ~20 secs your ingestion task ends, you should be able to see your Historical node trying to load the new segment.
 
-The following example was retrieved from a historical node configured to use Azure for deep storage:
+The following example was retrieved from a Historical node configured to use Azure for deep storage:
 
 ```
 2015-04-14T02:42:33,450 INFO [ZkCoordinator-0] org.apache.druid.server.coordination.ZkCoordinator - New request[LOAD: dde_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00
diff --git a/docs/content/development/overview.md b/docs/content/development/overview.md
index 2637cb4..1e0d6f5 100644
--- a/docs/content/development/overview.md
+++ b/docs/content/development/overview.md
@@ -48,7 +48,7 @@ the query logic is to start from `QueryResource.java`.
 
 ## Coordination
 
-Most of the coordination logic for historical nodes is on the Druid coordinator. The starting point here is `DruidCoordinator.java`.  
+Most of the coordination logic for Historical nodes is on the Druid Coordinator. The starting point here is `DruidCoordinator.java`.  
 Most of the coordination logic for (real-time) ingestion is in the Druid indexing service. The starting point here is `OverlordResource.java`.
 
 ## Real-time Ingestion
diff --git a/docs/content/development/router.md b/docs/content/development/router.md
index 79592ca..018fba1 100644
--- a/docs/content/development/router.md
+++ b/docs/content/development/router.md
@@ -24,7 +24,7 @@ title: "Router Node"
 
 # Router Node
 
-You should only ever need the router node if you have a Druid cluster well into the terabyte range. The router node can be used to route queries to different broker nodes. By default, the broker routes queries based on how [Rules](../operations/rule-configuration.html) are set up. For example, if 1 month of recent data is loaded into a `hot` cluster, queries that fall within the recent month can be routed to a dedicated set of brokers. Queries outside this range are routed to another set [...]
+You should only ever need the Router node if you have a Druid cluster well into the terabyte range. The Router node can be used to route queries to different Broker nodes. By default, the broker routes queries based on how [Rules](../operations/rule-configuration.html) are set up. For example, if 1 month of recent data is loaded into a `hot` cluster, queries that fall within the recent month can be routed to a dedicated set of brokers. Queries outside this range are routed to another set [...]
 
 Running
 -------
@@ -36,7 +36,7 @@ org.apache.druid.cli.Main server router
 Example Production Configuration
 --------------------------------
 
-In this example, we have two tiers in our production cluster: `hot` and `_default_tier`. Queries for the `hot` tier are routed through the `broker-hot` set of brokers, and queries for the `_default_tier` are routed through the `broker-cold` set of brokers. If any exceptions or network problems occur, queries are routed to the `broker-cold` set of brokers. In our example, we are running with a c3.2xlarge EC2 node. We assume a `common.runtime.properties` already exists.
+In this example, we have two tiers in our production cluster: `hot` and `_default_tier`. Queries for the `hot` tier are routed through the `broker-hot` set of Brokers, and queries for the `_default_tier` are routed through the `broker-cold` set of Brokers. If any exceptions or network problems occur, queries are routed to the `broker-cold` set of brokers. In our example, we are running with a c3.2xlarge EC2 node. We assume a `common.runtime.properties` already exists.
 
 JVM settings:
 
@@ -74,7 +74,7 @@ druid.router.tierToBrokerMap={"hot":"druid:broker-hot","_default_tier":"druid:br
 druid.router.http.numConnections=50
 druid.router.http.readTimeout=PT5M
 
-# Number of threads used by the router proxy http client
+# Number of threads used by the Router proxy http client
 druid.router.http.numMaxThreads=100
 
 druid.server.http.numThreads=100
@@ -83,22 +83,22 @@ druid.server.http.numThreads=100
 Runtime Configuration
 ---------------------
 
-The router module uses several of the default modules in [Configuration](../configuration/index.html) and has the following set of configurations as well:
+The Router module uses several of the default modules in [Configuration](../configuration/index.html) and has the following set of configurations as well:
 
 |Property|Possible Values|Description|Default|
 |--------|---------------|-----------|-------|
-|`druid.router.defaultBrokerServiceName`|Any string.|The default broker to connect to in case service discovery fails.|druid/broker|
-|`druid.router.tierToBrokerMap`|An ordered JSON map of tiers to broker names. The priority of brokers is based on the ordering.|Queries for a certain tier of data are routed to their appropriate broker.|{"_default_tier": "<defaultBrokerServiceName>"}|
+|`druid.router.defaultBrokerServiceName`|Any string.|The default Broker to connect to in case service discovery fails.|druid/broker|
+|`druid.router.tierToBrokerMap`|An ordered JSON map of tiers to Broker names. The priority of Brokers is based on the ordering.|Queries for a certain tier of data are routed to their appropriate Broker.|{"_default_tier": "<defaultBrokerServiceName>"}|
 |`druid.router.defaultRule`|Any string.|The default rule for all datasources.|"_default"|
 |`druid.router.pollPeriod`|Any ISO8601 duration.|How often to poll for new rules.|PT1M|
 |`druid.router.strategies`|An ordered JSON array of objects.|All custom strategies to use for routing.|[{"type":"timeBoundary"},{"type":"priority"}]|
-|`druid.router.avatica.balancer.type`|String representing an AvaticaConnectionBalancer name|Class to use for balancing Avatica queries across brokers|rendezvousHash|
-|`druid.router.http.maxRequestBufferSize`|Maximum size of the buffer used to write requests when forwarding them to the broker. This should be set to atleast the maxHeaderSize allowed on the broker|8 * 1024|
+|`druid.router.avatica.balancer.type`|String representing an AvaticaConnectionBalancer name|Class to use for balancing Avatica queries across Brokers|rendezvousHash|
+|`druid.router.http.maxRequestBufferSize`|Maximum size of the buffer used to write requests when forwarding them to the Broker. This should be set to atleast the maxHeaderSize allowed on the Broker|8 * 1024|
 
 
 Router Strategies
 -----------------
-The router has a configurable list of strategies for how it selects which brokers to route queries to. The order of the strategies matter because as soon as a strategy condition is matched, a broker is selected.
+The Router has a configurable list of strategies for how it selects which Brokers to route queries to. The order of the strategies matter because as soon as a strategy condition is matched, a Broker is selected.
 
 ### timeBoundary
 
@@ -108,7 +108,7 @@ The router has a configurable list of strategies for how it selects which broker
 }
 ```
 
-Including this strategy means all timeBoundary queries are always routed to the highest priority broker.
+Including this strategy means all timeBoundary queries are always routed to the highest priority Broker.
 
 ### priority
 
@@ -120,13 +120,13 @@ Including this strategy means all timeBoundary queries are always routed to the
 }
 ```
 
-Queries with a priority set to less than minPriority are routed to the lowest priority broker. Queries with priority set to greater than maxPriority are routed to the highest priority broker. By default, minPriority is 0 and maxPriority is 1. Using these default values, if a query with priority 0 (the default query priority is 0) is sent, the query skips the priority selection logic.
+Queries with a priority set to less than minPriority are routed to the lowest priority Broker. Queries with priority set to greater than maxPriority are routed to the highest priority Broker. By default, minPriority is 0 and maxPriority is 1. Using these default values, if a query with priority 0 (the default query priority is 0) is sent, the query skips the priority selection logic.
 
 ### JavaScript
 
 Allows defining arbitrary routing rules using a JavaScript function. The function is passed the configuration and the query to be executed, and returns the tier it should be routed to, or null for the default tier.
 
-*Example*: a function that sends queries containing more than three aggregators to the lowest priority broker.
+*Example*: a function that sends queries containing more than three aggregators to the lowest priority Broker.
 
 ```json
 {
@@ -143,15 +143,15 @@ JavaScript-based functionality is disabled by default. Please refer to the Druid
 Avatica Query Balancing
 --------------
 
-All Avatica JDBC requests with a given connection ID must be routed to the same broker, since Druid brokers do not share connection state with each other.
+All Avatica JDBC requests with a given connection ID must be routed to the same Broker, since Druid Brokers do not share connection state with each other.
 
-To accomplish this, Druid provides two built-in balancers that use rendezvous hashing and consistent hashing of a request's connection ID respectively to assign requests to brokers.
+To accomplish this, Druid provides two built-in balancers that use rendezvous hashing and consistent hashing of a request's connection ID respectively to assign requests to Brokers.
 
-Note that when multiple routers are used, all routers should have identical balancer configuration to ensure that they make the same routing decisions.
+Note that when multiple Routers are used, all Routers should have identical balancer configuration to ensure that they make the same routing decisions.
 
 ### Rendezvous Hash Balancer
 
-This balancer uses [Rendezvous Hashing](https://en.wikipedia.org/wiki/Rendezvous_hashing) on an Avatica request's connection ID to assign the request to a broker.
+This balancer uses [Rendezvous Hashing](https://en.wikipedia.org/wiki/Rendezvous_hashing) on an Avatica request's connection ID to assign the request to a Broker.
 
 To use this balancer, specify the following property:
 
@@ -163,7 +163,7 @@ If no `druid.router.avatica.balancer` property is set, the Router will also defa
 
 ### Consistent Hash Balancer
 
-This balancer uses [Consistent Hashing](https://en.wikipedia.org/wiki/Consistent_hashing) on an Avatica request's connection ID to assign the request to a broker.
+This balancer uses [Consistent Hashing](https://en.wikipedia.org/wiki/Consistent_hashing) on an Avatica request's connection ID to assign the request to a Broker.
 
 To use this balancer, specify the following property:
 
@@ -171,12 +171,12 @@ To use this balancer, specify the following property:
 druid.router.avatica.balancer.type=consistentHash
 ```
 
-This is a non-default implementation that is provided for experimentation purposes. The consistent hasher has longer setup times on initialization and when the set of brokers changes, but has a faster broker assignment time than the rendezous hasher when tested with 5 brokers. Benchmarks for both implementations have been provided in `ConsistentHasherBenchmark` and `RendezvousHasherBenchmark`. The consistent hasher also requires locking, while the rendezvous hasher does not.
+This is a non-default implementation that is provided for experimentation purposes. The consistent hasher has longer setup times on initialization and when the set of Brokers changes, but has a faster Broker assignment time than the rendezous hasher when tested with 5 Brokers. Benchmarks for both implementations have been provided in `ConsistentHasherBenchmark` and `RendezvousHasherBenchmark`. The consistent hasher also requires locking, while the rendezvous hasher does not.
 
 HTTP Endpoints
 --------------
 
-The router node exposes several HTTP endpoints for interactions.
+The Router node exposes several HTTP endpoints for interactions.
 
 ### GET
 
@@ -203,14 +203,14 @@ Returns the metrics of the datasource.
 Router as Management Proxy
 --------------------------
 
-The router can be configured to forward requests to the active coordinator or overlord node. This may be useful for
+The Router can be configured to forward requests to the active Coordinator or Overlord node. This may be useful for
 setting up a highly available cluster in situations where the HTTP redirect mechanism of the inactive -> active
-coordinator/overlord does not function correctly (servers are behind a load balancer, the hostname used in the redirect
+Coordinator/Overlord does not function correctly (servers are behind a load balancer, the hostname used in the redirect
 is only resolvable internally, etc.).
 
 ### Enabling the Management Proxy
 
-To enable this functionality, set the following in the router's runtime.properties:
+To enable this functionality, set the following in the Router's runtime.properties:
 
 ```
 druid.router.managementProxy.enabled=true
@@ -219,13 +219,13 @@ druid.router.managementProxy.enabled=true
 ### Routing
 
 The management proxy supports implicit and explicit routes. Implicit routes are those where the destination can be
-determined from the original request path based on Druid API path conventions. For the coordinator the convention is
-`/druid/coordinator/*` and for the overlord the convention is `/druid/indexer/*`. These are convenient because they mean
-that using the management proxy does not require modifying the API request other than issuing the request to the router
-instead of the coordinator or overlord. Most Druid API requests can be routed implicitly.
+determined from the original request path based on Druid API path conventions. For the Coordinator the convention is
+`/druid/coordinator/*` and for the Overlord the convention is `/druid/indexer/*`. These are convenient because they mean
+that using the management proxy does not require modifying the API request other than issuing the request to the Router
+instead of the Coordinator or Overlord. Most Druid API requests can be routed implicitly.
 
-Explicit routes are those where the request to the router contains a path prefix indicating which node the request
-should be routed to. For the coordinator this prefix is `/proxy/coordinator` and for the overlord it is `/proxy/overlord`.
+Explicit routes are those where the request to the Router contains a path prefix indicating which node the request
+should be routed to. For the Coordinator this prefix is `/proxy/coordinator` and for the Overlord it is `/proxy/overlord`.
 This is required for API calls with an ambiguous destination. For example, the `/status` API is present on all Druid
 nodes, so explicit routing needs to be used to indicate the proxy destination.
 
diff --git a/docs/content/ingestion/faq.md b/docs/content/ingestion/faq.md
index 13a0a95..e9584e8 100644
--- a/docs/content/ingestion/faq.md
+++ b/docs/content/ingestion/faq.md
@@ -56,19 +56,19 @@ Other common reasons that hand-off fails are as follows:
 
 1) Druid is unable to write to the metadata storage. Make sure your configurations are correct.
 
-2) Historical nodes are out of capacity and cannot download any more segments. You'll see exceptions in the coordinator logs if this occurs and the coordinator console will show the historicals are near capacity.
+2) Historical nodes are out of capacity and cannot download any more segments. You'll see exceptions in the Coordinator logs if this occurs and the Coordinator console will show the Historicals are near capacity.
 
-3) Segments are corrupt and cannot be downloaded. You'll see exceptions in your historical nodes if this occurs.
+3) Segments are corrupt and cannot be downloaded. You'll see exceptions in your Historical nodes if this occurs.
 
-4) Deep storage is improperly configured. Make sure that your segment actually exists in deep storage and that the coordinator logs have no errors.
+4) Deep storage is improperly configured. Make sure that your segment actually exists in deep storage and that the Coordinator logs have no errors.
 
 ## How do I get HDFS to work?
 
 Make sure to include the `druid-hdfs-storage` and all the hadoop configuration, dependencies (that can be obtained by running command `hadoop classpath` on a machine where hadoop has been setup) in the classpath. And, provide necessary HDFS settings as described in [Deep Storage](../dependencies/deep-storage.html) .
 
-## I don't see my Druid segments on my historical nodes
+## I don't see my Druid segments on my Historical nodes
 
-You can check the coordinator console located at `<COORDINATOR_IP>:<PORT>`. Make sure that your segments have actually loaded on [historical nodes](../design/historical.html). If your segments are not present, check the coordinator logs for messages about capacity of replication errors. One reason that segments are not downloaded is because historical nodes have maxSizes that are too small, making them incapable of downloading more data. You can change that with (for example):
+You can check the Coordinator console located at `<COORDINATOR_IP>:<PORT>`. Make sure that your segments have actually loaded on [Historical nodes](../design/historical.html). If your segments are not present, check the Coordinator logs for messages about capacity of replication errors. One reason that segments are not downloaded is because Historical nodes have maxSizes that are too small, making them incapable of downloading more data. You can change that with (for example):
 
 ```
 -Ddruid.segmentCache.locations=[{"path":"/tmp/druid/storageLocation","maxSize":"500000000000"}]
diff --git a/docs/content/ingestion/hadoop.md b/docs/content/ingestion/hadoop.md
index 7ae0eae..4f8174c 100644
--- a/docs/content/ingestion/hadoop.md
+++ b/docs/content/ingestion/hadoop.md
@@ -25,7 +25,7 @@ title: "Hadoop-based Batch Ingestion"
 # Hadoop-based Batch Ingestion
 
 Hadoop-based batch ingestion in Druid is supported via a Hadoop-ingestion task. These tasks can be posted to a running
-instance of a Druid [overlord](../design/overlord.html). 
+instance of a Druid [Overlord](../design/overlord.html). 
 
 ## Command Line Hadoop Indexer
 
@@ -105,7 +105,7 @@ A sample task is shown below:
 |type|The task type, this should always be "index_hadoop".|yes|
 |spec|A Hadoop Index Spec. See [Ingestion](../ingestion/ingestion-spec.html)|yes|
 |hadoopDependencyCoordinates|A JSON array of Hadoop dependency coordinates that Druid will 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|
-|classpathPrefix|Classpath that will be pre-appended for the peon process.|no|
+|classpathPrefix|Classpath that will be pre-appended for the Peon process.|no|
 
 also note that, druid automatically computes the classpath for hadoop job containers that run in hadoop cluster. But, in case of conflicts between hadoop and druid's dependencies, you can manually specify the classpath by setting `druid.extensions.hadoopContainerDruidClasspath` property. See the extensions config in [base druid configuration](../configuration/index.html#extensions).
 
diff --git a/docs/content/ingestion/index.md b/docs/content/ingestion/index.md
index fa473d9..db1edfa 100644
--- a/docs/content/ingestion/index.md
+++ b/docs/content/ingestion/index.md
@@ -291,9 +291,9 @@ For compaction documentation, please see [tasks](../ingestion/tasks.html).
 
 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.
+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.
+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).
 
diff --git a/docs/content/ingestion/locking-and-priority.md b/docs/content/ingestion/locking-and-priority.md
index 523ed77..0942466 100644
--- a/docs/content/ingestion/locking-and-priority.md
+++ b/docs/content/ingestion/locking-and-priority.md
@@ -26,7 +26,7 @@ title: "Task Locking & Priority"
 
 ## Locking
 
-Once an overlord node accepts a task, the task acquires locks for the data source and intervals specified in the task.
+Once an Overlord node accepts a task, the task acquires locks for the data source and intervals specified in the task.
 
 There are two lock types, i.e., _shared lock_ and _exclusive lock_.
 
@@ -75,5 +75,5 @@ The task context is used for various task configuration parameters. The followin
 
 <div class="note caution">
 When a task acquires a lock, it sends a request via HTTP and awaits until it receives a response containing the lock acquisition result.
-As a result, an HTTP timeout error can occur if `taskLockTimeout` is greater than `druid.server.http.maxIdleTime` of overlords.
+As a result, an HTTP timeout error can occur if `taskLockTimeout` is greater than `druid.server.http.maxIdleTime` of Overlords.
 </div>
diff --git a/docs/content/ingestion/native_tasks.md b/docs/content/ingestion/native_tasks.md
index 1f2c453..e5b2e7d 100644
--- a/docs/content/ingestion/native_tasks.md
+++ b/docs/content/ingestion/native_tasks.md
@@ -25,8 +25,8 @@ title: "Native Index Tasks"
 # Native Index Tasks
 
 Druid currently has two types of native batch indexing tasks, `index_parallel` which runs tasks
-in parallel on multiple middle manager nodes, and `index` which will run a single indexing task locally on a single
-middle manager.
+in parallel on multiple MiddleManager nodes, and `index` which will run a single indexing task locally on a single
+MiddleManager.
 
 Parallel Index Task
 --------------------------------
@@ -34,7 +34,7 @@ Parallel Index Task
 The Parallel Index Task is a task for parallel batch indexing. This task only uses Druid's resource and
 doesn't depend on other external systems like Hadoop. This task currently works in a single phase without shuffling intermediate
 data. `index_parallel` task is a supervisor task which basically generates multiple worker tasks and submits
-them to overlords. Each worker task reads input data and makes segments. Once they successfully generate segments for all
+them to Overlords. Each worker task reads input data and makes segments. Once they successfully generate segments for all
 input, they report the generated segment list to the supervisor task. The supervisor task periodically checks the worker
 task statuses. If one of them fails, it retries the failed task until the retrying number reaches the configured limit.
 If all worker tasks succeed, then it collects the reported list of generated segments and publishes those segments at once.
diff --git a/docs/content/ingestion/reports.md b/docs/content/ingestion/reports.md
index de37cd1..8696374 100644
--- a/docs/content/ingestion/reports.md
+++ b/docs/content/ingestion/reports.md
@@ -90,7 +90,7 @@ The `errorMsg` field shows a message describing the error that caused a task to
 
 The non-parallel [Native Batch Task](../native_tasks.md), the Hadoop batch task, and the tasks created by the Kafka Indexing Service support retrieval of row stats while the task is running.
 
-The live report can be accessed with a GET to the following URL on a peon running a task:
+The live report can be accessed with a GET to the following URL on a Peon running a task:
 
 ```
 http://<middlemanager-host>:<worker-port>/druid/worker/v1/chat/<task-id>/rowStats
@@ -143,7 +143,7 @@ http://<OVERLORD-HOST>:<OVERLORD-PORT>/druid/indexer/v1/supervisor/<supervisor-i
 
 ### Unparseable Events
 
-Current lists of unparseable events can be retrieved from a running task with a GET to the following peon API:
+Current lists of unparseable events can be retrieved from a running task with a GET to the following Peon API:
 
 ```
 http://<middlemanager-host>:<worker-port>/druid/worker/v1/chat/<task-id>/unparseableEvents
diff --git a/docs/content/ingestion/stream-pull.md b/docs/content/ingestion/stream-pull.md
index d3b7425..0ec8b7f 100644
--- a/docs/content/ingestion/stream-pull.md
+++ b/docs/content/ingestion/stream-pull.md
@@ -336,7 +336,7 @@ data segments on each of the N nodes, which is not possible with standard Kafka
 
 For example, let's say your topic is split across Kafka partitions 1, 2, & 3 and you have 2 real-time nodes with linear shard specs 1 & 2.
 Both of the real-time nodes are in the same consumer group. Real-time node 1 may consume data from partitions 1 & 3, and real-time node 2 may consume data from partition 2.
-Querying for your data through the broker will yield correct results.
+Querying for your data through the Broker will yield correct results.
 
 The problem arises if you want to replicate your data by creating real-time nodes 3 & 4. These new real-time nodes also
 have linear shard specs 1 & 2, and they will consume data from Kafka using a different consumer group. In this case,
diff --git a/docs/content/ingestion/stream-push.md b/docs/content/ingestion/stream-push.md
index b31b3ab..9552499 100644
--- a/docs/content/ingestion/stream-push.md
+++ b/docs/content/ingestion/stream-push.md
@@ -41,8 +41,8 @@ time). Older messages will not be processed in real-time. Historical data is bes
 ### Server
 
 Druid can use [Tranquility Server](https://github.com/druid-io/tranquility/blob/master/docs/server.md), which
-lets you send data to Druid without developing a JVM app. You can run Tranquility server colocated with Druid middleManagers
-and historical processes.
+lets you send data to Druid without developing a JVM app. You can run Tranquility server colocated with Druid MiddleManagers
+and Historical processes.
 
 Tranquility server is started by issuing:
 
@@ -125,7 +125,7 @@ The windowPeriod is the slack time permitted for events. For example, a windowPe
 than ten minutes in the future, will be dropped.
 
 These are important configurations because they influence how long tasks will be alive for, and how
-long data stays in the realtime system before being handed off to the historical nodes. For example,
+long data stays in the realtime system before being handed off to the Historical nodes. For example,
 if your configuration has segmentGranularity "hour" and windowPeriod ten minutes, tasks will stay
 around listening for events for an hour and ten minutes. For this reason, to prevent excessive
 buildup of tasks, it is recommended that your windowPeriod be less than your segmentGranularity.
diff --git a/docs/content/ingestion/tasks.md b/docs/content/ingestion/tasks.md
index 4c55b0e..41f7b52 100644
--- a/docs/content/ingestion/tasks.md
+++ b/docs/content/ingestion/tasks.md
@@ -24,7 +24,7 @@ title: "Tasks Overview"
 
 # Tasks Overview
 
-Tasks are run on middle managers and always operate on a single data source.
+Tasks are run on MiddleManagers and always operate on a single data source.
 
 Tasks are submitted using POST requests to the Overlord. Please see [Overlord Task API](../operations/api-reference.html#overlord-tasks) for API details.
 
diff --git a/docs/content/ingestion/update-existing-data.md b/docs/content/ingestion/update-existing-data.md
index 506e5b3..0575726 100644
--- a/docs/content/ingestion/update-existing-data.md
+++ b/docs/content/ingestion/update-existing-data.md
@@ -73,7 +73,7 @@ Here is what goes inside `ingestionSpec`:
 |-----|----|-----------|--------|
 |dataSource|String|Druid dataSource name from which you are loading the data.|yes|
 |intervals|List|A list of strings representing ISO-8601 Intervals.|yes|
-|segments|List|List of segments from which to read data from, by default it is obtained automatically. You can obtain list of segments to put here by making a POST query to coordinator at url /druid/coordinator/v1/metadata/datasources/segments?full with list of intervals specified in the request paylod e.g. ["2012-01-01T00:00:00.000/2012-01-03T00:00:00.000", "2012-01-05T00:00:00.000/2012-01-07T00:00:00.000"]. You may want to provide this list manually in order to ensure that segments rea [...]
+|segments|List|List of segments from which to read data from, by default it is obtained automatically. You can obtain list of segments to put here by making a POST query to Coordinator at url /druid/coordinator/v1/metadata/datasources/segments?full with list of intervals specified in the request paylod e.g. ["2012-01-01T00:00:00.000/2012-01-03T00:00:00.000", "2012-01-05T00:00:00.000/2012-01-07T00:00:00.000"]. You may want to provide this list manually in order to ensure that segments rea [...]
 |filter|JSON|See [Filters](../querying/filters.html)|no|
 |dimensions|Array of String|Name of dimension columns to load. By default, the list will be constructed from parseSpec. If parseSpec does not have an explicit list of dimensions then all the dimension columns present in stored data will be read.|no|
 |metrics|Array of String|Name of metric columns to load. By default, the list will be constructed from the "name" of all the configured aggregators.|no|
@@ -148,7 +148,7 @@ For example:
 }
 ```
 
-It is STRONGLY RECOMMENDED to provide list of segments in `dataSource` inputSpec explicitly so that your delta ingestion task is idempotent. You can obtain that list of segments by making following call to the coordinator.
+It is STRONGLY RECOMMENDED to provide list of segments in `dataSource` inputSpec explicitly so that your delta ingestion task is idempotent. You can obtain that list of segments by making following call to the Coordinator.
 POST `/druid/coordinator/v1/metadata/datasources/{dataSourceName}/segments?full`
 Request Body: [interval1, interval2,...] for example ["2012-01-01T00:00:00.000/2012-01-03T00:00:00.000", "2012-01-05T00:00:00.000/2012-01-07T00:00:00.000"]
 
diff --git a/docs/content/operations/api-reference.md b/docs/content/operations/api-reference.md
index 8389eb0..7a0b434 100644
--- a/docs/content/operations/api-reference.md
+++ b/docs/content/operations/api-reference.md
@@ -28,12 +28,15 @@ This page documents all of the API endpoints for each Druid service type.
 
 ## Table of Contents
   * [Common](#common)
-  * [Coordinator](#coordinator)
-  * [Overlord](#overlord)
-  * [MiddleManager](#middlemanager)
-  * [Peon](#peon)
-  * [Broker](#broker)
-  * [Historical](#historical)
+  * [Master Server](#master-server)
+    * [Coordinator](#coordinator)
+    * [Overlord](#overlord)
+  * [Data Server](#data-server)
+    * [MiddleManager](#middlemanager)
+    * [Peon](#peon)
+    * [Historical](#historical)
+  * [Query Server](#query-server)
+    * [Broker](#broker)
 
 ## Common
 
@@ -55,26 +58,30 @@ An endpoint that always returns a boolean "true" value with a 200 OK response, u
 
 Returns the current configuration properties of the node.
 
-## Coordinator
+## Master Server
 
-### Leadership
+This section documents the API endpoints for the processes that reside on Master servers (Coordinators and Overlords) in the suggested [three-server configuration](../design/processes.html#server-types).
 
-#### GET
+### Coordinator
+
+#### Leadership
+
+##### GET
 
 * `/druid/coordinator/v1/leader`
 
-Returns the current leader coordinator of the cluster.
+Returns the current leader Coordinator of the cluster.
 
 * `/druid/coordinator/v1/isLeader`
 
-Returns true if the coordinator receiving the request is the current leader.
+Returns true if the Coordinator receiving the request is the current leader.
 
-### Segment Loading
+#### Segment Loading
 
-#### GET
+##### GET
 
 Returns a JSON object with field "leader", either true or false, indicating if this server is the current leader
-coordinator of the cluster. In addition, returns HTTP 200 if the server is the current leader and HTTP 404 if not.
+Coordinator of the cluster. In addition, returns HTTP 200 if the server is the current leader and HTTP 404 if not.
 This is suitable for use as a load balancer status check if you only want the active leader to be considered in-service
 at the load balancer.
 
@@ -92,19 +99,19 @@ Returns the number of segments left to load in each tier until segments that sho
 
 * `/druid/coordinator/v1/loadqueue`
 
-Returns the ids of segments to load and drop for each historical node.
+Returns the ids of segments to load and drop for each Historical node.
 
 * `/druid/coordinator/v1/loadqueue?simple`
 
-Returns the number of segments to load and drop, as well as the total segment load and drop size in bytes for each historical node.
+Returns the number of segments to load and drop, as well as the total segment load and drop size in bytes for each Historical node.
 
 * `/druid/coordinator/v1/loadqueue?full`
 
-Returns the serialized JSON of segments to load and drop for each historical node.
+Returns the serialized JSON of segments to load and drop for each Historical node.
 
-### Metadata store information
+#### Metadata store information
 
-#### GET
+##### GET
 
 * `/druid/coordinator/v1/metadata/datasources`
 
@@ -134,7 +141,7 @@ Returns a list of all segments for a datasource with the full segment metadata a
 
 Returns full segment metadata for a specific segment as stored in the metadata store.
 
-#### POST
+##### POST
 
 * `/druid/coordinator/v1/metadata/datasources/{dataSourceName}/segments`
 
@@ -144,10 +151,9 @@ Returns a list of all segments, overlapping with any of given intervals,  for a
 
 Returns a list of all segments, overlapping with any of given intervals, for a datasource with the full segment metadata as stored in the metadata store. Request body is array of string intervals like [interval1, interval2,...] for example ["2012-01-01T00:00:00.000/2012-01-03T00:00:00.000", "2012-01-05T00:00:00.000/2012-01-07T00:00:00.000"]
 
+#### Datasources
 
-### Datasources
-
-#### GET
+##### GET
 
 * `/druid/coordinator/v1/datasources`
 
@@ -213,7 +219,7 @@ Returns full segment metadata for a specific segment in the cluster.
 
 Return the tiers that a datasource exists in.
 
-#### POST
+##### POST
 
 * `/druid/coordinator/v1/datasources/{dataSourceName}`
 
@@ -223,7 +229,7 @@ Enables all segments of datasource which are not overshadowed by others.
 
 Enables a segment of a datasource.
 
-#### DELETE<a name="coordinator-delete"></a>
+##### DELETE<a name="coordinator-delete"></a>
 
 * `/druid/coordinator/v1/datasources/{dataSourceName}`
 
@@ -240,9 +246,9 @@ Note that {interval} parameters are delimited by a `_` instead of a `/` (e.g., 2
 
 Disables a segment.
 
-### Retention Rules
+#### Retention Rules
 
-#### GET
+##### GET
 
 * `/druid/coordinator/v1/rules`
 
@@ -259,7 +265,7 @@ Returns all rules for a specified datasource and includes default datasource.
 
 * `/druid/coordinator/v1/rules/history?interval=<interval>`
 
- Returns audit history of rules for all datasources. default value of interval can be specified by setting `druid.audit.manager.auditHistoryMillis` (1 week if not configured) in coordinator runtime.properties
+ Returns audit history of rules for all datasources. default value of interval can be specified by setting `druid.audit.manager.auditHistoryMillis` (1 week if not configured) in Coordinator runtime.properties
 
 * `/druid/coordinator/v1/rules/history?count=<n>`
 
@@ -267,13 +273,13 @@ Returns all rules for a specified datasource and includes default datasource.
 
 * `/druid/coordinator/v1/rules/{dataSourceName}/history?interval=<interval>`
 
- Returns audit history of rules for a specified datasource. default value of interval can be specified by setting `druid.audit.manager.auditHistoryMillis` (1 week if not configured) in coordinator runtime.properties
+ Returns audit history of rules for a specified datasource. default value of interval can be specified by setting `druid.audit.manager.auditHistoryMillis` (1 week if not configured) in Coordinator runtime.properties
 
 * `/druid/coordinator/v1/rules/{dataSourceName}/history?count=<n>`
 
  Returns last <n> entries of audit history of rules for a specified datasource.
  
-#### POST
+##### POST
 
 * `/druid/coordinator/v1/rules/{dataSourceName}`
 
@@ -286,9 +292,9 @@ Optional Header Parameters for auditing the config change can also be specified.
 |`X-Druid-Author`| author making the config change|""|
 |`X-Druid-Comment`| comment describing the change being done|""|
 
-### Intervals
+#### Intervals
 
-#### GET
+##### GET
 
 Note that {interval} parameters are delimited by a `_` instead of a `/` (e.g., 2016-06-27_2016-06-28).
 
@@ -308,9 +314,9 @@ Returns total size and count for each interval within given isointerval.
 
 Returns total size and count for each datasource for each interval within given isointerval.
 
-### Compaction Configuration
+#### Compaction Configuration
 
-#### GET
+##### GET
 
 * `/druid/coordinator/v1/config/compaction`
 
@@ -320,7 +326,7 @@ Returns all compaction configs.
 
 Returns a compaction config of a dataSource.
 
-#### POST
+##### POST
 
 * `/druid/coordinator/v1/config/compaction/taskslots?ratio={someRatio}&max={someMaxSlots}`
 
@@ -334,15 +340,15 @@ will be set for them.
 
 Creates or updates the compaction config for a dataSource. See [Compaction Configuration](../configuration/index.html#compaction-dynamic-configuration) for configuration details.
 
-#### DELETE
+##### DELETE
 
 * `/druid/coordinator/v1/config/compaction/{dataSource}`
 
 Removes the compaction config for a dataSource.
 
-### Server Information
+#### Server Information
 
-#### GET
+##### GET
 
 * `/druid/coordinator/v1/servers`
 
@@ -360,15 +366,15 @@ Returns a list of server data objects in which each object has the following key
 - `priority`
 - `tier`
 
-## Overlord
+### Overlord
 
-### Leadership
+#### Leadership
 
-#### GET
+##### GET
 
 * `/druid/indexer/v1/leader` 
 
-Returns the current leader overlord of the cluster. If you have multiple overlords, just one is leading at any given time. The others are on standby.
+Returns the current leader Overlord of the cluster. If you have multiple Overlords, just one is leading at any given time. The others are on standby.
 
 * `/druid/indexer/v1/isLeader`
 
@@ -376,9 +382,9 @@ This returns a JSON object with field "leader", either true or false. In additio
 server is the current leader and HTTP 404 if not. This is suitable for use as a load balancer status check if you
 only want the active leader to be considered in-service at the load balancer.
 
-### Tasks<a name="overlord-tasks"></a> 
+#### Tasks<a name="overlord-tasks"></a> 
 
-#### GET
+##### GET
 
 * `/druid/indexer/v1/task/{taskId}/status`
 
@@ -396,11 +402,11 @@ This API is deprecated and will be removed in future releases.
 
 Retrieve a [task completion report](../ingestion/reports.html) for a task. Only works for completed tasks.
 
-#### POST
+##### POST
 
 * `/druid/indexer/v1/task` 
 
-Endpoint for submitting tasks and supervisor specs to the overlord. Returns the taskId of the submitted task.
+Endpoint for submitting tasks and supervisor specs to the Overlord. Returns the taskId of the submitted task.
 
 * `druid/indexer/v1/task/{taskId}/shutdown`
 
@@ -410,9 +416,9 @@ Shuts down a task.
 
 Shuts down all tasks for a dataSource.
 
-### Supervisors
+#### Supervisors
 
-#### GET
+##### GET
 
 * `/druid/indexer/v1/supervisor`
 
@@ -443,7 +449,7 @@ Returns an audit history of specs for all supervisors (current and past).
 
 Returns an audit history of specs for the supervisor with the provided ID.
 
-#### POST
+##### POST
 
 * `/druid/indexer/v1/supervisor`
 
@@ -486,28 +492,53 @@ This API is deprecated and will be removed in future releases.
 Please use the equivalent 'terminate' instead.
 </div>
 
-## MiddleManager
+## Data Server
+
+This section documents the API endpoints for the processes that reside on Data servers (MiddleManagers/Peons and Historicals) in the suggested [three-server configuration](../design/processes.html#server-types).
+
+### MiddleManager
 
 The MiddleManager does not have any API endpoints beyond the [common endpoints](#common).
 
-## Peon
+### Peon
 
 #### GET
 
 * `/druid/worker/v1/chat/{taskId}/rowStats`
 
-Retrieve a live row stats report from a peon. See [task reports](../ingestion/reports.html) for more details.
+Retrieve a live row stats report from a Peon. See [task reports](../ingestion/reports.html) for more details.
 
 * `/druid/worker/v1/chat/{taskId}/unparseableEvents`
 
-Retrieve an unparseable events report from a peon. See [task reports](../ingestion/reports.html) for more details.
+Retrieve an unparseable events report from a Peon. See [task reports](../ingestion/reports.html) for more details.
 
+### Historical
 
-## Broker
+#### Segment Loading
 
-### Datasource Information
+##### GET
 
-#### GET
+* `/druid/historical/v1/loadstatus`
+
+Returns JSON of the form `{"cacheInitialized":<value>}`, where value is either `true` or `false` indicating if all
+segments in the local cache have been loaded. This can be used to know when a Historical node is ready
+to be queried after a restart.
+
+* `/druid/historical/v1/readiness`
+
+Similar to `/druid/historical/v1/loadstatus`, but instead of returning JSON with a flag, responses 200 OK if segments
+in the local cache have been loaded, and 503 SERVICE UNAVAILABLE, if they haven't.
+
+
+## Query Server
+
+This section documents the API endpoints for the processes that reside on Query servers (Brokers) in the suggested [three-server configuration](../design/processes.html#server-types).
+
+### Broker
+
+#### Datasource Information
+
+##### GET
 
 * `/druid/v2/datasources`
 
@@ -545,18 +576,17 @@ if you're using SQL.
 
 Returns segment information lists including server locations for the given datasource and intervals. If "numCandidates" is not specified, it will return all servers for each interval.
 
-### Load Status
+#### Load Status
 
-#### GET
+##### GET
 
 * `/druid/broker/v1/loadstatus`
 
-Returns a flag indicating if the broker knows about all segments in Zookeeper. This can be used to know when a broker node is ready to be queried after a restart.
+Returns a flag indicating if the Broker knows about all segments in Zookeeper. This can be used to know when a Broker process is ready to be queried after a restart.
 
+#### Queries
 
-### Queries
-
-#### POST
+##### POST
 
 * `/druid/v2/`
 
@@ -566,20 +596,3 @@ The endpoint for submitting queries. Accepts an option `?pretty` that pretty pri
 
 Returns segment information lists including server locations for the given query..
 
-
-## Historical
-
-### Segment Loading
-
-#### GET
-
-* `/druid/historical/v1/loadstatus`
-
-Returns JSON of the form `{"cacheInitialized":<value>}`, where value is either `true` or `false` indicating if all
-segments in the local cache have been loaded. This can be used to know when a historical node is ready
-to be queried after a restart.
-
-* `/druid/historical/v1/readiness`
-
-Similar to `/druid/historical/v1/loadstatus`, but instead of returning JSON with a flag, responses 200 OK if segments
-in the local cache have been loaded, and 503 SERVICE UNAVAILABLE, if they haven't.
diff --git a/docs/content/operations/metrics.md b/docs/content/operations/metrics.md
index cb4056f..7bce8c0 100644
--- a/docs/content/operations/metrics.md
+++ b/docs/content/operations/metrics.md
@@ -53,7 +53,7 @@ Available Metrics
 |`query/bytes`|number of bytes returned in query response.|Common: dataSource, type, interval, hasFilters, duration, context, remoteAddress, id. Aggregation Queries: numMetrics, numComplexMetrics. GroupBy: numDimensions. TopN: threshold, dimension.| |
 |`query/node/time`|Milliseconds taken to query individual historical/realtime nodes.|id, status, server.|< 1s|
 |`query/node/bytes`|number of bytes returned from querying individual historical/realtime nodes.|id, status, server.| |
-|`query/node/ttfb`|Time to first byte. Milliseconds elapsed until broker starts receiving the response from individual historical/realtime nodes.|id, status, server.|< 1s|
+|`query/node/ttfb`|Time to first byte. Milliseconds elapsed until Broker starts receiving the response from individual historical/realtime nodes.|id, status, server.|< 1s|
 |`query/node/backpressure`|Milliseconds that the channel to this node has spent suspended due to backpressure.|id, status, server.| |
 |`query/intervalChunk/time`|Only emitted if interval chunking is enabled. Milliseconds required to query an interval chunk. This metric is deprecated and will be removed in the future because interval chunking is deprecated. See [Query Context](../querying/query-context.html).|id, status, chunkInterval (if interval chunking is enabled).|< 1s|
 |`query/count`|number of total queries|This metric is only available if the QueryCountStatsMonitor module is included.||
@@ -69,7 +69,7 @@ Available Metrics
 |`query/segment/time`|Milliseconds taken to query individual segment. Includes time to page in the segment from disk.|id, status, segment.|several hundred milliseconds|
 |`query/wait/time`|Milliseconds spent waiting for a segment to be scanned.|id, segment.|< several hundred milliseconds|
 |`segment/scan/pending`|Number of segments in queue waiting to be scanned.||Close to 0|
-|`query/segmentAndCache/time`|Milliseconds taken to query individual segment or hit the cache (if it is enabled on the historical node).|id, segment.|several hundred milliseconds|
+|`query/segmentAndCache/time`|Milliseconds taken to query individual segment or hit the cache (if it is enabled on the Historical node).|id, segment.|several hundred milliseconds|
 |`query/cpu/time`|Microseconds of CPU time taken to complete a query|Common: dataSource, type, interval, hasFilters, duration, context, remoteAddress, id. Aggregation Queries: numMetrics, numComplexMetrics. GroupBy: numDimensions. TopN: threshold, dimension.|Varies|
 |`query/count`|number of total queries|This metric is only available if the QueryCountStatsMonitor module is included.||
 |`query/success/count`|number of queries successfully processed|This metric is only available if the QueryCountStatsMonitor module is included.||
@@ -128,7 +128,7 @@ Memcached client metrics are reported as per the following. These metrics come d
 
 ## SQL Metrics
 
-If SQL is enabled, broker will emit the following metrics for SQL.
+If SQL is enabled, the Broker will emit the following metrics for SQL.
 
 |Metric|Description|Dimensions|Normal Value|
 |------|-----------|----------|------------|
@@ -190,7 +190,7 @@ Note: If the JVM does not support CPU time measurement for the current thread, i
 
 ## Coordination
 
-These metrics are for the Druid coordinator and are reset each time the coordinator runs the coordination logic.
+These metrics are for the Druid Coordinator and are reset each time the Coordinator runs the coordination logic.
 
 |Metric|Description|Dimensions|Normal Value|
 |------|-----------|----------|------------|
@@ -212,7 +212,7 @@ These metrics are for the Druid coordinator and are reset each time the coordina
 |`segment/unavailable/count`|Number of segments (not including replicas) left to load until segments that should be loaded in the cluster are available for queries.|datasource.|0|
 |`segment/underReplicated/count`|Number of segments (including replicas) left to load until segments that should be loaded in the cluster are available for queries.|tier, datasource.|0|
 
-If `emitBalancingStats` is set to `true` in the coordinator [dynamic configuration](../configuration/index.html#dynamic-configuration), then [log entries](../configuration/logging.html) for class `org.apache.druid.server.coordinator.helper.DruidCoordinatorLogger` will have extra information on balancing decisions.
+If `emitBalancingStats` is set to `true` in the Coordinator [dynamic configuration](../configuration/index.html#dynamic-configuration), then [log entries](../configuration/logging.html) for class `org.apache.druid.server.coordinator.helper.DruidCoordinatorLogger` will have extra information on balancing decisions.
 
 ## General Health
 
diff --git a/docs/content/operations/other-hadoop.md b/docs/content/operations/other-hadoop.md
index a7b4952..74a8696 100644
--- a/docs/content/operations/other-hadoop.md
+++ b/docs/content/operations/other-hadoop.md
@@ -56,7 +56,7 @@ Generally, you should only set one of these parameters, not both.
 These properties can be set in either one of the following ways:
 
 - Using the task definition, e.g. add `"mapreduce.job.classloader": "true"` to the `jobProperties` of the `tuningConfig` of your indexing task (see the [Hadoop batch ingestion documentation](../ingestion/hadoop.html)).
-- Using system properties, e.g. on the middleManager set `druid.indexer.runner.javaOpts=... -Dhadoop.mapreduce.job.classloader=true`.
+- Using system properties, e.g. on the MiddleManager set `druid.indexer.runner.javaOpts=... -Dhadoop.mapreduce.job.classloader=true`.
 
 ### Overriding specific classes
 
@@ -181,7 +181,7 @@ For more about building Druid, please see [Building Druid](../development/build.
 
 **Alternate workaround - 2**
 
-Another workaround solution is to build a custom fat jar of Druid using [sbt](http://www.scala-sbt.org/), which manually excludes all the conflicting Jackson dependencies, and then put this fat jar in the classpath of the command that starts overlord indexing service. To do this, please follow the following steps.
+Another workaround solution is to build a custom fat jar of Druid using [sbt](http://www.scala-sbt.org/), which manually excludes all the conflicting Jackson dependencies, and then put this fat jar in the classpath of the command that starts Overlord indexing service. To do this, please follow the following steps.
 
 (1) Download and install sbt.
 
diff --git a/docs/content/operations/performance-faq.md b/docs/content/operations/performance-faq.md
index 03b2c37..672ca9d 100644
--- a/docs/content/operations/performance-faq.md
+++ b/docs/content/operations/performance-faq.md
@@ -32,16 +32,16 @@ Improper configuration is by far the largest problem we see people trying to dep
 
 The size of the JVM heap really depends on the type of Druid node you are running. Below are a few considerations.
 
-[Broker nodes](../design/broker.html) uses the JVM heap mainly to merge results from historicals and real-times. Brokers also use off-heap memory and processing threads for groupBy queries. We recommend 20G-30G of heap here.
+[Broker nodes](../design/broker.html) uses the JVM heap mainly to merge results from Historicals and real-times. Brokers also use off-heap memory and processing threads for groupBy queries. We recommend 20G-30G of heap here.
 
-[Historical nodes](../design/historical.html) use off-heap memory to store intermediate results, and by default, all segments are memory mapped before they can be queried. Typically, the more memory is available on a historical node, the more segments can be served without the possibility of data being paged on to disk. On historicals, the JVM heap is used for [GroupBy queries](../querying/groupbyquery.html), some data structures used for intermediate computation, and general processing. [...]
+[Historical nodes](../design/historical.html) use off-heap memory to store intermediate results, and by default, all segments are memory mapped before they can be queried. Typically, the more memory is available on a Historical node, the more segments can be served without the possibility of data being paged on to disk. On Historicals, the JVM heap is used for [GroupBy queries](../querying/groupbyquery.html), some data structures used for intermediate computation, and general processing. [...]
 
 We recommend 250mb * (processing.numThreads) for the heap.
 
 [Coordinator nodes](../design/coordinator.html) do not require off-heap memory and the heap is used for loading information about all segments to determine what segments need to be loaded, dropped, moved, or replicated.
 
 ## How much direct memory does Druid use?
-Any Druid node that process queries (brokers, ingestion workers, and historical nodes) use two kinds of direct memory buffers with configurable size: processing buffers and merge buffers.
+Any Druid process that handles queries (Brokers, Peons, and Historicals) uses two kinds of direct memory buffers with configurable size: processing buffers and merge buffers.
 
 Each processing thread is allocated one processing buffer. Additionally, there is a shared pool of merge buffers (only used for GroupBy V2 queries currently).
 
diff --git a/docs/content/operations/recommendations.md b/docs/content/operations/recommendations.md
index cfac2ae..bb0acd3 100644
--- a/docs/content/operations/recommendations.md
+++ b/docs/content/operations/recommendations.md
@@ -70,7 +70,7 @@ We recommend using UTC timezone for all your events and across on your nodes, no
 
 # SSDs
 
-SSDs are highly recommended for historical and real-time nodes if you are not running a cluster that is entirely in memory. SSDs can greatly mitigate the time required to page data in and out of memory.
+SSDs are highly recommended for Historical and real-time nodes if you are not running a cluster that is entirely in memory. SSDs can greatly mitigate the time required to page data in and out of memory.
 
 # JBOD vs RAID
 Historical nodes store large number of segments on Disk and support specifying multiple paths for storing those. Typically, hosts have multiple disks configured with RAID which makes them look like a single disk to OS. RAID might have overheads specially if its not hardware controller based but software based. So, Historicals might get improved disk throughput with JBOD.
diff --git a/docs/content/operations/rolling-updates.md b/docs/content/operations/rolling-updates.md
index c154fc4..9b90803 100644
--- a/docs/content/operations/rolling-updates.md
+++ b/docs/content/operations/rolling-updates.md
@@ -82,7 +82,7 @@ to `<MiddleManager_IP:PORT>/druid/worker/v1/enable`.
 
 If autoscaling is enabled on your Overlord, then Overlord nodes can launch new Middle Manager nodes
 en masse and then gracefully terminate old ones as their tasks finish. This process is configured by
-setting `druid.indexer.runner.minWorkerVersion=#{VERSION}`. Each time you update your overlord node,
+setting `druid.indexer.runner.minWorkerVersion=#{VERSION}`. Each time you update your Overlord node,
 the `VERSION` value should be increased, which will trigger a mass launch of new Middle Managers.
 
 The config `druid.indexer.autoscale.workerVersion=#{VERSION}` also needs to be set.
@@ -94,7 +94,7 @@ Standalone real-time nodes can be updated one at a time in a rolling fashion.
 ## Broker
 
 Broker nodes can be updated one at a time in a rolling fashion. There needs to be some delay between
-updating each node as brokers must load the entire state of the cluster before they return valid
+updating each node as Brokers must load the entire state of the cluster before they return valid
 results.
 
 ## Coordinator
diff --git a/docs/content/operations/rule-configuration.md b/docs/content/operations/rule-configuration.md
index 9e67d5d..4d67789 100644
--- a/docs/content/operations/rule-configuration.md
+++ b/docs/content/operations/rule-configuration.md
@@ -24,16 +24,16 @@ title: "Retaining or Automatically Dropping Data"
 
 # Retaining or Automatically Dropping Data
 
-Coordinator nodes use rules to determine what data should be loaded to or dropped from the cluster. Rules are used for data retention and query execution, and are set on the coordinator console (http://coordinator_ip:port).
+Coordinator nodes use rules to determine what data should be loaded to or dropped from the cluster. Rules are used for data retention and query execution, and are set on the Coordinator console (http://coordinator_ip:port).
 
 There are three types of rules, i.e., load rules, drop rules, and broadcast rules. Load rules indicate how segments should be assigned to different historical node tiers and how many replicas of a segment should exist in each tier. 
 Drop rules indicate when segments should be dropped entirely from the cluster. Finally, broadcast rules indicate how segments of different data sources should be co-located in historical nodes.
 
-The coordinator loads a set of rules from the metadata storage. Rules may be specific to a certain datasource and/or a default set of rules can be configured. Rules are read in order and hence the ordering of rules is important. The coordinator will cycle through all available segments and match each segment with the first rule that applies. Each segment may only match a single rule.
+The Coordinator loads a set of rules from the metadata storage. Rules may be specific to a certain datasource and/or a default set of rules can be configured. Rules are read in order and hence the ordering of rules is important. The Coordinator will cycle through all available segments and match each segment with the first rule that applies. Each segment may only match a single rule.
 
-Note: It is recommended that the coordinator console is used to configure rules. However, the coordinator node does have HTTP endpoints to programmatically configure rules.
+Note: It is recommended that the Coordinator console is used to configure rules. However, the Coordinator node does have HTTP endpoints to programmatically configure rules.
 
-When a rule is updated, the change may not be reflected until the next time the coordinator runs. This will be fixed in the near future.
+When a rule is updated, the change may not be reflected until the next time the Coordinator runs. This will be fixed in the near future.
 
 Load Rules
 ----------
@@ -173,7 +173,7 @@ The interval of a segment will be compared against the specified period. The per
 Broadcast Rules
 ---------------
 
-Broadcast rules indicate how segments of different data sources should be co-located in historical nodes. 
+Broadcast rules indicate how segments of different data sources should be co-located in Historical nodes. 
 Once a broadcast rule is configured for a data source, all segments of the data source are broadcasted to the servers holding _any segments_ of the co-located data sources.
 
 ### Forever Broadcast Rule
@@ -234,9 +234,9 @@ If you want to always co-locate the segments of some data sources together, it i
 # Permanently Deleting Data
  
 Druid can fully drop data from the cluster, wipe the metadata store entry, and remove the data from deep storage for any segments that are 
-marked as unused (segments dropped from the cluster via rules are always marked as unused). You can submit a [kill task](../ingestion/tasks.html) to the [indexing service](../design/indexing-service.html) to do this.
+marked as unused (segments dropped from the cluster via rules are always marked as unused). You can submit a [kill task](../ingestion/tasks.html) to the [Overlord](../design/overlord.html) to do this.
 
 # Reloading Dropped Data
 
 Data that has been dropped from a Druid cluster cannot be reloaded using only rules. To reload dropped data in Druid, you must first set your retention period (i.e. changing the retention period from 1 month to 2 months), and 
-then enable the datasource in the Druid coordinator console, or through the Druid coordinator endpoints.
+then enable the datasource in the Druid Coordinator console, or through the Druid Coordinator endpoints.
diff --git a/docs/content/operations/segment-optimization.md b/docs/content/operations/segment-optimization.md
index 04d099e..2fea3ff 100644
--- a/docs/content/operations/segment-optimization.md
+++ b/docs/content/operations/segment-optimization.md
@@ -28,7 +28,7 @@ In Druid, it's important to optimize the segment size because
 
   1. Druid stores data in segments. If you're using the [best-effort roll-up](../design/index.html#roll-up-modes) mode,
   increasing the segment size might introduce further aggregation which reduces the dataSource size.
-  2. When a query is submitted, that query is distributed to all historicals and realtimes
+  2. When a query is submitted, that query is distributed to all Historicals and realtimes
   which hold the input segments of the query. Each node has a processing threads pool and use one thread per segment to
   process it. If the segment size is too large, data might not be well distributed over the
   whole cluster, thereby decreasing the degree of parallelism. If the segment size is too small,
@@ -39,8 +39,8 @@ It would be best if you can optimize the segment size at ingestion time, but som
 especially for the streaming ingestion because the amount of data ingested might vary over time. In this case,
 you can roughly set the segment size at ingestion time and optimize it later. You have two options:
 
-  - Turning on the [automatic compaction of coordinators](../design/coordinator.html#compacting-segments).
-  The coordinator periodically submits [compaction tasks](../ingestion/tasks.html#compaction-task) to re-index small segments.
+  - Turning on the [automatic compaction of Coordinators](../design/coordinator.html#compacting-segments).
+  The Coordinator periodically submits [compaction tasks](../ingestion/tasks.html#compaction-task) to re-index small segments.
   - Running periodic Hadoop batch ingestion jobs and using a `dataSource`
   inputSpec to read from the segments generated by the Kafka indexing tasks. This might be helpful if you want to compact a lot of segments in parallel.
   Details on how to do this can be found under ['Updating Existing Data'](../ingestion/update-existing-data.html).
diff --git a/docs/content/querying/caching.md b/docs/content/querying/caching.md
index cb84e1f..c83d067 100644
--- a/docs/content/querying/caching.md
+++ b/docs/content/querying/caching.md
@@ -34,7 +34,7 @@ can be enabled at either the Historical and Broker level (it is not recommended
 
 ## Query caching on Brokers
 
-Enabling caching on the broker can yield faster results than if query caches were enabled on Historicals for small clusters. This is 
+Enabling caching on the Broker can yield faster results than if query caches were enabled on Historicals for small clusters. This is 
 the recommended setup for smaller production clusters (< 20 servers). Take note that when caching is enabled on the Broker, 
 results from Historicals are returned on a per segment basis, and Historicals will not be able to do any local result merging.
 Result level caching is enabled only on the Broker side.
diff --git a/docs/content/querying/datasource.md b/docs/content/querying/datasource.md
index d9811e8..552e1cc 100644
--- a/docs/content/querying/datasource.md
+++ b/docs/content/querying/datasource.md
@@ -48,7 +48,7 @@ This data source unions two or more table data sources.
 ```
 
 Note that the data sources being unioned should have the same schema.
-Union Queries should be always sent to the broker/router node and are *NOT* supported directly by the historical nodes. 
+Union Queries should be always sent to a Broker/Router process and are *NOT* supported directly by the Historical processes. 
 
 ### Query Data Source
 
diff --git a/docs/content/querying/dimensionspecs.md b/docs/content/querying/dimensionspecs.md
index c6c636d..ce57f46 100644
--- a/docs/content/querying/dimensionspecs.md
+++ b/docs/content/querying/dimensionspecs.md
@@ -123,7 +123,7 @@ It is illegal to set `retainMissingValue = true` and also specify a `replaceMiss
 
 A property `optimize` can be supplied to allow optimization of lookup based extraction filter (by default `optimize = true`).
 
-The second kind where it is not possible to pass at query time due to their size, will be based on an external lookup table or resource that is already registered via configuration file or/and coordinator.
+The second kind where it is not possible to pass at query time due to their size, will be based on an external lookup table or resource that is already registered via configuration file or/and Coordinator.
 
 ```json
 {
@@ -375,7 +375,7 @@ A property of `injective` can override the lookup's own sense of whether or not
 configuration.
 
 A property `optimize` can be supplied to allow optimization of lookup based extraction filter (by default `optimize = true`).
-The optimization layer will run on the broker and it will rewrite the extraction filter as clause of selector filters.
+The optimization layer will run on the Broker and it will rewrite the extraction filter as clause of selector filters.
 For instance the following filter
 
 ```json
diff --git a/docs/content/querying/groupbyquery.md b/docs/content/querying/groupbyquery.md
index 1d502a9..6ba1402 100644
--- a/docs/content/querying/groupbyquery.md
+++ b/docs/content/querying/groupbyquery.md
@@ -226,22 +226,22 @@ Response for above query would look something like below...
 #### Strategies
 
 GroupBy queries can be executed using two different strategies. The default strategy for a cluster is determined by the
-"druid.query.groupBy.defaultStrategy" runtime property on the broker. This can be overridden using "groupByStrategy" in
+"druid.query.groupBy.defaultStrategy" runtime property on the Broker. This can be overridden using "groupByStrategy" in
 the query context. If neither the context field nor the property is set, the "v2" strategy will be used.
 
 - "v2", the default, is designed to offer better performance and memory management. This strategy generates
 per-segment results using a fully off-heap map. Data nodes merge the per-segment results using a fully off-heap
 concurrent facts map combined with an on-heap string dictionary. This may optionally involve spilling to disk. Data
-nodes return sorted results to the broker, which merges result streams using an N-way merge. The broker materializes
+nodes return sorted results to the Broker, which merges result streams using an N-way merge. The broker materializes
 the results if necessary (e.g. if the query sorts on columns other than its dimensions). Otherwise, it streams results
 back as they are merged.
 
-- "v1", a legacy engine, generates per-segment results on data nodes (historical, realtime, middleManager) using a map which
+- "v1", a legacy engine, generates per-segment results on data nodes (Historical, realtime, MiddleManager) using a map which
 is partially on-heap (dimension keys and the map itself) and partially off-heap (the aggregated values). Data nodes then
 merge the per-segment results using Druid's indexing mechanism. This merging is multi-threaded by default, but can
-optionally be single-threaded. The broker merges the final result set using Druid's indexing mechanism again. The broker
-merging is always single-threaded. Because the broker merges results using the indexing mechanism, it must materialize
-the full result set before returning any results. On both the data nodes and the broker, the merging index is fully
+optionally be single-threaded. The Broker merges the final result set using Druid's indexing mechanism again. The broker
+merging is always single-threaded. Because the Broker merges results using the indexing mechanism, it must materialize
+the full result set before returning any results. On both the data nodes and the Broker, the merging index is fully
 on-heap by default, but it can optionally store aggregated values off-heap.
 
 #### Differences between v1 and v2
@@ -257,9 +257,9 @@ that can complete successfully in one engine may exceed resource limits and fail
 - groupBy v1 imposes no limit on the number of concurrently running queries, whereas groupBy v2 controls memory usage
 by using a finite-sized merge buffer pool. By default, the number of merge buffers is 1/4 the number of processing
 threads. You can adjust this as necessary to balance concurrency and memory usage.
-- groupBy v1 supports caching on either the broker or historical nodes, whereas groupBy v2 only supports caching on
-historical nodes.
-- groupBy v1 supports using [chunkPeriod](query-context.html) to parallelize merging on the broker, whereas groupBy v2
+- groupBy v1 supports caching on either the Broker or Historical nodes, whereas groupBy v2 only supports caching on
+Historical nodes.
+- groupBy v1 supports using [chunkPeriod](query-context.html) to parallelize merging on the Broker, whereas groupBy v2
 ignores chunkPeriod.
 - groupBy v2 supports both array-based aggregation and hash-based aggregation. The array-based aggregation is used only
 when the grouping key is a single indexed string column. In array-based aggregation, the dictionary-encoded value is used
@@ -304,7 +304,7 @@ concurrent query load.
 
 ##### Limit pushdown optimization
 
-Druid pushes down the `limit` spec in groupBy queries to the segments on historicals wherever possible to early prune unnecessary intermediate results and minimize the amount of data transferred to brokers. By default, this technique is applied only when all fields in the `orderBy` spec is a subset of the grouping keys. This is because the `limitPushDown` doesn't guarantee the exact results if the `orderBy` spec includes any fields that are not in the grouping keys. However, you can enab [...]
+Druid pushes down the `limit` spec in groupBy queries to the segments on Historicals wherever possible to early prune unnecessary intermediate results and minimize the amount of data transferred to Brokers. By default, this technique is applied only when all fields in the `orderBy` spec is a subset of the grouping keys. This is because the `limitPushDown` doesn't guarantee the exact results if the `orderBy` spec includes any fields that are not in the grouping keys. However, you can enab [...]
 
 
 ##### Optimizing hash table
@@ -316,10 +316,10 @@ The default number of initial buckets is 1024 and the default max load factor of
 
 ##### Parallel combine
 
-Once a historical finishes aggregation using the hash table, it sorts the aggregated results and merges them before sending to the
-broker for N-way merge aggregation in the broker. By default, historicals use all their available processing threads
+Once a Historical finishes aggregation using the hash table, it sorts the aggregated results and merges them before sending to the
+Broker for N-way merge aggregation in the broker. By default, Historicals use all their available processing threads
 (configured by `druid.processing.numThreads`) for aggregation, but use a single thread for sorting and merging
-aggregates which is an http thread to send data to brokers.
+aggregates which is an http thread to send data to Brokers.
 
 This is to prevent some heavy groupBy queries from blocking other queries. In Druid, the processing threads are shared
 between all submitted queries and they are _not interruptible_. It means, if a heavy query takes all available
@@ -338,7 +338,7 @@ aggregates from hash tables including spilled ones. Usually, leaf nodes are slow
 need to read data from disk. As a result, less threads are used for intermediate nodes by default. You can change the
 degree of intermediate nodes. See `intermediateCombineDegree` in [Advanced groupBy v2 configurations](#groupby-v2-configurations).
 
-Please note that each historical needs two merge buffers to process a groupBy v2 query with parallel combine: one for
+Please note that each Historical needs two merge buffers to process a groupBy v2 query with parallel combine: one for
 computing intermediate aggregates from each segment and another for combining intermediate aggregates in parallel.
 
 
@@ -356,15 +356,15 @@ results acceptable.
 
 #### Nested groupBys
 
-Nested groupBys (dataSource of type "query") are performed differently for "v1" and "v2". The broker first runs the
+Nested groupBys (dataSource of type "query") are performed differently for "v1" and "v2". The Broker first runs the
 inner groupBy query in the usual way. "v1" strategy then materializes the inner query's results on-heap with Druid's
 indexing mechanism, and runs the outer query on these materialized results. "v2" strategy runs the outer query on the
 inner query's results stream with off-heap fact map and on-heap string dictionary that can spill to disk. Both
-strategy perform the outer query on the broker in a single-threaded fashion.
+strategy perform the outer query on the Broker in a single-threaded fashion.
 
 #### Configurations
 
-This section describes the configurations for groupBy queries. You can set the runtime properties in the `runtime.properties` file on broker, historical, and MiddleManager nodes. You can set the query context parameters through the [query context](query-context.html).
+This section describes the configurations for groupBy queries. You can set the runtime properties in the `runtime.properties` file on Broker, Historical, and MiddleManager nodes. You can set the query context parameters through the [query context](query-context.html).
   
 ##### Configurations for groupBy v2
 
@@ -424,7 +424,7 @@ Supported query contexts:
 |`intermediateCombineDegree`|Overrides the value of `druid.query.groupBy.intermediateCombineDegree`|None|
 |`numParallelCombineThreads`|Overrides the value of `druid.query.groupBy.numParallelCombineThreads`|None|
 |`sortByDimsFirst`|Sort the results first by dimension values and then by timestamp.|false|
-|`forceLimitPushDown`|When all fields in the orderby are part of the grouping key, the broker will push limit application down to the historical nodes. When the sorting order uses fields that are not in the grouping key, applying this optimization can result in approximate results with unknown accuracy, so this optimization is disabled by default in that case. Enabling this context flag turns on limit push down for limit/orderbys that contain non-grouping key columns.|false|
+|`forceLimitPushDown`|When all fields in the orderby are part of the grouping key, the Broker will push limit application down to the Historical nodes. When the sorting order uses fields that are not in the grouping key, applying this optimization can result in approximate results with unknown accuracy, so this optimization is disabled by default in that case. Enabling this context flag turns on limit push down for limit/orderbys that contain non-grouping key columns.|false|
 
 
 ##### GroupBy v1 configurations
diff --git a/docs/content/querying/lookups.md b/docs/content/querying/lookups.md
index 9a9feca..7c7ad81 100644
--- a/docs/content/querying/lookups.md
+++ b/docs/content/querying/lookups.md
@@ -93,11 +93,11 @@ Dynamic Configuration
 Dynamic lookup configuration is an <a href="../development/experimental.html">experimental</a> feature. Static
 configuration is no longer supported.
 </div>
-The following documents the behavior of the cluster-wide config which is accessible through the coordinator.
+The following documents the behavior of the cluster-wide config which is accessible through the Coordinator.
 The configuration is propagated through the concept of "tier" of servers.
 A "tier" is defined as a group of services which should receive a set of lookups.
-For example, you might have all historicals be part of `__default`, and Peons be part of individual tiers for the datasources they are tasked with.
-The tiers for lookups are completely independent of historical tiers.
+For example, you might have all Historicals be part of `__default`, and Peons be part of individual tiers for the datasources they are tasked with.
+The tiers for lookups are completely independent of Historical tiers.
 
 These configs are accessed using JSON through the following URI template
 
@@ -117,9 +117,9 @@ These endpoints will return one of the following results:
 * 200 if the request succeeded (`GET` only)
 
 ## Configuration propagation behavior
-The configuration is propagated to the query serving nodes (broker / router / peon / historical) by the coordinator.
-The query serving nodes have an internal API for managing lookups on the node and those are used by the coordinator.
-The coordinator periodically checks if any of the nodes need to load/drop lookups and updates them appropriately.
+The configuration is propagated to the query serving processes (Broker / Router / Peon / Historical) by the Coordinator.
+The query serving nodes have an internal API for managing lookups on the node and those are used by the Coordinator.
+The Coordinator periodically checks if any of the nodes need to load/drop lookups and updates them appropriately.
 
 # API for configuring lookups
 
@@ -288,7 +288,7 @@ To discover a list of tiers currently active in the cluster **instead of** ones
 A `GET` to `/druid/coordinator/v1/lookups/config/{tier}` will return a list of known lookup names for that tier.
 
 # Additional API related to status of configured lookups
-These end points can be used to get the propagation status of configured lookups to lookup nodes such as historicals.
+These end points can be used to get the propagation status of configured lookups to lookup nodes such as Historicals.
 
 ## List load status of all lookups
 `GET /druid/coordinator/v1/lookups/status` with optional query parameter `detailed`.
@@ -352,16 +352,16 @@ The return value will be the json representation of the factory.
 ```
 
 # Configuration
-See [Lookups Dynamic Configuration](../configuration/index.html#lookups-dynamic-configuration) for coordinator configuration.
+See [Lookups Dynamic Configuration](../configuration/index.html#lookups-dynamic-configuration) for Coordinator configuration.
 
 To configure a Broker / Router / Historical / Peon to announce itself as part of a lookup tier, use the `druid.zk.paths.lookupTier` property.
 
 |Property | Description | Default |
 |---------|-------------|---------|
 |`druid.lookup.lookupTier`| The tier for **lookups** for this node. This is independent of other tiers.|`__default`|
-|`druid.lookup.lookupTierIsDatasource`|For some things like indexing service tasks, the datasource is passed in the runtime properties of a task. This option fetches the tierName from the same value as the datasource for the task. It is suggested to only use this as peon options for the indexing service, if at all. If true, `druid.lookup.lookupTier` MUST NOT be specified|`"false"`|
+|`druid.lookup.lookupTierIsDatasource`|For some things like indexing service tasks, the datasource is passed in the runtime properties of a task. This option fetches the tierName from the same value as the datasource for the task. It is suggested to only use this as Peon options for the indexing service, if at all. If true, `druid.lookup.lookupTier` MUST NOT be specified|`"false"`|
 
-To configure the behavior of the dynamic configuration manager, use the following properties on the coordinator:
+To configure the behavior of the dynamic configuration manager, use the following properties on the Coordinator:
 
 |Property|Description|Default|
 |--------|-----------|-------|
@@ -372,20 +372,20 @@ To configure the behavior of the dynamic configuration manager, use the followin
 
 ## Saving configuration across restarts
 
-It is possible to save the configuration across restarts such that a node will not have to wait for coordinator action to re-populate its lookups. To do this the following property is set:
+It is possible to save the configuration across restarts such that a node will not have to wait for Coordinator action to re-populate its lookups. To do this the following property is set:
 
 |Property|Description|Default|
 |--------|-----------|-------|
 |`druid.lookup.snapshotWorkingDir`|Working path used to store snapshot of current lookup configuration, leaving this property null will disable snapshot/bootstrap utility|null|
-|`druid.lookup.enableLookupSyncOnStartup`|Enable the lookup synchronization process with coordinator on startup. The queryable nodes will fetch and load the lookups from the coordinator instead of waiting for the coordinator to load the lookups for them. Users may opt to disable this option if there are no lookups configured in the cluster.|true|
+|`druid.lookup.enableLookupSyncOnStartup`|Enable the lookup synchronization process with Coordinator on startup. The queryable nodes will fetch and load the lookups from the Coordinator instead of waiting for the Coordinator to load the lookups for them. Users may opt to disable this option if there are no lookups configured in the cluster.|true|
 |`druid.lookup.numLookupLoadingThreads`|Number of threads for loading the lookups in parallel on startup. This thread pool is destroyed once startup is done. It is not kept during the lifetime of the JVM|Available Processors / 2|
-|`druid.lookup.coordinatorFetchRetries`|How many times to retry to fetch the lookup bean list from coordinator, during the sync on startup.|3|
+|`druid.lookup.coordinatorFetchRetries`|How many times to retry to fetch the lookup bean list from Coordinator, during the sync on startup.|3|
 |`druid.lookup.lookupStartRetries`|How many times to retry to start each lookup, either during the sync on startup, or during the runtime.|3|
-|`druid.lookup.coordinatorRetryDelay`|How long to delay (in millis) between retries to fetch lookup list from the coordinator during the sync on startup.|60_000|
+|`druid.lookup.coordinatorRetryDelay`|How long to delay (in millis) between retries to fetch lookup list from the Coordinator during the sync on startup.|60_000|
 
 ## Introspect a Lookup
 
-The broker provides an API for lookup introspection if the lookup type implements a `LookupIntrospectHandler`. 
+The Broker provides an API for lookup introspection if the lookup type implements a `LookupIntrospectHandler`. 
 
 A `GET` request to `/druid/v1/lookups/introspect/{lookupId}` will return the map of complete values. 
 
@@ -436,6 +436,6 @@ ex: `GET /druid/v1/lookups/introspect/nato-phonetic/values`
 ## Druid version 0.10.0 to 0.10.1 upgrade/downgrade
 Overall druid cluster lookups configuration is persisted in metadata store and also individual lookup nodes optionally persist a snapshot of loaded lookups on disk.
 If upgrading from druid version 0.10.0 to 0.10.1, then migration for all persisted metadata is handled automatically.
-If downgrading from 0.10.1 to 0.9.0 then lookups updates done via coordinator while 0.10.1 was running, would be lost.
+If downgrading from 0.10.1 to 0.9.0 then lookups updates done via Coordinator while 0.10.1 was running, would be lost.
 
  
diff --git a/docs/content/querying/multitenancy.md b/docs/content/querying/multitenancy.md
index 642d6ea..ea74081 100644
--- a/docs/content/querying/multitenancy.md
+++ b/docs/content/querying/multitenancy.md
@@ -71,7 +71,7 @@ You can use this in concert with single-dimension partitioning to repartition yo
 
 ## Customizing data distribution
 
-Druid additionally supports multitenancy by providing configurable means of distributing data. Druid's historical nodes 
+Druid additionally supports multitenancy by providing configurable means of distributing data. Druid's Historical nodes 
 can be configured into [tiers](../operations/rule-configuration.html), and [rules](../operations/rule-configuration.html) 
 can be set that determines which segments go into which tiers. One use case of this is that recent data tends to be accessed 
 more frequently than older data. Tiering enables more recent segments to be hosted on more powerful hardware for better performance. 
@@ -95,5 +95,5 @@ Druid queries can optionally set a `priority` flag in the [query context](../que
 slow (download or reporting style queries) can be de-prioritized and more interactive queries can have higher priority. 
 
 Broker nodes can also be dedicated to a given tier. For example, one set of broker nodes can be dedicated to fast interactive queries, 
-and a second set of broker nodes can be dedicated to slower reporting queries. Druid also provides a [router](../development/router.html) 
-node that can route queries to different brokers based on various query parameters (datasource, interval, etc.).  
+and a second set of Broker nodes can be dedicated to slower reporting queries. Druid also provides a [Router](../development/router.html) 
+node that can route queries to different Brokers based on various query parameters (datasource, interval, etc.).  
diff --git a/docs/content/querying/query-context.md b/docs/content/querying/query-context.md
index 81b39d8..60fe1c7 100644
--- a/docs/content/querying/query-context.md
+++ b/docs/content/querying/query-context.md
@@ -28,7 +28,7 @@ The query context is used for various query configuration parameters. The follow
 
 |property         |default                                 | description          |
 |-----------------|----------------------------------------|----------------------|
-|timeout          | `druid.server.http.defaultQueryTimeout`| Query timeout in millis, beyond which unfinished queries will be cancelled. 0 timeout means `no timeout`. To set the default timeout, see [broker configuration](../configuration/index.html#broker) |
+|timeout          | `druid.server.http.defaultQueryTimeout`| Query timeout in millis, beyond which unfinished queries will be cancelled. 0 timeout means `no timeout`. To set the default timeout, see [Broker configuration](../configuration/index.html#broker) |
 |priority         | `0`                                    | Query Priority. Queries with higher priority get precedence for computational resources.|
 |queryId          | auto-generated                         | Unique identifier given to this query. If a query ID is set or known, this can be used to cancel the query |
 |useCache         | `true`                                 | Flag indicating whether to leverage the query cache for this query. When set to false, it disables reading from the query cache for this query. When set to true, Druid uses druid.broker.cache.useCache or druid.historical.cache.useCache to determine whether or not to read from the query cache |
@@ -37,11 +37,11 @@ The query context is used for various query configuration parameters. The follow
 |populateResultLevelCache    | `false`                                 | Flag indicating whether to save the results of the query to the result level cache. Primarily used for debugging. When set to false, it disables saving the results of this query to the query cache. When set to true, Druid uses druid.broker.cache.populateCache to determine whether or not to save the results of this query to the query cache |
 |bySegment        | `false`                                | Return "by segment" results. Primarily used for debugging, setting it to `true` returns results associated with the data segment they came from |
 |finalize         | `true`                                 | Flag indicating whether to "finalize" aggregation results. Primarily used for debugging. For instance, the `hyperUnique` aggregator will return the full HyperLogLog sketch instead of the estimated cardinality when this flag is set to `false` |
-|chunkPeriod      | `P0D` (off)                            | At the broker node level, long interval queries (of any type) may be broken into shorter interval queries to parallelize merging more than normal. Broken up queries will use a larger share of cluster resources, but, if you use groupBy "v1, it may be able to complete faster as a result. Use ISO 8601 periods. For example, if this property is set to `P1M` (one month), then a query covering a year would be broken into 12 smaller qu [...]
-|maxScatterGatherBytes| `druid.server.http.maxScatterGatherBytes` | Maximum number of bytes gathered from data nodes such as historicals and realtime processes to execute a query. This parameter can be used to further reduce `maxScatterGatherBytes` limit at query time. See [broker configuration](../configuration/index.html#broker) for more details.|
+|chunkPeriod      | `P0D` (off)                            | At the Broker node level, long interval queries (of any type) may be broken into shorter interval queries to parallelize merging more than normal. Broken up queries will use a larger share of cluster resources, but, if you use groupBy "v1, it may be able to complete faster as a result. Use ISO 8601 periods. For example, if this property is set to `P1M` (one month), then a query covering a year would be broken into 12 smaller qu [...]
+|maxScatterGatherBytes| `druid.server.http.maxScatterGatherBytes` | Maximum number of bytes gathered from data nodes such as Historicals and realtime processes to execute a query. This parameter can be used to further reduce `maxScatterGatherBytes` limit at query time. See [Broker configuration](../configuration/index.html#broker) for more details.|
 |maxQueuedBytes       | `druid.broker.http.maxQueuedBytes`        | Maximum number of bytes queued per query before exerting backpressure on the channel to the data server. Similar to `maxScatterGatherBytes`, except unlike that configuration, this one will trigger backpressure rather than query failure. Zero means disabled.|
-|serializeDateTimeAsLong| `false`       | If true, DateTime is serialized as long in the result returned by broker and the data transportation between broker and compute node|
-|serializeDateTimeAsLongInner| `false`  | If true, DateTime is serialized as long in the data transportation between broker and compute node|
+|serializeDateTimeAsLong| `false`       | If true, DateTime is serialized as long in the result returned by Broker and the data transportation between Broker and compute node|
+|serializeDateTimeAsLongInner| `false`  | If true, DateTime is serialized as long in the data transportation between Broker and compute node|
 
 In addition, some query types offer context parameters specific to that query type.
 
diff --git a/docs/content/querying/querying.md b/docs/content/querying/querying.md
index 1aa9ca5..c26e601 100644
--- a/docs/content/querying/querying.md
+++ b/docs/content/querying/querying.md
@@ -27,7 +27,7 @@ title: "Querying"
 Queries are made using an HTTP REST style request to queryable nodes ([Broker](../design/broker.html),
 [Historical](../design/historical.html). [Peons](../design/peons.html)) that are running stream ingestion tasks can also accept queries. The
 query is expressed in JSON and each of these node types expose the same
-REST query interface. For normal Druid operations, queries should be issued to the broker nodes. Queries can be posted
+REST query interface. For normal Druid operations, queries should be issued to the Broker nodes. Queries can be posted
 to the queryable nodes like this -
 
  ```bash
@@ -79,7 +79,7 @@ Where possible, we recommend using [Timeseries]() and [TopN]() queries instead o
 
 Queries can be cancelled explicitly using their unique identifier.  If the
 query identifier is set at the time of query, or is otherwise known, the following
-endpoint can be used on the broker or router to cancel the query.
+endpoint can be used on the Broker or Router to cancel the query.
 
 ```sh
 DELETE /druid/v2/{queryId}
diff --git a/docs/content/querying/scan-query.md b/docs/content/querying/scan-query.md
index 3963800..cb4791f 100644
--- a/docs/content/querying/scan-query.md
+++ b/docs/content/querying/scan-query.md
@@ -178,7 +178,7 @@ The format of the result when resultFormat equals to `compactedList`:
 The biggest difference between select query and scan query is that, scan query doesn't retain all rows in memory before rows can be returned to client.  
 It will cause memory pressure if too many rows required by select query.  
 Scan query doesn't have this issue.  
-Scan query can return all rows without issuing another pagination query, which is extremely useful when query against historical or realtime node directly.
+Scan query can return all rows without issuing another pagination query, which is extremely useful when query against Historical or realtime node directly.
 
 ## Legacy mode
 
diff --git a/docs/content/querying/searchquery.md b/docs/content/querying/searchquery.md
index 8a9c5b1..3727a5d 100644
--- a/docs/content/querying/searchquery.md
+++ b/docs/content/querying/searchquery.md
@@ -56,7 +56,7 @@ There are several main parts to a search query:
 |dataSource|A String or Object defining the data source to query, very similar to a table in a relational database. See [DataSource](../querying/datasource.html) for more information.|yes|
 |granularity|Defines the granularity of the query. See [Granularities](../querying/granularities.html).|yes|
 |filter|See [Filters](../querying/filters.html).|no|
-|limit| Defines the maximum number per historical node (parsed as int) of search results to return. |no (default to 1000)|
+|limit| Defines the maximum number per Historical node (parsed as int) of search results to return. |no (default to 1000)|
 |intervals|A JSON Object representing ISO-8601 Intervals. This defines the time ranges to run the query over.|yes|
 |searchDimensions|The dimensions to run the search over. Excluding this means the search is run over all dimensions.|no|
 |query|See [SearchQuerySpec](../querying/searchqueryspec.html).|yes|
@@ -105,7 +105,7 @@ The format of the result is:
 #### Strategies
 
 Search queries can be executed using two different strategies. The default strategy is determined by the
-"druid.query.search.searchStrategy" runtime property on the broker. This can be overridden using "searchStrategy" in the
+"druid.query.search.searchStrategy" runtime property on the Broker. This can be overridden using "searchStrategy" in the
 query context. If neither the context field nor the property is set, the "useIndexes" strategy will be used.
 
 - "useIndexes" strategy, the default, first categorizes search dimensions into two groups according to their support for
diff --git a/docs/content/querying/segmentmetadataquery.md b/docs/content/querying/segmentmetadataquery.md
index ae6d6b8..aa7b9f1 100644
--- a/docs/content/querying/segmentmetadataquery.md
+++ b/docs/content/querying/segmentmetadataquery.md
@@ -93,7 +93,7 @@ Only columns which are dimensions (ie, have type `STRING`) will have any cardina
 
 If an interval is not specified, the query will use a default interval that spans a configurable period before the end time of the most recent segment.
 
-The length of this default time period is set in the broker configuration via:
+The length of this default time period is set in the Broker configuration via:
   druid.query.segmentMetadata.defaultHistory
 
 ### toInclude
@@ -130,7 +130,7 @@ This is a list of properties that determines the amount of information returned
 
 By default, the "cardinality", "interval", and "minmax" types will be used. If a property is not needed, omitting it from this list will result in a more efficient query.
 
-The default analysis types can be set in the broker configuration via:
+The default analysis types can be set in the Broker configuration via:
   `druid.query.segmentMetadata.defaultAnalysisTypes`
 
 Types of column analyses are described below:
diff --git a/docs/content/querying/sql.md b/docs/content/querying/sql.md
index 4a0c0d5..257963d 100644
--- a/docs/content/querying/sql.md
+++ b/docs/content/querying/sql.md
@@ -31,12 +31,12 @@ subject to change.
 
 Druid SQL is a built-in SQL layer and an alternative to Druid's native JSON-based query language, and is powered by a
 parser and planner based on [Apache Calcite](https://calcite.apache.org/). Druid SQL translates SQL into native Druid
-queries on the query broker (the first node you query), which are then passed down to data nodes as native Druid
-queries. Other than the (slight) overhead of translating SQL on the broker, there isn't an additional performance
+queries on the query Broker (the first node you query), which are then passed down to data nodes as native Druid
+queries. Other than the (slight) overhead of translating SQL on the Broker, there isn't an additional performance
 penalty versus native queries.
 
 To enable Druid SQL, make sure you have set `druid.sql.enable = true` either in your common.runtime.properties or your
-broker's runtime.properties.
+Broker's runtime.properties.
 
 ## Query syntax
 
@@ -322,18 +322,18 @@ computed in memory. See the TopN documentation for more details.
 - [GroupBy](groupbyquery.html) is used for all other aggregations, including any nested aggregation queries. Druid's
 GroupBy is a traditional aggregation engine: it delivers exact results and rankings and supports a wide variety of
 features. GroupBy aggregates in memory if it can, but it may spill to disk if it doesn't have enough memory to complete
-your query. Results are streamed back from data nodes through the broker if you ORDER BY the same expressions in your
+your query. Results are streamed back from data nodes through the Broker if you ORDER BY the same expressions in your
 GROUP BY clause, or if you don't have an ORDER BY at all. If your query has an ORDER BY referencing expressions that
-don't appear in the GROUP BY clause (like aggregation functions) then the broker will materialize a list of results in
+don't appear in the GROUP BY clause (like aggregation functions) then the Broker will materialize a list of results in
 memory, up to a max of your LIMIT, if any. See the GroupBy documentation for details about tuning performance and memory
 use.
 
 If your query does nested aggregations (an aggregation subquery in your FROM clause) then Druid will execute it as a
 [nested GroupBy](groupbyquery.html#nested-groupbys). In nested GroupBys, the innermost aggregation is distributed, but
-all outer aggregations beyond that take place locally on the query broker.
+all outer aggregations beyond that take place locally on the query Broker.
 
 Semi-join queries containing WHERE clauses like `col IN (SELECT expr FROM ...)` are executed with a special process. The
-broker will first translate the subquery into a GroupBy to find distinct values of `expr`. Then, the broker will rewrite
+Broker will first translate the subquery into a GroupBy to find distinct values of `expr`. Then, the broker will rewrite
 the subquery to a literal filter, like `col IN (val1, val2, ...)` and run the outer query. The configuration parameter
 druid.sql.planner.maxSemiJoinRowsInMemory controls the maximum number of values that will be materialized for this kind
 of plan.
@@ -351,10 +351,10 @@ Druid SQL will use approximate algorithms in some situations:
 - The `COUNT(DISTINCT col)` aggregation functions by default uses a variant of
 [HyperLogLog](http://algo.inria.fr/flajolet/Publications/FlFuGaMe07.pdf), a fast approximate distinct counting
 algorithm. Druid SQL will switch to exact distinct counts if you set "useApproximateCountDistinct" to "false", either
-through query context or through broker configuration.
+through query context or through Broker configuration.
 - GROUP BY queries over a single column with ORDER BY and LIMIT may be executed using the TopN engine, which uses an
 approximate algorithm. Druid SQL will switch to an exact grouping algorithm if you set "useApproximateTopN" to "false",
-either through query context or through broker configuration.
+either through query context or through Broker configuration.
 - The APPROX_COUNT_DISTINCT and APPROX_QUANTILE aggregation functions always use approximate algorithms, regardless
 of configuration.
 
@@ -444,7 +444,7 @@ you've downloaded the Avatica client jar, add it to your classpath and use the c
 Example code:
 
 ```java
-// Connect to /druid/v2/sql/avatica/ on your broker.
+// Connect to /druid/v2/sql/avatica/ on your Broker.
 String url = "jdbc:avatica:remote:url=http://localhost:8082/druid/v2/sql/avatica/";
 
 // Set any connection context parameters you need here (see "Connection context" below).
@@ -469,8 +469,8 @@ so avoid those.
 
 #### Connection stickiness
 
-Druid's JDBC server does not share connection state between brokers. This means that if you're using JDBC and have
-multiple Druid brokers, you should either connect to a specific broker, or use a load balancer with sticky sessions
+Druid's JDBC server does not share connection state between Brokers. This means that if you're using JDBC and have
+multiple Druid Brokers, you should either connect to a specific Broker, or use a load balancer with sticky sessions
 enabled. The Druid Router node provides connection stickiness when balancing JDBC requests, and can be used to achieve
 the necessary stickiness even with a normal non-sticky load balancer. Please see the
 [Router](../development/router.html) documentation for more details.
@@ -491,15 +491,15 @@ Connection context can be specified as JDBC connection properties or as a "conte
 |Parameter|Description|Default value|
 |---------|-----------|-------------|
 |`sqlQueryId`|Unique identifier given to this SQL query. For HTTP client, it will be returned in `X-Druid-SQL-Query-Id` header.|auto-generated|
-|`sqlTimeZone`|Sets the time zone for this connection, which will affect how time functions and timestamp literals behave. Should be a time zone name like "America/Los_Angeles" or offset like "-08:00".|druid.sql.planner.sqlTimeZone on the broker (default: UTC)|
-|`useApproximateCountDistinct`|Whether to use an approximate cardinalty algorithm for `COUNT(DISTINCT foo)`.|druid.sql.planner.useApproximateCountDistinct on the broker (default: true)|
-|`useApproximateTopN`|Whether to use approximate [TopN queries](topnquery.html) when a SQL query could be expressed as such. If false, exact [GroupBy queries](groupbyquery.html) will be used instead.|druid.sql.planner.useApproximateTopN on the broker (default: true)|
-|`useFallback`|Whether to evaluate operations on the broker when they cannot be expressed as Druid queries. This option is not recommended for production since it can generate unscalable query plans. If false, SQL queries that cannot be translated to Druid queries will fail.|druid.sql.planner.useFallback on the broker (default: false)|
+|`sqlTimeZone`|Sets the time zone for this connection, which will affect how time functions and timestamp literals behave. Should be a time zone name like "America/Los_Angeles" or offset like "-08:00".|druid.sql.planner.sqlTimeZone on the Broker (default: UTC)|
+|`useApproximateCountDistinct`|Whether to use an approximate cardinalty algorithm for `COUNT(DISTINCT foo)`.|druid.sql.planner.useApproximateCountDistinct on the Broker (default: true)|
+|`useApproximateTopN`|Whether to use approximate [TopN queries](topnquery.html) when a SQL query could be expressed as such. If false, exact [GroupBy queries](groupbyquery.html) will be used instead.|druid.sql.planner.useApproximateTopN on the Broker (default: true)|
+|`useFallback`|Whether to evaluate operations on the Broker when they cannot be expressed as Druid queries. This option is not recommended for production since it can generate unscalable query plans. If false, SQL queries that cannot be translated to Druid queries will fail.|druid.sql.planner.useFallback on the Broker (default: false)|
 
 ### Retrieving metadata
 
-Druid brokers infer table and column metadata for each dataSource from segments loaded in the cluster, and use this to
-plan SQL queries. This metadata is cached on broker startup and also updated periodically in the background through
+Druid Brokers infer table and column metadata for each dataSource from segments loaded in the cluster, and use this to
+plan SQL queries. This metadata is cached on Broker startup and also updated periodically in the background through
 [SegmentMetadata queries](segmentmetadataquery.html). Background metadata refreshing is triggered by
 segments entering and exiting the cluster, and can also be throttled through configuration.
 
@@ -581,14 +581,14 @@ Segments table provides details on all Druid segments, whether they are publishe
 |version|Version string (generally an ISO8601 timestamp corresponding to when the segment set was first started). Higher version means the more recently created segment. Version comparing is based on string comparison.|
 |partition_num|Partition number (an integer, unique within a datasource+interval+version; may not necessarily be contiguous)|
 |num_replicas|Number of replicas of this segment currently being served|
-|num_rows|Number of rows in current segment, this value could be null if unkown to broker at query time|
+|num_rows|Number of rows in current segment, this value could be null if unkown to Broker at query time|
 |is_published|Boolean is represented as long type where 1 = true, 0 = false. 1 represents this segment has been published to the metadata store|
-|is_available|Boolean is represented as long type where 1 = true, 0 = false. 1 if this segment is currently being served by any server(historical or realtime)|
+|is_available|Boolean is represented as long type where 1 = true, 0 = false. 1 if this segment is currently being served by any server(Historical or realtime)|
 |is_realtime|Boolean is represented as long type where 1 = true, 0 = false. 1 if this segment is being served on any type of realtime tasks|
 |payload|JSON-serialized data segment payload|
 
 ### SERVERS table
-Servers table lists all data servers(any server that hosts a segment). It includes both historicals and peons.
+Servers table lists all data servers(any server that hosts a segment). It includes both Historicals and Peons.
 
 |Column|Notes|
 |------|-----|
@@ -596,7 +596,7 @@ Servers table lists all data servers(any server that hosts a segment). It includ
 |host|Hostname of the server|
 |plaintext_port|Unsecured port of the server, or -1 if plaintext traffic is disabled|
 |tls_port|TLS port of the server, or -1 if TLS is disabled|
-|server_type|Type of Druid service. Possible values include: historical, realtime and indexer_executor(peon).|
+|server_type|Type of Druid service. Possible values include: Historical, realtime and indexer_executor(Peon).|
 |tier|Distribution tier see [druid.server.tier](#../configuration/index.html#Historical-General-Configuration)|
 |current_size|Current size of segments in bytes on this server|
 |max_size|Max size in bytes this server recommends to assign to segments see [druid.server.maxSize](#../configuration/index.html#Historical-General-Configuration)|
@@ -638,7 +638,7 @@ check out [ingestion tasks](#../ingestion/tasks.html)
 |type|Task type, for example this value is "index" for indexing tasks. See [tasks-overview](../ingestion/tasks.md)|
 |datasource|Datasource name being indexed|
 |created_time|Timestamp in ISO8601 format corresponding to when the ingestion task was created. Note that this value is populated for completed and waiting tasks. For running and pending tasks this value is set to 1970-01-01T00:00:00Z|
-|queue_insertion_time|Timestamp in ISO8601 format corresponding to when this task was added to the queue on the overlord|
+|queue_insertion_time|Timestamp in ISO8601 format corresponding to when this task was added to the queue on the Overlord|
 |status|Status of a task can be RUNNING, FAILED, SUCCESS|
 |runner_status|Runner status of a completed task would be NONE, for in-progress tasks this can be RUNNING, WAITING, PENDING|
 |duration|Time it took to finish the task in milliseconds, this value is present only for completed tasks|
@@ -656,7 +656,7 @@ SELECT * FROM sys.tasks where status='FAILED';
 
 ## Server configuration
 
-The Druid SQL server is configured through the following properties on the broker.
+The Druid SQL server is configured through the following properties on the Broker.
 
 |Property|Description|Default|
 |--------|-----------|-------|
@@ -674,7 +674,7 @@ The Druid SQL server is configured through the following properties on the broke
 |`druid.sql.planner.selectThreshold`|Page size threshold for [Select queries](../querying/select-query.html). Select queries for larger resultsets will be issued back-to-back using pagination.|1000|
 |`druid.sql.planner.useApproximateCountDistinct`|Whether to use an approximate cardinalty algorithm for `COUNT(DISTINCT foo)`.|true|
 |`druid.sql.planner.useApproximateTopN`|Whether to use approximate [TopN queries](../querying/topnquery.html) when a SQL query could be expressed as such. If false, exact [GroupBy queries](../querying/groupbyquery.html) will be used instead.|true|
-|`druid.sql.planner.useFallback`|Whether to evaluate operations on the broker when they cannot be expressed as Druid queries. This option is not recommended for production since it can generate unscalable query plans. If false, SQL queries that cannot be translated to Druid queries will fail.|false|
+|`druid.sql.planner.useFallback`|Whether to evaluate operations on the Broker when they cannot be expressed as Druid queries. This option is not recommended for production since it can generate unscalable query plans. If false, SQL queries that cannot be translated to Druid queries will fail.|false|
 |`druid.sql.planner.requireTimeCondition`|Whether to require SQL to have filter conditions on __time column so that all generated native queries will have user specified intervals. If true, all queries wihout filter condition on __time column will fail|false|
 |`druid.sql.planner.sqlTimeZone`|Sets the default time zone for the server, which will affect how time functions and timestamp literals behave. Should be a time zone name like "America/Los_Angeles" or offset like "-08:00".|UTC|
 
diff --git a/docs/content/querying/topnquery.md b/docs/content/querying/topnquery.md
index 8963f7b..db662a8 100644
--- a/docs/content/querying/topnquery.md
+++ b/docs/content/querying/topnquery.md
@@ -26,7 +26,7 @@ title: "TopN queries"
 
 TopN queries return a sorted set of results for the values in a given dimension according to some criteria. Conceptually, they can be thought of as an approximate [GroupByQuery](../querying/groupbyquery.html) over a single dimension with an [Ordering](../querying/limitspec.html) spec. TopNs are much faster and resource efficient than GroupBys for this use case. These types of queries take a topN query object and return an array of JSON objects where each object represents a value asked f [...]
 
-TopNs are approximate in that each node will rank their top K results and only return those top K results to the broker. K, by default in Druid, is `max(1000, threshold)`. In practice, this means that if you ask for the top 1000 items ordered, the correctness of the first ~900 items will be 100%, and the ordering of the results after that is not guaranteed. TopNs can be made more accurate by increasing the threshold.
+TopNs are approximate in that each node will rank their top K results and only return those top K results to the Broker. K, by default in Druid, is `max(1000, threshold)`. In practice, this means that if you ask for the top 1000 items ordered, the correctness of the first ~900 items will be 100%, and the ordering of the results after that is not guaranteed. TopNs can be made more accurate by increasing the threshold.
 
 A topN query object looks like:
 
diff --git a/docs/content/toc.md b/docs/content/toc.md
index f6e0800..5d67c19 100644
--- a/docs/content/toc.md
+++ b/docs/content/toc.md
@@ -101,13 +101,12 @@ layout: toc
   * [Overview](/docs/VERSION/design/index.html)
   * Storage
     * [Segments](/docs/VERSION/design/segments.html)
-  * Node Types
-    * [Historical](/docs/VERSION/design/historical.html)
-    * [Broker](/docs/VERSION/design/broker.html)
+  * [Processes and Servers](/docs/VERSION/design/processes.html)
     * [Coordinator](/docs/VERSION/design/coordinator.html)
-    * [Indexing Service](/docs/VERSION/design/indexing-service.html)
-      * [Overlord](/docs/VERSION/design/overlord.html)
-      * [MiddleManager](/docs/VERSION/design/middlemanager.html)
+    * [Overlord](/docs/VERSION/design/overlord.html)
+    * [Broker](/docs/VERSION/design/broker.html)
+    * [Historical](/docs/VERSION/design/historical.html)
+    * [MiddleManager](/docs/VERSION/design/middlemanager.html)
       * [Peons](/docs/VERSION/design/peons.html)
     * [Realtime (Deprecated)](/docs/VERSION/design/realtime.html)
   * Dependencies
diff --git a/docs/content/tutorials/cluster.md b/docs/content/tutorials/cluster.md
index c4c2b19..579ae1d 100644
--- a/docs/content/tutorials/cluster.md
+++ b/docs/content/tutorials/cluster.md
@@ -27,12 +27,19 @@ title: "Clustering"
 Druid is designed to be deployed as a scalable, fault-tolerant cluster.
 
 In this document, we'll set up a simple cluster and discuss how it can be further configured to meet
-your needs. This simple cluster will feature scalable, fault-tolerant servers for Historicals and MiddleManagers, and a single
-coordination server to host the Coordinator and Overlord processes. In production, we recommend deploying Coordinators and Overlords in a fault-tolerant
-configuration as well.
+your needs. 
+
+This simple cluster will feature:
+ - A single Master server to host the Coordinator and Overlord processes
+ - Scalable, fault-tolerant Data servers running Historical and MiddleManager processes
+ - Query servers, hosting Druid Broker processes
+
+In production, we recommend deploying multiple Master servers with Coordinator and Overlord processes in a fault-tolerant configuration as well.
 
 ## Select hardware
 
+### Master Server
+
 The Coordinator and Overlord processes can be co-located on a single server that is responsible for handling the metadata and coordination needs of your cluster.
 The equivalent of an AWS [m3.xlarge](https://aws.amazon.com/ec2/instance-types/#M3) is sufficient for most clusters. This
 hardware offers:
@@ -41,6 +48,8 @@ hardware offers:
 - 15 GB RAM
 - 80 GB SSD storage
 
+### Data Server
+
 Historicals and MiddleManagers can be colocated on a single server to handle the actual data in your cluster. These servers benefit greatly from CPU, RAM,
 and SSDs. The equivalent of an AWS [r3.2xlarge](https://aws.amazon.com/ec2/instance-types/#r3) is a
 good starting point. This hardware offers:
@@ -49,6 +58,8 @@ good starting point. This hardware offers:
 - 61 GB RAM
 - 160 GB SSD storage
 
+### Query Server
+
 Druid Brokers accept queries and farm them out to the rest of the cluster. They also optionally maintain an
 in-memory query cache. These servers benefit greatly from CPU and RAM, and can also be deployed on
 the equivalent of an AWS [r3.2xlarge](https://aws.amazon.com/ec2/instance-types/#r3). This hardware
@@ -191,7 +202,7 @@ using this functionality, then at this point you should
 If you will be loading data from a Hadoop cluster, then at this point you should configure Druid to be aware
 of your cluster:
 
-- Update `druid.indexer.task.hadoopWorkingPath` in `conf/middleManager/runtime.properties` to
+- Update `druid.indexer.task.hadoopWorkingPath` in `conf/druid/middleManager/runtime.properties` to
 a path on HDFS that you'd like to use for temporary files required during the indexing process.
 `druid.indexer.task.hadoopWorkingPath=/tmp/druid-indexing` is a common choice.
 
@@ -207,8 +218,13 @@ For more info, please see [batch ingestion](../ingestion/batch-ingestion.html).
 
 ## Configure addresses for Druid coordination
 
-In this simple cluster, you will deploy a single Druid Coordinator, a
-single Druid Overlord, a single ZooKeeper instance, and an embedded Derby metadata store on the same server.
+In this simple cluster, you will deploy a single Master server containing the following:
+- A single Druid Coordinator process
+- A single Druid Overlord process
+- A single ZooKeeper istance
+- An embedded Derby metadata store
+
+The processes on the cluster need to be configured with the addresses of this ZK instance and the metadata store.
 
 In `conf/druid/_common/common.runtime.properties`, replace
 "zk.service.host" with the address of the machine that runs your ZK instance:
@@ -222,13 +238,13 @@ In `conf/druid/_common/common.runtime.properties`, replace
 - `druid.metadata.storage.connector.host`
 
 <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,
+In production, we recommend running 2 Master servers, each running a Druid Coordinator process
+and a Druid Overlord process. We also recommend running a ZooKeeper cluster on its own dedicated hardware,
 as well as replicated <a href = "../dependencies/metadata-storage.html">metadata storage</a>
 such as MySQL or PostgreSQL, on its own dedicated hardware.
 </div>
 
-## Tune Druid processes that serve queries
+## Tune processes on the Data Server
 
 Druid Historicals and MiddleManagers can be co-located on the same hardware. Both Druid processes benefit greatly from
 being tuned to the hardware they run on. If you are running Tranquility Server or Kafka, you can also colocate Tranquility with these two Druid processes.
@@ -255,7 +271,7 @@ Keep -XX:MaxDirectMemory >= numThreads*sizeBytes, otherwise Druid will fail to s
 Please see the Druid [configuration documentation](../configuration/index.html) for a full description of all
 possible configuration options.
 
-## Tune Druid Brokers
+## Tune Druid Brokers on the Query Server
 
 Druid Brokers also benefit greatly from being tuned to the hardware they
 run on. If you are using [r3.2xlarge](https://aws.amazon.com/ec2/instance-types/#r3) EC2 instances,
@@ -284,27 +300,34 @@ possible configuration options.
 If you're using a firewall or some other system that only allows traffic on specific ports, allow
 inbound connections on the following:
 
-- 1527 (Derby on your Coordinator; not needed if you are using a separate metadata store like MySQL or PostgreSQL)
+### Master Server
+- 1527 (Derby metadata store; not needed if you are using a separate metadata store like MySQL or PostgreSQL)
 - 2181 (ZooKeeper; not needed if you are using a separate ZooKeeper cluster)
 - 8081 (Coordinator)
-- 8082 (Broker)
-- 8083 (Historical)
-- 8084 (Standalone Realtime, if used)
-- 8088 (Router, if used)
 - 8090 (Overlord)
+
+### Data Server
+- 8083 (Historical)
 - 8091, 8100&ndash;8199 (Druid Middle Manager; you may need higher than port 8199 if you have a very high `druid.worker.capacity`)
+
+### Query Server
+- 8082 (Broker)
+- 8088 (Router, if used)
+
+### Other
 - 8200 (Tranquility Server, if used)
+- 8084 (Standalone Realtime, if used, deprecated)
 
 <div class="note caution">
 In production, we recommend deploying ZooKeeper and your metadata store on their own dedicated hardware,
-rather than on the Coordinator server.
+rather than on the Master server.
 </div>
 
-## Start Coordinator, Overlord, Zookeeper, and metadata store
+## Start Master Server
+
+Copy the Druid distribution and your edited configurations to your Master server. 
 
-Copy the Druid distribution and your edited configurations to your coordination
-server. If you have been editing the configurations on your local machine, you can use *rsync* to
-copy them:
+If you have been editing the configurations on your local machine, you can use *rsync* to copy them:
 
 ```bash
 rsync -az apache-druid-#{DRUIDVERSION}/ COORDINATION_SERVER:apache-druid-#{DRUIDVERSION}/
@@ -334,18 +357,18 @@ java `cat conf/druid/overlord/jvm.config | xargs` -cp conf/druid/_common:conf/dr
 You should see a log message printed out for each service that starts up. You can view detailed logs
 for any service by looking in the `var/log/druid` directory using another terminal.
 
-## Start Historicals and MiddleManagers
+## Start Data Server
 
-Copy the Druid distribution and your edited configurations to your servers set aside for the Druid Historicals and MiddleManagers.
+Copy the Druid distribution and your edited configurations to your Data servers set aside for the Druid Historicals and MiddleManagers.
 
-On each one, *cd* into the distribution and run this command to start a Data server:
+On each one, *cd* into the distribution and run this command to start the Data server processes:
 
 ```bash
 java `cat conf/druid/historical/jvm.config | xargs` -cp conf/druid/_common:conf/druid/historical:lib/* org.apache.druid.cli.Main server historical
 java `cat conf/druid/middleManager/jvm.config | xargs` -cp conf/druid/_common:conf/druid/middleManager:lib/* org.apache.druid.cli.Main server middleManager
 ```
 
-You can add more servers with Druid Historicals and MiddleManagers as needed.
+You can add more Data servers with Druid Historicals and MiddleManagers as needed.
 
 <div class="note info">
 For clusters with complex resource allocation needs, you can break apart Historicals and MiddleManagers and scale the components individually.
@@ -365,17 +388,17 @@ cd tranquility-distribution-0.8.0
 bin/tranquility <server or kafka> -configFile <path_to_druid_distro>/conf/tranquility/<server or kafka>.json
 ```
 
-## Start Druid Broker
+## Start Query Server
 
-Copy the Druid distribution and your edited configurations to your servers set aside for the Druid Brokers.
+Copy the Druid distribution and your edited configurations to your Query servers set aside for the Druid Brokers.
 
-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):
+On each Query server, *cd* into the distribution and run this command to start the Broker process (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/* org.apache.druid.cli.Main server broker
 ```
 
-You can add more Brokers as needed based on query load.
+You can add more Query servers as needed based on query load.
 
 ## Loading data
 
diff --git a/docs/content/tutorials/tutorial-batch.md b/docs/content/tutorials/tutorial-batch.md
index 972569f..41f62aa 100644
--- a/docs/content/tutorials/tutorial-batch.md
+++ b/docs/content/tutorials/tutorial-batch.md
@@ -34,7 +34,7 @@ don't need to have loaded any data yet.
 
 ## Preparing the data and the ingestion task spec
 
-A data load is initiated by submitting an *ingestion task* spec to the Druid overlord. For this tutorial, we'll be loading the sample Wikipedia page edits data.
+A data load is initiated by submitting an *ingestion task* spec to the Druid Overlord. For this tutorial, we'll be loading the sample Wikipedia page edits data.
 
 The Druid package includes the following sample native batch ingestion task spec at `quickstart/tutorial/wikipedia-index.json`, shown here for convenience,
 which has been configured to read the `quickstart/tutorial/wikiticker-2015-09-12-sampled.json.gz` input file:
@@ -117,7 +117,7 @@ a task that loads the `wikiticker-2015-09-12-sampled.json.gz` file included in t
 
 For convenience, the Druid package includes a batch ingestion helper script at `bin/post-index-task`.
 
-This script will POST an ingestion task to the Druid overlord and poll Druid until the data is available for querying.
+This script will POST an ingestion task to the Druid Overlord and poll Druid until the data is available for querying.
 
 Run the following command from Druid package root:
 
@@ -163,13 +163,13 @@ Which will print the ID of the task if the submission was successful:
 {"task":"index_wikipedia_2018-06-09T21:30:32.802Z"}
 ```
 
-To view the status of the ingestion task, go to the overlord console:
+To view the status of the ingestion task, go to the Overlord console:
 [http://localhost:8090/console.html](http://localhost:8090/console.html). You can refresh the console periodically, and after
 the task is successful, you should see a "SUCCESS" status for the task.
 
-After the ingestion task finishes, the data will be loaded by historical nodes and available for
+After the ingestion task finishes, the data will be loaded by Historical nodes and available for
 querying within a minute or two. You can monitor the progress of loading the data in the
-coordinator console, by checking whether there is a datasource "wikipedia" with a blue circle
+Coordinator console, by checking whether there is a datasource "wikipedia" with a blue circle
 indicating "fully available": [http://localhost:8081/#/](http://localhost:8081/#/).
 
 ![Coordinator console](../tutorials/img/tutorial-batch-01.png "Wikipedia 100% loaded")
diff --git a/docs/content/tutorials/tutorial-compaction.md b/docs/content/tutorials/tutorial-compaction.md
index daf1f3c..201ea00 100644
--- a/docs/content/tutorials/tutorial-compaction.md
+++ b/docs/content/tutorials/tutorial-compaction.md
@@ -97,11 +97,11 @@ After the task finishes, refresh the http://localhost:8081/#/datasources/compact
 
 The original 24 segments will eventually be marked as "unused" by the Coordinator and removed, with the new compacted segment remaining. 
 
-By default, the Druid coordinator will not mark segments as unused until the coordinator process has been up for at least 15 minutes, so you may see the old segment set and the new compacted set at the same time in the coordinator, e.g.:
+By default, the Druid Coordinator will not mark segments as unused until the Coordinator process has been up for at least 15 minutes, so you may see the old segment set and the new compacted set at the same time in the Coordinator, e.g.:
 
 ![Compacted segments intermediate state](../tutorials/img/tutorial-compaction-01.png "Compacted segments intermediate state")
 
-The new compacted segment has a more recent version than the original segments, so even when both sets of segments are shown by the coordinator, queries will only read from the new compacted segment.
+The new compacted segment has a more recent version than the original segments, so even when both sets of segments are shown by the Coordinator, queries will only read from the new compacted segment.
 
 Let's try running a COUNT(*) on `compaction-tutorial` again, where the row count should still be 39,244:
 
@@ -115,7 +115,7 @@ dsql> select count(*) from "compaction-tutorial";
 Retrieved 1 row in 1.30s.
 ```
 
-After the coordinator has been running for at least 15 minutes, the http://localhost:8081/#/datasources/compaction-tutorial page should show there is only 1 segment:
+After the Coordinator has been running for at least 15 minutes, the http://localhost:8081/#/datasources/compaction-tutorial page should show there is only 1 segment:
 
 ![Compacted segments final state](../tutorials/img/tutorial-compaction-02.png "Compacted segments final state")
 
diff --git a/docs/content/tutorials/tutorial-delete-data.md b/docs/content/tutorials/tutorial-delete-data.md
index 146658d..6eba8f0 100644
--- a/docs/content/tutorials/tutorial-delete-data.md
+++ b/docs/content/tutorials/tutorial-delete-data.md
@@ -108,7 +108,7 @@ On http://localhost:8081/#/datasources/deletion-tutorial, click one of the remai
 
 The top of the info box shows the full segment ID, e.g. `deletion-tutorial_2016-06-27T14:00:00.000Z_2016-06-27T15:00:00.000Z_2018-07-27T22:57:00.110Z` for the segment of hour 14.
 
-Let's disable the hour 14 segment by sending the following DELETE request to the coordinator, where {SEGMENT-ID} is the full segment ID shown in the info box:
+Let's disable the hour 14 segment by sending the following DELETE request to the Coordinator, where {SEGMENT-ID} is the full segment ID shown in the info box:
 
 ```bash
 curl -XDELETE http://localhost:8081/druid/coordinator/v1/datasources/deletion-tutorial/segments/{SEGMENT-ID}
diff --git a/docs/content/tutorials/tutorial-query.md b/docs/content/tutorials/tutorial-query.md
index 1cc2839..a5cc7e2 100644
--- a/docs/content/tutorials/tutorial-query.md
+++ b/docs/content/tutorials/tutorial-query.md
@@ -57,7 +57,7 @@ Druid's native query format is expressed in JSON. We have included a sample nati
 
 This query retrieves the 10 Wikipedia pages with the most page edits on 2015-09-12.
 
-Let's submit this query to the Druid broker:
+Let's submit this query to the Druid Broker:
 
 ```bash
 curl -X 'POST' -H 'Content-Type:application/json' -d @quickstart/tutorial/wikipedia-top-pages.json http://localhost:8082/druid/v2?pretty
@@ -114,7 +114,7 @@ The SQL queries are submitted as JSON over HTTP.
 
 ### TopN query example
 
-The tutorial package includes an example file that contains the SQL query shown above at `quickstart/tutorial/wikipedia-top-pages-sql.json`. Let's submit that query to the Druid broker:
+The tutorial package includes an example file that contains the SQL query shown above at `quickstart/tutorial/wikipedia-top-pages-sql.json`. Let's submit that query to the Druid Broker:
 
 ```bash
 curl -X 'POST' -H 'Content-Type:application/json' -d @quickstart/tutorial/wikipedia-top-pages-sql.json http://localhost:8082/druid/v2/sql
diff --git a/docs/img/druid-architecture.png b/docs/img/druid-architecture.png
index cac289e..1f1906e 100644
Binary files a/docs/img/druid-architecture.png and b/docs/img/druid-architecture.png differ


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