diff --git a/docs/api-reference/supervisor-api.md b/docs/api-reference/supervisor-api.md index c5f6c0762709..2e63b69c07f6 100644 --- a/docs/api-reference/supervisor-api.md +++ b/docs/api-reference/supervisor-api.md @@ -36,8 +36,8 @@ 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](../development/extensions-core/kafka-supervisor-operations.md) 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](../development/extensions-core/kafka-ingestion.md) and [Amazon Kinesis ingestion](../development/extensions-core/kinesis-ingestion.md) for supervisor-specific states.| +|`state`|String|Generic state of the supervisor. Available states:`UNHEALTHY_SUPERVISOR`, `UNHEALTHY_TASKS`, `PENDING`, `RUNNING`, `SUSPENDED`, `STOPPING`. See [Supervisor reference](../ingestion/supervisor.md#status-report) for more information.| +|`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](../ingestion/kafka-ingestion.md) and [Amazon Kinesis ingestion](../ingestion/kinesis-ingestion.md) 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.| @@ -1205,9 +1205,7 @@ Host: http://ROUTER_IP:ROUTER_PORT 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: -* [Amazon Kinesis](../development/extensions-core/kinesis-ingestion.md#get-supervisor-status-report) -* [Apache Kafka](../development/extensions-core/kafka-supervisor-operations.md#getting-supervisor-status-report) +For additional information about the status report, see [Supervisor reference](../ingestion/supervisor.md#status-report). #### URL @@ -1309,13 +1307,184 @@ Host: http://ROUTER_IP:ROUTER_PORT ``` +### Get supervisor health + +Retrieves the current health report for a single supervisor. The health of a supervisor is determined by the supervisor's `state` (as returned by the `/status` endpoint) and the `druid.supervisor.*` Overlord configuration thresholds. + +#### URL + +GET /druid/indexer/v1/supervisor/:supervisorId/health + +#### Responses + + + + + +*Supervisor is healthy* + + + + + +*Invalid supervisor ID* + + + + + +*Supervisor is unhealthy* + + + + + +--- + +#### Sample request + +The following example shows how to retrieve the health report for a supervisor with the name `social_media`. + + + + + +```shell +curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/social_media/health" +``` + + + + +```HTTP +GET /druid/indexer/v1/supervisor/social_media/health HTTP/1.1 +Host: http://ROUTER_IP:ROUTER_PORT +``` + + + + +#### Sample response + +
+ Click to show sample response + + ```json + { + "healthy": false + } + ``` +
+ +### Get supervisor ingestion 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 [Row stats](../ingestion/tasks.md#row-stats) for more information. + +#### URL + +GET /druid/indexer/v1/supervisor/:supervisorId/stats + +#### Responses + + + + + +*Successfully retrieved supervisor stats* + + + + + +*Invalid supervisor ID* + + + + + +--- + +#### Sample request + +The following example shows how to retrieve the current ingestion row counters for a supervisor with the name `custom_data`. + + + + + + +```shell +curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/custom_data/stats" +``` + + + + + +```HTTP +GET /druid/indexer/v1/supervisor/custom_data/stats HTTP/1.1 +Host: http://ROUTER_IP:ROUTER_PORT +``` + + + + +#### Sample response + +
+ Click to show sample response + + ```json + { + "0": { + "index_kafka_custom_data_881d621078f6b7c_ccplchbi": { + "movingAverages": { + "buildSegments": { + "5m": { + "processed": 53.401225142603316, + "processedBytes": 5226.400757148808, + "unparseable": 0.0, + "thrownAway": 0.0, + "processedWithError": 0.0 + }, + "15m": { + "processed": 56.92994990102502, + "processedBytes": 5571.772059828217, + "unparseable": 0.0, + "thrownAway": 0.0, + "processedWithError": 0.0 + }, + "1m": { + "processed": 37.134921285556636, + "processedBytes": 3634.2766230628677, + "unparseable": 0.0, + "thrownAway": 0.0, + "processedWithError": 0.0 + } + } + }, + "totals": { + "buildSegments": { + "processed": 665, + "processedBytes": 65079, + "processedWithError": 0, + "thrownAway": 0, + "unparseable": 0 + } + } + } + } + } + ``` +
+ ## 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. +Retrieves an audit history of specs for all supervisors. #### URL @@ -1325,7 +1494,7 @@ Retrieve an audit history of specs for all supervisors. - + *Successfully retrieved audit history* @@ -1339,7 +1508,7 @@ Retrieve an audit history of specs for all supervisors. - + ```shell @@ -1347,7 +1516,7 @@ curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/history" ``` - + ```HTTP @@ -1686,13 +1855,13 @@ Retrieves an audit history of specs for a single supervisor. - + *Successfully retrieved supervisor audit history* - + *Invalid supervisor ID* @@ -1716,7 +1885,7 @@ curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/wikipedia_stream/ ``` - + ```HTTP @@ -2044,11 +2213,22 @@ Host: http://ROUTER_IP:ROUTER_PORT ### Create or update a supervisor -Creates a new supervisor or updates an existing one for the same datasource with a new schema and configuration. +Creates a new supervisor spec or updates an existing one with new configuration and schema information. When updating a supervisor spec, the datasource must remain the same as the previous supervisor. + +You can define a supervisor spec for [Apache Kafka](../ingestion/kafka-ingestion.md) or [Amazon Kinesis](../ingestion/kinesis-ingestion.md) streaming ingestion methods. -You can define a supervisor spec for [Apache Kafka](../development/extensions-core/kafka-ingestion.md#define-a-supervisor-spec) or [Amazon Kinesis](../development/extensions-core/kinesis-ingestion.md#supervisor-spec) streaming ingestion methods. Once created, the supervisor persists in the metadata database. +The following table lists the properties of a supervisor spec: -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. +|Property|Type|Description|Required| +|--------|----|-----------|--------| +|`type`|String|The supervisor type. One of`kafka` or `kinesis`.|Yes| +|`spec`|Object|The container object for the supervisor configuration.|Yes| +|`ioConfig`|Object|The I/O configuration object to define the connection and I/O-related settings for the supervisor and indexing task.|Yes| +|`dataSchema`|Object|The schema for the indexing task to use during ingestion. See [`dataSchema`](../ingestion/ingestion-spec.md#dataschema) for more information.|Yes| +|`tuningConfig`|Object|The tuning configuration object to define performance-related settings for the supervisor and indexing tasks.|No| + +When you call this endpoint on an existing supervisor, 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. +This way, you can apply configuration changes without a pause in ingestion. #### URL @@ -2058,13 +2238,13 @@ When you call this endpoint on an existing supervisor for the same datasource, t - + *Successfully created a new supervisor or updated an existing supervisor* - + *Request body content type is not in JSON format* @@ -2080,8 +2260,7 @@ The following example uses JSON input format to create a supervisor spec for Kaf - - + ```shell curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor" \ @@ -2139,8 +2318,8 @@ curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor" \ ``` - + ```HTTP POST /druid/indexer/v1/supervisor HTTP/1.1 @@ -2218,6 +2397,7 @@ Content-Length: 1359 ### 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. +Indexing tasks remain suspended until you [resume the supervisor](#resume-a-supervisor). #### URL POST /druid/indexer/v1/supervisor/:supervisorId/suspend @@ -2226,19 +2406,19 @@ Suspends a single running supervisor. Returns the updated supervisor spec, where - + *Successfully shut down supervisor* - + *Supervisor already suspended* - + *Invalid supervisor ID* @@ -2254,7 +2434,7 @@ The following example shows how to suspend a running supervisor with the name `s - + ```shell @@ -2262,7 +2442,7 @@ curl --request POST "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/so ``` - + ```HTTP @@ -2592,7 +2772,7 @@ Suspends all supervisors. Note that this endpoint returns an HTTP `200 Success` - + *Successfully suspended all supervisors* @@ -2606,7 +2786,7 @@ Suspends all supervisors. Note that this endpoint returns an HTTP `200 Success` - + ```shell @@ -2614,7 +2794,7 @@ curl --request POST "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/su ``` - + ```HTTP @@ -2649,19 +2829,19 @@ Resumes indexing tasks for a supervisor. Returns an updated supervisor spec with - + *Successfully resumed supervisor* - + *Supervisor already running* - + *Invalid supervisor ID* @@ -2677,7 +2857,7 @@ The following example resumes a previously suspended supervisor with name `socia - + ```shell @@ -2685,7 +2865,7 @@ curl --request POST "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/so ``` - + ```HTTP @@ -3016,7 +3196,7 @@ Resumes all supervisors. Note that this endpoint returns an HTTP `200 Success` c - + *Successfully resumed all supervisors* @@ -3030,7 +3210,7 @@ Resumes all supervisors. Note that this endpoint returns an HTTP `200 Success` c - + ```shell @@ -3038,7 +3218,7 @@ curl --request POST "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/re ``` - + ```HTTP @@ -3063,10 +3243,16 @@ Host: http://ROUTER_IP:ROUTER_PORT ### Reset a supervisor -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. +The supervisor must be running for this endpoint to be available. + +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 restarts from the earliest or latest available position, depending on the platform: offsets in Kafka or sequence numbers in Kinesis. +After clearing all stored offsets in Kafka or sequence numbers in Kinesis, the supervisor kills and recreates active tasks, +so that tasks begin reading 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. +The indexing service keeps track of the latest persisted offsets in Kafka or sequence numbers in Kinesis to provide exactly-once ingestion guarantees across tasks. Subsequent tasks must start reading from where the previous task completed for Druid to accept the generated segments. If the messages at the expected starting offsets in Kafka or sequence numbers in Kinesis are no longer available, the supervisor refuses to start and in-flight tasks fail. Possible causes for missing messages include the message retention period elapsing or the topic being removed and re-created. Use the `reset` endpoint to recover from this condition. + #### URL POST /druid/indexer/v1/supervisor/:supervisorId/reset @@ -3075,13 +3261,13 @@ Use this endpoint to recover from a stopped state due to missing offsets in Kafk - + *Successfully reset supervisor* - + *Invalid supervisor ID* @@ -3097,7 +3283,7 @@ The following example shows how to reset a supervisor with the name `social_medi - + ```shell @@ -3105,7 +3291,7 @@ curl --request POST "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/so ``` - + ```HTTP @@ -3128,13 +3314,19 @@ Host: http://ROUTER_IP:ROUTER_PORT ``` -### Reset Offsets for a supervisor +### Reset offsets for a supervisor + +The supervisor must be running for this endpoint to be available. + +Resets the specified offsets for partitions without resetting the entire set. + +This endpoint clears only the specified offsets in Kafka or sequence numbers in Kinesis, prompting the supervisor to resume reading data from the specified offsets. +If there are no stored offsets, the specified offsets are set in the metadata store. -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. +After resetting stored offsets, the supervisor kills and recreates any active tasks pertaining to the specified partitions, +so that tasks begin reading specified offsets. For partitions that are not specified in this operation, the supervisor resumes from the last stored offset. -Use this endpoint to selectively reset offsets for partitions without resetting the entire set. +Use this endpoint with caution. It can cause skipped messages, leading to data loss or duplicate data. #### URL @@ -3180,8 +3372,7 @@ The following table defines the fields within the `partitions` object in the res #### 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 a kafka topic `ads_media_stream` and has the stored offsets: `{"0": 0, "1": 10, "2": 20, "3": 40}`. +The following example shows how to reset offsets for a Kafka supervisor with the name `social_media`. For example, the supervisor is reading from a Kafka topic `ads_media_stream` and has the stored offsets: `{"0": 0, "1": 10, "2": 20, "3": 40}`. @@ -3216,8 +3407,8 @@ Content-Type: application/json } ``` -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}`. +The example operation resets offsets only for partitions `0` and `2` to 100 and 650 respectively. After a successful reset, +when the supervisor's tasks restart, they resume reading from `{"0": 100, "1": 10, "2": 650, "3": 40}`. diff --git a/docs/assets/supervisor-info-dialog.png b/docs/assets/supervisor-info-dialog.png new file mode 100644 index 000000000000..3be424a413ef Binary files /dev/null and b/docs/assets/supervisor-info-dialog.png differ diff --git a/docs/assets/supervisor-view.png b/docs/assets/supervisor-view.png new file mode 100644 index 000000000000..e3100cdd3ba6 Binary files /dev/null and b/docs/assets/supervisor-view.png differ diff --git a/docs/configuration/extensions.md b/docs/configuration/extensions.md index 0b09fb75ce63..c9035319287b 100644 --- a/docs/configuration/extensions.md +++ b/docs/configuration/extensions.md @@ -45,8 +45,8 @@ Core extensions are maintained by Druid committers. |druid-hdfs-storage|HDFS deep storage.|[link](../development/extensions-core/hdfs.md)| |druid-histogram|Approximate histograms and quantiles aggregator. Deprecated, please use the [DataSketches quantiles aggregator](../development/extensions-core/datasketches-quantiles.md) from the `druid-datasketches` extension instead.|[link](../development/extensions-core/approximate-histograms.md)| |druid-kafka-extraction-namespace|Apache Kafka-based namespaced lookup. Requires namespace lookup extension.|[link](../development/extensions-core/kafka-extraction-namespace.md)| -|druid-kafka-indexing-service|Supervised exactly-once Apache Kafka ingestion for the indexing service.|[link](../development/extensions-core/kafka-ingestion.md)| -|druid-kinesis-indexing-service|Supervised exactly-once Kinesis ingestion for the indexing service.|[link](../development/extensions-core/kinesis-ingestion.md)| +|druid-kafka-indexing-service|Supervised exactly-once Apache Kafka ingestion for the indexing service.|[link](../ingestion/kafka-ingestion.md)| +|druid-kinesis-indexing-service|Supervised exactly-once Kinesis ingestion for the indexing service.|[link](../ingestion/kinesis-ingestion.md)| |druid-kerberos|Kerberos authentication for druid processes.|[link](../development/extensions-core/druid-kerberos.md)| |druid-lookups-cached-global|A module for [lookups](../querying/lookups.md) providing a jvm-global eager caching for lookups. It provides JDBC and URI implementations for fetching lookup data.|[link](../development/extensions-core/lookups-cached-global.md)| |druid-lookups-cached-single| Per lookup caching module to support the use cases where a lookup need to be isolated from the global pool of lookups |[link](../development/extensions-core/druid-lookups.md)| diff --git a/docs/configuration/index.md b/docs/configuration/index.md index 2e4d1f99722a..bdb0766f2de2 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -510,13 +510,13 @@ These properties specify the JDBC connection and other configuration around the |Property|Description|Default| |--------|-----------|-------| -|`druid.metadata.storage.type`|The type of metadata storage to use. Choose from `mysql`, `postgresql`, or `derby`.|`derby`| +|`druid.metadata.storage.type`|The type of metadata storage to use. One of `mysql`, `postgresql`, or `derby`.|`derby`| |`druid.metadata.storage.connector.connectURI`|The JDBC URI for the database to connect to|none| |`druid.metadata.storage.connector.user`|The username to connect with.|none| |`druid.metadata.storage.connector.password`|The [Password Provider](../operations/password-provider.md) or String password used to connect with.|none| |`druid.metadata.storage.connector.createTables`|If Druid requires a table and it doesn't exist, create it?|true| |`druid.metadata.storage.tables.base`|The base name for tables.|`druid`| -|`druid.metadata.storage.tables.dataSource`|The table to use to look for datasources created by [Kafka Indexing Service](../development/extensions-core/kafka-ingestion.md).|`druid_dataSource`| +|`druid.metadata.storage.tables.dataSource`|The table to use to look for datasources created by [Kafka Indexing Service](../ingestion/kafka-ingestion.md).|`druid_dataSource`| |`druid.metadata.storage.tables.pendingSegments`|The table to use to look for pending segments.|`druid_pendingSegments`| |`druid.metadata.storage.tables.segments`|The table to use to look for segments.|`druid_segments`| |`druid.metadata.storage.tables.rules`|The table to use to look for segment load/drop rules.|`druid_rules`| @@ -533,7 +533,7 @@ The configurations concern how to push and pull [Segments](../design/segments.md |Property|Description|Default| |--------|-----------|-------| -|`druid.storage.type`|The type of deep storage to use. Choose from `local`, `noop`, `s3`, `hdfs`, `c*`.|local| +|`druid.storage.type`|The type of deep storage to use. One of `local`, `noop`, `s3`, `hdfs`, `c*`.|local| #### Local deep storage @@ -1101,7 +1101,7 @@ These Overlord static configurations can be defined in the `overlord/runtime.pro |Property|Description|Default| |--------|-----------|-------| |`druid.indexer.runner.type`|Indicates whether tasks should be run locally using `local` or in a distributed environment using `remote`. The recommended option is `httpRemote`, which is similar to `remote` but uses HTTP to interact with Middle Managers instead of ZooKeeper.|`httpRemote`| -|`druid.indexer.storage.type`|Indicates whether incoming tasks should be stored locally (in heap) or in metadata storage. Choose from `local` or `metadata`. `local` is mainly for internal testing while `metadata` is recommended in production because storing incoming tasks in metadata storage allows for tasks to be resumed if the Overlord should fail.|`local`| +|`druid.indexer.storage.type`|Indicates whether incoming tasks should be stored locally (in heap) or in metadata storage. One of `local` or `metadata`. `local` is mainly for internal testing while `metadata` is recommended in production because storing incoming tasks in metadata storage allows for tasks to be resumed if the Overlord should fail.|`local`| |`druid.indexer.storage.recentlyFinishedThreshold`|Duration of time to store task results. Default is 24 hours. If you have hundreds of tasks running in a day, consider increasing this threshold.|`PT24H`| |`druid.indexer.tasklock.forceTimeChunkLock`|_**Setting this to false is still experimental**_
If set, all tasks are enforced to use time chunk lock. If not set, each task automatically chooses a lock type to use. This configuration can be overwritten by setting `forceTimeChunkLock` in the [task context](../ingestion/tasks.md#context). See [Task Locking & Priority](../ingestion/tasks.md#context) for more details about locking in tasks.|true| |`druid.indexer.tasklock.batchSegmentAllocation`| If set to true, Druid performs segment allocate actions in batches to improve throughput and reduce the average `task/action/run/time`. See [batching `segmentAllocate` actions](../ingestion/tasks.md#batching-segmentallocate-actions) for details.|true| @@ -1133,7 +1133,7 @@ If autoscaling is enabled, you can set these additional configs: |Property|Description|Default| |--------|-----------|-------| -|`druid.indexer.autoscale.strategy`|Sets the strategy to run when autoscaling is required. Choose from `noop`, `ec2` or `gce`.|`noop`| +|`druid.indexer.autoscale.strategy`|Sets the strategy to run when autoscaling is required. One of `noop`, `ec2` or `gce`.|`noop`| |`druid.indexer.autoscale.doAutoscale`|If set to true, autoscaling will be enabled.|false| |`druid.indexer.autoscale.provisionPeriod`|How often to check whether or not new MiddleManagers should be added.|`PT1M`| |`druid.indexer.autoscale.terminatePeriod`|How often to check when MiddleManagers should be removed.|`PT5M`| @@ -1159,7 +1159,7 @@ If autoscaling is enabled, you can set these additional configs: |`druid.supervisor.idleConfig.enabled`|If `true`, supervisor can become idle if there is no data on input stream/topic for some time.|false| |`druid.supervisor.idleConfig.inactiveAfterMillis`|Supervisor is marked as idle if all existing data has been read from input topic and no new data has been published for `inactiveAfterMillis` milliseconds.|`600_000`| -The `druid.supervisor.idleConfig.*` specified in the Overlord runtime properties defines the default behavior for the entire cluster. See [Idle Configuration in Kafka Supervisor IOConfig](../development/extensions-core/kafka-supervisor-reference.md#supervisor-io-configuration) to override it for an individual supervisor. +The `druid.supervisor.idleConfig.*` specification in the Overlord runtime properties defines the default behavior for the entire cluster. See [Idle Configuration in Kafka Supervisor IOConfig](../ingestion/kinesis-ingestion.md#io-configuration) to override it for an individual supervisor. #### Overlord dynamic configuration @@ -1483,7 +1483,7 @@ Additional Peon configs include: |Property|Description|Default| |--------|-----------|-------| -|`druid.peon.mode`|Choose from `local` and `remote`. Setting this property to `local` means you intend to run the Peon as a standalone process which is not recommended.|`remote`| +|`druid.peon.mode`|One of `local` or `remote`. Setting this property to `local` means you intend to run the Peon as a standalone process which is not recommended.|`remote`| |`druid.indexer.task.baseDir`|Base temporary working directory.|`System.getProperty("java.io.tmpdir")`| |`druid.indexer.task.baseTaskDir`|Base temporary working directory for tasks.|`${druid.indexer.task.baseDir}/persistent/task`| |`druid.indexer.task.batchProcessingMode`| Batch ingestion tasks have three operating modes to control construction and tracking for intermediary segments: `OPEN_SEGMENTS`, `CLOSED_SEGMENTS`, and `CLOSED_SEGMENT_SINKS`. `OPEN_SEGMENTS` uses the streaming ingestion code path and performs a `mmap` on intermediary segments to build a timeline to make these segments available to realtime queries. Batch ingestion doesn't require intermediary segments, so the default mode, `CLOSED_SEGMENTS`, eliminates `mmap` of intermediary segments. `CLOSED_SEGMENTS` mode still tracks the entire set of segments in heap. The `CLOSED_SEGMENTS_SINKS` mode is the most aggressive configuration and should have the smallest memory footprint. It eliminates in-memory tracking and `mmap` of intermediary segments produced during segment creation. `CLOSED_SEGMENTS_SINKS` mode isn't as well tested as other modes so is currently considered experimental. You can use `OPEN_SEGMENTS` mode if problems occur with the 2 newer modes. |`CLOSED_SEGMENTS`| diff --git a/docs/design/storage.md b/docs/design/storage.md index e981b4a82db0..73e0b85fa9a9 100644 --- a/docs/design/storage.md +++ b/docs/design/storage.md @@ -114,14 +114,14 @@ Druid has an architectural separation between ingestion and querying, as describ On the ingestion side, Druid's primary [ingestion methods](../ingestion/index.md#ingestion-methods) are all pull-based and offer transactional guarantees. This means that you are guaranteed that ingestion using these methods will publish in an all-or-nothing manner: -- Supervised "seekable-stream" ingestion methods like [Kafka](../development/extensions-core/kafka-ingestion.md) and [Kinesis](../development/extensions-core/kinesis-ingestion.md). With these methods, Druid commits stream offsets to its [metadata store](metadata-storage.md) alongside segment metadata, in the same transaction. Note that ingestion of data that has not yet been published can be rolled back if ingestion tasks fail. In this case, partially-ingested data is +- Supervised "seekable-stream" ingestion methods like [Kafka](../ingestion/kafka-ingestion.md) and [Kinesis](../ingestion/kinesis-ingestion.md). With these methods, Druid commits stream offsets to its [metadata store](metadata-storage.md) alongside segment metadata, in the same transaction. Note that ingestion of data that has not yet been published can be rolled back if ingestion tasks fail. In this case, partially-ingested data is discarded, and Druid will resume ingestion from the last committed set of stream offsets. This ensures exactly-once publishing behavior. - [Hadoop-based batch ingestion](../ingestion/hadoop.md). Each task publishes all segment metadata in a single transaction. - [Native batch ingestion](../ingestion/native-batch.md). In parallel mode, the supervisor task publishes all segment metadata in a single transaction after the subtasks are finished. In simple (single-task) mode, the single task publishes all segment metadata in a single transaction after it is complete. Additionally, some ingestion methods offer an _idempotency_ guarantee. This means that repeated executions of the same ingestion will not cause duplicate data to be ingested: -- Supervised "seekable-stream" ingestion methods like [Kafka](../development/extensions-core/kafka-ingestion.md) and [Kinesis](../development/extensions-core/kinesis-ingestion.md) are idempotent due to the fact that stream offsets and segment metadata are stored together and updated in lock-step. +- Supervised "seekable-stream" ingestion methods like [Kafka](../ingestion/kafka-ingestion.md) and [Kinesis](../ingestion/kinesis-ingestion.md) are idempotent due to the fact that stream offsets and segment metadata are stored together and updated in lock-step. - [Hadoop-based batch ingestion](../ingestion/hadoop.md) is idempotent unless one of your input sources is the same Druid datasource that you are ingesting into. In this case, running the same task twice is non-idempotent, because you are adding to existing data instead of overwriting it. - [Native batch ingestion](../ingestion/native-batch.md) is idempotent unless [`appendToExisting`](../ingestion/native-batch.md) is true, or one of your input sources is the same Druid datasource that you are ingesting into. In either of these two cases, running the same task twice is non-idempotent, because you are adding to existing data instead of overwriting it. diff --git a/docs/development/extensions-core/kafka-ingestion.md b/docs/development/extensions-core/kafka-ingestion.md deleted file mode 100644 index 329967747bfa..000000000000 --- a/docs/development/extensions-core/kafka-ingestion.md +++ /dev/null @@ -1,301 +0,0 @@ ---- -id: kafka-ingestion -title: "Apache Kafka ingestion" -sidebar_label: "Apache Kafka ingestion" -description: "Overview of the Kafka indexing service for Druid. Includes example supervisor specs to help you get started." ---- - - - -When you enable the Kafka indexing service, you can configure supervisors on the Overlord to manage the creation and lifetime of Kafka indexing tasks. - -Kafka indexing tasks read events using Kafka's own partition and offset mechanism to guarantee exactly-once ingestion. The supervisor oversees the state of the indexing tasks to: - - coordinate handoffs - - manage failures - - ensure that scalability and replication requirements are maintained. - - -This topic covers how to submit a supervisor spec to ingest event data, also known as message data, from Kafka. See the following for more information: -- For a reference of Kafka supervisor spec configuration options, see the [Kafka supervisor reference](./kafka-supervisor-reference.md). -- For operations reference information to help run and maintain Apache Kafka supervisors, see [Kafka supervisor operations](./kafka-supervisor-operations.md). -- For a walk-through, see the [Loading from Apache Kafka](../../tutorials/tutorial-kafka.md) tutorial. - -## Kafka support - -The Kafka indexing service supports transactional topics introduced in Kafka 0.11.x by default. The consumer for Kafka indexing service is incompatible with older Kafka brokers. If you are using an older version, refer to the [Kafka upgrade guide](https://kafka.apache.org/documentation/#upgrade). - -Additionally, you can set `isolation.level` to `read_uncommitted` in `consumerProperties` if either: -- You don't need Druid to consume transactional topics. -- You need Druid to consume older versions of Kafka. Make sure offsets are sequential, since there is no offset gap check in Druid anymore. - -If your Kafka cluster enables consumer-group based ACLs, you can set `group.id` in `consumerProperties` to override the default auto generated group id. - -## Load the Kafka indexing service - -To use the Kafka indexing service, load the `druid-kafka-indexing-service` extension on both the Overlord and the MiddleManagers. See [Loading extensions](../../configuration/extensions.md) for instructions on how to configure extensions. - -## Define a supervisor spec - -Similar to the ingestion spec for batch ingestion, the supervisor spec configures the data ingestion for Kafka streaming ingestion. A supervisor spec has the following sections: -- `dataSchema` to specify the Druid datasource name, primary timestamp, dimensions, metrics, transforms, and any necessary filters. -- `ioConfig` to configure Kafka connection settings and configure how Druid parses the data. Kafka-specific connection details go in the `consumerProperties`. The `ioConfig` is also where you define the input format (`inputFormat`) of your Kafka data. For supported formats for Kafka and information on how to configure the input format, see [Data formats](../../ingestion/data-formats.md). -- `tuningConfig` to control various tuning parameters specific to each ingestion method. -For a full description of all the fields and parameters in a Kafka supervisor spec, see the [Kafka supervisor reference](./kafka-supervisor-reference.md). - - -The following sections contain examples to help you get started with supervisor specs. - -### JSON input format supervisor spec example - -The following example demonstrates a supervisor spec for Kafka that uses the `JSON` input format. In this case Druid parses the event contents in JSON format: - -```json -{ - "type": "kafka", - "spec": { - "dataSchema": { - "dataSource": "metrics-kafka", - "timestampSpec": { - "column": "timestamp", - "format": "auto" - }, - "dimensionsSpec": { - "dimensions": [], - "dimensionExclusions": [ - "timestamp", - "value" - ] - }, - "metricsSpec": [ - { - "name": "count", - "type": "count" - }, - { - "name": "value_sum", - "fieldName": "value", - "type": "doubleSum" - }, - { - "name": "value_min", - "fieldName": "value", - "type": "doubleMin" - }, - { - "name": "value_max", - "fieldName": "value", - "type": "doubleMax" - } - ], - "granularitySpec": { - "type": "uniform", - "segmentGranularity": "HOUR", - "queryGranularity": "NONE" - } - }, - "ioConfig": { - "topic": "metrics", - "inputFormat": { - "type": "json" - }, - "consumerProperties": { - "bootstrap.servers": "localhost:9092" - }, - "taskCount": 1, - "replicas": 1, - "taskDuration": "PT1H" - }, - "tuningConfig": { - "type": "kafka", - "maxRowsPerSegment": 5000000 - } - } -} -``` - -### Kafka input format supervisor spec example - -If you want to parse the Kafka metadata fields in addition to the Kafka payload value contents, you can use the `kafka` input format. - -The `kafka` input format wraps around the payload parsing input format and augments the data it outputs with the Kafka event timestamp, -the Kafka topic name, the Kafka event headers, and the key field that itself can be parsed using any available InputFormat. - -For example, consider the following structure for a Kafka message that represents a fictitious wiki edit in a development environment: - -- **Kafka timestamp**: `1680795276351` -- **Kafka topic**: `wiki-edits` -- **Kafka headers**: - - `env=development` - - `zone=z1` -- **Kafka key**: `wiki-edit` -- **Kafka payload value**: `{"channel":"#sv.wikipedia","timestamp":"2016-06-27T00:00:11.080Z","page":"Salo Toraut","delta":31,"namespace":"Main"}` - -Using `{ "type": "json" }` as the input format would only parse the payload value. -To parse the Kafka metadata in addition to the payload, use the `kafka` input format. - -You would configure it as follows: - -- `valueFormat`: Define how to parse the payload value. Set this to the payload parsing input format (`{ "type": "json" }`). -- `timestampColumnName`: Supply a custom name for the Kafka timestamp in the Druid schema to avoid conflicts with columns from the payload. The default is `kafka.timestamp`. -- `topicColumnName`: Supply a custom name for the Kafka topic in the Druid schema to avoid conflicts with columns from the payload. The default is `kafka.topic`. This field is useful when ingesting data from multiple topics into same datasource. -- `headerFormat`: The default value `string` decodes strings in UTF-8 encoding from the Kafka header. - Other supported encoding formats include the following: - - `ISO-8859-1`: ISO Latin Alphabet No. 1, that is, ISO-LATIN-1. - - `US-ASCII`: Seven-bit ASCII. Also known as ISO646-US. The Basic Latin block of the Unicode character set. - - `UTF-16`: Sixteen-bit UCS Transformation Format, byte order identified by an optional byte-order mark. - - `UTF-16BE`: Sixteen-bit UCS Transformation Format, big-endian byte order. - - `UTF-16LE`: Sixteen-bit UCS Transformation Format, little-endian byte order. -- `headerColumnPrefix`: Supply a prefix to the Kafka headers to avoid any conflicts with columns from the payload. The default is `kafka.header.`. - Considering the header from the example, Druid maps the headers to the following columns: `kafka.header.env`, `kafka.header.zone`. -- `keyFormat`: Supply an input format to parse the key. Only the first value will be used. - If, as in the example, your key values are simple strings, then you can use the `tsv` format to parse them. - ``` - { - "type": "tsv", - "findColumnsFromHeader": false, - "columns": ["x"] - } - ``` - Note that for `tsv`,`csv`, and `regex` formats, you need to provide a `columns` array to make a valid input format. Only the first one is used, and its name will be ignored in favor of `keyColumnName`. -- `keyColumnName`: Supply the name for the Kafka key column to avoid conflicts with columns from the payload. The default is `kafka.key`. - -Putting it together, the following input format (that uses the default values for `timestampColumnName`, `topicColumnName`, `headerColumnPrefix`, and `keyColumnName`) - -```json -{ - "type": "kafka", - "valueFormat": { - "type": "json" - }, - "headerFormat": { - "type": "string" - }, - "keyFormat": { - "type": "tsv", - "findColumnsFromHeader": false, - "columns": ["x"] - } -} -``` - -would parse the example message as follows: - -```json -{ - "channel": "#sv.wikipedia", - "timestamp": "2016-06-27T00:00:11.080Z", - "page": "Salo Toraut", - "delta": 31, - "namespace": "Main", - "kafka.timestamp": 1680795276351, - "kafka.topic": "wiki-edits", - "kafka.header.env": "development", - "kafka.header.zone": "z1", - "kafka.key": "wiki-edit" -} -``` - -For more information on data formats, see [Data formats](../../ingestion/data-formats.md). - -Finally, add these Kafka metadata columns to the `dimensionsSpec` or set your `dimensionsSpec` to auto-detect columns. - -The following supervisor spec demonstrates how to ingest the Kafka header, key, timestamp, and topic into Druid dimensions: - -``` -{ - "type": "kafka", - "spec": { - "ioConfig": { - "type": "kafka", - "consumerProperties": { - "bootstrap.servers": "localhost:9092" - }, - "topic": "wiki-edits", - "inputFormat": { - "type": "kafka", - "valueFormat": { - "type": "json" - }, - "headerFormat": { - "type": "string" - }, - "keyFormat": { - "type": "tsv", - "findColumnsFromHeader": false, - "columns": ["x"] - } - }, - "useEarliestOffset": true - }, - "dataSchema": { - "dataSource": "wikiticker", - "timestampSpec": { - "column": "timestamp", - "format": "posix" - }, - "dimensionsSpec": "dimensionsSpec": { - "useSchemaDiscovery": true, - "includeAllDimensions": true - }, - "granularitySpec": { - "queryGranularity": "none", - "rollup": false, - "segmentGranularity": "day" - } - }, - "tuningConfig": { - "type": "kafka" - } - } -} -``` - -After Druid ingests the data, you can query the Kafka metadata columns as follows: - -```sql -SELECT - "kafka.header.env", - "kafka.key", - "kafka.timestamp", - "kafka.topic" -FROM "wikiticker" -``` - -This query returns: - -| `kafka.header.env` | `kafka.key` | `kafka.timestamp` | `kafka.topic` | -|--------------------|-----------|---------------|---------------| -| `development` | `wiki-edit` | `1680795276351` | `wiki-edits` | - -For more information, see [`kafka` data format](../../ingestion/data-formats.md#kafka). - -## Submit a supervisor spec - -Druid starts a supervisor for a dataSource when you submit a supervisor spec. You can use the data loader in the web console or you can submit a supervisor spec to the following endpoint: - -`http://:/druid/indexer/v1/supervisor` - -For example: - -``` -curl -X POST -H 'Content-Type: application/json' -d @supervisor-spec.json http://localhost:8090/druid/indexer/v1/supervisor -``` - -Where the file `supervisor-spec.json` contains your Kafka supervisor spec file. diff --git a/docs/development/extensions-core/kafka-supervisor-operations.md b/docs/development/extensions-core/kafka-supervisor-operations.md deleted file mode 100644 index b76a80f8cb9b..000000000000 --- a/docs/development/extensions-core/kafka-supervisor-operations.md +++ /dev/null @@ -1,287 +0,0 @@ ---- -id: kafka-supervisor-operations -title: "Apache Kafka supervisor operations reference" -sidebar_label: "Apache Kafka operations" -description: "Reference topic for running and maintaining Apache Kafka supervisors" ---- - -import Tabs from '@theme/Tabs'; -import TabItem from '@theme/TabItem'; - - -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](../../api-reference/supervisor-api.md). - -## Getting Supervisor Status Report - -`GET /druid/indexer/v1/supervisor//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 State|Corresponding State|Description| -|--------------|-------------------|-----------| -|UNHEALTHY_SUPERVISOR|UNHEALTHY_SUPERVISOR|The supervisor has encountered errors on the past `druid.supervisor.unhealthinessThreshold` iterations| -|UNHEALTHY_TASKS|UNHEALTHY_TASKS|The last `druid.supervisor.taskUnhealthinessThreshold` tasks have all failed| -|UNABLE_TO_CONNECT_TO_STREAM|UNHEALTHY_SUPERVISOR|The supervisor is encountering connectivity issues with Kafka and has not successfully connected in the past| -|LOST_CONTACT_WITH_STREAM|UNHEALTHY_SUPERVISOR|The supervisor is encountering connectivity issues with Kafka but has successfully connected in the past| -|PENDING (first iteration only)|PENDING|The supervisor has been initialized and hasn't started connecting to the stream| -|CONNECTING_TO_STREAM (first iteration only)|RUNNING|The supervisor is trying to connect to the stream and update partition data| -|DISCOVERING_INITIAL_TASKS (first iteration only)|RUNNING|The supervisor is discovering already-running tasks| -|CREATING_TASKS (first iteration only)|RUNNING|The supervisor is creating tasks and discovering state| -|RUNNING|RUNNING|The supervisor has started tasks and is waiting for taskDuration to elapse| -|IDLE|IDLE|The supervisor is not creating tasks since the input stream has not received any new data and all the existing data is read.| -|SUSPENDED|SUSPENDED|The supervisor has been suspended| -|STOPPING|STOPPING|The 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//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](../../ingestion/tasks.md#row-stats) for more information. - -## Supervisor Health Check - -`GET /druid/indexer/v1/supervisor//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//suspend` and `POST /druid/indexer/v1/supervisor//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//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//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}`. - - - - - - -```shell -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}}}' -``` - - - - -```HTTP -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_foo|ads_media_bar", - "partitionOffsetMap": { - "ads_media_foo:0": 3, - "ads_media_bar:1": 12 - } - } -} -``` - -The above operation will reset offsets for `ads_media_foo` partition 0 and `ads_media_bar` partition 1 to offsets 3 and 12 respectively. After a successful reset, -when the supervisor's tasks restart, they will resume reading from `{"ads_media_foo:0": 3, "ads_media_foo:1": 10, "ads_media_bar:0": 20, "ads_media_bar:1": 12}`. - - - - -#### Sample response - -
- Click to show sample response - - ```json -{ - "id": "social_media" -} - ``` -
- -## Terminating Supervisors - -The `POST /druid/indexer/v1/supervisor//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//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](../../operations/segment-optimization.md). -There is also ongoing work to support automatic segment compaction of sharded segments as well as compaction not requiring -Hadoop (see [here](https://github.com/apache/druid/pull/5102)). diff --git a/docs/development/extensions-core/kafka-supervisor-reference.md b/docs/development/extensions-core/kafka-supervisor-reference.md deleted file mode 100644 index 037ffe752eb7..000000000000 --- a/docs/development/extensions-core/kafka-supervisor-reference.md +++ /dev/null @@ -1,262 +0,0 @@ ---- -id: kafka-supervisor-reference -title: "Apache Kafka supervisor reference" -sidebar_label: "Apache Kafka supervisor" -description: "Reference topic for Apache Kafka supervisors" ---- - - - -This topic contains configuration reference information for the Apache Kafka supervisor for Apache Druid. - -The following table outlines the high-level configuration options: - -|Property|Type|Description|Required| -|--------|----|-----------|--------| -|`type`|String|The supervisor type. For Kafka streaming, set to `kafka`.|Yes| -|`spec`|Object|The container object for the supervisor configuration.|Yes| -|`ioConfig`|Object|The I/O configuration object to define the Kafka connection and I/O-related settings for the supervisor and indexing task. See [Supervisor I/O configuration](#supervisor-io-configuration).|Yes| -|`dataSchema`|Object|The schema for the Kafka indexing task to use during ingestion.|Yes| -|`tuningConfig`|Object|The tuning configuration object to define performance-related settings for the supervisor and indexing tasks. See [Supervisor tuning configuration](#supervisor-tuning-configuration).|No| - -## Supervisor I/O configuration - -The following table outlines the configuration options for `ioConfig`: - -|Property|Type|Description|Required|Default| -|--------|----|-----------|--------|-------| -|`topic`|String|The Kafka topic to read from. Must be a specific topic. Druid does not support topic patterns.|Yes|| -|`inputFormat`|Object|The input format to define input data parsing. See [Specifying data format](#specifying-data-format) for details about specifying the input format.|Yes|| -|`consumerProperties`|String, Object|A map of properties to pass to the Kafka consumer. See [Consumer properties](#consumer-properties).|Yes|| -|`pollTimeout`|Long|The length of time to wait for the Kafka consumer to poll records, in milliseconds.|No|100| -|`replicas`|Integer|The number of replica sets, where 1 is a single set of tasks (no replication). Druid always assigns replicate tasks to different workers to provide resiliency against process failure.|No|1| -|`taskCount`|Integer|The maximum number of reading tasks in a replica set. The maximum number of reading tasks equals `taskCount * replicas`. The total number of tasks, reading and publishing, is greater than this count. See [Capacity planning](./kafka-supervisor-operations.md#capacity-planning) for more details. When `taskCount > {numKafkaPartitions}`, the actual number of reading tasks is less than the `taskCount` value.|No|1| -|`taskDuration`|ISO 8601 period|The length of time before tasks stop reading and begin publishing segments.|No|PT1H| -|`startDelay`|ISO 8601 period|The period to wait before the supervisor starts managing tasks.|No|PT5S| -|`period`|ISO 8601 period|Determines how often the supervisor executes its management logic. Note that the supervisor also runs in response to certain events, such as tasks succeeding, failing, and reaching their task duration. The `period` value specifies the maximum time between iterations.|No|PT30S| -|`useEarliestOffset`|Boolean|If a supervisor manages a datasource for the first time, it obtains a set of starting offsets from Kafka. This flag determines whether it retrieves the earliest or latest offsets in Kafka. Under normal circumstances, subsequent tasks start from where the previous segments ended. Druid only uses `useEarliestOffset` on the first run.|No|`false`| -|`completionTimeout`|ISO 8601 period|The length of time to wait before declaring a publishing task as failed and terminating it. If the value is too low, your tasks may never publish. The publishing clock for a task begins roughly after `taskDuration` elapses.|No|PT30M| -|`lateMessageRejectionStartDateTime`|ISO 8601 date time|Configure tasks to reject messages with timestamps earlier than this date time. For example, if this property is set to `2016-01-01T11:00Z` and the supervisor creates a task at `2016-01-01T12:00Z`, Druid drops messages with timestamps earlier than `2016-01-01T11:00Z`. This can prevent concurrency issues if your data stream has late messages and you have multiple pipelines that need to operate on the same segments, such as a realtime and a nightly batch ingestion pipeline.|No|| -|`lateMessageRejectionPeriod`|ISO 8601 period|Configure tasks to reject messages with timestamps earlier than this period before the task was created. For example, if this property is set to `PT1H` and the supervisor creates a task at `2016-01-01T12:00Z`, Druid drops messages with timestamps earlier than `2016-01-01T11:00Z`. This may help prevent concurrency issues if your data stream has late messages and you have multiple pipelines that need to operate on the same segments, such as a realtime and a nightly batch ingestion pipeline. Note that you can specify only one of the late message rejection properties.|No|| -|`earlyMessageRejectionPeriod`|ISO 8601 period|Configure tasks to reject messages with timestamps later than this period after the task reached its task duration. For example, if this property is set to `PT1H`, the task duration is set to `PT1H` and the supervisor creates a task at `2016-01-01T12:00Z`, Druid drops messages with timestamps later than `2016-01-01T14:00Z`. Tasks sometimes run past their task duration, such as in cases of supervisor failover. Setting `earlyMessageRejectionPeriod` too low may cause Druid to drop messages unexpectedly whenever a task runs past its originally configured task duration.|No|| -|`autoScalerConfig`|Object|Defines auto scaling behavior for Kafka ingest tasks. See [Task autoscaler properties](#task-autoscaler-properties).|No|null| -|`idleConfig`|Object|Defines how and when the Kafka supervisor can become idle. See [Idle supervisor configuration](#idle-supervisor-configuration) for more details.|No|null| - -### Task autoscaler properties - -The following table outlines the configuration options for `autoScalerConfig`: - -|Property|Description|Required|Default| -|--------|-----------|--------|-------| -|`enableTaskAutoScaler`|Enable or disable autoscaling. `false` or blank disables the `autoScaler` even when `autoScalerConfig` is not null.|No|`false`| -|`taskCountMax`|Maximum number of ingestion tasks. Set `taskCountMax >= taskCountMin`. If `taskCountMax > {numKafkaPartitions}`, Druid only scales reading tasks up to the `{numKafkaPartitions}`. In this case, `taskCountMax` is ignored.|Yes|| -|`taskCountMin`|Minimum number of ingestion tasks. When you enable the autoscaler, Druid ignores the value of `taskCount` in `ioConfig` and starts with the `taskCountMin` number of tasks.|Yes|| -|`minTriggerScaleActionFrequencyMillis`|Minimum time interval between two scale actions.|No|600000| -|`autoScalerStrategy`|The algorithm of `autoScaler`. Only supports `lagBased`. See [Lag based autoscaler strategy related properties](#lag-based-autoscaler-strategy-related-properties) for details.|No|`lagBased`| - -### Lag based autoscaler strategy related properties - -The following table outlines the configuration options for `autoScalerStrategy`: - -|Property|Description|Required|Default| -|--------|-----------|--------|-------| -|`lagCollectionIntervalMillis`|The time period during which Druid collects lag metric points.|No|30000| -|`lagCollectionRangeMillis`|The total time window of lag collection. Use with `lagCollectionIntervalMillis` to specify the intervals at which to collect lag metric points.|No|600000| -|`scaleOutThreshold`|The threshold of scale out action.|No|6000000| -|`triggerScaleOutFractionThreshold`|Enables scale out action if `triggerScaleOutFractionThreshold` percent of lag points is higher than `scaleOutThreshold`.|No|0.3| -|`scaleInThreshold`|The threshold of scale in action.|No|1000000| -|`triggerScaleInFractionThreshold`|Enables scale in action if `triggerScaleInFractionThreshold` percent of lag points is lower than `scaleOutThreshold`.|No|0.9| -|`scaleActionStartDelayMillis`|The number of milliseconds to delay after the supervisor starts before the first scale logic check.|No|300000| -|`scaleActionPeriodMillis`|The frequency in milliseconds to check if a scale action is triggered.|No|60000| -|`scaleInStep`|The number of tasks to reduce at once when scaling down.|No|1| -|`scaleOutStep`|The number of tasks to add at once when scaling out.|No|2| - -### Ingesting from multiple topics - -To ingest data from multiple topics, you have to set `topicPattern` in the supervisor I/O configuration and not set `topic`. -You can pass multiple topics as a regex pattern as the value for `topicPattern` in the I/O configuration. For example, to -ingest data from clicks and impressions, set `topicPattern` to `clicks|impressions` in the I/O configuration. -Similarly, you can use `metrics-.*` as the value for `topicPattern` if you want to ingest from all the topics that -start with `metrics-`. If new topics are added to the cluster that match the regex, Druid automatically starts -ingesting from those new topics. A topic name that only matches partially such as `my-metrics-12` will not be -included for ingestion. If you enable multi-topic ingestion for a datasource, downgrading to a version older than -28.0.0 will cause the ingestion for that datasource to fail. - -When ingesting data from multiple topics, partitions are assigned based on the hashcode of the topic name and the -id of the partition within that topic. The partition assignment might not be uniform across all the tasks. It's also -assumed that partitions across individual topics have similar load. It is recommended that you have a higher number of -partitions for a high load topic and a lower number of partitions for a low load topic. Assuming that you want to -ingest from both high and low load topic in the same supervisor. - -## Idle supervisor configuration - -:::info - Note that idle state transitioning is currently designated as experimental. -::: - -|Property|Description|Required| -|--------|-----------|--------| -|`enabled`|If `true`, the supervisor becomes idle if there is no data on input stream/topic for some time.|No|`false`| -|`inactiveAfterMillis`|The supervisor becomes idle if all existing data has been read from input topic and no new data has been published for `inactiveAfterMillis` milliseconds.|No|`600_000`| - -When the supervisor enters the idle state, no new tasks are launched subsequent to the completion of the currently executing tasks. This strategy may lead to reduced costs for cluster operators while using topics that get sporadic data. - -The following example demonstrates supervisor spec with `lagBased` autoscaler and idle configuration enabled: - -```json -{ - "type": "kafka", - "spec": { - "dataSchema": { - ... - }, - "ioConfig": { - "topic": "metrics", - "inputFormat": { - "type": "json" - }, - "consumerProperties": { - "bootstrap.servers": "localhost:9092" - }, - "autoScalerConfig": { - "enableTaskAutoScaler": true, - "taskCountMax": 6, - "taskCountMin": 2, - "minTriggerScaleActionFrequencyMillis": 600000, - "autoScalerStrategy": "lagBased", - "lagCollectionIntervalMillis": 30000, - "lagCollectionRangeMillis": 600000, - "scaleOutThreshold": 6000000, - "triggerScaleOutFractionThreshold": 0.3, - "scaleInThreshold": 1000000, - "triggerScaleInFractionThreshold": 0.9, - "scaleActionStartDelayMillis": 300000, - "scaleActionPeriodMillis": 60000, - "scaleInStep": 1, - "scaleOutStep": 2 - }, - "taskCount":1, - "replicas":1, - "taskDuration":"PT1H", - "idleConfig": { - "enabled": true, - "inactiveAfterMillis": 600000 - } - }, - "tuningConfig":{ - ... - } - } -} -``` - -## Consumer properties - -Consumer properties must contain a property `bootstrap.servers` with a list of Kafka brokers in the form: `:,:,...`. -By default, `isolation.level` is set to `read_committed`. If you use older versions of Kafka servers without transactions support or don't want Druid to consume only committed transactions, set `isolation.level` to `read_uncommitted`. - -In some cases, you may need to fetch consumer properties at runtime. For example, when `bootstrap.servers` is not known upfront, or is not static. To enable SSL connections, you must provide passwords for `keystore`, `truststore` and `key` secretly. You can provide configurations at runtime with a dynamic config provider implementation like the environment variable config provider that comes with Druid. For more information, see [Dynamic config provider](../../operations/dynamic-config-provider.md). - -For example, if you are using SASL and SSL with Kafka, set the following environment variables for the Druid user on the machines running the Overlord and the Peon services: - -``` -export KAFKA_JAAS_CONFIG="org.apache.kafka.common.security.plain.PlainLoginModule required username='admin_user' password='admin_password';" -export SSL_KEY_PASSWORD=mysecretkeypassword -export SSL_KEYSTORE_PASSWORD=mysecretkeystorepassword -export SSL_TRUSTSTORE_PASSWORD=mysecrettruststorepassword -``` - -``` - "druid.dynamic.config.provider": { - "type": "environment", - "variables": { - "sasl.jaas.config": "KAFKA_JAAS_CONFIG", - "ssl.key.password": "SSL_KEY_PASSWORD", - "ssl.keystore.password": "SSL_KEYSTORE_PASSWORD", - "ssl.truststore.password": "SSL_TRUSTSTORE_PASSWORD" - } - } - } -``` - -Verify that you've changed the values for all configurations to match your own environment. You can use the environment variable config provider syntax in the **Consumer properties** field on the **Connect tab** in the **Load Data** UI in the web console. When connecting to Kafka, Druid replaces the environment variables with their corresponding values. - -You can provide SSL connections with [Password provider](../../operations/password-provider.md) interface to define the `keystore`, `truststore`, and `key`, but this feature is deprecated. - -## Specifying data format - -The Kafka indexing service supports both [`inputFormat`](../../ingestion/data-formats.md#input-format) and [`parser`](../../ingestion/data-formats.md#parser) to specify the data format. -Use the `inputFormat` to specify the data format for Kafka indexing service unless you need a format only supported by the legacy `parser`. - -Druid supports the following input formats: - -- `csv` -- `tsv` -- `json` -- `kafka` -- `avro_stream` -- `avro_ocf` -- `protobuf` - -For more information, see [Data formats](../../ingestion/data-formats.md). You can also read [`thrift`](../extensions-contrib/thrift.md) formats using `parser`. - -## Supervisor tuning configuration - -The `tuningConfig` object is optional. If you don't specify the `tuningConfig` object, Druid uses the default configuration settings. - -|Property|Type|Description|Required|Default| -|--------|----|-----------|--------|-------| -|`type`|String|The indexing task type. This should always be `kafka`.|Yes|| -|`maxRowsInMemory`|Integer|The number of rows to aggregate before persisting. This number represents the post-aggregation rows. It is not equivalent to the number of input events, but the resulting number of aggregated rows. Druid uses `maxRowsInMemory` to manage the required JVM heap size. The maximum heap memory usage for indexing scales is `maxRowsInMemory * (2 + maxPendingPersists)`. Normally, you do not need to set this, but depending on the nature of data, if rows are short in terms of bytes, you may not want to store a million rows in memory and this value should be set.|No|150000| -|`maxBytesInMemory`|Long|The number of bytes to aggregate in heap memory before persisting. This is based on a rough estimate of memory usage and not actual usage. Normally, this is computed internally. The maximum heap memory usage for indexing is `maxBytesInMemory * (2 + maxPendingPersists)`.|No|One-sixth of max JVM memory| -|`skipBytesInMemoryOverheadCheck`|Boolean|The calculation of `maxBytesInMemory` takes into account overhead objects created during ingestion and each intermediate persist. To exclude the bytes of these overhead objects from the `maxBytesInMemory` check, set `skipBytesInMemoryOverheadCheck` to `true`.|No|`false`| -|`maxRowsPerSegment`|Integer|The number of rows to store in a segment. This number is post-aggregation rows. Handoff occurs when `maxRowsPerSegment` or `maxTotalRows` is reached or every `intermediateHandoffPeriod`, whichever happens first.|No|5000000| -|`maxTotalRows`|Long|The number of rows to aggregate across all segments; this number is post-aggregation rows. Handoff happens either if `maxRowsPerSegment` or `maxTotalRows` is reached or every `intermediateHandoffPeriod`, whichever happens earlier.|No|20000000| -|`intermediateHandoffPeriod`|ISO 8601 period|The period that determines how often tasks hand off segments. Handoff occurs if `maxRowsPerSegment` or `maxTotalRows` is reached or every `intermediateHandoffPeriod`, whichever happens first.|No|P2147483647D| -|`intermediatePersistPeriod`|ISO 8601 period|The period that determines the rate at which intermediate persists occur.|No|PT10M| -|`maxPendingPersists`|Integer|Maximum number of persists that can be pending but not started. If a new intermediate persist exceeds this limit, Druid blocks ingestion until the currently running persist finishes. One persist can be running concurrently with ingestion, and none can be queued up. The maximum heap memory usage for indexing scales is `maxRowsInMemory * (2 + maxPendingPersists)`.|No|0| -|`numPersistThreads`|Integer|The number of threads to use to create and persist incremental segments on the disk. Higher ingestion data throughput results in a larger number of incremental segments, causing significant CPU time to be spent on the creation of the incremental segments on the disk. For datasources with number of columns running into hundreds or thousands, creation of the incremental segments may take up significant time, in the order of multiple seconds. In both of these scenarios, ingestion can stall or pause frequently, causing it to fall behind. You can use additional threads to parallelize the segment creation without blocking ingestion as long as there are sufficient CPU resources available.|No|1| -|`indexSpec`|Object|Defines how Druid indexes the data. See [IndexSpec](#indexspec) for more information.|No|| -|`indexSpecForIntermediatePersists`|Object|Defines segment storage format options to use at indexing time for intermediate persisted temporary segments. You can use `indexSpecForIntermediatePersists` to disable dimension/metric compression on intermediate segments to reduce memory required for final merging. However, disabling compression on intermediate segments might increase page cache use while they are used before getting merged into final segment published. See [IndexSpec](#indexspec) for possible values.|No|Same as `indexSpec`| -|`reportParseExceptions`|Boolean|DEPRECATED. If `true`, Druid throws exceptions encountered during parsing causing ingestion to halt. If `false`, Druid skips unparseable rows and fields. Setting `reportParseExceptions` to `true` overrides existing configurations for `maxParseExceptions` and `maxSavedParseExceptions`, setting `maxParseExceptions` to 0 and limiting `maxSavedParseExceptions` to not more than 1.|No|`false`| -|`handoffConditionTimeout`|Long|Number of milliseconds to wait for segment handoff. Set to a value >= 0, where 0 means to wait indefinitely.|No|900000 (15 minutes)| -|`resetOffsetAutomatically`|Boolean|Controls behavior when Druid needs to read Kafka messages that are no longer available, when `offsetOutOfRangeException` is encountered.
If `false`, the exception bubbles up causing tasks to fail and ingestion to halt. If this occurs, manual intervention is required to correct the situation, potentially using the [Reset Supervisor API](../../api-reference/supervisor-api.md). This mode is useful for production, since it will make you aware of issues with ingestion.
If `true`, Druid will automatically reset to the earlier or latest offset available in Kafka, based on the value of the `useEarliestOffset` property (earliest if `true`, latest if `false`). Note that this can lead to dropping data (if `useEarliestSequenceNumber` is `false`) or duplicating data (if `useEarliestSequenceNumber` is `true`) without your knowledge. Druid logs messages indicating that a reset has occurred without interrupting ingestion. This mode is useful for non-production situations since it enables Druid to recover from problems automatically, even if they lead to quiet dropping or duplicating of data. This feature behaves similarly to the Kafka `auto.offset.reset` consumer property.|No|`false`| -|`workerThreads`|Integer|The number of threads that the supervisor uses to handle requests/responses for worker tasks, along with any other internal asynchronous operation.|No|`min(10, taskCount)`| -|`chatAsync`|Boolean|If `true`, use asynchronous communication with indexing tasks, and ignore the `chatThreads` parameter. If `false`, use synchronous communication in a thread pool of size `chatThreads`.|No|`true`| -|`chatThreads`|Integer|The number of threads to use for communicating with indexing tasks. Ignored if `chatAsync` is `true`.|No|`min(10, taskCount * replicas)`| -|`chatRetries`|Integer|The number of times HTTP requests to indexing tasks are retried before considering tasks unresponsive.|No|8| -|`httpTimeout`| ISO 8601 period|The period of time to wait for a HTTP response from an indexing task.|No|PT10S| -|`shutdownTimeout`|ISO 8601 period|The period of time to wait for the supervisor to attempt a graceful shutdown of tasks before exiting.|No|PT80S| -|`offsetFetchPeriod`|ISO 8601 period|Determines how often the supervisor queries Kafka and the indexing tasks to fetch current offsets and calculate lag. If the user-specified value is below the minimum value of `PT5S`, the supervisor ignores the value and uses the minimum value instead.|No|PT30S| -|`segmentWriteOutMediumFactory`|Object|The segment write-out medium to use when creating segments. See [Additional Peon configuration: SegmentWriteOutMediumFactory](../../configuration/index.md#segmentwriteoutmediumfactory) for explanation and available options.|No|If not specified, Druid uses the value from `druid.peon.defaultSegmentWriteOutMediumFactory.type`.| -|`logParseExceptions`|Boolean|If `true`, Druid logs an error message when a parsing exception occurs, containing information about the row where the error occurred.|No|`false`| -|`maxParseExceptions`|Integer|The maximum number of parse exceptions that can occur before the task halts ingestion and fails. Overridden if `reportParseExceptions` is set.|No|unlimited| -|`maxSavedParseExceptions`|Integer|When a parse exception occurs, Druid keeps track of the most recent parse exceptions. `maxSavedParseExceptions` limits the number of saved exception instances. These saved exceptions are available after the task finishes in the [task completion report](../../ingestion/tasks.md#task-reports). Overridden if `reportParseExceptions` is set.|No|0| - -### IndexSpec - -The following table outlines the configuration options for `indexSpec`: - -|Property|Type|Description|Required|Default| -|--------|----|-----------|--------|-------| -|`bitmap`|Object|Compression format for bitmap indexes. Druid supports roaring and concise bitmap types.|No|Roaring| -|`dimensionCompression`|String|Compression format for dimension columns. Choose from `LZ4`, `LZF`, `ZSTD` or `uncompressed`.|No|`LZ4`| -|`metricCompression`|String|Compression format for primitive type metric columns. Choose from `LZ4`, `LZF`, `ZSTD`, `uncompressed` or `none`.|No|`LZ4`| -|`longEncoding`|String|Encoding format for metric and dimension columns with type long. Choose from `auto` or `longs`. `auto` encodes the values using offset or lookup table depending on column cardinality, and store them with variable size. `longs` stores the value as is with 8 bytes each.|No|`longs`| \ No newline at end of file diff --git a/docs/development/extensions-core/kinesis-ingestion.md b/docs/development/extensions-core/kinesis-ingestion.md deleted file mode 100644 index 7d0709d99cf9..000000000000 --- a/docs/development/extensions-core/kinesis-ingestion.md +++ /dev/null @@ -1,720 +0,0 @@ ---- -id: kinesis-ingestion -title: "Amazon Kinesis ingestion" -sidebar_label: "Amazon Kinesis" ---- -import Tabs from '@theme/Tabs'; -import TabItem from '@theme/TabItem'; - - - - -When you enable the Kinesis indexing service, you can configure supervisors on the Overlord to manage the creation and lifetime of Kinesis indexing tasks. These indexing tasks read events using Kinesis' own shard and sequence number mechanism to guarantee exactly-once ingestion. The supervisor oversees the state of the indexing tasks to coordinate handoffs, manage failures, and ensure that scalability and replication requirements are maintained. - -This topic contains configuration reference information for the Kinesis indexing service supervisor for Apache Druid. - -## Setup - -To use the Kinesis indexing service, you must first load the `druid-kinesis-indexing-service` core extension on both the Overlord and the Middle Manager. See [Loading extensions](../../configuration/extensions.md#loading-extensions) for more information. -Review the [Kinesis known issues](#kinesis-known-issues) before deploying the `druid-kinesis-indexing-service` extension to production. - -## Supervisor spec - -The following table outlines the high-level configuration options for the Kinesis supervisor object. -See [Supervisor API](../../api-reference/supervisor-api.md) for more information. - -|Property|Type|Description|Required| -|--------|----|-----------|--------| -|`type`|String|The supervisor type; this should always be `kinesis`.|Yes| -|`spec`|Object|The container object for the supervisor configuration.|Yes| -|`ioConfig`|Object|The [I/O configuration](#supervisor-io-configuration) object for configuring Kinesis connection and I/O-related settings for the supervisor and indexing task.|Yes| -|`dataSchema`|Object|The schema used by the Kinesis indexing task during ingestion. See [`dataSchema`](../../ingestion/ingestion-spec.md#dataschema) for more information.|Yes| -|`tuningConfig`|Object|The [tuning configuration](#supervisor-tuning-configuration) object for configuring performance-related settings for the supervisor and indexing tasks.|No| - -Druid starts a new supervisor when you define a supervisor spec. -To create a supervisor, send a `POST` request to the `/druid/indexer/v1/supervisor` endpoint. -Once created, the supervisor persists in the configured metadata database. There can only be a single supervisor per datasource, and submitting a second spec for the same datasource overwrites the previous one. - -When an Overlord gains leadership, either by being started or as a result of another Overlord failing, it spawns -a supervisor for each supervisor spec in the metadata database. The supervisor then discovers running Kinesis indexing -tasks and attempts 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 shard allocation, the tasks are killed and the -supervisor creates a new set of tasks. In this way, the supervisors persist across Overlord restarts and failovers. - -The following example shows how to submit a supervisor spec for a stream with the name `KinesisStream`. -In this example, `http://SERVICE_IP:SERVICE_PORT` is a placeholder for the server address of deployment and the service port. - - - - - -```shell -curl -X POST "http://SERVICE_IP:SERVICE_PORT/druid/indexer/v1/supervisor" \ --H "Content-Type: application/json" \ --d '{ - "type": "kinesis", - "spec": { - "ioConfig": { - "type": "kinesis", - "stream": "KinesisStream", - "inputFormat": { - "type": "json" - }, - "useEarliestSequenceNumber": true - }, - "tuningConfig": { - "type": "kinesis" - }, - "dataSchema": { - "dataSource": "KinesisStream", - "timestampSpec": { - "column": "timestamp", - "format": "iso" - }, - "dimensionsSpec": { - "dimensions": [ - "isRobot", - "channel", - "flags", - "isUnpatrolled", - "page", - "diffUrl", - { - "type": "long", - "name": "added" - }, - "comment", - { - "type": "long", - "name": "commentLength" - }, - "isNew", - "isMinor", - { - "type": "long", - "name": "delta" - }, - "isAnonymous", - "user", - { - "type": "long", - "name": "deltaBucket" - }, - { - "type": "long", - "name": "deleted" - }, - "namespace", - "cityName", - "countryName", - "regionIsoCode", - "metroCode", - "countryIsoCode", - "regionName" - ] - }, - "granularitySpec": { - "queryGranularity": "none", - "rollup": false, - "segmentGranularity": "hour" - } - } - } -}' -``` - - - -```HTTP -POST /druid/indexer/v1/supervisor -HTTP/1.1 -Host: http://SERVICE_IP:SERVICE_PORT -Content-Type: application/json - -{ - "type": "kinesis", - "spec": { - "ioConfig": { - "type": "kinesis", - "stream": "KinesisStream", - "inputFormat": { - "type": "json" - }, - "useEarliestSequenceNumber": true - }, - "tuningConfig": { - "type": "kinesis" - }, - "dataSchema": { - "dataSource": "KinesisStream", - "timestampSpec": { - "column": "timestamp", - "format": "iso" - }, - "dimensionsSpec": { - "dimensions": [ - "isRobot", - "channel", - "flags", - "isUnpatrolled", - "page", - "diffUrl", - { - "type": "long", - "name": "added" - }, - "comment", - { - "type": "long", - "name": "commentLength" - }, - "isNew", - "isMinor", - { - "type": "long", - "name": "delta" - }, - "isAnonymous", - "user", - { - "type": "long", - "name": "deltaBucket" - }, - { - "type": "long", - "name": "deleted" - }, - "namespace", - "cityName", - "countryName", - "regionIsoCode", - "metroCode", - "countryIsoCode", - "regionName" - ] - }, - "granularitySpec": { - "queryGranularity": "none", - "rollup": false, - "segmentGranularity": "hour" - } - } - } -} -``` - - - -## Supervisor I/O configuration - -The following table outlines the configuration options for `ioConfig`: - -|Property|Type|Description|Required|Default| -|--------|----|-----------|--------|-------| -|`stream`|String|The Kinesis stream to read.|Yes|| -|`inputFormat`|Object|The [input format](../../ingestion/data-formats.md#input-format) to specify how to parse input data. See [Specify data format](#specify-data-format) for more information.|Yes|| -|`endpoint`|String|The AWS Kinesis stream endpoint for a region. You can find a list of endpoints in the [AWS service endpoints](http://docs.aws.amazon.com/general/latest/gr/rande.html#ak_region) document.|No|`kinesis.us-east-1.amazonaws.com`| -|`replicas`|Integer|The number of replica sets, where 1 is a single set of tasks (no replication). Druid always assigns replicate tasks to different workers to provide resiliency against process failure.|No|1| -|`taskCount`|Integer|The maximum number of reading tasks in a replica set. Multiply `taskCount` and `replicas` to measure the maximum number of reading tasks.
The total number of tasks (reading and publishing) is higher than the maximum number of reading tasks. See [Capacity planning](#capacity-planning) for more details. When `taskCount > {numKinesisShards}`, the actual number of reading tasks is less than the `taskCount` value.|No|1| -|`taskDuration`|ISO 8601 period|The length of time before tasks stop reading and begin publishing their segments.|No|PT1H| -|`startDelay`|ISO 8601 period|The period to wait before the supervisor starts managing tasks.|No|PT5S| -|`period`|ISO 8601 period|Determines how often the supervisor executes its management logic. Note that the supervisor also runs in response to certain events, such as tasks succeeding, failing, and reaching their task duration, so this value specifies the maximum time between iterations.|No|PT30S| -|`useEarliestSequenceNumber`|Boolean|If a supervisor is managing a datasource for the first time, it obtains a set of starting sequence numbers from Kinesis. This flag determines whether a supervisor retrieves the earliest or latest sequence numbers in Kinesis. Under normal circumstances, subsequent tasks start from where the previous segments ended so this flag is only used on the first run.|No|`false`| -|`completionTimeout`|ISO 8601 period|The length of time to wait before Druid declares a publishing task has failed and terminates it. If this is set too low, your tasks may never publish. The publishing clock for a task begins roughly after `taskDuration` elapses.|No|PT6H| -|`lateMessageRejectionPeriod`|ISO 8601 period|Configure tasks to reject messages with timestamps earlier than this period before the task is created. For example, if `lateMessageRejectionPeriod` is set to `PT1H` and the supervisor creates a task at `2016-01-01T12:00Z`, messages with timestamps earlier than `2016-01-01T11:00Z` are dropped. This may help prevent concurrency issues if your data stream has late messages and you have multiple pipelines that need to operate on the same segments, such as a streaming and a nightly batch ingestion pipeline.|No|| -|`earlyMessageRejectionPeriod`|ISO 8601 period|Configure tasks to reject messages with timestamps later than this period after the task reached its `taskDuration`. For example, if `earlyMessageRejectionPeriod` is set to `PT1H`, the `taskDuration` is set to `PT1H` and the supervisor creates a task at `2016-01-01T12:00Z`. Messages with timestamps later than `2016-01-01T14:00Z` are dropped. **Note:** Tasks sometimes run past their task duration, for example, in cases of supervisor failover. Setting `earlyMessageRejectionPeriod` too low may cause messages to be dropped unexpectedly whenever a task runs past its originally configured task duration.|No|| -|`fetchDelayMillis`|Integer|Time in milliseconds to wait between subsequent calls to fetch records from Kinesis. See [Determine fetch settings](#determine-fetch-settings).|No|0| -|`awsAssumedRoleArn`|String|The AWS assumed role to use for additional permissions.|No|| -|`awsExternalId`|String|The AWS external ID to use for additional permissions.|No|| -|`autoScalerConfig`|Object|Defines autoscaling behavior for Kinesis ingest tasks. See [Task autoscaler properties](#task-autoscaler-properties) for more information.|No|null| - -### Task autoscaler properties - -The following table outlines the configuration options for `autoScalerConfig`: - -|Property|Description|Required|Default| -|--------|-----------|--------|-------| -|`enableTaskAutoScaler`|Enables the auto scaler. If not specified, Druid disables the auto scaler even when `autoScalerConfig` is not null.|No|`false`| -|`taskCountMax`|Maximum number of Kinesis ingestion tasks. Must be greater than or equal to `taskCountMin`. If greater than `{numKinesisShards}`, Druid sets the maximum number of reading tasks to `{numKinesisShards}` and ignores `taskCountMax`.|Yes|| -|`taskCountMin`|Minimum number of Kinesis ingestion tasks. When you enable the auto scaler, Druid ignores the value of `taskCount` in `IOConfig` and uses `taskCountMin` for the initial number of tasks to launch.|Yes|| -|`minTriggerScaleActionFrequencyMillis`|Minimum time interval between two scale actions.| No|600000| -|`autoScalerStrategy`|The algorithm of `autoScaler`. Druid only supports the `lagBased` strategy. See [Lag based autoscaler strategy related properties](#lag-based-autoscaler-strategy-related-properties) for more information.|No|Defaults to `lagBased`.| - -### Lag based autoscaler strategy related properties - -Unlike the Kafka indexing service, Kinesis reports lag metrics measured in time difference in milliseconds between the current sequence number and latest sequence number, rather than message count. - -The following table outlines the configuration options for `autoScalerStrategy`: - -|Property|Description|Required|Default| -|--------|-----------|--------|-------| -|`lagCollectionIntervalMillis`|The time period during which Druid collects lag metric points.|No|30000| -|`lagCollectionRangeMillis`|The total time window of lag collection. Use with `lagCollectionIntervalMillis` to specify the intervals at which to collect lag metric points.|No|600000| -|`scaleOutThreshold`|The threshold of scale out action. |No|6000000| -|`triggerScaleOutFractionThreshold`|Enables scale out action if `triggerScaleOutFractionThreshold` percent of lag points is higher than `scaleOutThreshold`.|No|0.3| -|`scaleInThreshold`|The threshold of scale in action.|No|1000000| -|`triggerScaleInFractionThreshold`|Enables scale in action if `triggerScaleInFractionThreshold` percent of lag points is lower than `scaleOutThreshold`.|No|0.9| -|`scaleActionStartDelayMillis`|The number of milliseconds to delay after the supervisor starts before the first scale logic check.|No|300000| -|`scaleActionPeriodMillis`|The frequency in milliseconds to check if a scale action is triggered.|No|60000| -|`scaleInStep`|The number of tasks to reduce at once when scaling down.|No|1| -|`scaleOutStep`|The number of tasks to add at once when scaling out.|No|2| - -The following example shows a supervisor spec with `lagBased` auto scaler enabled. - -
- Click to view the example - -```json -{ - "type": "kinesis", - "dataSchema": { - "dataSource": "metrics-kinesis", - "timestampSpec": { - "column": "timestamp", - "format": "auto" - }, - "dimensionsSpec": { - "dimensions": [], - "dimensionExclusions": [ - "timestamp", - "value" - ] - }, - "metricsSpec": [ - { - "name": "count", - "type": "count" - }, - { - "name": "value_sum", - "fieldName": "value", - "type": "doubleSum" - }, - { - "name": "value_min", - "fieldName": "value", - "type": "doubleMin" - }, - { - "name": "value_max", - "fieldName": "value", - "type": "doubleMax" - } - ], - "granularitySpec": { - "type": "uniform", - "segmentGranularity": "HOUR", - "queryGranularity": "NONE" - } - }, - "ioConfig": { - "stream": "metrics", - "autoScalerConfig": { - "enableTaskAutoScaler": true, - "taskCountMax": 6, - "taskCountMin": 2, - "minTriggerScaleActionFrequencyMillis": 600000, - "autoScalerStrategy": "lagBased", - "lagCollectionIntervalMillis": 30000, - "lagCollectionRangeMillis": 600000, - "scaleOutThreshold": 600000, - "triggerScaleOutFractionThreshold": 0.3, - "scaleInThreshold": 100000, - "triggerScaleInFractionThreshold": 0.9, - "scaleActionStartDelayMillis": 300000, - "scaleActionPeriodMillis": 60000, - "scaleInStep": 1, - "scaleOutStep": 2 - }, - "inputFormat": { - "type": "json" - }, - "endpoint": "kinesis.us-east-1.amazonaws.com", - "taskCount": 1, - "replicas": 1, - "taskDuration": "PT1H" - }, - "tuningConfig": { - "type": "kinesis", - "maxRowsPerSegment": 5000000 - } -} -``` - -
- -### Specify data format - -The Kinesis indexing service supports both [`inputFormat`](../../ingestion/data-formats.md#input-format) and [`parser`](../../ingestion/data-formats.md#parser) to specify the data format. -Use the `inputFormat` to specify the data format for the Kinesis indexing service unless you need a format only supported by the legacy `parser`. - -Supported values for `inputFormat` include: - -- `csv` -- `delimited` -- `json` -- `avro_stream` -- `avro_ocf` -- `protobuf` - -For more information, see [Data formats](../../ingestion/data-formats.md). You can also read [`thrift`](../extensions-contrib/thrift.md) formats using `parser`. - -## Supervisor tuning configuration - -The `tuningConfig` object is optional. If you don't specify the `tuningConfig` object, Druid uses the default configuration settings. - -The following table outlines the configuration options for `tuningConfig`: - -|Property|Type|Description|Required|Default| -|--------|----|-----------|--------|-------| -|`type`|String|The indexing task type. This should always be `kinesis`.|Yes|| -|`maxRowsInMemory`|Integer|The number of rows to aggregate before persisting. This number represents the post-aggregation rows. It is not equivalent to the number of input events, but the resulting number of aggregated rows. Druid uses `maxRowsInMemory` to manage the required JVM heap size. The maximum heap memory usage for indexing scales is `maxRowsInMemory * (2 + maxPendingPersists)`.|No|100000| -|`maxBytesInMemory`|Long| The number of bytes to aggregate in heap memory before persisting. This is based on a rough estimate of memory usage and not actual usage. Normally, this is computed internally. The maximum heap memory usage for indexing is `maxBytesInMemory * (2 + maxPendingPersists)`.|No|One-sixth of max JVM memory| -|`skipBytesInMemoryOverheadCheck`|Boolean|The calculation of `maxBytesInMemory` takes into account overhead objects created during ingestion and each intermediate persist. To exclude the bytes of these overhead objects from the `maxBytesInMemory` check, set `skipBytesInMemoryOverheadCheck` to `true`.|No|`false`| -|`maxRowsPerSegment`|Integer|The number of rows to aggregate into a segment; this number represents the post-aggregation rows. Handoff occurs when `maxRowsPerSegment` or `maxTotalRows` is reached or every `intermediateHandoffPeriod`, whichever happens first.|No|5000000| -|`maxTotalRows`|Long|The number of rows to aggregate across all segments; this number represents the post-aggregation rows. Handoff occurs when `maxRowsPerSegment` or `maxTotalRows` is reached or every `intermediateHandoffPeriod`, whichever happens first.|No|unlimited| -|`intermediateHandoffPeriod`|ISO 8601 period|The period that determines how often tasks hand off segments. Handoff occurs if `maxRowsPerSegment` or `maxTotalRows` is reached or every `intermediateHandoffPeriod`, whichever happens first.|No|P2147483647D| -|`intermediatePersistPeriod`|ISO 8601 period|The period that determines the rate at which intermediate persists occur.|No|PT10M| -|`maxPendingPersists`|Integer|Maximum number of persists that can be pending but not started. If a new intermediate persist exceeds this limit, Druid blocks ingestion until the currently running persist finishes. One persist can be running concurrently with ingestion, and none can be queued up. The maximum heap memory usage for indexing scales is `maxRowsInMemory * (2 + maxPendingPersists)`.|No|0| -|`indexSpec`|Object|Defines how Druid indexes the data. See [IndexSpec](#indexspec) for more information.|No|| -|`indexSpecForIntermediatePersists`|Object|Defines segment storage format options to use at indexing time for intermediate persisted temporary segments. You can use `indexSpecForIntermediatePersists` to disable dimension/metric compression on intermediate segments to reduce memory required for final merging. However, disabling compression on intermediate segments might increase page cache use while they are used before getting merged into final segment published. See [IndexSpec](#indexspec) for possible values.|No|Same as `indexSpec`| -|`reportParseExceptions`|Boolean|If `true`, Druid throws exceptions encountered during parsing causing ingestion to halt. If `false`, Druid skips unparseable rows and fields.|No|`false`| -|`handoffConditionTimeout`|Long|Number of milliseconds to wait for segment handoff. Set to a value >= 0, where 0 means to wait indefinitely.|No|0| -|`resetOffsetAutomatically`|Boolean|Controls behavior when Druid needs to read Kinesis messages that are no longer available.
If `false`, the exception bubbles up causing tasks to fail and ingestion to halt. If this occurs, manual intervention is required to correct the situation, potentially using the [Reset Supervisor API](../../api-reference/supervisor-api.md). This mode is useful for production, since it highlights issues with ingestion.
If `true`, Druid automatically resets to the earliest or latest sequence number available in Kinesis, based on the value of the `useEarliestSequenceNumber` property (earliest if `true`, latest if `false`). Note that this can lead to dropping data (if `useEarliestSequenceNumber` is `false`) or duplicating data (if `useEarliestSequenceNumber` is `true`) without your knowledge. Druid logs messages indicating that a reset has occurred without interrupting ingestion. This mode is useful for non-production situations since it enables Druid to recover from problems automatically, even if they lead to quiet dropping or duplicating of data.|No|`false`| -|`skipSequenceNumberAvailabilityCheck`|Boolean|Whether to enable checking if the current sequence number is still available in a particular Kinesis shard. If `false`, the indexing task attempts to reset the current sequence number, depending on the value of `resetOffsetAutomatically`.|No|`false`| -|`workerThreads`|Integer|The number of threads that the supervisor uses to handle requests/responses for worker tasks, along with any other internal asynchronous operation.|No| `min(10, taskCount)`| -|`chatRetries`|Integer|The number of times Druid retries HTTP requests to indexing tasks before considering tasks unresponsive.|No|8| -|`httpTimeout`|ISO 8601 period|The period of time to wait for a HTTP response from an indexing task.|No|PT10S| -|`shutdownTimeout`|ISO 8601 period|The period of time to wait for the supervisor to attempt a graceful shutdown of tasks before exiting.|No|PT80S| -|`recordBufferSizeBytes`|Integer| The size of the buffer (heap memory bytes) Druid uses between the Kinesis fetch threads and the main ingestion thread.|No| See [Determine fetch settings](#determine-fetch-settings) for defaults.| -|`recordBufferOfferTimeout`|Integer|The number of milliseconds to wait for space to become available in the buffer before timing out.|No|5000| -|`recordBufferFullWait`|Integer|The number of milliseconds to wait for the buffer to drain before Druid attempts to fetch records from Kinesis again.|No|5000| -|`fetchThreads`|Integer|The size of the pool of threads fetching data from Kinesis. There is no benefit in having more threads than Kinesis shards.|No| `procs * 2`, where `procs` is the number of processors available to the task.| -|`segmentWriteOutMediumFactory`|Object|The segment write-out medium to use when creating segments See [Additional Peon configuration: SegmentWriteOutMediumFactory](../../configuration/index.md#segmentwriteoutmediumfactory) for explanation and available options.|No|If not specified, Druid uses the value from `druid.peon.defaultSegmentWriteOutMediumFactory.type`.| -|`logParseExceptions`|Boolean|If `true`, Druid logs an error message when a parsing exception occurs, containing information about the row where the error occurred.|No|`false`| -|`maxParseExceptions`|Integer|The maximum number of parse exceptions that can occur before the task halts ingestion and fails. Overridden if `reportParseExceptions` is set.|No|unlimited| -|`maxSavedParseExceptions`|Integer|When a parse exception occurs, Druid keeps track of the most recent parse exceptions. `maxSavedParseExceptions` limits the number of saved exception instances. These saved exceptions are available after the task finishes in the [task completion report](../../ingestion/tasks.md#task-reports). Overridden if `reportParseExceptions` is set.|No|0| -|`maxBytesPerPoll`|Integer| The maximum number of bytes to be fetched from buffer per poll. At least one record is polled from the buffer regardless of this config.|No| 1000000 bytes| -|`repartitionTransitionDuration`|ISO 8601 period|When shards are split or merged, the supervisor recomputes shard to task group mappings. The supervisor also signals any running tasks created under the old mappings to stop early at current time + `repartitionTransitionDuration`. Stopping the tasks early allows Druid to begin reading from the new shards more quickly. The repartition transition wait time controlled by this property gives the stream additional time to write records to the new shards after the split or merge, which helps avoid issues with [empty shard handling](https://github.com/apache/druid/issues/7600).|No|PT2M| -|`offsetFetchPeriod`|ISO 8601 period|Determines how often the supervisor queries Kinesis and the indexing tasks to fetch current offsets and calculate lag. If the user-specified value is below the minimum value of PT5S, the supervisor ignores the value and uses the minimum value instead.|No|PT30S| -|`useListShards`|Boolean|Indicates if `listShards` API of AWS Kinesis SDK can be used to prevent `LimitExceededException` during ingestion. You must set the necessary `IAM` permissions.|No|`false`| - -### IndexSpec - -The following table outlines the configuration options for `indexSpec`: - -|Property|Type|Description|Required|Default| -|--------|----|-----------|--------|-------| -|`bitmap`|Object|Compression format for bitmap indexes. Druid supports roaring and concise bitmap types.|No|Roaring| -|`dimensionCompression`|String|Compression format for dimension columns. Choose from `LZ4`, `LZF`, or `uncompressed`.|No|`LZ4`| -|`metricCompression`|String|Compression format for primitive type metric columns. Choose from `LZ4`, `LZF`, `uncompressed`, or `none`.|No|`LZ4`| -|`longEncoding`|String|Encoding format for metric and dimension columns with type long. Choose from `auto` or `longs`. `auto` encodes the values using sequence number or lookup table depending on column cardinality and stores them with variable sizes. `longs` stores the value as is with 8 bytes each.|No|`longs`| - -## Operations - -This section describes how to use the [Supervisor API](../../api-reference/supervisor-api.md) with the Kinesis indexing service. - -### AWS authentication - -Druid uses AWS access and secret keys to authenticate Kinesis API requests. There are a few ways to provide this information to Druid: - -1. Using roles or short-term credentials: - - Druid looks for credentials set in [environment variables](https://docs.aws.amazon.com/cli/latest/userguide/cli-configure-envvars.html), -via [Web Identity Token](https://docs.aws.amazon.com/IAM/latest/UserGuide/id_roles_providers_oidc.html), in the -default [profile configuration file](https://docs.aws.amazon.com/cli/latest/userguide/cli-configure-files.html), and from the -EC2 instance profile provider (in this order). - -2. Using long-term security credentials: - - You can directly provide your AWS access key and AWS secret key in the `common.runtime.properties` file as shown in the example below: - -```properties -druid.kinesis.accessKey=AKIAWxxxxxxxxxx4NCKS -druid.kinesis.secretKey=Jbytxxxxxxxxxxx2+555 -``` - -> Note: AWS does not recommend providing long-term security credentials in configuration files since it might pose a security risk. -If you use this approach, it takes precedence over all other methods of providing credentials. - -To ingest data from Kinesis, ensure that the policy attached to your IAM role contains the necessary permissions. -The required permissions depend on the value of `useListShards`. - -If the `useListShards` flag is set to `true`, you need following permissions: - -- `ListStreams` to list your data streams. -- `Get*` required for `GetShardIterator`. -- `GetRecords` to get data records from a data stream's shard. -- `ListShards` to get the shards for a stream of interest. - -The following is an example policy: - -```json -[ - { - "Effect": "Allow", - "Action": ["kinesis:List*"], - "Resource": ["*"] - }, - { - "Effect": "Allow", - "Action": ["kinesis:Get*"], - "Resource": [] - } -] -``` - -If the `useListShards` flag is set to `false`, you need following permissions: - -- `ListStreams` to list your data streams. -- `Get*` required for `GetShardIterator`. -- `GetRecords` to get data records from a data stream's shard. -- `DescribeStream` to describe the specified data stream. - -The following is an example policy: - -```json -[ - { - "Effect": "Allow", - "Action": ["kinesis:ListStreams"], - "Resource": ["*"] - }, - { - "Effect": "Allow", - "Action": ["kinesis:DescribeStream"], - "Resource": ["*"] - }, - { - "Effect": "Allow", - "Action": ["kinesis:Get*"], - "Resource": [] - } -] -``` - -### Get supervisor status report - -To retrieve the current status report for a single supervisor, send a `GET` request to the `/druid/indexer/v1/supervisor/:supervisorId/status` endpoint. - -The report contains the state of the supervisor tasks, the latest sequence numbers, and an array of recently thrown exceptions reported as `recentErrors`. You can control the maximum size of the exceptions using the `druid.supervisor.maxStoredExceptionEvents` configuration. - -The two properties related to the supervisor's state are `state` and `detailedState`. The `state` property contains a small number of generic states that apply to any type of supervisor, while the `detailedState` property contains a more descriptive, implementation-specific state that may provide more insight into the supervisor's activities. - -Possible `state` values are `PENDING`, `RUNNING`, `SUSPENDED`, `STOPPING`, `UNHEALTHY_SUPERVISOR`, and `UNHEALTHY_TASKS`. - -The following table lists `detailedState` values and their corresponding `state` mapping: - -|Detailed state|Corresponding state|Description| -|--------------|-------------------|-----------| -|`UNHEALTHY_SUPERVISOR`|`UNHEALTHY_SUPERVISOR`|The supervisor encountered errors on previous `druid.supervisor.unhealthinessThreshold` iterations.| -|`UNHEALTHY_TASKS`|`UNHEALTHY_TASKS`|The last `druid.supervisor.taskUnhealthinessThreshold` tasks all failed.| -|`UNABLE_TO_CONNECT_TO_STREAM`|`UNHEALTHY_SUPERVISOR`|The supervisor is encountering connectivity issues with Kinesis and has not successfully connected in the past.| -|`LOST_CONTACT_WITH_STREAM`|`UNHEALTHY_SUPERVISOR`|The supervisor is encountering connectivity issues with Kinesis but has successfully connected in the past.| -|`PENDING` (first iteration only)|`PENDING`|The supervisor has been initialized but hasn't started connecting to the stream.| -|`CONNECTING_TO_STREAM` (first iteration only)|`RUNNING`|The supervisor is trying to connect to the stream and update partition data.| -|`DISCOVERING_INITIAL_TASKS` (first iteration only)|`RUNNING`|The supervisor is discovering already-running tasks.| -|`CREATING_TASKS` (first iteration only)|`RUNNING`|The supervisor is creating tasks and discovering state.| -|`RUNNING`|`RUNNING`|The supervisor has started tasks and is waiting for `taskDuration` to elapse.| -|`SUSPENDED`|`SUSPENDED`|The supervisor is suspended.| -|`STOPPING`|`STOPPING`|The 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 shards from Kinesis and determine the starting sequence number for each shard (either based on the last processed sequence number if continuing, or starting from the beginning or ending of the stream if this is a new stream). -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 the view of the state of the tasks under 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. - -The `detailedState` property shows additional values (marked with "first iteration only" in the preceding table) 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. For example, if the supervisor cannot connect to -Kinesis, if it's unable to read from the stream, or cannot 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 stopped, suspended, or hits a failure threshold and transitions to an unhealthy state. - -### Update existing supervisors - -To update an existing supervisor spec, send a `POST` request to the `/druid/indexer/v1/supervisor` endpoint. - -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 their segments, exiting itself. Druid then uses the provided configuration from the request body to create a new supervisor with a new set of tasks that start reading from the sequence numbers, 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. - -You can achieve seamless schema migrations by submitting the new schema using the `/druid/indexer/v1/supervisor` endpoint. - -### Suspend and resume a supervisor - -To suspend a supervisor, send a `POST` request to the `/druid/indexer/v1/supervisor/:supervisorId/suspend` endpoint. -Suspending a supervisor does not prevent it from operating and emitting logs and metrics. It ensures that no indexing tasks are running until the supervisor resumes. - -To resume a supervisor, send a `POST` request to the `/druid/indexer/v1/supervisor/:supervisorId/resume` endpoint. - -### Reset a supervisor - -The supervisor must be running for this endpoint to be available - -To reset a supervisor, send a `POST` request to the `/druid/indexer/v1/supervisor/:supervisorId/reset` endpoint. This endpoint clears stored -sequence numbers, prompting the supervisor to start reading from either the earliest or the -latest sequence numbers in Kinesis (depending on the value of `useEarliestSequenceNumber`). -After clearing stored sequence numbers, the supervisor kills and recreates active tasks, -so that tasks begin reading from valid sequence numbers. - -This endpoint is useful when you need to recover from a stopped state due to missing sequence numbers in Kinesis. -Use this endpoint with caution as it may result in skipped messages, leading to data loss or duplicate data. - -The indexing service keeps track of the latest -persisted sequence number to provide exactly-once ingestion guarantees across -tasks. -Subsequent tasks must start reading from where the previous task completed -for the generated segments to be accepted. If the messages at the expected starting sequence numbers are -no longer available in Kinesis (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 endpoint enables you to recover from this condition. - -### Resetting Offsets for a supervisor - -To reset partition offsets for a supervisor, send a `POST` request to the `/druid/indexer/v1/supervisor/:supervisorId/resetOffsets` endpoint. This endpoint clears stored -sequence numbers, prompting 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 specified offsets. For partitions that are not specified in this operation, the supervisor will resume from the last -stored offset. - -Use this endpoint with caution as it may result in skipped messages, leading to data loss or duplicate data. - -### Terminate a supervisor - -To terminate a supervisor and its associated indexing tasks, send a `POST` request to the `/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. -The tasks stop reading and begin publishing their segments immediately. -The call returns after all tasks have been signaled to stop but before the tasks finish publishing their segments. - -The terminated supervisor continues exists in the metadata store and its history can be retrieved. -The only way to restart a terminated supervisor is by submitting a functioning supervisor spec to `/druid/indexer/v1/supervisor`. - -## Capacity planning - -Kinesis indexing tasks run on Middle Managers and are limited by the resources available in the Middle Manager 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, such as batch processing, streaming tasks, and merging tasks. If your workers run out of capacity, Kinesis indexing tasks 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 Kinesis purges those sequence numbers. - -A running task can be in one of two states: reading or publishing. A task remains in reading state for the period defined in `taskDuration`, at which point it transitions to publishing state. A task remains in publishing state for as long as it takes to generate segments, push segments to deep storage, and have them 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 are `replicas * taskCount` reading tasks. An exception occurs if `taskCount > {numKinesisShards}`, in which case Druid uses `{numKinesisShards}` tasks. When `taskDuration` elapses, these tasks transition to publishing state and `replicas * taskCount` new reading tasks are created. To allow for reading tasks and publishing tasks to run concurrently, there should be a minimum capacity of: - -```text -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, load on Historical) is greater than `taskDuration`. This is a valid and correct scenario 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. - -## Shards and segment handoff - -Each Kinesis indexing task writes the events it consumes from Kinesis shards into a single segment for the segment granularity interval until it reaches one of the following limits: `maxRowsPerSegment`, `maxTotalRows`, or `intermediateHandoffPeriod`. -At this point, the task creates a new shard for this segment granularity to contain subsequent events. - -The Kinesis indexing task also performs incremental hand-offs so that the segments created by the task are not held up until the task duration is over. -When the task reaches one of the `maxRowsPerSegment`, `maxTotalRows`, or `intermediateHandoffPeriod` limits, it hands off all the segments and creates a new set of segments for further events. This allows the task to run for longer durations -without accumulating old segments locally on Middle Manager processes. - -The Kinesis 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](../../operations/segment-optimization.md). - -## Determine fetch settings - -Kinesis indexing tasks fetch records using `fetchThreads` threads. -If `fetchThreads` is higher than the number of Kinesis shards, the excess threads are unused. -Each fetch thread fetches up to 10 MB of records at once from a Kinesis shard, with a delay between fetches -of `fetchDelayMillis`. -The records fetched by each thread are pushed into a shared queue of size `recordBufferSizeBytes`. -The main runner thread for each task polls up to `maxRecordsPerPoll` records from the queue at once. - -The default values for these parameters are: - -- `fetchThreads`: Twice the number of processors available to the task. The number of processors available to the task -is the total number of processors on the server, divided by `druid.worker.capacity` (the number of task slots on that -particular server). This value is further limited so that the total data record data fetched at a given time does not -exceed 5% of the max heap configured, assuming that each thread fetches 10 MB of records at once. If the value specified -for this configuration is higher than this limit, no failure occurs, but a warning is logged, and the value is -implicitly lowered to the max allowed by this constraint. -- `fetchDelayMillis`: 0 (no delay between fetches). -- `recordBufferSizeBytes`: 100 MB or an estimated 10% of available heap, whichever is smaller. -- `maxBytesPerPoll`: 1000000. - -Kinesis places the following restrictions on calls to fetch records: - -- Each data record can be up to 1 MB in size. -- Each shard can support up to 5 transactions per second for reads. -- Each shard can read up to 2 MB per second. -- The maximum size of data that GetRecords can return is 10 MB. - -If the above limits are exceeded, Kinesis throws `ProvisionedThroughputExceededException` errors. If this happens, Druid -Kinesis tasks pause by `fetchDelayMillis` or 3 seconds, whichever is larger, and then attempt the call again. - -In most cases, the default settings for fetch parameters are sufficient to achieve good performance without excessive -memory usage. However, in some cases, you may need to adjust these parameters to control fetch rate -and memory usage more finely. Optimal values depend on the average size of a record and the number of consumers you -have reading from a given shard, which will be `replicas` unless you have other consumers also reading from this -Kinesis stream. - -## Deaggregation - -The Kinesis indexing service supports de-aggregation of multiple rows packed into a single record by the Kinesis -Producer Library's aggregate method for more efficient data transfer. - -## Resharding - -[Resharding](https://docs.aws.amazon.com/streams/latest/dev/kinesis-using-sdk-java-resharding.html) is an advanced operation that lets you adjust the number of shards in a stream to adapt to changes in the rate of data flowing through a stream. - -When changing the shard count for a Kinesis stream, there is a window of time around the resharding operation with early shutdown of Kinesis ingestion tasks and possible task failures. - -The early shutdowns and task failures are expected. They occur because the supervisor updates the shard to task group mappings as shards are closed and fully read. This ensures that tasks are not running -with an assignment of closed shards that have been fully read and balances distribution of active shards across tasks. - -This window with early task shutdowns and possible task failures concludes when: - -- All closed shards have been fully read and the Kinesis ingestion tasks have published the data from those shards, committing the "closed" state to metadata storage. -- Any remaining tasks that had inactive shards in the assignment have been shut down. These tasks would have been created before the closed shards were completely drained. - -Note that when the supervisor is running and detects new partitions, tasks read new partitions from the earliest offsets, irrespective of the `useEarliestSequence` setting. This is because these new shards were immediately discovered and are therefore unlikely to experience a lag. - -If resharding occurs when the supervisor is suspended and `useEarliestSequence` is set to `false`, resuming the supervisor causes tasks to read the new shards from the latest sequence. This is by design so that the consumer can catch up quickly with any lag accumulated while the supervisor was suspended. - -## Kinesis known issues - -Before you deploy the Kinesis extension to production, consider the following known issues: - -- Avoid implementing more than one Kinesis supervisor that reads from the same Kinesis stream for ingestion. Kinesis has a per-shard read throughput limit and having multiple supervisors on the same stream can reduce available read throughput for an individual supervisor's tasks. Multiple supervisors ingesting to the same Druid datasource can also cause increased contention for locks on the datasource. -- The only way to change the stream reset policy is to submit a new ingestion spec and set up a new supervisor. -- If ingestion tasks get stuck, the supervisor does not automatically recover. You should monitor ingestion tasks and investigate if your ingestion falls behind. -- A Kinesis supervisor can sometimes compare the checkpoint offset to retention window of the stream to see if it has fallen behind. These checks fetch the earliest sequence number for Kinesis which can result in `IteratorAgeMilliseconds` becoming very high in AWS CloudWatch. diff --git a/docs/development/extensions-core/protobuf.md b/docs/development/extensions-core/protobuf.md index 3c87809f72b5..08b9cc1185f0 100644 --- a/docs/development/extensions-core/protobuf.md +++ b/docs/development/extensions-core/protobuf.md @@ -30,7 +30,7 @@ for [stream ingestion](../../ingestion/index.md#streaming). See corresponding do ## Example: Load Protobuf messages from Kafka -This example demonstrates how to load Protobuf messages from Kafka. Please read the [Load from Kafka tutorial](../../tutorials/tutorial-kafka.md) first, and see [Kafka Indexing Service](./kafka-ingestion.md) documentation for more details. +This example demonstrates how to load Protobuf messages from Kafka. Please read the [Load from Kafka tutorial](../../tutorials/tutorial-kafka.md) first, and see [Kafka Indexing Service](../../ingestion/kafka-ingestion.md) documentation for more details. The files used in this example are found at [`./examples/quickstart/protobuf` in your Druid directory](https://github.com/apache/druid/tree/master/examples/quickstart/protobuf). diff --git a/docs/ingestion/data-formats.md b/docs/ingestion/data-formats.md index 7dd1b10c7fa9..c9c23896a286 100644 --- a/docs/ingestion/data-formats.md +++ b/docs/ingestion/data-formats.md @@ -398,8 +398,8 @@ For details, see the Schema Registry [documentation](http://docs.confluent.io/cu | url | String | Specifies the URL endpoint of the Schema Registry. | yes | | capacity | Integer | Specifies the max size of the cache (default = Integer.MAX_VALUE). | no | | urls | Array | Specifies the URL endpoints of the multiple Schema Registry instances. | yes (if `url` is not provided) | -| config | Json | To send additional configurations, configured for Schema Registry. This can be supplied via a [DynamicConfigProvider](../operations/dynamic-config-provider.md) | no | -| headers | Json | To send headers to the Schema Registry. This can be supplied via a [DynamicConfigProvider](../operations/dynamic-config-provider.md) | no | +| config | Json | To send additional configurations, configured for Schema Registry. This can be supplied via a [DynamicConfigProvider](../operations/dynamic-config-provider.md) | no | +| headers | Json | To send headers to the Schema Registry. This can be supplied via a [DynamicConfigProvider](../operations/dynamic-config-provider.md) | no | For a single schema registry instance, use Field `url` or `urls` for multi instances. @@ -549,52 +549,62 @@ For example: ### Kafka -`kafka` is a special input format that wraps a regular input format (which goes in `valueFormat`) and allows you -to parse the Kafka metadata (timestamp, headers, and key) that is part of Kafka messages. -It should only be used when ingesting from Apache Kafka. +The `kafka` input format lets you parse the Kafka metadata fields in addition to the Kafka payload value contents. +It should only be used when ingesting from Apache Kafka. -Configure the Kafka `inputFormat` as follows: +The `kafka` input format wraps around the payload parsing input format and augments the data it outputs with the Kafka event timestamp, topic name, event headers, and the key field that itself can be parsed using any available input format. -| Field | Type | Description | Required | -|-------|------|-------------|----------| -| `type` | String | Set value to `kafka`. | yes | -| `valueFormat` | [InputFormat](#input-format) | Any [InputFormat](#input-format) to parse the Kafka value payload. For details about specifying the input format, see [Specifying data format](../development/extensions-core/kafka-supervisor-reference.md#specifying-data-format). | yes | -| `timestampColumnName` | String | Name of the column for the kafka record's timestamp.| no (default = "kafka.timestamp") | -| `topicColumnName` | String |Name of the column for the kafka record's topic. It is useful when ingesting data from multiple topics.| no (default = "kafka.timestamp") | -| `headerColumnPrefix` | String | Custom prefix for all the header columns. | no (default = "kafka.header.") | -| `headerFormat` | Object | `headerFormat` specifies how to parse the Kafka headers. Supports String types. Because Kafka header values are bytes, the parser decodes them as UTF-8 encoded strings. To change this behavior, implement your own parser based on the encoding style. Change the 'encoding' type in `KafkaStringHeaderFormat` to match your custom implementation. | no | -| `keyFormat` | [InputFormat](#input-format) | Any [input format](#input-format) to parse the Kafka key. It only processes the first entry of the `inputFormat` field. For details, see [Specifying data format](../development/extensions-core/kafka-supervisor-reference.md#specifying-data-format). | no | -| `keyColumnName` | String | Name of the column for the kafka record's key.| no (default = "kafka.key") | +If there are conflicts between column names in the payload and those created from the metadata, the payload takes precedence. +This ensures that upgrading a Kafka ingestion to use the Kafka input format (by taking its existing input format and setting it as the `valueFormat`) can be done without losing any of the payload data. +Configure the Kafka `inputFormat` as follows: -The Kafka input format augments the payload with information from the Kafka timestamp, headers, and key. +| Field | Type | Description | Required | Default | +|-------|------|-------------|----------|---------| +| `type` | String | Set value to `kafka`. | yes || +| `valueFormat` | [InputFormat](#input-format) | The [input format](#input-format) to parse the Kafka value payload. | yes || +| `timestampColumnName` | String | The name of the column for the Kafka timestamp.| no |`kafka.timestamp`| +| `topicColumnName` | String |The name of the column for the Kafka topic. This field is useful when ingesting data from multiple topics into same datasource.| no |`kafka.topic`| +| `headerColumnPrefix` | String | The custom prefix for all the header columns. | no | `kafka.header`| +| `headerFormat` | Object | Specifies how to parse the Kafka headers. Supports String types. Because Kafka header values are bytes, the parser decodes them as UTF-8 encoded strings. To change this behavior, implement your own parser based on the encoding style. Change the `encoding` type in `KafkaStringHeaderFormat` to match your custom implementation. See [Header format](#header-format) for supported encoding formats.| no || +| `keyFormat` | [InputFormat](#input-format) | The [input format](#input-format) to parse the Kafka key. It only processes the first entry of the `inputFormat` field. If your key values are simple strings, you can use the `tsv` format to parse them. Note that for `tsv`,`csv`, and `regex` formats, you need to provide a `columns` array to make a valid input format. Only the first one is used, and its name will be ignored in favor of `keyColumnName`. | no || +| `keyColumnName` | String | The name of the column for the Kafka key.| no |`kafka.key`| + +#### Header format + +`headerFormat` supports the following encoding formats: + - `ISO-8859-1`: ISO Latin Alphabet No. 1, that is, ISO-LATIN-1. + - `US-ASCII`: Seven-bit ASCII. Also known as ISO646-US. The Basic Latin block of the Unicode character set. + - `UTF-8`: Eight-bit UCS Transformation Format. + - `UTF-16`: Sixteen-bit UCS Transformation Format, byte order identified by an optional byte-order mark. + - `UTF-16BE`: Sixteen-bit UCS Transformation Format, big-endian byte order. + - `UTF-16LE`: Sixteen-bit UCS Transformation Format, little-endian byte order. +- `headerColumnPrefix`: Supply a prefix to the Kafka headers to avoid any conflicts with columns from the payload. The default is `kafka.header.`. -If there are conflicts between column names in the payload and those created from the metadata, the payload takes precedence. -This ensures that upgrading a Kafka ingestion to use the Kafka input format (by taking its existing input format and setting it as the `valueFormat`) can be done without losing any of the payload data. +#### Example -Here is a minimal example that only augments the parsed payload with the Kafka timestamp column and kafka topic column: +Using `{ "type": "json" }` as the input format would only parse the payload value. +To parse the Kafka metadata in addition to the payload, use the `kafka` input format. -``` -"ioConfig": { - "inputFormat": { - "type": "kafka", - "valueFormat": { - "type": "json" - } - }, - ... -} -``` +For example, consider the following structure for a Kafka message that represents an edit in a development environment: -Here is a complete example: +- **Kafka timestamp**: `1680795276351` +- **Kafka topic**: `wiki-edits` +- **Kafka headers**: + - `env=development` + - `zone=z1` +- **Kafka key**: `wiki-edit` +- **Kafka payload value**: `{"channel":"#sv.wikipedia","timestamp":"2016-06-27T00:00:11.080Z","page":"Salo Toraut","delta":31,"namespace":"Main"}` -``` +You would configure it as follows: + +```json "ioConfig": { "inputFormat": { "type": "kafka", "valueFormat": { "type": "json" - } + }, "timestampColumnName": "kafka.timestamp", "topicColumnName": "kafka.topic", "headerFormat": { @@ -608,8 +618,24 @@ Here is a complete example: "columns": ["x"] }, "keyColumnName": "kafka.key", - }, - ... + } +} +``` + +You would parse the example message as follows: + +```json +{ + "channel": "#sv.wikipedia", + "timestamp": "2016-06-27T00:00:11.080Z", + "page": "Salo Toraut", + "delta": 31, + "namespace": "Main", + "kafka.timestamp": 1680795276351, + "kafka.topic": "wiki-edits", + "kafka.header.env": "development", + "kafka.header.zone": "z1", + "kafka.key": "wiki-edit" } ``` @@ -631,6 +657,80 @@ Similarly, if you want to use a timestamp extracted from the Kafka header: } ``` +Finally, add these Kafka metadata columns to the `dimensionsSpec` or set your `dimensionsSpec` to auto-detect columns. + +The following supervisor spec demonstrates how to ingest the Kafka header, key, timestamp, and topic into Druid dimensions: + +
+Click to view the example + +```json +{ + "type": "kafka", + "spec": { + "ioConfig": { + "type": "kafka", + "consumerProperties": { + "bootstrap.servers": "localhost:9092" + }, + "topic": "wiki-edits", + "inputFormat": { + "type": "kafka", + "valueFormat": { + "type": "json" + }, + "headerFormat": { + "type": "string" + }, + "keyFormat": { + "type": "tsv", + "findColumnsFromHeader": false, + "columns": ["x"] + } + }, + "useEarliestOffset": true + }, + "dataSchema": { + "dataSource": "wikiticker", + "timestampSpec": { + "column": "timestamp", + "format": "posix" + }, + "dimensionsSpec": "dimensionsSpec": { + "useSchemaDiscovery": true, + "includeAllDimensions": true + }, + "granularitySpec": { + "queryGranularity": "none", + "rollup": false, + "segmentGranularity": "day" + } + }, + "tuningConfig": { + "type": "kafka" + } + } +} +``` +
+ +After Druid ingests the data, you can query the Kafka metadata columns as follows: + +```sql +SELECT + "kafka.header.env", + "kafka.key", + "kafka.timestamp", + "kafka.topic" +FROM "wikiticker" +``` + +This query returns: + +| `kafka.header.env` | `kafka.key` | `kafka.timestamp` | `kafka.topic` | +|--------------------|-----------|---------------|---------------| +| `development` | `wiki-edit` | `1680795276351` | `wiki-edits` | + ## FlattenSpec You can use the `flattenSpec` object to flatten nested data, as an alternative to the Druid [nested columns](../querying/nested-columns.md) feature, and for nested input formats unsupported by the feature. It is an object within the `inputFormat` object. @@ -698,8 +798,8 @@ Each entry in the `fields` list can have the following components: ## Parser :::info - The Parser is deprecated for [native batch tasks](./native-batch.md), [Kafka indexing service](../development/extensions-core/kafka-ingestion.md), -and [Kinesis indexing service](../development/extensions-core/kinesis-ingestion.md). + The Parser is deprecated for [native batch tasks](./native-batch.md), [Kafka indexing service](../ingestion/kafka-ingestion.md), +and [Kinesis indexing service](../ingestion/kinesis-ingestion.md). Consider using the [input format](#input-format) instead for these types of ingestion. ::: @@ -1464,8 +1564,8 @@ Multiple Instances: ## ParseSpec :::info - The Parser is deprecated for [native batch tasks](./native-batch.md), [Kafka indexing service](../development/extensions-core/kafka-ingestion.md), -and [Kinesis indexing service](../development/extensions-core/kinesis-ingestion.md). + The Parser is deprecated for [native batch tasks](./native-batch.md), [Kafka indexing service](../ingestion/kafka-ingestion.md), +and [Kinesis indexing service](../ingestion/kinesis-ingestion.md). Consider using the [input format](#input-format) instead for these types of ingestion. ::: diff --git a/docs/ingestion/index.md b/docs/ingestion/index.md index 26c71d1ec0b9..d500b1d9bc25 100644 --- a/docs/ingestion/index.md +++ b/docs/ingestion/index.md @@ -54,7 +54,7 @@ page. There are two available options for streaming ingestion. Streaming ingestion is controlled by a continuously-running supervisor. -| **Method** | [Kafka](../development/extensions-core/kafka-ingestion.md) | [Kinesis](../development/extensions-core/kinesis-ingestion.md) | +| **Method** | [Kafka](../ingestion/kafka-ingestion.md) | [Kinesis](../ingestion/kinesis-ingestion.md) | |---|-----|--------------| | **Supervisor type** | `kafka` | `kinesis`| | **How it works** | Druid reads directly from Apache Kafka. | Druid reads directly from Amazon Kinesis.| diff --git a/docs/ingestion/ingestion-spec.md b/docs/ingestion/ingestion-spec.md index 4aec1de80c5c..6d1f9609d856 100644 --- a/docs/ingestion/ingestion-spec.md +++ b/docs/ingestion/ingestion-spec.md @@ -96,8 +96,8 @@ For more examples, refer to the documentation for each ingestion method. You can also load data visually, without the need to write an ingestion spec, using the "Load data" functionality available in Druid's [web console](../operations/web-console.md). Druid's visual data loader supports -[Kafka](../development/extensions-core/kafka-ingestion.md), -[Kinesis](../development/extensions-core/kinesis-ingestion.md), and +[Kafka](../ingestion/kafka-ingestion.md), +[Kinesis](../ingestion/kinesis-ingestion.md), and [native batch](native-batch.md) mode. ## `dataSchema` @@ -503,7 +503,7 @@ is: |skipBytesInMemoryOverheadCheck|The calculation of maxBytesInMemory takes into account overhead objects created during ingestion and each intermediate persist. Setting this to true can exclude the bytes of these overhead objects from maxBytesInMemory check.|false| |indexSpec|Defines segment storage format options to use at indexing time.|See [`indexSpec`](#indexspec) for more information.| |indexSpecForIntermediatePersists|Defines segment storage format options to use at indexing time for intermediate persisted temporary segments.|See [`indexSpec`](#indexspec) for more information.| -|Other properties|Each ingestion method has its own list of additional tuning properties. See the documentation for each method for a full list: [Kafka indexing service](../development/extensions-core/kafka-supervisor-reference.md#supervisor-tuning-configuration), [Kinesis indexing service](../development/extensions-core/kinesis-ingestion.md#supervisor-tuning-configuration), [Native batch](native-batch.md#tuningconfig), and [Hadoop-based](hadoop.md#tuningconfig).|| +|Other properties|Each ingestion method has its own list of additional tuning properties. See the documentation for each method for a full list: [Kafka indexing service](../ingestion/kafka-ingestion.md#tuning-configuration), [Kinesis indexing service](../ingestion/kinesis-ingestion.md#tuning-configuration), [Native batch](native-batch.md#tuningconfig), and [Hadoop-based](hadoop.md#tuningconfig).|| ### `indexSpec` diff --git a/docs/ingestion/kafka-ingestion.md b/docs/ingestion/kafka-ingestion.md new file mode 100644 index 000000000000..5d14b1ad6baf --- /dev/null +++ b/docs/ingestion/kafka-ingestion.md @@ -0,0 +1,449 @@ +--- +id: kafka-ingestion +title: "Apache Kafka ingestion" +sidebar_label: "Apache Kafka ingestion" +description: "Overview of the Kafka indexing service for Druid. Includes example supervisor specs to help you get started." +--- + + + +:::info +To use the Kafka indexing service, you must be on Apache Kafka version 0.11.x or higher. +If you are using an older version, refer to the [Apache Kafka upgrade guide](https://kafka.apache.org/documentation/#upgrade). +::: + +When you enable the Kafka indexing service, you can configure supervisors on the Overlord to manage the creation and lifetime of Kafka indexing tasks. +Kafka indexing tasks read events using Kafka partition and offset mechanism to guarantee exactly-once ingestion. The supervisor oversees the state of the indexing tasks to coordinate handoffs, manage failures, and ensure that scalability and replication requirements are maintained. + +This topic contains configuration information for the Kafka indexing service supervisor for Apache Druid. + +## Setup + +To use the Kafka indexing service, you must first load the `druid-kafka-indexing-service` extension on both the Overlord and the MiddleManager. See [Loading extensions](../configuration/extensions.md) for more information. + +## Supervisor spec configuration + +This section outlines the configuration properties that are specific to the Apache Kafka streaming ingestion method. For configuration properties shared across all streaming ingestion methods supported by Druid, see [Supervisor spec](supervisor.md#supervisor-spec). + +The following example shows a supervisor spec for the Kafka indexing service: + +
+ Click to view the example + +```json +{ + "type": "kafka", + "spec": { + "dataSchema": { + "dataSource": "metrics-kafka", + "timestampSpec": { + "column": "timestamp", + "format": "auto" + }, + "dimensionsSpec": { + "dimensions": [], + "dimensionExclusions": [ + "timestamp", + "value" + ] + }, + "metricsSpec": [ + { + "name": "count", + "type": "count" + }, + { + "name": "value_sum", + "fieldName": "value", + "type": "doubleSum" + }, + { + "name": "value_min", + "fieldName": "value", + "type": "doubleMin" + }, + { + "name": "value_max", + "fieldName": "value", + "type": "doubleMax" + } + ], + "granularitySpec": { + "type": "uniform", + "segmentGranularity": "HOUR", + "queryGranularity": "NONE" + } + }, + "ioConfig": { + "topic": "metrics", + "inputFormat": { + "type": "json" + }, + "consumerProperties": { + "bootstrap.servers": "localhost:9092" + }, + "taskCount": 1, + "replicas": 1, + "taskDuration": "PT1H" + }, + "tuningConfig": { + "type": "kafka", + "maxRowsPerSegment": 5000000 + } + } +} +``` + +
+ +### I/O configuration + +The following table outlines the `ioConfig` configuration properties specific to Kafka. +For configuration properties shared across all streaming ingestion methods, refer to [Supervisor I/O configuration](supervisor.md#io-configuration). + +|Property|Type|Description|Required|Default| +|--------|----|-----------|--------|-------| +|`topic`|String|The Kafka topic to read from. To ingest data from multiple topic, use `topicPattern`. |Yes if `topicPattern` isn't set.|| +|`topicPattern`|String|Multiple Kafka topics to read from, passed as a regex pattern. See [Ingest from multiple topics](#ingest-from-multiple-topics) for more information.|Yes if `topic` isn't set.|| +|`consumerProperties`|String, Object|A map of properties to pass to the Kafka consumer. See [Consumer properties](#consumer-properties) for details.|Yes. At the minimum, you must set the `bootstrap.servers` property to establish the initial connection to the Kafka cluster.|| +|`pollTimeout`|Long|The length of time to wait for the Kafka consumer to poll records, in milliseconds.|No|100| +|`useEarliestOffset`|Boolean|If a supervisor manages a datasource for the first time, it obtains a set of starting offsets from Kafka. This flag determines whether it retrieves the earliest or latest offsets in Kafka. Under normal circumstances, subsequent tasks start from where the previous segments ended. Druid only uses `useEarliestOffset` on the first run.|No|`false`| +|`idleConfig`|Object|Defines how and when the Kafka supervisor can become idle. See [Idle configuration](#idle-configuration) for more details.|No|null| + +#### Ingest from multiple topics + +:::info +If you enable multi-topic ingestion for a datasource, downgrading to a version older than +28.0.0 will cause the ingestion for that datasource to fail. +::: + +You can ingest data from one or multiple topics. +When ingesting data from multiple topics, Druid assigns partitions based on the hashcode of the topic name and the ID of the partition within that topic. The partition assignment might not be uniform across all the tasks. Druid assumes that partitions across individual topics have similar load. If you want to ingest from both high and low load topics in the same supervisor, it is recommended that you have a higher number of partitions for a high load topic and a lower number of partitions for a low load topic. + +To ingest data from multiple topics, use the `topicPattern` property instead of `topic`. +You pass multiple topics as a regex pattern. For example, to ingest data from clicks and impressions, set `topicPattern` to `clicks|impressions`. +Similarly, you can use `metrics-.*` as the value for `topicPattern` if you want to ingest from all the topics that start with `metrics-`. If you add a new topic that matches the regex to the cluster, Druid automatically starts ingesting from the new topic. Topic names that match partially, such as `my-metrics-12`, are not included for ingestion. + +#### Consumer properties + +Consumer properties control how a supervisor reads and processes event messages from a Kafka stream. For more information about consumers, refer to the [Apache Kafka documentation](https://kafka.apache.org/documentation/#consumerconfigs). + +The `consumerProperties` object must contain a `bootstrap.servers` property with a list of Kafka brokers in the form: `:,:,...`. +By default, `isolation.level` is set to `read_committed`. + +If you use older versions of Kafka servers without transactions support or don't want Druid to consume only committed transactions, set `isolation.level` to `read_uncommitted`. If you need Druid to consume older versions of Kafka, make sure offsets are sequential, since there is no offset gap check in Druid. + +If your Kafka cluster enables consumer-group based ACLs, you can set `group.id` in `consumerProperties` to override the default auto generated group ID. + +In some cases, you may need to fetch consumer properties at runtime. For example, when `bootstrap.servers` is not known upfront or is not static. To enable SSL connections, you must provide passwords for `keystore`, `truststore`, and `key` secretly. You can provide configurations at runtime with a dynamic config provider implementation like the environment variable config provider that comes with Druid. For more information, see [Dynamic config provider](../operations/dynamic-config-provider.md). + +For example, if you are using SASL and SSL with Kafka, set the following environment variables for the Druid user on the machines running the Overlord and the Peon services: + +``` +export KAFKA_JAAS_CONFIG="org.apache.kafka.common.security.plain.PlainLoginModule required username='admin_user' password='admin_password';" +export SSL_KEY_PASSWORD=mysecretkeypassword +export SSL_KEYSTORE_PASSWORD=mysecretkeystorepassword +export SSL_TRUSTSTORE_PASSWORD=mysecrettruststorepassword +``` + +```json +"druid.dynamic.config.provider": { + "type": "environment", + "variables": { + "sasl.jaas.config": "KAFKA_JAAS_CONFIG", + "ssl.key.password": "SSL_KEY_PASSWORD", + "ssl.keystore.password": "SSL_KEYSTORE_PASSWORD", + "ssl.truststore.password": "SSL_TRUSTSTORE_PASSWORD" + } +} +``` + +Verify that you've changed the values for all configurations to match your own environment. In the Druid data loader interface, you can use the environment variable config provider syntax in the **Consumer properties** field on the **Connect tab**. When connecting to Kafka, Druid replaces the environment variables with their corresponding values. + +You can provide SSL connections with [Password provider](../operations/password-provider.md) interface to define the `keystore`, `truststore`, and `key`, but this feature is deprecated. + +#### Idle configuration + +:::info +Idle state transitioning is currently designated as experimental. +::: + +When the supervisor enters the idle state, no new tasks are launched subsequent to the completion of the currently executing tasks. This strategy may lead to reduced costs for cluster operators while using topics that get sporadic data. + +The following table outlines the configuration options for `idleConfig`: + +|Property|Description|Required| +|--------|-----------|--------| +|`enabled`|If `true`, the supervisor becomes idle if there is no data on input stream or topic for some time.|No|`false`| +|`inactiveAfterMillis`|The supervisor becomes idle if all existing data has been read from input topic and no new data has been published for `inactiveAfterMillis` milliseconds.|No|`600_000`| + +The following example shows a supervisor spec with idle configuration enabled: + +
+ Click to view the example + +```json +{ + "type": "kafka", + "spec": { + "dataSchema": {...}, + "ioConfig": { + "topic": "metrics", + "inputFormat": { + "type": "json" + }, + "consumerProperties": { + "bootstrap.servers": "localhost:9092" + }, + "autoScalerConfig": { + "enableTaskAutoScaler": true, + "taskCountMax": 6, + "taskCountMin": 2, + "minTriggerScaleActionFrequencyMillis": 600000, + "autoScalerStrategy": "lagBased", + "lagCollectionIntervalMillis": 30000, + "lagCollectionRangeMillis": 600000, + "scaleOutThreshold": 6000000, + "triggerScaleOutFractionThreshold": 0.3, + "scaleInThreshold": 1000000, + "triggerScaleInFractionThreshold": 0.9, + "scaleActionStartDelayMillis": 300000, + "scaleActionPeriodMillis": 60000, + "scaleInStep": 1, + "scaleOutStep": 2 + }, + "taskCount": 1, + "replicas": 1, + "taskDuration": "PT1H", + "idleConfig": { + "enabled": true, + "inactiveAfterMillis": 600000 + } + }, + "tuningConfig": {...} + } +} +``` +
+ +#### Data format + +The Kafka indexing service supports both [`inputFormat`](data-formats.md#input-format) and [`parser`](data-formats.md#parser) to specify the data format. Use the `inputFormat` to specify the data format for the Kafka indexing service unless you need a format only supported by the legacy `parser`. For more information, see [Source input formats](data-formats.md). + +The Kinesis indexing service supports the following values for `inputFormat`: + +* `csv` +* `tvs` +* `json` +* `kafka` +* `avro_stream` +* `avro_ocf` +* `protobuf` + +You can use `parser` to read [`thrift`](../development/extensions-contrib/thrift.md) formats. + +##### Kafka input format supervisor spec example + +The `kafka` input format lets you parse the Kafka metadata fields in addition to the Kafka payload value contents. + +The `kafka` input format wraps around the payload parsing input format and augments the data it outputs with the Kafka event timestamp, the Kafka topic name, the Kafka event headers, and the key field that itself can be parsed using any available input format. + +For example, consider the following structure for a Kafka message that represents a wiki edit in a development environment: + +- **Kafka timestamp**: `1680795276351` +- **Kafka topic**: `wiki-edits` +- **Kafka headers**: + - `env=development` + - `zone=z1` +- **Kafka key**: `wiki-edit` +- **Kafka payload value**: `{"channel":"#sv.wikipedia","timestamp":"2016-06-27T00:00:11.080Z","page":"Salo Toraut","delta":31,"namespace":"Main"}` + +Using `{ "type": "json" }` as the input format only parses the payload value. +To parse the Kafka metadata in addition to the payload, use the `kafka` input format. + +You configure it as follows: + +- `valueFormat`: Define how to parse the payload value. Set this to the payload parsing input format (`{ "type": "json" }`). +- `timestampColumnName`: Supply a custom name for the Kafka timestamp in the Druid schema to avoid conflicts with columns from the payload. The default is `kafka.timestamp`. +- `topicColumnName`: Supply a custom name for the Kafka topic in the Druid schema to avoid conflicts with columns from the payload. The default is `kafka.topic`. This field is useful when ingesting data from multiple topics into the same datasource. +- `headerFormat`: The default value `string` decodes strings in UTF-8 encoding from the Kafka header. + Other supported encoding formats include the following: + - `ISO-8859-1`: ISO Latin Alphabet No. 1, that is, ISO-LATIN-1. + - `US-ASCII`: Seven-bit ASCII. Also known as ISO646-US. The Basic Latin block of the Unicode character set. + - `UTF-16`: Sixteen-bit UCS Transformation Format, byte order identified by an optional byte-order mark. + - `UTF-16BE`: Sixteen-bit UCS Transformation Format, big-endian byte order. + - `UTF-16LE`: Sixteen-bit UCS Transformation Format, little-endian byte order. +- `headerColumnPrefix`: Supply a prefix to the Kafka headers to avoid any conflicts with columns from the payload. The default is `kafka.header.`. + Considering the header from the example, Druid maps the headers to the following columns: `kafka.header.env`, `kafka.header.zone`. +- `keyFormat`: Supply an input format to parse the key. Only the first value is used. + If, as in the example, your key values are simple strings, then you can use the `tsv` format to parse them. + ```json + { + "type": "tsv", + "findColumnsFromHeader": false, + "columns": ["x"] + } + ``` + Note that for `tsv`,`csv`, and `regex` formats, you need to provide a `columns` array to make a valid input format. Only the first one is used, and its name will be ignored in favor of `keyColumnName`. +- `keyColumnName`: Supply the name for the Kafka key column to avoid conflicts with columns from the payload. The default is `kafka.key`. + +The following input format uses default values for `timestampColumnName`, `topicColumnName`, `headerColumnPrefix`, and `keyColumnName`: + +```json +{ + "type": "kafka", + "valueFormat": { + "type": "json" + }, + "headerFormat": { + "type": "string" + }, + "keyFormat": { + "type": "tsv", + "findColumnsFromHeader": false, + "columns": ["x"] + } +} +``` + +It parses the example message as follows: + +```json +{ + "channel": "#sv.wikipedia", + "timestamp": "2016-06-27T00:00:11.080Z", + "page": "Salo Toraut", + "delta": 31, + "namespace": "Main", + "kafka.timestamp": 1680795276351, + "kafka.topic": "wiki-edits", + "kafka.header.env": "development", + "kafka.header.zone": "z1", + "kafka.key": "wiki-edit" +} +``` + +Finally, add these Kafka metadata columns to the `dimensionsSpec` or set your `dimensionsSpec` to auto-detect columns. + +The following supervisor spec demonstrates how to ingest the Kafka header, key, timestamp, and topic into Druid dimensions: + +
+ Click to view the example + +```json +{ + "type": "kafka", + "spec": { + "ioConfig": { + "type": "kafka", + "consumerProperties": { + "bootstrap.servers": "localhost:9092" + }, + "topic": "wiki-edits", + "inputFormat": { + "type": "kafka", + "valueFormat": { + "type": "json" + }, + "headerFormat": { + "type": "string" + }, + "keyFormat": { + "type": "tsv", + "findColumnsFromHeader": false, + "columns": ["x"] + } + }, + "useEarliestOffset": true + }, + "dataSchema": { + "dataSource": "wikiticker", + "timestampSpec": { + "column": "timestamp", + "format": "posix" + }, + "dimensionsSpec": "dimensionsSpec": { + "useSchemaDiscovery": true, + "includeAllDimensions": true + }, + "granularitySpec": { + "queryGranularity": "none", + "rollup": false, + "segmentGranularity": "day" + } + }, + "tuningConfig": { + "type": "kafka" + } + } +} +``` +
+ +After Druid ingests the data, you can query the Kafka metadata columns as follows: + +```sql +SELECT + "kafka.header.env", + "kafka.key", + "kafka.timestamp", + "kafka.topic" +FROM "wikiticker" +``` + +This query returns: + +|`kafka.header.env`|`kafka.key`|`kafka.timestamp`|`kafka.topic`| +|------------------|-----------|-----------------|-------------| +|`development`|`wiki-edit`|`1680795276351`|`wiki-edits`| + +### Tuning configuration + +The following table outlines the `tuningConfig` configuration properties specific to Kafka. +For configuration properties shared across all streaming ingestion methods, refer to [Supervisor tuning configuration](supervisor.md#tuning-configuration). + +|Property|Type|Description|Required|Default| +|--------|----|-----------|--------|-------| +|`numPersistThreads`|Integer|The number of threads to use to create and persist incremental segments on the disk. Higher ingestion data throughput results in a larger number of incremental segments, causing significant CPU time to be spent on the creation of the incremental segments on the disk. For datasources with number of columns running into hundreds or thousands, creation of the incremental segments may take up significant time, in the order of multiple seconds. In both of these scenarios, ingestion can stall or pause frequently, causing it to fall behind. You can use additional threads to parallelize the segment creation without blocking ingestion as long as there are sufficient CPU resources available.|No|1| +|`chatAsync`|Boolean|If `true`, use asynchronous communication with indexing tasks, and ignore the `chatThreads` parameter. If `false`, use synchronous communication in a thread pool of size `chatThreads`.|No|`true`| +|`chatThreads`|Integer|The number of threads to use for communicating with indexing tasks. Ignored if `chatAsync` is `true`.|No|`min(10, taskCount * replicas)`| + +## Deployment notes on Kafka partitions and Druid segments + +Druid assigns Kafka partitions to each Kafka indexing task. 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 limits: `maxRowsPerSegment`, `maxTotalRows`, or `intermediateHandoffPeriod`. 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 don't 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 set of segments for further events. This allows the task to run for longer durations without accumulating old segments locally on MiddleManager services. + +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 information on how to optimize the segment size, see [Segment size optimization](../operations/segment-optimization.md). + +## Learn more + +See the following topics for more information: + +* [Supervisor API](../api-reference/supervisor-api.md) for how to manage and monitor supervisors using the API. +* [Supervisor](../ingestion/supervisor.md) for supervisor status and capacity planning. +* [Loading from Apache Kafka](../tutorials/tutorial-kafka.md) for a tutorial on streaming data from Apache Kafka. +* [Kafka input format](../ingestion/data-formats.md#kafka) to learn about the `kafka` input format. \ No newline at end of file diff --git a/docs/ingestion/kinesis-ingestion.md b/docs/ingestion/kinesis-ingestion.md new file mode 100644 index 000000000000..fb4bfde235a0 --- /dev/null +++ b/docs/ingestion/kinesis-ingestion.md @@ -0,0 +1,334 @@ +--- +id: kinesis-ingestion +title: "Amazon Kinesis ingestion" +sidebar_label: "Amazon Kinesis ingestion" +--- +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + + + +When you enable the Kinesis indexing service, you can configure supervisors on the Overlord to manage the creation and lifetime of Kinesis indexing tasks. Kinesis indexing tasks read events using the Kinesis shard and sequence number mechanism to guarantee exactly-once ingestion. The supervisor oversees the state of the indexing tasks to coordinate handoffs, manage failures, and ensure that scalability and replication requirements are maintained. + +This topic contains configuration information for the Kinesis indexing service supervisor for Apache Druid. + +## Setup + +To use the Kinesis indexing service, you must first load the `druid-kinesis-indexing-service` core extension on both the Overlord and the MiddleManager. See [Loading extensions](../configuration/extensions.md#loading-extensions) for more information. + +Review [Known issues](#known-issues) before deploying the `druid-kinesis-indexing-service` extension to production. + +## Supervisor spec configuration + +This section outlines the configuration properties that are specific to the Amazon Kinesis streaming ingestion method. For configuration properties shared across all streaming ingestion methods supported by Druid, see [Supervisor spec](supervisor.md#supervisor-spec). + +The following example shows a supervisor spec for a stream with the name `KinesisStream`: + +
Click to view the example + +```json +{ + "type": "kinesis", + "spec": { + "ioConfig": { + "type": "kinesis", + "stream": "KinesisStream", + "inputFormat": { + "type": "json" + }, + "useEarliestSequenceNumber": true + }, + "tuningConfig": { + "type": "kinesis" + }, + "dataSchema": { + "dataSource": "KinesisStream", + "timestampSpec": { + "column": "timestamp", + "format": "iso" + }, + "dimensionsSpec": { + "dimensions": [ + "isRobot", + "channel", + "flags", + "isUnpatrolled", + "page", + "diffUrl", + { + "type": "long", + "name": "added" + }, + "comment", + { + "type": "long", + "name": "commentLength" + }, + "isNew", + "isMinor", + { + "type": "long", + "name": "delta" + }, + "isAnonymous", + "user", + { + "type": "long", + "name": "deltaBucket" + }, + { + "type": "long", + "name": "deleted" + }, + "namespace", + "cityName", + "countryName", + "regionIsoCode", + "metroCode", + "countryIsoCode", + "regionName" + ] + }, + "granularitySpec": { + "queryGranularity": "none", + "rollup": false, + "segmentGranularity": "hour" + } + } + } +} +``` +
+ +### I/O configuration + +The following table outlines the `ioConfig` configuration properties specific to Kinesis. +For configuration properties shared across all streaming ingestion methods, refer to [Supervisor I/O configuration](supervisor.md#io-configuration). + +|Property|Type|Description|Required|Default| +|--------|----|-----------|--------|-------| +|`stream`|String|The Kinesis stream to read.|Yes|| +|`endpoint`|String|The AWS Kinesis stream endpoint for a region. You can find a list of endpoints in the [AWS service endpoints](http://docs.aws.amazon.com/general/latest/gr/rande.html#ak_region) document.|No|`kinesis.us-east-1.amazonaws.com`| +|`useEarliestSequenceNumber`|Boolean|If a supervisor is managing a datasource for the first time, it obtains a set of starting sequence numbers from Kinesis. This flag determines whether a supervisor retrieves the earliest or latest sequence numbers in Kinesis. Under normal circumstances, subsequent tasks start from where the previous segments ended so this flag is only used on the first run.|No|`false`| +|`fetchDelayMillis`|Integer|Time in milliseconds to wait between subsequent calls to fetch records from Kinesis. See [Determine fetch settings](#determine-fetch-settings).|No|0| +|`awsAssumedRoleArn`|String|The AWS assumed role to use for additional permissions.|No|| +|`awsExternalId`|String|The AWS external ID to use for additional permissions.|No|| + +#### Data format + +The Kinesis indexing service supports both [`inputFormat`](data-formats.md#input-format) and [`parser`](data-formats.md#parser) to specify the data format. Use the `inputFormat` to specify the data format for the Kinesis indexing service unless you need a format only supported by the legacy `parser`. For more information, see [Source input formats](data-formats.md). + +The Kinesis indexing service supports the following values for `inputFormat`: + +* `csv` +* `tvs` +* `json` +* `avro_stream` +* `avro_ocf` +* `protobuf` + +You can use `parser` to read [`thrift`](../development/extensions-contrib/thrift.md) formats. + +### Tuning configuration + +The following table outlines the `tuningConfig` configuration properties specific to Kinesis. +For configuration properties shared across all streaming ingestion methods, refer to [Supervisor tuning configuration](supervisor.md#tuning-configuration). + +|Property|Type|Description|Required|Default| +|--------|----|-----------|--------|-------| +|`skipSequenceNumberAvailabilityCheck`|Boolean|Whether to enable checking if the current sequence number is still available in a particular Kinesis shard. If `false`, the indexing task attempts to reset the current sequence number, depending on the value of `resetOffsetAutomatically`.|No|`false`| +|`recordBufferSizeBytes`|Integer| The size of the buffer (heap memory bytes) Druid uses between the Kinesis fetch threads and the main ingestion thread.|No| See [Determine fetch settings](#determine-fetch-settings) for defaults.| +|`recordBufferOfferTimeout`|Integer|The number of milliseconds to wait for space to become available in the buffer before timing out.|No|5000| +|`recordBufferFullWait`|Integer|The number of milliseconds to wait for the buffer to drain before Druid attempts to fetch records from Kinesis again.|No|5000| +|`fetchThreads`|Integer|The size of the pool of threads fetching data from Kinesis. There is no benefit in having more threads than Kinesis shards.|No| `procs * 2`, where `procs` is the number of processors available to the task.| +|`maxBytesPerPoll`|Integer| The maximum number of bytes to be fetched from buffer per poll. At least one record is polled from the buffer regardless of this config.|No| 1000000 bytes| +|`repartitionTransitionDuration`|ISO 8601 period|When shards are split or merged, the supervisor recomputes shard to task group mappings. The supervisor also signals any running tasks created under the old mappings to stop early at current time + `repartitionTransitionDuration`. Stopping the tasks early allows Druid to begin reading from the new shards more quickly. The repartition transition wait time controlled by this property gives the stream additional time to write records to the new shards after the split or merge, which helps avoid issues with [empty shard handling](https://github.com/apache/druid/issues/7600).|No|`PT2M`| +|`useListShards`|Boolean|Indicates if `listShards` API of AWS Kinesis SDK can be used to prevent `LimitExceededException` during ingestion. You must set the necessary `IAM` permissions.|No|`false`| + +## AWS authentication + +Druid uses AWS access and secret keys to authenticate Kinesis API requests. There are a few ways to provide this information to Druid: + +1. Using roles or short-term credentials: + + Druid looks for credentials set in [environment variables](https://docs.aws.amazon.com/cli/latest/userguide/cli-configure-envvars.html), via [Web Identity Token](https://docs.aws.amazon.com/IAM/latest/UserGuide/id_roles_providers_oidc.html), in the default [profile configuration file](https://docs.aws.amazon.com/cli/latest/userguide/cli-configure-files.html), and from the EC2 instance profile provider (in this order). + +2. Using long-term security credentials: + + You can directly provide your AWS access key and AWS secret key in the `common.runtime.properties` file as shown in the example below: + + ```properties + druid.kinesis.accessKey=AKIAWxxxxxxxxxx4NCKS + druid.kinesis.secretKey=Jbytxxxxxxxxxxx2+555 + ``` + +:::info +AWS does not recommend providing long-term security credentials in configuration files since it might pose a security risk. +If you use this approach, it takes precedence over all other methods of providing credentials. +::: + +To ingest data from Kinesis, ensure that the policy attached to your IAM role contains the necessary permissions. +The required permissions depend on the value of `useListShards`. + +If the `useListShards` flag is set to `true`, you need following permissions: + +- `ListStreams` to list your data streams. +- `Get*` required for `GetShardIterator`. +- `GetRecords` to get data records from a data stream's shard. +- `ListShards` to get the shards for a stream of interest. + +The following is an example policy: + +```json +[ + { + "Effect": "Allow", + "Action": ["kinesis:List*"], + "Resource": ["*"] + }, + { + "Effect": "Allow", + "Action": ["kinesis:Get*"], + "Resource": [] + } +] +``` + +If the `useListShards` flag is set to `false`, you need following permissions: + +- `ListStreams` to list your data streams. +- `Get*` required for `GetShardIterator`. +- `GetRecords` to get data records from a data stream's shard. +- `DescribeStream` to describe the specified data stream. + +The following is an example policy: + +```json +[ + { + "Effect": "Allow", + "Action": ["kinesis:ListStreams"], + "Resource": ["*"] + }, + { + "Effect": "Allow", + "Action": ["kinesis:DescribeStream"], + "Resource": ["*"] + }, + { + "Effect": "Allow", + "Action": ["kinesis:Get*"], + "Resource": [] + } +] +``` + +## Shards and segment handoff + +Each Kinesis indexing task writes the events it consumes from Kinesis shards into a single segment for the segment granularity interval until it reaches one of the following limits: `maxRowsPerSegment`, `maxTotalRows`, or `intermediateHandoffPeriod`. +At this point, the task creates a new shard for this segment granularity to contain subsequent events. + +The Kinesis indexing task also performs incremental hand-offs so that the segments created by the task are not held up until the task duration is over. +When the task reaches one of the `maxRowsPerSegment`, `maxTotalRows`, or `intermediateHandoffPeriod` limits, it hands off all the segments and creates a new set of segments for further events. This allows the task to run for longer durations +without accumulating old segments locally on MiddleManager services. + +The Kinesis 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 information on how to optimize the segment size, see [Segment size optimization](../operations/segment-optimization.md). + +## Determine fetch settings + +Kinesis indexing tasks fetch records using `fetchThreads` threads. +If `fetchThreads` is higher than the number of Kinesis shards, the excess threads are unused. +Each fetch thread fetches up to 10 MB of records at once from a Kinesis shard, with a delay between fetches of `fetchDelayMillis`. +The records fetched by each thread are pushed into a shared queue of size `recordBufferSizeBytes`. + +The default values for these parameters are: + +- `fetchThreads`: Twice the number of processors available to the task. The number of processors available to the task +is the total number of processors on the server, divided by `druid.worker.capacity` (the number of task slots on that +particular server). This value is further limited so that the total data record data fetched at a given time does not +exceed 5% of the max heap configured, assuming that each thread fetches 10 MB of records at once. If the value specified +for this configuration is higher than this limit, no failure occurs, but a warning is logged, and the value is +implicitly lowered to the max allowed by this constraint. +- `fetchDelayMillis`: 0 (no delay between fetches). +- `recordBufferSizeBytes`: 100 MB or an estimated 10% of available heap, whichever is smaller. +- `maxBytesPerPoll`: 1000000. + +Kinesis places the following restrictions on calls to fetch records: + +- Each data record can be up to 1 MB in size. +- Each shard can support up to 5 transactions per second for reads. +- Each shard can read up to 2 MB per second. +- The maximum size of data that GetRecords can return is 10 MB. + +If the above limits are exceeded, Kinesis throws `ProvisionedThroughputExceededException` errors. If this happens, Druid +Kinesis tasks pause by `fetchDelayMillis` or 3 seconds, whichever is larger, and then attempt the call again. + +In most cases, the default settings for fetch parameters are sufficient to achieve good performance without excessive +memory usage. However, in some cases, you may need to adjust these parameters to control fetch rate +and memory usage more finely. Optimal values depend on the average size of a record and the number of consumers you +have reading from a given shard, which will be `replicas` unless you have other consumers also reading from this +Kinesis stream. + +## Deaggregation + +The Kinesis indexing service supports de-aggregation of multiple rows stored within a single [Kinesis Data Streams](https://docs.aws.amazon.com/streams/latest/dev/introduction.html) record for more efficient data transfer. + +## Resharding + +[Resharding](https://docs.aws.amazon.com/streams/latest/dev/kinesis-using-sdk-java-resharding.html) is an advanced operation that lets you adjust the number of shards in a stream to adapt to changes in the rate of data flowing through a stream. + +When changing the shard count for a Kinesis stream, there is a window of time around the resharding operation with early shutdown of Kinesis ingestion tasks and possible task failures. + +The early shutdowns and task failures are expected. They occur because the supervisor updates the shard to task group mappings as shards are closed and fully read. This ensures that tasks are not running +with an assignment of closed shards that have been fully read and balances distribution of active shards across tasks. + +This window with early task shutdowns and possible task failures concludes when: + +- All closed shards have been fully read and the Kinesis ingestion tasks have published the data from those shards, committing the "closed" state to metadata storage. +- Any remaining tasks that had inactive shards in the assignment have been shut down. These tasks would have been created before the closed shards were completely drained. + +Note that when the supervisor is running and detects new partitions, tasks read new partitions from the earliest offsets, irrespective of the `useEarliestSequence` setting. This is because these new shards were immediately discovered and are therefore unlikely to experience a lag. + +If resharding occurs when the supervisor is suspended and `useEarliestSequence` is set to `false`, resuming the supervisor causes tasks to read the new shards from the latest sequence. This is by design so that the consumer can catch up quickly with any lag accumulated while the supervisor was suspended. + +## Known issues + +Before you deploy the `druid-kinesis-indexing-service` extension to production, consider the following known issues: + +- Kinesis imposes a read throughput limit per shard. If you have multiple supervisors reading from the same Kinesis stream, consider adding more shards to ensure sufficient read throughput for all supervisors. +- A Kinesis supervisor can sometimes compare the checkpoint offset to retention window of the stream to see if it has fallen behind. These checks fetch the earliest sequence number for Kinesis which can result in `IteratorAgeMilliseconds` becoming very high in AWS CloudWatch. + +## Learn more + +See the following topics for more information: + +* [Supervisor API](../api-reference/supervisor-api.md) for how to manage and monitor supervisors using the API. +* [Supervisor](../ingestion/supervisor.md) for supervisor status and capacity planning. \ No newline at end of file diff --git a/docs/ingestion/partitioning.md b/docs/ingestion/partitioning.md index 422c07de8058..6cf5b0a74d28 100644 --- a/docs/ingestion/partitioning.md +++ b/docs/ingestion/partitioning.md @@ -69,8 +69,8 @@ The following table shows how each ingestion method handles partitioning: |[Native batch](native-batch.md)|Configured using [`partitionsSpec`](native-batch.md#partitionsspec) inside the `tuningConfig`.| |[SQL](../multi-stage-query/index.md)|Configured using [`PARTITIONED BY`](../multi-stage-query/concepts.md#partitioning) and [`CLUSTERED BY`](../multi-stage-query/concepts.md#clustering).| |[Hadoop](hadoop.md)|Configured using [`partitionsSpec`](hadoop.md#partitionsspec) inside the `tuningConfig`.| -|[Kafka indexing service](../development/extensions-core/kafka-ingestion.md)|Kafka topic partitioning defines how Druid partitions the datasource. You can also [reindex](../data-management/update.md#reindex) or [compact](../data-management/compaction.md) to repartition after initial ingestion.| -|[Kinesis indexing service](../development/extensions-core/kinesis-ingestion.md)|Kinesis stream sharding defines how Druid partitions the datasource. You can also [reindex](../data-management/update.md#reindex) or [compact](../data-management/compaction.md) to repartition after initial ingestion.| +|[Kafka indexing service](../ingestion/kafka-ingestion.md)|Kafka topic partitioning defines how Druid partitions the datasource. You can also [reindex](../data-management/update.md#reindex) or [compact](../data-management/compaction.md) to repartition after initial ingestion.| +|[Kinesis indexing service](../ingestion/kinesis-ingestion.md)|Kinesis stream sharding defines how Druid partitions the datasource. You can also [reindex](../data-management/update.md#reindex) or [compact](../data-management/compaction.md) to repartition after initial ingestion.| ## Learn more diff --git a/docs/ingestion/rollup.md b/docs/ingestion/rollup.md index 241ffba367ec..212708649a43 100644 --- a/docs/ingestion/rollup.md +++ b/docs/ingestion/rollup.md @@ -87,8 +87,8 @@ The following table shows how each method handles rollup: |[Native batch](native-batch.md)|`index_parallel` and `index` type may be either perfect or best-effort, based on configuration.| |[SQL-based batch](../multi-stage-query/index.md)|Always perfect.| |[Hadoop](hadoop.md)|Always perfect.| -|[Kafka indexing service](../development/extensions-core/kafka-ingestion.md)|Always best-effort.| -|[Kinesis indexing service](../development/extensions-core/kinesis-ingestion.md)|Always best-effort.| +|[Kafka indexing service](../ingestion/kafka-ingestion.md)|Always best-effort.| +|[Kinesis indexing service](../ingestion/kinesis-ingestion.md)|Always best-effort.| ## Learn more diff --git a/docs/ingestion/standalone-realtime.md b/docs/ingestion/standalone-realtime.md index 7a3a9e0e35a6..94a8565baad7 100644 --- a/docs/ingestion/standalone-realtime.md +++ b/docs/ingestion/standalone-realtime.md @@ -41,5 +41,5 @@ suffered from limitations which made it not possible to achieve exactly once ing The extensions `druid-kafka-eight`, `druid-kafka-eight-simpleConsumer`, `druid-rabbitmq`, and `druid-rocketmq` were also removed at this time, since they were built to operate on the realtime nodes. -Please consider using the [Kafka Indexing Service](../development/extensions-core/kafka-ingestion.md) or -[Kinesis Indexing Service](../development/extensions-core/kinesis-ingestion.md) for stream pull ingestion instead. +Please consider using the [Kafka Indexing Service](../ingestion/kafka-ingestion.md) or +[Kinesis Indexing Service](../ingestion/kinesis-ingestion.md) for stream pull ingestion instead. diff --git a/docs/ingestion/streaming.md b/docs/ingestion/streaming.md new file mode 100644 index 000000000000..f0f777b6c627 --- /dev/null +++ b/docs/ingestion/streaming.md @@ -0,0 +1,35 @@ +--- +id: streaming +title: "Streaming ingestion" +--- + + + +Apache Druid can consume data streams from the following external streaming sources: + +* Apache Kafka through the bundled [Kafka indexing service](kafka-ingestion.md) extension. +* Amazon Kinesis through the bundled [Kinesis indexing service](kinesis-ingestion.md) extension. + +Each indexing service provides real-time data ingestion with exactly-once stream processing guarantee. +To use either of the streaming ingestion methods, you must first load the associated extension on both the Overlord and the MiddleManager. See [Loading extensions](../configuration/extensions.md#loading-extensions) for more information. + +Streaming ingestion is controlled by a continuously running [supervisor](supervisor.md). +The supervisor oversees the state of indexing tasks to coordinate handoffs, manage failures, and ensure that scalability and replication requirements are maintained. +You start a supervisor by submitting a JSON specification, often referred to as the supervisor spec, either though the Druid web console or using the [Supervisor API](../api-reference/supervisor-api.md). \ No newline at end of file diff --git a/docs/ingestion/supervisor.md b/docs/ingestion/supervisor.md new file mode 100644 index 000000000000..b5411f4c1685 --- /dev/null +++ b/docs/ingestion/supervisor.md @@ -0,0 +1,366 @@ +--- +id: supervisor +title: Supervisor +sidebar_label: Supervisor +--- + + + +A supervisor manages streaming ingestion from external streaming sources into Apache Druid. +Supervisors oversee the state of indexing tasks to coordinate handoffs, manage failures, and ensure that the scalability and replication requirements are maintained. + +## Supervisor spec + +Druid uses a JSON specification, often referred to as the supervisor spec, to define streaming ingestion tasks. +The supervisor spec specifies how Druid should consume, process, and index streaming data. + +The following table outlines the high-level configuration options for a supervisor spec: + +|Property|Type|Description|Required| +|--------|----|-----------|--------| +|`type`|String|The supervisor type. One of `kafka`or `kinesis`.|Yes| +|`spec`|Object|The container object for the supervisor configuration.|Yes| +|`spec.dataSchema`|Object|The schema for the indexing task to use during ingestion. See [`dataSchema`](../ingestion/ingestion-spec.md#dataschema) for more information.|Yes| +|`spec.ioConfig`|Object|The I/O configuration object to define the connection and I/O-related settings for the supervisor and indexing tasks.|Yes| +|`spec.tuningConfig`|Object|The tuning configuration object to define performance-related settings for the supervisor and indexing tasks.|No| + +### I/O configuration + +The following table outlines the `ioConfig` configuration properties that apply to both Apache Kafka and Amazon Kinesis ingestion methods. +For configuration properties specific to Apache Kafka and Amazon Kinesis, see [Kafka I/O configuration](kafka-ingestion.md#io-configuration) and [Kinesis I/O configuration](kinesis-ingestion.md#io-configuration) respectively. + +|Property|Type|Description|Required|Default| +|--------|----|-----------|--------|-------| +|`inputFormat`|Object|The [input format](../ingestion/data-formats.md#input-format) to define input data parsing.|Yes|| +|`autoScalerConfig`|Object|Defines auto scaling behavior for ingestion tasks. See [Task autoscaler](#task-autoscaler) for more information.|No|null| +|`taskCount`|Integer|The maximum number of reading tasks in a replica set. Multiply `taskCount` and replicas to measure the maximum number of reading tasks. The total number of tasks, reading and publishing, is higher than the maximum number of reading tasks. See [Capacity planning](../ingestion/supervisor.md#capacity-planning) for more details. When `taskCount` is greater than the number of Kafka partitions or Kinesis shards, the actual number of reading tasks is less than the `taskCount` value.|No|1| +|`replicas`|Integer|The number of replica sets, where 1 is a single set of tasks (no replication). Druid always assigns replicate tasks to different workers to provide resiliency against process failure.|No|1| +|`taskDuration`|ISO 8601 period|The length of time before tasks stop reading and begin publishing segments.|No|`PT1H`| +|`startDelay`|ISO 8601 period|The period to wait before the supervisor starts managing tasks.|No|`PT5S`| +|`period`|ISO 8601 period|Determines how often the supervisor executes its management logic. Note that the supervisor also runs in response to certain events, such as tasks succeeding, failing, and reaching their task duration. The `period` value specifies the maximum time between iterations.|No|`PT30S`| +|`completionTimeout`|ISO 8601 period|The length of time to wait before declaring a publishing task as failed and terminating it. If the value is too low, tasks may never publish. The publishing clock for a task begins roughly after `taskDuration` elapses.|No|`PT30M`| +|`lateMessageRejectionStartDateTime`|ISO 8601 date time|Configures tasks to reject messages with timestamps earlier than this date time. For example, if this property is set to `2016-01-01T11:00Z` and the supervisor creates a task at `2016-01-01T12:00Z`, Druid drops messages with timestamps earlier than `2016-01-01T11:00Z`. This can prevent concurrency issues if your data stream has late messages and you have multiple pipelines that need to operate on the same segments, such as a realtime and a nightly batch ingestion pipeline.|No|| +|`lateMessageRejectionPeriod`|ISO 8601 period|Configures tasks to reject messages with timestamps earlier than this period before the task was created. For example, if this property is set to `PT1H` and the supervisor creates a task at `2016-01-01T12:00Z`, Druid drops messages with timestamps earlier than `2016-01-01T11:00Z`. This may help prevent concurrency issues if your data stream has late messages and you have multiple pipelines that need to operate on the same segments, such as a streaming and a nightly batch ingestion pipeline. You can specify only one of the late message rejection properties.|No|| +|`earlyMessageRejectionPeriod`|ISO 8601 period|Configures tasks to reject messages with timestamps later than this period after the task reached its task duration. For example, if this property is set to `PT1H`, the task duration is set to `PT1H` and the supervisor creates a task at `2016-01-01T12:00Z`, Druid drops messages with timestamps later than `2016-01-01T14:00Z`. Tasks sometimes run past their task duration, such as in cases of supervisor failover. Setting `earlyMessageRejectionPeriod` too low may cause Druid to drop messages unexpectedly whenever a task runs past its originally configured task duration.|No|| + +#### Task autoscaler + +You can optionally configure autoscaling behavior for ingestion tasks using the `autoScalerConfig` property of the `ioConfig` object. + +The following table outlines the configuration properties for `autoScalerConfig`: + +|Property|Description|Required|Default| +|--------|-----------|--------|-------| +|`enableTaskAutoScaler`|Enables the autoscaler. If not specified, Druid disables the autoscaler even when `autoScalerConfig` is not null.|No|`false`| +|`taskCountMax`|The maximum number of ingestion tasks. Must be greater than or equal to `taskCountMin`. If `taskCountMax` is greater than the number of Kafka partitions or Kinesis shards, Druid set the maximum number of reading tasks to the number of Kafka partitions or Kinesis shards and ignores `taskCountMax`.|Yes|| +|`taskCountMin`|The minimum number of ingestion tasks. When you enable the autoscaler, Druid ignores the value of `taskCount` in `ioConfig` and starts with the `taskCountMin` number of tasks to launch.|Yes|| +|`minTriggerScaleActionFrequencyMillis`|The minimum time interval between two scale actions.| No|600000| +|`autoScalerStrategy`|The algorithm of autoscaler. Druid only supports the `lagBased` strategy. See [Autoscaler strategy](#autoscaler-strategy) for more information.|No|`lagBased`| + +##### Autoscaler strategy + +:::info +Unlike the Kafka indexing service, Kinesis reports lag metrics measured in time difference in milliseconds between the current sequence number and latest sequence number, rather than message count. +::: + +The following table outlines the configuration properties related to the `lagBased` autoscaler strategy: + +|Property|Description|Required|Default| +|--------|-----------|--------|-------| +|`lagCollectionIntervalMillis`|The time period during which Druid collects lag metric points.|No|30000| +|`lagCollectionRangeMillis`|The total time window of lag collection. Use with `lagCollectionIntervalMillis` to specify the intervals at which to collect lag metric points.|No|600000| +|`scaleOutThreshold`|The threshold of scale out action. |No|6000000| +|`triggerScaleOutFractionThreshold`|Enables scale out action if `triggerScaleOutFractionThreshold` percent of lag points is higher than `scaleOutThreshold`.|No|0.3| +|`scaleInThreshold`|The threshold of scale in action.|No|1000000| +|`triggerScaleInFractionThreshold`|Enables scale in action if `triggerScaleInFractionThreshold` percent of lag points is lower than `scaleOutThreshold`.|No|0.9| +|`scaleActionStartDelayMillis`|The number of milliseconds to delay after the supervisor starts before the first scale logic check.|No|300000| +|`scaleActionPeriodMillis`|The frequency in milliseconds to check if a scale action is triggered.|No|60000| +|`scaleInStep`|The number of tasks to reduce at once when scaling down.|No|1| +|`scaleOutStep`|The number of tasks to add at once when scaling out.|No|2| + +The following example shows a supervisor spec with `lagBased` autoscaler: + +
+ Click to view the example + +```json +{ + "type": "kinesis", + "dataSchema": { + "dataSource": "metrics-kinesis", + "timestampSpec": { + "column": "timestamp", + "format": "auto" + }, + "dimensionsSpec": { + "dimensions": [], + "dimensionExclusions": [ + "timestamp", + "value" + ] + }, + "metricsSpec": [ + { + "name": "count", + "type": "count" + }, + { + "name": "value_sum", + "fieldName": "value", + "type": "doubleSum" + }, + { + "name": "value_min", + "fieldName": "value", + "type": "doubleMin" + }, + { + "name": "value_max", + "fieldName": "value", + "type": "doubleMax" + } + ], + "granularitySpec": { + "type": "uniform", + "segmentGranularity": "HOUR", + "queryGranularity": "NONE" + } + }, + "ioConfig": { + "stream": "metrics", + "autoScalerConfig": { + "enableTaskAutoScaler": true, + "taskCountMax": 6, + "taskCountMin": 2, + "minTriggerScaleActionFrequencyMillis": 600000, + "autoScalerStrategy": "lagBased", + "lagCollectionIntervalMillis": 30000, + "lagCollectionRangeMillis": 600000, + "scaleOutThreshold": 600000, + "triggerScaleOutFractionThreshold": 0.3, + "scaleInThreshold": 100000, + "triggerScaleInFractionThreshold": 0.9, + "scaleActionStartDelayMillis": 300000, + "scaleActionPeriodMillis": 60000, + "scaleInStep": 1, + "scaleOutStep": 2 + }, + "inputFormat": { + "type": "json" + }, + "endpoint": "kinesis.us-east-1.amazonaws.com", + "taskCount": 1, + "replicas": 1, + "taskDuration": "PT1H" + }, + "tuningConfig": { + "type": "kinesis", + "maxRowsPerSegment": 5000000 + } +} +``` +
+ +### Tuning configuration + +The `tuningConfig` object is optional. If you don't specify the `tuningConfig` object, Druid uses the default configuration settings. + +The following table outlines the `tuningConfig` configuration properties that apply to both Apache Kafka and Amazon Kinesis ingestion methods. +For configuration properties specific to Apache Kafka and Amazon Kinesis, see [Kafka tuning configuration](kafka-ingestion.md#tuning-configuration) and [Kinesis tuning configuration](kinesis-ingestion.md#tuning-configuration) respectively. + +|Property|Type|Description|Required|Default| +|--------|----|-----------|--------|-------| +|`type`|String|The tuning type code for the ingestion method. One of `kafka` or `kinesis`.|Yes|| +|`maxRowsInMemory`|Integer|The number of rows to aggregate before persisting. This number represents the post-aggregation rows. It is not equivalent to the number of input events, but the resulting number of aggregated rows. Druid uses `maxRowsInMemory` to manage the required JVM heap size. The maximum heap memory usage for indexing scales is `maxRowsInMemory * (2 + maxPendingPersists)`. Normally, you do not need to set this, but depending on the nature of data, if rows are short in terms of bytes, you may not want to store a million rows in memory and this value should be set.|No|150000| +|`maxBytesInMemory`|Long|The number of bytes to aggregate in heap memory before persisting. This is based on a rough estimate of memory usage and not actual usage. Normally, this is computed internally. The maximum heap memory usage for indexing is `maxBytesInMemory * (2 + maxPendingPersists)`.|No|One-sixth of max JVM memory| +|`skipBytesInMemoryOverheadCheck`|Boolean|The calculation of `maxBytesInMemory` takes into account overhead objects created during ingestion and each intermediate persist. To exclude the bytes of these overhead objects from the `maxBytesInMemory` check, set `skipBytesInMemoryOverheadCheck` to `true`.|No|`false`| +|`maxRowsPerSegment`|Integer|The number of rows to store in a segment. This number is post-aggregation rows. Handoff occurs when `maxRowsPerSegment` or `maxTotalRows` is reached or every `intermediateHandoffPeriod`, whichever happens first.|No|5000000| +|`maxTotalRows`|Long|The number of rows to aggregate across all segments; this number is post-aggregation rows. Handoff happens either if `maxRowsPerSegment` or `maxTotalRows` is reached or every `intermediateHandoffPeriod`, whichever happens earlier.|No|20000000| +|`intermediateHandoffPeriod`|ISO 8601 period|The period that determines how often tasks hand off segments. Handoff occurs if `maxRowsPerSegment` or `maxTotalRows` is reached or every `intermediateHandoffPeriod`, whichever happens first.|No|`P2147483647D`| +|`intermediatePersistPeriod`|ISO 8601 period|The period that determines the rate at which intermediate persists occur.|No|`PT10M`| +|`maxPendingPersists`|Integer|Maximum number of persists that can be pending but not started. If a new intermediate persist exceeds this limit, Druid blocks ingestion until the currently running persist finishes. One persist can be running concurrently with ingestion, and none can be queued up. The maximum heap memory usage for indexing scales is `maxRowsInMemory * (2 + maxPendingPersists)`.|No|0| +|`indexSpec`|Object|Defines segment storage format options to use at indexing time. See [IndexSpec](../ingestion/ingestion-spec.md#indexspec) for more information.|No|| +|`indexSpecForIntermediatePersists`|Object|Defines segment storage format options to use at indexing time for intermediate persisted temporary segments. You can use `indexSpecForIntermediatePersists` to disable dimension/metric compression on intermediate segments to reduce memory required for final merging. However, disabling compression on intermediate segments might increase page cache use while they are used before getting merged into final segment published.|No|| +|`reportParseExceptions`|Boolean|DEPRECATED. If `true`, Druid throws exceptions encountered during parsing causing ingestion to halt. If `false`, Druid skips unparseable rows and fields. Setting `reportParseExceptions` to `true` overrides existing configurations for `maxParseExceptions` and `maxSavedParseExceptions`, setting `maxParseExceptions` to 0 and limiting `maxSavedParseExceptions` to not more than 1.|No|`false`| +|`handoffConditionTimeout`|Long|Number of milliseconds to wait for segment handoff. Set to a value >= 0, where 0 means to wait indefinitely.|No|900000 (15 minutes) for Kafka. 0 for Kinesis.| +|`resetOffsetAutomatically`|Boolean|Resets partitions when the sequence number is unavailable. If set to `true`, Druid resets partitions to the earliest or latest Kafka sequence number or Kinesis offset, based on the value of `useEarliestSequenceNumber` or `useEarliestOffset` (earliest if `true`, latest if `false`). If set to `false`, the exception bubbles up causing tasks to fail and ingestion to halt. If this occurs, manual intervention is required to correct the situation, potentially through [resetting the supervisor](../api-reference/supervisor-api.md#reset-a-supervisor).|No|`false`| +|`workerThreads`|Integer|The number of threads that the supervisor uses to handle requests/responses for worker tasks, along with any other internal asynchronous operation.|No|`min(10, taskCount)`| +|`chatRetries`|Integer|The number of times Druid retries HTTP requests to indexing tasks before considering tasks unresponsive.|No|8| +|`httpTimeout`|ISO 8601 period|The period of time to wait for a HTTP response from an indexing task.|No|`PT10S`| +|`shutdownTimeout`|ISO 8601 period|The period of time to wait for the supervisor to attempt a graceful shutdown of tasks before exiting.|No|`PT80S`| +|`offsetFetchPeriod`|ISO 8601 period|Determines how often the supervisor queries the streaming source and the indexing tasks to fetch current offsets and calculate lag. If the user-specified value is below the minimum value of `PT5S`, the supervisor ignores the value and uses the minimum value instead.|No|`PT30S`| +|`segmentWriteOutMediumFactory`|Object|The segment write-out medium to use when creating segments. See [Additional Peon configuration: SegmentWriteOutMediumFactory](../configuration/index.md#segmentwriteoutmediumfactory) for explanation and available options.|No|If not specified, Druid uses the value from `druid.peon.defaultSegmentWriteOutMediumFactory.type`.| +|`logParseExceptions`|Boolean|If `true`, Druid logs an error message when a parsing exception occurs, containing information about the row where the error occurred.|No|`false`| +|`maxParseExceptions`|Integer|The maximum number of parse exceptions that can occur before the task halts ingestion and fails. Overridden if `reportParseExceptions` is set.|No|unlimited| +|`maxSavedParseExceptions`|Integer|When a parse exception occurs, Druid keeps track of the most recent parse exceptions. `maxSavedParseExceptions` limits the number of saved exception instances. These saved exceptions are available after the task finishes in the [task completion report](../ingestion/tasks.md#task-reports). Overridden if `reportParseExceptions` is set.|No|0| + +## Start a supervisor + +Druid starts a new supervisor when you submit a supervisor spec. +You can submit the supervisor spec using the Druid console [data loader](../operations/web-console.md#data-loader) or by calling the [Supervisor API](../api-reference/supervisor-api.md). + +The following screenshot shows the [Supervisors](../operations/web-console.md#supervisors) view of the Druid web console for a cluster with two supervisors: + +![Supervisors view](../assets/supervisor-view.png) + +Once started, the supervisor persists in the configured metadata database. There can only be one supervisor per datasource. Submitting a second supervisor spec for the same datasource overwrites the previous one. + +When an Overlord gains leadership, either by being started or as a result of another Overlord failing, it spawns a supervisor for each supervisor spec in the metadata database. The supervisor then discovers running indexing tasks and attempts to adopt them if they are compatible with the supervisor's configuration. If they are not compatible, the tasks are terminated and the supervisor creates a new set of tasks. This way, the supervisor ingestion tasks persist across Overlord restarts and failovers. + +### Schema and configuration changes + +Schema and configuration changes are handled by submitting the new supervisor spec. The Overlord initiates a graceful shutdown of the existing supervisor. The running supervisor signals its tasks to stop reading and begin publishing, exiting itself. Druid then uses the provided configuration to create a new supervisor. Druid submits a new schema while retaining existing publishing tasks and starts new tasks at the previous task offsets. +This way, configuration changes can be applied without requiring any pause in ingestion. + +## Status report + +The supervisor status report contains the state of the supervisor tasks and an array of recently thrown exceptions reported as `recentErrors`. +You can control the maximum size of the exceptions using the `druid.supervisor.maxStoredExceptionEvents` configuration. + +To view the supervisor status in the web console, navigate to the **Supervisors** view and click the supervisor ID to open the **Supervisor** dialog. +Click **Status** in the left navigation pane to display the status: + +![Supervisors info dialog](../assets/supervisor-info-dialog.png) + +The following example shows the status of a supervisor with the name `social_media`: + +
+ Click to view the example + +```json +{ + "dataSource": "social_media", + "stream": "social_media", + "partitions": 1, + "replicas": 1, + "durationSeconds": 3600, + "activeTasks": [ + { + "id": "index_kafka_social_media_8ff3096f21fe448_jajnddno", + "startingOffsets": { + "0": 0 + }, + "startTime": "2024-01-30T21:21:41.696Z", + "remainingSeconds": 479, + "type": "ACTIVE", + "currentOffsets": { + "0": 50000 + }, + "lag": { + "0": 0 + } + } + ], + "publishingTasks": [], + "latestOffsets": { + "0": 50000 + }, + "minimumLag": { + "0": 0 + }, + "aggregateLag": 0, + "offsetsLastUpdated": "2024-01-30T22:13:19.335Z", + "suspended": false, + "healthy": true, + "state": "RUNNING", + "detailedState": "RUNNING", + "recentErrors": [] +} +``` +
+ +The status report contains two properties that correspond to the state of the supervisor: `state` and `detailedState`. The `state` property contains a small number of generic states that apply to any type of supervisor. The `detailedState` property contains a more descriptive, implementation-specific state that may provide more insight into the supervisor's activities. + +Possible `state` values are `PENDING`, `RUNNING`, `SUSPENDED`, `STOPPING`, `UNHEALTHY_SUPERVISOR`, and `UNHEALTHY_TASKS`. + +The following table lists `detailedState` values and their corresponding `state` mapping: + +|`detailedState`|`state`|Description| +|--------------|-------------------|-----------| +|`UNHEALTHY_SUPERVISOR`|`UNHEALTHY_SUPERVISOR`|The supervisor encountered errors on previous `druid.supervisor.unhealthinessThreshold` iterations.| +|`UNHEALTHY_TASKS`|`UNHEALTHY_TASKS`|The last `druid.supervisor.taskUnhealthinessThreshold` tasks all failed.| +|`UNABLE_TO_CONNECT_TO_STREAM`|`UNHEALTHY_SUPERVISOR`|The supervisor is encountering connectivity issues with the stream and hasn't successfully connected in the past.| +|`LOST_CONTACT_WITH_STREAM`|`UNHEALTHY_SUPERVISOR`|The supervisor is encountering connectivity issues with the stream but has successfully connected in the past.| +|`PENDING` (first iteration only)|`PENDING`|The supervisor has been initialized but hasn't started connecting to the stream.| +|`CONNECTING_TO_STREAM` (first iteration only)|`RUNNING`|The supervisor is trying to connect to the stream and update partition data.| +|`DISCOVERING_INITIAL_TASKS` (first iteration only)|`RUNNING`|The supervisor is discovering already-running tasks.| +|`CREATING_TASKS` (first iteration only)|`RUNNING`|The supervisor is creating tasks and discovering state.| +|`RUNNING`|`RUNNING`|The supervisor has started tasks and is waiting for `taskDuration` to elapse.| +|`IDLE`|`IDLE`|The supervisor is not creating tasks since the input stream has not received any new data and all the existing data is read.| +|`SUSPENDED`|`SUSPENDED`|The supervisor is suspended.| +|`STOPPING`|`STOPPING`|The 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 units of parallelism, such as Kinesis shards or Kafka partitions, and determine the starting sequence number or offset for each unit (either based on the last processed sequence number or offset if continuing, or starting from the beginning or ending of the stream if this is a new stream). +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 the view of the state of the tasks under 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. + +The `detailedState` property shows additional values (marked with "first iteration only" in the preceding table) 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. For example, if the supervisor cannot connect to +the stream, if it's unable to read from the stream, or cannot 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 stopped, suspended, or hits a failure threshold and transitions to an unhealthy state. + +:::info +For the Kafka indexing service, the consumer lag per partition may be reported as negative values if the supervisor hasn't received the latest offset response from Kafka. The aggregate lag value will always be >= 0. +::: + +## SUPERVISORS system table + +Druid exposes system information through special system schemas. You can query the `sys.supervisors` table to retrieve information about the supervisor internals. +The following example shows how to retrieve supervisor tasks information filtered by health status: + +```sql +SELECT * FROM sys.supervisors WHERE healthy=0; +``` + +For more information on the supervisors system table, see [SUPERVISORS table](../querying/sql-metadata-tables.md#supervisors-table). + +## Capacity planning + +Indexing tasks run on MiddleManagers and are 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, such as batch processing, streaming tasks, and merging tasks. If your workers run out of capacity, indexing tasks 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 the stream purges those sequence numbers. + +A running task can be in one of two states: reading or publishing. A task remains in reading state for the period defined in `taskDuration`, at which point it transitions to publishing state. A task remains in publishing state for as long as it takes to generate segments, push segments to deep storage, and have them loaded and served by a Historical service or until `completionTimeout` elapses. + +The number of reading tasks is controlled by `replicas` and `taskCount`. In general, there are `replicas * taskCount` reading tasks. An exception occurs if `taskCount` is over the number of shards in Kinesis or partitions in Kafka, in which case Druid uses the number of shards or partitions. When `taskDuration` elapses, these tasks transition to publishing state and `replicas * taskCount` new reading tasks are created. To allow for reading tasks and publishing tasks to run concurrently, there should be a minimum capacity of: + +```text +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, load on Historical) is greater than `taskDuration`. This is a valid and correct scenario 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. + +## Learn more + +See the following topics for more information: + +* [Supervisor API](../api-reference/supervisor-api.md) for how to manage and monitor supervisors using the API. +* [Apache Kafka ingestion](../ingestion/kafka-ingestion.md) to learn about ingesting data from an Apache Kafka stream. +* [Amazon Kinesis ingestion](../ingestion/kinesis-ingestion.md) to learn about ingesting data from an Amazon Kinesis stream. \ No newline at end of file diff --git a/docs/ingestion/tasks.md b/docs/ingestion/tasks.md index 4b3081cef911..d9f0758ede6a 100644 --- a/docs/ingestion/tasks.md +++ b/docs/ingestion/tasks.md @@ -438,7 +438,7 @@ Task storage sizes are configured through a combination of three properties: While it seems like one task might use multiple directories, only one directory from the list of base directories will be used for any given task, as such, each task is only given a singular directory for scratch space. -The actual amount of memory assigned to any given task is computed by determining the largest size that enables all task slots to be given an equivalent amount of disk storage. For example, with 5 slots, 2 directories (A and B) and a size of 300 GB, 3 slots would be given to directory A, 2 slots to directory B and each slot would be allowed 100 GB +The actual amount of memory assigned to any given task is computed by determining the largest size that enables all task slots to be given an equivalent amount of disk storage. For example, with 5 slots, 2 directories (A and B) and a size of 300 GB, 3 slots would be given to directory A, 2 slots to directory B and each slot would be allowed 100 GB ## All task types @@ -453,12 +453,12 @@ See [Hadoop-based ingestion](hadoop.md). ### `index_kafka` Submitted automatically, on your behalf, by a -[Kafka-based ingestion supervisor](../development/extensions-core/kafka-ingestion.md). +[Kafka-based ingestion supervisor](../ingestion/kafka-ingestion.md). ### `index_kinesis` Submitted automatically, on your behalf, by a -[Kinesis-based ingestion supervisor](../development/extensions-core/kinesis-ingestion.md). +[Kinesis-based ingestion supervisor](../ingestion/kinesis-ingestion.md). ### `compact` diff --git a/docs/ingestion/tranquility.md b/docs/ingestion/tranquility.md index f66464456188..9cc0636fd6cc 100644 --- a/docs/ingestion/tranquility.md +++ b/docs/ingestion/tranquility.md @@ -30,7 +30,7 @@ release. It may still work with the latest Druid servers, but not all features a due to limitations of older Druid APIs on the Tranquility side. For new projects that require streaming ingestion, we recommend using Druid's native support for -[Apache Kafka](../development/extensions-core/kafka-ingestion.md) or -[Amazon Kinesis](../development/extensions-core/kinesis-ingestion.md). +[Apache Kafka](../ingestion/kafka-ingestion.md) or +[Amazon Kinesis](../ingestion/kinesis-ingestion.md). For more details, check out the [Tranquility GitHub page](https://github.com/druid-io/tranquility/). diff --git a/docs/operations/basic-cluster-tuning.md b/docs/operations/basic-cluster-tuning.md index 7bc7c1e2761d..e48b4b0c4f60 100644 --- a/docs/operations/basic-cluster-tuning.md +++ b/docs/operations/basic-cluster-tuning.md @@ -256,7 +256,7 @@ The total memory usage of the MiddleManager + Tasks: ###### Kafka/Kinesis ingestion -If you use the [Kafka Indexing Service](../development/extensions-core/kafka-ingestion.md) or [Kinesis Indexing Service](../development/extensions-core/kinesis-ingestion.md), the number of tasks required will depend on the number of partitions and your taskCount/replica settings. +If you use the [Kafka Indexing Service](../ingestion/kafka-ingestion.md) or [Kinesis Indexing Service](../ingestion/kinesis-ingestion.md), the number of tasks required will depend on the number of partitions and your taskCount/replica settings. On top of those requirements, allocating more task slots in your cluster is a good idea, so that you have free task slots available for other tasks, such as [compaction tasks](../data-management/compaction.md). diff --git a/docs/operations/dynamic-config-provider.md b/docs/operations/dynamic-config-provider.md index a0413d856229..b641efd7a0ea 100644 --- a/docs/operations/dynamic-config-provider.md +++ b/docs/operations/dynamic-config-provider.md @@ -24,7 +24,7 @@ title: "Dynamic Config Providers" Druid relies on dynamic config providers to supply multiple related sets of credentials, secrets, and configurations within a Druid extension. Dynamic config providers are intended to eventually replace [PasswordProvider](./password-provider.md). -By default, Druid includes an environment variable dynamic config provider that supports Kafka consumer configuration in [Kafka ingestion](../development/extensions-core/kafka-ingestion.md). +By default, Druid includes an environment variable dynamic config provider that supports Kafka consumer configuration in [Kafka ingestion](../ingestion/kafka-ingestion.md). To develop a custom extension of the `DynamicConfigProvider` interface that is registered at Druid process startup, see [Adding a new DynamicConfigProvider implementation](../development/modules.md#adding-a-new-dynamicconfigprovider-implementation). diff --git a/docs/operations/metrics.md b/docs/operations/metrics.md index 8fefc8b6e133..1510f35199d6 100644 --- a/docs/operations/metrics.md +++ b/docs/operations/metrics.md @@ -201,7 +201,7 @@ field in the `context` field of the ingestion spec. `tags` is expected to be a m ### Ingestion metrics for Kafka -These metrics apply to the [Kafka indexing service](../development/extensions-core/kafka-ingestion.md). +These metrics apply to the [Kafka indexing service](../ingestion/kafka-ingestion.md). |Metric|Description|Dimensions|Normal value| |------|-----------|----------|------------| @@ -212,7 +212,7 @@ These metrics apply to the [Kafka indexing service](../development/extensions-co ### Ingestion metrics for Kinesis -These metrics apply to the [Kinesis indexing service](../development/extensions-core/kinesis-ingestion.md). +These metrics apply to the [Kinesis indexing service](../ingestion/kinesis-ingestion.md). |Metric|Description|Dimensions|Normal value| |------|-----------|----------|------------| diff --git a/docs/querying/arrays.md b/docs/querying/arrays.md index 904802c2b1fc..dbeb3ec6e028 100644 --- a/docs/querying/arrays.md +++ b/docs/querying/arrays.md @@ -42,7 +42,7 @@ The following sections describe inserting, filtering, and grouping behavior base ## Ingesting arrays ### Native batch and streaming ingestion -When using native [batch](../ingestion/native-batch.md) or streaming ingestion such as with [Apache Kafka](../development/extensions-core/kafka-ingestion.md), arrays can be ingested using the [`"auto"`](../ingestion/ingestion-spec.md#dimension-objects) type dimension schema which is shared with [type-aware schema discovery](../ingestion/schema-design.md#type-aware-schema-discovery). +When using native [batch](../ingestion/native-batch.md) or streaming ingestion such as with [Apache Kafka](../ingestion/kafka-ingestion.md), arrays can be ingested using the [`"auto"`](../ingestion/ingestion-spec.md#dimension-objects) type dimension schema which is shared with [type-aware schema discovery](../ingestion/schema-design.md#type-aware-schema-discovery). When ingesting from TSV or CSV data, you can specify the array delimiters using the `listDelimiter` field in the `inputFormat`. JSON data must be formatted as a JSON array to be ingested as an array type. JSON data does not require `inputFormat` configuration. @@ -238,7 +238,7 @@ Avoid confusing string arrays with [multi-value dimensions](multi-value-dimensio Use care during ingestion to ensure you get the type you want. -To get arrays when performing an ingestion using JSON ingestion specs, such as [native batch](../ingestion/native-batch.md) or streaming ingestion such as with [Apache Kafka](../development/extensions-core/kafka-ingestion.md), use dimension type `auto` or enable `useSchemaDiscovery`. When performing a [SQL-based ingestion](../multi-stage-query/index.md), write a query that generates arrays and set the context parameter `"arrayIngestMode": "array"`. Arrays may contain strings or numbers. +To get arrays when performing an ingestion using JSON ingestion specs, such as [native batch](../ingestion/native-batch.md) or streaming ingestion such as with [Apache Kafka](../ingestion/kafka-ingestion.md), use dimension type `auto` or enable `useSchemaDiscovery`. When performing a [SQL-based ingestion](../multi-stage-query/index.md), write a query that generates arrays and set the context parameter `"arrayIngestMode": "array"`. Arrays may contain strings or numbers. To get multi-value dimensions when performing an ingestion using JSON ingestion specs, use dimension type `string` and do not enable `useSchemaDiscovery`. When performing a [SQL-based ingestion](../multi-stage-query/index.md), wrap arrays in [`ARRAY_TO_MV`](multi-value-dimensions.md#sql-based-ingestion), which ensures you get multi-value dimensions in any `arrayIngestMode`. Multi-value dimensions can only contain strings. diff --git a/docs/querying/multi-value-dimensions.md b/docs/querying/multi-value-dimensions.md index 9680d5603974..2b33737a36fc 100644 --- a/docs/querying/multi-value-dimensions.md +++ b/docs/querying/multi-value-dimensions.md @@ -49,7 +49,7 @@ The following sections describe inserting, filtering, and grouping behavior base ## Ingestion ### Native batch and streaming ingestion -When using native [batch](../ingestion/native-batch.md) or streaming ingestion such as with [Apache Kafka](../development/extensions-core/kafka-ingestion.md), the Druid web console data loader can detect multi-value dimensions and configure the `dimensionsSpec` accordingly. +When using native [batch](../ingestion/native-batch.md) or streaming ingestion such as with [Apache Kafka](../ingestion/kafka-ingestion.md), the Druid web console data loader can detect multi-value dimensions and configure the `dimensionsSpec` accordingly. For TSV or CSV data, you can specify the multi-value delimiters using the `listDelimiter` field in the `inputFormat`. JSON data must be formatted as a JSON array to be ingested as a multi-value dimension. JSON data does not require `inputFormat` configuration. @@ -507,7 +507,7 @@ Avoid confusing string arrays with [multi-value dimensions](multi-value-dimensio Use care during ingestion to ensure you get the type you want. -To get arrays when performing an ingestion using JSON ingestion specs, such as [native batch](../ingestion/native-batch.md) or streaming ingestion such as with [Apache Kafka](../development/extensions-core/kafka-ingestion.md), use dimension type `auto` or enable `useSchemaDiscovery`. When performing a [SQL-based ingestion](../multi-stage-query/index.md), write a query that generates arrays and set the context parameter `"arrayIngestMode": "array"`. Arrays may contain strings or numbers. +To get arrays when performing an ingestion using JSON ingestion specs, such as [native batch](../ingestion/native-batch.md) or streaming ingestion such as with [Apache Kafka](../ingestion/kafka-ingestion.md), use dimension type `auto` or enable `useSchemaDiscovery`. When performing a [SQL-based ingestion](../multi-stage-query/index.md), write a query that generates arrays and set the context parameter `"arrayIngestMode": "array"`. Arrays may contain strings or numbers. To get multi-value dimensions when performing an ingestion using JSON ingestion specs, use dimension type `string` and do not enable `useSchemaDiscovery`. When performing a [SQL-based ingestion](../multi-stage-query/index.md), wrap arrays in [`ARRAY_TO_MV`](multi-value-dimensions.md#sql-based-ingestion), which ensures you get multi-value dimensions in any `arrayIngestMode`. Multi-value dimensions can only contain strings. diff --git a/docs/querying/nested-columns.md b/docs/querying/nested-columns.md index 01a86e49a78a..3641d4d46aa9 100644 --- a/docs/querying/nested-columns.md +++ b/docs/querying/nested-columns.md @@ -227,7 +227,7 @@ PARTITIONED BY ALL You can ingest nested data into Druid using the [streaming method](../ingestion/index.md#streaming)—for example, from a Kafka topic. -When you [define your supervisor spec](../development/extensions-core/kafka-ingestion.md#define-a-supervisor-spec), include a dimension with type `json` for each nested column. For example, the following supervisor spec from the [Kafka ingestion tutorial](../tutorials/tutorial-kafka.md) contains dimensions for the nested columns `event`, `agent`, and `geo_ip` in datasource `kttm-kafka`. +When you [define your supervisor spec](../ingestion/supervisor.md#start-a-supervisor), include a dimension with type `json` for each nested column. For example, the following supervisor spec from the [Kafka ingestion tutorial](../tutorials/tutorial-kafka.md) contains dimensions for the nested columns `event`, `agent`, and `geo_ip` in datasource `kttm-kafka`. ```json { diff --git a/docs/querying/sql-metadata-tables.md b/docs/querying/sql-metadata-tables.md index 331774d34abc..4fbbfb2ff4c0 100644 --- a/docs/querying/sql-metadata-tables.md +++ b/docs/querying/sql-metadata-tables.md @@ -299,8 +299,8 @@ The supervisors table provides information about supervisors. |Column|Type|Notes| |------|-----|-----| |supervisor_id|VARCHAR|Supervisor task identifier| -|state|VARCHAR|Basic state of the supervisor. Available states: `UNHEALTHY_SUPERVISOR`, `UNHEALTHY_TASKS`, `PENDING`, `RUNNING`, `SUSPENDED`, `STOPPING`. Check [Kafka Docs](../development/extensions-core/kafka-supervisor-operations.md) for details.| -|detailed_state|VARCHAR|Supervisor specific state. (See documentation of the specific supervisor for details, e.g. [Kafka](../development/extensions-core/kafka-ingestion.md) or [Kinesis](../development/extensions-core/kinesis-ingestion.md))| +|state|VARCHAR|Basic state of the supervisor. Available states: `UNHEALTHY_SUPERVISOR`, `UNHEALTHY_TASKS`, `PENDING`, `RUNNING`, `SUSPENDED`, `STOPPING`. See [Supervisor reference](../ingestion/supervisor.md) for more information.| +|detailed_state|VARCHAR|Supervisor specific state. See documentation of the specific supervisor for details: [Kafka](../ingestion/kafka-ingestion.md) or [Kinesis](../ingestion/kinesis-ingestion.md).| |healthy|BIGINT|Boolean represented as long type where 1 = true, 0 = false. 1 indicates a healthy supervisor| |type|VARCHAR|Type of supervisor, e.g. `kafka`, `kinesis` or `materialized_view`| |source|VARCHAR|Source of the supervisor, e.g. Kafka topic or Kinesis stream| diff --git a/docs/tutorials/tutorial-kafka.md b/docs/tutorials/tutorial-kafka.md index 9e74f467c4f3..4bad708d57d4 100644 --- a/docs/tutorials/tutorial-kafka.md +++ b/docs/tutorials/tutorial-kafka.md @@ -295,6 +295,4 @@ Check out the [Querying data tutorial](../tutorials/tutorial-query.md) to run so For more information, see the following topics: -- [Apache Kafka ingestion](../development/extensions-core/kafka-ingestion.md) for more information on loading data from Kafka streams. -- [Apache Kafka supervisor reference](../development/extensions-core/kafka-supervisor-reference.md) for Kafka supervisor configuration information. -- [Apache Kafka supervisor operations reference](../development/extensions-core/kafka-supervisor-operations.md) for information on running and maintaining Kafka supervisors for Druid. +- [Apache Kafka ingestion](../ingestion/kafka-ingestion.md) for information on loading data from Kafka streams and maintaining Kafka supervisors for Druid. diff --git a/website/redirects.js b/website/redirects.js index db3160513e66..bddbbc95893f 100644 --- a/website/redirects.js +++ b/website/redirects.js @@ -118,9 +118,15 @@ const Redirects=[ "from": [ "/docs/latest/development/community-extensions/kafka-simple.html", "/docs/latest/development/community-extensions/rabbitmq.html", - "/docs/latest/development/kafka-simple-consumer-firehose.html" + "/docs/latest/development/kafka-simple-consumer-firehose.html", + "/docs/latest/development/extensions-core/kafka-supervisor-operations.html", + "/docs/latest/development/extensions-core/kafka-supervisor-reference.html" ], - "to": "/docs/latest/development/extensions-core/kafka-ingestion" + "to": "/docs/latest/ingestion/kafka-ingestion" + }, + { + "from": "/docs/latest/development/extensions-core/kinesis-ingestion.html", + "to": "/docs/latest/ingestion/kinesis-ingestion" }, { "from": "/docs/latest/development/extensions-contrib/orc.html", diff --git a/website/sidebars.json b/website/sidebars.json index d007ffb0c24d..e3178726ce5e 100644 --- a/website/sidebars.json +++ b/website/sidebars.json @@ -79,10 +79,10 @@ "type": "category", "label": "Streaming", "items": [ - "development/extensions-core/kafka-ingestion", - "development/extensions-core/kafka-supervisor-reference", - "development/extensions-core/kafka-supervisor-operations", - "development/extensions-core/kinesis-ingestion" + "ingestion/streaming", + "ingestion/supervisor", + "ingestion/kafka-ingestion", + "ingestion/kinesis-ingestion" ] }, {