You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by da...@apache.org on 2018/08/09 20:27:46 UTC

[incubator-druid] branch master updated: Unified API doc page (#6128)

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

davidlim 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 2b0f03a  Unified API doc page (#6128)
2b0f03a is described below

commit 2b0f03acb9a6bbf064f18c662ed9fe8c175edd65
Author: Jonathan Wei <jo...@users.noreply.github.com>
AuthorDate: Thu Aug 9 13:27:42 2018 -0700

    Unified API doc page (#6128)
    
    * Unified API doc page
    
    * PR comments
    
    * Fix metadata endpoint
---
 docs/content/design/broker.md                      |  43 +--
 docs/content/design/coordinator.md                 | 292 +-----------------
 docs/content/design/historical.md                  |  18 +-
 docs/content/design/indexing-service.md            |  56 +---
 docs/content/design/middlemanager.md               |   6 +-
 docs/content/design/peons.md                       |   5 +
 .../coordinator.md => operations/api-reference.md} | 327 ++++++++++++---------
 docs/content/toc.md                                |   7 +
 8 files changed, 208 insertions(+), 546 deletions(-)

diff --git a/docs/content/design/broker.md b/docs/content/design/broker.md
index 7f20dfa..1c3c346 100644
--- a/docs/content/design/broker.md
+++ b/docs/content/design/broker.md
@@ -31,45 +31,4 @@ historical nodes. Once the historical nodes return their results, the broker wil
 HTTP Endpoints
 --------------
 
-The broker node exposes several HTTP endpoints for interactions.
-
-### GET
-
-* `/status`
-
-Returns the Druid version, loaded extensions, memory used, total memory and other useful information about the node.
-
-* `/druid/v2/datasources`
-
-Returns a list of queryable datasources.
-
-* `/druid/v2/datasources/{dataSourceName}`
-
-Returns the dimensions and metrics of the datasource. Optionally, you can provide request parameter "full" to get list of served intervals with dimensions and metrics being served for those intervals. You can also provide request param "interval" explicitly to refer to a particular interval.
-
-If no interval is specified, a default interval spanning a configurable period before the current time will be used. The duration of this interval is specified in ISO8601 format via:
-
-druid.query.segmentMetadata.defaultHistory
-
-* `/druid/v2/datasources/{dataSourceName}/dimensions`
-
-Returns the dimensions of the datasource.
-
-* `/druid/v2/datasources/{dataSourceName}/metrics`
-
-Returns the metrics of the datasource.
-
-* `/druid/v2/datasources/{dataSourceName}/candidates?intervals={comma-separated-intervals-in-ISO8601-format}&numCandidates={numCandidates}`
-
-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.
-
-* `/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.
-
-
-### POST
-
-* `/druid/v2/candidates/`
-
-Returns segment information lists including server locations for the given query.
+For a list of API endpoints supported by the Broker, please see the [API reference](../operations/api-reference.html#broker).
diff --git a/docs/content/design/coordinator.md b/docs/content/design/coordinator.md
index 9723fb2..462c616 100644
--- a/docs/content/design/coordinator.md
+++ b/docs/content/design/coordinator.md
@@ -82,297 +82,7 @@ If it finds such segments, it simply skips compacting them.
 HTTP Endpoints
 --------------
 
-The coordinator node exposes several HTTP endpoints for interactions.
-
-### GET
-
-* `/status`
-
-Returns the Druid version, loaded extensions, memory used, total memory and other useful information about the node.
-
-#### Coordinator information
-
-* `/druid/coordinator/v1/leader`
-
-Returns the current leader coordinator of the cluster.
-
-* `/druid/coordinator/v1/isLeader`
-
-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.
-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.
-
-* `/druid/coordinator/v1/loadstatus`
-
-Returns the percentage of segments actually loaded in the cluster versus segments that should be loaded in the cluster.
-
- * `/druid/coordinator/v1/loadstatus?simple`
-
-Returns the number of segments left to load until segments that should be loaded in the cluster are available for queries. This does not include replication.
-
-* `/druid/coordinator/v1/loadstatus?full`
-
-Returns the number of segments left to load in each tier until segments that should be loaded in the cluster are all available. This includes replication.
-
-* `/druid/coordinator/v1/loadqueue`
-
-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.
-
-* `/druid/coordinator/v1/loadqueue?full`
-
-Returns the serialized JSON of segments to load and drop for each historical node.
-
-#### Metadata store information
-
-* `/druid/coordinator/v1/metadata/datasources`
-
-Returns a list of the names of enabled datasources in the cluster.
-
-* `/druid/coordinator/v1/metadata/datasources?includeDisabled`
-
-Returns a list of the names of enabled and disabled datasources in the cluster.
-
-* `/druid/coordinator/v1/metadata/datasources?full`
-
-Returns a list of all enabled datasources with all metadata about those datasources as stored in the metadata store.
-
-* `/druid/coordinator/v1/metadata/datasources/{dataSourceName}`
-
-Returns full metadata for a datasource as stored in the metadata store.
-
-* `/druid/coordinator/v1/metadata/datasources/{dataSourceName}/segments`
-
-Returns a list of all segments for a datasource as stored in the metadata store.
-
-* `/druid/coordinator/v1/metadata/datasources/{dataSourceName}/segments?full`
-
-Returns a list of all segments for a datasource with the full segment metadata as stored in the metadata store.
-
-* POST `/druid/coordinator/v1/metadata/datasources/{dataSourceName}/segments`
-
-Returns a list of all segments, overlapping with any of given intervals,  for a datasource 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"]
-
-* POST `/druid/coordinator/v1/metadata/datasources/{dataSourceName}/segments?full`
-
-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"]
-
-* `/druid/coordinator/v1/metadata/datasources/{dataSourceName}/segments/{segmentId}`
-
-Returns full segment metadata for a specific segment as stored in the metadata store.
-
-#### Datasources information
-
-* `/druid/coordinator/v1/datasources`
-
-Returns a list of datasource names found in the cluster.
-
-* `/druid/coordinator/v1/datasources?simple`
-
-Returns a list of JSON objects containing the name and properties of datasources found in the cluster.  Properties include segment count, total segment byte size, minTime, and maxTime.
-
-* `/druid/coordinator/v1/datasources?full`
-
-Returns a list of datasource names found in the cluster with all metadata about those datasources.
-
-* `/druid/coordinator/v1/datasources/{dataSourceName}`
-
-Returns a JSON object containing the name and properties of a datasource. Properties include segment count, total segment byte size, minTime, and maxTime.
-
-* `/druid/coordinator/v1/datasources/{dataSourceName}?full`
-
-Returns full metadata for a datasource .
-
-* `/druid/coordinator/v1/datasources/{dataSourceName}/intervals`
-
-Returns a set of segment intervals.
-
-* `/druid/coordinator/v1/datasources/{dataSourceName}/intervals?simple`
-
-Returns a map of an interval to a JSON object containing the total byte size of segments and number of segments for that interval.
-
-* `/druid/coordinator/v1/datasources/{dataSourceName}/intervals?full`
-
-Returns a map of an interval to a map of segment metadata to a set of server names that contain the segment for that interval.
-
-* `/druid/coordinator/v1/datasources/{dataSourceName}/intervals/{interval}`
-
-Returns a set of segment ids for an ISO8601 interval. Note that {interval} parameters are delimited by a `_` instead of a `/` (e.g., 2016-06-27_2016-06-28).
-
-* `/druid/coordinator/v1/datasources/{dataSourceName}/intervals/{interval}?simple`
-
-Returns a map of segment intervals contained within the specified interval to a JSON object containing the total byte size of segments and number of segments for an interval.
-
-* `/druid/coordinator/v1/datasources/{dataSourceName}/intervals/{interval}?full`
-
-Returns a map of segment intervals contained within the specified interval to a map of segment metadata to a set of server names that contain the segment for an interval.
-
-* `/druid/coordinator/v1/datasources/{dataSourceName}/intervals/{interval}/serverview`
-
-Returns a map of segment intervals contained within the specified interval to information about the servers that contain the segment for an interval.
-
-* `/druid/coordinator/v1/datasources/{dataSourceName}/segments`
-
-Returns a list of all segments for a datasource in the cluster.
-
-* `/druid/coordinator/v1/datasources/{dataSourceName}/segments?full`
-
-Returns a list of all segments for a datasource in the cluster with the full segment metadata.
-
-* `/druid/coordinator/v1/datasources/{dataSourceName}/segments/{segmentId}`
-
-Returns full segment metadata for a specific segment in the cluster.
-
-* `/druid/coordinator/v1/datasources/{dataSourceName}/tiers`
-
-Return the tiers that a datasource exists in.
-
-#### Rules
-
-* `/druid/coordinator/v1/rules`
-
-Returns all rules as JSON objects for all datasources in the cluster including the default datasource.
-
-* `/druid/coordinator/v1/rules/{dataSourceName}`
-
-Returns all rules for a specified datasource.
-
-
-* `/druid/coordinator/v1/rules/{dataSourceName}?full`
-
-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
-
-* `/druid/coordinator/v1/rules/history?count=<n>`
-
- Returns last <n> entries of audit history of rules for all datasources.
-
-* `/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
-
-* `/druid/coordinator/v1/rules/{dataSourceName}/history?count=<n>`
-
- Returns last <n> entries of audit history of rules for a specified datasource.
-
-#### Intervals
-
-Note that {interval} parameters are delimited by a `_` instead of a `/` (e.g., 2016-06-27_2016-06-28).
-
-* `/druid/coordinator/v1/intervals`
-
-Returns all intervals for all datasources with total size and count.
-
-* `/druid/coordinator/v1/intervals/{interval}`
-
-Returns aggregated total size and count for all intervals that intersect given isointerval.
-
-* `/druid/coordinator/v1/intervals/{interval}?simple`
-
-Returns total size and count for each interval within given isointerval.
-
-* `/druid/coordinator/v1/intervals/{interval}?full`
-
-Returns total size and count for each datasource for each interval within given isointerval.
-
-#### Compaction Configs
-
-* `/druid/coordinator/v1/config/compaction/`
-
-Returns all compaction configs.
-
-* `/druid/coordinator/v1/config/compaction/{dataSource}`
-
-Returns a compaction config of a dataSource.
-
-#### Servers
-
-* `/druid/coordinator/v1/servers`
-
-Returns a list of servers URLs using the format `{hostname}:{port}`. Note that
-nodes that run with different types will appear multiple times with different
-ports.
-
-* `/druid/coordinator/v1/servers?simple`
-
-Returns a list of server data objects in which each object has the following keys:
-- `host`: host URL include (`{hostname}:{port}`)
-- `type`: node type (`indexer-executor`, `historical`)
-- `currSize`: storage size currently used
-- `maxSize`: maximum storage size
-- `priority`
-- `tier`
-
-### POST
-
-#### Datasources
-
-* `/druid/coordinator/v1/datasources/{dataSourceName}`
-
-Enables all segments of datasource which are not overshadowed by others.
-
-* `/druid/coordinator/v1/datasources/{dataSourceName}/segments/{segmentId}`
-
-Enables a segment.
-
-#### Rules
-
-* `/druid/coordinator/v1/rules/{dataSourceName}`
-
-POST with a list of rules in JSON form to update rules.
-
-Optional Header Parameters for auditing the config change can also be specified.
-
-|Header Param Name| Description | Default |
-|----------|-------------|---------|
-|`X-Druid-Author`| author making the config change|""|
-|`X-Druid-Comment`| comment describing the change being done|""|
-
-#### Compaction Configs
-
-* `/druid/coordinator/v1/config/compaction?slotRatio={someRatio}&maxSlots={someMaxSlots}`
-
-Update the capacity for compaction tasks. `slotRatio` and `maxSlots` are used to limit the max number of compaction tasks.
-They mean the ratio of the total task slots to the copmaction task slots and the maximum number of task slots for compaction tasks, respectively.
-The actual max number of compaction tasks is `min(maxSlots, slotRatio * total task slots)`.
-Note that `slotRatio` and `maxSlots` are optional and can be omitted. If they are omitted, default values (0.1 and unbounded)
-will be set for them.
-
-* `/druid/coordinator/v1/config/compaction/{dataSource}`
-
-Creates or updates the compaction config for a dataSource. See [Compaction Configuration](../configuration/coordinator.html#compaction-configuration) for configuration details.
-
-### DELETE
-
-#### Datasources
-
-* `/druid/coordinator/v1/datasources/{dataSourceName}`
-
-Disables a datasource.
-
-* `/druid/coordinator/v1/datasources/{dataSourceName}/intervals/{interval}`
-* `@Deprecated. /druid/coordinator/v1/datasources/{dataSourceName}?kill=true&interval={myISO8601Interval}`
-
-Runs a [Kill task](../ingestion/tasks.html) for a given interval and datasource.
-
-Note that {interval} parameters are delimited by a `_` instead of a `/` (e.g., 2016-06-27_2016-06-28).
-
-* `/druid/coordinator/v1/datasources/{dataSourceName}/segments/{segmentId}`
-
-Disables a segment.
-
-#### Compaction Configs
-
-* `/druid/coordinator/v1/config/compaction/{dataSource}`
-
-Removes the compaction config for a dataSource.
+For a list of API endpoints supported by the Coordinator, please see the [API reference](../operations/api-reference.html#coordinator).
 
 The Coordinator Console
 ------------------
diff --git a/docs/content/design/historical.md b/docs/content/design/historical.md
index d6fd0f7..04f99dc 100644
--- a/docs/content/design/historical.md
+++ b/docs/content/design/historical.md
@@ -40,21 +40,5 @@ A historical can be configured to log and report metrics for every query it serv
 HTTP Endpoints
 --------------
 
-The historical node exposes several HTTP endpoints for interactions.
+For a list of API endpoints supported by the Historical, please see the [API reference](../operations/api-reference.html#historical).
 
-### GET
-
-* `/status`
-
-Returns the Druid version, loaded extensions, memory used, total memory and other useful information about the node.
-
-* `/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/design/indexing-service.md b/docs/content/design/indexing-service.md
index 43087b7..4435e69 100644
--- a/docs/content/design/indexing-service.md
+++ b/docs/content/design/indexing-service.md
@@ -33,52 +33,9 @@ In local mode overlord is also responsible for creating peons for executing task
 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.
 This mode is recommended if you intend to use the indexing service as the single endpoint for all Druid indexing.
 
-#### Leadership status
+#### HTTP Endpoints
 
-If you have multiple overlords, just one is leading at any given time. The others are on standby. To get the current
-leader overlord of the cluster, call:
-
-
-```
-http://<OVERLORD_IP>:<port>/druid/indexer/v1/leader
-```
-
-To see if a given server is the current leader overlord of the cluster, call:
-
-```
-http://<OVERLORD_IP>:<port>/druid/indexer/v1/isLeader
-```
-
-This returns a JSON object with field "leader", either true or false. In addition, this call 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.
-
-#### Submitting Tasks and Querying Task Status
-
-Tasks are submitted to the overlord node in the form of JSON objects. Tasks can be submitted via POST requests to:
-
-```
-http://<OVERLORD_IP>:<port>/druid/indexer/v1/task
-```
-this will return the taskId of the submitted task.
-
-Tasks can be shut down via POST requests to:
-
-```
-http://<OVERLORD_IP>:<port>/druid/indexer/v1/task/{taskId}/shutdown
-```
-
-Task statuses can be retrieved via GET requests to:
-
-```
-http://<OVERLORD_IP>:<port>/druid/indexer/v1/task/{taskId}/status
-```
-
-Task segments can be retrieved via GET requests to:
-
-```
-http://<OVERLORD_IP>:<port>/druid/indexer/v1/task/{taskId}/segments
-```
+For a list of API endpoints supported by the Overlord, please see the [API reference](../operations/api-reference.html#overlord).
 
 #### Overlord Console
 
@@ -120,12 +77,3 @@ Tasks
 -----
 
 See [Tasks](../ingestion/tasks.html).
-
-HTTP Endpoints
---------------
-
-### GET
-
-* `/status`
-
-Returns the Druid version, loaded extensions, memory used, total memory and other useful information about the node.
diff --git a/docs/content/design/middlemanager.md b/docs/content/design/middlemanager.md
index 39d1be8..cc1de96 100644
--- a/docs/content/design/middlemanager.md
+++ b/docs/content/design/middlemanager.md
@@ -20,8 +20,4 @@ io.druid.cli.Main server middleManager
 HTTP Endpoints
 --------------
 
-### GET
-
-* `/status`
-
-Returns the Druid version, loaded extensions, memory used, total memory and other useful information about the node.
+For a list of API endpoints supported by the MiddleManager, please see the [API reference](../operations/api-reference.html#middlemanager).
diff --git a/docs/content/design/peons.md b/docs/content/design/peons.md
index f4082ba..76bbc1f 100644
--- a/docs/content/design/peons.md
+++ b/docs/content/design/peons.md
@@ -21,3 +21,8 @@ io.druid.cli.Main internal peon <task_file> <status_file>
 
 The task file contains the task JSON object.
 The status file indicates where the task status will be output.
+
+HTTP Endpoints
+--------------
+
+For a list of API endpoints supported by the Peon, please see the [API reference](../operations/api-reference.html#peon).
diff --git a/docs/content/design/coordinator.md b/docs/content/operations/api-reference.md
similarity index 52%
copy from docs/content/design/coordinator.md
copy to docs/content/operations/api-reference.md
index 9723fb2..d933c4f 100644
--- a/docs/content/design/coordinator.md
+++ b/docs/content/operations/api-reference.md
@@ -1,96 +1,45 @@
 ---
 layout: doc_page
 ---
-Coordinator Node
-================
-For Coordinator Node Configuration, see [Coordinator Configuration](../configuration/coordinator.html).
 
-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.
+# API Reference
 
-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 [...]
+This page documents all of the API endpoints for each Druid service type.
 
-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  [...]
+## Table of Contents
+  * [Common](#common)
+  * [Coordinator](#coordinator)
+  * [Overlord](#overlord)
+  * [MiddleManager](#middlemanager)
+  * [Peon](#peon)
+  * [Broker](#broker)
+  * [Historical](#historical)
 
-### Running
+## Common
 
-```
-io.druid.cli.Main server coordinator
-```
+The following endpoints are supported by all nodes.
 
-Rules
------
+### Node information
 
-Segments can be automatically loaded and dropped from the cluster based on a set of rules. For more information on rules, see [Rule Configuration](../operations/rule-configuration.html).
+#### GET
 
-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.
-
-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 [...]
-
-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 [...]
-
-Compacting Segments
--------------------
-
-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 usage. Note that the data for an interval
-cannot be compacted across the segments.
-
-The coordinator first finds the segments to compact together based on the [segment search policy](#segment-search-policy).
-Once it finds some segments, it launches a [compact task](../ingestion/tasks.html#compaction-task) to compact those segments.
-The maximum number of running compact tasks is `max(sum of worker capacity * slotRatio, maxSlots)`.
-Note that even though `max(sum of worker capacity * slotRatio, maxSlots)` = 1, at least one compact task is always submitted
-once a compaction is configured for a dataSource. See [HTTP Endpoints](#http-endpoints) to set those values.
-
-Compact tasks might fail due to some reasons.
-
-- If the input segments of a compact task are removed or overshadowed before it starts, that compact task fails immediately.
-- If a task of a higher priority acquires a lock for an interval overlapping with the interval of a compact task, the compact task fails.
-
-Once a compact task fails, the coordinator simply finds the segments for the interval of the failed task again, and launches a new compact task in the next run.
-
-To use this feature, you need to set some configurations for dataSources you want to compact.
-Please see [Compaction Configuration](../configuration/coordinator.html#compaction-configuration) for more details.
-
-### Segment Search Policy
-
-#### Newest Segment First Policy
-
-This policy searches the segments of _all dataSources_ in inverse order of their intervals.
-For example, let me assume there are 3 dataSources (`ds1`, `ds2`, `ds3`) and 5 segments (`seg_ds1_2017-10-01_2017-10-02`, `seg_ds1_2017-11-01_2017-11-02`, `seg_ds2_2017-08-01_2017-08-02`, `seg_ds3_2017-07-01_2017-07-02`, `seg_ds3_2017-12-01_2017-12-02`) for those dataSources.
-The segment name indicates its dataSource and interval. The search result of newestSegmentFirstPolicy is [`seg_ds3_2017-12-01_2017-12-02`, `seg_ds1_2017-11-01_2017-11-02`, `seg_ds1_2017-10-01_2017-10-02`, `seg_ds2_2017-08-01_2017-08-02`, `seg_ds3_2017-07-01_2017-07-02`].
-
-Every run, this policy starts searching from the (very latest interval - [skipOffsetFromLatest](../configuration/coordinator.html#compaction-configuration)).
-This is to handle the late segments ingested to realtime dataSources.
+* `/status`
 
-<div class="note caution">
-This policy currently cannot handle the situation when there are a lot of small segments which have the same interval,
-and their total size exceeds <a href="../configuration/coordinator.html#compaction-config">targetCompactionSizebytes</a>.
-If it finds such segments, it simply skips compacting them.
-</div>
+Returns the Druid version, loaded extensions, memory used, total memory and other useful information about the node.
 
+* `/status/health`
 
-HTTP Endpoints
---------------
+An endpoint that always returns a boolean "true" value with a 200 OK response, useful for automated health checks.
 
-The coordinator node exposes several HTTP endpoints for interactions.
+* `/status/properties`
 
-### GET
+Returns the current configuration properties of the node.
 
-* `/status`
+## Coordinator
 
-Returns the Druid version, loaded extensions, memory used, total memory and other useful information about the node.
+### Leadership
 
-#### Coordinator information
+#### GET
 
 * `/druid/coordinator/v1/leader`
 
@@ -98,6 +47,12 @@ Returns the current leader coordinator of the cluster.
 
 * `/druid/coordinator/v1/isLeader`
 
+Returns true if the coordinator receiving the request is the current leader.
+
+### Segment Loading
+
+#### 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.
 This is suitable for use as a load balancer status check if you only want the active leader to be considered in-service
@@ -127,7 +82,9 @@ Returns the number of segments to load and drop, as well as the total segment lo
 
 Returns the serialized JSON of segments to load and drop for each historical node.
 
-#### Metadata store information
+### Metadata store information
+
+#### GET
 
 * `/druid/coordinator/v1/metadata/datasources`
 
@@ -153,19 +110,24 @@ Returns a list of all segments for a datasource as stored in the metadata store.
 
 Returns a list of all segments for a datasource with the full segment metadata as stored in the metadata store.
 
-* POST `/druid/coordinator/v1/metadata/datasources/{dataSourceName}/segments`
+* `/druid/coordinator/v1/metadata/datasources/{dataSourceName}/segments/{segmentId}`
+
+Returns full segment metadata for a specific segment as stored in the metadata store.
+
+#### POST
+
+* `/druid/coordinator/v1/metadata/datasources/{dataSourceName}/segments`
 
 Returns a list of all segments, overlapping with any of given intervals,  for a datasource 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"]
 
-* POST `/druid/coordinator/v1/metadata/datasources/{dataSourceName}/segments?full`
+* `/druid/coordinator/v1/metadata/datasources/{dataSourceName}/segments?full`
 
 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"]
 
-* `/druid/coordinator/v1/metadata/datasources/{dataSourceName}/segments/{segmentId}`
 
-Returns full segment metadata for a specific segment as stored in the metadata store.
+### Datasources
 
-#### Datasources information
+#### GET
 
 * `/druid/coordinator/v1/datasources`
 
@@ -231,7 +193,36 @@ Returns full segment metadata for a specific segment in the cluster.
 
 Return the tiers that a datasource exists in.
 
-#### Rules
+#### POST
+
+* `/druid/coordinator/v1/datasources/{dataSourceName}`
+
+Enables all segments of datasource which are not overshadowed by others.
+
+* `/druid/coordinator/v1/datasources/{dataSourceName}/segments/{segmentId}`
+
+Enables a segment of a datasource.
+
+#### DELETE
+
+* `/druid/coordinator/v1/datasources/{dataSourceName}`
+
+Disables a datasource.
+
+* `/druid/coordinator/v1/datasources/{dataSourceName}/intervals/{interval}`
+* `@Deprecated. /druid/coordinator/v1/datasources/{dataSourceName}?kill=true&interval={myISO8601Interval}`
+
+Runs a [Kill task](../ingestion/tasks.html) for a given interval and datasource.
+
+Note that {interval} parameters are delimited by a `_` instead of a `/` (e.g., 2016-06-27_2016-06-28).
+
+* `/druid/coordinator/v1/datasources/{dataSourceName}/segments/{segmentId}`
+
+Disables a segment.
+
+### Retention Rules
+
+#### GET
 
 * `/druid/coordinator/v1/rules`
 
@@ -261,8 +252,23 @@ Returns all rules for a specified datasource and includes default datasource.
 * `/druid/coordinator/v1/rules/{dataSourceName}/history?count=<n>`
 
  Returns last <n> entries of audit history of rules for a specified datasource.
+ 
+#### POST
+
+* `/druid/coordinator/v1/rules/{dataSourceName}`
+
+POST with a list of rules in JSON form to update rules.
+
+Optional Header Parameters for auditing the config change can also be specified.
+
+|Header Param Name| Description | Default |
+|----------|-------------|---------|
+|`X-Druid-Author`| author making the config change|""|
+|`X-Druid-Comment`| comment describing the change being done|""|
+
+### Intervals
 
-#### Intervals
+#### GET
 
 Note that {interval} parameters are delimited by a `_` instead of a `/` (e.g., 2016-06-27_2016-06-28).
 
@@ -282,7 +288,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 Configs
+### Compaction Configuration
+
+#### GET
 
 * `/druid/coordinator/v1/config/compaction/`
 
@@ -292,7 +300,29 @@ Returns all compaction configs.
 
 Returns a compaction config of a dataSource.
 
-#### Servers
+#### POST
+
+* `/druid/coordinator/v1/config/compaction?slotRatio={someRatio}&maxSlots={someMaxSlots}`
+
+Update the capacity for compaction tasks. `slotRatio` and `maxSlots` are used to limit the max number of compaction tasks.
+They mean the ratio of the total task slots to the copmaction task slots and the maximum number of task slots for compaction tasks, respectively.
+The actual max number of compaction tasks is `min(maxSlots, slotRatio * total task slots)`.
+Note that `slotRatio` and `maxSlots` are optional and can be omitted. If they are omitted, default values (0.1 and unbounded)
+will be set for them.
+
+* `/druid/coordinator/v1/config/compaction/{dataSource}`
+
+Creates or updates the compaction config for a dataSource. See [Compaction Configuration](../configuration/coordinator.html#compaction-configuration) for configuration details.
+
+#### DELETE
+
+* `/druid/coordinator/v1/config/compaction/{dataSource}`
+
+Removes the compaction config for a dataSource.
+
+### Server Information
+
+#### GET
 
 * `/druid/coordinator/v1/servers`
 
@@ -310,96 +340,119 @@ Returns a list of server data objects in which each object has the following key
 - `priority`
 - `tier`
 
-### POST
+## Overlord
 
-#### Datasources
+### Leadership
 
-* `/druid/coordinator/v1/datasources/{dataSourceName}`
+#### GET
 
-Enables all segments of datasource which are not overshadowed by others.
+* `/druid/indexer/v1/leader` 
 
-* `/druid/coordinator/v1/datasources/{dataSourceName}/segments/{segmentId}`
+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.
 
-Enables a segment.
+* `/druid/indexer/v1/isLeader`
 
-#### Rules
+This returns a JSON object with field "leader", either true or false. In addition, this call 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.
 
-* `/druid/coordinator/v1/rules/{dataSourceName}`
+### Tasks
 
-POST with a list of rules in JSON form to update rules.
+#### GET
 
-Optional Header Parameters for auditing the config change can also be specified.
+* `/druid/indexer/v1/task/{taskId}/status`
 
-|Header Param Name| Description | Default |
-|----------|-------------|---------|
-|`X-Druid-Author`| author making the config change|""|
-|`X-Druid-Comment`| comment describing the change being done|""|
+Retrieve the status of a task.
 
-#### Compaction Configs
+* `/druid/indexer/v1/task/{taskId}/segments`
 
-* `/druid/coordinator/v1/config/compaction?slotRatio={someRatio}&maxSlots={someMaxSlots}`
+Retrieve information about the segments of a task.
 
-Update the capacity for compaction tasks. `slotRatio` and `maxSlots` are used to limit the max number of compaction tasks.
-They mean the ratio of the total task slots to the copmaction task slots and the maximum number of task slots for compaction tasks, respectively.
-The actual max number of compaction tasks is `min(maxSlots, slotRatio * total task slots)`.
-Note that `slotRatio` and `maxSlots` are optional and can be omitted. If they are omitted, default values (0.1 and unbounded)
-will be set for them.
+#### POST
 
-* `/druid/coordinator/v1/config/compaction/{dataSource}`
+* `/druid/indexer/v1/task` 
 
-Creates or updates the compaction config for a dataSource. See [Compaction Configuration](../configuration/coordinator.html#compaction-configuration) for configuration details.
+Endpoint for submitting tasks and supervisor specs to the overlord. Returns the taskId of the submitted task.
 
-### DELETE
+* `druid/indexer/v1/task/{taskId}/shutdown`
 
-#### Datasources
+Shuts down a task.
 
-* `/druid/coordinator/v1/datasources/{dataSourceName}`
 
-Disables a datasource.
+## MiddleManager
 
-* `/druid/coordinator/v1/datasources/{dataSourceName}/intervals/{interval}`
-* `@Deprecated. /druid/coordinator/v1/datasources/{dataSourceName}?kill=true&interval={myISO8601Interval}`
+The MiddleManager does not have any API endpoints beyond the [common endpoints](#common).
 
-Runs a [Kill task](../ingestion/tasks.html) for a given interval and datasource.
+## Peon
 
-Note that {interval} parameters are delimited by a `_` instead of a `/` (e.g., 2016-06-27_2016-06-28).
+The Peon does not have any API endpoints beyond the [common endpoints](#common).
 
-* `/druid/coordinator/v1/datasources/{dataSourceName}/segments/{segmentId}`
+## Broker
 
-Disables a segment.
 
-#### Compaction Configs
+### Datasource Information
 
-* `/druid/coordinator/v1/config/compaction/{dataSource}`
+#### GET
 
-Removes the compaction config for a dataSource.
+* `/druid/v2/datasources`
 
-The Coordinator Console
-------------------
+Returns a list of queryable datasources.
 
-The Druid coordinator exposes a web GUI for displaying cluster information and rule configuration. After the coordinator starts, the console can be accessed at:
+* `/druid/v2/datasources/{dataSourceName}`
 
-```
-http://<COORDINATOR_IP>:<COORDINATOR_PORT>
-```
+Returns the dimensions and metrics of the datasource. Optionally, you can provide request parameter "full" to get list of served intervals with dimensions and metrics being served for those intervals. You can also provide request param "interval" explicitly to refer to a particular interval.
 
- 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.
+If no interval is specified, a default interval spanning a configurable period before the current time will be used. The duration of this interval is specified in ISO8601 format via:
 
-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.
+druid.query.segmentMetadata.defaultHistory
+
+* `/druid/v2/datasources/{dataSourceName}/dimensions`
+
+Returns the dimensions of the datasource.
+
+* `/druid/v2/datasources/{dataSourceName}/metrics`
+
+Returns the metrics of the datasource.
+
+* `/druid/v2/datasources/{dataSourceName}/candidates?intervals={comma-separated-intervals-in-ISO8601-format}&numCandidates={numCandidates}`
+
+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
+
+#### 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.
+
+
+### Queries
+
+#### POST
+
+* `/druid/v2/`
+
+The endpoint for submitting queries. Accepts an option `?pretty` that pretty prints the results.
+
+* `/druid/v2/candidates/`
+
+Returns segment information lists including server locations for the given query..
 
-FAQ
----
 
-1. **Do clients ever contact the coordinator node?**
+## Historical
 
-    The coordinator is not involved in a query.
+### Segment Loading
 
-    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.
+#### GET
 
-    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.
+* `/druid/historical/v1/loadstatus`
 
-2. **Does it matter if the coordinator node starts up before or after other processes?**
+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.
 
-    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.
+* `/druid/historical/v1/readiness`
 
-    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.
+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/toc.md b/docs/content/toc.md
index a9d91c8..fcf905a 100644
--- a/docs/content/toc.md
+++ b/docs/content/toc.md
@@ -67,6 +67,13 @@ layout: toc
     * [ZooKeeper](/docs/VERSION/dependencies/zookeeper.html)
 
 ## Operations
+  * [API Reference](/docs/VERSION/operations/api-reference.html)
+    * [Coordinator](/docs/VERSION/operations/api-reference.html#coordinator)
+    * [Overlord](/docs/VERSION/operations/api-reference.html#overlord)
+    * [MiddleManager](/docs/VERSION/operations/api-reference.html#middlemanager)
+    * [Peon](/docs/VERSION/operations/api-reference.html#peon)
+    * [Broker](/docs/VERSION/operations/api-reference.html#broker)
+    * [Historical](/docs/VERSION/operations/api-reference.html#historical)
   * [Good Practices](/docs/VERSION/operations/recommendations.html)
   * [Including Extensions](/docs/VERSION/operations/including-extensions.html)
   * [Data Retention](/docs/VERSION/operations/rule-configuration.html)


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