Skip to main content

Apache Kafka supervisor operations reference

This topic contains operations reference information to run and maintain Apache Kafka supervisors for Apache Druid. It includes descriptions of how some supervisor APIs work within Kafka Indexing Service.

For all supervisor APIs, see Supervisor API reference.

Getting Supervisor Status Report

GET /druid/indexer/v1/supervisor/<supervisorId>/status returns a snapshot report of the current state of the tasks managed by the given supervisor. This includes the latest offsets as reported by Kafka, the consumer lag per partition, as well as the aggregate lag of all partitions. The consumer lag per partition may be reported as negative values if the supervisor has not received a recent latest offset response from Kafka. The aggregate lag value will always be >= 0.

The status report also contains the supervisor's state and a list of recently thrown exceptions (reported as recentErrors, whose max size can be controlled using the druid.supervisor.maxStoredExceptionEvents configuration). There are two fields related to the supervisor's state - state and detailedState. The state field will always be one of a small number of generic states that are applicable to any type of supervisor, while the detailedState field will contain a more descriptive, implementation-specific state that may provide more insight into the supervisor's activities than the generic state field.

The list of possible state values are: [PENDING, RUNNING, SUSPENDED, STOPPING, UNHEALTHY_SUPERVISOR, UNHEALTHY_TASKS]

The list of detailedState values and their corresponding state mapping is as follows:

Detailed StateCorresponding StateDescription
UNHEALTHY_SUPERVISORUNHEALTHY_SUPERVISORThe supervisor has encountered errors on the past druid.supervisor.unhealthinessThreshold iterations
UNHEALTHY_TASKSUNHEALTHY_TASKSThe last druid.supervisor.taskUnhealthinessThreshold tasks have all failed
UNABLE_TO_CONNECT_TO_STREAMUNHEALTHY_SUPERVISORThe supervisor is encountering connectivity issues with Kafka and has not successfully connected in the past
LOST_CONTACT_WITH_STREAMUNHEALTHY_SUPERVISORThe supervisor is encountering connectivity issues with Kafka but has successfully connected in the past
PENDING (first iteration only)PENDINGThe supervisor has been initialized and hasn't started connecting to the stream
CONNECTING_TO_STREAM (first iteration only)RUNNINGThe supervisor is trying to connect to the stream and update partition data
DISCOVERING_INITIAL_TASKS (first iteration only)RUNNINGThe supervisor is discovering already-running tasks
CREATING_TASKS (first iteration only)RUNNINGThe supervisor is creating tasks and discovering state
RUNNINGRUNNINGThe supervisor has started tasks and is waiting for taskDuration to elapse
IDLEIDLEThe supervisor is not creating tasks since the input stream has not received any new data and all the existing data is read.
SUSPENDEDSUSPENDEDThe supervisor has been suspended
STOPPINGSTOPPINGThe supervisor is stopping

On each iteration of the supervisor's run loop, the supervisor completes the following tasks in sequence: 1) Fetch the list of partitions from Kafka and determine the starting offset for each partition (either based on the last processed offset if continuing, or starting from the beginning or ending of the stream if this is a new topic). 2) Discover any running indexing tasks that are writing to the supervisor's datasource and adopt them if they match the supervisor's configuration, else signal them to stop. 3) Send a status request to each supervised task to update our view of the state of the tasks under our supervision. 4) Handle tasks that have exceeded taskDuration and should transition from the reading to publishing state. 5) Handle tasks that have finished publishing and signal redundant replica tasks to stop. 6) Handle tasks that have failed and clean up the supervisor's internal state. 7) Compare the list of healthy tasks to the requested taskCount and replicas configurations and create additional tasks if required in case supervisor is not idle.

The detailedState field will show additional values (those marked with "first iteration only") the first time the supervisor executes this run loop after startup or after resuming from a suspension. This is intended to surface initialization-type issues, where the supervisor is unable to reach a stable state (perhaps because it can't connect to Kafka, it can't read from the Kafka topic, or it can't communicate with existing tasks). Once the supervisor is stable - that is, once it has completed a full execution without encountering any issues - detailedState will show a RUNNING state until it is idle, stopped, suspended, or hits a task failure threshold and transitions to an unhealthy state.

Getting Supervisor Ingestion Stats Report

GET /druid/indexer/v1/supervisor/<supervisorId>/stats returns a snapshot of the current ingestion row counters for each task being managed by the supervisor, along with moving averages for the row counters.

See Task Reports: Row Stats for more information.

Supervisor Health Check

GET /druid/indexer/v1/supervisor/<supervisorId>/health returns 200 OK if the supervisor is healthy and 503 Service Unavailable if it is unhealthy. Healthiness is determined by the supervisor's state (as returned by the /status endpoint) and the druid.supervisor.* Overlord configuration thresholds.

Updating Existing Supervisors

POST /druid/indexer/v1/supervisor can be used to update existing supervisor spec. Calling this endpoint when there is already an existing supervisor for the same dataSource will cause:

  • The running supervisor to signal its managed tasks to stop reading and begin publishing.
  • The running supervisor to exit.
  • A new supervisor to be created using the configuration provided in the request body. This supervisor will retain the existing publishing tasks and will create new tasks starting at the offsets the publishing tasks ended on.

Seamless schema migrations can thus be achieved by simply submitting the new schema using this endpoint.

Suspending and Resuming Supervisors

You can suspend and resume a supervisor using POST /druid/indexer/v1/supervisor/<supervisorId>/suspend and POST /druid/indexer/v1/supervisor/<supervisorId>/resume, respectively.

Note that the supervisor itself will still be operating and emitting logs and metrics, it will just ensure that no indexing tasks are running until the supervisor is resumed.

Resetting Supervisors

The POST /druid/indexer/v1/supervisor/<supervisorId>/reset operation clears stored offsets, causing the supervisor to start reading offsets from either the earliest or latest offsets in Kafka (depending on the value of useEarliestOffset). After clearing stored offsets, the supervisor kills and recreates any active tasks, so that tasks begin reading from valid offsets.

Use care when using this operation! Resetting the supervisor may cause Kafka messages to be skipped or read twice, resulting in missing or duplicate data.

The reason for using this operation is to recover from a state in which the supervisor ceases operating due to missing offsets. The indexing service keeps track of the latest persisted Kafka offsets in order to provide exactly-once ingestion guarantees across tasks. Subsequent tasks must start reading from where the previous task completed in order for the generated segments to be accepted. If the messages at the expected starting offsets are no longer available in Kafka (typically because the message retention period has elapsed or the topic was removed and re-created) the supervisor will refuse to start and in flight tasks will fail. This operation enables you to recover from this condition.

Note that the supervisor must be running for this endpoint to be available.

Resetting Offsets for a Supervisor

The supervisor must be running for this endpoint to be available.

The POST /druid/indexer/v1/supervisor/<supervisorId>/resetOffsets operation clears stored offsets, causing the supervisor to start reading from the specified offsets. After resetting stored offsets, the supervisor kills and recreates any active tasks pertaining to the specified partitions, so that tasks begin reading from specified offsets. For partitions that are not specified in this operation, the supervisor will resume from the last stored offset.

Use care when using this operation! Resetting offsets for a supervisor may cause Kafka messages to be skipped or read twice, resulting in missing or duplicate data.

Sample request

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 two kafka topics ads_media_foo and ads_media_bar and has the stored offsets: {"ads_media_foo:0": 0, "ads_media_foo:1": 10, "ads_media_bar:0": 20, "ads_media_bar:1": 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_foo|ads_media_bar","partitionOffsetMap":{"ads_media_foo:0": 3, "ads_media_bar:1": 12}}}'

Sample response

Click to show sample response
{
"id": "social_media"
}

Terminating Supervisors

The POST /druid/indexer/v1/supervisor/<supervisorId>/terminate operation terminates a supervisor and causes all associated indexing tasks managed by this supervisor to immediately stop and begin publishing their segments. This supervisor will still exist in the metadata store and its history may be retrieved with the supervisor history API, but will not be listed in the 'get supervisors' API response nor can it's configuration or status report be retrieved. The only way this supervisor can start again is by submitting a functioning supervisor spec to the create API.

Capacity Planning

Kafka indexing tasks run on MiddleManagers and are thus limited by the resources available in the MiddleManager cluster. In particular, you should make sure that you have sufficient worker capacity (configured using the druid.worker.capacity property) to handle the configuration in the supervisor spec. Note that worker capacity is shared across all types of indexing tasks, so you should plan your worker capacity to handle your total indexing load (e.g. batch processing, realtime tasks, merging tasks, etc.). If your workers run out of capacity, Kafka indexing tasks will queue and wait for the next available worker. This may cause queries to return partial results but will not result in data loss (assuming the tasks run before Kafka purges those offsets).

A running task will normally be in one of two states: reading or publishing. A task will remain in reading state for taskDuration, at which point it will transition to publishing state. A task will remain in publishing state for as long as it takes to generate segments, push segments to deep storage, and have them be loaded and served by a Historical process (or until completionTimeout elapses).

The number of reading tasks is controlled by replicas and taskCount. In general, there will be replicas * taskCount reading tasks, the exception being if taskCount > {numKafkaPartitions} in which case {numKafkaPartitions} tasks will be used instead. When taskDuration elapses, these tasks will transition to publishing state and replicas * taskCount new reading tasks will be created. Therefore to allow for reading tasks and publishing tasks to run concurrently, there should be a minimum capacity of:

workerCapacity = 2 * replicas * taskCount

This value is for the ideal situation in which there is at most one set of tasks publishing while another set is reading. In some circumstances, it is possible to have multiple sets of tasks publishing simultaneously. This would happen if the time-to-publish (generate segment, push to deep storage, loaded on Historical) > taskDuration. This is a valid scenario (correctness-wise) but requires additional worker capacity to support. In general, it is a good idea to have taskDuration be large enough that the previous set of tasks finishes publishing before the current set begins.

Supervisor Persistence

When a supervisor spec is submitted via the POST /druid/indexer/v1/supervisor endpoint, it is persisted in the configured metadata database. There can only be a single supervisor per dataSource, and submitting a second spec for the same dataSource will overwrite the previous one.

When an Overlord gains leadership, either by being started or as a result of another Overlord failing, it will spawn a supervisor for each supervisor spec in the metadata database. The supervisor will then discover running Kafka indexing tasks and will attempt to adopt them if they are compatible with the supervisor's configuration. If they are not compatible because they have a different ingestion spec or partition allocation, the tasks will be killed and the supervisor will create a new set of tasks. In this way, the supervisors are persistent across Overlord restarts and fail-overs.

A supervisor is stopped via the POST /druid/indexer/v1/supervisor/<supervisorId>/terminate endpoint. This places a tombstone marker in the database (to prevent the supervisor from being reloaded on a restart) and then gracefully shuts down the currently running supervisor. When a supervisor is shut down in this way, it will instruct its managed tasks to stop reading and begin publishing their segments immediately. The call to the shutdown endpoint will return after all tasks have been signaled to stop but before the tasks finish publishing their segments.

Schema/Configuration Changes

Schema and configuration changes are handled by submitting the new supervisor spec via the same POST /druid/indexer/v1/supervisor endpoint used to initially create the supervisor. The Overlord will initiate a graceful shutdown of the existing supervisor which will cause the tasks being managed by that supervisor to stop reading and begin publishing their segments. A new supervisor will then be started which will create a new set of tasks that will start reading from the offsets where the previous now-publishing tasks left off, but using the updated schema. In this way, configuration changes can be applied without requiring any pause in ingestion.

Deployment notes on Kafka partitions and Druid segments

Druid assigns each Kafka indexing task Kafka partitions. A task writes the events it consumes from Kafka into a single segment for the segment granularity interval until it reaches one of the following: maxRowsPerSegment, maxTotalRows or intermediateHandoffPeriod limit. At this point, the task creates a new partition for this segment granularity to contain subsequent events.

The Kafka Indexing Task also does incremental hand-offs. Therefore segments become available as they are ready and you do not have to wait for all segments until the end of the task duration. When the task reaches one of maxRowsPerSegment, maxTotalRows, or intermediateHandoffPeriod, it hands off all the segments and creates a new new set of segments will be created for further events. This allows the task to run for longer durations without accumulating old segments locally on Middle Manager processes.

The Kafka Indexing Service may still produce some small segments. For example, consider the following scenario:

  • Task duration is 4 hours
  • Segment granularity is set to an HOUR
  • The supervisor was started at 9:10 After 4 hours at 13:10, Druid starts a new set of tasks. The events for the interval 13:00 - 14:00 may be split across existing tasks and the new set of tasks which could result in small segments. To merge them together into new segments of an ideal size (in the range of ~500-700 MB per segment), you can schedule re-indexing tasks, optionally with a different segment granularity.

For more detail, see Segment size optimization. There is also ongoing work to support automatic segment compaction of sharded segments as well as compaction not requiring Hadoop (see here).