You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2021/08/25 19:21:59 UTC

[GitHub] [druid] jihoonson commented on a change in pull request #11624: updates Kafka and Kinesis to use . Fixes some typos and other style i…

jihoonson commented on a change in pull request #11624:
URL: https://github.com/apache/druid/pull/11624#discussion_r696025116



##########
File path: docs/development/extensions-core/kafka-ingestion.md
##########
@@ -23,129 +23,130 @@ sidebar_label: "Apache Kafka"
   ~ under the License.
   -->
 
+When you enable the Kafka indexing service, you can configure *supervisors* on the Overlord to manage manage the creation and lifetime of Kafka indexing tasks. These 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.
 
-The Kafka indexing service enables the configuration of *supervisors* on the Overlord, which facilitate ingestion from
-Kafka by managing the creation and lifetime of Kafka indexing tasks. These indexing tasks read events using Kafka's own
-partition and offset mechanism and are therefore able to provide guarantees of exactly-once ingestion.
-The supervisor oversees the state of the indexing tasks to coordinate handoffs,
-manage failures, and ensure that the scalability and replication requirements are maintained.
+To use the Kafka indexing service, load the `druid-kafka-indexing-service` core Apache Druid extension. See [Including Extensions](../../development/extensions.md#loading-extensions)).
 
-This service is provided in the `druid-kafka-indexing-service` core Apache Druid extension (see
-[Including Extensions](../../development/extensions.md#loading-extensions)).
+This topic covers the ingestion spec for Kafka. For a general `ingestionSpec` reference, see [Ingestion specs](../../ingestion/ingestion-spec.md). For a walk-through, check out the [Loading from Apache Kafka](../../tutorials/tutorial-kafka.md) tutorial.
 
-> The Kafka indexing service supports transactional topics which were introduced in Kafka 0.11.x. It is the default behavior of Druid and make the
-> Kafka consumer that Druid uses incompatible with older brokers. Ensure that your Kafka brokers are version 0.11.x or
-> better before using this functionality. Refer [Kafka upgrade guide](https://kafka.apache.org/documentation/#upgrade)
-> if you are using older version of Kafka brokers.
-> In addition, users could set `isolation.level` `read_uncommitted` in `consumerProperties`, if don't need Druid to consume transactional topics or need Druid to consume older versions of Kafka.
-> Make sure offsets are sequential, since there is no offset gap check in Druid anymore.
+## 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).
 
-> If your Kafka cluster enables consumer-group based ACLs, you can set `group.id` in `consumerProperties` to override the default auto generated group id.
+Additionally, you can set `isolation.level` `read_uncommitted` in `consumerProperties` if either:
+- You don't need Druid to consume transactional topics.
+- You need Druid to consume older versions of Kafka.
 
-## Tutorial
+Make sure offsets are sequential, since there is no offset gap check in Druid anymore.

Review comment:
       I think this applies only in one of the above cases (non-transactional topic or older kafka versions), so it could be better to make it more obvious.

##########
File path: docs/development/extensions-core/kafka-ingestion.md
##########
@@ -23,129 +23,130 @@ sidebar_label: "Apache Kafka"
   ~ under the License.
   -->
 
+When you enable the Kafka indexing service, you can configure *supervisors* on the Overlord to manage manage the creation and lifetime of Kafka indexing tasks. These 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.
 
-The Kafka indexing service enables the configuration of *supervisors* on the Overlord, which facilitate ingestion from
-Kafka by managing the creation and lifetime of Kafka indexing tasks. These indexing tasks read events using Kafka's own
-partition and offset mechanism and are therefore able to provide guarantees of exactly-once ingestion.
-The supervisor oversees the state of the indexing tasks to coordinate handoffs,
-manage failures, and ensure that the scalability and replication requirements are maintained.
+To use the Kafka indexing service, load the `druid-kafka-indexing-service` core Apache Druid extension. See [Including Extensions](../../development/extensions.md#loading-extensions)).
 
-This service is provided in the `druid-kafka-indexing-service` core Apache Druid extension (see
-[Including Extensions](../../development/extensions.md#loading-extensions)).
+This topic covers the ingestion spec for Kafka. For a general `ingestionSpec` reference, see [Ingestion specs](../../ingestion/ingestion-spec.md). For a walk-through, check out the [Loading from Apache Kafka](../../tutorials/tutorial-kafka.md) tutorial.
 
-> The Kafka indexing service supports transactional topics which were introduced in Kafka 0.11.x. It is the default behavior of Druid and make the
-> Kafka consumer that Druid uses incompatible with older brokers. Ensure that your Kafka brokers are version 0.11.x or
-> better before using this functionality. Refer [Kafka upgrade guide](https://kafka.apache.org/documentation/#upgrade)
-> if you are using older version of Kafka brokers.
-> In addition, users could set `isolation.level` `read_uncommitted` in `consumerProperties`, if don't need Druid to consume transactional topics or need Druid to consume older versions of Kafka.
-> Make sure offsets are sequential, since there is no offset gap check in Druid anymore.
+## 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).
 
-> If your Kafka cluster enables consumer-group based ACLs, you can set `group.id` in `consumerProperties` to override the default auto generated group id.
+Additionally, you can set `isolation.level` `read_uncommitted` in `consumerProperties` if either:

Review comment:
       ```suggestion
   Additionally, you can set `isolation.level` to `read_uncommitted` in `consumerProperties` if either:
   ```

##########
File path: docs/development/extensions-core/kafka-ingestion.md
##########
@@ -23,129 +23,130 @@ sidebar_label: "Apache Kafka"
   ~ under the License.
   -->
 
+When you enable the Kafka indexing service, you can configure *supervisors* on the Overlord to manage manage the creation and lifetime of Kafka indexing tasks. These 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.
 
-The Kafka indexing service enables the configuration of *supervisors* on the Overlord, which facilitate ingestion from
-Kafka by managing the creation and lifetime of Kafka indexing tasks. These indexing tasks read events using Kafka's own
-partition and offset mechanism and are therefore able to provide guarantees of exactly-once ingestion.
-The supervisor oversees the state of the indexing tasks to coordinate handoffs,
-manage failures, and ensure that the scalability and replication requirements are maintained.
+To use the Kafka indexing service, load the `druid-kafka-indexing-service` core Apache Druid extension. See [Including Extensions](../../development/extensions.md#loading-extensions)).
 
-This service is provided in the `druid-kafka-indexing-service` core Apache Druid extension (see
-[Including Extensions](../../development/extensions.md#loading-extensions)).
+This topic covers the ingestion spec for Kafka. For a general `ingestionSpec` reference, see [Ingestion specs](../../ingestion/ingestion-spec.md). For a walk-through, check out the [Loading from Apache Kafka](../../tutorials/tutorial-kafka.md) tutorial.
 
-> The Kafka indexing service supports transactional topics which were introduced in Kafka 0.11.x. It is the default behavior of Druid and make the
-> Kafka consumer that Druid uses incompatible with older brokers. Ensure that your Kafka brokers are version 0.11.x or
-> better before using this functionality. Refer [Kafka upgrade guide](https://kafka.apache.org/documentation/#upgrade)
-> if you are using older version of Kafka brokers.
-> In addition, users could set `isolation.level` `read_uncommitted` in `consumerProperties`, if don't need Druid to consume transactional topics or need Druid to consume older versions of Kafka.
-> Make sure offsets are sequential, since there is no offset gap check in Druid anymore.
+## 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).
 
-> If your Kafka cluster enables consumer-group based ACLs, you can set `group.id` in `consumerProperties` to override the default auto generated group id.
+Additionally, you can set `isolation.level` `read_uncommitted` in `consumerProperties` if either:
+- You don't need Druid to consume transactional topics.
+- You need Druid to consume older versions of Kafka.
 
-## Tutorial
+Make sure offsets are sequential, since there is no offset gap check in Druid anymore.
 
-This page contains reference documentation for Apache Kafka-based ingestion.
-For a walk-through instead, check out the [Loading from Apache Kafka](../../tutorials/tutorial-kafka.md) tutorial.
+If your Kafka cluster enables consumer-group based ACLs, you can set `group.id` in `consumerProperties` to override the default auto generated group id.
 
 ## Submitting a Supervisor Spec
 
-The Kafka indexing service requires that the `druid-kafka-indexing-service` extension be loaded on both the Overlord and the
-MiddleManagers. A supervisor for a dataSource is started by submitting a supervisor spec via HTTP POST to
-`http://<OVERLORD_IP>:<OVERLORD_PORT>/druid/indexer/v1/supervisor`, for example:
+To use the Kafka indexing service, load the `druid-kafka-indexing-service` extension on both the Overlord and the MiddleManagers. Druid starts a supervisor for a dataSource when you submit a supervisor spec. You can use the following endpoint:
+
+`http://<OVERLORD_IP>:<OVERLORD_PORT>/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
 ```
 
-A sample supervisor spec is shown below:
+Where the file `supervisor-spec.json` contains a Kafka supervisor spec:
 
 ```json
 {
   "type": "kafka",
-  "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"
+  "spec": {
+    "dataSchema": {
+      "dataSource": "metrics-kafka",
+      "timestampSpec": {
+        "column": "timestamp",
+        "format": "auto"
       },
-      {
-        "name": "value_max",
-        "fieldName": "value",
-        "type": "doubleMax"
+      "dimensionsSpec": {
+        "dimensions": [],
+        "dimensionExclusions": [
+          "timestamp",
+          "value"
+       ]
+     },

Review comment:
       ```suggestion
            ]
          },
   ```

##########
File path: docs/development/extensions-core/kafka-ingestion.md
##########
@@ -23,129 +23,130 @@ sidebar_label: "Apache Kafka"
   ~ under the License.
   -->
 
+When you enable the Kafka indexing service, you can configure *supervisors* on the Overlord to manage manage the creation and lifetime of Kafka indexing tasks. These 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:

Review comment:
       ```suggestion
   When you enable the Kafka indexing service, you can configure *supervisors* on the Overlord to manage the creation and lifetime of Kafka indexing tasks. These 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:
   ```

##########
File path: docs/development/extensions-core/kinesis-ingestion.md
##########
@@ -23,87 +23,92 @@ sidebar_label: "Amazon Kinesis"
   ~ under the License.
   -->
 
+When you enable the Kinesis indexing service, you can configure *supervisors* on the Overlord to manage manage the creation and lifetime of Kafka indexing tasks. These 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.
 
-Similar to the [Kafka indexing service](./kafka-ingestion.md), the Kinesis indexing service for Apache Druid enables the configuration of *supervisors* on the Overlord. These supervisors facilitate ingestion from Kinesis by managing the creation and lifetime of Kinesis indexing tasks. These indexing tasks read events using Kinesis's own
-Shards and Sequence Number mechanism and are therefore able to provide guarantees of exactly-once ingestion.
-The supervisor oversees the state of the indexing tasks to coordinate handoffs, manage failures,
-and ensure that the scalability and replication requirements are maintained.
 
-The Kinesis indexing service is provided as the `druid-kinesis-indexing-service` core Apache Druid extension (see
+To use the Kinesis indexing service load the `druid-kinesis-indexing-service` core Apache Druid extension (see

Review comment:
       ```suggestion
   To use the Kinesis indexing service, load the `druid-kinesis-indexing-service` core Apache Druid extension (see
   ```

##########
File path: docs/development/extensions-core/kinesis-ingestion.md
##########
@@ -23,87 +23,92 @@ sidebar_label: "Amazon Kinesis"
   ~ under the License.
   -->
 
+When you enable the Kinesis indexing service, you can configure *supervisors* on the Overlord to manage manage the creation and lifetime of Kafka indexing tasks. These 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:

Review comment:
       ```suggestion
   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:
   ```

##########
File path: docs/development/extensions-core/kafka-ingestion.md
##########
@@ -519,22 +485,18 @@ and begin publishing their segments. A new supervisor will then be started which
 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
+### 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.
 
-#### On the Subject of Segments
+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.
 
-Each Kafka Indexing Task puts events consumed from Kafka partitions assigned to it in a single segment for each segment
-granular interval until maxRowsPerSegment, maxTotalRows or intermediateHandoffPeriod limit is reached, at this point a new partition
-for this segment granularity is created for further events. Kafka Indexing Task also does incremental hand-offs which
-means that all the segments created by a task will not be held up till the task duration is over. As soon as maxRowsPerSegment,
-maxTotalRows or intermediateHandoffPeriod limit is hit, all the segments held by the task at that point in time will be handed-off
-and new set of segments will be created for further events. This means that the task can run for longer durations of time
-without accumulating old segments locally on Middle Manager processes and it is encouraged to do so.
+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. If you encounter problems, you can schedule re-indexing tasks to merge segments together into new segments of an ideal size (in the range of ~500-700 MB per segment).

Review comment:
       I think it could be better if it specifies what problems we are talking about. Maybe 
   
   ```suggestion
   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.
   ```

##########
File path: docs/development/extensions-core/kafka-ingestion.md
##########
@@ -23,129 +23,130 @@ sidebar_label: "Apache Kafka"
   ~ under the License.
   -->
 
+When you enable the Kafka indexing service, you can configure *supervisors* on the Overlord to manage manage the creation and lifetime of Kafka indexing tasks. These 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.
 
-The Kafka indexing service enables the configuration of *supervisors* on the Overlord, which facilitate ingestion from
-Kafka by managing the creation and lifetime of Kafka indexing tasks. These indexing tasks read events using Kafka's own
-partition and offset mechanism and are therefore able to provide guarantees of exactly-once ingestion.
-The supervisor oversees the state of the indexing tasks to coordinate handoffs,
-manage failures, and ensure that the scalability and replication requirements are maintained.
+To use the Kafka indexing service, load the `druid-kafka-indexing-service` core Apache Druid extension. See [Including Extensions](../../development/extensions.md#loading-extensions)).
 
-This service is provided in the `druid-kafka-indexing-service` core Apache Druid extension (see
-[Including Extensions](../../development/extensions.md#loading-extensions)).
+This topic covers the ingestion spec for Kafka. For a general `ingestionSpec` reference, see [Ingestion specs](../../ingestion/ingestion-spec.md). For a walk-through, check out the [Loading from Apache Kafka](../../tutorials/tutorial-kafka.md) tutorial.
 
-> The Kafka indexing service supports transactional topics which were introduced in Kafka 0.11.x. It is the default behavior of Druid and make the
-> Kafka consumer that Druid uses incompatible with older brokers. Ensure that your Kafka brokers are version 0.11.x or
-> better before using this functionality. Refer [Kafka upgrade guide](https://kafka.apache.org/documentation/#upgrade)
-> if you are using older version of Kafka brokers.
-> In addition, users could set `isolation.level` `read_uncommitted` in `consumerProperties`, if don't need Druid to consume transactional topics or need Druid to consume older versions of Kafka.
-> Make sure offsets are sequential, since there is no offset gap check in Druid anymore.
+## 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).
 
-> If your Kafka cluster enables consumer-group based ACLs, you can set `group.id` in `consumerProperties` to override the default auto generated group id.
+Additionally, you can set `isolation.level` `read_uncommitted` in `consumerProperties` if either:
+- You don't need Druid to consume transactional topics.
+- You need Druid to consume older versions of Kafka.
 
-## Tutorial
+Make sure offsets are sequential, since there is no offset gap check in Druid anymore.
 
-This page contains reference documentation for Apache Kafka-based ingestion.
-For a walk-through instead, check out the [Loading from Apache Kafka](../../tutorials/tutorial-kafka.md) tutorial.
+If your Kafka cluster enables consumer-group based ACLs, you can set `group.id` in `consumerProperties` to override the default auto generated group id.
 
 ## Submitting a Supervisor Spec
 
-The Kafka indexing service requires that the `druid-kafka-indexing-service` extension be loaded on both the Overlord and the
-MiddleManagers. A supervisor for a dataSource is started by submitting a supervisor spec via HTTP POST to
-`http://<OVERLORD_IP>:<OVERLORD_PORT>/druid/indexer/v1/supervisor`, for example:
+To use the Kafka indexing service, load the `druid-kafka-indexing-service` extension on both the Overlord and the MiddleManagers. Druid starts a supervisor for a dataSource when you submit a supervisor spec. You can use the following endpoint:
+
+`http://<OVERLORD_IP>:<OVERLORD_PORT>/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
 ```
 
-A sample supervisor spec is shown below:
+Where the file `supervisor-spec.json` contains a Kafka supervisor spec:
 
 ```json
 {
   "type": "kafka",
-  "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"
+  "spec": {
+    "dataSchema": {
+      "dataSource": "metrics-kafka",
+      "timestampSpec": {
+        "column": "timestamp",
+        "format": "auto"
       },
-      {
-        "name": "value_max",
-        "fieldName": "value",
-        "type": "doubleMax"
+      "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"
       }
-    ],
-    "granularitySpec": {
-      "type": "uniform",
-      "segmentGranularity": "HOUR",
-      "queryGranularity": "NONE"
-    }
-  },
-  "ioConfig": {
-    "topic": "metrics",
-    "inputFormat": {
-      "type": "json"
     },
-    "consumerProperties": {
-      "bootstrap.servers": "localhost:9092"
+    "ioConfig": {
+      "topic": "metrics",
+      "inputFormat": {
+        "type": "json"
+      },
+      "consumerProperties": {
+        "bootstrap.servers": "localhost:9092"
+      },
+      "taskCount": 1,
+      "replicas": 1,
+      "taskDuration": "PT1H"
     },
-    "taskCount": 1,
-    "replicas": 1,
-    "taskDuration": "PT1H"
-  },
-  "tuningConfig": {
-    "type": "kafka",
-    "maxRowsPerSegment": 5000000
-  }
+    "tuningConfig": {
+      "type": "kafka",
+      "maxRowsPerSegment": 5000000
+    }
+  } 
 }
 ```
 
 ## Supervisor Configuration
 
 |Field|Description|Required|
 |--------|-----------|---------|
-|`type`|The supervisor type, this should always be `kafka`.|yes|
-|`dataSchema`|The schema that will be used by the Kafka indexing task during ingestion. See [`dataSchema`](../../ingestion/ingestion-spec.md#dataschema) for details.|yes|
-|`ioConfig`|A KafkaSupervisorIOConfig object for configuring Kafka connection and I/O-related settings for the supervisor and indexing task. See [KafkaSupervisorIOConfig](#kafkasupervisorioconfig) below.|yes|
-|`tuningConfig`|A KafkaSupervisorTuningConfig object for configuring performance-related settings for the supervisor and indexing tasks. See [KafkaSupervisorTuningConfig](#kafkasupervisortuningconfig) below.|no|
+|`type`|Supervisor type. For Kafka streaming, set to `kafka`.|yes|
+|`spec`| Container object for the supervisor configuration. | yes |
+|`dataSchema`|Schema for the Kafka indexing task to use during ingestion.|yes|
+|`ioConfig`|A `KafkaSupervisorIOConfig` object to define the Kafka connection and I/O-related settings for the supervisor and indexing task. See [KafkaSupervisorIOConfig](#kafkasupervisorioconfig).|yes|
+|`tuningConfig`|A KafkaSupervisorTuningConfig object to define performance-related settings for the supervisor and indexing tasks. See [KafkaSupervisorTuningConfig](#kafkasupervisortuningconfig).|no|
 
 ### KafkaSupervisorIOConfig
 
 |Field|Type|Description|Required|
 |-----|----|-----------|--------|
-|`topic`|String|The Kafka topic to read from. This must be a specific topic as topic patterns are not supported.|yes|
-|`inputFormat`|Object|[`inputFormat`](../../ingestion/data-formats.md#input-format) to specify how to parse input data. See [the below section](#specifying-data-format) for details about specifying the input format.|yes|
-|`consumerProperties`|Map<String, Object>|A map of properties to be passed to the Kafka consumer. See [next section](#more-on-consumerproperties) for more information.|yes|
+|`topic`|String|The Kafka topic to read from. Must be a specific topic. Topic patterns are not supported.|yes|
+|`inputFormat`|Object|`inputFormat` to define input data parsing. See [Specifying data format](#specifying-data-format) for details about specifying the input format.|yes|
+|`consumerProperties`|Map<String, Object>|A map of properties to pass to the Kafka consumer. See [More on consumer properties](#more-on-consumerproperties).|yes|
 |`pollTimeout`|Long|The length of time to wait for the Kafka consumer to poll records, in milliseconds|no (default == 100)|
-|`replicas`|Integer|The number of replica sets, where 1 means a single set of tasks (no replication). Replica tasks will always be assigned to different workers to provide resiliency against process failure.|no (default == 1)|
-|`taskCount`|Integer|The maximum number of *reading* tasks in a *replica set*. This means that the maximum number of reading tasks will be `taskCount * replicas` and the total number of tasks (*reading* + *publishing*) will be higher than this. See [Capacity Planning](#capacity-planning) below for more details. The number of reading tasks will be less than `taskCount` if `taskCount > {numKafkaPartitions}`.|no (default == 1)|
-|`taskDuration`|ISO8601 Period|The length of time before tasks stop reading and begin publishing their segment.|no (default == PT1H)|
+|`replicas`|Integer|The number of replica sets. "1" means a single set of tasks without replication. Druid always assigns replica tasks to different workers to provide resiliency against process task failure.|no (default == 1)|

Review comment:
       This is actually for being resilient against middleManager/indexer failures. 
   
   ```suggestion
   |`replicas`|Integer|The number of replica sets. "1" means a single set of tasks without replication. Druid always assigns replica tasks to different workers to provide resiliency against worker failure.|no (default == 1)|
   ```

##########
File path: docs/development/extensions-core/kafka-ingestion.md
##########
@@ -23,129 +23,130 @@ sidebar_label: "Apache Kafka"
   ~ under the License.
   -->
 
+When you enable the Kafka indexing service, you can configure *supervisors* on the Overlord to manage manage the creation and lifetime of Kafka indexing tasks. These 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.
 
-The Kafka indexing service enables the configuration of *supervisors* on the Overlord, which facilitate ingestion from
-Kafka by managing the creation and lifetime of Kafka indexing tasks. These indexing tasks read events using Kafka's own
-partition and offset mechanism and are therefore able to provide guarantees of exactly-once ingestion.
-The supervisor oversees the state of the indexing tasks to coordinate handoffs,
-manage failures, and ensure that the scalability and replication requirements are maintained.
+To use the Kafka indexing service, load the `druid-kafka-indexing-service` core Apache Druid extension. See [Including Extensions](../../development/extensions.md#loading-extensions)).
 
-This service is provided in the `druid-kafka-indexing-service` core Apache Druid extension (see
-[Including Extensions](../../development/extensions.md#loading-extensions)).
+This topic covers the ingestion spec for Kafka. For a general `ingestionSpec` reference, see [Ingestion specs](../../ingestion/ingestion-spec.md). For a walk-through, check out the [Loading from Apache Kafka](../../tutorials/tutorial-kafka.md) tutorial.
 
-> The Kafka indexing service supports transactional topics which were introduced in Kafka 0.11.x. It is the default behavior of Druid and make the
-> Kafka consumer that Druid uses incompatible with older brokers. Ensure that your Kafka brokers are version 0.11.x or
-> better before using this functionality. Refer [Kafka upgrade guide](https://kafka.apache.org/documentation/#upgrade)
-> if you are using older version of Kafka brokers.
-> In addition, users could set `isolation.level` `read_uncommitted` in `consumerProperties`, if don't need Druid to consume transactional topics or need Druid to consume older versions of Kafka.
-> Make sure offsets are sequential, since there is no offset gap check in Druid anymore.
+## 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).
 
-> If your Kafka cluster enables consumer-group based ACLs, you can set `group.id` in `consumerProperties` to override the default auto generated group id.
+Additionally, you can set `isolation.level` `read_uncommitted` in `consumerProperties` if either:
+- You don't need Druid to consume transactional topics.
+- You need Druid to consume older versions of Kafka.
 
-## Tutorial
+Make sure offsets are sequential, since there is no offset gap check in Druid anymore.
 
-This page contains reference documentation for Apache Kafka-based ingestion.
-For a walk-through instead, check out the [Loading from Apache Kafka](../../tutorials/tutorial-kafka.md) tutorial.
+If your Kafka cluster enables consumer-group based ACLs, you can set `group.id` in `consumerProperties` to override the default auto generated group id.
 
 ## Submitting a Supervisor Spec
 
-The Kafka indexing service requires that the `druid-kafka-indexing-service` extension be loaded on both the Overlord and the
-MiddleManagers. A supervisor for a dataSource is started by submitting a supervisor spec via HTTP POST to
-`http://<OVERLORD_IP>:<OVERLORD_PORT>/druid/indexer/v1/supervisor`, for example:
+To use the Kafka indexing service, load the `druid-kafka-indexing-service` extension on both the Overlord and the MiddleManagers. Druid starts a supervisor for a dataSource when you submit a supervisor spec. You can use the following endpoint:
+
+`http://<OVERLORD_IP>:<OVERLORD_PORT>/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
 ```
 
-A sample supervisor spec is shown below:
+Where the file `supervisor-spec.json` contains a Kafka supervisor spec:
 
 ```json
 {
   "type": "kafka",
-  "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"
+  "spec": {
+    "dataSchema": {
+      "dataSource": "metrics-kafka",
+      "timestampSpec": {
+        "column": "timestamp",
+        "format": "auto"
       },
-      {
-        "name": "value_max",
-        "fieldName": "value",
-        "type": "doubleMax"
+      "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"
       }
-    ],
-    "granularitySpec": {
-      "type": "uniform",
-      "segmentGranularity": "HOUR",
-      "queryGranularity": "NONE"
-    }
-  },
-  "ioConfig": {
-    "topic": "metrics",
-    "inputFormat": {
-      "type": "json"
     },
-    "consumerProperties": {
-      "bootstrap.servers": "localhost:9092"
+    "ioConfig": {
+      "topic": "metrics",
+      "inputFormat": {
+        "type": "json"
+      },
+      "consumerProperties": {
+        "bootstrap.servers": "localhost:9092"
+      },
+      "taskCount": 1,
+      "replicas": 1,
+      "taskDuration": "PT1H"
     },
-    "taskCount": 1,
-    "replicas": 1,
-    "taskDuration": "PT1H"
-  },
-  "tuningConfig": {
-    "type": "kafka",
-    "maxRowsPerSegment": 5000000
-  }
+    "tuningConfig": {
+      "type": "kafka",
+      "maxRowsPerSegment": 5000000
+    }
+  } 
 }
 ```
 
 ## Supervisor Configuration
 
 |Field|Description|Required|
 |--------|-----------|---------|
-|`type`|The supervisor type, this should always be `kafka`.|yes|
-|`dataSchema`|The schema that will be used by the Kafka indexing task during ingestion. See [`dataSchema`](../../ingestion/ingestion-spec.md#dataschema) for details.|yes|
-|`ioConfig`|A KafkaSupervisorIOConfig object for configuring Kafka connection and I/O-related settings for the supervisor and indexing task. See [KafkaSupervisorIOConfig](#kafkasupervisorioconfig) below.|yes|
-|`tuningConfig`|A KafkaSupervisorTuningConfig object for configuring performance-related settings for the supervisor and indexing tasks. See [KafkaSupervisorTuningConfig](#kafkasupervisortuningconfig) below.|no|
+|`type`|Supervisor type. For Kafka streaming, set to `kafka`.|yes|
+|`spec`| Container object for the supervisor configuration. | yes |
+|`dataSchema`|Schema for the Kafka indexing task to use during ingestion.|yes|
+|`ioConfig`|A `KafkaSupervisorIOConfig` object to define the Kafka connection and I/O-related settings for the supervisor and indexing task. See [KafkaSupervisorIOConfig](#kafkasupervisorioconfig).|yes|
+|`tuningConfig`|A KafkaSupervisorTuningConfig object to define performance-related settings for the supervisor and indexing tasks. See [KafkaSupervisorTuningConfig](#kafkasupervisortuningconfig).|no|
 
 ### KafkaSupervisorIOConfig
 
 |Field|Type|Description|Required|
 |-----|----|-----------|--------|
-|`topic`|String|The Kafka topic to read from. This must be a specific topic as topic patterns are not supported.|yes|
-|`inputFormat`|Object|[`inputFormat`](../../ingestion/data-formats.md#input-format) to specify how to parse input data. See [the below section](#specifying-data-format) for details about specifying the input format.|yes|
-|`consumerProperties`|Map<String, Object>|A map of properties to be passed to the Kafka consumer. See [next section](#more-on-consumerproperties) for more information.|yes|
+|`topic`|String|The Kafka topic to read from. Must be a specific topic. Topic patterns are not supported.|yes|
+|`inputFormat`|Object|`inputFormat` to define input data parsing. See [Specifying data format](#specifying-data-format) for details about specifying the input format.|yes|
+|`consumerProperties`|Map<String, Object>|A map of properties to pass to the Kafka consumer. See [More on consumer properties](#more-on-consumerproperties).|yes|
 |`pollTimeout`|Long|The length of time to wait for the Kafka consumer to poll records, in milliseconds|no (default == 100)|
-|`replicas`|Integer|The number of replica sets, where 1 means a single set of tasks (no replication). Replica tasks will always be assigned to different workers to provide resiliency against process failure.|no (default == 1)|
-|`taskCount`|Integer|The maximum number of *reading* tasks in a *replica set*. This means that the maximum number of reading tasks will be `taskCount * replicas` and the total number of tasks (*reading* + *publishing*) will be higher than this. See [Capacity Planning](#capacity-planning) below for more details. The number of reading tasks will be less than `taskCount` if `taskCount > {numKafkaPartitions}`.|no (default == 1)|
-|`taskDuration`|ISO8601 Period|The length of time before tasks stop reading and begin publishing their segment.|no (default == PT1H)|
+|`replicas`|Integer|The number of replica sets. "1" means a single set of tasks without replication. Druid always assigns replica tasks to different workers to provide resiliency against process task failure.|no (default == 1)|
+|`taskCount`|Integer|The maximum number of *reading* tasks in a *replica set*. The maximum number of reading tasks equals `taskCount * replicas`. Therefore, the total number of tasks, *reading* + *publishing*, is greater than this count. See [Capacity Planning](#capacity-planning) for more details. When `taskCount > {numKafkaPartitions}`, number of reading tasks is less than the `taskCount` value.|no (default == 1)|

Review comment:
       ```suggestion
   |`taskCount`|Integer|The maximum number of *reading* tasks in a *replica set*. The maximum number of reading tasks equals `taskCount * replicas`. Therefore, the total number of tasks, *reading* + *publishing*, is greater than this count. See [Capacity Planning](#capacity-planning) for more details. When `taskCount > {numKafkaPartitions}`, the actual number of reading tasks is less than the `taskCount` value.|no (default == 1)|
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org