diff --git a/docs/api-reference/service-status-api.md b/docs/api-reference/service-status-api.md
index 614d216ca0d..88059b1450a 100644
--- a/docs/api-reference/service-status-api.md
+++ b/docs/api-reference/service-status-api.md
@@ -23,154 +23,1198 @@ sidebar_label: Service status
~ under the License.
-->
-This document describes the API endpoints to retrieve service (process) status, cluster information for Apache Druid
+
+This document describes the API endpoints to retrieve service status, cluster information for Apache Druid.
+
+In this document, `http://SERVICE_IP:SERVICE_PORT` is a placeholder for the server address of deployment and the service port. For example, on the quickstart configuration, replace `http://ROUTER_IP:ROUTER_PORT` with `http://localhost:8888`.
## Common
-All processes support the following endpoints.
+All services support the following endpoints.
-### Process information
+You can use each endpoint with the ports for each type of service. The following table contains port addresses for a local configuration:
-`GET /status`
+|Service|Port address|
+| ------ | ------------ |
+| Coordinator|8081|
+| Overlord|8081|
+| Router|8888|
+| Broker|8082|
+| Historical|8083|
+| MiddleManager|8091|
-Returns the Druid version, loaded extensions, memory used, total memory, and other useful information about the process.
+### Get service information
-`GET /status/health`
+Retrieves the Druid version, loaded extensions, memory used, total memory, and other useful information about the individual service.
-Always returns a boolean `true` value with a 200 OK response, useful for automated health checks.
+Modify the host and port for the endpoint to match the service to query. Refer to the [default service ports](#common) for the port numbers.
-`GET /status/properties`
+#### URL
-Returns the current configuration properties of the process.
+GET
/status
-`GET /status/selfDiscovered/status`
+#### Responses
-Returns a JSON map of the form `{"selfDiscovered": true/false}`, indicating whether the node has received a confirmation
-from the central node discovery mechanism (currently ZooKeeper) of the Druid cluster that the node has been added to the
-cluster. It is recommended to not consider a Druid node "healthy" or "ready" in automated deployment/container
-management systems until it returns `{"selfDiscovered": true}` from this endpoint. This is because a node may be
-isolated from the rest of the cluster due to network issues and it doesn't make sense to consider nodes "healthy" in
-this case. Also, when nodes such as Brokers use ZooKeeper segment discovery for building their view of the Druid cluster
-(as opposed to HTTP segment discovery), they may be unusable until the ZooKeeper client is fully initialized and starts
-to receive data from the ZooKeeper cluster. `{"selfDiscovered": true}` is a proxy event indicating that the ZooKeeper
-client on the node has started to receive data from the ZooKeeper cluster and it's expected that all segments and other
-nodes will be discovered by this node timely from this point.
+
-`GET /status/selfDiscovered`
+
-Similar to `/status/selfDiscovered/status`, but returns 200 OK response with empty body if the node has discovered itself
-and 503 SERVICE UNAVAILABLE if the node hasn't discovered itself yet. This endpoint might be useful because some
-monitoring checks such as AWS load balancer health checks are not able to look at the response body.
+
-## Master server
+*Successfully retrieved service information*
-### Coordinator
+
-#### Leadership
+---
-`GET /druid/coordinator/v1/leader`
+#### Sample request
-Returns the current leader Coordinator of the cluster.
+
-`GET /druid/coordinator/v1/isLeader`
+
-Returns a JSON object with `leader` parameter, 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.
-
-
-
-### Overlord
-
-#### Leadership
-
-`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.
-
-`GET /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.
-
-## Data server
-
-### MiddleManager
-
-`GET /druid/worker/v1/enabled`
-
-Check whether a MiddleManager is in an enabled or disabled state. Returns JSON object keyed by the combined `druid.host`
-and `druid.port` with the boolean state as the value.
-
-```json
-{"localhost:8091":true}
+```shell
+curl "http://ROUTER_IP:ROUTER_PORT/status"
```
-`GET /druid/worker/v1/tasks`
+
-Retrieve a list of active tasks being run on MiddleManager. Returns JSON list of taskid strings. Normal usage should
-prefer to use the `/druid/indexer/v1/tasks` [Tasks API](./tasks-api.md) or one of it's task state specific variants instead.
-
-```json
-["index_wikiticker_2019-02-11T02:20:15.316Z"]
+```http
+GET /status HTTP/1.1
+Host: http://ROUTER_IP:ROUTER_PORT
```
-`GET /druid/worker/v1/task/{taskid}/log`
+
-Retrieve task log output stream by task id. Normal usage should prefer to use the `/druid/indexer/v1/task/{taskId}/log`
-[Tasks API](./tasks-api.md) instead.
+#### Sample response
-`POST /druid/worker/v1/disable`
+
+ Click to show sample response
+
+ ```json
+ {
+ "version": "26.0.0",
+ "modules": [
+ {
+ "name": "org.apache.druid.common.aws.AWSModule",
+ "artifact": "druid-aws-common",
+ "version": "26.0.0"
+ },
+ {
+ "name": "org.apache.druid.common.gcp.GcpModule",
+ "artifact": "druid-gcp-common",
+ "version": "26.0.0"
+ },
+ {
+ "name": "org.apache.druid.storage.hdfs.HdfsStorageDruidModule",
+ "artifact": "druid-hdfs-storage",
+ "version": "26.0.0"
+ },
+ {
+ "name": "org.apache.druid.indexing.kafka.KafkaIndexTaskModule",
+ "artifact": "druid-kafka-indexing-service",
+ "version": "26.0.0"
+ },
+ {
+ "name": "org.apache.druid.query.aggregation.datasketches.theta.SketchModule",
+ "artifact": "druid-datasketches",
+ "version": "26.0.0"
+ },
+ {
+ "name": "org.apache.druid.query.aggregation.datasketches.theta.oldapi.OldApiSketchModule",
+ "artifact": "druid-datasketches",
+ "version": "26.0.0"
+ },
+ {
+ "name": "org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchModule",
+ "artifact": "druid-datasketches",
+ "version": "26.0.0"
+ },
+ {
+ "name": "org.apache.druid.query.aggregation.datasketches.tuple.ArrayOfDoublesSketchModule",
+ "artifact": "druid-datasketches",
+ "version": "26.0.0"
+ },
+ {
+ "name": "org.apache.druid.query.aggregation.datasketches.hll.HllSketchModule",
+ "artifact": "druid-datasketches",
+ "version": "26.0.0"
+ },
+ {
+ "name": "org.apache.druid.query.aggregation.datasketches.kll.KllSketchModule",
+ "artifact": "druid-datasketches",
+ "version": "26.0.0"
+ },
+ {
+ "name": "org.apache.druid.msq.guice.MSQExternalDataSourceModule",
+ "artifact": "druid-multi-stage-query",
+ "version": "26.0.0"
+ },
+ {
+ "name": "org.apache.druid.msq.guice.MSQIndexingModule",
+ "artifact": "druid-multi-stage-query",
+ "version": "26.0.0"
+ },
+ {
+ "name": "org.apache.druid.msq.guice.MSQDurableStorageModule",
+ "artifact": "druid-multi-stage-query",
+ "version": "26.0.0"
+ },
+ {
+ "name": "org.apache.druid.msq.guice.MSQServiceClientModule",
+ "artifact": "druid-multi-stage-query",
+ "version": "26.0.0"
+ },
+ {
+ "name": "org.apache.druid.msq.guice.MSQSqlModule",
+ "artifact": "druid-multi-stage-query",
+ "version": "26.0.0"
+ },
+ {
+ "name": "org.apache.druid.msq.guice.SqlTaskModule",
+ "artifact": "druid-multi-stage-query",
+ "version": "26.0.0"
+ }
+ ],
+ "memory": {
+ "maxMemory": 268435456,
+ "totalMemory": 268435456,
+ "freeMemory": 139060688,
+ "usedMemory": 129374768,
+ "directMemory": 134217728
+ }
+ }
+ ```
+
-Disable a MiddleManager, causing it to stop accepting new tasks but complete all existing tasks. Returns JSON object
-keyed by the combined `druid.host` and `druid.port`:
+### Get service health
-```json
-{"localhost:8091":"disabled"}
+Retrieves the online status of the individual Druid service. It is a simple health check to determine if the service is running and accessible. If online, it will always return a boolean `true` value, indicating that the service can receive API calls. This endpoint is suitable for automated health checks.
+
+Modify the host and port for the endpoint to match the service to query. Refer to the [default service ports](#common) for the port numbers.
+
+Additional checks for readiness should use the [Historical segment readiness](#get-segment-readiness) and [Broker query readiness](#get-broker-query-readiness) endpoints.
+
+#### URL
+
+GET
/status/health
+
+#### Responses
+
+
+
+
+
+
+
+*Successfully retrieved service health*
+
+
+
+#### Sample request
+
+
+
+
+
+```shell
+curl "http://ROUTER_IP:ROUTER_PORT/status/health"
```
-`POST /druid/worker/v1/enable`
+
-Enable a MiddleManager, allowing it to accept new tasks again if it was previously disabled. Returns JSON object
-keyed by the combined `druid.host` and `druid.port`:
-
-```json
-{"localhost:8091":"enabled"}
+```http
+GET /status/health HTTP/1.1
+Host: http://ROUTER_IP:ROUTER_PORT
```
-`POST /druid/worker/v1/task/{taskid}/shutdown`
+
-Shutdown a running task by `taskid`. Normal usage should prefer to use the `/druid/indexer/v1/task/{taskId}/shutdown`
-[Tasks API](./tasks-api.md) instead. Returns JSON:
+#### Sample response
-```json
-{"task":"index_kafka_wikiticker_f7011f8ffba384b_fpeclode"}
+
+ Click to show sample response
+
+ ```json
+ true
+ ```
+
+
+
+
+### Get configuration properties
+
+Retrieves the current configuration properties of the individual service queried.
+
+Modify the host and port for the endpoint to match the service to query. Refer to the [default service ports](#common) for the port numbers.
+
+#### URL
+
+GET
/status/properties
+
+#### Responses
+
+
+
+
+
+
+
+*Successfully retrieved service configuration properties*
+
+
+
+#### Sample request
+
+
+
+
+
+```shell
+curl "http://ROUTER_IP:ROUTER_PORT/status/properties"
```
+
+
+```http
+GET /status/properties HTTP/1.1
+Host: http://ROUTER_IP:ROUTER_PORT
+```
+
+
+
+#### Sample response
+
+
+ Click to show sample response
+
+ ```json
+ {
+{
+ "gopherProxySet": "false",
+ "awt.toolkit": "sun.lwawt.macosx.LWCToolkit",
+ "druid.monitoring.monitors": "[\"org.apache.druid.java.util.metrics.JvmMonitor\"]",
+ "java.specification.version": "11",
+ "sun.cpu.isalist": "",
+ "druid.plaintextPort": "8888",
+ "sun.jnu.encoding": "UTF-8",
+ "druid.indexing.doubleStorage": "double",
+ "druid.metadata.storage.connector.port": "1527",
+ "java.class.path": "/Users/genericUserPath",
+ "log4j.shutdownHookEnabled": "true",
+ "java.vm.vendor": "Homebrew",
+ "sun.arch.data.model": "64",
+ "druid.extensions.loadList": "[\"druid-hdfs-storage\", \"druid-kafka-indexing-service\", \"druid-datasketches\", \"druid-multi-stage-query\"]",
+ "java.vendor.url": "https://github.com/Homebrew/homebrew-core/issues",
+ "druid.router.coordinatorServiceName": "druid/coordinator",
+ "user.timezone": "UTC",
+ "druid.global.http.eagerInitialization": "false",
+ "os.name": "Mac OS X",
+ "java.vm.specification.version": "11",
+ "sun.java.launcher": "SUN_STANDARD",
+ "user.country": "US",
+ "sun.boot.library.path": "/opt/homebrew/Cellar/openjdk@11/11.0.19/libexec/openjdk.jdk/Contents/Home/lib",
+ "sun.java.command": "org.apache.druid.cli.Main server router",
+ "http.nonProxyHosts": "local|*.local|169.254/16|*.169.254/16",
+ "jdk.debug": "release",
+ "druid.metadata.storage.connector.host": "localhost",
+ "sun.cpu.endian": "little",
+ "druid.zk.paths.base": "/druid",
+ "user.home": "/Users/genericUser",
+ "user.language": "en",
+ "java.specification.vendor": "Oracle Corporation",
+ "java.version.date": "2023-04-18",
+ "java.home": "/opt/homebrew/Cellar/openjdk@11/11.0.19/libexec/openjdk.jdk/Contents/Home",
+ "druid.service": "druid/router",
+ "druid.selectors.coordinator.serviceName": "druid/coordinator",
+ "druid.metadata.storage.connector.connectURI": "jdbc:derby://localhost:1527/var/druid/metadata.db;create=true",
+ "file.separator": "/",
+ "druid.selectors.indexing.serviceName": "druid/overlord",
+ "java.vm.compressedOopsMode": "Zero based",
+ "druid.metadata.storage.type": "derby",
+ "line.separator": "\n",
+ "druid.log.path": "/Users/genericUserPath",
+ "java.vm.specification.vendor": "Oracle Corporation",
+ "java.specification.name": "Java Platform API Specification",
+ "druid.indexer.logs.directory": "var/druid/indexing-logs",
+ "java.awt.graphicsenv": "sun.awt.CGraphicsEnvironment",
+ "druid.router.defaultBrokerServiceName": "druid/broker",
+ "druid.storage.storageDirectory": "var/druid/segments",
+ "sun.management.compiler": "HotSpot 64-Bit Tiered Compilers",
+ "ftp.nonProxyHosts": "local|*.local|169.254/16|*.169.254/16",
+ "java.runtime.version": "11.0.19+0",
+ "user.name": "genericUser",
+ "druid.indexer.logs.type": "file",
+ "druid.host": "localhost",
+ "log4j2.is.webapp": "false",
+ "path.separator": ":",
+ "os.version": "12.6.5",
+ "druid.lookup.enableLookupSyncOnStartup": "false",
+ "java.runtime.name": "OpenJDK Runtime Environment",
+ "druid.zk.service.host": "localhost",
+ "file.encoding": "UTF-8",
+ "druid.sql.planner.useGroupingSetForExactDistinct": "true",
+ "druid.router.managementProxy.enabled": "true",
+ "java.vm.name": "OpenJDK 64-Bit Server VM",
+ "java.vendor.version": "Homebrew",
+ "druid.startup.logging.logProperties": "true",
+ "java.vendor.url.bug": "https://github.com/Homebrew/homebrew-core/issues",
+ "log4j.shutdownCallbackRegistry": "org.apache.druid.common.config.Log4jShutdown",
+ "java.io.tmpdir": "var/tmp",
+ "druid.sql.enable": "true",
+ "druid.emitter.logging.logLevel": "info",
+ "java.version": "11.0.19",
+ "user.dir": "/Users/genericUser/Downloads/apache-druid-26.0.0",
+ "os.arch": "aarch64",
+ "java.vm.specification.name": "Java Virtual Machine Specification",
+ "druid.node.type": "router",
+ "java.awt.printerjob": "sun.lwawt.macosx.CPrinterJob",
+ "sun.os.patch.level": "unknown",
+ "java.util.logging.manager": "org.apache.logging.log4j.jul.LogManager",
+ "java.library.path": "/Users/genericUserPath",
+ "java.vendor": "Homebrew",
+ "java.vm.info": "mixed mode",
+ "java.vm.version": "11.0.19+0",
+ "druid.emitter": "noop",
+ "sun.io.unicode.encoding": "UnicodeBig",
+ "druid.storage.type": "local",
+ "druid.expressions.useStrictBooleans": "true",
+ "java.class.version": "55.0",
+ "socksNonProxyHosts": "local|*.local|169.254/16|*.169.254/16",
+ "druid.server.hiddenProperties": "[\"druid.s3.accessKey\",\"druid.s3.secretKey\",\"druid.metadata.storage.connector.password\", \"password\", \"key\", \"token\", \"pwd\"]"
+}
+```
+
+
+
+
+### Get node discovery status and cluster integration confirmation
+
+Retrieves a JSON map of the form `{"selfDiscovered": true/false}`, indicating whether the node has received a confirmation from the central node discovery mechanism (currently ZooKeeper) of the Druid cluster that the node has been added to the cluster.
+
+Only consider a Druid node "healthy" or "ready" in automated deployment/container management systems when this endpoint returns `{"selfDiscovered": true}`. Nodes experiencing network issues may become isolated and are not healthy.
+For nodes that use Zookeeper segment discovery, a response of `{"selfDiscovered": true}` indicates that the node's Zookeeper client has started receiving data from the Zookeeper cluster, enabling timely discovery of segments and other nodes.
+
+#### URL
+
+GET
/status/selfDiscovered/status
+
+#### Responses
+
+
+
+
+
+
+
+*Node was successfully added to the cluster*
+
+
+
+#### Sample request
+
+
+
+
+
+```shell
+curl "http://ROUTER_IP:ROUTER_PORT/status/selfDiscovered/status"
+```
+
+
+
+```http
+GET /status/selfDiscovered/status HTTP/1.1
+Host: http://ROUTER_IP:ROUTER_PORT
+```
+
+
+
+#### Sample response
+
+
+ Click to show sample response
+
+ ```json
+ {
+ "selfDiscovered": true
+ }
+ ```
+
+
+
+
+### Get node self-discovery status
+
+Returns an HTTP status code to indicate node discovery within the Druid cluster. This endpoint is similar to the `status/selfDiscovered/status` endpoint, but relies on HTTP status codes alone.
+Use this endpoint for monitoring checks that are unable to examine the response body. For example, AWS load balancer health checks.
+
+#### URL
+
+GET
/status/selfDiscovered
+
+#### Responses
+
+
+
+
+
+
+
+*Successfully retrieved node status*
+
+
+
+
+
+*Unsuccessful node self-discovery*
+
+
+
+#### Sample request
+
+
+
+
+
+```shell
+curl "http://ROUTER_IP:ROUTER_PORT/status/selfDiscovered"
+```
+
+
+
+```http
+GET /status/selfDiscovered HTTP/1.1
+Host: http://ROUTER_IP:ROUTER_PORT
+```
+
+
+
+#### Sample response
+
+A successful response to this endpoint results in an empty response body.
+
+## Coordinator
+
+### Get Coordinator leader address
+
+Retrieves the address of the current leader Coordinator of the cluster. If any request is sent to a non-leader Coordinator, the request is automatically redirected to the leader Coordinator.
+
+#### URL
+
+GET
/druid/coordinator/v1/leader
+
+#### Responses
+
+
+
+
+
+
+
+*Successfully retrieved leader Coordinator address*
+
+
+
+---
+
+#### Sample request
+
+
+
+
+
+```shell
+curl "http://ROUTER_IP:ROUTER_PORT/druid/coordinator/v1/leader"
+```
+
+
+
+```http
+GET /druid/coordinator/v1/leader HTTP/1.1
+Host: http://ROUTER_IP:ROUTER_PORT
+```
+
+
+
+#### Sample response
+
+
+ Click to show sample response
+
+ ```json
+ http://localhost:8081
+ ```
+
+
+
+### Get Coordinator leader status
+
+Retrieves a JSON object with a `leader` key. Returns `true` if this server is the current leader Coordinator of the cluster. To get the individual address of the leader Coordinator node, see the [leader endpoint](#get-coordinator-leader-address).
+
+Use this endpoint as a load balancer status check when you only want the active leader to be considered in-service at the load balancer.
+
+#### URL
+
+GET
/druid/coordinator/v1/isLeader
+
+#### Responses
+
+
+
+
+
+
+
+*Current server is the leader*
+
+
+
+---
+
+#### Sample request
+
+
+
+
+
+```shell
+curl "http://COORDINATOR_IP:COORDINATOR_PORT/druid/coordinator/v1/isLeader"
+```
+
+
+
+```http
+GET /druid/coordinator/v1/isLeader HTTP/1.1
+Host: http://COORDINATOR_IP:COORDINATOR_PORT
+```
+
+
+
+#### Sample response
+
+
+ Click to show sample response
+
+ ```json
+ {
+ "leader": true
+ }
+ ```
+
+
+
+## Overlord
+
+### Get Overlord leader address
+
+Retrieves the address of the current leader Overlord of the cluster. In a cluster of multiple Overlords, only one Overlord assumes the leading role, while the remaining Overlords remain on standby.
+
+#### URL
+
+GET
/druid/indexer/v1/leader
+
+#### Responses
+
+
+
+
+
+
+
+*Successfully retrieved leader Overlord address*
+
+
+
+---
+
+#### Sample request
+
+
+
+
+
+```shell
+curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/leader"
+```
+
+
+
+```http
+GET /druid/indexer/v1/leader HTTP/1.1
+Host: http://ROUTER_IP:ROUTER_PORT
+```
+
+
+
+#### Sample response
+
+
+ Click to show sample response
+
+ ```json
+ http://localhost:8081
+ ```
+
+
+
+
+### Get Overlord leader status
+
+Retrieves a JSON object with a `leader` property. The value can be `true` or `false`, indicating if this server is the current leader Overlord of the cluster. To get the individual address of the leader Overlord node, see the [leader endpoint](#get-overlord-leader-address).
+
+Use this endpoint as a load balancer status check when you only want the active leader to be considered in-service at the load balancer.
+
+#### URL
+
+GET
/druid/indexer/v1/isLeader
+
+#### Responses
+
+
+
+
+
+
+
+*Current server is the leader*
+
+
+
+
+
+*Current server is not the leader*
+
+
+
+---
+
+#### Sample request
+
+
+
+
+
+```shell
+curl "http://OVERLORD_IP:OVERLORD_PORT/druid/indexer/v1/isLeader"
+```
+
+
+
+```http
+GET /druid/indexer/v1/isLeader HTTP/1.1
+Host: http://OVERLORD_IP:OVERLORD_PORT
+```
+
+
+
+#### Sample response
+
+
+ Click to show sample response
+
+ ```json
+ {
+ "leader": true
+ }
+ ```
+
+
+
+
+## MiddleManager
+
+### Get MiddleManager state status
+
+Retrieves the enabled state of the MiddleManager. Returns JSON object keyed by the combined `druid.host` and `druid.port` with a boolean `true` or `false` state as the value.
+
+#### URL
+
+GET
/druid/worker/v1/enabled
+
+#### Responses
+
+
+
+
+
+
+
+*Successfully retrieved MiddleManager state*
+
+
+
+---
+
+#### Sample request
+
+
+
+
+
+```shell
+curl "http://MIDDLEMANAGER_IP:MIDDLEMANAGER_PORT/druid/worker/v1/enabled"
+```
+
+
+
+```http
+GET /druid/worker/v1/enabled HTTP/1.1
+Host: http://MIDDLEMANAGER_IP:MIDDLEMANAGER_PORT
+```
+
+
+
+#### Sample response
+
+
+ Click to show sample response
+
+ ```json
+ {
+ "localhost:8091": true
+ }
+ ```
+
+
+
+### Get active tasks
+
+Retrieves a list of active tasks being run on MiddleManager. Returns JSON list of task ID strings. Note that for normal usage, you should use the `/druid/indexer/v1/tasks` [Tasks API](./tasks-api.md) endpoint or one of the task state specific variants instead.
+
+#### URL
+
+GET
/druid/worker/v1/tasks
+
+#### Responses
+
+
+
+
+
+
+
+*Successfully retrieved active tasks*
+
+
+
+---
+
+#### Sample request
+
+
+
+
+
+```shell
+curl "http://MIDDLEMANAGER_IP:MIDDLEMANAGER_PORT/druid/worker/v1/tasks"
+```
+
+
+
+```http
+GET /druid/worker/v1/tasks HTTP/1.1
+Host: http://MIDDLEMANAGER_IP:MIDDLEMANAGER_PORT
+```
+
+
+
+#### Sample response
+
+
+ Click to show sample response
+
+ ```json
+ [
+ "index_parallel_wikipedia_mgchefio_2023-06-13T22:18:05.360Z"
+ ]
+ ```
+
+
+
+### Get task log
+
+Retrieves task log output stream by task ID. For normal usage, you should use the `/druid/indexer/v1/task/{taskId}/log`
+[Tasks API](./tasks-api.md) endpoint instead.
+
+#### URL
+
+GET
/druid/worker/v1/task/:taskId/log
+
+### Shut down running task
+
+Shuts down a running task by ID. For normal usage, you should use the `/druid/indexer/v1/task/:taskId/shutdown`
+[Tasks API](./tasks-api.md) endpoint instead.
+
+#### URL
+
+POST
/druid/worker/v1/task/:taskId/shutdown
+
+#### Responses
+
+
+
+
+
+
+*Successfully shut down a task*
+
+
+
+---
+
+#### Sample request
+
+The following example shuts down a task with specified ID `index_kafka_wikiticker_f7011f8ffba384b_fpeclode`.
+
+
+
+
+
+```shell
+curl "http://MIDDLEMANAGER_IP:MIDDLEMANAGER_PORT/druid/worker/v1/task/index_kafka_wikiticker_f7011f8ffba384b_fpeclode/shutdown"
+```
+
+
+
+```http
+POST /druid/worker/v1/task/index_kafka_wikiticker_f7011f8ffba384b_fpeclode/shutdown HTTP/1.1
+Host: http://MIDDLEMANAGER_IP:MIDDLEMANAGER_PORT
+```
+
+
+
+#### Sample response
+
+
+ Click to show sample response
+
+ ```json
+ {
+ "task":"index_kafka_wikiticker_f7011f8ffba384b_fpeclode"
+ }
+ ```
+
+
+
+### Disable MiddleManager
+
+Disables a MiddleManager, causing it to stop accepting new tasks but complete all existing tasks. Returns a JSON object
+keyed by the combined `druid.host` and `druid.port`.
+
+#### URL
+
+POST
/druid/worker/v1/disable
+
+#### Responses
+
+
+
+
+
+
+
+*Successfully disabled MiddleManager*
+
+
+
+#### Sample request
+
+
+
+
+
+```shell
+curl "http://MIDDLEMANAGER_IP:MIDDLEMANAGER_PORT/druid/worker/v1/disable"
+```
+
+
+
+```http
+POST /druid/worker/v1/disable HTTP/1.1
+Host: http://MIDDLEMANAGER_IP:MIDDLEMANAGER_PORT
+```
+
+
+
+#### Sample response
+
+
+ Click to show sample response
+
+ ```json
+ {
+ "localhost:8091":"disabled"
+ }
+ ```
+
+
+
+### Enable MiddleManager
+
+Enables a MiddleManager, allowing it to accept new tasks again if it was previously disabled. Returns a JSON object
+keyed by the combined `druid.host` and `druid.port`.
+
+#### URL
+
+POST
/druid/worker/v1/enable
+
+#### Responses
+
+
+
+
+
+
+
+*Successfully enabled MiddleManager*
+
+
+
+#### Sample request
+
+
+
+
+
+```shell
+curl "http://MIDDLEMANAGER_IP:MIDDLEMANAGER_PORT/druid/worker/v1/enable"
+```
+
+
+
+```http
+POST /druid/worker/v1/enable HTTP/1.1
+Host: http://MIDDLEMANAGER_IP:MIDDLEMANAGER_PORT
+```
+
+
+
+#### Sample response
+
+
+ Click to show sample response
+
+ ```json
+ {
+ "localhost:8091":"enabled"
+ }
+ ```
+
+
## Historical
-### Segment loading
-`GET /druid/historical/v1/loadstatus`
+### Get segment load status
-Returns JSON of the form `{"cacheInitialized":}`, 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 process is ready
-to be queried after a restart.
+Retrieves a JSON object of the form `{"cacheInitialized":value}`, where value is either `true` or `false` indicating if all segments in the local cache have been loaded.
-`GET /druid/historical/v1/readiness`
+Use this endpoint to know when a Broker service is ready to accept queries after a restart.
-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.
+#### URL
+GET
/druid/historical/v1/loadstatus
+
+#### Responses
+
+
+
+
+
+
+
+*Successfully retrieved status*
+
+
+
+#### Sample request
+
+
+
+
+
+```shell
+curl "http://HISTORICAL_IP:HISTORICAL_PORT/druid/historical/v1/loadstatus"
+```
+
+
+
+```http
+GET /druid/historical/v1/loadstatus HTTP/1.1
+Host: http://HISTORICAL_IP:HISTORICAL_PORT
+```
+
+
+
+#### Sample response
+
+
+ Click to show sample response
+
+ ```json
+ {
+ "cacheInitialized": true
+ }
+ ```
+
+
+
+### Get segment readiness
+
+Retrieves a status code to indicate if all segments in the local cache have been loaded. Similar to `/druid/historical/v1/loadstatus`, but instead of returning JSON with a flag, it returns status codes.
+
+#### URL
+
+GET
/druid/historical/v1/readiness
+
+#### Responses
+
+
+
+
+
+
+
+*Segments in local cache successfully loaded*
+
+
+
+
+
+*Segments in local cache have not been loaded*
+
+
+
+#### Sample request
+
+
+
+
+
+```shell
+curl "http://HISTORICAL_IP:HISTORICAL_PORT/druid/historical/v1/readiness"
+```
+
+
+
+```http
+GET /druid/historical/v1/readiness HTTP/1.1
+Host: http://HISTORICAL_IP:HISTORICAL_PORT
+```
+
+
+
+#### Sample response
+
+A successful response to this endpoint results in an empty response body.
## Load Status
-`GET /druid/broker/v1/loadstatus`
+### Get Broker query load status
-Returns a flag indicating if the Broker knows about all segments in the cluster. This can be used to know when a Broker process is ready to be queried after a restart.
+Retrieves a flag indicating if the Broker knows about all segments in the cluster. Use this endpoint to know when a Broker service is ready to accept queries after a restart.
+
+#### URL
+
+GET
/druid/broker/v1/loadstatus
+
+#### Responses
+
+
+
+
+
+
+
+*Segments successfully loaded*
+
+
+
+#### Sample request
+
+
+
+
+
+```shell
+curl "http://BROKER_IP:BROKER_PORT/druid/broker/v1/loadstatus"
+```
+
+
+
+```http
+GET /druid/broker/v1/loadstatus HTTP/1.1
+Host: http://:
+```
+
+
+
+#### Sample response
+
+
+ Click to show sample response
+
+ ```json
+ {
+ "inventoryInitialized": true
+ }
+ ```
+
+
+
+### Get Broker query readiness
+
+Retrieves a status code to indicate Broker readiness. Readiness signifies the Broker knows about all segments in the cluster and is ready to accept queries after a restart. Similar to `/druid/broker/v1/loadstatus`, but instead of returning a JSON, it returns status codes.
+
+#### URL
+
+GET
/druid/broker/v1/readiness
+
+#### Responses
+
+
+
+
+
+
+
+*Segments successfully loaded*
+
+
+
+
+
+*Segments have not been loaded*
+
+
+
+#### Sample request
+
+
+
+
+
+```shell
+curl "http://BROKER_IP:BROKER_PORT/druid/broker/v1/readiness"
+```
+
+
+
+```http
+GET /druid/broker/v1/readiness HTTP/1.1
+Host: http://BROKER_IP:BROKER_PORT
+```
+
+
+
+#### Sample response
+
+A successful response to this endpoint results in an empty response body.
-`GET /druid/broker/v1/readiness`
-Similar to `/druid/broker/v1/loadstatus`, but instead of returning a JSON, responses 200 OK if its ready and otherwise 503 SERVICE UNAVAILABLE.
diff --git a/website/.spelling b/website/.spelling
index 984a658bca1..d4d3c1f5c6f 100644
--- a/website/.spelling
+++ b/website/.spelling
@@ -2289,3 +2289,6 @@ prometheus-emitter
kubernetes-overlord-extensions
UCS
ISO646-US
+selfDiscovered
+isLeader
+loadstatus
\ No newline at end of file