Supervisor API

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.

Supervisor information

The following table lists the properties of a supervisor object:

PropertyTypeDescription
idStringUnique identifier.
stateStringGeneric state of the supervisor. Available states:UNHEALTHY_SUPERVISOR, UNHEALTHY_TASKS, PENDING, RUNNING, SUSPENDED, STOPPING. See Apache Kafka operations for details.
detailedStateStringDetailed 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.
healthyBooleanSupervisor health indicator.
specObjectContainer object for the supervisor configuration.
suspendedBooleanIndicates whether the supervisor is in a suspended state.

Get an array of active supervisor IDs

Returns an array of strings representing the names of active supervisors. If there are no active supervisors, it returns an empty array.

URL

GET /druid/indexer/v1/supervisor

Responses

  • 200 SUCCESS

Successfully retrieved array of active supervisor IDs


Sample request

  • cURL
  • HTTP
  1. curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor"
  1. GET /druid/indexer/v1/supervisor HTTP/1.1
  2. Host: http://ROUTER_IP:ROUTER_PORT

Sample response

Click to show sample response

  1. [
  2. "wikipedia_stream",
  3. "social_media"
  4. ]

Get an array of active supervisor objects

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.

URL

GET /druid/indexer/v1/supervisor?full

Responses

  • 200 SUCCESS

Successfully retrieved supervisor objects


Sample request

  • cURL
  • HTTP
  1. curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor?full=null"
  1. GET /druid/indexer/v1/supervisor?full=null HTTP/1.1
  2. Host: http://ROUTER_IP:ROUTER_PORT

Sample response

Click to show sample response

  1. [
  2. {
  3. "id": "wikipedia_stream",
  4. "state": "RUNNING",
  5. "detailedState": "CONNECTING_TO_STREAM",
  6. "healthy": true,
  7. "spec": {
  8. "type": "kafka",
  9. "spec": {
  10. "dataSchema": {
  11. "dataSource": "wikipedia_stream",
  12. "timestampSpec": {
  13. "column": "__time",
  14. "format": "iso",
  15. "missingValue": null
  16. },
  17. "dimensionsSpec": {
  18. "dimensions": [
  19. {
  20. "type": "string",
  21. "name": "username",
  22. "multiValueHandling": "SORTED_ARRAY",
  23. "createBitmapIndex": true
  24. },
  25. {
  26. "type": "string",
  27. "name": "post_title",
  28. "multiValueHandling": "SORTED_ARRAY",
  29. "createBitmapIndex": true
  30. },
  31. {
  32. "type": "long",
  33. "name": "views",
  34. "multiValueHandling": "SORTED_ARRAY",
  35. "createBitmapIndex": false
  36. },
  37. {
  38. "type": "long",
  39. "name": "upvotes",
  40. "multiValueHandling": "SORTED_ARRAY",
  41. "createBitmapIndex": false
  42. },
  43. {
  44. "type": "long",
  45. "name": "comments",
  46. "multiValueHandling": "SORTED_ARRAY",
  47. "createBitmapIndex": false
  48. },
  49. {
  50. "type": "string",
  51. "name": "edited",
  52. "multiValueHandling": "SORTED_ARRAY",
  53. "createBitmapIndex": true
  54. }
  55. ],
  56. "dimensionExclusions": [
  57. "__time"
  58. ],
  59. "includeAllDimensions": false,
  60. "useSchemaDiscovery": false
  61. },
  62. "metricsSpec": [],
  63. "granularitySpec": {
  64. "type": "uniform",
  65. "segmentGranularity": "HOUR",
  66. "queryGranularity": {
  67. "type": "none"
  68. },
  69. "rollup": false,
  70. "intervals": []
  71. },
  72. "transformSpec": {
  73. "filter": null,
  74. "transforms": []
  75. }
  76. },
  77. "ioConfig": {
  78. "topic": "social_media",
  79. "inputFormat": {
  80. "type": "json",
  81. "keepNullColumns": false,
  82. "assumeNewlineDelimited": false,
  83. "useJsonNodeReader": false
  84. },
  85. "replicas": 1,
  86. "taskCount": 1,
  87. "taskDuration": "PT3600S",
  88. "consumerProperties": {
  89. "bootstrap.servers": "localhost:9042"
  90. },
  91. "autoScalerConfig": null,
  92. "pollTimeout": 100,
  93. "startDelay": "PT5S",
  94. "period": "PT30S",
  95. "useEarliestOffset": true,
  96. "completionTimeout": "PT1800S",
  97. "lateMessageRejectionPeriod": null,
  98. "earlyMessageRejectionPeriod": null,
  99. "lateMessageRejectionStartDateTime": null,
  100. "configOverrides": null,
  101. "idleConfig": null,
  102. "stream": "social_media",
  103. "useEarliestSequenceNumber": true
  104. },
  105. "tuningConfig": {
  106. "type": "kafka",
  107. "appendableIndexSpec": {
  108. "type": "onheap",
  109. "preserveExistingMetrics": false
  110. },
  111. "maxRowsInMemory": 150000,
  112. "maxBytesInMemory": 0,
  113. "skipBytesInMemoryOverheadCheck": false,
  114. "maxRowsPerSegment": 5000000,
  115. "maxTotalRows": null,
  116. "intermediatePersistPeriod": "PT10M",
  117. "maxPendingPersists": 0,
  118. "indexSpec": {
  119. "bitmap": {
  120. "type": "roaring"
  121. },
  122. "dimensionCompression": "lz4",
  123. "stringDictionaryEncoding": {
  124. "type": "utf8"
  125. },
  126. "metricCompression": "lz4",
  127. "longEncoding": "longs"
  128. },
  129. "indexSpecForIntermediatePersists": {
  130. "bitmap": {
  131. "type": "roaring"
  132. },
  133. "dimensionCompression": "lz4",
  134. "stringDictionaryEncoding": {
  135. "type": "utf8"
  136. },
  137. "metricCompression": "lz4",
  138. "longEncoding": "longs"
  139. },
  140. "reportParseExceptions": false,
  141. "handoffConditionTimeout": 0,
  142. "resetOffsetAutomatically": false,
  143. "segmentWriteOutMediumFactory": null,
  144. "workerThreads": null,
  145. "chatThreads": null,
  146. "chatRetries": 8,
  147. "httpTimeout": "PT10S",
  148. "shutdownTimeout": "PT80S",
  149. "offsetFetchPeriod": "PT30S",
  150. "intermediateHandoffPeriod": "P2147483647D",
  151. "logParseExceptions": false,
  152. "maxParseExceptions": 2147483647,
  153. "maxSavedParseExceptions": 0,
  154. "skipSequenceNumberAvailabilityCheck": false,
  155. "repartitionTransitionDuration": "PT120S"
  156. }
  157. },
  158. "dataSchema": {
  159. "dataSource": "wikipedia_stream",
  160. "timestampSpec": {
  161. "column": "__time",
  162. "format": "iso",
  163. "missingValue": null
  164. },
  165. "dimensionsSpec": {
  166. "dimensions": [
  167. {
  168. "type": "string",
  169. "name": "username",
  170. "multiValueHandling": "SORTED_ARRAY",
  171. "createBitmapIndex": true
  172. },
  173. {
  174. "type": "string",
  175. "name": "post_title",
  176. "multiValueHandling": "SORTED_ARRAY",
  177. "createBitmapIndex": true
  178. },
  179. {
  180. "type": "long",
  181. "name": "views",
  182. "multiValueHandling": "SORTED_ARRAY",
  183. "createBitmapIndex": false
  184. },
  185. {
  186. "type": "long",
  187. "name": "upvotes",
  188. "multiValueHandling": "SORTED_ARRAY",
  189. "createBitmapIndex": false
  190. },
  191. {
  192. "type": "long",
  193. "name": "comments",
  194. "multiValueHandling": "SORTED_ARRAY",
  195. "createBitmapIndex": false
  196. },
  197. {
  198. "type": "string",
  199. "name": "edited",
  200. "multiValueHandling": "SORTED_ARRAY",
  201. "createBitmapIndex": true
  202. }
  203. ],
  204. "dimensionExclusions": [
  205. "__time"
  206. ],
  207. "includeAllDimensions": false,
  208. "useSchemaDiscovery": false
  209. },
  210. "metricsSpec": [],
  211. "granularitySpec": {
  212. "type": "uniform",
  213. "segmentGranularity": "HOUR",
  214. "queryGranularity": {
  215. "type": "none"
  216. },
  217. "rollup": false,
  218. "intervals": []
  219. },
  220. "transformSpec": {
  221. "filter": null,
  222. "transforms": []
  223. }
  224. },
  225. "tuningConfig": {
  226. "type": "kafka",
  227. "appendableIndexSpec": {
  228. "type": "onheap",
  229. "preserveExistingMetrics": false
  230. },
  231. "maxRowsInMemory": 150000,
  232. "maxBytesInMemory": 0,
  233. "skipBytesInMemoryOverheadCheck": false,
  234. "maxRowsPerSegment": 5000000,
  235. "maxTotalRows": null,
  236. "intermediatePersistPeriod": "PT10M",
  237. "maxPendingPersists": 0,
  238. "indexSpec": {
  239. "bitmap": {
  240. "type": "roaring"
  241. },
  242. "dimensionCompression": "lz4",
  243. "stringDictionaryEncoding": {
  244. "type": "utf8"
  245. },
  246. "metricCompression": "lz4",
  247. "longEncoding": "longs"
  248. },
  249. "indexSpecForIntermediatePersists": {
  250. "bitmap": {
  251. "type": "roaring"
  252. },
  253. "dimensionCompression": "lz4",
  254. "stringDictionaryEncoding": {
  255. "type": "utf8"
  256. },
  257. "metricCompression": "lz4",
  258. "longEncoding": "longs"
  259. },
  260. "reportParseExceptions": false,
  261. "handoffConditionTimeout": 0,
  262. "resetOffsetAutomatically": false,
  263. "segmentWriteOutMediumFactory": null,
  264. "workerThreads": null,
  265. "chatThreads": null,
  266. "chatRetries": 8,
  267. "httpTimeout": "PT10S",
  268. "shutdownTimeout": "PT80S",
  269. "offsetFetchPeriod": "PT30S",
  270. "intermediateHandoffPeriod": "P2147483647D",
  271. "logParseExceptions": false,
  272. "maxParseExceptions": 2147483647,
  273. "maxSavedParseExceptions": 0,
  274. "skipSequenceNumberAvailabilityCheck": false,
  275. "repartitionTransitionDuration": "PT120S"
  276. },
  277. "ioConfig": {
  278. "topic": "social_media",
  279. "inputFormat": {
  280. "type": "json",
  281. "keepNullColumns": false,
  282. "assumeNewlineDelimited": false,
  283. "useJsonNodeReader": false
  284. },
  285. "replicas": 1,
  286. "taskCount": 1,
  287. "taskDuration": "PT3600S",
  288. "consumerProperties": {
  289. "bootstrap.servers": "localhost:9042"
  290. },
  291. "autoScalerConfig": null,
  292. "pollTimeout": 100,
  293. "startDelay": "PT5S",
  294. "period": "PT30S",
  295. "useEarliestOffset": true,
  296. "completionTimeout": "PT1800S",
  297. "lateMessageRejectionPeriod": null,
  298. "earlyMessageRejectionPeriod": null,
  299. "lateMessageRejectionStartDateTime": null,
  300. "configOverrides": null,
  301. "idleConfig": null,
  302. "stream": "social_media",
  303. "useEarliestSequenceNumber": true
  304. },
  305. "context": null,
  306. "suspended": false
  307. },
  308. "suspended": false
  309. },
  310. {
  311. "id": "social_media",
  312. "state": "RUNNING",
  313. "detailedState": "RUNNING",
  314. "healthy": true,
  315. "spec": {
  316. "type": "kafka",
  317. "spec": {
  318. "dataSchema": {
  319. "dataSource": "social_media",
  320. "timestampSpec": {
  321. "column": "__time",
  322. "format": "iso",
  323. "missingValue": null
  324. },
  325. "dimensionsSpec": {
  326. "dimensions": [
  327. {
  328. "type": "string",
  329. "name": "username",
  330. "multiValueHandling": "SORTED_ARRAY",
  331. "createBitmapIndex": true
  332. },
  333. {
  334. "type": "string",
  335. "name": "post_title",
  336. "multiValueHandling": "SORTED_ARRAY",
  337. "createBitmapIndex": true
  338. },
  339. {
  340. "type": "long",
  341. "name": "views",
  342. "multiValueHandling": "SORTED_ARRAY",
  343. "createBitmapIndex": false
  344. },
  345. {
  346. "type": "long",
  347. "name": "upvotes",
  348. "multiValueHandling": "SORTED_ARRAY",
  349. "createBitmapIndex": false
  350. },
  351. {
  352. "type": "long",
  353. "name": "comments",
  354. "multiValueHandling": "SORTED_ARRAY",
  355. "createBitmapIndex": false
  356. },
  357. {
  358. "type": "string",
  359. "name": "edited",
  360. "multiValueHandling": "SORTED_ARRAY",
  361. "createBitmapIndex": true
  362. }
  363. ],
  364. "dimensionExclusions": [
  365. "__time"
  366. ],
  367. "includeAllDimensions": false,
  368. "useSchemaDiscovery": false
  369. },
  370. "metricsSpec": [],
  371. "granularitySpec": {
  372. "type": "uniform",
  373. "segmentGranularity": "HOUR",
  374. "queryGranularity": {
  375. "type": "none"
  376. },
  377. "rollup": false,
  378. "intervals": []
  379. },
  380. "transformSpec": {
  381. "filter": null,
  382. "transforms": []
  383. }
  384. },
  385. "ioConfig": {
  386. "topic": "social_media",
  387. "inputFormat": {
  388. "type": "json",
  389. "keepNullColumns": false,
  390. "assumeNewlineDelimited": false,
  391. "useJsonNodeReader": false
  392. },
  393. "replicas": 1,
  394. "taskCount": 1,
  395. "taskDuration": "PT3600S",
  396. "consumerProperties": {
  397. "bootstrap.servers": "localhost:9094"
  398. },
  399. "autoScalerConfig": null,
  400. "pollTimeout": 100,
  401. "startDelay": "PT5S",
  402. "period": "PT30S",
  403. "useEarliestOffset": true,
  404. "completionTimeout": "PT1800S",
  405. "lateMessageRejectionPeriod": null,
  406. "earlyMessageRejectionPeriod": null,
  407. "lateMessageRejectionStartDateTime": null,
  408. "configOverrides": null,
  409. "idleConfig": null,
  410. "stream": "social_media",
  411. "useEarliestSequenceNumber": true
  412. },
  413. "tuningConfig": {
  414. "type": "kafka",
  415. "appendableIndexSpec": {
  416. "type": "onheap",
  417. "preserveExistingMetrics": false
  418. },
  419. "maxRowsInMemory": 150000,
  420. "maxBytesInMemory": 0,
  421. "skipBytesInMemoryOverheadCheck": false,
  422. "maxRowsPerSegment": 5000000,
  423. "maxTotalRows": null,
  424. "intermediatePersistPeriod": "PT10M",
  425. "maxPendingPersists": 0,
  426. "indexSpec": {
  427. "bitmap": {
  428. "type": "roaring"
  429. },
  430. "dimensionCompression": "lz4",
  431. "stringDictionaryEncoding": {
  432. "type": "utf8"
  433. },
  434. "metricCompression": "lz4",
  435. "longEncoding": "longs"
  436. },
  437. "indexSpecForIntermediatePersists": {
  438. "bitmap": {
  439. "type": "roaring"
  440. },
  441. "dimensionCompression": "lz4",
  442. "stringDictionaryEncoding": {
  443. "type": "utf8"
  444. },
  445. "metricCompression": "lz4",
  446. "longEncoding": "longs"
  447. },
  448. "reportParseExceptions": false,
  449. "handoffConditionTimeout": 0,
  450. "resetOffsetAutomatically": false,
  451. "segmentWriteOutMediumFactory": null,
  452. "workerThreads": null,
  453. "chatThreads": null,
  454. "chatRetries": 8,
  455. "httpTimeout": "PT10S",
  456. "shutdownTimeout": "PT80S",
  457. "offsetFetchPeriod": "PT30S",
  458. "intermediateHandoffPeriod": "P2147483647D",
  459. "logParseExceptions": false,
  460. "maxParseExceptions": 2147483647,
  461. "maxSavedParseExceptions": 0,
  462. "skipSequenceNumberAvailabilityCheck": false,
  463. "repartitionTransitionDuration": "PT120S"
  464. }
  465. },
  466. "dataSchema": {
  467. "dataSource": "social_media",
  468. "timestampSpec": {
  469. "column": "__time",
  470. "format": "iso",
  471. "missingValue": null
  472. },
  473. "dimensionsSpec": {
  474. "dimensions": [
  475. {
  476. "type": "string",
  477. "name": "username",
  478. "multiValueHandling": "SORTED_ARRAY",
  479. "createBitmapIndex": true
  480. },
  481. {
  482. "type": "string",
  483. "name": "post_title",
  484. "multiValueHandling": "SORTED_ARRAY",
  485. "createBitmapIndex": true
  486. },
  487. {
  488. "type": "long",
  489. "name": "views",
  490. "multiValueHandling": "SORTED_ARRAY",
  491. "createBitmapIndex": false
  492. },
  493. {
  494. "type": "long",
  495. "name": "upvotes",
  496. "multiValueHandling": "SORTED_ARRAY",
  497. "createBitmapIndex": false
  498. },
  499. {
  500. "type": "long",
  501. "name": "comments",
  502. "multiValueHandling": "SORTED_ARRAY",
  503. "createBitmapIndex": false
  504. },
  505. {
  506. "type": "string",
  507. "name": "edited",
  508. "multiValueHandling": "SORTED_ARRAY",
  509. "createBitmapIndex": true
  510. }
  511. ],
  512. "dimensionExclusions": [
  513. "__time"
  514. ],
  515. "includeAllDimensions": false,
  516. "useSchemaDiscovery": false
  517. },
  518. "metricsSpec": [],
  519. "granularitySpec": {
  520. "type": "uniform",
  521. "segmentGranularity": "HOUR",
  522. "queryGranularity": {
  523. "type": "none"
  524. },
  525. "rollup": false,
  526. "intervals": []
  527. },
  528. "transformSpec": {
  529. "filter": null,
  530. "transforms": []
  531. }
  532. },
  533. "tuningConfig": {
  534. "type": "kafka",
  535. "appendableIndexSpec": {
  536. "type": "onheap",
  537. "preserveExistingMetrics": false
  538. },
  539. "maxRowsInMemory": 150000,
  540. "maxBytesInMemory": 0,
  541. "skipBytesInMemoryOverheadCheck": false,
  542. "maxRowsPerSegment": 5000000,
  543. "maxTotalRows": null,
  544. "intermediatePersistPeriod": "PT10M",
  545. "maxPendingPersists": 0,
  546. "indexSpec": {
  547. "bitmap": {
  548. "type": "roaring"
  549. },
  550. "dimensionCompression": "lz4",
  551. "stringDictionaryEncoding": {
  552. "type": "utf8"
  553. },
  554. "metricCompression": "lz4",
  555. "longEncoding": "longs"
  556. },
  557. "indexSpecForIntermediatePersists": {
  558. "bitmap": {
  559. "type": "roaring"
  560. },
  561. "dimensionCompression": "lz4",
  562. "stringDictionaryEncoding": {
  563. "type": "utf8"
  564. },
  565. "metricCompression": "lz4",
  566. "longEncoding": "longs"
  567. },
  568. "reportParseExceptions": false,
  569. "handoffConditionTimeout": 0,
  570. "resetOffsetAutomatically": false,
  571. "segmentWriteOutMediumFactory": null,
  572. "workerThreads": null,
  573. "chatThreads": null,
  574. "chatRetries": 8,
  575. "httpTimeout": "PT10S",
  576. "shutdownTimeout": "PT80S",
  577. "offsetFetchPeriod": "PT30S",
  578. "intermediateHandoffPeriod": "P2147483647D",
  579. "logParseExceptions": false,
  580. "maxParseExceptions": 2147483647,
  581. "maxSavedParseExceptions": 0,
  582. "skipSequenceNumberAvailabilityCheck": false,
  583. "repartitionTransitionDuration": "PT120S"
  584. },
  585. "ioConfig": {
  586. "topic": "social_media",
  587. "inputFormat": {
  588. "type": "json",
  589. "keepNullColumns": false,
  590. "assumeNewlineDelimited": false,
  591. "useJsonNodeReader": false
  592. },
  593. "replicas": 1,
  594. "taskCount": 1,
  595. "taskDuration": "PT3600S",
  596. "consumerProperties": {
  597. "bootstrap.servers": "localhost:9094"
  598. },
  599. "autoScalerConfig": null,
  600. "pollTimeout": 100,
  601. "startDelay": "PT5S",
  602. "period": "PT30S",
  603. "useEarliestOffset": true,
  604. "completionTimeout": "PT1800S",
  605. "lateMessageRejectionPeriod": null,
  606. "earlyMessageRejectionPeriod": null,
  607. "lateMessageRejectionStartDateTime": null,
  608. "configOverrides": null,
  609. "idleConfig": null,
  610. "stream": "social_media",
  611. "useEarliestSequenceNumber": true
  612. },
  613. "context": null,
  614. "suspended": false
  615. },
  616. "suspended": false
  617. }
  618. ]

Get an array of supervisor states

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.

URL

GET /druid/indexer/v1/supervisor?state=true

Responses

  • 200 SUCCESS

Successfully retrieved supervisor state objects


Sample request

  • cURL
  • HTTP
  1. curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor?state=true"
  1. GET /druid/indexer/v1/supervisor?state=true HTTP/1.1
  2. Host: http://ROUTER_IP:ROUTER_PORT

Sample response

Click to show sample response

  1. [
  2. {
  3. "id": "wikipedia_stream",
  4. "state": "UNHEALTHY_SUPERVISOR",
  5. "detailedState": "UNABLE_TO_CONNECT_TO_STREAM",
  6. "healthy": false,
  7. "suspended": false
  8. },
  9. {
  10. "id": "social_media",
  11. "state": "RUNNING",
  12. "detailedState": "RUNNING",
  13. "healthy": true,
  14. "suspended": false
  15. }
  16. ]

Get supervisor specification

Retrieves the specification for a single supervisor. The returned specification includes the dataSchema, ioConfig, and tuningConfig objects.

URL

GET /druid/indexer/v1/supervisor/:supervisorId

Responses

  • 200 SUCCESS
  • 404 NOT FOUND

Successfully retrieved supervisor spec

Invalid supervisor ID


Sample request

The following example shows how to retrieve the specification of a supervisor with the name wikipedia_stream.

  • cURL
  • HTTP
  1. curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/wikipedia_stream"
  1. GET /druid/indexer/v1/supervisor/wikipedia_stream HTTP/1.1
  2. Host: http://ROUTER_IP:ROUTER_PORT

Sample response

Click to show sample response

  1. {
  2. "type": "kafka",
  3. "spec": {
  4. "dataSchema": {
  5. "dataSource": "social_media",
  6. "timestampSpec": {
  7. "column": "__time",
  8. "format": "iso",
  9. "missingValue": null
  10. },
  11. "dimensionsSpec": {
  12. "dimensions": [
  13. {
  14. "type": "string",
  15. "name": "username",
  16. "multiValueHandling": "SORTED_ARRAY",
  17. "createBitmapIndex": true
  18. },
  19. {
  20. "type": "string",
  21. "name": "post_title",
  22. "multiValueHandling": "SORTED_ARRAY",
  23. "createBitmapIndex": true
  24. },
  25. {
  26. "type": "long",
  27. "name": "views",
  28. "multiValueHandling": "SORTED_ARRAY",
  29. "createBitmapIndex": false
  30. },
  31. {
  32. "type": "long",
  33. "name": "upvotes",
  34. "multiValueHandling": "SORTED_ARRAY",
  35. "createBitmapIndex": false
  36. },
  37. {
  38. "type": "long",
  39. "name": "comments",
  40. "multiValueHandling": "SORTED_ARRAY",
  41. "createBitmapIndex": false
  42. },
  43. {
  44. "type": "string",
  45. "name": "edited",
  46. "multiValueHandling": "SORTED_ARRAY",
  47. "createBitmapIndex": true
  48. }
  49. ],
  50. "dimensionExclusions": [
  51. "__time"
  52. ],
  53. "includeAllDimensions": false,
  54. "useSchemaDiscovery": false
  55. },
  56. "metricsSpec": [],
  57. "granularitySpec": {
  58. "type": "uniform",
  59. "segmentGranularity": "HOUR",
  60. "queryGranularity": {
  61. "type": "none"
  62. },
  63. "rollup": false,
  64. "intervals": []
  65. },
  66. "transformSpec": {
  67. "filter": null,
  68. "transforms": []
  69. }
  70. },
  71. "ioConfig": {
  72. "topic": "social_media",
  73. "inputFormat": {
  74. "type": "json",
  75. "keepNullColumns": false,
  76. "assumeNewlineDelimited": false,
  77. "useJsonNodeReader": false
  78. },
  79. "replicas": 1,
  80. "taskCount": 1,
  81. "taskDuration": "PT3600S",
  82. "consumerProperties": {
  83. "bootstrap.servers": "localhost:9094"
  84. },
  85. "autoScalerConfig": null,
  86. "pollTimeout": 100,
  87. "startDelay": "PT5S",
  88. "period": "PT30S",
  89. "useEarliestOffset": true,
  90. "completionTimeout": "PT1800S",
  91. "lateMessageRejectionPeriod": null,
  92. "earlyMessageRejectionPeriod": null,
  93. "lateMessageRejectionStartDateTime": null,
  94. "configOverrides": null,
  95. "idleConfig": null,
  96. "stream": "social_media",
  97. "useEarliestSequenceNumber": true
  98. },
  99. "tuningConfig": {
  100. "type": "kafka",
  101. "appendableIndexSpec": {
  102. "type": "onheap",
  103. "preserveExistingMetrics": false
  104. },
  105. "maxRowsInMemory": 150000,
  106. "maxBytesInMemory": 0,
  107. "skipBytesInMemoryOverheadCheck": false,
  108. "maxRowsPerSegment": 5000000,
  109. "maxTotalRows": null,
  110. "intermediatePersistPeriod": "PT10M",
  111. "maxPendingPersists": 0,
  112. "indexSpec": {
  113. "bitmap": {
  114. "type": "roaring"
  115. },
  116. "dimensionCompression": "lz4",
  117. "stringDictionaryEncoding": {
  118. "type": "utf8"
  119. },
  120. "metricCompression": "lz4",
  121. "longEncoding": "longs"
  122. },
  123. "indexSpecForIntermediatePersists": {
  124. "bitmap": {
  125. "type": "roaring"
  126. },
  127. "dimensionCompression": "lz4",
  128. "stringDictionaryEncoding": {
  129. "type": "utf8"
  130. },
  131. "metricCompression": "lz4",
  132. "longEncoding": "longs"
  133. },
  134. "reportParseExceptions": false,
  135. "handoffConditionTimeout": 0,
  136. "resetOffsetAutomatically": false,
  137. "segmentWriteOutMediumFactory": null,
  138. "workerThreads": null,
  139. "chatThreads": null,
  140. "chatRetries": 8,
  141. "httpTimeout": "PT10S",
  142. "shutdownTimeout": "PT80S",
  143. "offsetFetchPeriod": "PT30S",
  144. "intermediateHandoffPeriod": "P2147483647D",
  145. "logParseExceptions": false,
  146. "maxParseExceptions": 2147483647,
  147. "maxSavedParseExceptions": 0,
  148. "skipSequenceNumberAvailabilityCheck": false,
  149. "repartitionTransitionDuration": "PT120S"
  150. }
  151. },
  152. "dataSchema": {
  153. "dataSource": "social_media",
  154. "timestampSpec": {
  155. "column": "__time",
  156. "format": "iso",
  157. "missingValue": null
  158. },
  159. "dimensionsSpec": {
  160. "dimensions": [
  161. {
  162. "type": "string",
  163. "name": "username",
  164. "multiValueHandling": "SORTED_ARRAY",
  165. "createBitmapIndex": true
  166. },
  167. {
  168. "type": "string",
  169. "name": "post_title",
  170. "multiValueHandling": "SORTED_ARRAY",
  171. "createBitmapIndex": true
  172. },
  173. {
  174. "type": "long",
  175. "name": "views",
  176. "multiValueHandling": "SORTED_ARRAY",
  177. "createBitmapIndex": false
  178. },
  179. {
  180. "type": "long",
  181. "name": "upvotes",
  182. "multiValueHandling": "SORTED_ARRAY",
  183. "createBitmapIndex": false
  184. },
  185. {
  186. "type": "long",
  187. "name": "comments",
  188. "multiValueHandling": "SORTED_ARRAY",
  189. "createBitmapIndex": false
  190. },
  191. {
  192. "type": "string",
  193. "name": "edited",
  194. "multiValueHandling": "SORTED_ARRAY",
  195. "createBitmapIndex": true
  196. }
  197. ],
  198. "dimensionExclusions": [
  199. "__time"
  200. ],
  201. "includeAllDimensions": false,
  202. "useSchemaDiscovery": false
  203. },
  204. "metricsSpec": [],
  205. "granularitySpec": {
  206. "type": "uniform",
  207. "segmentGranularity": "HOUR",
  208. "queryGranularity": {
  209. "type": "none"
  210. },
  211. "rollup": false,
  212. "intervals": []
  213. },
  214. "transformSpec": {
  215. "filter": null,
  216. "transforms": []
  217. }
  218. },
  219. "tuningConfig": {
  220. "type": "kafka",
  221. "appendableIndexSpec": {
  222. "type": "onheap",
  223. "preserveExistingMetrics": false
  224. },
  225. "maxRowsInMemory": 150000,
  226. "maxBytesInMemory": 0,
  227. "skipBytesInMemoryOverheadCheck": false,
  228. "maxRowsPerSegment": 5000000,
  229. "maxTotalRows": null,
  230. "intermediatePersistPeriod": "PT10M",
  231. "maxPendingPersists": 0,
  232. "indexSpec": {
  233. "bitmap": {
  234. "type": "roaring"
  235. },
  236. "dimensionCompression": "lz4",
  237. "stringDictionaryEncoding": {
  238. "type": "utf8"
  239. },
  240. "metricCompression": "lz4",
  241. "longEncoding": "longs"
  242. },
  243. "indexSpecForIntermediatePersists": {
  244. "bitmap": {
  245. "type": "roaring"
  246. },
  247. "dimensionCompression": "lz4",
  248. "stringDictionaryEncoding": {
  249. "type": "utf8"
  250. },
  251. "metricCompression": "lz4",
  252. "longEncoding": "longs"
  253. },
  254. "reportParseExceptions": false,
  255. "handoffConditionTimeout": 0,
  256. "resetOffsetAutomatically": false,
  257. "segmentWriteOutMediumFactory": null,
  258. "workerThreads": null,
  259. "chatThreads": null,
  260. "chatRetries": 8,
  261. "httpTimeout": "PT10S",
  262. "shutdownTimeout": "PT80S",
  263. "offsetFetchPeriod": "PT30S",
  264. "intermediateHandoffPeriod": "P2147483647D",
  265. "logParseExceptions": false,
  266. "maxParseExceptions": 2147483647,
  267. "maxSavedParseExceptions": 0,
  268. "skipSequenceNumberAvailabilityCheck": false,
  269. "repartitionTransitionDuration": "PT120S"
  270. },
  271. "ioConfig": {
  272. "topic": "social_media",
  273. "inputFormat": {
  274. "type": "json",
  275. "keepNullColumns": false,
  276. "assumeNewlineDelimited": false,
  277. "useJsonNodeReader": false
  278. },
  279. "replicas": 1,
  280. "taskCount": 1,
  281. "taskDuration": "PT3600S",
  282. "consumerProperties": {
  283. "bootstrap.servers": "localhost:9094"
  284. },
  285. "autoScalerConfig": null,
  286. "pollTimeout": 100,
  287. "startDelay": "PT5S",
  288. "period": "PT30S",
  289. "useEarliestOffset": true,
  290. "completionTimeout": "PT1800S",
  291. "lateMessageRejectionPeriod": null,
  292. "earlyMessageRejectionPeriod": null,
  293. "lateMessageRejectionStartDateTime": null,
  294. "configOverrides": null,
  295. "idleConfig": null,
  296. "stream": "social_media",
  297. "useEarliestSequenceNumber": true
  298. },
  299. "context": null,
  300. "suspended": false
  301. }

Get supervisor status

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:

URL

GET /druid/indexer/v1/supervisor/:supervisorId/status

Responses

  • 200 SUCCESS
  • 404 NOT FOUND

Successfully retrieved supervisor status

Invalid supervisor ID


Sample request

The following example shows how to retrieve the status of a supervisor with the name social_media.

  • cURL
  • HTTP
  1. curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/social_media/status"
  1. GET /druid/indexer/v1/supervisor/social_media/status HTTP/1.1
  2. Host: http://ROUTER_IP:ROUTER_PORT

Sample response

Click to show sample response

  1. {
  2. "id": "social_media",
  3. "generationTime": "2023-07-05T23:24:43.934Z",
  4. "payload": {
  5. "dataSource": "social_media",
  6. "stream": "social_media",
  7. "partitions": 1,
  8. "replicas": 1,
  9. "durationSeconds": 3600,
  10. "activeTasks": [
  11. {
  12. "id": "index_kafka_social_media_ab72ae4127c591c_flcbhdlh",
  13. "startingOffsets": {
  14. "0": 3176381
  15. },
  16. "startTime": "2023-07-05T23:21:39.321Z",
  17. "remainingSeconds": 3415,
  18. "type": "ACTIVE",
  19. "currentOffsets": {
  20. "0": 3296632
  21. },
  22. "lag": {
  23. "0": 3
  24. }
  25. }
  26. ],
  27. "publishingTasks": [],
  28. "latestOffsets": {
  29. "0": 3296635
  30. },
  31. "minimumLag": {
  32. "0": 3
  33. },
  34. "aggregateLag": 3,
  35. "offsetsLastUpdated": "2023-07-05T23:24:30.212Z",
  36. "suspended": false,
  37. "healthy": true,
  38. "state": "RUNNING",
  39. "detailedState": "RUNNING",
  40. "recentErrors": []
  41. }
  42. }

Audit history

An audit history provides a comprehensive log of events, including supervisor configuration, creation, suspension, and modification history.

Get audit history for all supervisors

Retrieve an audit history of specs for all supervisors.

URL

GET /druid/indexer/v1/supervisor/history

Responses

  • 200 SUCCESS

Successfully retrieved audit history


Sample request

  • cURL
  • HTTP
  1. curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/history"
  1. GET /druid/indexer/v1/supervisor/history HTTP/1.1
  2. Host: http://ROUTER_IP:ROUTER_PORT

Sample response

Click to show sample response

  1. {
  2. "social_media": [
  3. {
  4. "spec": {
  5. "type": "kafka",
  6. "spec": {
  7. "dataSchema": {
  8. "dataSource": "social_media",
  9. "timestampSpec": {
  10. "column": "__time",
  11. "format": "iso",
  12. "missingValue": null
  13. },
  14. "dimensionsSpec": {
  15. "dimensions": [
  16. {
  17. "type": "string",
  18. "name": "username",
  19. "multiValueHandling": "SORTED_ARRAY",
  20. "createBitmapIndex": true
  21. },
  22. {
  23. "type": "string",
  24. "name": "post_title",
  25. "multiValueHandling": "SORTED_ARRAY",
  26. "createBitmapIndex": true
  27. },
  28. {
  29. "type": "long",
  30. "name": "views",
  31. "multiValueHandling": "SORTED_ARRAY",
  32. "createBitmapIndex": false
  33. },
  34. {
  35. "type": "long",
  36. "name": "upvotes",
  37. "multiValueHandling": "SORTED_ARRAY",
  38. "createBitmapIndex": false
  39. },
  40. {
  41. "type": "long",
  42. "name": "comments",
  43. "multiValueHandling": "SORTED_ARRAY",
  44. "createBitmapIndex": false
  45. },
  46. {
  47. "type": "string",
  48. "name": "edited",
  49. "multiValueHandling": "SORTED_ARRAY",
  50. "createBitmapIndex": true
  51. }
  52. ],
  53. "dimensionExclusions": [
  54. "__time"
  55. ],
  56. "includeAllDimensions": false,
  57. "useSchemaDiscovery": false
  58. },
  59. "metricsSpec": [],
  60. "granularitySpec": {
  61. "type": "uniform",
  62. "segmentGranularity": "HOUR",
  63. "queryGranularity": {
  64. "type": "none"
  65. },
  66. "rollup": false,
  67. "intervals": []
  68. },
  69. "transformSpec": {
  70. "filter": null,
  71. "transforms": []
  72. }
  73. },
  74. "ioConfig": {
  75. "topic": "social_media",
  76. "inputFormat": {
  77. "type": "json",
  78. "keepNullColumns": false,
  79. "assumeNewlineDelimited": false,
  80. "useJsonNodeReader": false
  81. },
  82. "replicas": 1,
  83. "taskCount": 1,
  84. "taskDuration": "PT3600S",
  85. "consumerProperties": {
  86. "bootstrap.servers": "localhost:9094"
  87. },
  88. "autoScalerConfig": null,
  89. "pollTimeout": 100,
  90. "startDelay": "PT5S",
  91. "period": "PT30S",
  92. "useEarliestOffset": true,
  93. "completionTimeout": "PT1800S",
  94. "lateMessageRejectionPeriod": null,
  95. "earlyMessageRejectionPeriod": null,
  96. "lateMessageRejectionStartDateTime": null,
  97. "configOverrides": null,
  98. "idleConfig": null,
  99. "stream": "social_media",
  100. "useEarliestSequenceNumber": true
  101. },
  102. "tuningConfig": {
  103. "type": "kafka",
  104. "appendableIndexSpec": {
  105. "type": "onheap",
  106. "preserveExistingMetrics": false
  107. },
  108. "maxRowsInMemory": 150000,
  109. "maxBytesInMemory": 0,
  110. "skipBytesInMemoryOverheadCheck": false,
  111. "maxRowsPerSegment": 5000000,
  112. "maxTotalRows": null,
  113. "intermediatePersistPeriod": "PT10M",
  114. "maxPendingPersists": 0,
  115. "indexSpec": {
  116. "bitmap": {
  117. "type": "roaring"
  118. },
  119. "dimensionCompression": "lz4",
  120. "stringDictionaryEncoding": {
  121. "type": "utf8"
  122. },
  123. "metricCompression": "lz4",
  124. "longEncoding": "longs"
  125. },
  126. "indexSpecForIntermediatePersists": {
  127. "bitmap": {
  128. "type": "roaring"
  129. },
  130. "dimensionCompression": "lz4",
  131. "stringDictionaryEncoding": {
  132. "type": "utf8"
  133. },
  134. "metricCompression": "lz4",
  135. "longEncoding": "longs"
  136. },
  137. "reportParseExceptions": false,
  138. "handoffConditionTimeout": 0,
  139. "resetOffsetAutomatically": false,
  140. "segmentWriteOutMediumFactory": null,
  141. "workerThreads": null,
  142. "chatThreads": null,
  143. "chatRetries": 8,
  144. "httpTimeout": "PT10S",
  145. "shutdownTimeout": "PT80S",
  146. "offsetFetchPeriod": "PT30S",
  147. "intermediateHandoffPeriod": "P2147483647D",
  148. "logParseExceptions": false,
  149. "maxParseExceptions": 2147483647,
  150. "maxSavedParseExceptions": 0,
  151. "skipSequenceNumberAvailabilityCheck": false,
  152. "repartitionTransitionDuration": "PT120S"
  153. }
  154. },
  155. "dataSchema": {
  156. "dataSource": "social_media",
  157. "timestampSpec": {
  158. "column": "__time",
  159. "format": "iso",
  160. "missingValue": null
  161. },
  162. "dimensionsSpec": {
  163. "dimensions": [
  164. {
  165. "type": "string",
  166. "name": "username",
  167. "multiValueHandling": "SORTED_ARRAY",
  168. "createBitmapIndex": true
  169. },
  170. {
  171. "type": "string",
  172. "name": "post_title",
  173. "multiValueHandling": "SORTED_ARRAY",
  174. "createBitmapIndex": true
  175. },
  176. {
  177. "type": "long",
  178. "name": "views",
  179. "multiValueHandling": "SORTED_ARRAY",
  180. "createBitmapIndex": false
  181. },
  182. {
  183. "type": "long",
  184. "name": "upvotes",
  185. "multiValueHandling": "SORTED_ARRAY",
  186. "createBitmapIndex": false
  187. },
  188. {
  189. "type": "long",
  190. "name": "comments",
  191. "multiValueHandling": "SORTED_ARRAY",
  192. "createBitmapIndex": false
  193. },
  194. {
  195. "type": "string",
  196. "name": "edited",
  197. "multiValueHandling": "SORTED_ARRAY",
  198. "createBitmapIndex": true
  199. }
  200. ],
  201. "dimensionExclusions": [
  202. "__time"
  203. ],
  204. "includeAllDimensions": false,
  205. "useSchemaDiscovery": false
  206. },
  207. "metricsSpec": [],
  208. "granularitySpec": {
  209. "type": "uniform",
  210. "segmentGranularity": "HOUR",
  211. "queryGranularity": {
  212. "type": "none"
  213. },
  214. "rollup": false,
  215. "intervals": []
  216. },
  217. "transformSpec": {
  218. "filter": null,
  219. "transforms": []
  220. }
  221. },
  222. "tuningConfig": {
  223. "type": "kafka",
  224. "appendableIndexSpec": {
  225. "type": "onheap",
  226. "preserveExistingMetrics": false
  227. },
  228. "maxRowsInMemory": 150000,
  229. "maxBytesInMemory": 0,
  230. "skipBytesInMemoryOverheadCheck": false,
  231. "maxRowsPerSegment": 5000000,
  232. "maxTotalRows": null,
  233. "intermediatePersistPeriod": "PT10M",
  234. "maxPendingPersists": 0,
  235. "indexSpec": {
  236. "bitmap": {
  237. "type": "roaring"
  238. },
  239. "dimensionCompression": "lz4",
  240. "stringDictionaryEncoding": {
  241. "type": "utf8"
  242. },
  243. "metricCompression": "lz4",
  244. "longEncoding": "longs"
  245. },
  246. "indexSpecForIntermediatePersists": {
  247. "bitmap": {
  248. "type": "roaring"
  249. },
  250. "dimensionCompression": "lz4",
  251. "stringDictionaryEncoding": {
  252. "type": "utf8"
  253. },
  254. "metricCompression": "lz4",
  255. "longEncoding": "longs"
  256. },
  257. "reportParseExceptions": false,
  258. "handoffConditionTimeout": 0,
  259. "resetOffsetAutomatically": false,
  260. "segmentWriteOutMediumFactory": null,
  261. "workerThreads": null,
  262. "chatThreads": null,
  263. "chatRetries": 8,
  264. "httpTimeout": "PT10S",
  265. "shutdownTimeout": "PT80S",
  266. "offsetFetchPeriod": "PT30S",
  267. "intermediateHandoffPeriod": "P2147483647D",
  268. "logParseExceptions": false,
  269. "maxParseExceptions": 2147483647,
  270. "maxSavedParseExceptions": 0,
  271. "skipSequenceNumberAvailabilityCheck": false,
  272. "repartitionTransitionDuration": "PT120S"
  273. },
  274. "ioConfig": {
  275. "topic": "social_media",
  276. "inputFormat": {
  277. "type": "json",
  278. "keepNullColumns": false,
  279. "assumeNewlineDelimited": false,
  280. "useJsonNodeReader": false
  281. },
  282. "replicas": 1,
  283. "taskCount": 1,
  284. "taskDuration": "PT3600S",
  285. "consumerProperties": {
  286. "bootstrap.servers": "localhost:9094"
  287. },
  288. "autoScalerConfig": null,
  289. "pollTimeout": 100,
  290. "startDelay": "PT5S",
  291. "period": "PT30S",
  292. "useEarliestOffset": true,
  293. "completionTimeout": "PT1800S",
  294. "lateMessageRejectionPeriod": null,
  295. "earlyMessageRejectionPeriod": null,
  296. "lateMessageRejectionStartDateTime": null,
  297. "configOverrides": null,
  298. "idleConfig": null,
  299. "stream": "social_media",
  300. "useEarliestSequenceNumber": true
  301. },
  302. "context": null,
  303. "suspended": false
  304. },
  305. "version": "2023-07-03T18:51:02.970Z"
  306. }
  307. ]
  308. }

Get audit history for a specific supervisor

Retrieves an audit history of specs for a single supervisor.

URL

GET /druid/indexer/v1/supervisor/:supervisorId/history

Responses

  • 200 SUCCESS
  • 404 NOT FOUND

Successfully retrieved supervisor audit history

Invalid supervisor ID


Sample request

The following example shows how to retrieve the audit history of a supervisor with the name wikipedia_stream.

  • cURL
  • HTTP
  1. curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/wikipedia_stream/history"
  1. GET /druid/indexer/v1/supervisor/wikipedia_stream/history HTTP/1.1
  2. Host: http://ROUTER_IP:ROUTER_PORT

Sample response

Click to show sample response

  1. [
  2. {
  3. "spec": {
  4. "type": "kafka",
  5. "spec": {
  6. "dataSchema": {
  7. "dataSource": "wikipedia_stream",
  8. "timestampSpec": {
  9. "column": "__time",
  10. "format": "iso",
  11. "missingValue": null
  12. },
  13. "dimensionsSpec": {
  14. "dimensions": [
  15. {
  16. "type": "string",
  17. "name": "username",
  18. "multiValueHandling": "SORTED_ARRAY",
  19. "createBitmapIndex": true
  20. },
  21. {
  22. "type": "string",
  23. "name": "post_title",
  24. "multiValueHandling": "SORTED_ARRAY",
  25. "createBitmapIndex": true
  26. },
  27. {
  28. "type": "long",
  29. "name": "views",
  30. "multiValueHandling": "SORTED_ARRAY",
  31. "createBitmapIndex": false
  32. },
  33. {
  34. "type": "long",
  35. "name": "upvotes",
  36. "multiValueHandling": "SORTED_ARRAY",
  37. "createBitmapIndex": false
  38. },
  39. {
  40. "type": "long",
  41. "name": "comments",
  42. "multiValueHandling": "SORTED_ARRAY",
  43. "createBitmapIndex": false
  44. },
  45. {
  46. "type": "string",
  47. "name": "edited",
  48. "multiValueHandling": "SORTED_ARRAY",
  49. "createBitmapIndex": true
  50. }
  51. ],
  52. "dimensionExclusions": [
  53. "__time"
  54. ],
  55. "includeAllDimensions": false,
  56. "useSchemaDiscovery": false
  57. },
  58. "metricsSpec": [],
  59. "granularitySpec": {
  60. "type": "uniform",
  61. "segmentGranularity": "HOUR",
  62. "queryGranularity": {
  63. "type": "none"
  64. },
  65. "rollup": false,
  66. "intervals": []
  67. },
  68. "transformSpec": {
  69. "filter": null,
  70. "transforms": []
  71. }
  72. },
  73. "ioConfig": {
  74. "topic": "social_media",
  75. "inputFormat": {
  76. "type": "json",
  77. "keepNullColumns": false,
  78. "assumeNewlineDelimited": false,
  79. "useJsonNodeReader": false
  80. },
  81. "replicas": 1,
  82. "taskCount": 1,
  83. "taskDuration": "PT3600S",
  84. "consumerProperties": {
  85. "bootstrap.servers": "localhost:9042"
  86. },
  87. "autoScalerConfig": null,
  88. "pollTimeout": 100,
  89. "startDelay": "PT5S",
  90. "period": "PT30S",
  91. "useEarliestOffset": true,
  92. "completionTimeout": "PT1800S",
  93. "lateMessageRejectionPeriod": null,
  94. "earlyMessageRejectionPeriod": null,
  95. "lateMessageRejectionStartDateTime": null,
  96. "configOverrides": null,
  97. "idleConfig": null,
  98. "stream": "social_media",
  99. "useEarliestSequenceNumber": true
  100. },
  101. "tuningConfig": {
  102. "type": "kafka",
  103. "appendableIndexSpec": {
  104. "type": "onheap",
  105. "preserveExistingMetrics": false
  106. },
  107. "maxRowsInMemory": 150000,
  108. "maxBytesInMemory": 0,
  109. "skipBytesInMemoryOverheadCheck": false,
  110. "maxRowsPerSegment": 5000000,
  111. "maxTotalRows": null,
  112. "intermediatePersistPeriod": "PT10M",
  113. "maxPendingPersists": 0,
  114. "indexSpec": {
  115. "bitmap": {
  116. "type": "roaring"
  117. },
  118. "dimensionCompression": "lz4",
  119. "stringDictionaryEncoding": {
  120. "type": "utf8"
  121. },
  122. "metricCompression": "lz4",
  123. "longEncoding": "longs"
  124. },
  125. "indexSpecForIntermediatePersists": {
  126. "bitmap": {
  127. "type": "roaring"
  128. },
  129. "dimensionCompression": "lz4",
  130. "stringDictionaryEncoding": {
  131. "type": "utf8"
  132. },
  133. "metricCompression": "lz4",
  134. "longEncoding": "longs"
  135. },
  136. "reportParseExceptions": false,
  137. "handoffConditionTimeout": 0,
  138. "resetOffsetAutomatically": false,
  139. "segmentWriteOutMediumFactory": null,
  140. "workerThreads": null,
  141. "chatThreads": null,
  142. "chatRetries": 8,
  143. "httpTimeout": "PT10S",
  144. "shutdownTimeout": "PT80S",
  145. "offsetFetchPeriod": "PT30S",
  146. "intermediateHandoffPeriod": "P2147483647D",
  147. "logParseExceptions": false,
  148. "maxParseExceptions": 2147483647,
  149. "maxSavedParseExceptions": 0,
  150. "skipSequenceNumberAvailabilityCheck": false,
  151. "repartitionTransitionDuration": "PT120S"
  152. }
  153. },
  154. "dataSchema": {
  155. "dataSource": "wikipedia_stream",
  156. "timestampSpec": {
  157. "column": "__time",
  158. "format": "iso",
  159. "missingValue": null
  160. },
  161. "dimensionsSpec": {
  162. "dimensions": [
  163. {
  164. "type": "string",
  165. "name": "username",
  166. "multiValueHandling": "SORTED_ARRAY",
  167. "createBitmapIndex": true
  168. },
  169. {
  170. "type": "string",
  171. "name": "post_title",
  172. "multiValueHandling": "SORTED_ARRAY",
  173. "createBitmapIndex": true
  174. },
  175. {
  176. "type": "long",
  177. "name": "views",
  178. "multiValueHandling": "SORTED_ARRAY",
  179. "createBitmapIndex": false
  180. },
  181. {
  182. "type": "long",
  183. "name": "upvotes",
  184. "multiValueHandling": "SORTED_ARRAY",
  185. "createBitmapIndex": false
  186. },
  187. {
  188. "type": "long",
  189. "name": "comments",
  190. "multiValueHandling": "SORTED_ARRAY",
  191. "createBitmapIndex": false
  192. },
  193. {
  194. "type": "string",
  195. "name": "edited",
  196. "multiValueHandling": "SORTED_ARRAY",
  197. "createBitmapIndex": true
  198. }
  199. ],
  200. "dimensionExclusions": [
  201. "__time"
  202. ],
  203. "includeAllDimensions": false,
  204. "useSchemaDiscovery": false
  205. },
  206. "metricsSpec": [],
  207. "granularitySpec": {
  208. "type": "uniform",
  209. "segmentGranularity": "HOUR",
  210. "queryGranularity": {
  211. "type": "none"
  212. },
  213. "rollup": false,
  214. "intervals": []
  215. },
  216. "transformSpec": {
  217. "filter": null,
  218. "transforms": []
  219. }
  220. },
  221. "tuningConfig": {
  222. "type": "kafka",
  223. "appendableIndexSpec": {
  224. "type": "onheap",
  225. "preserveExistingMetrics": false
  226. },
  227. "maxRowsInMemory": 150000,
  228. "maxBytesInMemory": 0,
  229. "skipBytesInMemoryOverheadCheck": false,
  230. "maxRowsPerSegment": 5000000,
  231. "maxTotalRows": null,
  232. "intermediatePersistPeriod": "PT10M",
  233. "maxPendingPersists": 0,
  234. "indexSpec": {
  235. "bitmap": {
  236. "type": "roaring"
  237. },
  238. "dimensionCompression": "lz4",
  239. "stringDictionaryEncoding": {
  240. "type": "utf8"
  241. },
  242. "metricCompression": "lz4",
  243. "longEncoding": "longs"
  244. },
  245. "indexSpecForIntermediatePersists": {
  246. "bitmap": {
  247. "type": "roaring"
  248. },
  249. "dimensionCompression": "lz4",
  250. "stringDictionaryEncoding": {
  251. "type": "utf8"
  252. },
  253. "metricCompression": "lz4",
  254. "longEncoding": "longs"
  255. },
  256. "reportParseExceptions": false,
  257. "handoffConditionTimeout": 0,
  258. "resetOffsetAutomatically": false,
  259. "segmentWriteOutMediumFactory": null,
  260. "workerThreads": null,
  261. "chatThreads": null,
  262. "chatRetries": 8,
  263. "httpTimeout": "PT10S",
  264. "shutdownTimeout": "PT80S",
  265. "offsetFetchPeriod": "PT30S",
  266. "intermediateHandoffPeriod": "P2147483647D",
  267. "logParseExceptions": false,
  268. "maxParseExceptions": 2147483647,
  269. "maxSavedParseExceptions": 0,
  270. "skipSequenceNumberAvailabilityCheck": false,
  271. "repartitionTransitionDuration": "PT120S"
  272. },
  273. "ioConfig": {
  274. "topic": "social_media",
  275. "inputFormat": {
  276. "type": "json",
  277. "keepNullColumns": false,
  278. "assumeNewlineDelimited": false,
  279. "useJsonNodeReader": false
  280. },
  281. "replicas": 1,
  282. "taskCount": 1,
  283. "taskDuration": "PT3600S",
  284. "consumerProperties": {
  285. "bootstrap.servers": "localhost:9042"
  286. },
  287. "autoScalerConfig": null,
  288. "pollTimeout": 100,
  289. "startDelay": "PT5S",
  290. "period": "PT30S",
  291. "useEarliestOffset": true,
  292. "completionTimeout": "PT1800S",
  293. "lateMessageRejectionPeriod": null,
  294. "earlyMessageRejectionPeriod": null,
  295. "lateMessageRejectionStartDateTime": null,
  296. "configOverrides": null,
  297. "idleConfig": null,
  298. "stream": "social_media",
  299. "useEarliestSequenceNumber": true
  300. },
  301. "context": null,
  302. "suspended": false
  303. },
  304. "version": "2023-07-05T20:59:16.872Z"
  305. }
  306. ]

Manage supervisors

Create or update a supervisor

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.

URL

POST /druid/indexer/v1/supervisor

Responses

  • 200 SUCCESS
  • 415 UNSUPPORTED MEDIA TYPE

Successfully created a new supervisor or updated an existing supervisor

Request body content type is not in JSON format


Sample request

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
  1. curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor" \
  2. --header 'Content-Type: application/json' \
  3. --data '{
  4. "type": "kafka",
  5. "spec": {
  6. "ioConfig": {
  7. "type": "kafka",
  8. "consumerProperties": {
  9. "bootstrap.servers": "localhost:9094"
  10. },
  11. "topic": "social_media",
  12. "inputFormat": {
  13. "type": "json"
  14. },
  15. "useEarliestOffset": true
  16. },
  17. "tuningConfig": {
  18. "type": "kafka"
  19. },
  20. "dataSchema": {
  21. "dataSource": "social_media",
  22. "timestampSpec": {
  23. "column": "__time",
  24. "format": "iso"
  25. },
  26. "dimensionsSpec": {
  27. "dimensions": [
  28. "username",
  29. "post_title",
  30. {
  31. "type": "long",
  32. "name": "views"
  33. },
  34. {
  35. "type": "long",
  36. "name": "upvotes"
  37. },
  38. {
  39. "type": "long",
  40. "name": "comments"
  41. },
  42. "edited"
  43. ]
  44. },
  45. "granularitySpec": {
  46. "queryGranularity": "none",
  47. "rollup": false,
  48. "segmentGranularity": "hour"
  49. }
  50. }
  51. }
  52. }'
  1. POST /druid/indexer/v1/supervisor HTTP/1.1
  2. Host: http://ROUTER_IP:ROUTER_PORT
  3. Content-Type: application/json
  4. Content-Length: 1359
  5. {
  6. "type": "kafka",
  7. "spec": {
  8. "ioConfig": {
  9. "type": "kafka",
  10. "consumerProperties": {
  11. "bootstrap.servers": "localhost:9094"
  12. },
  13. "topic": "social_media",
  14. "inputFormat": {
  15. "type": "json"
  16. },
  17. "useEarliestOffset": true
  18. },
  19. "tuningConfig": {
  20. "type": "kafka"
  21. },
  22. "dataSchema": {
  23. "dataSource": "social_media",
  24. "timestampSpec": {
  25. "column": "__time",
  26. "format": "iso"
  27. },
  28. "dimensionsSpec": {
  29. "dimensions": [
  30. "username",
  31. "post_title",
  32. {
  33. "type": "long",
  34. "name": "views"
  35. },
  36. {
  37. "type": "long",
  38. "name": "upvotes"
  39. },
  40. {
  41. "type": "long",
  42. "name": "comments"
  43. },
  44. "edited"
  45. ]
  46. },
  47. "granularitySpec": {
  48. "queryGranularity": "none",
  49. "rollup": false,
  50. "segmentGranularity": "hour"
  51. }
  52. }
  53. }
  54. }

Sample response

Click to show sample response

  1. {
  2. "id": "social_media"
  3. }

Suspend a running supervisor

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.

URL

POST /druid/indexer/v1/supervisor/:supervisorId/suspend

Responses

  • 200 SUCCESS
  • 400 BAD REQUEST
  • 404 NOT FOUND

Successfully shut down supervisor

Supervisor already suspended

Invalid supervisor ID


Sample request

The following example shows how to suspend a running supervisor with the name social_media.

  • cURL
  • HTTP
  1. curl --request POST "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/social_media/suspend"
  1. POST /druid/indexer/v1/supervisor/social_media/suspend HTTP/1.1
  2. Host: http://ROUTER_IP:ROUTER_PORT

Sample response

Click to show sample response

  1. {
  2. "type": "kafka",
  3. "spec": {
  4. "dataSchema": {
  5. "dataSource": "social_media",
  6. "timestampSpec": {
  7. "column": "__time",
  8. "format": "iso",
  9. "missingValue": null
  10. },
  11. "dimensionsSpec": {
  12. "dimensions": [
  13. {
  14. "type": "string",
  15. "name": "username",
  16. "multiValueHandling": "SORTED_ARRAY",
  17. "createBitmapIndex": true
  18. },
  19. {
  20. "type": "string",
  21. "name": "post_title",
  22. "multiValueHandling": "SORTED_ARRAY",
  23. "createBitmapIndex": true
  24. },
  25. {
  26. "type": "long",
  27. "name": "views",
  28. "multiValueHandling": "SORTED_ARRAY",
  29. "createBitmapIndex": false
  30. },
  31. {
  32. "type": "long",
  33. "name": "upvotes",
  34. "multiValueHandling": "SORTED_ARRAY",
  35. "createBitmapIndex": false
  36. },
  37. {
  38. "type": "long",
  39. "name": "comments",
  40. "multiValueHandling": "SORTED_ARRAY",
  41. "createBitmapIndex": false
  42. },
  43. {
  44. "type": "string",
  45. "name": "edited",
  46. "multiValueHandling": "SORTED_ARRAY",
  47. "createBitmapIndex": true
  48. }
  49. ],
  50. "dimensionExclusions": [
  51. "__time"
  52. ],
  53. "includeAllDimensions": false,
  54. "useSchemaDiscovery": false
  55. },
  56. "metricsSpec": [],
  57. "granularitySpec": {
  58. "type": "uniform",
  59. "segmentGranularity": "HOUR",
  60. "queryGranularity": {
  61. "type": "none"
  62. },
  63. "rollup": false,
  64. "intervals": []
  65. },
  66. "transformSpec": {
  67. "filter": null,
  68. "transforms": []
  69. }
  70. },
  71. "ioConfig": {
  72. "topic": "social_media",
  73. "inputFormat": {
  74. "type": "json",
  75. "keepNullColumns": false,
  76. "assumeNewlineDelimited": false,
  77. "useJsonNodeReader": false
  78. },
  79. "replicas": 1,
  80. "taskCount": 1,
  81. "taskDuration": "PT3600S",
  82. "consumerProperties": {
  83. "bootstrap.servers": "localhost:9094"
  84. },
  85. "autoScalerConfig": null,
  86. "pollTimeout": 100,
  87. "startDelay": "PT5S",
  88. "period": "PT30S",
  89. "useEarliestOffset": true,
  90. "completionTimeout": "PT1800S",
  91. "lateMessageRejectionPeriod": null,
  92. "earlyMessageRejectionPeriod": null,
  93. "lateMessageRejectionStartDateTime": null,
  94. "configOverrides": null,
  95. "idleConfig": null,
  96. "stream": "social_media",
  97. "useEarliestSequenceNumber": true
  98. },
  99. "tuningConfig": {
  100. "type": "kafka",
  101. "appendableIndexSpec": {
  102. "type": "onheap",
  103. "preserveExistingMetrics": false
  104. },
  105. "maxRowsInMemory": 150000,
  106. "maxBytesInMemory": 0,
  107. "skipBytesInMemoryOverheadCheck": false,
  108. "maxRowsPerSegment": 5000000,
  109. "maxTotalRows": null,
  110. "intermediatePersistPeriod": "PT10M",
  111. "maxPendingPersists": 0,
  112. "indexSpec": {
  113. "bitmap": {
  114. "type": "roaring"
  115. },
  116. "dimensionCompression": "lz4",
  117. "stringDictionaryEncoding": {
  118. "type": "utf8"
  119. },
  120. "metricCompression": "lz4",
  121. "longEncoding": "longs"
  122. },
  123. "indexSpecForIntermediatePersists": {
  124. "bitmap": {
  125. "type": "roaring"
  126. },
  127. "dimensionCompression": "lz4",
  128. "stringDictionaryEncoding": {
  129. "type": "utf8"
  130. },
  131. "metricCompression": "lz4",
  132. "longEncoding": "longs"
  133. },
  134. "reportParseExceptions": false,
  135. "handoffConditionTimeout": 0,
  136. "resetOffsetAutomatically": false,
  137. "segmentWriteOutMediumFactory": null,
  138. "workerThreads": null,
  139. "chatThreads": null,
  140. "chatRetries": 8,
  141. "httpTimeout": "PT10S",
  142. "shutdownTimeout": "PT80S",
  143. "offsetFetchPeriod": "PT30S",
  144. "intermediateHandoffPeriod": "P2147483647D",
  145. "logParseExceptions": false,
  146. "maxParseExceptions": 2147483647,
  147. "maxSavedParseExceptions": 0,
  148. "skipSequenceNumberAvailabilityCheck": false,
  149. "repartitionTransitionDuration": "PT120S"
  150. }
  151. },
  152. "dataSchema": {
  153. "dataSource": "social_media",
  154. "timestampSpec": {
  155. "column": "__time",
  156. "format": "iso",
  157. "missingValue": null
  158. },
  159. "dimensionsSpec": {
  160. "dimensions": [
  161. {
  162. "type": "string",
  163. "name": "username",
  164. "multiValueHandling": "SORTED_ARRAY",
  165. "createBitmapIndex": true
  166. },
  167. {
  168. "type": "string",
  169. "name": "post_title",
  170. "multiValueHandling": "SORTED_ARRAY",
  171. "createBitmapIndex": true
  172. },
  173. {
  174. "type": "long",
  175. "name": "views",
  176. "multiValueHandling": "SORTED_ARRAY",
  177. "createBitmapIndex": false
  178. },
  179. {
  180. "type": "long",
  181. "name": "upvotes",
  182. "multiValueHandling": "SORTED_ARRAY",
  183. "createBitmapIndex": false
  184. },
  185. {
  186. "type": "long",
  187. "name": "comments",
  188. "multiValueHandling": "SORTED_ARRAY",
  189. "createBitmapIndex": false
  190. },
  191. {
  192. "type": "string",
  193. "name": "edited",
  194. "multiValueHandling": "SORTED_ARRAY",
  195. "createBitmapIndex": true
  196. }
  197. ],
  198. "dimensionExclusions": [
  199. "__time"
  200. ],
  201. "includeAllDimensions": false,
  202. "useSchemaDiscovery": false
  203. },
  204. "metricsSpec": [],
  205. "granularitySpec": {
  206. "type": "uniform",
  207. "segmentGranularity": "HOUR",
  208. "queryGranularity": {
  209. "type": "none"
  210. },
  211. "rollup": false,
  212. "intervals": []
  213. },
  214. "transformSpec": {
  215. "filter": null,
  216. "transforms": []
  217. }
  218. },
  219. "tuningConfig": {
  220. "type": "kafka",
  221. "appendableIndexSpec": {
  222. "type": "onheap",
  223. "preserveExistingMetrics": false
  224. },
  225. "maxRowsInMemory": 150000,
  226. "maxBytesInMemory": 0,
  227. "skipBytesInMemoryOverheadCheck": false,
  228. "maxRowsPerSegment": 5000000,
  229. "maxTotalRows": null,
  230. "intermediatePersistPeriod": "PT10M",
  231. "maxPendingPersists": 0,
  232. "indexSpec": {
  233. "bitmap": {
  234. "type": "roaring"
  235. },
  236. "dimensionCompression": "lz4",
  237. "stringDictionaryEncoding": {
  238. "type": "utf8"
  239. },
  240. "metricCompression": "lz4",
  241. "longEncoding": "longs"
  242. },
  243. "indexSpecForIntermediatePersists": {
  244. "bitmap": {
  245. "type": "roaring"
  246. },
  247. "dimensionCompression": "lz4",
  248. "stringDictionaryEncoding": {
  249. "type": "utf8"
  250. },
  251. "metricCompression": "lz4",
  252. "longEncoding": "longs"
  253. },
  254. "reportParseExceptions": false,
  255. "handoffConditionTimeout": 0,
  256. "resetOffsetAutomatically": false,
  257. "segmentWriteOutMediumFactory": null,
  258. "workerThreads": null,
  259. "chatThreads": null,
  260. "chatRetries": 8,
  261. "httpTimeout": "PT10S",
  262. "shutdownTimeout": "PT80S",
  263. "offsetFetchPeriod": "PT30S",
  264. "intermediateHandoffPeriod": "P2147483647D",
  265. "logParseExceptions": false,
  266. "maxParseExceptions": 2147483647,
  267. "maxSavedParseExceptions": 0,
  268. "skipSequenceNumberAvailabilityCheck": false,
  269. "repartitionTransitionDuration": "PT120S"
  270. },
  271. "ioConfig": {
  272. "topic": "social_media",
  273. "inputFormat": {
  274. "type": "json",
  275. "keepNullColumns": false,
  276. "assumeNewlineDelimited": false,
  277. "useJsonNodeReader": false
  278. },
  279. "replicas": 1,
  280. "taskCount": 1,
  281. "taskDuration": "PT3600S",
  282. "consumerProperties": {
  283. "bootstrap.servers": "localhost:9094"
  284. },
  285. "autoScalerConfig": null,
  286. "pollTimeout": 100,
  287. "startDelay": "PT5S",
  288. "period": "PT30S",
  289. "useEarliestOffset": true,
  290. "completionTimeout": "PT1800S",
  291. "lateMessageRejectionPeriod": null,
  292. "earlyMessageRejectionPeriod": null,
  293. "lateMessageRejectionStartDateTime": null,
  294. "configOverrides": null,
  295. "idleConfig": null,
  296. "stream": "social_media",
  297. "useEarliestSequenceNumber": true
  298. },
  299. "context": null,
  300. "suspended": true
  301. }

Suspend all supervisors

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.

URL

POST /druid/indexer/v1/supervisor/suspendAll

Responses

  • 200 SUCCESS

Successfully suspended all supervisors


Sample request

  • cURL
  • HTTP
  1. curl --request POST "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/suspendAll"
  1. POST /druid/indexer/v1/supervisor/suspendAll HTTP/1.1
  2. Host: http://ROUTER_IP:ROUTER_PORT

Sample response

Click to show sample response

  1. {
  2. "status": "success"
  3. }

Resume a supervisor

Resumes indexing tasks for a supervisor. Returns an updated supervisor spec with the suspended property set to false.

URL

POST /druid/indexer/v1/supervisor/:supervisorId/resume

Responses

  • 200 SUCCESS
  • 400 BAD REQUEST
  • 404 NOT FOUND

Successfully resumed supervisor

Supervisor already running

Invalid supervisor ID


Sample request

The following example resumes a previously suspended supervisor with name social_media.

  • cURL
  • HTTP
  1. curl --request POST "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/social_media/resume"
  1. POST /druid/indexer/v1/supervisor/social_media/resume HTTP/1.1
  2. Host: http://ROUTER_IP:ROUTER_PORT

Sample response

Click to show sample response

  1. {
  2. "type": "kafka",
  3. "spec": {
  4. "dataSchema": {
  5. "dataSource": "social_media",
  6. "timestampSpec": {
  7. "column": "__time",
  8. "format": "iso",
  9. "missingValue": null
  10. },
  11. "dimensionsSpec": {
  12. "dimensions": [
  13. {
  14. "type": "string",
  15. "name": "username",
  16. "multiValueHandling": "SORTED_ARRAY",
  17. "createBitmapIndex": true
  18. },
  19. {
  20. "type": "string",
  21. "name": "post_title",
  22. "multiValueHandling": "SORTED_ARRAY",
  23. "createBitmapIndex": true
  24. },
  25. {
  26. "type": "long",
  27. "name": "views",
  28. "multiValueHandling": "SORTED_ARRAY",
  29. "createBitmapIndex": false
  30. },
  31. {
  32. "type": "long",
  33. "name": "upvotes",
  34. "multiValueHandling": "SORTED_ARRAY",
  35. "createBitmapIndex": false
  36. },
  37. {
  38. "type": "long",
  39. "name": "comments",
  40. "multiValueHandling": "SORTED_ARRAY",
  41. "createBitmapIndex": false
  42. },
  43. {
  44. "type": "string",
  45. "name": "edited",
  46. "multiValueHandling": "SORTED_ARRAY",
  47. "createBitmapIndex": true
  48. }
  49. ],
  50. "dimensionExclusions": [
  51. "__time"
  52. ],
  53. "includeAllDimensions": false,
  54. "useSchemaDiscovery": false
  55. },
  56. "metricsSpec": [],
  57. "granularitySpec": {
  58. "type": "uniform",
  59. "segmentGranularity": "HOUR",
  60. "queryGranularity": {
  61. "type": "none"
  62. },
  63. "rollup": false,
  64. "intervals": []
  65. },
  66. "transformSpec": {
  67. "filter": null,
  68. "transforms": []
  69. }
  70. },
  71. "ioConfig": {
  72. "topic": "social_media",
  73. "inputFormat": {
  74. "type": "json",
  75. "keepNullColumns": false,
  76. "assumeNewlineDelimited": false,
  77. "useJsonNodeReader": false
  78. },
  79. "replicas": 1,
  80. "taskCount": 1,
  81. "taskDuration": "PT3600S",
  82. "consumerProperties": {
  83. "bootstrap.servers": "localhost:9094"
  84. },
  85. "autoScalerConfig": null,
  86. "pollTimeout": 100,
  87. "startDelay": "PT5S",
  88. "period": "PT30S",
  89. "useEarliestOffset": true,
  90. "completionTimeout": "PT1800S",
  91. "lateMessageRejectionPeriod": null,
  92. "earlyMessageRejectionPeriod": null,
  93. "lateMessageRejectionStartDateTime": null,
  94. "configOverrides": null,
  95. "idleConfig": null,
  96. "stream": "social_media",
  97. "useEarliestSequenceNumber": true
  98. },
  99. "tuningConfig": {
  100. "type": "kafka",
  101. "appendableIndexSpec": {
  102. "type": "onheap",
  103. "preserveExistingMetrics": false
  104. },
  105. "maxRowsInMemory": 150000,
  106. "maxBytesInMemory": 0,
  107. "skipBytesInMemoryOverheadCheck": false,
  108. "maxRowsPerSegment": 5000000,
  109. "maxTotalRows": null,
  110. "intermediatePersistPeriod": "PT10M",
  111. "maxPendingPersists": 0,
  112. "indexSpec": {
  113. "bitmap": {
  114. "type": "roaring"
  115. },
  116. "dimensionCompression": "lz4",
  117. "stringDictionaryEncoding": {
  118. "type": "utf8"
  119. },
  120. "metricCompression": "lz4",
  121. "longEncoding": "longs"
  122. },
  123. "indexSpecForIntermediatePersists": {
  124. "bitmap": {
  125. "type": "roaring"
  126. },
  127. "dimensionCompression": "lz4",
  128. "stringDictionaryEncoding": {
  129. "type": "utf8"
  130. },
  131. "metricCompression": "lz4",
  132. "longEncoding": "longs"
  133. },
  134. "reportParseExceptions": false,
  135. "handoffConditionTimeout": 0,
  136. "resetOffsetAutomatically": false,
  137. "segmentWriteOutMediumFactory": null,
  138. "workerThreads": null,
  139. "chatThreads": null,
  140. "chatRetries": 8,
  141. "httpTimeout": "PT10S",
  142. "shutdownTimeout": "PT80S",
  143. "offsetFetchPeriod": "PT30S",
  144. "intermediateHandoffPeriod": "P2147483647D",
  145. "logParseExceptions": false,
  146. "maxParseExceptions": 2147483647,
  147. "maxSavedParseExceptions": 0,
  148. "skipSequenceNumberAvailabilityCheck": false,
  149. "repartitionTransitionDuration": "PT120S"
  150. }
  151. },
  152. "dataSchema": {
  153. "dataSource": "social_media",
  154. "timestampSpec": {
  155. "column": "__time",
  156. "format": "iso",
  157. "missingValue": null
  158. },
  159. "dimensionsSpec": {
  160. "dimensions": [
  161. {
  162. "type": "string",
  163. "name": "username",
  164. "multiValueHandling": "SORTED_ARRAY",
  165. "createBitmapIndex": true
  166. },
  167. {
  168. "type": "string",
  169. "name": "post_title",
  170. "multiValueHandling": "SORTED_ARRAY",
  171. "createBitmapIndex": true
  172. },
  173. {
  174. "type": "long",
  175. "name": "views",
  176. "multiValueHandling": "SORTED_ARRAY",
  177. "createBitmapIndex": false
  178. },
  179. {
  180. "type": "long",
  181. "name": "upvotes",
  182. "multiValueHandling": "SORTED_ARRAY",
  183. "createBitmapIndex": false
  184. },
  185. {
  186. "type": "long",
  187. "name": "comments",
  188. "multiValueHandling": "SORTED_ARRAY",
  189. "createBitmapIndex": false
  190. },
  191. {
  192. "type": "string",
  193. "name": "edited",
  194. "multiValueHandling": "SORTED_ARRAY",
  195. "createBitmapIndex": true
  196. }
  197. ],
  198. "dimensionExclusions": [
  199. "__time"
  200. ],
  201. "includeAllDimensions": false,
  202. "useSchemaDiscovery": false
  203. },
  204. "metricsSpec": [],
  205. "granularitySpec": {
  206. "type": "uniform",
  207. "segmentGranularity": "HOUR",
  208. "queryGranularity": {
  209. "type": "none"
  210. },
  211. "rollup": false,
  212. "intervals": []
  213. },
  214. "transformSpec": {
  215. "filter": null,
  216. "transforms": []
  217. }
  218. },
  219. "tuningConfig": {
  220. "type": "kafka",
  221. "appendableIndexSpec": {
  222. "type": "onheap",
  223. "preserveExistingMetrics": false
  224. },
  225. "maxRowsInMemory": 150000,
  226. "maxBytesInMemory": 0,
  227. "skipBytesInMemoryOverheadCheck": false,
  228. "maxRowsPerSegment": 5000000,
  229. "maxTotalRows": null,
  230. "intermediatePersistPeriod": "PT10M",
  231. "maxPendingPersists": 0,
  232. "indexSpec": {
  233. "bitmap": {
  234. "type": "roaring"
  235. },
  236. "dimensionCompression": "lz4",
  237. "stringDictionaryEncoding": {
  238. "type": "utf8"
  239. },
  240. "metricCompression": "lz4",
  241. "longEncoding": "longs"
  242. },
  243. "indexSpecForIntermediatePersists": {
  244. "bitmap": {
  245. "type": "roaring"
  246. },
  247. "dimensionCompression": "lz4",
  248. "stringDictionaryEncoding": {
  249. "type": "utf8"
  250. },
  251. "metricCompression": "lz4",
  252. "longEncoding": "longs"
  253. },
  254. "reportParseExceptions": false,
  255. "handoffConditionTimeout": 0,
  256. "resetOffsetAutomatically": false,
  257. "segmentWriteOutMediumFactory": null,
  258. "workerThreads": null,
  259. "chatThreads": null,
  260. "chatRetries": 8,
  261. "httpTimeout": "PT10S",
  262. "shutdownTimeout": "PT80S",
  263. "offsetFetchPeriod": "PT30S",
  264. "intermediateHandoffPeriod": "P2147483647D",
  265. "logParseExceptions": false,
  266. "maxParseExceptions": 2147483647,
  267. "maxSavedParseExceptions": 0,
  268. "skipSequenceNumberAvailabilityCheck": false,
  269. "repartitionTransitionDuration": "PT120S"
  270. },
  271. "ioConfig": {
  272. "topic": "social_media",
  273. "inputFormat": {
  274. "type": "json",
  275. "keepNullColumns": false,
  276. "assumeNewlineDelimited": false,
  277. "useJsonNodeReader": false
  278. },
  279. "replicas": 1,
  280. "taskCount": 1,
  281. "taskDuration": "PT3600S",
  282. "consumerProperties": {
  283. "bootstrap.servers": "localhost:9094"
  284. },
  285. "autoScalerConfig": null,
  286. "pollTimeout": 100,
  287. "startDelay": "PT5S",
  288. "period": "PT30S",
  289. "useEarliestOffset": true,
  290. "completionTimeout": "PT1800S",
  291. "lateMessageRejectionPeriod": null,
  292. "earlyMessageRejectionPeriod": null,
  293. "lateMessageRejectionStartDateTime": null,
  294. "configOverrides": null,
  295. "idleConfig": null,
  296. "stream": "social_media",
  297. "useEarliestSequenceNumber": true
  298. },
  299. "context": null,
  300. "suspended": false
  301. }

Resume all supervisors

Resumes all supervisors. Note that this endpoint returns an HTTP 200 Success code even if there are no supervisors or suspended supervisors to resume.

URL

POST /druid/indexer/v1/supervisor/resumeAll

Responses

  • 200 SUCCESS

Successfully resumed all supervisors


Sample request

  • cURL
  • HTTP
  1. curl --request POST "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/resumeAll"
  1. POST /druid/indexer/v1/supervisor/resumeAll HTTP/1.1
  2. Host: http://ROUTER_IP:ROUTER_PORT

Sample response

Click to show sample response

  1. {
  2. "status": "success"
  3. }

Reset a supervisor

Resets the specified supervisor. This endpoint clears 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.

URL

POST /druid/indexer/v1/supervisor/:supervisorId/reset

Responses

  • 200 SUCCESS
  • 404 NOT FOUND

Successfully reset supervisor

Invalid supervisor ID


Sample request

The following example shows how to reset a supervisor with the name social_media.

  • cURL
  • HTTP
  1. curl --request POST "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/social_media/reset"
  1. POST /druid/indexer/v1/supervisor/social_media/reset HTTP/1.1
  2. Host: http://ROUTER_IP:ROUTER_PORT

Sample response

Click to show sample response

  1. {
  2. "id": "social_media"
  3. }

Terminate a supervisor

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.

URL

POST /druid/indexer/v1/supervisor/:supervisorId/terminate

Responses

  • 200 SUCCESS
  • 404 NOT FOUND

Successfully terminated a supervisor

Invalid supervisor ID or supervisor not running


Sample request

  • cURL
  • HTTP
  1. curl --request POST "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/social_media/terminate"
  1. POST /druid/indexer/v1/supervisor/social_media/terminate HTTP/1.1
  2. Host: http://ROUTER_IP:ROUTER_PORT

Sample response

Click to show sample response

  1. {
  2. "id": "social_media"
  3. }

Terminate all supervisors

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.

URL

POST /druid/indexer/v1/supervisor/terminateAll

Responses

  • 200 SUCCESS

Successfully terminated all supervisors


Sample request

  • cURL
  • HTTP
  1. curl --request POST "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/terminateAll"
  1. POST /druid/indexer/v1/supervisor/terminateAll HTTP/1.1
  2. Host: http://ROUTER_IP:ROUTER_PORT

Sample response

Click to show sample response

  1. {
  2. "status": "success"
  3. }

Shut down a supervisor

Shuts down a supervisor. This endpoint is deprecated and will be removed in future releases. Use the equivalent terminate endpoint instead.

URL

POST /druid/indexer/v1/supervisor/:supervisorId/shutdown