import Tabs from ‘@theme/Tabs’; import TabItem from ‘@theme/TabItem’;
This topic describes the API endpoints to manage and monitor supervisors for Apache Druid.
In this topic, http://ROUTER_IP:ROUTER_PORT
is a placeholder for your Router service address and port. Replace it with the information for your deployment. For example, use http://localhost:8888
for quickstart deployments.
The following table lists the properties of a supervisor object:
Property | Type | Description |
---|---|---|
id | String | Unique identifier. |
state | String | Generic state of the supervisor. Available states:UNHEALTHY_SUPERVISOR , UNHEALTHY_TASKS , PENDING , RUNNING , SUSPENDED , STOPPING . See Apache Kafka operations for details. |
detailedState | String | Detailed state of the supervisor. This property contains a more descriptive, implementation-specific state that may provide more insight into the supervisor's activities than the state property. See Apache Kafka ingestion and Amazon Kinesis ingestion for supervisor-specific states. |
healthy | Boolean | Supervisor health indicator. |
spec | Object | Container object for the supervisor configuration. |
suspended | Boolean | Indicates whether the supervisor is in a suspended state. |
Returns an array of strings representing the names of active supervisors. If there are no active supervisors, it returns an empty array.
GET /druid/indexer/v1/supervisor
Successfully retrieved array of active supervisor IDs
curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor"
GET /druid/indexer/v1/supervisor HTTP/1.1 Host: http://ROUTER_IP:ROUTER_PORT
[ "wikipedia_stream", "social_media" ]
Retrieves an array of active supervisor objects. If there are no active supervisors, it returns an empty array. For reference on the supervisor object properties, see the preceding table.
GET /druid/indexer/v1/supervisor?full
Successfully retrieved supervisor objects
curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor?full=null"
GET /druid/indexer/v1/supervisor?full=null HTTP/1.1 Host: http://ROUTER_IP:ROUTER_PORT
[ { "id": "wikipedia_stream", "state": "RUNNING", "detailedState": "CONNECTING_TO_STREAM", "healthy": true, "spec": { "type": "kafka", "spec": { "dataSchema": { "dataSource": "wikipedia_stream", "timestampSpec": { "column": "__time", "format": "iso", "missingValue": null }, "dimensionsSpec": { "dimensions": [ { "type": "string", "name": "username", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true }, { "type": "string", "name": "post_title", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true }, { "type": "long", "name": "views", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "long", "name": "upvotes", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "long", "name": "comments", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "string", "name": "edited", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true } ], "dimensionExclusions": [ "__time" ], "includeAllDimensions": false, "useSchemaDiscovery": false }, "metricsSpec": [], "granularitySpec": { "type": "uniform", "segmentGranularity": "HOUR", "queryGranularity": { "type": "none" }, "rollup": false, "intervals": [] }, "transformSpec": { "filter": null, "transforms": [] } }, "ioConfig": { "topic": "social_media", "inputFormat": { "type": "json", "keepNullColumns": false, "assumeNewlineDelimited": false, "useJsonNodeReader": false }, "replicas": 1, "taskCount": 1, "taskDuration": "PT3600S", "consumerProperties": { "bootstrap.servers": "localhost:9042" }, "autoScalerConfig": null, "pollTimeout": 100, "startDelay": "PT5S", "period": "PT30S", "useEarliestOffset": true, "completionTimeout": "PT1800S", "lateMessageRejectionPeriod": null, "earlyMessageRejectionPeriod": null, "lateMessageRejectionStartDateTime": null, "configOverrides": null, "idleConfig": null, "stream": "social_media", "useEarliestSequenceNumber": true }, "tuningConfig": { "type": "kafka", "appendableIndexSpec": { "type": "onheap", "preserveExistingMetrics": false }, "maxRowsInMemory": 150000, "maxBytesInMemory": 0, "skipBytesInMemoryOverheadCheck": false, "maxRowsPerSegment": 5000000, "maxTotalRows": null, "intermediatePersistPeriod": "PT10M", "maxPendingPersists": 0, "indexSpec": { "bitmap": { "type": "roaring" }, "dimensionCompression": "lz4", "stringDictionaryEncoding": { "type": "utf8" }, "metricCompression": "lz4", "longEncoding": "longs" }, "indexSpecForIntermediatePersists": { "bitmap": { "type": "roaring" }, "dimensionCompression": "lz4", "stringDictionaryEncoding": { "type": "utf8" }, "metricCompression": "lz4", "longEncoding": "longs" }, "reportParseExceptions": false, "handoffConditionTimeout": 0, "resetOffsetAutomatically": false, "segmentWriteOutMediumFactory": null, "workerThreads": null, "chatRetries": 8, "httpTimeout": "PT10S", "shutdownTimeout": "PT80S", "offsetFetchPeriod": "PT30S", "intermediateHandoffPeriod": "P2147483647D", "logParseExceptions": false, "maxParseExceptions": 2147483647, "maxSavedParseExceptions": 0, "skipSequenceNumberAvailabilityCheck": false, "repartitionTransitionDuration": "PT120S" } }, "dataSchema": { "dataSource": "wikipedia_stream", "timestampSpec": { "column": "__time", "format": "iso", "missingValue": null }, "dimensionsSpec": { "dimensions": [ { "type": "string", "name": "username", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true }, { "type": "string", "name": "post_title", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true }, { "type": "long", "name": "views", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "long", "name": "upvotes", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "long", "name": "comments", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "string", "name": "edited", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true } ], "dimensionExclusions": [ "__time" ], "includeAllDimensions": false, "useSchemaDiscovery": false }, "metricsSpec": [], "granularitySpec": { "type": "uniform", "segmentGranularity": "HOUR", "queryGranularity": { "type": "none" }, "rollup": false, "intervals": [] }, "transformSpec": { "filter": null, "transforms": [] } }, "tuningConfig": { "type": "kafka", "appendableIndexSpec": { "type": "onheap", "preserveExistingMetrics": false }, "maxRowsInMemory": 150000, "maxBytesInMemory": 0, "skipBytesInMemoryOverheadCheck": false, "maxRowsPerSegment": 5000000, "maxTotalRows": null, "intermediatePersistPeriod": "PT10M", "maxPendingPersists": 0, "indexSpec": { "bitmap": { "type": "roaring" }, "dimensionCompression": "lz4", "stringDictionaryEncoding": { "type": "utf8" }, "metricCompression": "lz4", "longEncoding": "longs" }, "indexSpecForIntermediatePersists": { "bitmap": { "type": "roaring" }, "dimensionCompression": "lz4", "stringDictionaryEncoding": { "type": "utf8" }, "metricCompression": "lz4", "longEncoding": "longs" }, "reportParseExceptions": false, "handoffConditionTimeout": 0, "resetOffsetAutomatically": false, "segmentWriteOutMediumFactory": null, "workerThreads": null, "chatRetries": 8, "httpTimeout": "PT10S", "shutdownTimeout": "PT80S", "offsetFetchPeriod": "PT30S", "intermediateHandoffPeriod": "P2147483647D", "logParseExceptions": false, "maxParseExceptions": 2147483647, "maxSavedParseExceptions": 0, "skipSequenceNumberAvailabilityCheck": false, "repartitionTransitionDuration": "PT120S" }, "ioConfig": { "topic": "social_media", "inputFormat": { "type": "json", "keepNullColumns": false, "assumeNewlineDelimited": false, "useJsonNodeReader": false }, "replicas": 1, "taskCount": 1, "taskDuration": "PT3600S", "consumerProperties": { "bootstrap.servers": "localhost:9042" }, "autoScalerConfig": null, "pollTimeout": 100, "startDelay": "PT5S", "period": "PT30S", "useEarliestOffset": true, "completionTimeout": "PT1800S", "lateMessageRejectionPeriod": null, "earlyMessageRejectionPeriod": null, "lateMessageRejectionStartDateTime": null, "configOverrides": null, "idleConfig": null, "stream": "social_media", "useEarliestSequenceNumber": true }, "context": null, "suspended": false }, "suspended": false }, { "id": "social_media", "state": "RUNNING", "detailedState": "RUNNING", "healthy": true, "spec": { "type": "kafka", "spec": { "dataSchema": { "dataSource": "social_media", "timestampSpec": { "column": "__time", "format": "iso", "missingValue": null }, "dimensionsSpec": { "dimensions": [ { "type": "string", "name": "username", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true }, { "type": "string", "name": "post_title", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true }, { "type": "long", "name": "views", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "long", "name": "upvotes", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "long", "name": "comments", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "string", "name": "edited", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true } ], "dimensionExclusions": [ "__time" ], "includeAllDimensions": false, "useSchemaDiscovery": false }, "metricsSpec": [], "granularitySpec": { "type": "uniform", "segmentGranularity": "HOUR", "queryGranularity": { "type": "none" }, "rollup": false, "intervals": [] }, "transformSpec": { "filter": null, "transforms": [] } }, "ioConfig": { "topic": "social_media", "inputFormat": { "type": "json", "keepNullColumns": false, "assumeNewlineDelimited": false, "useJsonNodeReader": false }, "replicas": 1, "taskCount": 1, "taskDuration": "PT3600S", "consumerProperties": { "bootstrap.servers": "localhost:9094" }, "autoScalerConfig": null, "pollTimeout": 100, "startDelay": "PT5S", "period": "PT30S", "useEarliestOffset": true, "completionTimeout": "PT1800S", "lateMessageRejectionPeriod": null, "earlyMessageRejectionPeriod": null, "lateMessageRejectionStartDateTime": null, "configOverrides": null, "idleConfig": null, "stream": "social_media", "useEarliestSequenceNumber": true }, "tuningConfig": { "type": "kafka", "appendableIndexSpec": { "type": "onheap", "preserveExistingMetrics": false }, "maxRowsInMemory": 150000, "maxBytesInMemory": 0, "skipBytesInMemoryOverheadCheck": false, "maxRowsPerSegment": 5000000, "maxTotalRows": null, "intermediatePersistPeriod": "PT10M", "maxPendingPersists": 0, "indexSpec": { "bitmap": { "type": "roaring" }, "dimensionCompression": "lz4", "stringDictionaryEncoding": { "type": "utf8" }, "metricCompression": "lz4", "longEncoding": "longs" }, "indexSpecForIntermediatePersists": { "bitmap": { "type": "roaring" }, "dimensionCompression": "lz4", "stringDictionaryEncoding": { "type": "utf8" }, "metricCompression": "lz4", "longEncoding": "longs" }, "reportParseExceptions": false, "handoffConditionTimeout": 0, "resetOffsetAutomatically": false, "segmentWriteOutMediumFactory": null, "workerThreads": null, "chatRetries": 8, "httpTimeout": "PT10S", "shutdownTimeout": "PT80S", "offsetFetchPeriod": "PT30S", "intermediateHandoffPeriod": "P2147483647D", "logParseExceptions": false, "maxParseExceptions": 2147483647, "maxSavedParseExceptions": 0, "skipSequenceNumberAvailabilityCheck": false, "repartitionTransitionDuration": "PT120S" } }, "dataSchema": { "dataSource": "social_media", "timestampSpec": { "column": "__time", "format": "iso", "missingValue": null }, "dimensionsSpec": { "dimensions": [ { "type": "string", "name": "username", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true }, { "type": "string", "name": "post_title", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true }, { "type": "long", "name": "views", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "long", "name": "upvotes", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "long", "name": "comments", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "string", "name": "edited", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true } ], "dimensionExclusions": [ "__time" ], "includeAllDimensions": false, "useSchemaDiscovery": false }, "metricsSpec": [], "granularitySpec": { "type": "uniform", "segmentGranularity": "HOUR", "queryGranularity": { "type": "none" }, "rollup": false, "intervals": [] }, "transformSpec": { "filter": null, "transforms": [] } }, "tuningConfig": { "type": "kafka", "appendableIndexSpec": { "type": "onheap", "preserveExistingMetrics": false }, "maxRowsInMemory": 150000, "maxBytesInMemory": 0, "skipBytesInMemoryOverheadCheck": false, "maxRowsPerSegment": 5000000, "maxTotalRows": null, "intermediatePersistPeriod": "PT10M", "maxPendingPersists": 0, "indexSpec": { "bitmap": { "type": "roaring" }, "dimensionCompression": "lz4", "stringDictionaryEncoding": { "type": "utf8" }, "metricCompression": "lz4", "longEncoding": "longs" }, "indexSpecForIntermediatePersists": { "bitmap": { "type": "roaring" }, "dimensionCompression": "lz4", "stringDictionaryEncoding": { "type": "utf8" }, "metricCompression": "lz4", "longEncoding": "longs" }, "reportParseExceptions": false, "handoffConditionTimeout": 0, "resetOffsetAutomatically": false, "segmentWriteOutMediumFactory": null, "workerThreads": null, "chatRetries": 8, "httpTimeout": "PT10S", "shutdownTimeout": "PT80S", "offsetFetchPeriod": "PT30S", "intermediateHandoffPeriod": "P2147483647D", "logParseExceptions": false, "maxParseExceptions": 2147483647, "maxSavedParseExceptions": 0, "skipSequenceNumberAvailabilityCheck": false, "repartitionTransitionDuration": "PT120S" }, "ioConfig": { "topic": "social_media", "inputFormat": { "type": "json", "keepNullColumns": false, "assumeNewlineDelimited": false, "useJsonNodeReader": false }, "replicas": 1, "taskCount": 1, "taskDuration": "PT3600S", "consumerProperties": { "bootstrap.servers": "localhost:9094" }, "autoScalerConfig": null, "pollTimeout": 100, "startDelay": "PT5S", "period": "PT30S", "useEarliestOffset": true, "completionTimeout": "PT1800S", "lateMessageRejectionPeriod": null, "earlyMessageRejectionPeriod": null, "lateMessageRejectionStartDateTime": null, "configOverrides": null, "idleConfig": null, "stream": "social_media", "useEarliestSequenceNumber": true }, "context": null, "suspended": false }, "suspended": false } ]
Retrieves an array of objects representing active supervisors and their current state. If there are no active supervisors, it returns an empty array. For reference on the supervisor object properties, see the preceding table.
GET /druid/indexer/v1/supervisor?state=true
Successfully retrieved supervisor state objects
curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor?state=true"
GET /druid/indexer/v1/supervisor?state=true HTTP/1.1 Host: http://ROUTER_IP:ROUTER_PORT
[ { "id": "wikipedia_stream", "state": "UNHEALTHY_SUPERVISOR", "detailedState": "UNABLE_TO_CONNECT_TO_STREAM", "healthy": false, "suspended": false }, { "id": "social_media", "state": "RUNNING", "detailedState": "RUNNING", "healthy": true, "suspended": false } ]
Retrieves the specification for a single supervisor. The returned specification includes the dataSchema
, ioConfig
, and tuningConfig
objects.
GET /druid/indexer/v1/supervisor/:supervisorId
Successfully retrieved supervisor spec
Invalid supervisor ID
The following example shows how to retrieve the specification of a supervisor with the name wikipedia_stream
.
curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/wikipedia_stream"
GET /druid/indexer/v1/supervisor/wikipedia_stream HTTP/1.1 Host: http://ROUTER_IP:ROUTER_PORT
{ "type": "kafka", "spec": { "dataSchema": { "dataSource": "social_media", "timestampSpec": { "column": "__time", "format": "iso", "missingValue": null }, "dimensionsSpec": { "dimensions": [ { "type": "string", "name": "username", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true }, { "type": "string", "name": "post_title", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true }, { "type": "long", "name": "views", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "long", "name": "upvotes", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "long", "name": "comments", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "string", "name": "edited", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true } ], "dimensionExclusions": [ "__time" ], "includeAllDimensions": false, "useSchemaDiscovery": false }, "metricsSpec": [], "granularitySpec": { "type": "uniform", "segmentGranularity": "HOUR", "queryGranularity": { "type": "none" }, "rollup": false, "intervals": [] }, "transformSpec": { "filter": null, "transforms": [] } }, "ioConfig": { "topic": "social_media", "inputFormat": { "type": "json", "keepNullColumns": false, "assumeNewlineDelimited": false, "useJsonNodeReader": false }, "replicas": 1, "taskCount": 1, "taskDuration": "PT3600S", "consumerProperties": { "bootstrap.servers": "localhost:9094" }, "autoScalerConfig": null, "pollTimeout": 100, "startDelay": "PT5S", "period": "PT30S", "useEarliestOffset": true, "completionTimeout": "PT1800S", "lateMessageRejectionPeriod": null, "earlyMessageRejectionPeriod": null, "lateMessageRejectionStartDateTime": null, "configOverrides": null, "idleConfig": null, "stream": "social_media", "useEarliestSequenceNumber": true }, "tuningConfig": { "type": "kafka", "appendableIndexSpec": { "type": "onheap", "preserveExistingMetrics": false }, "maxRowsInMemory": 150000, "maxBytesInMemory": 0, "skipBytesInMemoryOverheadCheck": false, "maxRowsPerSegment": 5000000, "maxTotalRows": null, "intermediatePersistPeriod": "PT10M", "maxPendingPersists": 0, "indexSpec": { "bitmap": { "type": "roaring" }, "dimensionCompression": "lz4", "stringDictionaryEncoding": { "type": "utf8" }, "metricCompression": "lz4", "longEncoding": "longs" }, "indexSpecForIntermediatePersists": { "bitmap": { "type": "roaring" }, "dimensionCompression": "lz4", "stringDictionaryEncoding": { "type": "utf8" }, "metricCompression": "lz4", "longEncoding": "longs" }, "reportParseExceptions": false, "handoffConditionTimeout": 0, "resetOffsetAutomatically": false, "segmentWriteOutMediumFactory": null, "workerThreads": null, "chatRetries": 8, "httpTimeout": "PT10S", "shutdownTimeout": "PT80S", "offsetFetchPeriod": "PT30S", "intermediateHandoffPeriod": "P2147483647D", "logParseExceptions": false, "maxParseExceptions": 2147483647, "maxSavedParseExceptions": 0, "skipSequenceNumberAvailabilityCheck": false, "repartitionTransitionDuration": "PT120S" } }, "dataSchema": { "dataSource": "social_media", "timestampSpec": { "column": "__time", "format": "iso", "missingValue": null }, "dimensionsSpec": { "dimensions": [ { "type": "string", "name": "username", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true }, { "type": "string", "name": "post_title", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true }, { "type": "long", "name": "views", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "long", "name": "upvotes", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "long", "name": "comments", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "string", "name": "edited", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true } ], "dimensionExclusions": [ "__time" ], "includeAllDimensions": false, "useSchemaDiscovery": false }, "metricsSpec": [], "granularitySpec": { "type": "uniform", "segmentGranularity": "HOUR", "queryGranularity": { "type": "none" }, "rollup": false, "intervals": [] }, "transformSpec": { "filter": null, "transforms": [] } }, "tuningConfig": { "type": "kafka", "appendableIndexSpec": { "type": "onheap", "preserveExistingMetrics": false }, "maxRowsInMemory": 150000, "maxBytesInMemory": 0, "skipBytesInMemoryOverheadCheck": false, "maxRowsPerSegment": 5000000, "maxTotalRows": null, "intermediatePersistPeriod": "PT10M", "maxPendingPersists": 0, "indexSpec": { "bitmap": { "type": "roaring" }, "dimensionCompression": "lz4", "stringDictionaryEncoding": { "type": "utf8" }, "metricCompression": "lz4", "longEncoding": "longs" }, "indexSpecForIntermediatePersists": { "bitmap": { "type": "roaring" }, "dimensionCompression": "lz4", "stringDictionaryEncoding": { "type": "utf8" }, "metricCompression": "lz4", "longEncoding": "longs" }, "reportParseExceptions": false, "handoffConditionTimeout": 0, "resetOffsetAutomatically": false, "segmentWriteOutMediumFactory": null, "workerThreads": null, "chatRetries": 8, "httpTimeout": "PT10S", "shutdownTimeout": "PT80S", "offsetFetchPeriod": "PT30S", "intermediateHandoffPeriod": "P2147483647D", "logParseExceptions": false, "maxParseExceptions": 2147483647, "maxSavedParseExceptions": 0, "skipSequenceNumberAvailabilityCheck": false, "repartitionTransitionDuration": "PT120S" }, "ioConfig": { "topic": "social_media", "inputFormat": { "type": "json", "keepNullColumns": false, "assumeNewlineDelimited": false, "useJsonNodeReader": false }, "replicas": 1, "taskCount": 1, "taskDuration": "PT3600S", "consumerProperties": { "bootstrap.servers": "localhost:9094" }, "autoScalerConfig": null, "pollTimeout": 100, "startDelay": "PT5S", "period": "PT30S", "useEarliestOffset": true, "completionTimeout": "PT1800S", "lateMessageRejectionPeriod": null, "earlyMessageRejectionPeriod": null, "lateMessageRejectionStartDateTime": null, "configOverrides": null, "idleConfig": null, "stream": "social_media", "useEarliestSequenceNumber": true }, "context": null, "suspended": false }
Retrieves the current status report for a single supervisor. The report contains the state of the supervisor tasks and an array of recently thrown exceptions.
For additional information about the status report, see the topic for each streaming ingestion methods:
GET /druid/indexer/v1/supervisor/:supervisorId/status
Successfully retrieved supervisor status
Invalid supervisor ID
The following example shows how to retrieve the status of a supervisor with the name social_media
.
curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/social_media/status"
GET /druid/indexer/v1/supervisor/social_media/status HTTP/1.1 Host: http://ROUTER_IP:ROUTER_PORT
{ "id": "social_media", "generationTime": "2023-07-05T23:24:43.934Z", "payload": { "dataSource": "social_media", "stream": "social_media", "partitions": 1, "replicas": 1, "durationSeconds": 3600, "activeTasks": [ { "id": "index_kafka_social_media_ab72ae4127c591c_flcbhdlh", "startingOffsets": { "0": 3176381 }, "startTime": "2023-07-05T23:21:39.321Z", "remainingSeconds": 3415, "type": "ACTIVE", "currentOffsets": { "0": 3296632 }, "lag": { "0": 3 } } ], "publishingTasks": [], "latestOffsets": { "0": 3296635 }, "minimumLag": { "0": 3 }, "aggregateLag": 3, "offsetsLastUpdated": "2023-07-05T23:24:30.212Z", "suspended": false, "healthy": true, "state": "RUNNING", "detailedState": "RUNNING", "recentErrors": [] } }
An audit history provides a comprehensive log of events, including supervisor configuration, creation, suspension, and modification history.
Retrieve an audit history of specs for all supervisors.
GET /druid/indexer/v1/supervisor/history
Successfully retrieved audit history
curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/history"
GET /druid/indexer/v1/supervisor/history HTTP/1.1 Host: http://ROUTER_IP:ROUTER_PORT
{ "social_media": [ { "spec": { "type": "kafka", "spec": { "dataSchema": { "dataSource": "social_media", "timestampSpec": { "column": "__time", "format": "iso", "missingValue": null }, "dimensionsSpec": { "dimensions": [ { "type": "string", "name": "username", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true }, { "type": "string", "name": "post_title", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true }, { "type": "long", "name": "views", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "long", "name": "upvotes", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "long", "name": "comments", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "string", "name": "edited", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true } ], "dimensionExclusions": [ "__time" ], "includeAllDimensions": false, "useSchemaDiscovery": false }, "metricsSpec": [], "granularitySpec": { "type": "uniform", "segmentGranularity": "HOUR", "queryGranularity": { "type": "none" }, "rollup": false, "intervals": [] }, "transformSpec": { "filter": null, "transforms": [] } }, "ioConfig": { "topic": "social_media", "inputFormat": { "type": "json", "keepNullColumns": false, "assumeNewlineDelimited": false, "useJsonNodeReader": false }, "replicas": 1, "taskCount": 1, "taskDuration": "PT3600S", "consumerProperties": { "bootstrap.servers": "localhost:9094" }, "autoScalerConfig": null, "pollTimeout": 100, "startDelay": "PT5S", "period": "PT30S", "useEarliestOffset": true, "completionTimeout": "PT1800S", "lateMessageRejectionPeriod": null, "earlyMessageRejectionPeriod": null, "lateMessageRejectionStartDateTime": null, "configOverrides": null, "idleConfig": null, "stream": "social_media", "useEarliestSequenceNumber": true }, "tuningConfig": { "type": "kafka", "appendableIndexSpec": { "type": "onheap", "preserveExistingMetrics": false }, "maxRowsInMemory": 150000, "maxBytesInMemory": 0, "skipBytesInMemoryOverheadCheck": false, "maxRowsPerSegment": 5000000, "maxTotalRows": null, "intermediatePersistPeriod": "PT10M", "maxPendingPersists": 0, "indexSpec": { "bitmap": { "type": "roaring" }, "dimensionCompression": "lz4", "stringDictionaryEncoding": { "type": "utf8" }, "metricCompression": "lz4", "longEncoding": "longs" }, "indexSpecForIntermediatePersists": { "bitmap": { "type": "roaring" }, "dimensionCompression": "lz4", "stringDictionaryEncoding": { "type": "utf8" }, "metricCompression": "lz4", "longEncoding": "longs" }, "reportParseExceptions": false, "handoffConditionTimeout": 0, "resetOffsetAutomatically": false, "segmentWriteOutMediumFactory": null, "workerThreads": null, "chatRetries": 8, "httpTimeout": "PT10S", "shutdownTimeout": "PT80S", "offsetFetchPeriod": "PT30S", "intermediateHandoffPeriod": "P2147483647D", "logParseExceptions": false, "maxParseExceptions": 2147483647, "maxSavedParseExceptions": 0, "skipSequenceNumberAvailabilityCheck": false, "repartitionTransitionDuration": "PT120S" } }, "dataSchema": { "dataSource": "social_media", "timestampSpec": { "column": "__time", "format": "iso", "missingValue": null }, "dimensionsSpec": { "dimensions": [ { "type": "string", "name": "username", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true }, { "type": "string", "name": "post_title", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true }, { "type": "long", "name": "views", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "long", "name": "upvotes", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "long", "name": "comments", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "string", "name": "edited", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true } ], "dimensionExclusions": [ "__time" ], "includeAllDimensions": false, "useSchemaDiscovery": false }, "metricsSpec": [], "granularitySpec": { "type": "uniform", "segmentGranularity": "HOUR", "queryGranularity": { "type": "none" }, "rollup": false, "intervals": [] }, "transformSpec": { "filter": null, "transforms": [] } }, "tuningConfig": { "type": "kafka", "appendableIndexSpec": { "type": "onheap", "preserveExistingMetrics": false }, "maxRowsInMemory": 150000, "maxBytesInMemory": 0, "skipBytesInMemoryOverheadCheck": false, "maxRowsPerSegment": 5000000, "maxTotalRows": null, "intermediatePersistPeriod": "PT10M", "maxPendingPersists": 0, "indexSpec": { "bitmap": { "type": "roaring" }, "dimensionCompression": "lz4", "stringDictionaryEncoding": { "type": "utf8" }, "metricCompression": "lz4", "longEncoding": "longs" }, "indexSpecForIntermediatePersists": { "bitmap": { "type": "roaring" }, "dimensionCompression": "lz4", "stringDictionaryEncoding": { "type": "utf8" }, "metricCompression": "lz4", "longEncoding": "longs" }, "reportParseExceptions": false, "handoffConditionTimeout": 0, "resetOffsetAutomatically": false, "segmentWriteOutMediumFactory": null, "workerThreads": null, "chatRetries": 8, "httpTimeout": "PT10S", "shutdownTimeout": "PT80S", "offsetFetchPeriod": "PT30S", "intermediateHandoffPeriod": "P2147483647D", "logParseExceptions": false, "maxParseExceptions": 2147483647, "maxSavedParseExceptions": 0, "skipSequenceNumberAvailabilityCheck": false, "repartitionTransitionDuration": "PT120S" }, "ioConfig": { "topic": "social_media", "inputFormat": { "type": "json", "keepNullColumns": false, "assumeNewlineDelimited": false, "useJsonNodeReader": false }, "replicas": 1, "taskCount": 1, "taskDuration": "PT3600S", "consumerProperties": { "bootstrap.servers": "localhost:9094" }, "autoScalerConfig": null, "pollTimeout": 100, "startDelay": "PT5S", "period": "PT30S", "useEarliestOffset": true, "completionTimeout": "PT1800S", "lateMessageRejectionPeriod": null, "earlyMessageRejectionPeriod": null, "lateMessageRejectionStartDateTime": null, "configOverrides": null, "idleConfig": null, "stream": "social_media", "useEarliestSequenceNumber": true }, "context": null, "suspended": false }, "version": "2023-07-03T18:51:02.970Z" } ] }
Retrieves an audit history of specs for a single supervisor.
GET /druid/indexer/v1/supervisor/:supervisorId/history
Successfully retrieved supervisor audit history
Invalid supervisor ID
The following example shows how to retrieve the audit history of a supervisor with the name wikipedia_stream
.
curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/wikipedia_stream/history"
GET /druid/indexer/v1/supervisor/wikipedia_stream/history HTTP/1.1 Host: http://ROUTER_IP:ROUTER_PORT
[ { "spec": { "type": "kafka", "spec": { "dataSchema": { "dataSource": "wikipedia_stream", "timestampSpec": { "column": "__time", "format": "iso", "missingValue": null }, "dimensionsSpec": { "dimensions": [ { "type": "string", "name": "username", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true }, { "type": "string", "name": "post_title", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true }, { "type": "long", "name": "views", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "long", "name": "upvotes", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "long", "name": "comments", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "string", "name": "edited", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true } ], "dimensionExclusions": [ "__time" ], "includeAllDimensions": false, "useSchemaDiscovery": false }, "metricsSpec": [], "granularitySpec": { "type": "uniform", "segmentGranularity": "HOUR", "queryGranularity": { "type": "none" }, "rollup": false, "intervals": [] }, "transformSpec": { "filter": null, "transforms": [] } }, "ioConfig": { "topic": "social_media", "inputFormat": { "type": "json", "keepNullColumns": false, "assumeNewlineDelimited": false, "useJsonNodeReader": false }, "replicas": 1, "taskCount": 1, "taskDuration": "PT3600S", "consumerProperties": { "bootstrap.servers": "localhost:9042" }, "autoScalerConfig": null, "pollTimeout": 100, "startDelay": "PT5S", "period": "PT30S", "useEarliestOffset": true, "completionTimeout": "PT1800S", "lateMessageRejectionPeriod": null, "earlyMessageRejectionPeriod": null, "lateMessageRejectionStartDateTime": null, "configOverrides": null, "idleConfig": null, "stream": "social_media", "useEarliestSequenceNumber": true }, "tuningConfig": { "type": "kafka", "appendableIndexSpec": { "type": "onheap", "preserveExistingMetrics": false }, "maxRowsInMemory": 150000, "maxBytesInMemory": 0, "skipBytesInMemoryOverheadCheck": false, "maxRowsPerSegment": 5000000, "maxTotalRows": null, "intermediatePersistPeriod": "PT10M", "maxPendingPersists": 0, "indexSpec": { "bitmap": { "type": "roaring" }, "dimensionCompression": "lz4", "stringDictionaryEncoding": { "type": "utf8" }, "metricCompression": "lz4", "longEncoding": "longs" }, "indexSpecForIntermediatePersists": { "bitmap": { "type": "roaring" }, "dimensionCompression": "lz4", "stringDictionaryEncoding": { "type": "utf8" }, "metricCompression": "lz4", "longEncoding": "longs" }, "reportParseExceptions": false, "handoffConditionTimeout": 0, "resetOffsetAutomatically": false, "segmentWriteOutMediumFactory": null, "workerThreads": null, "chatRetries": 8, "httpTimeout": "PT10S", "shutdownTimeout": "PT80S", "offsetFetchPeriod": "PT30S", "intermediateHandoffPeriod": "P2147483647D", "logParseExceptions": false, "maxParseExceptions": 2147483647, "maxSavedParseExceptions": 0, "skipSequenceNumberAvailabilityCheck": false, "repartitionTransitionDuration": "PT120S" } }, "dataSchema": { "dataSource": "wikipedia_stream", "timestampSpec": { "column": "__time", "format": "iso", "missingValue": null }, "dimensionsSpec": { "dimensions": [ { "type": "string", "name": "username", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true }, { "type": "string", "name": "post_title", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true }, { "type": "long", "name": "views", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "long", "name": "upvotes", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "long", "name": "comments", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "string", "name": "edited", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true } ], "dimensionExclusions": [ "__time" ], "includeAllDimensions": false, "useSchemaDiscovery": false }, "metricsSpec": [], "granularitySpec": { "type": "uniform", "segmentGranularity": "HOUR", "queryGranularity": { "type": "none" }, "rollup": false, "intervals": [] }, "transformSpec": { "filter": null, "transforms": [] } }, "tuningConfig": { "type": "kafka", "appendableIndexSpec": { "type": "onheap", "preserveExistingMetrics": false }, "maxRowsInMemory": 150000, "maxBytesInMemory": 0, "skipBytesInMemoryOverheadCheck": false, "maxRowsPerSegment": 5000000, "maxTotalRows": null, "intermediatePersistPeriod": "PT10M", "maxPendingPersists": 0, "indexSpec": { "bitmap": { "type": "roaring" }, "dimensionCompression": "lz4", "stringDictionaryEncoding": { "type": "utf8" }, "metricCompression": "lz4", "longEncoding": "longs" }, "indexSpecForIntermediatePersists": { "bitmap": { "type": "roaring" }, "dimensionCompression": "lz4", "stringDictionaryEncoding": { "type": "utf8" }, "metricCompression": "lz4", "longEncoding": "longs" }, "reportParseExceptions": false, "handoffConditionTimeout": 0, "resetOffsetAutomatically": false, "segmentWriteOutMediumFactory": null, "workerThreads": null, "chatRetries": 8, "httpTimeout": "PT10S", "shutdownTimeout": "PT80S", "offsetFetchPeriod": "PT30S", "intermediateHandoffPeriod": "P2147483647D", "logParseExceptions": false, "maxParseExceptions": 2147483647, "maxSavedParseExceptions": 0, "skipSequenceNumberAvailabilityCheck": false, "repartitionTransitionDuration": "PT120S" }, "ioConfig": { "topic": "social_media", "inputFormat": { "type": "json", "keepNullColumns": false, "assumeNewlineDelimited": false, "useJsonNodeReader": false }, "replicas": 1, "taskCount": 1, "taskDuration": "PT3600S", "consumerProperties": { "bootstrap.servers": "localhost:9042" }, "autoScalerConfig": null, "pollTimeout": 100, "startDelay": "PT5S", "period": "PT30S", "useEarliestOffset": true, "completionTimeout": "PT1800S", "lateMessageRejectionPeriod": null, "earlyMessageRejectionPeriod": null, "lateMessageRejectionStartDateTime": null, "configOverrides": null, "idleConfig": null, "stream": "social_media", "useEarliestSequenceNumber": true }, "context": null, "suspended": false }, "version": "2023-07-05T20:59:16.872Z" } ]
Creates a new supervisor or updates an existing one for the same datasource with a new schema and configuration.
You can define a supervisor spec for Apache Kafka or Amazon Kinesis streaming ingestion methods. Once created, the supervisor persists in the metadata database.
When you call this endpoint on an existing supervisor for the same datasource, the running supervisor signals its tasks to stop reading and begin publishing, exiting itself. Druid then uses the provided configuration from the request body to create a new supervisor. Druid submits a new schema while retaining existing publishing tasks and starts new tasks at the previous task offsets.
POST /druid/indexer/v1/supervisor
Successfully created a new supervisor or updated an existing supervisor
Request body content type is not in JSON format
The following example uses JSON input format to create a supervisor spec for Kafka with a social_media
datasource and social_media
topic.
curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor" \ --header 'Content-Type: application/json' \ --data '{ "type": "kafka", "spec": { "ioConfig": { "type": "kafka", "consumerProperties": { "bootstrap.servers": "localhost:9094" }, "topic": "social_media", "inputFormat": { "type": "json" }, "useEarliestOffset": true }, "tuningConfig": { "type": "kafka" }, "dataSchema": { "dataSource": "social_media", "timestampSpec": { "column": "__time", "format": "iso" }, "dimensionsSpec": { "dimensions": [ "username", "post_title", { "type": "long", "name": "views" }, { "type": "long", "name": "upvotes" }, { "type": "long", "name": "comments" }, "edited" ] }, "granularitySpec": { "queryGranularity": "none", "rollup": false, "segmentGranularity": "hour" } } } }'
POST /druid/indexer/v1/supervisor HTTP/1.1 Host: http://ROUTER_IP:ROUTER_PORT Content-Type: application/json Content-Length: 1359 { "type": "kafka", "spec": { "ioConfig": { "type": "kafka", "consumerProperties": { "bootstrap.servers": "localhost:9094" }, "topic": "social_media", "inputFormat": { "type": "json" }, "useEarliestOffset": true }, "tuningConfig": { "type": "kafka" }, "dataSchema": { "dataSource": "social_media", "timestampSpec": { "column": "__time", "format": "iso" }, "dimensionsSpec": { "dimensions": [ "username", "post_title", { "type": "long", "name": "views" }, { "type": "long", "name": "upvotes" }, { "type": "long", "name": "comments" }, "edited" ] }, "granularitySpec": { "queryGranularity": "none", "rollup": false, "segmentGranularity": "hour" } } } }
{ "id": "social_media" }
Suspends a single running supervisor. Returns the updated supervisor spec, where the suspended
property is set to true
. The suspended supervisor continues to emit logs and metrics.
POST /druid/indexer/v1/supervisor/:supervisorId/suspend
Successfully shut down supervisor
Supervisor already suspended
Invalid supervisor ID
The following example shows how to suspend a running supervisor with the name social_media
.
curl --request POST "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/social_media/suspend"
POST /druid/indexer/v1/supervisor/social_media/suspend HTTP/1.1 Host: http://ROUTER_IP:ROUTER_PORT
{ "type": "kafka", "spec": { "dataSchema": { "dataSource": "social_media", "timestampSpec": { "column": "__time", "format": "iso", "missingValue": null }, "dimensionsSpec": { "dimensions": [ { "type": "string", "name": "username", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true }, { "type": "string", "name": "post_title", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true }, { "type": "long", "name": "views", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "long", "name": "upvotes", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "long", "name": "comments", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "string", "name": "edited", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true } ], "dimensionExclusions": [ "__time" ], "includeAllDimensions": false, "useSchemaDiscovery": false }, "metricsSpec": [], "granularitySpec": { "type": "uniform", "segmentGranularity": "HOUR", "queryGranularity": { "type": "none" }, "rollup": false, "intervals": [] }, "transformSpec": { "filter": null, "transforms": [] } }, "ioConfig": { "topic": "social_media", "inputFormat": { "type": "json", "keepNullColumns": false, "assumeNewlineDelimited": false, "useJsonNodeReader": false }, "replicas": 1, "taskCount": 1, "taskDuration": "PT3600S", "consumerProperties": { "bootstrap.servers": "localhost:9094" }, "autoScalerConfig": null, "pollTimeout": 100, "startDelay": "PT5S", "period": "PT30S", "useEarliestOffset": true, "completionTimeout": "PT1800S", "lateMessageRejectionPeriod": null, "earlyMessageRejectionPeriod": null, "lateMessageRejectionStartDateTime": null, "configOverrides": null, "idleConfig": null, "stream": "social_media", "useEarliestSequenceNumber": true }, "tuningConfig": { "type": "kafka", "appendableIndexSpec": { "type": "onheap", "preserveExistingMetrics": false }, "maxRowsInMemory": 150000, "maxBytesInMemory": 0, "skipBytesInMemoryOverheadCheck": false, "maxRowsPerSegment": 5000000, "maxTotalRows": null, "intermediatePersistPeriod": "PT10M", "maxPendingPersists": 0, "indexSpec": { "bitmap": { "type": "roaring" }, "dimensionCompression": "lz4", "stringDictionaryEncoding": { "type": "utf8" }, "metricCompression": "lz4", "longEncoding": "longs" }, "indexSpecForIntermediatePersists": { "bitmap": { "type": "roaring" }, "dimensionCompression": "lz4", "stringDictionaryEncoding": { "type": "utf8" }, "metricCompression": "lz4", "longEncoding": "longs" }, "reportParseExceptions": false, "handoffConditionTimeout": 0, "resetOffsetAutomatically": false, "segmentWriteOutMediumFactory": null, "workerThreads": null, "chatRetries": 8, "httpTimeout": "PT10S", "shutdownTimeout": "PT80S", "offsetFetchPeriod": "PT30S", "intermediateHandoffPeriod": "P2147483647D", "logParseExceptions": false, "maxParseExceptions": 2147483647, "maxSavedParseExceptions": 0, "skipSequenceNumberAvailabilityCheck": false, "repartitionTransitionDuration": "PT120S" } }, "dataSchema": { "dataSource": "social_media", "timestampSpec": { "column": "__time", "format": "iso", "missingValue": null }, "dimensionsSpec": { "dimensions": [ { "type": "string", "name": "username", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true }, { "type": "string", "name": "post_title", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true }, { "type": "long", "name": "views", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "long", "name": "upvotes", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "long", "name": "comments", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "string", "name": "edited", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true } ], "dimensionExclusions": [ "__time" ], "includeAllDimensions": false, "useSchemaDiscovery": false }, "metricsSpec": [], "granularitySpec": { "type": "uniform", "segmentGranularity": "HOUR", "queryGranularity": { "type": "none" }, "rollup": false, "intervals": [] }, "transformSpec": { "filter": null, "transforms": [] } }, "tuningConfig": { "type": "kafka", "appendableIndexSpec": { "type": "onheap", "preserveExistingMetrics": false }, "maxRowsInMemory": 150000, "maxBytesInMemory": 0, "skipBytesInMemoryOverheadCheck": false, "maxRowsPerSegment": 5000000, "maxTotalRows": null, "intermediatePersistPeriod": "PT10M", "maxPendingPersists": 0, "indexSpec": { "bitmap": { "type": "roaring" }, "dimensionCompression": "lz4", "stringDictionaryEncoding": { "type": "utf8" }, "metricCompression": "lz4", "longEncoding": "longs" }, "indexSpecForIntermediatePersists": { "bitmap": { "type": "roaring" }, "dimensionCompression": "lz4", "stringDictionaryEncoding": { "type": "utf8" }, "metricCompression": "lz4", "longEncoding": "longs" }, "reportParseExceptions": false, "handoffConditionTimeout": 0, "resetOffsetAutomatically": false, "segmentWriteOutMediumFactory": null, "workerThreads": null, "chatRetries": 8, "httpTimeout": "PT10S", "shutdownTimeout": "PT80S", "offsetFetchPeriod": "PT30S", "intermediateHandoffPeriod": "P2147483647D", "logParseExceptions": false, "maxParseExceptions": 2147483647, "maxSavedParseExceptions": 0, "skipSequenceNumberAvailabilityCheck": false, "repartitionTransitionDuration": "PT120S" }, "ioConfig": { "topic": "social_media", "inputFormat": { "type": "json", "keepNullColumns": false, "assumeNewlineDelimited": false, "useJsonNodeReader": false }, "replicas": 1, "taskCount": 1, "taskDuration": "PT3600S", "consumerProperties": { "bootstrap.servers": "localhost:9094" }, "autoScalerConfig": null, "pollTimeout": 100, "startDelay": "PT5S", "period": "PT30S", "useEarliestOffset": true, "completionTimeout": "PT1800S", "lateMessageRejectionPeriod": null, "earlyMessageRejectionPeriod": null, "lateMessageRejectionStartDateTime": null, "configOverrides": null, "idleConfig": null, "stream": "social_media", "useEarliestSequenceNumber": true }, "context": null, "suspended": true }
Suspends all supervisors. Note that this endpoint returns an HTTP 200 Success
code message even if there are no supervisors or running supervisors to suspend.
POST /druid/indexer/v1/supervisor/suspendAll
Successfully suspended all supervisors
curl --request POST "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/suspendAll"
POST /druid/indexer/v1/supervisor/suspendAll HTTP/1.1 Host: http://ROUTER_IP:ROUTER_PORT
{ "status": "success" }
Resumes indexing tasks for a supervisor. Returns an updated supervisor spec with the suspended
property set to false
.
POST /druid/indexer/v1/supervisor/:supervisorId/resume
Successfully resumed supervisor
Supervisor already running
Invalid supervisor ID
The following example resumes a previously suspended supervisor with name social_media
.
curl --request POST "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/social_media/resume"
POST /druid/indexer/v1/supervisor/social_media/resume HTTP/1.1 Host: http://ROUTER_IP:ROUTER_PORT
{ "type": "kafka", "spec": { "dataSchema": { "dataSource": "social_media", "timestampSpec": { "column": "__time", "format": "iso", "missingValue": null }, "dimensionsSpec": { "dimensions": [ { "type": "string", "name": "username", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true }, { "type": "string", "name": "post_title", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true }, { "type": "long", "name": "views", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "long", "name": "upvotes", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "long", "name": "comments", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "string", "name": "edited", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true } ], "dimensionExclusions": [ "__time" ], "includeAllDimensions": false, "useSchemaDiscovery": false }, "metricsSpec": [], "granularitySpec": { "type": "uniform", "segmentGranularity": "HOUR", "queryGranularity": { "type": "none" }, "rollup": false, "intervals": [] }, "transformSpec": { "filter": null, "transforms": [] } }, "ioConfig": { "topic": "social_media", "inputFormat": { "type": "json", "keepNullColumns": false, "assumeNewlineDelimited": false, "useJsonNodeReader": false }, "replicas": 1, "taskCount": 1, "taskDuration": "PT3600S", "consumerProperties": { "bootstrap.servers": "localhost:9094" }, "autoScalerConfig": null, "pollTimeout": 100, "startDelay": "PT5S", "period": "PT30S", "useEarliestOffset": true, "completionTimeout": "PT1800S", "lateMessageRejectionPeriod": null, "earlyMessageRejectionPeriod": null, "lateMessageRejectionStartDateTime": null, "configOverrides": null, "idleConfig": null, "stream": "social_media", "useEarliestSequenceNumber": true }, "tuningConfig": { "type": "kafka", "appendableIndexSpec": { "type": "onheap", "preserveExistingMetrics": false }, "maxRowsInMemory": 150000, "maxBytesInMemory": 0, "skipBytesInMemoryOverheadCheck": false, "maxRowsPerSegment": 5000000, "maxTotalRows": null, "intermediatePersistPeriod": "PT10M", "maxPendingPersists": 0, "indexSpec": { "bitmap": { "type": "roaring" }, "dimensionCompression": "lz4", "stringDictionaryEncoding": { "type": "utf8" }, "metricCompression": "lz4", "longEncoding": "longs" }, "indexSpecForIntermediatePersists": { "bitmap": { "type": "roaring" }, "dimensionCompression": "lz4", "stringDictionaryEncoding": { "type": "utf8" }, "metricCompression": "lz4", "longEncoding": "longs" }, "reportParseExceptions": false, "handoffConditionTimeout": 0, "resetOffsetAutomatically": false, "segmentWriteOutMediumFactory": null, "workerThreads": null, "chatRetries": 8, "httpTimeout": "PT10S", "shutdownTimeout": "PT80S", "offsetFetchPeriod": "PT30S", "intermediateHandoffPeriod": "P2147483647D", "logParseExceptions": false, "maxParseExceptions": 2147483647, "maxSavedParseExceptions": 0, "skipSequenceNumberAvailabilityCheck": false, "repartitionTransitionDuration": "PT120S" } }, "dataSchema": { "dataSource": "social_media", "timestampSpec": { "column": "__time", "format": "iso", "missingValue": null }, "dimensionsSpec": { "dimensions": [ { "type": "string", "name": "username", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true }, { "type": "string", "name": "post_title", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true }, { "type": "long", "name": "views", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "long", "name": "upvotes", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "long", "name": "comments", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": false }, { "type": "string", "name": "edited", "multiValueHandling": "SORTED_ARRAY", "createBitmapIndex": true } ], "dimensionExclusions": [ "__time" ], "includeAllDimensions": false, "useSchemaDiscovery": false }, "metricsSpec": [], "granularitySpec": { "type": "uniform", "segmentGranularity": "HOUR", "queryGranularity": { "type": "none" }, "rollup": false, "intervals": [] }, "transformSpec": { "filter": null, "transforms": [] } }, "tuningConfig": { "type": "kafka", "appendableIndexSpec": { "type": "onheap", "preserveExistingMetrics": false }, "maxRowsInMemory": 150000, "maxBytesInMemory": 0, "skipBytesInMemoryOverheadCheck": false, "maxRowsPerSegment": 5000000, "maxTotalRows": null, "intermediatePersistPeriod": "PT10M", "maxPendingPersists": 0, "indexSpec": { "bitmap": { "type": "roaring" }, "dimensionCompression": "lz4", "stringDictionaryEncoding": { "type": "utf8" }, "metricCompression": "lz4", "longEncoding": "longs" }, "indexSpecForIntermediatePersists": { "bitmap": { "type": "roaring" }, "dimensionCompression": "lz4", "stringDictionaryEncoding": { "type": "utf8" }, "metricCompression": "lz4", "longEncoding": "longs" }, "reportParseExceptions": false, "handoffConditionTimeout": 0, "resetOffsetAutomatically": false, "segmentWriteOutMediumFactory": null, "workerThreads": null, "chatRetries": 8, "httpTimeout": "PT10S", "shutdownTimeout": "PT80S", "offsetFetchPeriod": "PT30S", "intermediateHandoffPeriod": "P2147483647D", "logParseExceptions": false, "maxParseExceptions": 2147483647, "maxSavedParseExceptions": 0, "skipSequenceNumberAvailabilityCheck": false, "repartitionTransitionDuration": "PT120S" }, "ioConfig": { "topic": "social_media", "inputFormat": { "type": "json", "keepNullColumns": false, "assumeNewlineDelimited": false, "useJsonNodeReader": false }, "replicas": 1, "taskCount": 1, "taskDuration": "PT3600S", "consumerProperties": { "bootstrap.servers": "localhost:9094" }, "autoScalerConfig": null, "pollTimeout": 100, "startDelay": "PT5S", "period": "PT30S", "useEarliestOffset": true, "completionTimeout": "PT1800S", "lateMessageRejectionPeriod": null, "earlyMessageRejectionPeriod": null, "lateMessageRejectionStartDateTime": null, "configOverrides": null, "idleConfig": null, "stream": "social_media", "useEarliestSequenceNumber": true }, "context": null, "suspended": false }
Resumes all supervisors. Note that this endpoint returns an HTTP 200 Success
code even if there are no supervisors or suspended supervisors to resume.
POST /druid/indexer/v1/supervisor/resumeAll
Successfully resumed all supervisors
curl --request POST "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/resumeAll"
POST /druid/indexer/v1/supervisor/resumeAll HTTP/1.1 Host: http://ROUTER_IP:ROUTER_PORT
{ "status": "success" }
Resets the specified supervisor. This endpoint clears all stored offsets in Kafka or sequence numbers in Kinesis, prompting the supervisor to resume data reading. The supervisor will start from the earliest or latest available position, depending on the platform (offsets in Kafka or sequence numbers in Kinesis). It kills and recreates active tasks to read from valid positions.
Use this endpoint to recover from a stopped state due to missing offsets in Kafka or sequence numbers in Kinesis. Use this endpoint with caution as it may result in skipped messages and lead to data loss or duplicate data.
POST /druid/indexer/v1/supervisor/:supervisorId/reset
Successfully reset supervisor
Invalid supervisor ID
The following example shows how to reset a supervisor with the name social_media
.
curl --request POST "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/social_media/reset"
POST /druid/indexer/v1/supervisor/social_media/reset HTTP/1.1 Host: http://ROUTER_IP:ROUTER_PORT
{ "id": "social_media" }
Resets the specified offsets for a supervisor. This endpoint clears only the specified offsets in Kafka or sequence numbers in Kinesis, prompting the supervisor to resume data reading. If there are no stored offsets, the specified offsets will be set in the metadata store. The supervisor will start from the reset offsets for the partitions specified and for the other partitions from the stored offset. It kills and recreates active tasks pertaining to the partitions specified to read from valid offsets.
Use this endpoint to selectively reset offsets for partitions without resetting the entire set.
POST /druid/indexer/v1/supervisor/:supervisorId/resetOffsets
Successfully reset offsets
Invalid supervisor ID
This section presents the structure and details of the reset offsets metadata payload.
Field | Type | Description | Required |
---|---|---|---|
type | String | The type of reset offsets metadata payload. It must match the supervisor's type . Possible values: kafka or kinesis . | Yes |
partitions | Object | An object representing the reset metadata. See below for details. | Yes |
The following table defines the fields within the partitions
object in the reset offsets metadata payload.
Field | Type | Description | Required |
---|---|---|---|
type | String | Must be set as end . Indicates the end sequence numbers for the reset offsets. | Yes |
stream | String | The stream to be reset. It must be a valid stream consumed by the supervisor. | Yes |
partitionOffsetMap | Object | A map of partitions to corresponding offsets for the stream to be reset. | Yes |
The following example shows how to reset offsets for a kafka supervisor with the name social_media
. Let's say the supervisor is reading from a kafka topic ads_media_stream
and has the stored offsets: {"0": 0, "1": 10, "2": 20, "3": 40}
.
curl --request POST "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/social_media/resetOffsets" --header 'Content-Type: application/json' --data-raw '{"type":"kafka","partitions":{"type":"end","stream":"ads_media_stream","partitionOffsetMap":{"0":100, "2": 650}}}'
POST /druid/indexer/v1/supervisor/social_media/resetOffsets HTTP/1.1 Host: http://ROUTER_IP:ROUTER_PORT Content-Type: application/json { "type": "kafka", "partitions": { "type": "end", "stream": "ads_media_stream", "partitionOffsetMap": { "0": 100, "2": 650 } } }
The above operation will reset offsets only for partitions 0 and 2 to 100 and 650 respectively. After a successful reset, when the supervisor's tasks restart, they will resume reading from {"0": 100, "1": 10, "2": 650, "3": 40}
.
{ "id": "social_media" }
Terminates a supervisor and its associated indexing tasks, triggering the publishing of their segments. When terminated, a tombstone marker is placed in the database to prevent reloading on restart.
The terminated supervisor still exists in the metadata store and its history can be retrieved.
POST /druid/indexer/v1/supervisor/:supervisorId/terminate
Successfully terminated a supervisor
Invalid supervisor ID or supervisor not running
curl --request POST "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/social_media/terminate"
POST /druid/indexer/v1/supervisor/social_media/terminate HTTP/1.1 Host: http://ROUTER_IP:ROUTER_PORT
{ "id": "social_media" }
Terminates all supervisors. Terminated supervisors still exist in the metadata store and their history can be retrieved. Note that this endpoint returns an HTTP 200 Success
code even if there are no supervisors or running supervisors to terminate.
POST /druid/indexer/v1/supervisor/terminateAll
Successfully terminated all supervisors
curl --request POST "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/terminateAll"
POST /druid/indexer/v1/supervisor/terminateAll HTTP/1.1 Host: http://ROUTER_IP:ROUTER_PORT
{ "status": "success" }
Shuts down a supervisor. This endpoint is deprecated and will be removed in future releases. Use the equivalent terminate endpoint instead.
POST /druid/indexer/v1/supervisor/:supervisorId/shutdown