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