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 2018/12/21 19:49:26 UTC

[GitHub] dclim closed pull request #6431: Add Kinesis Indexing Service to core Druid

dclim closed pull request #6431: Add Kinesis Indexing Service to core Druid
URL: https://github.com/apache/incubator-druid/pull/6431
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/distribution/pom.xml b/distribution/pom.xml
index 6169d736af2..96b256e9722 100644
--- a/distribution/pom.xml
+++ b/distribution/pom.xml
@@ -139,6 +139,8 @@
                                         <argument>-c</argument>
                                         <argument>org.apache.druid.extensions:druid-kafka-indexing-service</argument>
                                         <argument>-c</argument>
+                                        <argument>org.apache.druid.extensions:druid-kinesis-indexing-service</argument>
+                                        <argument>-c</argument>
                                         <argument>org.apache.druid.extensions:druid-lookups-cached-global</argument>
                                         <argument>-c</argument>
                                         <argument>org.apache.druid.extensions:druid-lookups-cached-single</argument>
@@ -317,4 +319,4 @@
             </build>
         </profile>
     </profiles>
-</project>
+</project>
\ No newline at end of file
diff --git a/docs/content/development/extensions-core/kafka-ingestion.md b/docs/content/development/extensions-core/kafka-ingestion.md
index 2e0909706fa..aa771b7a27c 100644
--- a/docs/content/development/extensions-core/kafka-ingestion.md
+++ b/docs/content/development/extensions-core/kafka-ingestion.md
@@ -28,7 +28,7 @@ The Kafka indexing service enables the configuration of *supervisors* on the Ove
 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. They are also
 able to read non-recent events from Kafka and are not subject to the window period considerations imposed on other
-ingestion mechanisms. The supervisor oversees the state of the indexing tasks to coordinate handoffs, manage failures,
+ingestion mechanisms using Tranquility. The supervisor oversees the state of the indexing tasks to coordinate handoffs, manage failures,
 and ensure that the scalability and replication requirements are maintained.
 
 This service is provided in the `druid-kafka-indexing-service` core extension (see
@@ -125,8 +125,8 @@ A sample supervisor spec is shown below:
 |--------|-----------|---------|
 |`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 [Ingestion Spec DataSchema](../../ingestion/ingestion-spec.html#dataschema).|yes|
-|`tuningConfig`|A KafkaSupervisorTuningConfig to configure the supervisor and indexing tasks, see below.|no|
 |`ioConfig`|A KafkaSupervisorIOConfig to configure the supervisor and indexing tasks, see below.|yes|
+|`tuningConfig`|A KafkaSupervisorTuningConfig to configure the supervisor and indexing tasks, see below.|no|
 
 ### KafkaSupervisorTuningConfig
 
@@ -312,7 +312,7 @@ compatible because they have a different ingestion spec or partition allocation,
 supervisor will create a new set of tasks. In this way, the supervisors are persistent across overlord restarts and
 fail-overs.
 
-A supervisor is stopped via the `POST /druid/indexer/v1/supervisor/<supervisorId>/shutdown` endpoint. This places a
+A supervisor is stopped via the `POST /druid/indexer/v1/supervisor/<supervisorId>/terminate` endpoint. This places a
 tombstone marker in the database (to prevent the supervisor from being reloaded on a restart) and then gracefully
 shuts down the currently running supervisor. When a supervisor is shut down in this way, it will instruct its
 managed tasks to stop reading and begin publishing their segments immediately. The call to the shutdown endpoint will
diff --git a/docs/content/development/extensions-core/kinesis-ingestion.md b/docs/content/development/extensions-core/kinesis-ingestion.md
new file mode 100644
index 00000000000..8b3583b0d49
--- /dev/null
+++ b/docs/content/development/extensions-core/kinesis-ingestion.md
@@ -0,0 +1,392 @@
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one
+  ~ or more contributor license agreements.  See the NOTICE file
+  ~ distributed with this work for additional information
+  ~ regarding copyright ownership.  The ASF licenses this file
+  ~ to you under the Apache License, Version 2.0 (the
+  ~ "License"); you may not use this file except in compliance
+  ~ with the License.  You may obtain a copy of the License at
+  ~
+  ~   http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing,
+  ~ software distributed under the License is distributed on an
+  ~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+  ~ KIND, either express or implied.  See the License for the
+  ~ specific language governing permissions and limitations
+  ~ under the License.
+  -->
+
+---
+layout: doc_page
+---
+
+# Kinesis Indexing Service
+
+Similar to the [Kafka indexing service](./kafka-ingestion.html), the Kinesis indexing service enables the configuration of *supervisors* on the Overlord, which 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. They are also
+able to read non-recent events from Kinesis and are not subject to the window period considerations imposed on other
+ingestion mechanisms using Tranquility. 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 extension (see
+[Including Extensions](../../operations/including-extensions.html)). Please note that this is
+currently designated as an *experimental feature* and is subject to the usual
+[experimental caveats](../experimental.html).
+
+## Submitting a Supervisor Spec
+
+The Kinesis indexing service requires that the `druid-kinesis-indexing-service` extension be loaded on both the overlord
+and the middle managers. 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:
+
+```
+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:
+
+```json
+{
+  "type": "kinesis",
+  "dataSchema": {
+    "dataSource": "metrics-kinesis",
+    "parser": {
+      "type": "string",
+      "parseSpec": {
+        "format": "json",
+        "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"
+    }
+  },
+  "tuningConfig": {
+    "type": "kinesis",
+    "maxRowsPerSegment": 5000000
+  },
+  "ioConfig": {
+    "stream": "metrics",
+    "endpoint": "kinesis.us-east-1.amazonaws.com",
+    "taskCount": 1,
+    "replicas": 1,
+    "taskDuration": "PT1H",
+    "recordsPerFetch": 2000,
+    "fetchDelayMillis": 1000
+  }
+}
+```
+
+## Supervisor Configuration
+
+|Field|Description|Required|
+|--------|-----------|---------|
+|`type`|The supervisor type, this should always be `kinesis`.|yes|
+|`dataSchema`|The schema that will be used by the Kinesis indexing task during ingestion, see [Ingestion Spec DataSchema](../../ingestion/ingestion-spec.html#dataschema).|yes|
+|`ioConfig`|A KinesisSupervisorIOConfig to configure the supervisor and indexing tasks, see below.|yes|
+|`tuningConfig`|A KinesisSupervisorTuningConfig to configure the supervisor and indexing tasks, see below.|no|
+
+### KinesisSupervisorTuningConfig
+
+The tuningConfig is optional and default parameters will be used if no tuningConfig is specified.
+
+|Field|Type|Description|Required|
+|-----|----|-----------|--------|
+|`type`|String|The indexing task type, this should always be `kinesis`.|yes|
+|`maxRowsInMemory`|Integer|The number of rows to aggregate before persisting. This number is the post-aggregation rows, so it is not equivalent to the number of input events, but the number of aggregated rows that those events result in. This is used to manage the required JVM heap size. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 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 and user does not need to set it. The maximum heap memory usage for indexing is maxBytesInMemory * (2 + maxPendingPersists).  |no (default == One-sixth of max JVM memory)|
+|`maxRowsPerSegment`|Integer|The number of rows to aggregate into a segment; this number is post-aggregation rows. Handoff will happen either if `maxRowsPerSegment` or `maxTotalRows` is hit or every `intermediateHandoffPeriod`, whichever happens earlier.|no (default == 5000000)|
+|`maxTotalRows`|Long|The number of rows to aggregate across all segments; this number is post-aggregation rows. Handoff will happen either if `maxRowsPerSegment` or `maxTotalRows` is hit or every `intermediateHandoffPeriod`, whichever happens earlier.|no (default == unlimited)|
+|`intermediatePersistPeriod`|ISO8601 Period|The period that determines the rate at which intermediate persists occur.|no (default == PT10M)|
+|`maxPendingPersists`|Integer|Maximum number of persists that can be pending but not started. If this limit would be exceeded by a new intermediate persist, ingestion will block until the currently-running persist finishes. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 0, meaning one persist can be running concurrently with ingestion, and none can be queued up)|
+|`indexSpec`|Object|Tune how data is indexed, see 'IndexSpec' below for more details.|no|
+|`reportParseExceptions`|Boolean|If true, exceptions encountered during parsing will be thrown and will halt ingestion; if false, unparseable rows and fields will be skipped.|no (default == false)|
+|`handoffConditionTimeout`|Long|Milliseconds to wait for segment handoff. It must be >= 0, where 0 means to wait forever.|no (default == 0)|
+|`resetOffsetAutomatically`|Boolean|Whether to reset the consumer sequence numbers if the next sequence number that it is trying to fetch is less than the earliest available sequence number for that particular shard. The sequence number will be reset to either the earliest or latest sequence number depending on `useEarliestOffset` property of `KinesisSupervisorIOConfig` (see below). This situation typically occurs when messages in Kinesis are no longer available for consumption and therefore won't be ingested into Druid. If set to false then ingestion for that particular shard will halt and manual intervention is required to correct the situation, please see `Reset Supervisor` API below.|no (default == false)|
+|`skipSequenceNumberAvailabilityCheck`|Boolean|Whether to enable checking if the current sequence number is still available in a particular Kinesis shard. If set to false, the indexing task will attempt to reset the current sequence number (or not), depending on the value of `resetOffsetAutomatically`. |no (default == false)|
+|`workerThreads`|Integer|The number of threads that will be used by the supervisor for asynchronous operations.|no (default == min(10, taskCount))|
+|`chatThreads`|Integer|The number of threads that will be used for communicating with indexing tasks.|no (default == min(10, taskCount * replicas))|
+|`chatRetries`|Integer|The number of times HTTP requests to indexing tasks will be retried before considering tasks unresponsive.|no (default == 8)|
+|`httpTimeout`|ISO8601 Period|How long to wait for a HTTP response from an indexing task.|no (default == PT10S)|
+|`shutdownTimeout`|ISO8601 Period|How long to wait for the supervisor to attempt a graceful shutdown of tasks before exiting.|no (default == PT80S)|
+|`recordBufferSize`|Integer|Size of the buffer (number of events) used between the Kinesis fetch threads and the main ingestion thread.|no (default == 10000)|
+|`recordBufferOfferTimeout`|Integer|Length of time in milliseconds to wait for space to become available in the buffer before timing out.|no (default == 5000)|
+|`recordBufferFullWait`|Integer|Length of time in milliseconds to wait for the buffer to drain before attempting to fetch records from Kinesis again.|no (default == 5000)|
+|`fetchSequenceNumberTimeout`|Integer|Length of time in milliseconds to wait for Kinesis to return the earliest or latest sequence number for a shard. Kinesis will not return the latest sequence number if no data is actively being written to that shard. In this case, this fetch call will repeatedly timeout and retry until fresh data is written to the stream.|no (default == 60000)|
+|`fetchThreads`|Integer|Size of the pool of threads fetching data from Kinesis. There is no benefit in having more threads than Kinesis shards.|no (default == max(1, {numProcessors} - 1))|
+|`segmentWriteOutMediumFactory`|Object|Segment write-out medium to use when creating segments. See below for more information.|no (not specified by default, the value from `druid.peon.defaultSegmentWriteOutMediumFactory.type` is used)|
+|`intermediateHandoffPeriod`|ISO8601 Period|How often the tasks should hand off segments. Handoff will happen either if `maxRowsPerSegment` or `maxTotalRows` is hit or every `intermediateHandoffPeriod`, whichever happens earlier.|no (default == P2147483647D)|
+|`logParseExceptions`|Boolean|If true, log an error message when a parsing exception occurs, containing information about the row where the error occurred.|no, default == 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 default|
+|`maxSavedParseExceptions`|Integer|When a parse exception occurs, Druid can keep track of the most recent parse exceptions. "maxSavedParseExceptions" limits how many exception instances will be saved. These saved exceptions will be made available after the task finishes in the [task completion report](../../ingestion/reports.html). Overridden if `reportParseExceptions` is set.|no, default == 0|
+|`maxRecordsPerPoll`|Integer| The maximum number of records/events to be fetched from buffer per poll. The actual maximum will be `Max(maxRecordsPerPoll, Max(bufferSize, 1)) |no, default == 100|
+
+#### IndexSpec
+
+|Field|Type|Description|Required|
+|-----|----|-----------|--------|
+|bitmap|Object|Compression format for bitmap indexes. Should be a JSON object; see below for options.|no (defaults to Concise)|
+|dimensionCompression|String|Compression format for dimension columns. Choose from `LZ4`, `LZF`, or `uncompressed`.|no (default == `LZ4`)|
+|metricCompression|String|Compression format for metric columns. Choose from `LZ4`, `LZF`, `uncompressed`, or `none`.|no (default == `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 store them with variable size. `longs` stores the value as is with 8 bytes each.|no (default == `longs`)|
+
+##### Bitmap types
+
+For Concise bitmaps:
+
+|Field|Type|Description|Required|
+|-----|----|-----------|--------|
+|`type`|String|Must be `concise`.|yes|
+
+For Roaring bitmaps:
+
+|Field|Type|Description|Required|
+|-----|----|-----------|--------|
+|`type`|String|Must be `roaring`.|yes|
+|`compressRunOnSerialization`|Boolean|Use a run-length encoding where it is estimated as more space efficient.|no (default == `true`)|
+
+#### SegmentWriteOutMediumFactory
+
+|Field|Type|Description|Required|
+|-----|----|-----------|--------|
+|`type`|String|See [Additional Peon Configuration: SegmentWriteOutMediumFactory](../../configuration/index.html#segmentwriteoutmediumfactory) for explanation and available options.|yes|
+
+### KinesisSupervisorIOConfig
+
+|Field|Type|Description|Required|
+|-----|----|-----------|--------|
+|`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 [here](http://docs.aws.amazon.com/general/latest/gr/rande.html#ak_region).|no (default == kinesis.us-east-1.amazonaws.com)|
+|`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 node 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' below for more details. The number of reading tasks will be less than `taskCount` if `taskCount > {numKinesisshards}`.|no (default == 1)|
+|`taskDuration`|ISO8601 Period|The length of time before tasks stop reading and begin publishing their segment.|no (default == PT1H)|
+|`startDelay`|ISO8601 Period|The period to wait before the supervisor starts managing tasks.|no (default == PT5S)|
+|`period`|ISO8601 Period|How often the supervisor will execute its management logic. Note that the supervisor will also run in response to certain events (such as tasks succeeding, failing, and reaching their taskDuration) so this value specifies the maximum time between iterations.|no (default == PT30S)|
+|`useEarliestSequenceNumber`|Boolean|If a supervisor is managing a dataSource for the first time, it will obtain a set of starting sequence numbers from Kinesis. This flag determines whether it retrieves the earliest or latest sequence numbers in Kinesis. Under normal circumstances, subsequent tasks will start from where the previous segments ended so this flag will only be used on first run.|no (default == false)|
+|`completionTimeout`|ISO8601 Period|The length of time to wait before declaring a publishing task as failed and terminating it. If this is set too low, your tasks may never publish. The publishing clock for a task begins roughly after `taskDuration` elapses.|no (default == PT6H)|
+|`lateMessageRejectionPeriod`|ISO8601 Period|Configure tasks to reject messages with timestamps earlier than this period before the task was created; for example if this is set to `PT1H` and the supervisor creates a task at *2016-01-01T12:00Z*, messages with timestamps earlier than *2016-01-01T11:00Z* will be 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 (e.g. a realtime and a nightly batch ingestion pipeline).|no (default == none)|
+|`earlyMessageRejectionPeriod`|ISO8601 Period|Configure tasks to reject messages with timestamps later than this period after the task reached its taskDuration; for example if this 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* will be 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 (default == none)|
+|`recordsPerFetch`|Integer|The number of records to request per GetRecords call to Kinesis. See 'Determining Fetch Settings' below.|no (default == 2000)|
+|`fetchDelayMillis`|Integer|Time in milliseconds to wait between subsequent GetRecords calls to Kinesis. See 'Determining Fetch Settings' below.|no (default == 1000)|
+|`awsAssumedRoleArn`|String|The AWS assumed role to use for additional permissions.|no|
+|`awsExternalId`|String|The AWS external id to use for additional permissions.|no|
+|`deaggregate`|Boolean|Whether to use the de-aggregate function of the KCL. See below for details.|no|
+
+## Operations
+
+This section gives descriptions of how some supervisor APIs work specifically in Kinesis Indexing Service.
+For all supervisor APIs, please check [Supervisor APIs](../../operations/api-reference.html#supervisors).
+
+### AWS Authentication
+To authenticate with AWS, you must provide your AWS access key and AWS secret key via runtime.properties, for example:
+```
+-Ddruid.kinesis.accessKey=123 -Ddruid.kinesis.secretKey=456
+```
+The AWS access key ID and secret access key are used for Kinesis API requests. If this is not provided, the service will look for credentials set in environment variables, in the default profile configuration file, and from the EC2 instance profile provider (in this order).
+
+### Getting Supervisor Status Report
+
+`GET /druid/indexer/v1/supervisor/<supervisorId>/status` returns a snapshot report of the current state of the tasks managed by the given supervisor. This includes the latest
+sequence numbers as reported by Kinesis. Unlike the Kafka Indexing Service, stats about lag is not yet supported.
+
+### 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 sequence numbers the publishing tasks ended on.
+
+Seamless schema migrations can thus be achieved by simply submitting the new schema using this endpoint.
+
+### Suspending and Resuming Supervisors
+
+You can suspend and resume a supervisor using `POST /druid/indexer/v1/supervisor/<supervisorId>/suspend` and `POST /druid/indexer/v1/supervisor/<supervisorId>/resume`, respectively.
+
+Note that the supervisor itself will still be operating and emitting logs and metrics,
+it will just ensure that no indexing tasks are running until the supervisor is resumed.
+
+### Resetting Supervisors
+
+To reset a running supervisor, you can use `POST /druid/indexer/v1/supervisor/<supervisorId>/reset`.
+
+The indexing service keeps track of the latest persisted Kinesis sequence number 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 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 can be used to clear the stored sequence numbers which will cause the supervisor to start reading from
+either the earliest or latest sequence numbers in Kinesis (depending on the value of `useEarliestSequenceNumber`). The supervisor must be
+running for this endpoint to be available. After the stored sequence numbers are cleared, the supervisor will automatically kill
+and re-create any active tasks so that tasks begin reading from valid sequence numbers.
+
+Note that since the stored sequence numbers are necessary to guarantee exactly-once ingestion, resetting them with this endpoint
+may cause some Kinesis messages to be skipped or to be read twice.
+
+### Terminating Supervisors
+
+`POST /druid/indexer/v1/supervisor/<supervisorId>/terminate` 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 it's 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
+
+Kinesis indexing tasks run on middle managers and are thus 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
+(e.g. batch processing, realtime tasks, merging tasks, etc.). If your workers run out of capacity, Kinesis 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 Kinesis purges those sequence numbers).
+
+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 node
+(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 > {numKinesisShards} in which case {numKinesisShards} 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 Kinesis 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 shard allocation, the tasks will be killed and the
+supervisor will create a new set of tasks. In this way, the supervisors are persistent across overlord restarts and
+fail-overs.
+
+A supervisor is stopped via the `POST /druid/indexer/v1/supervisor/<supervisorId>/terminate` endpoint. This places a
+tombstone marker in the database (to prevent the supervisor from being reloaded on a restart) and then gracefully
+shuts down the currently running supervisor. When a supervisor is shut down in this way, it will instruct its
+managed tasks to stop reading and begin publishing their segments immediately. The call to the shutdown endpoint will
+return after all tasks have been signalled 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 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.
+
+### Deployment Notes
+
+#### On the Subject of Segments
+
+Each Kinesis Indexing Task puts events consumed from Kinesis Shards 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 shard
+for this segment granularity is created for further events. Kinesis 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 nodes and it is encouraged to do so.
+
+Kinesis Indexing Service may still produce some small segments. Lets say the task duration is 4 hours, segment granularity
+is set to an HOUR and Supervisor was started at 9:10 then after 4 hours at 13:10, new set of tasks will be started and
+events for the interval 13:00 - 14:00 may be split across previous and new set of tasks. If you see it becoming a problem then
+one can schedule re-indexing tasks be run to merge segments together into new segments of an ideal size (in the range of ~500-700 MB per segment).
+Details on how to optimize the segment size can be found on [Segment size optimization](../../operations/segment-optimization.html).
+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/incubator-druid/pull/5102)).
+
+### Determining Fetch Settings
+Internally, the Kinesis Indexing Service uses the Kinesis Record Supplier abstraction for fetching Kinesis data records and storing the records
+locally. The way the Kinesis Record Supplier fetches records is to have a separate thread run the fetching operation per each Kinesis Shard, the
+max number of threads is determined by `fetchThreads`. For example, a Kinesis stream with 3 shards will have 3 threads, each fetching from a shard separately.
+There is a delay between each fetching operation, which is controlled by `fetchDelayMillis`. The maximum number of records to be fetched per thread per
+operation is controlled by `recordsPerFetch`. Note that this is not the same as `maxRecordsPerPoll`.
+
+The records fetched by each thread will be pushed to a queue in the order that they are fetched. The records are stored in this queue until `poll()` is called
+by either the supervisor or the indexing task. `poll()` will attempt to drain the internal buffer queue up to a limit of `max(maxRecordsPerPoll, q.size())`.
+Here `maxRecordsPerPoll` controls the theoretical maximum records to drain out of the buffer queue, so setting this parameter to a reasonable value is essential
+in preventing the queue from overflowing or memory exceeding heap size.
+
+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.
+
+Values for `recordsPerFetch` and `fetchDelayMillis` should be chosen to maximize throughput under the above constraints.
+The values that you choose will 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).
+
+If the above limits are violated, AWS will throw ProvisionedThroughputExceededException errors on subsequent calls to
+read data. When this happens, the Kinesis indexing service will pause by `fetchDelayMillis` and then attempt the call
+again.
+
+Internally, each indexing task maintains a buffer that stores the fetched but not yet processed record. `recordsPerFetch` and `fetchDelayMillis`
+control this behavior. The number of records that the indexing task fetch from the buffer is controlled by `maxRecordsPerPoll`, which
+determines the number of records to be processed per each ingestion loop in the task.
+
+## Deaggregation
+See [issue](https://github.com/apache/incubator-druid/issues/6714)
+
+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. Currently, enabling the de-aggregate functionality
+requires the user to manually provide the Kinesis Client Library on the classpath, since this library has a license not
+compatible with Apache projects.
+
+To enable this feature, add the `amazon-kinesis-client` (tested on version `1.9.2`) jar file ([link](https://mvnrepository.com/artifact/com.amazonaws/amazon-kinesis-client/1.9.2)) under `dist/druid/extensions/druid-kinesis-indexing-service/`.
+Then when submitting a supervisor-spec, set `deaggregate` to true.
\ No newline at end of file
diff --git a/docs/content/development/extensions.md b/docs/content/development/extensions.md
index 1f9d925dad9..a5b8a5b3f57 100644
--- a/docs/content/development/extensions.md
+++ b/docs/content/development/extensions.md
@@ -51,6 +51,7 @@ Core extensions are maintained by Druid committers.
 |druid-kafka-eight|Kafka ingest firehose (high level consumer) for realtime nodes.|[link](../development/extensions-core/kafka-eight-firehose.html)|
 |druid-kafka-extraction-namespace|Kafka-based namespaced lookup. Requires namespace lookup extension.|[link](../development/extensions-core/kafka-extraction-namespace.html)|
 |druid-kafka-indexing-service|Supervised exactly-once Kafka ingestion for the indexing service.|[link](../development/extensions-core/kafka-ingestion.html)|
+|druid-kinesis-indexing-service|Supervised exactly-once Kinesis ingestion for the indexing service.|[link](../development/extensions-core/kinesis-ingestion.html)|
 |druid-kerberos|Kerberos authentication for druid nodes.|[link](../development/extensions-core/druid-kerberos.html)|
 |druid-lookups-cached-global|A module for [lookups](../querying/lookups.html) 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.html)|
 |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.html)|
diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java
index c05021df7bb..e8f62620866 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java
@@ -19,185 +19,48 @@
 
 package org.apache.druid.indexing.kafka;
 
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.core.type.TypeReference;
-import com.google.common.base.Function;
-import com.google.common.base.Joiner;
+import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Supplier;
-import com.google.common.base.Throwables;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Sets;
-import com.google.common.primitives.Longs;
-import com.google.common.util.concurrent.FutureCallback;
-import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.SettableFuture;
-import org.apache.druid.data.input.Committer;
-import org.apache.druid.data.input.InputRow;
 import org.apache.druid.data.input.impl.InputRowParser;
-import org.apache.druid.discovery.DiscoveryDruidNode;
-import org.apache.druid.discovery.LookupNodeService;
-import org.apache.druid.discovery.NodeType;
-import org.apache.druid.indexer.IngestionState;
-import org.apache.druid.indexer.TaskStatus;
-import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReport;
-import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData;
-import org.apache.druid.indexing.common.TaskRealtimeMetricsMonitorBuilder;
-import org.apache.druid.indexing.common.TaskReport;
 import org.apache.druid.indexing.common.TaskToolbox;
-import org.apache.druid.indexing.common.actions.CheckPointDataSourceMetadataAction;
-import org.apache.druid.indexing.common.actions.ResetDataSourceMetadataAction;
-import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction;
-import org.apache.druid.indexing.common.stats.RowIngestionMeters;
 import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory;
-import org.apache.druid.indexing.common.task.IndexTaskUtils;
-import org.apache.druid.indexing.common.task.RealtimeIndexTask;
-import org.apache.druid.indexing.kafka.KafkaIndexTask.Status;
-import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata;
+import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner;
+import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
+import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord;
+import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber;
+import org.apache.druid.indexing.seekablestream.common.RecordSupplier;
+import org.apache.druid.indexing.seekablestream.common.StreamPartition;
 import org.apache.druid.java.util.common.ISE;
-import org.apache.druid.java.util.common.StringUtils;
-import org.apache.druid.java.util.common.collect.Utils;
-import org.apache.druid.java.util.common.parsers.ParseException;
 import org.apache.druid.java.util.emitter.EmittingLogger;
-import org.apache.druid.segment.indexing.RealtimeIOConfig;
-import org.apache.druid.segment.realtime.FireDepartment;
-import org.apache.druid.segment.realtime.FireDepartmentMetrics;
-import org.apache.druid.segment.realtime.appenderator.Appenderator;
-import org.apache.druid.segment.realtime.appenderator.AppenderatorDriverAddResult;
-import org.apache.druid.segment.realtime.appenderator.SegmentsAndMetadata;
-import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorDriver;
-import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher;
 import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider;
-import org.apache.druid.server.security.Access;
-import org.apache.druid.server.security.Action;
 import org.apache.druid.server.security.AuthorizerMapper;
-import org.apache.druid.timeline.DataSegment;
 import org.apache.druid.utils.CircularBuffer;
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.apache.kafka.clients.consumer.ConsumerRecords;
-import org.apache.kafka.clients.consumer.KafkaConsumer;
 import org.apache.kafka.clients.consumer.OffsetOutOfRangeException;
 import org.apache.kafka.common.TopicPartition;
-import org.joda.time.DateTime;
 
+import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
-import javax.servlet.http.HttpServletRequest;
-import javax.ws.rs.Consumes;
-import javax.ws.rs.DefaultValue;
-import javax.ws.rs.GET;
-import javax.ws.rs.POST;
-import javax.ws.rs.Path;
-import javax.ws.rs.Produces;
-import javax.ws.rs.QueryParam;
-import javax.ws.rs.core.Context;
-import javax.ws.rs.core.MediaType;
-import javax.ws.rs.core.Response;
-import java.io.File;
+import javax.validation.constraints.NotNull;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.Map.Entry;
 import java.util.Set;
 import java.util.TreeMap;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-import java.util.concurrent.RejectedExecutionException;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.locks.Condition;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
 import java.util.stream.Collectors;
 
 /**
  * Kafka indexing task runner supporting incremental segments publishing
  */
-public class IncrementalPublishingKafkaIndexTaskRunner implements KafkaIndexTaskRunner
+public class IncrementalPublishingKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner<Integer, Long>
 {
   private static final EmittingLogger log = new EmittingLogger(IncrementalPublishingKafkaIndexTaskRunner.class);
-  private static final String METADATA_NEXT_PARTITIONS = "nextPartitions";
-  private static final String METADATA_PUBLISH_PARTITIONS = "publishPartitions";
-
-  private final Map<Integer, Long> endOffsets;
-  private final Map<Integer, Long> nextOffsets = new ConcurrentHashMap<>();
-  private final Map<Integer, Long> lastPersistedOffsets = new ConcurrentHashMap<>();
-
-  // The pause lock and associated conditions are to support coordination between the Jetty threads and the main
-  // ingestion loop. The goal is to provide callers of the API a guarantee that if pause() returns successfully
-  // the ingestion loop has been stopped at the returned offsets and will not ingest any more data until resumed. The
-  // fields are used as follows (every step requires acquiring [pauseLock]):
-  //   Pausing:
-  //   - In pause(), [pauseRequested] is set to true and then execution waits for [status] to change to PAUSED, with the
-  //     condition checked when [hasPaused] is signalled.
-  //   - In possiblyPause() called from the main loop, if [pauseRequested] is true, [status] is set to PAUSED,
-  //     [hasPaused] is signalled, and execution pauses until [pauseRequested] becomes false, either by being set or by
-  //     the [pauseMillis] timeout elapsing. [pauseRequested] is checked when [shouldResume] is signalled.
-  //   Resuming:
-  //   - In resume(), [pauseRequested] is set to false, [shouldResume] is signalled, and execution waits for [status] to
-  //     change to something other than PAUSED, with the condition checked when [shouldResume] is signalled.
-  //   - In possiblyPause(), when [shouldResume] is signalled, if [pauseRequested] has become false the pause loop ends,
-  //     [status] is changed to STARTING and [shouldResume] is signalled.
-
-  private final Lock pauseLock = new ReentrantLock();
-  private final Condition hasPaused = pauseLock.newCondition();
-  private final Condition shouldResume = pauseLock.newCondition();
-
-  private final AtomicBoolean stopRequested = new AtomicBoolean(false);
-  private final AtomicBoolean publishOnStop = new AtomicBoolean(false);
-
-  // [statusLock] is used to synchronize the Jetty thread calling stopGracefully() with the main run thread. It prevents
-  // the main run thread from switching into a publishing state while the stopGracefully() thread thinks it's still in
-  // a pre-publishing state. This is important because stopGracefully() will try to use the [stopRequested] flag to stop
-  // the main thread where possible, but this flag is not honored once publishing has begun so in this case we must
-  // interrupt the thread. The lock ensures that if the run thread is about to transition into publishing state, it
-  // blocks until after stopGracefully() has set [stopRequested] and then does a final check on [stopRequested] before
-  // transitioning to publishing state.
-  private final Object statusLock = new Object();
-
-  private final Lock pollRetryLock = new ReentrantLock();
-  private final Condition isAwaitingRetry = pollRetryLock.newCondition();
-
   private final KafkaIndexTask task;
-  private final KafkaIOConfig ioConfig;
-  private final KafkaTuningConfig tuningConfig;
-  private final InputRowParser<ByteBuffer> parser;
-  private final AuthorizerMapper authorizerMapper;
-  private final Optional<ChatHandlerProvider> chatHandlerProvider;
-  private final CircularBuffer<Throwable> savedParseExceptions;
-  private final String topic;
-  private final RowIngestionMeters rowIngestionMeters;
-
-  private final Set<String> publishingSequences = Sets.newConcurrentHashSet();
-  private final List<ListenableFuture<SegmentsAndMetadata>> publishWaitList = new ArrayList<>();
-  private final List<ListenableFuture<SegmentsAndMetadata>> handOffWaitList = new ArrayList<>();
-
-  private volatile DateTime startTime;
-  private volatile Status status = Status.NOT_STARTED; // this is only ever set by the task runner thread (runThread)
-  private volatile TaskToolbox toolbox;
-  private volatile Thread runThread;
-  private volatile Appenderator appenderator;
-  private volatile StreamAppenderatorDriver driver;
-  private volatile FireDepartmentMetrics fireDepartmentMetrics;
-  private volatile IngestionState ingestionState;
-
-  private volatile boolean pauseRequested = false;
-  private volatile long nextCheckpointTime;
-
-  private volatile CopyOnWriteArrayList<SequenceMetadata> sequences;
-  private volatile Throwable backgroundThreadException;
 
   public IncrementalPublishingKafkaIndexTaskRunner(
       KafkaIndexTask task,
@@ -208,854 +71,85 @@ public IncrementalPublishingKafkaIndexTaskRunner(
       RowIngestionMetersFactory rowIngestionMetersFactory
   )
   {
+    super(
+        task,
+        parser,
+        authorizerMapper,
+        chatHandlerProvider,
+        savedParseExceptions,
+        rowIngestionMetersFactory
+    );
     this.task = task;
-    this.ioConfig = task.getIOConfig();
-    this.tuningConfig = task.getTuningConfig();
-    this.parser = parser;
-    this.authorizerMapper = authorizerMapper;
-    this.chatHandlerProvider = chatHandlerProvider;
-    this.savedParseExceptions = savedParseExceptions;
-    this.topic = ioConfig.getStartPartitions().getTopic();
-    this.rowIngestionMeters = rowIngestionMetersFactory.createRowIngestionMeters();
-
-    this.endOffsets = new ConcurrentHashMap<>(ioConfig.getEndPartitions().getPartitionOffsetMap());
-    this.sequences = new CopyOnWriteArrayList<>();
-    this.ingestionState = IngestionState.NOT_STARTED;
-
-    resetNextCheckpointTime();
   }
 
   @Override
-  public TaskStatus run(TaskToolbox toolbox)
+  protected Long getSequenceNumberToStoreAfterRead(@NotNull Long sequenceNumber)
   {
-    try {
-      return runInternal(toolbox);
-    }
-    catch (Exception e) {
-      log.error(e, "Encountered exception while running task.");
-      final String errorMsg = Throwables.getStackTraceAsString(e);
-      toolbox.getTaskReportFileWriter().write(getTaskCompletionReports(errorMsg));
-      return TaskStatus.failure(
-          task.getId(),
-          errorMsg
-      );
-    }
+    return sequenceNumber + 1;
   }
 
-  private TaskStatus runInternal(TaskToolbox toolbox) throws Exception
-  {
-    log.info("Starting up!");
-
-    startTime = DateTimes.nowUtc();
-    status = Status.STARTING;
-    this.toolbox = toolbox;
-
-    if (!restoreSequences()) {
-      final TreeMap<Integer, Map<Integer, Long>> checkpoints = getCheckPointsFromContext(toolbox, task);
-      if (checkpoints != null) {
-        Iterator<Entry<Integer, Map<Integer, Long>>> sequenceOffsets = checkpoints.entrySet().iterator();
-        Map.Entry<Integer, Map<Integer, Long>> previous = sequenceOffsets.next();
-        while (sequenceOffsets.hasNext()) {
-          Map.Entry<Integer, Map<Integer, Long>> current = sequenceOffsets.next();
-          sequences.add(new SequenceMetadata(
-              previous.getKey(),
-              StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), previous.getKey()),
-              previous.getValue(),
-              current.getValue(),
-              true
-          ));
-          previous = current;
-        }
-        sequences.add(new SequenceMetadata(
-            previous.getKey(),
-            StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), previous.getKey()),
-            previous.getValue(),
-            endOffsets,
-            false
-        ));
-      } else {
-        sequences.add(new SequenceMetadata(
-            0,
-            StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), 0),
-            ioConfig.getStartPartitions().getPartitionOffsetMap(),
-            endOffsets,
-            false
-        ));
-      }
-    }
-    log.info("Starting with sequences:  %s", sequences);
-
-    if (chatHandlerProvider.isPresent()) {
-      log.info("Found chat handler of class[%s]", chatHandlerProvider.get().getClass().getName());
-      chatHandlerProvider.get().register(task.getId(), this, false);
-    } else {
-      log.warn("No chat handler detected");
-    }
-
-    runThread = Thread.currentThread();
-
-    // Set up FireDepartmentMetrics
-    final FireDepartment fireDepartmentForMetrics = new FireDepartment(
-        task.getDataSchema(),
-        new RealtimeIOConfig(null, null, null),
-        null
-    );
-    fireDepartmentMetrics = fireDepartmentForMetrics.getMetrics();
-    toolbox.getMonitorScheduler()
-           .addMonitor(TaskRealtimeMetricsMonitorBuilder.build(task, fireDepartmentForMetrics, rowIngestionMeters));
-
-    final String lookupTier = task.getContextValue(RealtimeIndexTask.CTX_KEY_LOOKUP_TIER);
-    LookupNodeService lookupNodeService = lookupTier == null ?
-                                          toolbox.getLookupNodeService() :
-                                          new LookupNodeService(lookupTier);
-    DiscoveryDruidNode discoveryDruidNode = new DiscoveryDruidNode(
-        toolbox.getDruidNode(),
-        NodeType.PEON,
-        ImmutableMap.of(
-            toolbox.getDataNodeService().getName(), toolbox.getDataNodeService(),
-            lookupNodeService.getName(), lookupNodeService
-        )
-    );
-
-    Throwable caughtExceptionOuter = null;
-    try (final KafkaConsumer<byte[], byte[]> consumer = task.newConsumer()) {
-      toolbox.getDataSegmentServerAnnouncer().announce();
-      toolbox.getDruidNodeAnnouncer().announce(discoveryDruidNode);
-
-      appenderator = task.newAppenderator(fireDepartmentMetrics, toolbox);
-      driver = task.newDriver(appenderator, toolbox, fireDepartmentMetrics);
-
-      final String topic = ioConfig.getStartPartitions().getTopic();
-
-      // Start up, set up initial offsets.
-      final Object restoredMetadata = driver.startJob();
-      if (restoredMetadata == null) {
-        // no persist has happened so far
-        // so either this is a brand new task or replacement of a failed task
-        Preconditions.checkState(sequences.get(0).startOffsets.entrySet().stream().allMatch(
-            partitionOffsetEntry -> Longs.compare(
-                partitionOffsetEntry.getValue(),
-                ioConfig.getStartPartitions()
-                        .getPartitionOffsetMap()
-                        .get(partitionOffsetEntry.getKey())
-            ) >= 0
-        ), "Sequence offsets are not compatible with start offsets of task");
-        nextOffsets.putAll(sequences.get(0).startOffsets);
-      } else {
-        final Map<String, Object> restoredMetadataMap = (Map) restoredMetadata;
-        final KafkaPartitions restoredNextPartitions = toolbox.getObjectMapper().convertValue(
-            restoredMetadataMap.get(METADATA_NEXT_PARTITIONS),
-            KafkaPartitions.class
-        );
-        nextOffsets.putAll(restoredNextPartitions.getPartitionOffsetMap());
-
-        // Sanity checks.
-        if (!restoredNextPartitions.getTopic().equals(ioConfig.getStartPartitions().getTopic())) {
-          throw new ISE(
-              "WTF?! Restored topic[%s] but expected topic[%s]",
-              restoredNextPartitions.getTopic(),
-              ioConfig.getStartPartitions().getTopic()
-          );
-        }
-
-        if (!nextOffsets.keySet().equals(ioConfig.getStartPartitions().getPartitionOffsetMap().keySet())) {
-          throw new ISE(
-              "WTF?! Restored partitions[%s] but expected partitions[%s]",
-              nextOffsets.keySet(),
-              ioConfig.getStartPartitions().getPartitionOffsetMap().keySet()
-          );
-        }
-        // sequences size can be 0 only when all sequences got published and task stopped before it could finish
-        // which is super rare
-        if (sequences.size() == 0 || sequences.get(sequences.size() - 1).isCheckpointed()) {
-          this.endOffsets.putAll(sequences.size() == 0
-                                 ? nextOffsets
-                                 : sequences.get(sequences.size() - 1).getEndOffsets());
-          log.info("End offsets changed to [%s]", endOffsets);
-        }
-      }
-
-      // Set up committer.
-      final Supplier<Committer> committerSupplier = () -> {
-        final Map<Integer, Long> snapshot = ImmutableMap.copyOf(nextOffsets);
-        lastPersistedOffsets.clear();
-        lastPersistedOffsets.putAll(snapshot);
-
-        return new Committer()
-        {
-          @Override
-          public Object getMetadata()
-          {
-            return ImmutableMap.of(
-                METADATA_NEXT_PARTITIONS, new KafkaPartitions(
-                    ioConfig.getStartPartitions().getTopic(),
-                    snapshot
-                )
-            );
-          }
-
-          @Override
-          public void run()
-          {
-            // Do nothing.
-          }
-        };
-      };
-
-      // restart publishing of sequences (if any)
-      maybePersistAndPublishSequences(committerSupplier);
-
-      Set<Integer> assignment = assignPartitionsAndSeekToNext(consumer, topic);
-
-      ingestionState = IngestionState.BUILD_SEGMENTS;
-
-      // Main loop.
-      // Could eventually support leader/follower mode (for keeping replicas more in sync)
-      boolean stillReading = !assignment.isEmpty();
-      status = Status.READING;
-      Throwable caughtExceptionInner = null;
-      try {
-        while (stillReading) {
-          if (possiblyPause()) {
-            // The partition assignments may have changed while paused by a call to setEndOffsets() so reassign
-            // partitions upon resuming. This is safe even if the end offsets have not been modified.
-            assignment = assignPartitionsAndSeekToNext(consumer, topic);
-
-            if (assignment.isEmpty()) {
-              log.info("All partitions have been fully read");
-              publishOnStop.set(true);
-              stopRequested.set(true);
-            }
-          }
-
-          // if stop is requested or task's end offset is set by call to setEndOffsets method with finish set to true
-          if (stopRequested.get() || sequences.get(sequences.size() - 1).isCheckpointed()) {
-            status = Status.PUBLISHING;
-          }
-
-          if (stopRequested.get()) {
-            break;
-          }
-
-          if (backgroundThreadException != null) {
-            throw new RuntimeException(backgroundThreadException);
-          }
-
-          checkPublishAndHandoffFailure();
-
-          maybePersistAndPublishSequences(committerSupplier);
-
-          // The retrying business is because the KafkaConsumer throws OffsetOutOfRangeException if the seeked-to
-          // offset is not present in the topic-partition. This can happen if we're asking a task to read from data
-          // that has not been written yet (which is totally legitimate). So let's wait for it to show up.
-          ConsumerRecords<byte[], byte[]> records = ConsumerRecords.empty();
-          try {
-            records = consumer.poll(KafkaIndexTask.POLL_TIMEOUT_MILLIS);
-          }
-          catch (OffsetOutOfRangeException e) {
-            log.warn("OffsetOutOfRangeException with message [%s]", e.getMessage());
-            possiblyResetOffsetsOrWait(e.offsetOutOfRangePartitions(), consumer, toolbox);
-            stillReading = !assignment.isEmpty();
-          }
-
-          SequenceMetadata sequenceToCheckpoint = null;
-          for (ConsumerRecord<byte[], byte[]> record : records) {
-            log.trace(
-                "Got topic[%s] partition[%d] offset[%,d].",
-                record.topic(),
-                record.partition(),
-                record.offset()
-            );
-
-            if (record.offset() < endOffsets.get(record.partition())) {
-              if (record.offset() != nextOffsets.get(record.partition())) {
-                if (ioConfig.isSkipOffsetGaps()) {
-                  log.warn(
-                      "Skipped to offset[%,d] after offset[%,d] in partition[%d].",
-                      record.offset(),
-                      nextOffsets.get(record.partition()),
-                      record.partition()
-                  );
-                } else {
-                  throw new ISE(
-                      "WTF?! Got offset[%,d] after offset[%,d] in partition[%d].",
-                      record.offset(),
-                      nextOffsets.get(record.partition()),
-                      record.partition()
-                  );
-                }
-              }
-
-              try {
-                final byte[] valueBytes = record.value();
-                final List<InputRow> rows = valueBytes == null
-                                            ? Utils.nullableListOf((InputRow) null)
-                                            : parser.parseBatch(ByteBuffer.wrap(valueBytes));
-                boolean isPersistRequired = false;
-
-                final SequenceMetadata sequenceToUse = sequences
-                    .stream()
-                    .filter(sequenceMetadata -> sequenceMetadata.canHandle(record))
-                    .findFirst()
-                    .orElse(null);
-
-                if (sequenceToUse == null) {
-                  throw new ISE(
-                      "WTH?! cannot find any valid sequence for record with partition [%d] and offset [%d]. Current sequences: %s",
-                      record.partition(),
-                      record.offset(),
-                      sequences
-                  );
-                }
-
-                for (InputRow row : rows) {
-                  if (row != null && task.withinMinMaxRecordTime(row)) {
-                    final AppenderatorDriverAddResult addResult = driver.add(
-                        row,
-                        sequenceToUse.getSequenceName(),
-                        committerSupplier,
-                        // skip segment lineage check as there will always be one segment
-                        // for combination of sequence and segment granularity.
-                        // It is necessary to skip it as the task puts messages polled from all the
-                        // assigned Kafka partitions into a single Druid segment, thus ordering of
-                        // messages among replica tasks across assigned partitions is not guaranteed
-                        // which may cause replica tasks to ask for segments with different interval
-                        // in different order which might cause SegmentAllocateAction to fail.
-                        true,
-                        // do not allow incremental persists to happen until all the rows from this batch
-                        // of rows are indexed
-                        false
-                    );
-
-                    if (addResult.isOk()) {
-                      // If the number of rows in the segment exceeds the threshold after adding a row,
-                      // move the segment out from the active segments of BaseAppenderatorDriver to make a new segment.
-                      if (addResult.isPushRequired(tuningConfig) && !sequenceToUse.isCheckpointed()) {
-                        sequenceToCheckpoint = sequenceToUse;
-                      }
-                      isPersistRequired |= addResult.isPersistRequired();
-                    } else {
-                      // Failure to allocate segment puts determinism at risk, bail out to be safe.
-                      // May want configurable behavior here at some point.
-                      // If we allow continuing, then consider blacklisting the interval for a while to avoid constant checks.
-                      throw new ISE("Could not allocate segment for row with timestamp[%s]", row.getTimestamp());
-                    }
-
-                    if (addResult.getParseException() != null) {
-                      handleParseException(addResult.getParseException(), record);
-                    } else {
-                      rowIngestionMeters.incrementProcessed();
-                    }
-                  } else {
-                    rowIngestionMeters.incrementThrownAway();
-                  }
-                }
-                if (isPersistRequired) {
-                  Futures.addCallback(
-                      driver.persistAsync(committerSupplier.get()),
-                      new FutureCallback<Object>()
-                      {
-                        @Override
-                        public void onSuccess(@Nullable Object result)
-                        {
-                          log.info("Persist completed with metadata [%s]", result);
-                        }
-
-                        @Override
-                        public void onFailure(Throwable t)
-                        {
-                          log.error("Persist failed, dying");
-                          backgroundThreadException = t;
-                        }
-                      }
-                  );
-                }
-              }
-              catch (ParseException e) {
-                handleParseException(e, record);
-              }
-
-              nextOffsets.put(record.partition(), record.offset() + 1);
-            }
-
-            if (nextOffsets.get(record.partition()).equals(endOffsets.get(record.partition()))
-                && assignment.remove(record.partition())) {
-              log.info("Finished reading topic[%s], partition[%,d].", record.topic(), record.partition());
-              KafkaIndexTask.assignPartitions(consumer, topic, assignment);
-              stillReading = !assignment.isEmpty();
-            }
-          }
-
-          if (System.currentTimeMillis() > nextCheckpointTime) {
-            sequenceToCheckpoint = sequences.get(sequences.size() - 1);
-          }
-
-          if (sequenceToCheckpoint != null && stillReading) {
-            Preconditions.checkArgument(
-                sequences.get(sequences.size() - 1)
-                         .getSequenceName()
-                         .equals(sequenceToCheckpoint.getSequenceName()),
-                "Cannot checkpoint a sequence [%s] which is not the latest one, sequences %s",
-                sequenceToCheckpoint,
-                sequences
-            );
-            requestPause();
-            final CheckPointDataSourceMetadataAction checkpointAction = new CheckPointDataSourceMetadataAction(
-                task.getDataSource(),
-                ioConfig.getTaskGroupId(),
-                task.getIOConfig().getBaseSequenceName(),
-                new KafkaDataSourceMetadata(new KafkaPartitions(topic, sequenceToCheckpoint.getStartOffsets())),
-                new KafkaDataSourceMetadata(new KafkaPartitions(topic, nextOffsets))
-            );
-            if (!toolbox.getTaskActionClient().submit(checkpointAction)) {
-              throw new ISE("Checkpoint request with offsets [%s] failed, dying", nextOffsets);
-            }
-          }
-        }
-        ingestionState = IngestionState.COMPLETED;
-      }
-      catch (Exception e) {
-        // (1) catch all exceptions while reading from kafka
-        caughtExceptionInner = e;
-        log.error(e, "Encountered exception in run() before persisting.");
-        throw e;
-      }
-      finally {
-        log.info("Persisting all pending data");
-        try {
-          driver.persist(committerSupplier.get()); // persist pending data
-        }
-        catch (Exception e) {
-          if (caughtExceptionInner != null) {
-            caughtExceptionInner.addSuppressed(e);
-          } else {
-            throw e;
-          }
-        }
-      }
-
-      synchronized (statusLock) {
-        if (stopRequested.get() && !publishOnStop.get()) {
-          throw new InterruptedException("Stopping without publishing");
-        }
-
-        status = Status.PUBLISHING;
-      }
-
-      for (SequenceMetadata sequenceMetadata : sequences) {
-        if (!publishingSequences.contains(sequenceMetadata.getSequenceName())) {
-          // this is done to prevent checks in sequence specific commit supplier from failing
-          sequenceMetadata.setEndOffsets(nextOffsets);
-          sequenceMetadata.updateAssignments(nextOffsets);
-          publishingSequences.add(sequenceMetadata.getSequenceName());
-          // persist already done in finally, so directly add to publishQueue
-          publishAndRegisterHandoff(sequenceMetadata);
-        }
-      }
-
-      if (backgroundThreadException != null) {
-        throw new RuntimeException(backgroundThreadException);
-      }
-
-      // Wait for publish futures to complete.
-      Futures.allAsList(publishWaitList).get();
-
-      // Wait for handoff futures to complete.
-      // Note that every publishing task (created by calling AppenderatorDriver.publish()) has a corresponding
-      // handoffFuture. handoffFuture can throw an exception if 1) the corresponding publishFuture failed or 2) it
-      // failed to persist sequences. It might also return null if handoff failed, but was recoverable.
-      // See publishAndRegisterHandoff() for details.
-      List<SegmentsAndMetadata> handedOffList = Collections.emptyList();
-      if (tuningConfig.getHandoffConditionTimeout() == 0) {
-        handedOffList = Futures.allAsList(handOffWaitList).get();
-      } else {
-        try {
-          handedOffList = Futures.allAsList(handOffWaitList)
-                                 .get(tuningConfig.getHandoffConditionTimeout(), TimeUnit.MILLISECONDS);
-        }
-        catch (TimeoutException e) {
-          // Handoff timeout is not an indexing failure, but coordination failure. We simply ignore timeout exception
-          // here.
-          log.makeAlert("Timed out after [%d] millis waiting for handoffs", tuningConfig.getHandoffConditionTimeout())
-             .addData("TaskId", task.getId())
-             .emit();
-        }
-      }
-
-      for (SegmentsAndMetadata handedOff : handedOffList) {
-        log.info(
-            "Handoff completed for segments[%s] with metadata[%s].",
-            Joiner.on(", ").join(
-                handedOff.getSegments().stream().map(DataSegment::getIdentifier).collect(Collectors.toList())
-            ),
-            Preconditions.checkNotNull(handedOff.getCommitMetadata(), "commitMetadata")
-        );
-      }
-
-      appenderator.close();
-    }
-    catch (InterruptedException | RejectedExecutionException e) {
-      // (2) catch InterruptedException and RejectedExecutionException thrown for the whole ingestion steps including
-      // the final publishing.
-      caughtExceptionOuter = e;
-      try {
-        Futures.allAsList(publishWaitList).cancel(true);
-        Futures.allAsList(handOffWaitList).cancel(true);
-        if (appenderator != null) {
-          appenderator.closeNow();
-        }
-      }
-      catch (Exception e2) {
-        e.addSuppressed(e2);
-      }
-
-      // handle the InterruptedException that gets wrapped in a RejectedExecutionException
-      if (e instanceof RejectedExecutionException
-          && (e.getCause() == null || !(e.getCause() instanceof InterruptedException))) {
-        throw e;
-      }
-
-      // if we were interrupted because we were asked to stop, handle the exception and return success, else rethrow
-      if (!stopRequested.get()) {
-        Thread.currentThread().interrupt();
-        throw e;
-      }
-
-      log.info("The task was asked to stop before completing");
-    }
-    catch (Exception e) {
-      // (3) catch all other exceptions thrown for the whole ingestion steps including the final publishing.
-      caughtExceptionOuter = e;
-      try {
-        Futures.allAsList(publishWaitList).cancel(true);
-        Futures.allAsList(handOffWaitList).cancel(true);
-        if (appenderator != null) {
-          appenderator.closeNow();
-        }
-      }
-      catch (Exception e2) {
-        e.addSuppressed(e2);
-      }
-      throw e;
-    }
-    finally {
-      try {
-        if (driver != null) {
-          driver.close();
-        }
-        if (chatHandlerProvider.isPresent()) {
-          chatHandlerProvider.get().unregister(task.getId());
-        }
-
-        toolbox.getDruidNodeAnnouncer().unannounce(discoveryDruidNode);
-        toolbox.getDataSegmentServerAnnouncer().unannounce();
-      }
-      catch (Exception e) {
-        if (caughtExceptionOuter != null) {
-          caughtExceptionOuter.addSuppressed(e);
-        } else {
-          throw e;
-        }
-      }
-    }
-
-    toolbox.getTaskReportFileWriter().write(getTaskCompletionReports(null));
-    return TaskStatus.success(task.getId());
-  }
-
-  private void checkPublishAndHandoffFailure() throws ExecutionException, InterruptedException
-  {
-    // Check if any publishFuture failed.
-    final List<ListenableFuture<SegmentsAndMetadata>> publishFinished = publishWaitList
-        .stream()
-        .filter(Future::isDone)
-        .collect(Collectors.toList());
-
-    for (ListenableFuture<SegmentsAndMetadata> publishFuture : publishFinished) {
-      // If publishFuture failed, the below line will throw an exception and catched by (1), and then (2) or (3).
-      publishFuture.get();
-    }
-
-    publishWaitList.removeAll(publishFinished);
-
-    // Check if any handoffFuture failed.
-    final List<ListenableFuture<SegmentsAndMetadata>> handoffFinished = handOffWaitList
-        .stream()
-        .filter(Future::isDone)
-        .collect(Collectors.toList());
-
-    for (ListenableFuture<SegmentsAndMetadata> handoffFuture : handoffFinished) {
-      // If handoffFuture failed, the below line will throw an exception and catched by (1), and then (2) or (3).
-      handoffFuture.get();
-    }
-
-    handOffWaitList.removeAll(handoffFinished);
-  }
-
-  private void publishAndRegisterHandoff(SequenceMetadata sequenceMetadata)
-  {
-    log.info("Publishing segments for sequence [%s]", sequenceMetadata);
-
-    final ListenableFuture<SegmentsAndMetadata> publishFuture = Futures.transform(
-        driver.publish(
-            sequenceMetadata.createPublisher(toolbox, ioConfig.isUseTransaction()),
-            sequenceMetadata.getCommitterSupplier(topic, lastPersistedOffsets).get(),
-            Collections.singletonList(sequenceMetadata.getSequenceName())
-        ),
-        (Function<SegmentsAndMetadata, SegmentsAndMetadata>) publishedSegmentsAndMetadata -> {
-          if (publishedSegmentsAndMetadata == null) {
-            throw new ISE(
-                "Transaction failure publishing segments for sequence [%s]",
-                sequenceMetadata
-            );
-          } else {
-            return publishedSegmentsAndMetadata;
-          }
-        }
-    );
-    publishWaitList.add(publishFuture);
-
-    // Create a handoffFuture for every publishFuture. The created handoffFuture must fail if publishFuture fails.
-    final SettableFuture<SegmentsAndMetadata> handoffFuture = SettableFuture.create();
-    handOffWaitList.add(handoffFuture);
-
-    Futures.addCallback(
-        publishFuture,
-        new FutureCallback<SegmentsAndMetadata>()
-        {
-          @Override
-          public void onSuccess(SegmentsAndMetadata publishedSegmentsAndMetadata)
-          {
-            log.info(
-                "Published segments[%s] with metadata[%s].",
-                publishedSegmentsAndMetadata.getSegments()
-                                            .stream()
-                                            .map(DataSegment::getIdentifier)
-                                            .collect(Collectors.toList()),
-                Preconditions.checkNotNull(publishedSegmentsAndMetadata.getCommitMetadata(), "commitMetadata")
-            );
-
-            sequences.remove(sequenceMetadata);
-            publishingSequences.remove(sequenceMetadata.getSequenceName());
-            try {
-              persistSequences();
-            }
-            catch (IOException e) {
-              log.error(e, "Unable to persist state, dying");
-              handoffFuture.setException(e);
-              throw new RuntimeException(e);
-            }
-
-            Futures.transform(
-                driver.registerHandoff(publishedSegmentsAndMetadata),
-                new Function<SegmentsAndMetadata, Void>()
-                {
-                  @Nullable
-                  @Override
-                  public Void apply(@Nullable SegmentsAndMetadata handoffSegmentsAndMetadata)
-                  {
-                    if (handoffSegmentsAndMetadata == null) {
-                      log.warn(
-                          "Failed to handoff segments[%s]",
-                          publishedSegmentsAndMetadata.getSegments()
-                                                      .stream()
-                                                      .map(DataSegment::getIdentifier)
-                                                      .collect(Collectors.toList())
-                      );
-                    }
-                    handoffFuture.set(handoffSegmentsAndMetadata);
-                    return null;
-                  }
-                }
-            );
-          }
-
-          @Override
-          public void onFailure(Throwable t)
-          {
-            log.error(t, "Error while publishing segments for sequence[%s]", sequenceMetadata);
-            handoffFuture.setException(t);
-          }
-        }
-    );
-  }
-
-  private static File getSequencesPersistFile(TaskToolbox toolbox)
-  {
-    return new File(toolbox.getPersistDir(), "sequences.json");
-  }
-
-  private boolean restoreSequences() throws IOException
-  {
-    final File sequencesPersistFile = getSequencesPersistFile(toolbox);
-    if (sequencesPersistFile.exists()) {
-      sequences = new CopyOnWriteArrayList<>(
-          toolbox.getObjectMapper().<List<SequenceMetadata>>readValue(
-              sequencesPersistFile,
-              new TypeReference<List<SequenceMetadata>>()
-              {
-              }
-          )
-      );
-      return true;
-    } else {
-      return false;
-    }
-  }
-
-  private synchronized void persistSequences() throws IOException
-  {
-    log.info("Persisting Sequences Metadata [%s]", sequences);
-    toolbox.getObjectMapper().writerWithType(
-        new TypeReference<List<SequenceMetadata>>()
-        {
-        }
-    ).writeValue(getSequencesPersistFile(toolbox), sequences);
-  }
-
-  private Map<String, TaskReport> getTaskCompletionReports(@Nullable String errorMsg)
-  {
-    return TaskReport.buildTaskReports(
-        new IngestionStatsAndErrorsTaskReport(
-            task.getId(),
-            new IngestionStatsAndErrorsTaskReportData(
-                ingestionState,
-                getTaskCompletionUnparseableEvents(),
-                getTaskCompletionRowStats(),
-                errorMsg
-            )
-        )
-    );
-  }
-
-  private Map<String, Object> getTaskCompletionUnparseableEvents()
-  {
-    Map<String, Object> unparseableEventsMap = new HashMap<>();
-    List<String> buildSegmentsParseExceptionMessages = IndexTaskUtils.getMessagesFromSavedParseExceptions(
-        savedParseExceptions
-    );
-    if (buildSegmentsParseExceptionMessages != null) {
-      unparseableEventsMap.put(RowIngestionMeters.BUILD_SEGMENTS, buildSegmentsParseExceptionMessages);
-    }
-    return unparseableEventsMap;
-  }
-
-  private Map<String, Object> getTaskCompletionRowStats()
-  {
-    Map<String, Object> metrics = new HashMap<>();
-    metrics.put(
-        RowIngestionMeters.BUILD_SEGMENTS,
-        rowIngestionMeters.getTotals()
-    );
-    return metrics;
-  }
-
-  private void maybePersistAndPublishSequences(Supplier<Committer> committerSupplier)
-      throws InterruptedException
-  {
-    for (SequenceMetadata sequenceMetadata : sequences) {
-      sequenceMetadata.updateAssignments(nextOffsets);
-      if (!sequenceMetadata.isOpen() && !publishingSequences.contains(sequenceMetadata.getSequenceName())) {
-        publishingSequences.add(sequenceMetadata.getSequenceName());
-        try {
-          Object result = driver.persist(committerSupplier.get());
-          log.info(
-              "Persist completed with results: [%s], adding sequence [%s] to publish queue",
-              result,
-              sequenceMetadata
-          );
-          publishAndRegisterHandoff(sequenceMetadata);
-        }
-        catch (InterruptedException e) {
-          log.warn("Interrupted while persisting sequence [%s]", sequenceMetadata);
-          throw e;
-        }
-      }
-    }
-  }
-
-  private Set<Integer> assignPartitionsAndSeekToNext(KafkaConsumer consumer, String topic)
-  {
-    // Initialize consumer assignment.
-    final Set<Integer> assignment = new HashSet<>();
-    for (Map.Entry<Integer, Long> entry : nextOffsets.entrySet()) {
-      final long endOffset = endOffsets.get(entry.getKey());
-      if (entry.getValue() < endOffset) {
-        assignment.add(entry.getKey());
-      } else if (entry.getValue() == endOffset) {
-        log.info("Finished reading partition[%d].", entry.getKey());
-      } else {
-        throw new ISE(
-            "WTF?! Cannot start from offset[%,d] > endOffset[%,d]",
-            entry.getValue(),
-            endOffset
-        );
-      }
+  @Nonnull
+  @Override
+  protected List<OrderedPartitionableRecord<Integer, Long>> getRecords(
+      RecordSupplier<Integer, Long> recordSupplier,
+      TaskToolbox toolbox
+  ) throws Exception
+  {
+    // Handles OffsetOutOfRangeException, which is thrown if the seeked-to
+    // offset is not present in the topic-partition. This can happen if we're asking a task to read from data
+    // that has not been written yet (which is totally legitimate). So let's wait for it to show up.
+    List<OrderedPartitionableRecord<Integer, Long>> records = new ArrayList<>();
+    try {
+      records = recordSupplier.poll(KafkaIndexTask.POLL_TIMEOUT_MILLIS);
     }
-
-    KafkaIndexTask.assignPartitions(consumer, topic, assignment);
-
-    // Seek to starting offsets.
-    for (final int partition : assignment) {
-      final long offset = nextOffsets.get(partition);
-      log.info("Seeking partition[%d] to offset[%,d].", partition, offset);
-      consumer.seek(new TopicPartition(topic, partition), offset);
+    catch (OffsetOutOfRangeException e) {
+      log.warn("OffsetOutOfRangeException with message [%s]", e.getMessage());
+      possiblyResetOffsetsOrWait(e.offsetOutOfRangePartitions(), recordSupplier, toolbox);
     }
 
-    return assignment;
+    return records;
   }
 
-  /**
-   * Checks if the pauseRequested flag was set and if so blocks until pauseRequested is cleared.
-   * <p/>
-   * Sets paused = true and signals paused so callers can be notified when the pause command has been accepted.
-   * <p/>
-   *
-   * @return true if a pause request was handled, false otherwise
-   */
-  private boolean possiblyPause() throws InterruptedException
+  @Override
+  protected SeekableStreamPartitions<Integer, Long> deserializeSeekableStreamPartitionsFromMetadata(
+      ObjectMapper mapper,
+      Object object
+  )
   {
-    pauseLock.lockInterruptibly();
-    try {
-      if (pauseRequested) {
-        status = Status.PAUSED;
-        hasPaused.signalAll();
-
-        while (pauseRequested) {
-          log.info("Pausing ingestion until resumed");
-          shouldResume.await();
-        }
-
-        status = Status.READING;
-        shouldResume.signalAll();
-        log.info("Ingestion loop resumed");
-        return true;
-      }
-    }
-    finally {
-      pauseLock.unlock();
-    }
-
-    return false;
+    return mapper.convertValue(object, mapper.getTypeFactory().constructParametrizedType(
+        SeekableStreamPartitions.class,
+        SeekableStreamPartitions.class,
+        Integer.class,
+        Long.class
+    ));
   }
 
   private void possiblyResetOffsetsOrWait(
       Map<TopicPartition, Long> outOfRangePartitions,
-      KafkaConsumer<byte[], byte[]> consumer,
+      RecordSupplier<Integer, Long> recordSupplier,
       TaskToolbox taskToolbox
   ) throws InterruptedException, IOException
   {
     final Map<TopicPartition, Long> resetPartitions = new HashMap<>();
     boolean doReset = false;
-    if (tuningConfig.isResetOffsetAutomatically()) {
+    if (task.getTuningConfig().isResetOffsetAutomatically()) {
       for (Map.Entry<TopicPartition, Long> outOfRangePartition : outOfRangePartitions.entrySet()) {
         final TopicPartition topicPartition = outOfRangePartition.getKey();
         final long nextOffset = outOfRangePartition.getValue();
         // seek to the beginning to get the least available offset
-        consumer.seekToBeginning(Collections.singletonList(topicPartition));
-        final long leastAvailableOffset = consumer.position(topicPartition);
+        StreamPartition<Integer> streamPartition = StreamPartition.of(
+            topicPartition.topic(),
+            topicPartition.partition()
+        );
+        final Long leastAvailableOffset = recordSupplier.getEarliestSequenceNumber(streamPartition);
+        if (leastAvailableOffset == null) {
+          throw new ISE(
+              "got null sequence number for partition[%s] when fetching from kafka!",
+              topicPartition.partition()
+          );
+        }
         // reset the seek
-        consumer.seek(topicPartition, nextOffset);
+        recordSupplier.seek(streamPartition, nextOffset);
         // Reset consumer offset if resetOffsetAutomatically is set to true
         // and the current message offset in the kafka partition is more than the
         // next message offset that we are trying to fetch
@@ -1067,7 +161,12 @@ private void possiblyResetOffsetsOrWait(
     }
 
     if (doReset) {
-      sendResetRequestAndWait(resetPartitions, taskToolbox);
+      sendResetRequestAndWait(resetPartitions.entrySet()
+                                             .stream()
+                                             .collect(Collectors.toMap(x -> StreamPartition.of(
+                                                 x.getKey().topic(),
+                                                 x.getKey().partition()
+                                             ), Map.Entry::getValue)), taskToolbox);
     } else {
       log.warn("Retrying in %dms", task.getPollRetryMs());
       pollRetryLock.lockInterruptibly();
@@ -1083,721 +182,56 @@ private void possiblyResetOffsetsOrWait(
     }
   }
 
-  private void handleParseException(ParseException pe, ConsumerRecord<byte[], byte[]> record)
-  {
-    if (pe.isFromPartiallyValidRow()) {
-      rowIngestionMeters.incrementProcessedWithError();
-    } else {
-      rowIngestionMeters.incrementUnparseable();
-    }
-
-    if (tuningConfig.isLogParseExceptions()) {
-      log.error(
-          pe,
-          "Encountered parse exception on row from partition[%d] offset[%d]",
-          record.partition(),
-          record.offset()
-      );
-    }
-
-    if (savedParseExceptions != null) {
-      savedParseExceptions.add(pe);
-    }
-
-    if (rowIngestionMeters.getUnparseable() + rowIngestionMeters.getProcessedWithError()
-        > tuningConfig.getMaxParseExceptions()) {
-      log.error("Max parse exceptions exceeded, terminating task...");
-      throw new RuntimeException("Max parse exceptions exceeded, terminating task...");
-    }
-  }
-
-  private boolean isPaused()
-  {
-    return status == Status.PAUSED;
-  }
-
-  private void requestPause()
-  {
-    pauseRequested = true;
-  }
-
-  private void sendResetRequestAndWait(Map<TopicPartition, Long> outOfRangePartitions, TaskToolbox taskToolbox)
-      throws IOException
-  {
-    Map<Integer, Long> partitionOffsetMap = new HashMap<>();
-    for (Map.Entry<TopicPartition, Long> outOfRangePartition : outOfRangePartitions.entrySet()) {
-      partitionOffsetMap.put(outOfRangePartition.getKey().partition(), outOfRangePartition.getValue());
-    }
-    boolean result = taskToolbox.getTaskActionClient()
-                                .submit(new ResetDataSourceMetadataAction(
-                                    task.getDataSource(),
-                                    new KafkaDataSourceMetadata(new KafkaPartitions(
-                                        ioConfig.getStartPartitions()
-                                                .getTopic(),
-                                        partitionOffsetMap
-                                    ))
-                                ));
-
-    if (result) {
-      log.makeAlert("Resetting Kafka offsets for datasource [%s]", task.getDataSource())
-         .addData("partitions", partitionOffsetMap.keySet())
-         .emit();
-      // wait for being killed by supervisor
-      requestPause();
-    } else {
-      log.makeAlert("Failed to send reset request for partitions [%s]", partitionOffsetMap.keySet()).emit();
-    }
-  }
-
-  /**
-   * Authorizes action to be performed on this task's datasource
-   *
-   * @return authorization result
-   */
-  private Access authorizationCheck(final HttpServletRequest req, Action action)
-  {
-    return IndexTaskUtils.datasourceAuthorizationCheck(req, action, task.getDataSource(), authorizerMapper);
-  }
-
   @Override
-  public Appenderator getAppenderator()
-  {
-    return appenderator;
-  }
-
-  @Override
-  public RowIngestionMeters getRowIngestionMeters()
-  {
-    return rowIngestionMeters;
-  }
-
-  @Override
-  public void stopGracefully()
-  {
-    log.info("Stopping gracefully (status: [%s])", status);
-    stopRequested.set(true);
-
-    synchronized (statusLock) {
-      if (status == Status.PUBLISHING) {
-        runThread.interrupt();
-        return;
-      }
-    }
-
-    try {
-      if (pauseLock.tryLock(KafkaIndexTask.LOCK_ACQUIRE_TIMEOUT_SECONDS, TimeUnit.SECONDS)) {
-        try {
-          if (pauseRequested) {
-            pauseRequested = false;
-            shouldResume.signalAll();
-          }
-        }
-        finally {
-          pauseLock.unlock();
-        }
-      } else {
-        log.warn("While stopping: failed to acquire pauseLock before timeout, interrupting run thread");
-        runThread.interrupt();
-        return;
-      }
-
-      if (pollRetryLock.tryLock(KafkaIndexTask.LOCK_ACQUIRE_TIMEOUT_SECONDS, TimeUnit.SECONDS)) {
-        try {
-          isAwaitingRetry.signalAll();
-        }
-        finally {
-          pollRetryLock.unlock();
-        }
-      } else {
-        log.warn("While stopping: failed to acquire pollRetryLock before timeout, interrupting run thread");
-        runThread.interrupt();
-      }
-    }
-    catch (Exception e) {
-      Throwables.propagate(e);
-    }
-  }
-
-  @POST
-  @Path("/stop")
-  public Response stop(@Context final HttpServletRequest req)
-  {
-    authorizationCheck(req, Action.WRITE);
-    stopGracefully();
-    return Response.status(Response.Status.OK).build();
-  }
-
-  @GET
-  @Path("/status")
-  @Produces(MediaType.APPLICATION_JSON)
-  public Status getStatusHTTP(@Context final HttpServletRequest req)
-  {
-    authorizationCheck(req, Action.READ);
-    return status;
-  }
-
-  @Override
-  public Status getStatus()
-  {
-    return status;
-  }
-
-  @GET
-  @Path("/offsets/current")
-  @Produces(MediaType.APPLICATION_JSON)
-  public Map<Integer, Long> getCurrentOffsets(@Context final HttpServletRequest req)
+  protected SeekableStreamDataSourceMetadata<Integer, Long> createDataSourceMetadata(
+      SeekableStreamPartitions<Integer, Long> partitions
+  )
   {
-    authorizationCheck(req, Action.READ);
-    return getCurrentOffsets();
+    return new KafkaDataSourceMetadata(partitions);
   }
 
   @Override
-  public Map<Integer, Long> getCurrentOffsets()
-  {
-    return nextOffsets;
-  }
-
-  @GET
-  @Path("/offsets/end")
-  @Produces(MediaType.APPLICATION_JSON)
-  public Map<Integer, Long> getEndOffsetsHTTP(@Context final HttpServletRequest req)
+  protected OrderedSequenceNumber<Long> createSequenceNumber(Long sequenceNumber)
   {
-    authorizationCheck(req, Action.READ);
-    return getEndOffsets();
+    return KafkaSequenceNumber.of(sequenceNumber);
   }
 
   @Override
-  public Map<Integer, Long> getEndOffsets()
-  {
-    return endOffsets;
-  }
-
-  @POST
-  @Path("/offsets/end")
-  @Consumes(MediaType.APPLICATION_JSON)
-  @Produces(MediaType.APPLICATION_JSON)
-  public Response setEndOffsetsHTTP(
-      Map<Integer, Long> offsets,
-      @QueryParam("finish") @DefaultValue("true") final boolean finish,
-      // this field is only for internal purposes, shouldn't be usually set by users
-      @Context final HttpServletRequest req
-  ) throws InterruptedException
-  {
-    authorizationCheck(req, Action.WRITE);
-    return setEndOffsets(offsets, finish);
-  }
-
-  @GET
-  @Path("/rowStats")
-  @Produces(MediaType.APPLICATION_JSON)
-  public Response getRowStats(
-      @Context final HttpServletRequest req
-  )
-  {
-    authorizationCheck(req, Action.READ);
-    Map<String, Object> returnMap = new HashMap<>();
-    Map<String, Object> totalsMap = new HashMap<>();
-    Map<String, Object> averagesMap = new HashMap<>();
-
-    totalsMap.put(
-        RowIngestionMeters.BUILD_SEGMENTS,
-        rowIngestionMeters.getTotals()
-    );
-    averagesMap.put(
-        RowIngestionMeters.BUILD_SEGMENTS,
-        rowIngestionMeters.getMovingAverages()
-    );
-
-    returnMap.put("movingAverages", averagesMap);
-    returnMap.put("totals", totalsMap);
-    return Response.ok(returnMap).build();
-  }
-
-  @GET
-  @Path("/unparseableEvents")
-  @Produces(MediaType.APPLICATION_JSON)
-  public Response getUnparseableEvents(
-      @Context final HttpServletRequest req
+  protected void possiblyResetDataSourceMetadata(
+      TaskToolbox toolbox,
+      RecordSupplier<Integer, Long> recordSupplier,
+      Set<StreamPartition<Integer>> assignment,
+      Map<Integer, Long> currOffsets
   )
   {
-    authorizationCheck(req, Action.READ);
-    List<String> events = IndexTaskUtils.getMessagesFromSavedParseExceptions(savedParseExceptions);
-    return Response.ok(events).build();
+    // do nothing
   }
 
   @Override
-  public Response setEndOffsets(
-      Map<Integer, Long> offsets,
-      final boolean finish // this field is only for internal purposes, shouldn't be usually set by users
-  ) throws InterruptedException
-  {
-    if (offsets == null) {
-      return Response.status(Response.Status.BAD_REQUEST)
-                     .entity("Request body must contain a map of { partition:endOffset }")
-                     .build();
-    } else if (!endOffsets.keySet().containsAll(offsets.keySet())) {
-      return Response.status(Response.Status.BAD_REQUEST)
-                     .entity(
-                         StringUtils.format(
-                             "Request contains partitions not being handled by this task, my partitions: %s",
-                             endOffsets.keySet()
-                         )
-                     )
-                     .build();
-    } else {
-      try {
-        pauseLock.lockInterruptibly();
-        // Perform all sequence related checks before checking for isPaused()
-        // and after acquiring pauseLock to correctly guard against duplicate requests
-        Preconditions.checkState(sequences.size() > 0, "WTH?! No Sequences found to set end offsets");
-
-        final SequenceMetadata latestSequence = sequences.get(sequences.size() - 1);
-        if ((latestSequence.getStartOffsets().equals(offsets) && !finish) ||
-            (latestSequence.getEndOffsets().equals(offsets) && finish)) {
-          log.warn("Ignoring duplicate request, end offsets already set for sequences [%s]", sequences);
-          return Response.ok(offsets).build();
-        } else if (latestSequence.isCheckpointed()) {
-          return Response.status(Response.Status.BAD_REQUEST)
-                         .entity(StringUtils.format(
-                             "WTH?! Sequence [%s] has already endOffsets set, cannot set to [%s]",
-                             latestSequence,
-                             offsets
-                         )).build();
-        } else if (!isPaused()) {
-          return Response.status(Response.Status.BAD_REQUEST)
-                         .entity("Task must be paused before changing the end offsets")
-                         .build();
-        }
-
-        for (Map.Entry<Integer, Long> entry : offsets.entrySet()) {
-          if (entry.getValue().compareTo(nextOffsets.get(entry.getKey())) < 0) {
-            return Response.status(Response.Status.BAD_REQUEST)
-                           .entity(
-                               StringUtils.format(
-                                   "End offset must be >= current offset for partition [%s] (current: %s)",
-                                   entry.getKey(),
-                                   nextOffsets.get(entry.getKey())
-                               )
-                           )
-                           .build();
-          }
-        }
-
-        resetNextCheckpointTime();
-        latestSequence.setEndOffsets(offsets);
-
-        if (finish) {
-          log.info("Updating endOffsets from [%s] to [%s]", endOffsets, offsets);
-          endOffsets.putAll(offsets);
-        } else {
-          // create new sequence
-          final SequenceMetadata newSequence = new SequenceMetadata(
-              latestSequence.getSequenceId() + 1,
-              StringUtils.format("%s_%d", ioConfig.getBaseSequenceName(), latestSequence.getSequenceId() + 1),
-              offsets,
-              endOffsets,
-              false
-          );
-          sequences.add(newSequence);
-        }
-
-        persistSequences();
-      }
-      catch (Exception e) {
-        log.error(e, "Unable to set end offsets, dying");
-        backgroundThreadException = e;
-        // should resume to immediately finish kafka index task as failed
-        resume();
-        return Response.status(Response.Status.INTERNAL_SERVER_ERROR)
-                       .entity(Throwables.getStackTraceAsString(e))
-                       .build();
-      }
-      finally {
-        pauseLock.unlock();
-      }
-    }
-
-    resume();
-
-    return Response.ok(offsets).build();
-  }
-
-  private void resetNextCheckpointTime()
-  {
-    nextCheckpointTime = DateTimes.nowUtc().plus(tuningConfig.getIntermediateHandoffPeriod()).getMillis();
-  }
-
-  @GET
-  @Path("/checkpoints")
-  @Produces(MediaType.APPLICATION_JSON)
-  public Map<Integer, Map<Integer, Long>> getCheckpointsHTTP(@Context final HttpServletRequest req)
+  protected boolean isEndSequenceOffsetsExclusive()
   {
-    authorizationCheck(req, Action.READ);
-    return getCheckpoints();
-  }
-
-  private Map<Integer, Map<Integer, Long>> getCheckpoints()
-  {
-    TreeMap<Integer, Map<Integer, Long>> result = new TreeMap<>();
-    result.putAll(
-        sequences.stream().collect(Collectors.toMap(SequenceMetadata::getSequenceId, SequenceMetadata::getStartOffsets))
-    );
-    return result;
-  }
-
-  /**
-   * Signals the ingestion loop to pause.
-   *
-   * @return one of the following Responses: 400 Bad Request if the task has started publishing; 202 Accepted if the
-   * method has timed out and returned before the task has paused; 200 OK with a map of the current partition offsets
-   * in the response body if the task successfully paused
-   */
-  @POST
-  @Path("/pause")
-  @Produces(MediaType.APPLICATION_JSON)
-  public Response pauseHTTP(
-      @Context final HttpServletRequest req
-  ) throws InterruptedException
-  {
-    authorizationCheck(req, Action.WRITE);
-    return pause();
+    return true;
   }
 
   @Override
-  public Response pause() throws InterruptedException
-  {
-    if (!(status == Status.PAUSED || status == Status.READING)) {
-      return Response.status(Response.Status.BAD_REQUEST)
-                     .entity(StringUtils.format("Can't pause, task is not in a pausable state (state: [%s])", status))
-                     .build();
-    }
-
-    pauseLock.lockInterruptibly();
-    try {
-      pauseRequested = true;
-
-      pollRetryLock.lockInterruptibly();
-      try {
-        isAwaitingRetry.signalAll();
-      }
-      finally {
-        pollRetryLock.unlock();
-      }
-
-      if (isPaused()) {
-        shouldResume.signalAll(); // kick the monitor so it re-awaits with the new pauseMillis
-      }
-
-      long nanos = TimeUnit.SECONDS.toNanos(2);
-      while (!isPaused()) {
-        if (nanos <= 0L) {
-          return Response.status(Response.Status.ACCEPTED)
-                         .entity("Request accepted but task has not yet paused")
-                         .build();
-        }
-        nanos = hasPaused.awaitNanos(nanos);
-      }
-    }
-    finally {
-      pauseLock.unlock();
-    }
-
-    try {
-      return Response.ok().entity(toolbox.getObjectMapper().writeValueAsString(getCurrentOffsets())).build();
-    }
-    catch (JsonProcessingException e) {
-      throw Throwables.propagate(e);
-    }
-  }
-
-  @POST
-  @Path("/resume")
-  public Response resumeHTTP(@Context final HttpServletRequest req) throws InterruptedException
+  protected boolean isStartingSequenceOffsetsExclusive()
   {
-    authorizationCheck(req, Action.WRITE);
-    resume();
-    return Response.status(Response.Status.OK).build();
+    return false;
   }
 
   @Override
-  public void resume() throws InterruptedException
-  {
-    pauseLock.lockInterruptibly();
-    try {
-      pauseRequested = false;
-      shouldResume.signalAll();
-
-      long nanos = TimeUnit.SECONDS.toNanos(5);
-      while (isPaused()) {
-        if (nanos <= 0L) {
-          throw new RuntimeException("Resume command was not accepted within 5 seconds");
-        }
-        nanos = shouldResume.awaitNanos(nanos);
-      }
-    }
-    finally {
-      pauseLock.unlock();
-    }
-  }
-
-  @GET
-  @Path("/time/start")
-  @Produces(MediaType.APPLICATION_JSON)
-  public DateTime getStartTime(@Context final HttpServletRequest req)
-  {
-    authorizationCheck(req, Action.WRITE);
-    return startTime;
-  }
-
-  private static class SequenceMetadata
+  protected boolean isEndOfShard(Long seqNum)
   {
-    /**
-     * Lock for accessing {@link #endOffsets} and {@link #checkpointed}. This lock is required because
-     * {@link #setEndOffsets)} can be called by both the main thread and the HTTP thread.
-     */
-    private final ReentrantLock lock = new ReentrantLock();
-
-    private final int sequenceId;
-    private final String sequenceName;
-    private final Map<Integer, Long> startOffsets;
-    private final Map<Integer, Long> endOffsets;
-    private final Set<Integer> assignments;
-    private final boolean sentinel;
-    private boolean checkpointed;
-
-    @JsonCreator
-    public SequenceMetadata(
-        @JsonProperty("sequenceId") int sequenceId,
-        @JsonProperty("sequenceName") String sequenceName,
-        @JsonProperty("startOffsets") Map<Integer, Long> startOffsets,
-        @JsonProperty("endOffsets") Map<Integer, Long> endOffsets,
-        @JsonProperty("checkpointed") boolean checkpointed
-    )
-    {
-      Preconditions.checkNotNull(sequenceName);
-      Preconditions.checkNotNull(startOffsets);
-      Preconditions.checkNotNull(endOffsets);
-      this.sequenceId = sequenceId;
-      this.sequenceName = sequenceName;
-      this.startOffsets = ImmutableMap.copyOf(startOffsets);
-      this.endOffsets = new HashMap<>(endOffsets);
-      this.assignments = new HashSet<>(startOffsets.keySet());
-      this.checkpointed = checkpointed;
-      this.sentinel = false;
-    }
-
-    @JsonProperty
-    public int getSequenceId()
-    {
-      return sequenceId;
-    }
-
-    @JsonProperty
-    public boolean isCheckpointed()
-    {
-      lock.lock();
-      try {
-        return checkpointed;
-      }
-      finally {
-        lock.unlock();
-      }
-    }
-
-    @JsonProperty
-    public String getSequenceName()
-    {
-      return sequenceName;
-    }
-
-    @JsonProperty
-    public Map<Integer, Long> getStartOffsets()
-    {
-      return startOffsets;
-    }
-
-    @JsonProperty
-    public Map<Integer, Long> getEndOffsets()
-    {
-      lock.lock();
-      try {
-        return endOffsets;
-      }
-      finally {
-        lock.unlock();
-      }
-    }
-
-    @JsonProperty
-    public boolean isSentinel()
-    {
-      return sentinel;
-    }
-
-    void setEndOffsets(Map<Integer, Long> newEndOffsets)
-    {
-      lock.lock();
-      try {
-        endOffsets.putAll(newEndOffsets);
-        checkpointed = true;
-      }
-      finally {
-        lock.unlock();
-      }
-    }
-
-    void updateAssignments(Map<Integer, Long> nextPartitionOffset)
-    {
-      lock.lock();
-      try {
-        assignments.clear();
-        nextPartitionOffset.forEach((key, value) -> {
-          if (Longs.compare(endOffsets.get(key), nextPartitionOffset.get(key)) > 0) {
-            assignments.add(key);
-          }
-        });
-      }
-      finally {
-        lock.unlock();
-      }
-    }
-
-    boolean isOpen()
-    {
-      return !assignments.isEmpty();
-    }
-
-    boolean canHandle(ConsumerRecord<byte[], byte[]> record)
-    {
-      lock.lock();
-      try {
-        final Long partitionEndOffset = endOffsets.get(record.partition());
-        return isOpen()
-               && partitionEndOffset != null
-               && record.offset() >= startOffsets.get(record.partition())
-               && record.offset() < partitionEndOffset;
-      }
-      finally {
-        lock.unlock();
-      }
-    }
-
-    @Override
-    public String toString()
-    {
-      lock.lock();
-      try {
-        return "SequenceMetadata{" +
-               "sequenceName='" + sequenceName + '\'' +
-               ", sequenceId=" + sequenceId +
-               ", startOffsets=" + startOffsets +
-               ", endOffsets=" + endOffsets +
-               ", assignments=" + assignments +
-               ", sentinel=" + sentinel +
-               ", checkpointed=" + checkpointed +
-               '}';
-      }
-      finally {
-        lock.unlock();
-      }
-    }
-
-    Supplier<Committer> getCommitterSupplier(String topic, Map<Integer, Long> lastPersistedOffsets)
-    {
-      // Set up committer.
-      return () ->
-          new Committer()
-          {
-            @Override
-            public Object getMetadata()
-            {
-              lock.lock();
-
-              try {
-                Preconditions.checkState(
-                    assignments.isEmpty(),
-                    "This committer can be used only once all the records till offsets [%s] have been consumed, also make"
-                    + " sure to call updateAssignments before using this committer",
-                    endOffsets
-                );
-
-                // merge endOffsets for this sequence with globally lastPersistedOffsets
-                // This is done because this committer would be persisting only sub set of segments
-                // corresponding to the current sequence. Generally, lastPersistedOffsets should already
-                // cover endOffsets but just to be sure take max of offsets and persist that
-                for (Map.Entry<Integer, Long> partitionOffset : endOffsets.entrySet()) {
-                  lastPersistedOffsets.put(
-                      partitionOffset.getKey(),
-                      Math.max(
-                          partitionOffset.getValue(),
-                          lastPersistedOffsets.getOrDefault(partitionOffset.getKey(), 0L)
-                      )
-                  );
-                }
-
-                // Publish metadata can be different from persist metadata as we are going to publish only
-                // subset of segments
-                return ImmutableMap.of(
-                    METADATA_NEXT_PARTITIONS, new KafkaPartitions(topic, lastPersistedOffsets),
-                    METADATA_PUBLISH_PARTITIONS, new KafkaPartitions(topic, endOffsets)
-                );
-              }
-              finally {
-                lock.unlock();
-              }
-            }
-
-            @Override
-            public void run()
-            {
-              // Do nothing.
-            }
-          };
-    }
-
-    TransactionalSegmentPublisher createPublisher(TaskToolbox toolbox, boolean useTransaction)
-    {
-      return (segments, commitMetadata) -> {
-        final KafkaPartitions finalPartitions = toolbox.getObjectMapper().convertValue(
-            ((Map) Preconditions.checkNotNull(commitMetadata, "commitMetadata")).get(METADATA_PUBLISH_PARTITIONS),
-            KafkaPartitions.class
-        );
-
-        // Sanity check, we should only be publishing things that match our desired end state.
-        if (!getEndOffsets().equals(finalPartitions.getPartitionOffsetMap())) {
-          throw new ISE(
-              "WTF?! Driver for sequence [%s], attempted to publish invalid metadata[%s].",
-              toString(),
-              commitMetadata
-          );
-        }
-
-        final SegmentTransactionalInsertAction action;
-
-        if (useTransaction) {
-          action = new SegmentTransactionalInsertAction(
-              segments,
-              new KafkaDataSourceMetadata(new KafkaPartitions(finalPartitions.getTopic(), getStartOffsets())),
-              new KafkaDataSourceMetadata(finalPartitions)
-          );
-        } else {
-          action = new SegmentTransactionalInsertAction(segments, null, null);
-        }
-
-        log.info("Publishing with isTransaction[%s].", useTransaction);
-
-        return toolbox.getTaskActionClient().submit(action);
-      };
-    }
+    return false;
   }
 
   @Nullable
-  private static TreeMap<Integer, Map<Integer, Long>> getCheckPointsFromContext(
+  @Override
+  protected TreeMap<Integer, Map<Integer, Long>> getCheckPointsFromContext(
       TaskToolbox toolbox,
-      KafkaIndexTask task
+      String checkpointsString
   ) throws IOException
   {
-    final String checkpointsString = task.getContextValue("checkpoints");
     if (checkpointsString != null) {
       log.info("Checkpoints [%s]", checkpointsString);
       return toolbox.getObjectMapper().readValue(
@@ -1811,3 +245,4 @@ TransactionalSegmentPublisher createPublisher(TaskToolbox toolbox, boolean useTr
     }
   }
 }
+
diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java
index 16909892771..55b03c000f3 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java
@@ -21,133 +21,28 @@
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
-import org.apache.druid.indexing.overlord.DataSourceMetadata;
-import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata;
+import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
 
-import java.util.HashMap;
 import java.util.Map;
-import java.util.Objects;
 
-public class KafkaDataSourceMetadata implements DataSourceMetadata
+public class KafkaDataSourceMetadata extends SeekableStreamDataSourceMetadata<Integer, Long>
 {
-  private final KafkaPartitions kafkaPartitions;
 
   @JsonCreator
   public KafkaDataSourceMetadata(
-      @JsonProperty("partitions") KafkaPartitions kafkaPartitions
+      @JsonProperty("partitions") SeekableStreamPartitions<Integer, Long> kafkaPartitions
   )
   {
-    this.kafkaPartitions = kafkaPartitions;
-  }
-
-  @JsonProperty("partitions")
-  public KafkaPartitions getKafkaPartitions()
-  {
-    return kafkaPartitions;
-  }
-
-  @Override
-  public boolean isValidStart()
-  {
-    return true;
-  }
-
-  @Override
-  public boolean matches(DataSourceMetadata other)
-  {
-    if (getClass() != other.getClass()) {
-      return false;
-    }
-
-    return plus(other).equals(other.plus(this));
-  }
-
-  @Override
-  public DataSourceMetadata plus(DataSourceMetadata other)
-  {
-    if (!(other instanceof KafkaDataSourceMetadata)) {
-      throw new IAE(
-          "Expected instance of %s, got %s",
-          KafkaDataSourceMetadata.class.getCanonicalName(),
-          other.getClass().getCanonicalName()
-      );
-    }
-
-    final KafkaDataSourceMetadata that = (KafkaDataSourceMetadata) other;
-
-    if (that.getKafkaPartitions().getTopic().equals(kafkaPartitions.getTopic())) {
-      // Same topic, merge offsets.
-      final Map<Integer, Long> newMap = new HashMap<>();
-
-      for (Map.Entry<Integer, Long> entry : kafkaPartitions.getPartitionOffsetMap().entrySet()) {
-        newMap.put(entry.getKey(), entry.getValue());
-      }
-
-      for (Map.Entry<Integer, Long> entry : that.getKafkaPartitions().getPartitionOffsetMap().entrySet()) {
-        newMap.put(entry.getKey(), entry.getValue());
-      }
-
-      return new KafkaDataSourceMetadata(new KafkaPartitions(kafkaPartitions.getTopic(), newMap));
-    } else {
-      // Different topic, prefer "other".
-      return other;
-    }
-  }
-
-  @Override
-  public DataSourceMetadata minus(DataSourceMetadata other)
-  {
-    if (!(other instanceof KafkaDataSourceMetadata)) {
-      throw new IAE(
-          "Expected instance of %s, got %s",
-          KafkaDataSourceMetadata.class.getCanonicalName(),
-          other.getClass().getCanonicalName()
-      );
-    }
-
-    final KafkaDataSourceMetadata that = (KafkaDataSourceMetadata) other;
-
-    if (that.getKafkaPartitions().getTopic().equals(kafkaPartitions.getTopic())) {
-      // Same topic, remove partitions present in "that" from "this"
-      final Map<Integer, Long> newMap = new HashMap<>();
-
-      for (Map.Entry<Integer, Long> entry : kafkaPartitions.getPartitionOffsetMap().entrySet()) {
-        if (!that.getKafkaPartitions().getPartitionOffsetMap().containsKey(entry.getKey())) {
-          newMap.put(entry.getKey(), entry.getValue());
-        }
-      }
-
-      return new KafkaDataSourceMetadata(new KafkaPartitions(kafkaPartitions.getTopic(), newMap));
-    } else {
-      // Different topic, prefer "this".
-      return this;
-    }
+    super(kafkaPartitions);
   }
 
   @Override
-  public boolean equals(Object o)
-  {
-    if (this == o) {
-      return true;
-    }
-    if (o == null || getClass() != o.getClass()) {
-      return false;
-    }
-    KafkaDataSourceMetadata that = (KafkaDataSourceMetadata) o;
-    return Objects.equals(kafkaPartitions, that.kafkaPartitions);
-  }
-
-  @Override
-  public int hashCode()
-  {
-    return Objects.hash(kafkaPartitions);
-  }
-
-  @Override
-  public String toString()
+  protected SeekableStreamDataSourceMetadata<Integer, Long> createConcreteDataSourceMetaData(
+      String streamId,
+      Map<Integer, Long> newMap
+  )
   {
-    return "KafkaDataSourceMetadata{" +
-           "kafkaPartitions=" + kafkaPartitions +
-           '}';
+    return new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(streamId, newMap));
   }
 }
diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java
index bb73651e6e8..22ede084b3a 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java
@@ -24,77 +24,32 @@
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Joiner;
-import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import org.apache.druid.data.input.InputRow;
-import org.apache.druid.data.input.impl.InputRowParser;
-import org.apache.druid.indexer.TaskStatus;
-import org.apache.druid.indexing.appenderator.ActionBasedSegmentAllocator;
-import org.apache.druid.indexing.appenderator.ActionBasedUsedSegmentChecker;
-import org.apache.druid.indexing.common.TaskToolbox;
-import org.apache.druid.indexing.common.actions.SegmentAllocateAction;
-import org.apache.druid.indexing.common.actions.TaskActionClient;
 import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory;
-import org.apache.druid.indexing.common.task.AbstractTask;
-import org.apache.druid.indexing.common.task.RealtimeIndexTask;
 import org.apache.druid.indexing.common.task.TaskResource;
-import org.apache.druid.indexing.common.task.Tasks;
-import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisor;
-import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig;
-import org.apache.druid.java.util.common.Intervals;
-import org.apache.druid.java.util.common.StringUtils;
-import org.apache.druid.java.util.common.parsers.ParseException;
-import org.apache.druid.java.util.emitter.EmittingLogger;
-import org.apache.druid.metadata.PasswordProvider;
-import org.apache.druid.query.NoopQueryRunner;
-import org.apache.druid.query.Query;
-import org.apache.druid.query.QueryRunner;
+import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask;
+import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner;
+import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor;
 import org.apache.druid.segment.indexing.DataSchema;
-import org.apache.druid.segment.realtime.FireDepartmentMetrics;
-import org.apache.druid.segment.realtime.appenderator.Appenderator;
-import org.apache.druid.segment.realtime.appenderator.Appenderators;
-import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorDriver;
-import org.apache.druid.segment.realtime.firehose.ChatHandler;
 import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider;
 import org.apache.druid.server.security.AuthorizerMapper;
-import org.apache.druid.utils.CircularBuffer;
 import org.apache.kafka.clients.consumer.KafkaConsumer;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.serialization.ByteArrayDeserializer;
 
-import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 
-public class KafkaIndexTask extends AbstractTask implements ChatHandler
+public class KafkaIndexTask extends SeekableStreamIndexTask<Integer, Long>
 {
-  public enum Status
-  {
-    NOT_STARTED,
-    STARTING,
-    READING,
-    PAUSED,
-    PUBLISHING
-    // ideally this should be called FINISHING now as the task does incremental publishes
-    // through out its lifetime
-  }
-
-  private static final EmittingLogger log = new EmittingLogger(KafkaIndexTask.class);
   private static final String TYPE = "index_kafka";
   static final long POLL_TIMEOUT_MILLIS = TimeUnit.MILLISECONDS.toMillis(100);
-  static final long LOCK_ACQUIRE_TIMEOUT_SECONDS = 15;
 
-  private final DataSchema dataSchema;
-  private final InputRowParser<ByteBuffer> parser;
-  private final KafkaTuningConfig tuningConfig;
-  private final KafkaIOConfig ioConfig;
-  private final Optional<ChatHandlerProvider> chatHandlerProvider;
-  private final KafkaIndexTaskRunner runner;
+  private final KafkaIndexTaskIOConfig ioConfig;
   private final ObjectMapper configMapper;
 
   // This value can be tuned in some tests
@@ -105,8 +60,8 @@ public KafkaIndexTask(
       @JsonProperty("id") String id,
       @JsonProperty("resource") TaskResource taskResource,
       @JsonProperty("dataSchema") DataSchema dataSchema,
-      @JsonProperty("tuningConfig") KafkaTuningConfig tuningConfig,
-      @JsonProperty("ioConfig") KafkaIOConfig ioConfig,
+      @JsonProperty("tuningConfig") KafkaIndexTaskTuningConfig tuningConfig,
+      @JsonProperty("ioConfig") KafkaIndexTaskIOConfig ioConfig,
       @JsonProperty("context") Map<String, Object> context,
       @JacksonInject ChatHandlerProvider chatHandlerProvider,
       @JacksonInject AuthorizerMapper authorizerMapper,
@@ -115,46 +70,20 @@ public KafkaIndexTask(
   )
   {
     super(
-        id == null ? makeTaskId(dataSchema.getDataSource()) : id,
-        StringUtils.format("%s_%s", TYPE, dataSchema.getDataSource()),
+        id == null ? getFormattedId(dataSchema.getDataSource(), TYPE) : id,
         taskResource,
-        dataSchema.getDataSource(),
-        context
+        dataSchema,
+        tuningConfig,
+        ioConfig,
+        context,
+        chatHandlerProvider,
+        authorizerMapper,
+        rowIngestionMetersFactory,
+        getFormattedGroupId(dataSchema.getDataSource(), TYPE)
     );
-
-    this.dataSchema = Preconditions.checkNotNull(dataSchema, "dataSchema");
-    this.parser = Preconditions.checkNotNull((InputRowParser<ByteBuffer>) dataSchema.getParser(), "parser");
-    this.tuningConfig = Preconditions.checkNotNull(tuningConfig, "tuningConfig");
-    this.ioConfig = Preconditions.checkNotNull(ioConfig, "ioConfig");
-    this.chatHandlerProvider = Optional.fromNullable(chatHandlerProvider);
     this.configMapper = configMapper;
-    final CircularBuffer<Throwable> savedParseExceptions;
-    if (tuningConfig.getMaxSavedParseExceptions() > 0) {
-      savedParseExceptions = new CircularBuffer<>(tuningConfig.getMaxSavedParseExceptions());
-    } else {
-      savedParseExceptions = null;
-    }
+    this.ioConfig = ioConfig;
 
-    if (context != null && context.get(KafkaSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED) != null
-        && ((boolean) context.get(KafkaSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED))) {
-      runner = new IncrementalPublishingKafkaIndexTaskRunner(
-          this,
-          parser,
-          authorizerMapper,
-          this.chatHandlerProvider,
-          savedParseExceptions,
-          rowIngestionMetersFactory
-      );
-    } else {
-      runner = new LegacyKafkaIndexTaskRunner(
-          this,
-          parser,
-          authorizerMapper,
-          this.chatHandlerProvider,
-          savedParseExceptions,
-          rowIngestionMetersFactory
-      );
-    }
   }
 
   long getPollRetryMs()
@@ -162,126 +91,7 @@ long getPollRetryMs()
     return pollRetryMs;
   }
 
-  private static String makeTaskId(String dataSource)
-  {
-    return Joiner.on("_").join(TYPE, dataSource, RealtimeIndexTask.makeRandomId());
-  }
-
-  @Override
-  public int getPriority()
-  {
-    return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_REALTIME_TASK_PRIORITY);
-  }
-
-  @Override
-  public String getType()
-  {
-    return TYPE;
-  }
-
-  @Override
-  public boolean isReady(TaskActionClient taskActionClient)
-  {
-    return true;
-  }
-
-  @JsonProperty
-  public DataSchema getDataSchema()
-  {
-    return dataSchema;
-  }
-
-  @JsonProperty
-  public KafkaTuningConfig getTuningConfig()
-  {
-    return tuningConfig;
-  }
-
-  @JsonProperty("ioConfig")
-  public KafkaIOConfig getIOConfig()
-  {
-    return ioConfig;
-  }
-
-
-  @Override
-  public TaskStatus run(final TaskToolbox toolbox)
-  {
-    return runner.run(toolbox);
-  }
-
-  @Override
-  public boolean canRestore()
-  {
-    return true;
-  }
-
-  @Override
-  public void stopGracefully()
-  {
-    runner.stopGracefully();
-  }
-
-  @Override
-  public <T> QueryRunner<T> getQueryRunner(Query<T> query)
-  {
-    if (runner.getAppenderator() == null) {
-      // Not yet initialized, no data yet, just return a noop runner.
-      return new NoopQueryRunner<>();
-    }
-
-    return (queryPlus, responseContext) -> queryPlus.run(runner.getAppenderator(), responseContext);
-  }
-
-  Appenderator newAppenderator(FireDepartmentMetrics metrics, TaskToolbox toolbox)
-  {
-    return Appenderators.createRealtime(
-        dataSchema,
-        tuningConfig.withBasePersistDirectory(toolbox.getPersistDir()),
-        metrics,
-        toolbox.getSegmentPusher(),
-        toolbox.getObjectMapper(),
-        toolbox.getIndexIO(),
-        toolbox.getIndexMergerV9(),
-        toolbox.getQueryRunnerFactoryConglomerate(),
-        toolbox.getSegmentAnnouncer(),
-        toolbox.getEmitter(),
-        toolbox.getQueryExecutorService(),
-        toolbox.getCache(),
-        toolbox.getCacheConfig(),
-        toolbox.getCachePopulatorStats()
-    );
-  }
-
-  StreamAppenderatorDriver newDriver(
-      final Appenderator appenderator,
-      final TaskToolbox toolbox,
-      final FireDepartmentMetrics metrics
-  )
-  {
-    return new StreamAppenderatorDriver(
-        appenderator,
-        new ActionBasedSegmentAllocator(
-            toolbox.getTaskActionClient(),
-            dataSchema,
-            (schema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> new SegmentAllocateAction(
-                schema.getDataSource(),
-                row.getTimestamp(),
-                schema.getGranularitySpec().getQueryGranularity(),
-                schema.getGranularitySpec().getSegmentGranularity(),
-                sequenceName,
-                previousSegmentId,
-                skipSegmentLineageCheck
-            )
-        ),
-        toolbox.getSegmentHandoffNotifierFactory(),
-        new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()),
-        toolbox.getDataSegmentKiller(),
-        toolbox.getObjectMapper(),
-        metrics
-    );
-  }
-
+  @Deprecated
   KafkaConsumer<byte[], byte[]> newConsumer()
   {
     ClassLoader currCtxCl = Thread.currentThread().getContextClassLoader();
@@ -290,7 +100,11 @@ StreamAppenderatorDriver newDriver(
 
       final Properties props = new Properties();
 
-      addConsumerPropertiesFromConfig(props, configMapper, ioConfig.getConsumerProperties());
+      KafkaRecordSupplier.addConsumerPropertiesFromConfig(
+          props,
+          configMapper,
+          ioConfig.getConsumerProperties()
+      );
 
       props.setProperty("enable.auto.commit", "false");
       props.setProperty("auto.offset.reset", "none");
@@ -304,25 +118,7 @@ StreamAppenderatorDriver newDriver(
     }
   }
 
-  public static void addConsumerPropertiesFromConfig(Properties properties, ObjectMapper configMapper, Map<String, Object> consumerProperties)
-  {
-    // Extract passwords before SSL connection to Kafka
-    for (Map.Entry<String, Object> entry : consumerProperties.entrySet()) {
-      String propertyKey = entry.getKey();
-      if (propertyKey.equals(KafkaSupervisorIOConfig.TRUST_STORE_PASSWORD_KEY)
-          || propertyKey.equals(KafkaSupervisorIOConfig.KEY_STORE_PASSWORD_KEY)
-          || propertyKey.equals(KafkaSupervisorIOConfig.KEY_PASSWORD_KEY)) {
-        PasswordProvider configPasswordProvider = configMapper.convertValue(
-            entry.getValue(),
-            PasswordProvider.class
-        );
-        properties.setProperty(propertyKey, configPasswordProvider.getPassword());
-      } else {
-        properties.setProperty(propertyKey, String.valueOf(entry.getValue()));
-      }
-    }
-  }
-
+  @Deprecated
   static void assignPartitions(
       final KafkaConsumer consumer,
       final String topic,
@@ -330,45 +126,62 @@ static void assignPartitions(
   )
   {
     consumer.assign(
-        Lists.newArrayList(
+        new ArrayList<>(
             partitions.stream().map(n -> new TopicPartition(topic, n)).collect(Collectors.toList())
         )
     );
   }
 
-  boolean withinMinMaxRecordTime(final InputRow row)
+  @Override
+  protected SeekableStreamIndexTaskRunner<Integer, Long> createTaskRunner()
   {
-    final boolean beforeMinimumMessageTime = ioConfig.getMinimumMessageTime().isPresent()
-                                             && ioConfig.getMinimumMessageTime().get().isAfter(row.getTimestamp());
-
-    final boolean afterMaximumMessageTime = ioConfig.getMaximumMessageTime().isPresent()
-                                            && ioConfig.getMaximumMessageTime().get().isBefore(row.getTimestamp());
-
-    if (!Intervals.ETERNITY.contains(row.getTimestamp())) {
-      final String errorMsg = StringUtils.format(
-          "Encountered row with timestamp that cannot be represented as a long: [%s]",
-          row
+    if (context != null && context.get(SeekableStreamSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED) != null
+        && ((boolean) context.get(SeekableStreamSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED))) {
+      return new IncrementalPublishingKafkaIndexTaskRunner(
+          this,
+          parser,
+          authorizerMapper,
+          chatHandlerProvider,
+          savedParseExceptions,
+          rowIngestionMetersFactory
+      );
+    } else {
+      return new LegacyKafkaIndexTaskRunner(
+          this,
+          parser,
+          authorizerMapper,
+          chatHandlerProvider,
+          savedParseExceptions,
+          rowIngestionMetersFactory
       );
-      throw new ParseException(errorMsg);
     }
+  }
+
+  @Override
+  protected KafkaRecordSupplier newTaskRecordSupplier()
+  {
+    ClassLoader currCtxCl = Thread.currentThread().getContextClassLoader();
+    try {
+      Thread.currentThread().setContextClassLoader(getClass().getClassLoader());
+
+      final Map<String, Object> props = new HashMap<>(((KafkaIndexTaskIOConfig) super.ioConfig).getConsumerProperties());
 
-    if (log.isDebugEnabled()) {
-      if (beforeMinimumMessageTime) {
-        log.debug(
-            "CurrentTimeStamp[%s] is before MinimumMessageTime[%s]",
-            row.getTimestamp(),
-            ioConfig.getMinimumMessageTime().get()
-        );
-      } else if (afterMaximumMessageTime) {
-        log.debug(
-            "CurrentTimeStamp[%s] is after MaximumMessageTime[%s]",
-            row.getTimestamp(),
-            ioConfig.getMaximumMessageTime().get()
-        );
-      }
+      props.put("auto.offset.reset", "none");
+      props.put("key.deserializer", ByteArrayDeserializer.class.getName());
+      props.put("value.deserializer", ByteArrayDeserializer.class.getName());
+
+      return new KafkaRecordSupplier(props, configMapper);
+    }
+    finally {
+      Thread.currentThread().setContextClassLoader(currCtxCl);
     }
+  }
 
-    return !beforeMinimumMessageTime && !afterMaximumMessageTime;
+  @Override
+  @JsonProperty
+  public KafkaIndexTaskTuningConfig getTuningConfig()
+  {
+    return (KafkaIndexTaskTuningConfig) super.getTuningConfig();
   }
 
   @VisibleForTesting
@@ -377,15 +190,16 @@ void setPollRetryMs(long retryMs)
     this.pollRetryMs = retryMs;
   }
 
-  @VisibleForTesting
-  Appenderator getAppenderator()
+  @Override
+  @JsonProperty("ioConfig")
+  public KafkaIndexTaskIOConfig getIOConfig()
   {
-    return runner.getAppenderator();
+    return (KafkaIndexTaskIOConfig) super.getIOConfig();
   }
 
-  @VisibleForTesting
-  KafkaIndexTaskRunner getRunner()
+  @Override
+  public String getType()
   {
-    return runner;
+    return TYPE;
   }
 }
diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClient.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClient.java
index 30f1c89a910..a915f44e09d 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClient.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClient.java
@@ -19,37 +19,15 @@
 
 package org.apache.druid.indexing.kafka;
 
-import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.util.concurrent.ListenableFuture;
-import org.apache.druid.indexing.common.IndexTaskClient;
 import org.apache.druid.indexing.common.TaskInfoProvider;
-import org.apache.druid.indexing.kafka.KafkaIndexTask.Status;
-import org.apache.druid.java.util.common.ISE;
-import org.apache.druid.java.util.common.RE;
-import org.apache.druid.java.util.common.StringUtils;
-import org.apache.druid.java.util.common.jackson.JacksonUtils;
-import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClient;
 import org.apache.druid.java.util.http.client.HttpClient;
-import org.apache.druid.java.util.http.client.response.FullResponseHolder;
-import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
-import org.joda.time.DateTime;
 import org.joda.time.Duration;
 
-import javax.annotation.Nullable;
-import java.io.IOException;
-import java.util.Collections;
-import java.util.Map;
-import java.util.TreeMap;
-
-public class KafkaIndexTaskClient extends IndexTaskClient
+public class KafkaIndexTaskClient extends SeekableStreamIndexTaskClient<Integer, Long>
 {
-  private static final EmittingLogger log = new EmittingLogger(KafkaIndexTaskClient.class);
-  private static final TreeMap<Integer, Map<Integer, Long>> EMPTY_TREE_MAP = new TreeMap<>();
-
-  public KafkaIndexTaskClient(
+  KafkaIndexTaskClient(
       HttpClient httpClient,
       ObjectMapper jsonMapper,
       TaskInfoProvider taskInfoProvider,
@@ -59,315 +37,26 @@ public KafkaIndexTaskClient(
       long numRetries
   )
   {
-    super(httpClient, jsonMapper, taskInfoProvider, httpTimeout, dataSource, numThreads, numRetries);
-  }
-
-  public boolean stop(final String id, final boolean publish)
-  {
-    log.debug("Stop task[%s] publish[%s]", id, publish);
-
-    try {
-      final FullResponseHolder response = submitRequestWithEmptyContent(
-          id,
-          HttpMethod.POST,
-          "stop",
-          publish ? "publish=true" : null,
-          true
-      );
-      return isSuccess(response);
-    }
-    catch (NoTaskLocationException e) {
-      return false;
-    }
-    catch (TaskNotRunnableException e) {
-      log.info("Task [%s] couldn't be stopped because it is no longer running", id);
-      return true;
-    }
-    catch (Exception e) {
-      log.warn(e, "Exception while stopping task [%s]", id);
-      return false;
-    }
-  }
-
-  public boolean resume(final String id)
-  {
-    log.debug("Resume task[%s]", id);
-
-    try {
-      final FullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.POST, "resume", null, true);
-      return isSuccess(response);
-    }
-    catch (NoTaskLocationException | IOException e) {
-      log.warn(e, "Exception while stopping task [%s]", id);
-      return false;
-    }
-  }
-
-  public Map<Integer, Long> pause(final String id)
-  {
-    log.debug("Pause task[%s]", id);
-
-    try {
-      final FullResponseHolder response = submitRequestWithEmptyContent(
-          id,
-          HttpMethod.POST,
-          "pause",
-          null,
-          true
-      );
-
-      final HttpResponseStatus responseStatus = response.getStatus();
-      final String responseContent = response.getContent();
-
-      if (responseStatus.equals(HttpResponseStatus.OK)) {
-        log.info("Task [%s] paused successfully", id);
-        return deserialize(responseContent, new TypeReference<Map<Integer, Long>>()
-        {
-        });
-      } else if (responseStatus.equals(HttpResponseStatus.ACCEPTED)) {
-        // The task received the pause request, but its status hasn't been changed yet.
-        while (true) {
-          final Status status = getStatus(id);
-          if (status == KafkaIndexTask.Status.PAUSED) {
-            return getCurrentOffsets(id, true);
-          }
-
-          final Duration delay = newRetryPolicy().getAndIncrementRetryDelay();
-          if (delay == null) {
-            throw new ISE(
-                "Task [%s] failed to change its status from [%s] to [%s], aborting",
-                id,
-                status,
-                Status.PAUSED
-            );
-          } else {
-            final long sleepTime = delay.getMillis();
-            log.info(
-                "Still waiting for task [%s] to change its status to [%s]; will try again in [%s]",
-                id,
-                Status.PAUSED,
-                new Duration(sleepTime).toString()
-            );
-            Thread.sleep(sleepTime);
-          }
-        }
-      } else {
-        throw new ISE(
-            "Pause request for task [%s] failed with response [%s] : [%s]",
-            id,
-            responseStatus,
-            responseContent
-        );
-      }
-    }
-    catch (NoTaskLocationException e) {
-      log.error("Exception [%s] while pausing Task [%s]", e.getMessage(), id);
-      return ImmutableMap.of();
-    }
-    catch (IOException | InterruptedException e) {
-      throw new RE(e, "Exception [%s] while pausing Task [%s]", e.getMessage(), id);
-    }
-  }
-
-  public KafkaIndexTask.Status getStatus(final String id)
-  {
-    log.debug("GetStatus task[%s]", id);
-
-    try {
-      final FullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "status", null, true);
-      return deserialize(response.getContent(), KafkaIndexTask.Status.class);
-    }
-    catch (NoTaskLocationException e) {
-      return KafkaIndexTask.Status.NOT_STARTED;
-    }
-    catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  @Nullable
-  public DateTime getStartTime(final String id)
-  {
-    log.debug("GetStartTime task[%s]", id);
-
-    try {
-      final FullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "time/start", null, true);
-      return response.getContent() == null || response.getContent().isEmpty()
-             ? null
-             : deserialize(response.getContent(), DateTime.class);
-    }
-    catch (NoTaskLocationException e) {
-      return null;
-    }
-    catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  public Map<String, Object> getMovingAverages(final String id)
-  {
-    log.debug("GetMovingAverages task[%s]", id);
-
-    try {
-      final FullResponseHolder response = submitRequestWithEmptyContent(
-          id,
-          HttpMethod.GET,
-          "rowStats",
-          null,
-          true
-      );
-      return response.getContent() == null || response.getContent().isEmpty()
-             ? Collections.emptyMap()
-             : deserialize(response.getContent(), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT);
-    }
-    catch (NoTaskLocationException e) {
-      return Collections.emptyMap();
-    }
-    catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  public Map<Integer, Long> getCurrentOffsets(final String id, final boolean retry)
-  {
-    log.debug("GetCurrentOffsets task[%s] retry[%s]", id, retry);
-
-    try {
-      final FullResponseHolder response = submitRequestWithEmptyContent(
-          id,
-          HttpMethod.GET,
-          "offsets/current",
-          null,
-          retry
-      );
-      return deserialize(response.getContent(), new TypeReference<Map<Integer, Long>>()
-      {
-      });
-    }
-    catch (NoTaskLocationException e) {
-      return ImmutableMap.of();
-    }
-    catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  public TreeMap<Integer, Map<Integer, Long>> getCheckpoints(final String id, final boolean retry)
-  {
-    log.debug("GetCheckpoints task[%s] retry[%s]", id, retry);
-    try {
-      final FullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "checkpoints", null, retry);
-      return deserialize(
-          response.getContent(),
-          new TypeReference<TreeMap<Integer, Map<Integer, Long>>>()
-          {
-          }
-      );
-    }
-    catch (NoTaskLocationException e) {
-      return EMPTY_TREE_MAP;
-    }
-    catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  public ListenableFuture<TreeMap<Integer, Map<Integer, Long>>> getCheckpointsAsync(
-      final String id,
-      final boolean retry
-  )
-  {
-    return doAsync(() -> getCheckpoints(id, retry));
-  }
-
-  public Map<Integer, Long> getEndOffsets(final String id)
-  {
-    log.debug("GetEndOffsets task[%s]", id);
-
-    try {
-      final FullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "offsets/end", null, true);
-      return deserialize(response.getContent(), new TypeReference<Map<Integer, Long>>()
-      {
-      });
-    }
-    catch (NoTaskLocationException e) {
-      return ImmutableMap.of();
-    }
-    catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  public boolean setEndOffsets(
-      final String id,
-      final Map<Integer, Long> endOffsets,
-      final boolean finalize
-  ) throws IOException
-  {
-    log.debug("SetEndOffsets task[%s] endOffsets[%s] finalize[%s]", id, endOffsets, finalize);
-
-    try {
-      final FullResponseHolder response = submitJsonRequest(
-          id,
-          HttpMethod.POST,
-          "offsets/end",
-          StringUtils.format("finish=%s", finalize),
-          serialize(endOffsets),
-          true
-      );
-      return isSuccess(response);
-    }
-    catch (NoTaskLocationException e) {
-      return false;
-    }
+    super(
+        httpClient,
+        jsonMapper,
+        taskInfoProvider,
+        dataSource,
+        numThreads,
+        httpTimeout,
+        numRetries
+    );
   }
 
-  public ListenableFuture<Boolean> stopAsync(final String id, final boolean publish)
-  {
-    return doAsync(() -> stop(id, publish));
-  }
-
-  public ListenableFuture<Boolean> resumeAsync(final String id)
-  {
-    return doAsync(() -> resume(id));
-  }
-
-  public ListenableFuture<Map<Integer, Long>> pauseAsync(final String id)
-  {
-    return doAsync(() -> pause(id));
-  }
-
-  public ListenableFuture<KafkaIndexTask.Status> getStatusAsync(final String id)
-  {
-    return doAsync(() -> getStatus(id));
-  }
-
-  public ListenableFuture<DateTime> getStartTimeAsync(final String id)
-  {
-    return doAsync(() -> getStartTime(id));
-  }
-
-  public ListenableFuture<Map<Integer, Long>> getCurrentOffsetsAsync(final String id, final boolean retry)
-  {
-    return doAsync(() -> getCurrentOffsets(id, retry));
-  }
-
-  public ListenableFuture<Map<Integer, Long>> getEndOffsetsAsync(final String id)
-  {
-    return doAsync(() -> getEndOffsets(id));
-  }
-
-  public ListenableFuture<Boolean> setEndOffsetsAsync(
-      final String id,
-      final Map<Integer, Long> endOffsets,
-      final boolean finalize
-  )
+  @Override
+  protected Class<Integer> getPartitionType()
   {
-    return doAsync(() -> setEndOffsets(id, endOffsets, finalize));
+    return Integer.class;
   }
 
-  public ListenableFuture<Map<String, Object>> getMovingAveragesAsync(final String id)
+  @Override
+  protected Class<Long> getSequenceType()
   {
-    return doAsync(() -> getMovingAverages(id));
+    return Long.class;
   }
 }
diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientFactory.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientFactory.java
index d7f7f61be4c..cfa7c723655 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientFactory.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientFactory.java
@@ -24,23 +24,19 @@
 import org.apache.druid.guice.annotations.EscalatedGlobal;
 import org.apache.druid.guice.annotations.Json;
 import org.apache.druid.indexing.common.TaskInfoProvider;
-import org.apache.druid.indexing.common.task.IndexTaskClientFactory;
+import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory;
 import org.apache.druid.java.util.http.client.HttpClient;
 import org.joda.time.Duration;
 
-public class KafkaIndexTaskClientFactory implements IndexTaskClientFactory<KafkaIndexTaskClient>
+public class KafkaIndexTaskClientFactory extends SeekableStreamIndexTaskClientFactory<KafkaIndexTaskClient>
 {
-  private HttpClient httpClient;
-  private ObjectMapper mapper;
-
   @Inject
   public KafkaIndexTaskClientFactory(
       @EscalatedGlobal HttpClient httpClient,
       @Json ObjectMapper mapper
   )
   {
-    this.httpClient = httpClient;
-    this.mapper = mapper;
+    super(httpClient, mapper);
   }
 
   @Override
@@ -53,8 +49,8 @@ public KafkaIndexTaskClient build(
   )
   {
     return new KafkaIndexTaskClient(
-        httpClient,
-        mapper,
+        getHttpClient(),
+        getMapper(),
         taskInfoProvider,
         dataSource,
         numThreads,
diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskIOConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskIOConfig.java
new file mode 100644
index 00000000000..fc5c28751bc
--- /dev/null
+++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskIOConfig.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.indexing.kafka;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig;
+import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
+import org.joda.time.DateTime;
+
+import javax.annotation.Nullable;
+import java.util.Map;
+
+public class KafkaIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig<Integer, Long>
+{
+  private final Map<String, Object> consumerProperties;
+
+  @JsonCreator
+  public KafkaIndexTaskIOConfig(
+      @JsonProperty("taskGroupId") @Nullable Integer taskGroupId, // can be null for backward compabitility
+      @JsonProperty("baseSequenceName") String baseSequenceName,
+      @JsonProperty("startPartitions") SeekableStreamPartitions<Integer, Long> startPartitions,
+      @JsonProperty("endPartitions") SeekableStreamPartitions<Integer, Long> endPartitions,
+      @JsonProperty("consumerProperties") Map<String, Object> consumerProperties,
+      @JsonProperty("useTransaction") Boolean useTransaction,
+      @JsonProperty("minimumMessageTime") DateTime minimumMessageTime,
+      @JsonProperty("maximumMessageTime") DateTime maximumMessageTime,
+      @JsonProperty("skipOffsetGaps") Boolean skipOffsetGaps
+  )
+  {
+    super(
+        taskGroupId,
+        baseSequenceName,
+        startPartitions,
+        endPartitions,
+        useTransaction,
+        minimumMessageTime,
+        maximumMessageTime,
+        skipOffsetGaps,
+        null
+    );
+
+    this.consumerProperties = Preconditions.checkNotNull(consumerProperties, "consumerProperties");
+
+    for (int partition : endPartitions.getPartitionSequenceNumberMap().keySet()) {
+      Preconditions.checkArgument(
+          endPartitions.getPartitionSequenceNumberMap()
+                       .get(partition)
+                       .compareTo(startPartitions.getPartitionSequenceNumberMap().get(partition)) >= 0,
+          "end offset must be >= start offset for partition[%s]",
+          partition
+      );
+    }
+  }
+
+  @JsonProperty
+  public Map<String, Object> getConsumerProperties()
+  {
+    return consumerProperties;
+  }
+
+  @Override
+  public String toString()
+  {
+    return "KafkaIndexTaskIOConfig{" +
+           "taskGroupId=" + getTaskGroupId() +
+           ", baseSequenceName='" + getBaseSequenceName() + '\'' +
+           ", startPartitions=" + getStartPartitions() +
+           ", endPartitions=" + getEndPartitions() +
+           ", consumerProperties=" + consumerProperties +
+           ", useTransaction=" + isUseTransaction() +
+           ", minimumMessageTime=" + getMinimumMessageTime() +
+           ", maximumMessageTime=" + getMaximumMessageTime() +
+           ", skipOffsetGaps=" + isSkipOffsetGaps() +
+           '}';
+  }
+}
diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskModule.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskModule.java
index 751421057ab..47b5df82aeb 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskModule.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskModule.java
@@ -26,9 +26,9 @@
 import com.google.inject.Binder;
 import com.google.inject.TypeLiteral;
 import org.apache.druid.guice.LazySingleton;
-import org.apache.druid.indexing.common.task.IndexTaskClientFactory;
 import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpec;
 import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorTuningConfig;
+import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory;
 import org.apache.druid.initialization.DruidModule;
 
 import java.util.List;
@@ -43,7 +43,7 @@
             .registerSubtypes(
                 new NamedType(KafkaIndexTask.class, "index_kafka"),
                 new NamedType(KafkaDataSourceMetadata.class, "kafka"),
-                new NamedType(KafkaIOConfig.class, "kafka"),
+                new NamedType(KafkaIndexTaskIOConfig.class, "kafka"),
                 new NamedType(KafkaSupervisorTuningConfig.class, "kafka"),
                 new NamedType(KafkaSupervisorSpec.class, "kafka")
             )
@@ -54,7 +54,7 @@
   public void configure(Binder binder)
   {
     binder.bind(
-        new TypeLiteral<IndexTaskClientFactory<KafkaIndexTaskClient>>()
+        new TypeLiteral<SeekableStreamIndexTaskClientFactory<KafkaIndexTaskClient>>()
         {
         }
     ).to(KafkaIndexTaskClientFactory.class).in(LazySingleton.class);
diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskRunner.java
deleted file mode 100644
index 348cfcc8c0b..00000000000
--- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskRunner.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.indexing.kafka;
-
-import com.google.common.annotations.VisibleForTesting;
-import org.apache.druid.indexer.TaskStatus;
-import org.apache.druid.indexing.common.TaskToolbox;
-import org.apache.druid.indexing.common.stats.RowIngestionMeters;
-import org.apache.druid.indexing.kafka.KafkaIndexTask.Status;
-import org.apache.druid.segment.realtime.appenderator.Appenderator;
-import org.apache.druid.segment.realtime.firehose.ChatHandler;
-
-import javax.ws.rs.core.Response;
-import java.util.Map;
-
-/**
- * This class is used by only {@link KafkaIndexTask}. We currently have two implementations of this interface, i.e.,
- * {@link IncrementalPublishingKafkaIndexTaskRunner} and {@link LegacyKafkaIndexTaskRunner}. The latter one was used in
- * the versions prior to 0.12.0, but being kept to support rolling update from them.
- *
- * We don't have a good reason for having this interface except for better code maintenance for the latest kakfa
- * indexing algorithm. As a result, this interface can be removed in the future when {@link LegacyKafkaIndexTaskRunner}
- * is removed and it's no longer useful.
- */
-public interface KafkaIndexTaskRunner extends ChatHandler
-{
-  Appenderator getAppenderator();
-
-  TaskStatus run(TaskToolbox toolbox);
-
-  void stopGracefully();
-
-  // The below methods are mostly for unit testing.
-
-  @VisibleForTesting
-  RowIngestionMeters getRowIngestionMeters();
-  @VisibleForTesting
-  Status getStatus();
-
-  @VisibleForTesting
-  Map<Integer, Long> getCurrentOffsets();
-  @VisibleForTesting
-  Map<Integer, Long> getEndOffsets();
-  @VisibleForTesting
-  Response setEndOffsets(
-      Map<Integer, Long> offsets,
-      boolean finish // this field is only for internal purposes, shouldn't be usually set by users
-  ) throws InterruptedException;
-
-  @VisibleForTesting
-  Response pause() throws InterruptedException;
-  @VisibleForTesting
-  void resume() throws InterruptedException;
-}
diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfig.java
new file mode 100644
index 00000000000..7cee8779065
--- /dev/null
+++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfig.java
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.indexing.kafka;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig;
+import org.apache.druid.segment.IndexSpec;
+import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory;
+import org.joda.time.Period;
+
+import javax.annotation.Nullable;
+import java.io.File;
+
+@JsonTypeName("KafkaTuningConfig")
+public class KafkaIndexTaskTuningConfig extends SeekableStreamIndexTaskTuningConfig
+{
+  @JsonCreator
+  public KafkaIndexTaskTuningConfig(
+      @JsonProperty("maxRowsInMemory") @Nullable Integer maxRowsInMemory,
+      @JsonProperty("maxBytesInMemory") @Nullable Long maxBytesInMemory,
+      @JsonProperty("maxRowsPerSegment") @Nullable Integer maxRowsPerSegment,
+      @JsonProperty("maxTotalRows") @Nullable Long maxTotalRows,
+      @JsonProperty("intermediatePersistPeriod") @Nullable Period intermediatePersistPeriod,
+      @JsonProperty("basePersistDirectory") @Nullable File basePersistDirectory,
+      @JsonProperty("maxPendingPersists") @Nullable Integer maxPendingPersists,
+      @JsonProperty("indexSpec") @Nullable IndexSpec indexSpec,
+      // This parameter is left for compatibility when reading existing configs, to be removed in Druid 0.12.
+      @JsonProperty("buildV9Directly") @Nullable Boolean buildV9Directly,
+      @Deprecated @JsonProperty("reportParseExceptions") @Nullable Boolean reportParseExceptions,
+      @JsonProperty("handoffConditionTimeout") @Nullable Long handoffConditionTimeout,
+      @JsonProperty("resetOffsetAutomatically") @Nullable Boolean resetOffsetAutomatically,
+      @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory,
+      @JsonProperty("intermediateHandoffPeriod") @Nullable Period intermediateHandoffPeriod,
+      @JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions,
+      @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions,
+      @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions
+  )
+  {
+    super(
+        maxRowsInMemory,
+        maxBytesInMemory,
+        maxRowsPerSegment,
+        maxTotalRows,
+        intermediatePersistPeriod,
+        basePersistDirectory,
+        maxPendingPersists,
+        indexSpec,
+        true,
+        reportParseExceptions,
+        handoffConditionTimeout,
+        resetOffsetAutomatically,
+        false,
+        segmentWriteOutMediumFactory,
+        intermediateHandoffPeriod,
+        logParseExceptions,
+        maxParseExceptions,
+        maxSavedParseExceptions
+    );
+  }
+
+  @Override
+  public KafkaIndexTaskTuningConfig withBasePersistDirectory(File dir)
+  {
+    return new KafkaIndexTaskTuningConfig(
+        getMaxRowsInMemory(),
+        getMaxBytesInMemory(),
+        getMaxRowsPerSegment(),
+        getMaxTotalRows(),
+        getIntermediatePersistPeriod(),
+        dir,
+        getMaxPendingPersists(),
+        getIndexSpec(),
+        true,
+        isReportParseExceptions(),
+        getHandoffConditionTimeout(),
+        isResetOffsetAutomatically(),
+        getSegmentWriteOutMediumFactory(),
+        getIntermediateHandoffPeriod(),
+        isLogParseExceptions(),
+        getMaxParseExceptions(),
+        getMaxSavedParseExceptions()
+    );
+  }
+
+
+  @Override
+  public String toString()
+  {
+    return "KafkaIndexTaskTuningConfig{" +
+           "maxRowsInMemory=" + getMaxRowsInMemory() +
+           ", maxRowsPerSegment=" + getMaxRowsPerSegment() +
+           ", maxTotalRows=" + getMaxTotalRows() +
+           ", maxBytesInMemory=" + getMaxBytesInMemory() +
+           ", intermediatePersistPeriod=" + getIntermediatePersistPeriod() +
+           ", basePersistDirectory=" + getBasePersistDirectory() +
+           ", maxPendingPersists=" + getMaxPendingPersists() +
+           ", indexSpec=" + getIndexSpec() +
+           ", reportParseExceptions=" + isReportParseExceptions() +
+           ", handoffConditionTimeout=" + getHandoffConditionTimeout() +
+           ", resetOffsetAutomatically=" + isResetOffsetAutomatically() +
+           ", segmentWriteOutMediumFactory=" + getSegmentWriteOutMediumFactory() +
+           ", intermediateHandoffPeriod=" + getIntermediateHandoffPeriod() +
+           ", logParseExceptions=" + isLogParseExceptions() +
+           ", maxParseExceptions=" + getMaxParseExceptions() +
+           ", maxSavedParseExceptions=" + getMaxSavedParseExceptions() +
+           '}';
+  }
+
+}
diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaPartitions.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaPartitions.java
deleted file mode 100644
index a5b36e9e8e7..00000000000
--- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaPartitions.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.indexing.kafka;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableMap;
-import org.apache.druid.java.util.common.StringUtils;
-
-import java.util.Map;
-import java.util.Objects;
-
-public class KafkaPartitions
-{
-  private final String topic;
-  private final Map<Integer, Long> partitionOffsetMap;
-
-  @JsonCreator
-  public KafkaPartitions(
-      @JsonProperty("topic") final String topic,
-      @JsonProperty("partitionOffsetMap") final Map<Integer, Long> partitionOffsetMap
-  )
-  {
-    this.topic = topic;
-    this.partitionOffsetMap = ImmutableMap.copyOf(partitionOffsetMap);
-
-    // Validate partitionOffsetMap
-    for (Map.Entry<Integer, Long> entry : partitionOffsetMap.entrySet()) {
-      Preconditions.checkArgument(
-          entry.getValue() >= 0,
-          StringUtils.format(
-              "partition[%d] offset[%d] invalid",
-              entry.getKey(),
-              entry.getValue()
-          )
-      );
-    }
-  }
-
-  @JsonProperty
-  public String getTopic()
-  {
-    return topic;
-  }
-
-  @JsonProperty
-  public Map<Integer, Long> getPartitionOffsetMap()
-  {
-    return partitionOffsetMap;
-  }
-
-  @Override
-  public boolean equals(Object o)
-  {
-    if (this == o) {
-      return true;
-    }
-    if (o == null || getClass() != o.getClass()) {
-      return false;
-    }
-    KafkaPartitions that = (KafkaPartitions) o;
-    return Objects.equals(topic, that.topic) &&
-           Objects.equals(partitionOffsetMap, that.partitionOffsetMap);
-  }
-
-  @Override
-  public int hashCode()
-  {
-    return Objects.hash(topic, partitionOffsetMap);
-  }
-
-  @Override
-  public String toString()
-  {
-    return "KafkaPartitions{" +
-           "topic='" + topic + '\'' +
-           ", partitionOffsetMap=" + partitionOffsetMap +
-           '}';
-  }
-}
diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java
new file mode 100644
index 00000000000..935404cbc7c
--- /dev/null
+++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java
@@ -0,0 +1,214 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.indexing.kafka;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableList;
+import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig;
+import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord;
+import org.apache.druid.indexing.seekablestream.common.RecordSupplier;
+import org.apache.druid.indexing.seekablestream.common.StreamPartition;
+import org.apache.druid.indexing.seekablestream.utils.RandomIdUtils;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.metadata.PasswordProvider;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+
+import javax.annotation.Nonnull;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class KafkaRecordSupplier implements RecordSupplier<Integer, Long>
+{
+  private final KafkaConsumer<byte[], byte[]> consumer;
+  private final Map<String, Object> consumerProperties;
+  private final ObjectMapper sortingMapper;
+  private boolean closed;
+
+  public KafkaRecordSupplier(
+      Map<String, Object> consumerProperties,
+      ObjectMapper sortingMapper
+  )
+  {
+    this.consumerProperties = consumerProperties;
+    this.sortingMapper = sortingMapper;
+    this.consumer = getKafkaConsumer();
+  }
+
+  @Override
+  public void assign(Set<StreamPartition<Integer>> streamPartitions)
+  {
+    consumer.assign(streamPartitions
+                        .stream()
+                        .map(x -> new TopicPartition(x.getStream(), x.getPartitionId()))
+                        .collect(Collectors.toSet()));
+    seekToEarliest(streamPartitions);
+  }
+
+  @Override
+  public void seek(StreamPartition<Integer> partition, Long sequenceNumber)
+  {
+    consumer.seek(new TopicPartition(partition.getStream(), partition.getPartitionId()), sequenceNumber);
+  }
+
+  @Override
+  public void seekToEarliest(Set<StreamPartition<Integer>> partitions)
+  {
+    consumer.seekToBeginning(partitions
+                                 .stream()
+                                 .map(e -> new TopicPartition(e.getStream(), e.getPartitionId()))
+                                 .collect(Collectors.toList()));
+  }
+
+  @Override
+  public void seekToLatest(Set<StreamPartition<Integer>> partitions)
+  {
+    consumer.seekToEnd(partitions
+                           .stream()
+                           .map(e -> new TopicPartition(e.getStream(), e.getPartitionId()))
+                           .collect(Collectors.toList()));
+  }
+
+  @Override
+  public Set<StreamPartition<Integer>> getAssignment()
+  {
+    Set<TopicPartition> topicPartitions = consumer.assignment();
+    return topicPartitions
+        .stream()
+        .map(e -> new StreamPartition<>(e.topic(), e.partition()))
+        .collect(Collectors.toSet());
+  }
+
+  @Nonnull
+  @Override
+  public List<OrderedPartitionableRecord<Integer, Long>> poll(long timeout)
+  {
+    List<OrderedPartitionableRecord<Integer, Long>> polledRecords = new ArrayList<>();
+    for (ConsumerRecord<byte[], byte[]> record : consumer.poll(timeout)) {
+      polledRecords.add(new OrderedPartitionableRecord<>(
+          record.topic(),
+          record.partition(),
+          record.offset(),
+          record.value() == null ? null : ImmutableList.of(record.value())
+      ));
+    }
+    return polledRecords;
+  }
+
+  @Override
+  public Long getLatestSequenceNumber(StreamPartition<Integer> partition)
+  {
+    Long currPos = consumer.position(new TopicPartition(partition.getStream(), partition.getPartitionId()));
+    seekToLatest(Collections.singleton(partition));
+    Long nextPos = consumer.position(new TopicPartition(partition.getStream(), partition.getPartitionId()));
+    seek(partition, currPos);
+    return nextPos;
+  }
+
+  @Override
+  public Long getEarliestSequenceNumber(StreamPartition<Integer> partition)
+  {
+    Long currPos = consumer.position(new TopicPartition(partition.getStream(), partition.getPartitionId()));
+    seekToEarliest(Collections.singleton(partition));
+    Long nextPos = consumer.position(new TopicPartition(partition.getStream(), partition.getPartitionId()));
+    seek(partition, currPos);
+    return nextPos;
+  }
+
+  @Override
+  public Long getPosition(StreamPartition<Integer> partition)
+  {
+    return consumer.position(new TopicPartition(partition.getStream(), partition.getPartitionId()));
+  }
+
+  @Override
+  public Set<Integer> getPartitionIds(String stream)
+  {
+    List<PartitionInfo> partitions = consumer.partitionsFor(stream);
+    if (partitions == null) {
+      throw new ISE("Topic [%s] is not found in KafkaConsumer's list of topics", stream);
+    }
+    return partitions.stream().map(PartitionInfo::partition).collect(Collectors.toSet());
+  }
+
+  @Override
+  public void close()
+  {
+    if (closed) {
+      return;
+    }
+    closed = true;
+    consumer.close();
+  }
+
+  public static void addConsumerPropertiesFromConfig(
+      Properties properties,
+      ObjectMapper configMapper,
+      Map<String, Object> consumerProperties
+  )
+  {
+    // Extract passwords before SSL connection to Kafka
+    for (Map.Entry<String, Object> entry : consumerProperties.entrySet()) {
+      String propertyKey = entry.getKey();
+      if (propertyKey.equals(KafkaSupervisorIOConfig.TRUST_STORE_PASSWORD_KEY)
+          || propertyKey.equals(KafkaSupervisorIOConfig.KEY_STORE_PASSWORD_KEY)
+          || propertyKey.equals(KafkaSupervisorIOConfig.KEY_PASSWORD_KEY)) {
+        PasswordProvider configPasswordProvider = configMapper.convertValue(
+            entry.getValue(),
+            PasswordProvider.class
+        );
+        properties.setProperty(propertyKey, configPasswordProvider.getPassword());
+      } else {
+        properties.setProperty(propertyKey, String.valueOf(entry.getValue()));
+      }
+    }
+  }
+
+  private KafkaConsumer<byte[], byte[]> getKafkaConsumer()
+  {
+    final Properties props = new Properties();
+
+    props.setProperty("metadata.max.age.ms", "10000");
+    props.setProperty("group.id", StringUtils.format("kafka-supervisor-%s", RandomIdUtils.getRandomId()));
+
+    addConsumerPropertiesFromConfig(props, sortingMapper, consumerProperties);
+
+    props.setProperty("enable.auto.commit", "false");
+
+    ClassLoader currCtxCl = Thread.currentThread().getContextClassLoader();
+    try {
+      Thread.currentThread().setContextClassLoader(getClass().getClassLoader());
+      return new KafkaConsumer<>(props, new ByteArrayDeserializer(), new ByteArrayDeserializer());
+    }
+    finally {
+      Thread.currentThread().setContextClassLoader(currCtxCl);
+    }
+  }
+
+}
diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaSequenceNumber.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaSequenceNumber.java
new file mode 100644
index 00000000000..e9033521b07
--- /dev/null
+++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaSequenceNumber.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.indexing.kafka;
+
+import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber;
+
+import javax.validation.constraints.NotNull;
+
+public class KafkaSequenceNumber extends OrderedSequenceNumber<Long>
+{
+  private KafkaSequenceNumber(Long sequenceNumber)
+  {
+    super(sequenceNumber, false);
+  }
+
+  public static KafkaSequenceNumber of(Long sequenceNumber)
+  {
+    return new KafkaSequenceNumber(sequenceNumber);
+  }
+
+  @Override
+  public int compareTo(
+      @NotNull OrderedSequenceNumber<Long> o
+  )
+  {
+    return this.get().compareTo(o.get());
+  }
+
+}
diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java
index 75d26884249..53320c664da 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java
@@ -45,7 +45,14 @@
 import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory;
 import org.apache.druid.indexing.common.task.IndexTaskUtils;
 import org.apache.druid.indexing.common.task.RealtimeIndexTask;
-import org.apache.druid.indexing.kafka.KafkaIndexTask.Status;
+import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata;
+import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask;
+import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner;
+import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
+import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord;
+import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber;
+import org.apache.druid.indexing.seekablestream.common.RecordSupplier;
+import org.apache.druid.indexing.seekablestream.common.StreamPartition;
 import org.apache.druid.java.util.common.DateTimes;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.StringUtils;
@@ -74,6 +81,7 @@
 import org.apache.kafka.common.TopicPartition;
 import org.joda.time.DateTime;
 
+import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
 import javax.servlet.http.HttpServletRequest;
 import javax.ws.rs.Consumes;
@@ -93,6 +101,7 @@
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.TreeMap;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.Future;
 import java.util.concurrent.RejectedExecutionException;
@@ -109,7 +118,7 @@
  * This class will be removed in a future release.
  */
 @Deprecated
-public class LegacyKafkaIndexTaskRunner implements KafkaIndexTaskRunner
+public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner<Integer, Long>
 {
   private static final EmittingLogger log = new EmittingLogger(LegacyKafkaIndexTaskRunner.class);
   private static final String METADATA_NEXT_PARTITIONS = "nextPartitions";
@@ -153,8 +162,8 @@
   private final Condition isAwaitingRetry = pollRetryLock.newCondition();
 
   private final KafkaIndexTask task;
-  private final KafkaIOConfig ioConfig;
-  private final KafkaTuningConfig tuningConfig;
+  private final KafkaIndexTaskIOConfig ioConfig;
+  private final KafkaIndexTaskTuningConfig tuningConfig;
   private final InputRowParser<ByteBuffer> parser;
   private final AuthorizerMapper authorizerMapper;
   private final Optional<ChatHandlerProvider> chatHandlerProvider;
@@ -181,6 +190,14 @@
       RowIngestionMetersFactory rowIngestionMetersFactory
   )
   {
+    super(
+        task,
+        parser,
+        authorizerMapper,
+        chatHandlerProvider,
+        savedParseExceptions,
+        rowIngestionMetersFactory
+    );
     this.task = task;
     this.ioConfig = task.getIOConfig();
     this.tuningConfig = task.getTuningConfig();
@@ -190,7 +207,7 @@
     this.savedParseExceptions = savedParseExceptions;
     this.rowIngestionMeters = rowIngestionMetersFactory.createRowIngestionMeters();
 
-    this.endOffsets.putAll(ioConfig.getEndPartitions().getPartitionOffsetMap());
+    this.endOffsets.putAll(ioConfig.getEndPartitions().getPartitionSequenceNumberMap());
     this.ingestionState = IngestionState.NOT_STARTED;
   }
 
@@ -274,34 +291,39 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception
 
       appenderator = appenderator0;
 
-      final String topic = ioConfig.getStartPartitions().getTopic();
+      final String topic = ioConfig.getStartPartitions().getStream();
 
       // Start up, set up initial offsets.
       final Object restoredMetadata = driver.startJob();
       if (restoredMetadata == null) {
-        nextOffsets.putAll(ioConfig.getStartPartitions().getPartitionOffsetMap());
+        nextOffsets.putAll(ioConfig.getStartPartitions().getPartitionSequenceNumberMap());
       } else {
         final Map<String, Object> restoredMetadataMap = (Map) restoredMetadata;
-        final KafkaPartitions restoredNextPartitions = toolbox.getObjectMapper().convertValue(
+        final SeekableStreamPartitions<Integer, Long> restoredNextPartitions = toolbox.getObjectMapper().convertValue(
             restoredMetadataMap.get(METADATA_NEXT_PARTITIONS),
-            KafkaPartitions.class
+            toolbox.getObjectMapper().getTypeFactory().constructParametrizedType(
+                SeekableStreamPartitions.class,
+                SeekableStreamPartitions.class,
+                Integer.class,
+                Long.class
+            )
         );
-        nextOffsets.putAll(restoredNextPartitions.getPartitionOffsetMap());
+        nextOffsets.putAll(restoredNextPartitions.getPartitionSequenceNumberMap());
 
         // Sanity checks.
-        if (!restoredNextPartitions.getTopic().equals(ioConfig.getStartPartitions().getTopic())) {
+        if (!restoredNextPartitions.getStream().equals(ioConfig.getStartPartitions().getStream())) {
           throw new ISE(
               "WTF?! Restored topic[%s] but expected topic[%s]",
-              restoredNextPartitions.getTopic(),
-              ioConfig.getStartPartitions().getTopic()
+              restoredNextPartitions.getStream(),
+              ioConfig.getStartPartitions().getStream()
           );
         }
 
-        if (!nextOffsets.keySet().equals(ioConfig.getStartPartitions().getPartitionOffsetMap().keySet())) {
+        if (!nextOffsets.keySet().equals(ioConfig.getStartPartitions().getPartitionSequenceNumberMap().keySet())) {
           throw new ISE(
               "WTF?! Restored partitions[%s] but expected partitions[%s]",
               nextOffsets.keySet(),
-              ioConfig.getStartPartitions().getPartitionOffsetMap().keySet()
+              ioConfig.getStartPartitions().getPartitionSequenceNumberMap().keySet()
           );
         }
       }
@@ -326,8 +348,8 @@ public Committer get()
             public Object getMetadata()
             {
               return ImmutableMap.of(
-                  METADATA_NEXT_PARTITIONS, new KafkaPartitions(
-                      ioConfig.getStartPartitions().getTopic(),
+                  METADATA_NEXT_PARTITIONS, new SeekableStreamPartitions<>(
+                      ioConfig.getStartPartitions().getStream(),
                       snapshot
                   )
               );
@@ -494,13 +516,20 @@ public void run()
       }
 
       final TransactionalSegmentPublisher publisher = (segments, commitMetadata) -> {
-        final KafkaPartitions finalPartitions = toolbox.getObjectMapper().convertValue(
+        final SeekableStreamPartitions<Integer, Long> finalPartitions = toolbox.getObjectMapper().convertValue(
             ((Map) Preconditions.checkNotNull(commitMetadata, "commitMetadata")).get(METADATA_NEXT_PARTITIONS),
-            KafkaPartitions.class
+            toolbox.getObjectMapper()
+                   .getTypeFactory()
+                   .constructParametrizedType(
+                       SeekableStreamPartitions.class,
+                       SeekableStreamPartitions.class,
+                       Integer.class,
+                       Long.class
+                   )
         );
 
         // Sanity check, we should only be publishing things that match our desired end state.
-        if (!endOffsets.equals(finalPartitions.getPartitionOffsetMap())) {
+        if (!endOffsets.equals(finalPartitions.getPartitionSequenceNumberMap())) {
           throw new ISE("WTF?! Driver attempted to publish invalid metadata[%s].", commitMetadata);
         }
 
@@ -530,9 +559,9 @@ public void run()
       ).get();
 
       final List<String> publishedSegments = published.getSegments()
-                                                       .stream()
-                                                       .map(DataSegment::getIdentifier)
-                                                       .collect(Collectors.toList());
+                                                      .stream()
+                                                      .map(DataSegment::getIdentifier)
+                                                      .collect(Collectors.toList());
 
       log.info(
           "Published segments[%s] with metadata[%s].",
@@ -596,6 +625,21 @@ public void run()
     );
   }
 
+  @Override
+  protected boolean isEndOfShard(Long seqNum)
+  {
+    return false;
+  }
+
+  @Nonnull
+  @Override
+  protected List<OrderedPartitionableRecord<Integer, Long>> getRecords(
+      RecordSupplier<Integer, Long> recordSupplier, TaskToolbox toolbox
+  )
+  {
+    throw new UnsupportedOperationException();
+  }
+
   private Set<Integer> assignPartitionsAndSeekToNext(KafkaConsumer consumer, String topic)
   {
     // Initialize consumer assignment.
@@ -615,7 +659,7 @@ public void run()
       }
     }
 
-    task.assignPartitions(consumer, topic, assignment);
+    KafkaIndexTask.assignPartitions(consumer, topic, assignment);
 
     // Seek to starting offsets.
     for (final int partition : assignment) {
@@ -661,6 +705,39 @@ private boolean possiblyPause() throws InterruptedException
     return false;
   }
 
+  @Override
+  protected void possiblyResetDataSourceMetadata(
+      TaskToolbox toolbox,
+      RecordSupplier<Integer, Long> recordSupplier,
+      Set<StreamPartition<Integer>> assignment,
+      Map<Integer, Long> currOffsets
+  )
+  {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  protected boolean isEndSequenceOffsetsExclusive()
+  {
+    return false;
+  }
+
+  @Override
+  protected boolean isStartingSequenceOffsetsExclusive()
+  {
+    return false;
+  }
+
+
+  @Override
+  protected SeekableStreamPartitions<Integer, Long> deserializeSeekableStreamPartitionsFromMetadata(
+      ObjectMapper mapper,
+      Object object
+  )
+  {
+    throw new UnsupportedOperationException();
+  }
+
   private void possiblyResetOffsetsOrWait(
       Map<TopicPartition, Long> outOfRangePartitions,
       KafkaConsumer<byte[], byte[]> consumer,
@@ -689,7 +766,7 @@ private void possiblyResetOffsetsOrWait(
     }
 
     if (doReset) {
-      sendResetRequestAndWait(resetPartitions, taskToolbox);
+      sendResetRequestAndWaitLegacy(resetPartitions, taskToolbox);
     } else {
       log.warn("Retrying in %dms", task.getPollRetryMs());
       pollRetryLock.lockInterruptibly();
@@ -705,7 +782,7 @@ private void possiblyResetOffsetsOrWait(
     }
   }
 
-  private void sendResetRequestAndWait(Map<TopicPartition, Long> outOfRangePartitions, TaskToolbox taskToolbox)
+  private void sendResetRequestAndWaitLegacy(Map<TopicPartition, Long> outOfRangePartitions, TaskToolbox taskToolbox)
       throws IOException
   {
     Map<Integer, Long> partitionOffsetMap = new HashMap<>();
@@ -715,9 +792,9 @@ private void sendResetRequestAndWait(Map<TopicPartition, Long> outOfRangePartiti
     boolean result = taskToolbox.getTaskActionClient()
                                 .submit(new ResetDataSourceMetadataAction(
                                     task.getDataSource(),
-                                    new KafkaDataSourceMetadata(new KafkaPartitions(
+                                    new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(
                                         ioConfig.getStartPartitions()
-                                                .getTopic(),
+                                                .getStream(),
                                         partitionOffsetMap
                                     ))
                                 ));
@@ -738,6 +815,12 @@ private void requestPause()
     pauseRequested = true;
   }
 
+  @Override
+  protected Long getSequenceNumberToStoreAfterRead(Long sequenceNumber)
+  {
+    throw new UnsupportedOperationException();
+  }
+
   private void handleParseException(ParseException pe, ConsumerRecord<byte[], byte[]> record)
   {
     if (pe.isFromPartiallyValidRow()) {
@@ -817,7 +900,7 @@ public void stopGracefully()
     }
 
     try {
-      if (pauseLock.tryLock(KafkaIndexTask.LOCK_ACQUIRE_TIMEOUT_SECONDS, TimeUnit.SECONDS)) {
+      if (pauseLock.tryLock(SeekableStreamIndexTask.LOCK_ACQUIRE_TIMEOUT_SECONDS, TimeUnit.SECONDS)) {
         try {
           if (pauseRequested) {
             pauseRequested = false;
@@ -833,7 +916,7 @@ public void stopGracefully()
         return;
       }
 
-      if (pollRetryLock.tryLock(KafkaIndexTask.LOCK_ACQUIRE_TIMEOUT_SECONDS, TimeUnit.SECONDS)) {
+      if (pollRetryLock.tryLock(SeekableStreamIndexTask.LOCK_ACQUIRE_TIMEOUT_SECONDS, TimeUnit.SECONDS)) {
         try {
           isAwaitingRetry.signalAll();
         }
@@ -860,6 +943,7 @@ private Access authorizationCheck(final HttpServletRequest req, Action action)
     return IndexTaskUtils.datasourceAuthorizationCheck(req, action, task.getDataSource(), authorizerMapper);
   }
 
+  @Override
   @POST
   @Path("/stop")
   public Response stop(@Context final HttpServletRequest req)
@@ -869,6 +953,7 @@ public Response stop(@Context final HttpServletRequest req)
     return Response.status(Response.Status.OK).build();
   }
 
+  @Override
   @GET
   @Path("/status")
   @Produces(MediaType.APPLICATION_JSON)
@@ -884,6 +969,7 @@ public Status getStatus()
     return status;
   }
 
+  @Override
   @GET
   @Path("/offsets/current")
   @Produces(MediaType.APPLICATION_JSON)
@@ -899,6 +985,7 @@ public Status getStatus()
     return nextOffsets;
   }
 
+  @Override
   @GET
   @Path("/offsets/end")
   @Produces(MediaType.APPLICATION_JSON)
@@ -915,10 +1002,10 @@ public Status getStatus()
   }
 
   @Override
-  public Response setEndOffsets(Map<Integer, Long> offsets, boolean finish) throws InterruptedException
+  public Response setEndOffsets(Map<Integer, Long> sequenceNumbers, boolean finish) throws InterruptedException
   {
     // finish is not used in this mode
-    return setEndOffsets(offsets);
+    return setEndOffsets(sequenceNumbers);
   }
 
   @POST
@@ -934,6 +1021,7 @@ public Response setEndOffsetsHTTP(
     return setEndOffsets(offsets);
   }
 
+  @Override
   @GET
   @Path("/rowStats")
   @Produces(MediaType.APPLICATION_JSON)
@@ -960,6 +1048,7 @@ public Response getRowStats(
     return Response.ok(returnMap).build();
   }
 
+  @Override
   @GET
   @Path("/unparseableEvents")
   @Produces(MediaType.APPLICATION_JSON)
@@ -1037,6 +1126,7 @@ private boolean isPaused()
    * method has timed out and returned before the task has paused; 200 OK with a map of the current partition offsets
    * in the response body if the task successfully paused
    */
+  @Override
   @POST
   @Path("/pause")
   @Produces(MediaType.APPLICATION_JSON)
@@ -1095,6 +1185,7 @@ public Response pause() throws InterruptedException
     }
   }
 
+  @Override
   @POST
   @Path("/resume")
   public Response resumeHTTP(@Context final HttpServletRequest req) throws InterruptedException
@@ -1125,6 +1216,21 @@ public void resume() throws InterruptedException
     }
   }
 
+  @Override
+  protected SeekableStreamDataSourceMetadata<Integer, Long> createDataSourceMetadata(
+      SeekableStreamPartitions<Integer, Long> partitions
+  )
+  {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  protected OrderedSequenceNumber<Long> createSequenceNumber(Long sequenceNumber)
+  {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
   @GET
   @Path("/time/start")
   @Produces(MediaType.APPLICATION_JSON)
@@ -1133,4 +1239,15 @@ public DateTime getStartTime(@Context final HttpServletRequest req)
     authorizationCheck(req, Action.WRITE);
     return startTime;
   }
+
+  @Nullable
+  @Override
+  protected TreeMap<Integer, Map<Integer, Long>> getCheckPointsFromContext(
+      TaskToolbox toolbox,
+      String checkpointsString
+  )
+  {
+    throw new UnsupportedOperationException();
+  }
+
 }
diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java
index b0c4ba56ec5..6f4434ea3e9 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java
@@ -21,95 +21,52 @@
 
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.core.type.TypeReference;
-import com.fasterxml.jackson.databind.MapperFeature;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Function;
 import com.google.common.base.Joiner;
-import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Predicate;
-import com.google.common.base.Throwables;
-import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
-import com.google.common.primitives.Longs;
-import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.ListeningExecutorService;
-import com.google.common.util.concurrent.MoreExecutors;
-import org.apache.commons.codec.digest.DigestUtils;
-import org.apache.druid.indexer.TaskLocation;
-import org.apache.druid.indexer.TaskStatus;
-import org.apache.druid.indexing.common.IndexTaskClient;
-import org.apache.druid.indexing.common.TaskInfoProvider;
 import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory;
-import org.apache.druid.indexing.common.task.RealtimeIndexTask;
 import org.apache.druid.indexing.common.task.Task;
 import org.apache.druid.indexing.common.task.TaskResource;
 import org.apache.druid.indexing.kafka.KafkaDataSourceMetadata;
-import org.apache.druid.indexing.kafka.KafkaIOConfig;
 import org.apache.druid.indexing.kafka.KafkaIndexTask;
-import org.apache.druid.indexing.kafka.KafkaIndexTaskClient;
 import org.apache.druid.indexing.kafka.KafkaIndexTaskClientFactory;
-import org.apache.druid.indexing.kafka.KafkaPartitions;
-import org.apache.druid.indexing.kafka.KafkaTuningConfig;
+import org.apache.druid.indexing.kafka.KafkaIndexTaskIOConfig;
+import org.apache.druid.indexing.kafka.KafkaIndexTaskTuningConfig;
+import org.apache.druid.indexing.kafka.KafkaRecordSupplier;
+import org.apache.druid.indexing.kafka.KafkaSequenceNumber;
 import org.apache.druid.indexing.overlord.DataSourceMetadata;
 import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
 import org.apache.druid.indexing.overlord.TaskMaster;
-import org.apache.druid.indexing.overlord.TaskQueue;
-import org.apache.druid.indexing.overlord.TaskRunner;
-import org.apache.druid.indexing.overlord.TaskRunnerListener;
-import org.apache.druid.indexing.overlord.TaskRunnerWorkItem;
 import org.apache.druid.indexing.overlord.TaskStorage;
-import org.apache.druid.indexing.overlord.supervisor.Supervisor;
-import org.apache.druid.indexing.overlord.supervisor.SupervisorReport;
-import org.apache.druid.java.util.common.DateTimes;
-import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask;
+import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig;
+import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig;
+import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
+import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber;
+import org.apache.druid.indexing.seekablestream.common.RecordSupplier;
+import org.apache.druid.indexing.seekablestream.common.StreamPartition;
+import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor;
+import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig;
+import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload;
+import org.apache.druid.indexing.seekablestream.utils.RandomIdUtils;
 import org.apache.druid.java.util.common.ISE;
-import org.apache.druid.java.util.common.Pair;
-import org.apache.druid.java.util.common.RetryUtils;
 import org.apache.druid.java.util.common.StringUtils;
-import org.apache.druid.java.util.common.concurrent.Execs;
 import org.apache.druid.java.util.emitter.EmittingLogger;
 import org.apache.druid.java.util.emitter.service.ServiceEmitter;
 import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
-import org.apache.druid.metadata.EntryExistsException;
 import org.apache.druid.server.metrics.DruidMonitorSchedulerConfig;
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.common.PartitionInfo;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.serialization.ByteArrayDeserializer;
 import org.joda.time.DateTime;
 
-import javax.annotation.Nullable;
-import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.Properties;
 import java.util.Set;
-import java.util.SortedMap;
 import java.util.TreeMap;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.LinkedBlockingDeque;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-import java.util.concurrent.atomic.AtomicInteger;
 import java.util.stream.Collectors;
-import java.util.stream.Stream;
 
 /**
  * Supervisor responsible for managing the KafkaIndexTasks for a single dataSource. At a high level, the class accepts a
@@ -119,2228 +76,194 @@
  * tasks to satisfy the desired number of replicas. As tasks complete, new tasks are queued to process the next range of
  * Kafka offsets.
  */
-public class KafkaSupervisor implements Supervisor
+public class KafkaSupervisor extends SeekableStreamSupervisor<Integer, Long>
 {
   private static final EmittingLogger log = new EmittingLogger(KafkaSupervisor.class);
-  private static final long MAX_RUN_FREQUENCY_MILLIS = 1000; // prevent us from running too often in response to events
-  private static final long NOT_SET = -1;
-  private static final long MINIMUM_FUTURE_TIMEOUT_IN_SECONDS = 120;
   private static final long MINIMUM_GET_OFFSET_PERIOD_MILLIS = 5000;
   private static final long INITIAL_GET_OFFSET_DELAY_MILLIS = 15000;
   private static final long INITIAL_EMIT_LAG_METRIC_DELAY_MILLIS = 25000;
-  private static final int MAX_INITIALIZATION_RETRIES = 20;
+  private static final Long NOT_SET = -1L;
+  private static final Long END_OF_PARTITION = Long.MAX_VALUE;
 
-  public static final String IS_INCREMENTAL_HANDOFF_SUPPORTED = "IS_INCREMENTAL_HANDOFF_SUPPORTED";
-
-  // Internal data structures
-  // --------------------------------------------------------
-
-  /**
-   * A TaskGroup is the main data structure used by KafkaSupervisor to organize and monitor Kafka partitions and
-   * indexing tasks. All the tasks in a TaskGroup should always be doing the same thing (reading the same partitions and
-   * starting from the same offset) and if [replicas] is configured to be 1, a TaskGroup will contain a single task (the
-   * exception being if the supervisor started up and discovered and adopted some already running tasks). At any given
-   * time, there should only be up to a maximum of [taskCount] actively-reading task groups (tracked in the [taskGroups]
-   * map) + zero or more pending-completion task groups (tracked in [pendingCompletionTaskGroups]).
-   */
-  private class TaskGroup
-  {
-    final int groupId;
-
-    // This specifies the partitions and starting offsets for this task group. It is set on group creation from the data
-    // in [partitionGroups] and never changes during the lifetime of this task group, which will live until a task in
-    // this task group has completed successfully, at which point this will be destroyed and a new task group will be
-    // created with new starting offsets. This allows us to create replacement tasks for failed tasks that process the
-    // same offsets, even if the values in [partitionGroups] has been changed.
-    final ImmutableMap<Integer, Long> partitionOffsets;
-
-    final ConcurrentHashMap<String, TaskData> tasks = new ConcurrentHashMap<>();
-    final Optional<DateTime> minimumMessageTime;
-    final Optional<DateTime> maximumMessageTime;
-    DateTime completionTimeout; // is set after signalTasksToFinish(); if not done by timeout, take corrective action
-    final TreeMap<Integer, Map<Integer, Long>> sequenceOffsets = new TreeMap<>();
-    final String baseSequenceName;
-
-    TaskGroup(
-        int groupId,
-        ImmutableMap<Integer, Long> partitionOffsets,
-        Optional<DateTime> minimumMessageTime,
-        Optional<DateTime> maximumMessageTime
-    )
-    {
-      this.groupId = groupId;
-      this.partitionOffsets = partitionOffsets;
-      this.minimumMessageTime = minimumMessageTime;
-      this.maximumMessageTime = maximumMessageTime;
-      this.sequenceOffsets.put(0, partitionOffsets);
-      this.baseSequenceName = generateSequenceName(partitionOffsets, minimumMessageTime, maximumMessageTime);
-    }
-
-    int addNewCheckpoint(Map<Integer, Long> checkpoint)
-    {
-      sequenceOffsets.put(sequenceOffsets.lastKey() + 1, checkpoint);
-      return sequenceOffsets.lastKey();
-    }
-
-    Set<String> taskIds()
-    {
-      return tasks.keySet();
-    }
-  }
-
-  private static class TaskData
-  {
-    @Nullable
-    volatile TaskStatus status;
-    @Nullable
-    volatile DateTime startTime;
-    volatile Map<Integer, Long> currentOffsets = new HashMap<>();
-
-    @Override
-    public String toString()
-    {
-      return "TaskData{" +
-             "status=" + status +
-             ", startTime=" + startTime +
-             ", currentOffsets=" + currentOffsets +
-             '}';
-    }
-  }
-
-  // Map<{group ID}, {actively reading task group}>; see documentation for TaskGroup class
-  private final ConcurrentHashMap<Integer, TaskGroup> taskGroups = new ConcurrentHashMap<>();
-
-  // After telling a taskGroup to stop reading and begin publishing a segment, it is moved from [taskGroups] to here so
-  // we can monitor its status while we queue new tasks to read the next range of offsets. This is a list since we could
-  // have multiple sets of tasks publishing at once if time-to-publish > taskDuration.
-  // Map<{group ID}, List<{pending completion task groups}>>
-  private final ConcurrentHashMap<Integer, CopyOnWriteArrayList<TaskGroup>> pendingCompletionTaskGroups = new ConcurrentHashMap<>();
-
-  // The starting offset for a new partition in [partitionGroups] is initially set to NOT_SET. When a new task group
-  // is created and is assigned partitions, if the offset in [partitionGroups] is NOT_SET it will take the starting
-  // offset value from the metadata store, and if it can't find it there, from Kafka. Once a task begins
-  // publishing, the offset in partitionGroups will be updated to the ending offset of the publishing-but-not-yet-
-  // completed task, which will cause the next set of tasks to begin reading from where the previous task left
-  // off. If that previous task now fails, we will set the offset in [partitionGroups] back to NOT_SET which will
-  // cause successive tasks to again grab their starting offset from metadata store. This mechanism allows us to
-  // start up successive tasks without waiting for the previous tasks to succeed and still be able to handle task
-  // failures during publishing.
-  // Map<{group ID}, Map<{partition ID}, {startingOffset}>>
-  private final ConcurrentHashMap<Integer, ConcurrentHashMap<Integer, Long>> partitionGroups = new ConcurrentHashMap<>();
-  // --------------------------------------------------------
-
-  private final TaskStorage taskStorage;
-  private final TaskMaster taskMaster;
-  private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator;
-  private final KafkaIndexTaskClient taskClient;
-  private final ObjectMapper sortingMapper;
-  private final KafkaSupervisorSpec spec;
-  private final ServiceEmitter emitter;
-  private final DruidMonitorSchedulerConfig monitorSchedulerConfig;
-  private final String dataSource;
-  private final KafkaSupervisorIOConfig ioConfig;
-  private final KafkaSupervisorTuningConfig tuningConfig;
-  private final KafkaTuningConfig taskTuningConfig;
-  private final String supervisorId;
-  private final TaskInfoProvider taskInfoProvider;
-  private final long futureTimeoutInSeconds; // how long to wait for async operations to complete
-  private final RowIngestionMetersFactory rowIngestionMetersFactory;
-
-  private final ExecutorService exec;
-  private final ScheduledExecutorService scheduledExec;
-  private final ScheduledExecutorService reportingExec;
-  private final ListeningExecutorService workerExec;
-  private final BlockingQueue<Notice> notices = new LinkedBlockingDeque<>();
-  private final Object stopLock = new Object();
-  private final Object stateChangeLock = new Object();
-  private final Object consumerLock = new Object();
-
-  private boolean listenerRegistered = false;
-  private long lastRunTime;
-
-  private int initRetryCounter = 0;
-
-  private volatile DateTime firstRunTime;
-  private volatile KafkaConsumer consumer;
-
-  private volatile boolean lifecycleStarted = false;
-  private volatile boolean started = false;
-  private volatile boolean stopped = false;
-  private volatile Map<Integer, Long> latestOffsetsFromKafka;
-  private volatile DateTime offsetsLastUpdated;
-
-  public KafkaSupervisor(
-      final TaskStorage taskStorage,
-      final TaskMaster taskMaster,
-      final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator,
-      final KafkaIndexTaskClientFactory taskClientFactory,
-      final ObjectMapper mapper,
-      final KafkaSupervisorSpec spec,
-      final RowIngestionMetersFactory rowIngestionMetersFactory
-  )
-  {
-    this.taskStorage = taskStorage;
-    this.taskMaster = taskMaster;
-    this.indexerMetadataStorageCoordinator = indexerMetadataStorageCoordinator;
-    this.sortingMapper = mapper.copy().configure(MapperFeature.SORT_PROPERTIES_ALPHABETICALLY, true);
-    this.spec = spec;
-    this.emitter = spec.getEmitter();
-    this.monitorSchedulerConfig = spec.getMonitorSchedulerConfig();
-    this.rowIngestionMetersFactory = rowIngestionMetersFactory;
-
-    this.dataSource = spec.getDataSchema().getDataSource();
-    this.ioConfig = spec.getIoConfig();
-    this.tuningConfig = spec.getTuningConfig();
-    this.taskTuningConfig = KafkaTuningConfig.copyOf(this.tuningConfig);
-    this.supervisorId = StringUtils.format("KafkaSupervisor-%s", dataSource);
-    this.exec = Execs.singleThreaded(supervisorId);
-    this.scheduledExec = Execs.scheduledSingleThreaded(supervisorId + "-Scheduler-%d");
-    this.reportingExec = Execs.scheduledSingleThreaded(supervisorId + "-Reporting-%d");
-
-    int workerThreads = (this.tuningConfig.getWorkerThreads() != null
-                         ? this.tuningConfig.getWorkerThreads()
-                         : Math.min(10, this.ioConfig.getTaskCount()));
-    this.workerExec = MoreExecutors.listeningDecorator(Execs.multiThreaded(workerThreads, supervisorId + "-Worker-%d"));
-    log.info("Created worker pool with [%d] threads for dataSource [%s]", workerThreads, this.dataSource);
-
-    this.taskInfoProvider = new TaskInfoProvider()
-    {
-      @Override
-      public TaskLocation getTaskLocation(final String id)
-      {
-        Preconditions.checkNotNull(id, "id");
-        Optional<TaskRunner> taskRunner = taskMaster.getTaskRunner();
-        if (taskRunner.isPresent()) {
-          Optional<? extends TaskRunnerWorkItem> item = Iterables.tryFind(
-              taskRunner.get().getRunningTasks(),
-              (Predicate<TaskRunnerWorkItem>) taskRunnerWorkItem -> id.equals(taskRunnerWorkItem.getTaskId())
-          );
-
-          if (item.isPresent()) {
-            return item.get().getLocation();
-          }
-        } else {
-          log.error("Failed to get task runner because I'm not the leader!");
-        }
-
-        return TaskLocation.unknown();
-      }
-
-      @Override
-      public Optional<TaskStatus> getTaskStatus(String id)
-      {
-        return taskStorage.getStatus(id);
-      }
-    };
-
-    this.futureTimeoutInSeconds = Math.max(
-        MINIMUM_FUTURE_TIMEOUT_IN_SECONDS,
-        tuningConfig.getChatRetries() * (tuningConfig.getHttpTimeout().getStandardSeconds()
-                                         + IndexTaskClient.MAX_RETRY_WAIT_SECONDS)
-    );
-
-    int chatThreads = (this.tuningConfig.getChatThreads() != null
-                       ? this.tuningConfig.getChatThreads()
-                       : Math.min(10, this.ioConfig.getTaskCount() * this.ioConfig.getReplicas()));
-    this.taskClient = taskClientFactory.build(
-        taskInfoProvider,
-        dataSource,
-        chatThreads,
-        this.tuningConfig.getHttpTimeout(),
-        this.tuningConfig.getChatRetries()
-    );
-    log.info(
-        "Created taskClient with dataSource[%s] chatThreads[%d] httpTimeout[%s] chatRetries[%d]",
-        dataSource,
-        chatThreads,
-        this.tuningConfig.getHttpTimeout(),
-        this.tuningConfig.getChatRetries()
-    );
-  }
-
-  @Override
-  public void start()
-  {
-    synchronized (stateChangeLock) {
-      Preconditions.checkState(!lifecycleStarted, "already started");
-      Preconditions.checkState(!exec.isShutdown(), "already stopped");
-
-      // Try normal initialization first, if that fails then schedule periodic initialization retries
-      try {
-        tryInit();
-      }
-      catch (Exception e) {
-        if (!started) {
-          log.warn("First initialization attempt failed for KafkaSupervisor[%s], starting retries...", dataSource);
-
-          exec.submit(
-              () -> {
-                try {
-                  RetryUtils.retry(
-                      () -> {
-                        tryInit();
-                        return 0;
-                      },
-                      (throwable) -> {
-                        return !started;
-                      },
-                      0,
-                      MAX_INITIALIZATION_RETRIES,
-                      null,
-                      null
-                  );
-                }
-                catch (Exception e2) {
-                  log.makeAlert(
-                      "Failed to initialize after %s retries, aborting. Please resubmit the supervisor spec to restart this supervisor [%s]",
-                      MAX_INITIALIZATION_RETRIES,
-                      supervisorId
-                  ).emit();
-                  throw new RuntimeException(e2);
-                }
-              }
-          );
-        }
-      }
-
-      lifecycleStarted = true;
-    }
-  }
-
-  @Override
-  public void stop(boolean stopGracefully)
-  {
-    synchronized (stateChangeLock) {
-      Preconditions.checkState(lifecycleStarted, "lifecycle not started");
-
-      log.info("Beginning shutdown of KafkaSupervisor[%s]", dataSource);
-
-      try {
-        scheduledExec.shutdownNow(); // stop recurring executions
-        reportingExec.shutdownNow();
-
-        if (started) {
-          Optional<TaskRunner> taskRunner = taskMaster.getTaskRunner();
-          if (taskRunner.isPresent()) {
-            taskRunner.get().unregisterListener(supervisorId);
-          }
-
-          // Stopping gracefully will synchronize the end offsets of the tasks and signal them to publish, and will block
-          // until the tasks have acknowledged or timed out. We want this behavior when we're explicitly shut down through
-          // the API, but if we shut down for other reasons (e.g. we lose leadership) we want to just stop and leave the
-          // tasks as they are.
-          synchronized (stopLock) {
-            if (stopGracefully) {
-              log.info("Posting GracefulShutdownNotice, signalling managed tasks to complete and publish");
-              notices.add(new GracefulShutdownNotice());
-            } else {
-              log.info("Posting ShutdownNotice");
-              notices.add(new ShutdownNotice());
-            }
-
-            long shutdownTimeoutMillis = tuningConfig.getShutdownTimeout().getMillis();
-            long endTime = System.currentTimeMillis() + shutdownTimeoutMillis;
-            while (!stopped) {
-              long sleepTime = endTime - System.currentTimeMillis();
-              if (sleepTime <= 0) {
-                log.info("Timed out while waiting for shutdown (timeout [%,dms])", shutdownTimeoutMillis);
-                stopped = true;
-                break;
-              }
-              stopLock.wait(sleepTime);
-            }
-          }
-          log.info("Shutdown notice handled");
-        }
-
-        taskClient.close();
-        workerExec.shutdownNow();
-        exec.shutdownNow();
-        started = false;
-
-        log.info("KafkaSupervisor[%s] has stopped", dataSource);
-      }
-      catch (Exception e) {
-        log.makeAlert(e, "Exception stopping KafkaSupervisor[%s]", dataSource)
-           .emit();
-      }
-    }
-  }
-
-  private boolean someTaskGroupsPendingCompletion(Integer groupId)
-  {
-    CopyOnWriteArrayList<TaskGroup> taskGroups = pendingCompletionTaskGroups.get(groupId);
-    return taskGroups != null && taskGroups.size() > 0;
-  }
-
-  @Override
-  public SupervisorReport getStatus()
-  {
-    return generateReport(true);
-  }
-
-  @Override
-  public Map<String, Map<String, Object>> getStats()
-  {
-    try {
-      return getCurrentTotalStats();
-    }
-    catch (InterruptedException ie) {
-      Thread.currentThread().interrupt();
-      log.error(ie, "getStats() interrupted.");
-      throw new RuntimeException(ie);
-    }
-    catch (ExecutionException | TimeoutException eete) {
-      throw new RuntimeException(eete);
-    }
-  }
-
-  @Override
-  public void reset(DataSourceMetadata dataSourceMetadata)
-  {
-    log.info("Posting ResetNotice");
-    notices.add(new ResetNotice(dataSourceMetadata));
-  }
-
-  @Override
-  public void checkpoint(
-      @Nullable Integer taskGroupId,
-      @Deprecated String baseSequenceName,
-      DataSourceMetadata previousCheckPoint,
-      DataSourceMetadata currentCheckPoint
-  )
-  {
-    Preconditions.checkNotNull(previousCheckPoint, "previousCheckpoint");
-    Preconditions.checkNotNull(currentCheckPoint, "current checkpoint cannot be null");
-    Preconditions.checkArgument(
-        ioConfig.getTopic().equals(((KafkaDataSourceMetadata) currentCheckPoint).getKafkaPartitions().getTopic()),
-        "Supervisor topic [%s] and topic in checkpoint [%s] does not match",
-        ioConfig.getTopic(),
-        ((KafkaDataSourceMetadata) currentCheckPoint).getKafkaPartitions().getTopic()
-    );
-
-    log.info("Checkpointing [%s] for taskGroup [%s]", currentCheckPoint, taskGroupId);
-    notices.add(
-        new CheckpointNotice(
-            taskGroupId,
-            baseSequenceName,
-            (KafkaDataSourceMetadata) previousCheckPoint,
-            (KafkaDataSourceMetadata) currentCheckPoint
-        )
-    );
-  }
-
-  public void possiblyRegisterListener()
-  {
-    // getTaskRunner() sometimes fails if the task queue is still being initialized so retry later until we succeed
-
-    if (listenerRegistered) {
-      return;
-    }
-
-    Optional<TaskRunner> taskRunner = taskMaster.getTaskRunner();
-    if (taskRunner.isPresent()) {
-      taskRunner.get().registerListener(
-          new TaskRunnerListener()
-          {
-            @Override
-            public String getListenerId()
-            {
-              return supervisorId;
-            }
-
-            @Override
-            public void locationChanged(final String taskId, final TaskLocation newLocation)
-            {
-              // do nothing
-            }
-
-            @Override
-            public void statusChanged(String taskId, TaskStatus status)
-            {
-              notices.add(new RunNotice());
-            }
-          }, MoreExecutors.sameThreadExecutor()
-      );
-
-      listenerRegistered = true;
-    }
-  }
-
-  private interface Notice
-  {
-    void handle() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException;
-  }
-
-  private class RunNotice implements Notice
-  {
-    @Override
-    public void handle() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException
-    {
-      long nowTime = System.currentTimeMillis();
-      if (nowTime - lastRunTime < MAX_RUN_FREQUENCY_MILLIS) {
-        return;
-      }
-      lastRunTime = nowTime;
-
-      runInternal();
-    }
-  }
-
-  private class GracefulShutdownNotice extends ShutdownNotice
-  {
-    @Override
-    public void handle() throws InterruptedException, ExecutionException, TimeoutException
-    {
-      gracefulShutdownInternal();
-      super.handle();
-    }
-  }
-
-  private class ShutdownNotice implements Notice
-  {
-    @Override
-    public void handle() throws InterruptedException, ExecutionException, TimeoutException
-    {
-      consumer.close();
-
-      synchronized (stopLock) {
-        stopped = true;
-        stopLock.notifyAll();
-      }
-    }
-  }
-
-  private class ResetNotice implements Notice
-  {
-    final DataSourceMetadata dataSourceMetadata;
-
-    ResetNotice(DataSourceMetadata dataSourceMetadata)
-    {
-      this.dataSourceMetadata = dataSourceMetadata;
-    }
-
-    @Override
-    public void handle()
-    {
-      resetInternal(dataSourceMetadata);
-    }
-  }
-
-  private class CheckpointNotice implements Notice
-  {
-    @Nullable
-    private final Integer nullableTaskGroupId;
-    @Deprecated
-    private final String baseSequenceName;
-    private final KafkaDataSourceMetadata previousCheckpoint;
-    private final KafkaDataSourceMetadata currentCheckpoint;
-
-    CheckpointNotice(
-        @Nullable Integer nullableTaskGroupId,
-        @Deprecated String baseSequenceName,
-        KafkaDataSourceMetadata previousCheckpoint,
-        KafkaDataSourceMetadata currentCheckpoint
-    )
-    {
-      this.baseSequenceName = baseSequenceName;
-      this.nullableTaskGroupId = nullableTaskGroupId;
-      this.previousCheckpoint = previousCheckpoint;
-      this.currentCheckpoint = currentCheckpoint;
-    }
-
-    @Override
-    public void handle() throws ExecutionException, InterruptedException
-    {
-      // Find taskGroupId using taskId if it's null. It can be null while rolling update.
-      final int taskGroupId;
-      if (nullableTaskGroupId == null) {
-        // We search taskId in taskGroups and pendingCompletionTaskGroups sequentially. This should be fine because
-        // 1) a taskGroup can be moved from taskGroups to pendingCompletionTaskGroups in RunNotice
-        //    (see checkTaskDuration()).
-        // 2) Notices are proceesed by a single thread. So, CheckpointNotice and RunNotice cannot be processed at the
-        //    same time.
-        final java.util.Optional<Integer> maybeGroupId = taskGroups
-            .entrySet()
-            .stream()
-            .filter(entry -> {
-              final TaskGroup taskGroup = entry.getValue();
-              return taskGroup.baseSequenceName.equals(baseSequenceName);
-            })
-            .findAny()
-            .map(Entry::getKey);
-
-        if (maybeGroupId.isPresent()) {
-          taskGroupId = maybeGroupId.get();
-        } else {
-          taskGroupId = pendingCompletionTaskGroups
-              .entrySet()
-              .stream()
-              .filter(entry -> {
-                final List<TaskGroup> taskGroups = entry.getValue();
-                return taskGroups.stream().anyMatch(group -> group.baseSequenceName.equals(baseSequenceName));
-              })
-              .findAny()
-              .orElseThrow(() -> new ISE("Cannot find taskGroup for baseSequenceName[%s]", baseSequenceName))
-              .getKey();
-        }
-      } else {
-        taskGroupId = nullableTaskGroupId;
-      }
-
-      // check for consistency
-      // if already received request for this sequenceName and dataSourceMetadata combination then return
-      final TaskGroup taskGroup = taskGroups.get(taskGroupId);
-
-      if (isValidTaskGroup(taskGroupId, taskGroup)) {
-        final TreeMap<Integer, Map<Integer, Long>> checkpoints = taskGroup.sequenceOffsets;
-
-        // check validity of previousCheckpoint
-        int index = checkpoints.size();
-        for (int sequenceId : checkpoints.descendingKeySet()) {
-          Map<Integer, Long> checkpoint = checkpoints.get(sequenceId);
-          // We have already verified the topic of the current checkpoint is same with that in ioConfig.
-          // See checkpoint().
-          if (checkpoint.equals(previousCheckpoint.getKafkaPartitions().getPartitionOffsetMap())) {
-            break;
-          }
-          index--;
-        }
-        if (index == 0) {
-          throw new ISE("No such previous checkpoint [%s] found", previousCheckpoint);
-        } else if (index < checkpoints.size()) {
-          // if the found checkpoint is not the latest one then already checkpointed by a replica
-          Preconditions.checkState(index == checkpoints.size() - 1, "checkpoint consistency failure");
-          log.info("Already checkpointed with offsets [%s]", checkpoints.lastEntry().getValue());
-          return;
-        }
-        final Map<Integer, Long> newCheckpoint = checkpointTaskGroup(taskGroup, false).get();
-        taskGroup.addNewCheckpoint(newCheckpoint);
-        log.info("Handled checkpoint notice, new checkpoint is [%s] for taskGroup [%s]", newCheckpoint, taskGroupId);
-      }
-    }
-
-    private boolean isValidTaskGroup(int taskGroupId, @Nullable TaskGroup taskGroup)
-    {
-      if (taskGroup == null) {
-        // taskGroup might be in pendingCompletionTaskGroups or partitionGroups
-        if (pendingCompletionTaskGroups.containsKey(taskGroupId)) {
-          log.warn(
-              "Ignoring checkpoint request because taskGroup[%d] has already stopped indexing and is waiting for "
-              + "publishing segments",
-              taskGroupId
-          );
-          return false;
-        } else if (partitionGroups.containsKey(taskGroupId)) {
-          log.warn("Ignoring checkpoint request because taskGroup[%d] is inactive", taskGroupId);
-          return false;
-        } else {
-          throw new ISE("WTH?! cannot find taskGroup [%s] among all taskGroups [%s]", taskGroupId, taskGroups);
-        }
-      }
-
-      return true;
-    }
-  }
-
-  @VisibleForTesting
-  void resetInternal(DataSourceMetadata dataSourceMetadata)
-  {
-    if (dataSourceMetadata == null) {
-      // Reset everything
-      boolean result = indexerMetadataStorageCoordinator.deleteDataSourceMetadata(dataSource);
-      log.info("Reset dataSource[%s] - dataSource metadata entry deleted? [%s]", dataSource, result);
-      taskGroups.values().forEach(group -> killTasksInGroup(group, "DataSourceMetadata is not found while reset"));
-      taskGroups.clear();
-      partitionGroups.clear();
-    } else if (!(dataSourceMetadata instanceof KafkaDataSourceMetadata)) {
-      throw new IAE("Expected KafkaDataSourceMetadata but found instance of [%s]", dataSourceMetadata.getClass());
-    } else {
-      // Reset only the partitions in dataSourceMetadata if it has not been reset yet
-      final KafkaDataSourceMetadata resetKafkaMetadata = (KafkaDataSourceMetadata) dataSourceMetadata;
-
-      if (resetKafkaMetadata.getKafkaPartitions().getTopic().equals(ioConfig.getTopic())) {
-        // metadata can be null
-        final DataSourceMetadata metadata = indexerMetadataStorageCoordinator.getDataSourceMetadata(dataSource);
-        if (metadata != null && !(metadata instanceof KafkaDataSourceMetadata)) {
-          throw new IAE(
-              "Expected KafkaDataSourceMetadata from metadata store but found instance of [%s]",
-              metadata.getClass()
-          );
-        }
-        final KafkaDataSourceMetadata currentMetadata = (KafkaDataSourceMetadata) metadata;
-
-        // defend against consecutive reset requests from replicas
-        // as well as the case where the metadata store do not have an entry for the reset partitions
-        boolean doReset = false;
-        for (Entry<Integer, Long> resetPartitionOffset : resetKafkaMetadata.getKafkaPartitions()
-                                                                           .getPartitionOffsetMap()
-                                                                           .entrySet()) {
-          final Long partitionOffsetInMetadataStore = currentMetadata == null
-                                                      ? null
-                                                      : currentMetadata.getKafkaPartitions()
-                                                                       .getPartitionOffsetMap()
-                                                                       .get(resetPartitionOffset.getKey());
-          final TaskGroup partitionTaskGroup = taskGroups.get(
-              getTaskGroupIdForPartition(resetPartitionOffset.getKey())
-          );
-          final boolean isSameOffset = partitionTaskGroup != null
-                                       && partitionTaskGroup.partitionOffsets.get(resetPartitionOffset.getKey())
-                                                                             .equals(resetPartitionOffset.getValue());
-          if (partitionOffsetInMetadataStore != null || isSameOffset) {
-            doReset = true;
-            break;
-          }
-        }
-
-        if (!doReset) {
-          log.info("Ignoring duplicate reset request [%s]", dataSourceMetadata);
-          return;
-        }
-
-        boolean metadataUpdateSuccess = false;
-        if (currentMetadata == null) {
-          metadataUpdateSuccess = true;
-        } else {
-          final DataSourceMetadata newMetadata = currentMetadata.minus(resetKafkaMetadata);
-          try {
-            metadataUpdateSuccess = indexerMetadataStorageCoordinator.resetDataSourceMetadata(dataSource, newMetadata);
-          }
-          catch (IOException e) {
-            log.error("Resetting DataSourceMetadata failed [%s]", e.getMessage());
-            Throwables.propagate(e);
-          }
-        }
-        if (metadataUpdateSuccess) {
-          resetKafkaMetadata.getKafkaPartitions().getPartitionOffsetMap().keySet().forEach(partition -> {
-            final int groupId = getTaskGroupIdForPartition(partition);
-            killTaskGroupForPartitions(ImmutableSet.of(partition), "DataSourceMetadata is updated while reset");
-            taskGroups.remove(groupId);
-            partitionGroups.get(groupId).replaceAll((partitionId, offset) -> NOT_SET);
-          });
-        } else {
-          throw new ISE("Unable to reset metadata");
-        }
-      } else {
-        log.warn(
-            "Reset metadata topic [%s] and supervisor's topic [%s] do not match",
-            resetKafkaMetadata.getKafkaPartitions().getTopic(),
-            ioConfig.getTopic()
-        );
-      }
-    }
-  }
-
-  private void killTaskGroupForPartitions(Set<Integer> partitions, String reasonFormat, Object... args)
-  {
-    for (Integer partition : partitions) {
-      killTasksInGroup(taskGroups.get(getTaskGroupIdForPartition(partition)), reasonFormat, args);
-    }
-  }
-
-  private void killTasksInGroup(TaskGroup taskGroup, String reasonFormat, Object... args)
-  {
-    if (taskGroup != null) {
-      for (String taskId : taskGroup.tasks.keySet()) {
-        killTask(taskId, reasonFormat, args);
-      }
-    }
-  }
-
-  @VisibleForTesting
-  void gracefulShutdownInternal() throws ExecutionException, InterruptedException, TimeoutException
-  {
-    // Prepare for shutdown by 1) killing all tasks that haven't been assigned to a worker yet, and 2) causing all
-    // running tasks to begin publishing by setting their startTime to a very long time ago so that the logic in
-    // checkTaskDuration() will be triggered. This is better than just telling these tasks to publish whatever they
-    // have, as replicas that are supposed to publish the same segment may not have read the same set of offsets.
-    for (TaskGroup taskGroup : taskGroups.values()) {
-      for (Entry<String, TaskData> entry : taskGroup.tasks.entrySet()) {
-        if (taskInfoProvider.getTaskLocation(entry.getKey()).equals(TaskLocation.unknown())) {
-          killTask(entry.getKey(), "Killing task for graceful shutdown");
-        } else {
-          entry.getValue().startTime = DateTimes.EPOCH;
-        }
-      }
-    }
-
-    checkTaskDuration();
-  }
-
-  @VisibleForTesting
-  void runInternal() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException
-  {
-    possiblyRegisterListener();
-    updatePartitionDataFromKafka();
-    discoverTasks();
-    updateTaskStatus();
-    checkTaskDuration();
-    checkPendingCompletionTasks();
-    checkCurrentTaskState();
-
-    // if supervisor is not suspended, ensure required tasks are running
-    // if suspended, ensure tasks have been requested to gracefully stop
-    if (!spec.isSuspended()) {
-      log.info("[%s] supervisor is running.", dataSource);
-      createNewTasks();
-    } else {
-      log.info("[%s] supervisor is suspended.", dataSource);
-      gracefulShutdownInternal();
-    }
-
-    if (log.isDebugEnabled()) {
-      log.debug(generateReport(true).toString());
-    } else {
-      log.info(generateReport(false).toString());
-    }
-  }
-
-  String generateSequenceName(
-      Map<Integer, Long> startPartitions,
-      Optional<DateTime> minimumMessageTime,
-      Optional<DateTime> maximumMessageTime
-  )
-  {
-    StringBuilder sb = new StringBuilder();
-
-    for (Entry<Integer, Long> entry : startPartitions.entrySet()) {
-      sb.append(StringUtils.format("+%d(%d)", entry.getKey(), entry.getValue()));
-    }
-    String partitionOffsetStr = sb.toString().substring(1);
-
-    String minMsgTimeStr = (minimumMessageTime.isPresent() ? String.valueOf(minimumMessageTime.get().getMillis()) : "");
-    String maxMsgTimeStr = (maximumMessageTime.isPresent() ? String.valueOf(maximumMessageTime.get().getMillis()) : "");
-
-    String dataSchema, tuningConfig;
-    try {
-      dataSchema = sortingMapper.writeValueAsString(spec.getDataSchema());
-      tuningConfig = sortingMapper.writeValueAsString(taskTuningConfig);
-    }
-    catch (JsonProcessingException e) {
-      throw Throwables.propagate(e);
-    }
-
-    String hashCode = DigestUtils.sha1Hex(dataSchema
-                                          + tuningConfig
-                                          + partitionOffsetStr
-                                          + minMsgTimeStr
-                                          + maxMsgTimeStr)
-                                 .substring(0, 15);
-
-    return Joiner.on("_").join("index_kafka", dataSource, hashCode);
-  }
-
-  @VisibleForTesting
-  protected void tryInit()
-  {
-    synchronized (stateChangeLock) {
-      if (started) {
-        log.warn("SUpervisor was already started, skipping init");
-        return;
-      }
-
-      if (stopped) {
-        log.warn("Supervisor was already stopped, skipping init.");
-        return;
-      }
-
-      try {
-        consumer = getKafkaConsumer();
-
-        exec.submit(
-            () -> {
-              try {
-                long pollTimeout = Math.max(ioConfig.getPeriod().getMillis(), MAX_RUN_FREQUENCY_MILLIS);
-                while (!Thread.currentThread().isInterrupted() && !stopped) {
-                  final Notice notice = notices.poll(pollTimeout, TimeUnit.MILLISECONDS);
-                  if (notice == null) {
-                    continue;
-                  }
-
-                  try {
-                    notice.handle();
-                  }
-                  catch (Throwable e) {
-                    log.makeAlert(e, "KafkaSupervisor[%s] failed to handle notice", dataSource)
-                       .addData("noticeClass", notice.getClass().getSimpleName())
-                       .emit();
-                  }
-                }
-              }
-              catch (InterruptedException e) {
-                log.info("KafkaSupervisor[%s] interrupted, exiting", dataSource);
-              }
-            }
-        );
-        firstRunTime = DateTimes.nowUtc().plus(ioConfig.getStartDelay());
-        scheduledExec.scheduleAtFixedRate(
-            buildRunTask(),
-            ioConfig.getStartDelay().getMillis(),
-            Math.max(ioConfig.getPeriod().getMillis(), MAX_RUN_FREQUENCY_MILLIS),
-            TimeUnit.MILLISECONDS
-        );
-
-        reportingExec.scheduleAtFixedRate(
-            updateCurrentAndLatestOffsets(),
-            ioConfig.getStartDelay().getMillis() + INITIAL_GET_OFFSET_DELAY_MILLIS, // wait for tasks to start up
-            Math.max(
-                tuningConfig.getOffsetFetchPeriod().getMillis(), MINIMUM_GET_OFFSET_PERIOD_MILLIS
-            ),
-            TimeUnit.MILLISECONDS
-        );
-
-        reportingExec.scheduleAtFixedRate(
-            emitLag(),
-            ioConfig.getStartDelay().getMillis() + INITIAL_EMIT_LAG_METRIC_DELAY_MILLIS, // wait for tasks to start up
-            monitorSchedulerConfig.getEmitterPeriod().getMillis(),
-            TimeUnit.MILLISECONDS
-        );
-
-        started = true;
-        log.info(
-            "Started KafkaSupervisor[%s], first run in [%s], with spec: [%s]",
-            dataSource,
-            ioConfig.getStartDelay(),
-            spec.toString()
-        );
-      }
-      catch (Exception e) {
-        if (consumer != null) {
-          consumer.close();
-        }
-        initRetryCounter++;
-        log.makeAlert(e, "Exception starting KafkaSupervisor[%s]", dataSource)
-           .emit();
-
-        throw new RuntimeException(e);
-      }
-    }
-  }
-
-  private KafkaConsumer<byte[], byte[]> getKafkaConsumer()
-  {
-    final Properties props = new Properties();
-
-    props.setProperty("metadata.max.age.ms", "10000");
-    props.setProperty("group.id", StringUtils.format("kafka-supervisor-%s", RealtimeIndexTask.makeRandomId()));
-
-    KafkaIndexTask.addConsumerPropertiesFromConfig(props, sortingMapper, ioConfig.getConsumerProperties());
-
-    props.setProperty("enable.auto.commit", "false");
-
-    ClassLoader currCtxCl = Thread.currentThread().getContextClassLoader();
-    try {
-      Thread.currentThread().setContextClassLoader(getClass().getClassLoader());
-      return new KafkaConsumer<>(props, new ByteArrayDeserializer(), new ByteArrayDeserializer());
-    }
-    finally {
-      Thread.currentThread().setContextClassLoader(currCtxCl);
-    }
-  }
-
-  private void updatePartitionDataFromKafka()
-  {
-    List<PartitionInfo> partitions;
-    try {
-      synchronized (consumerLock) {
-        partitions = consumer.partitionsFor(ioConfig.getTopic());
-      }
-    }
-    catch (Exception e) {
-      log.warn(
-          e,
-          "Unable to get partition data from Kafka for brokers [%s], are the brokers up?",
-          ioConfig.getConsumerProperties().get(KafkaSupervisorIOConfig.BOOTSTRAP_SERVERS_KEY)
-      );
-      return;
-    }
-
-    int numPartitions = (partitions != null ? partitions.size() : 0);
-
-    log.debug("Found [%d] Kafka partitions for topic [%s]", numPartitions, ioConfig.getTopic());
-
-    for (int partition = 0; partition < numPartitions; partition++) {
-      int taskGroupId = getTaskGroupIdForPartition(partition);
-
-      ConcurrentHashMap<Integer, Long> partitionMap = partitionGroups.computeIfAbsent(
-          taskGroupId,
-          k -> new ConcurrentHashMap<>()
-      );
-
-      // The starting offset for a new partition in [partitionGroups] is initially set to NOT_SET; when a new task group
-      // is created and is assigned partitions, if the offset in [partitionGroups] is NOT_SET it will take the starting
-      // offset value from the metadata store, and if it can't find it there, from Kafka. Once a task begins
-      // publishing, the offset in partitionGroups will be updated to the ending offset of the publishing-but-not-yet-
-      // completed task, which will cause the next set of tasks to begin reading from where the previous task left
-      // off. If that previous task now fails, we will set the offset in [partitionGroups] back to NOT_SET which will
-      // cause successive tasks to again grab their starting offset from metadata store. This mechanism allows us to
-      // start up successive tasks without waiting for the previous tasks to succeed and still be able to handle task
-      // failures during publishing.
-      if (partitionMap.putIfAbsent(partition, NOT_SET) == null) {
-        log.info(
-            "New partition [%d] discovered for topic [%s], added to task group [%d]",
-            partition,
-            ioConfig.getTopic(),
-            taskGroupId
-        );
-      }
-    }
-  }
-
-  private void discoverTasks() throws ExecutionException, InterruptedException, TimeoutException
-  {
-    int taskCount = 0;
-    List<String> futureTaskIds = new ArrayList<>();
-    List<ListenableFuture<Boolean>> futures = new ArrayList<>();
-    List<Task> tasks = taskStorage.getActiveTasks();
-    final Map<Integer, TaskGroup> taskGroupsToVerify = new HashMap<>();
-
-    for (Task task : tasks) {
-      if (!(task instanceof KafkaIndexTask) || !dataSource.equals(task.getDataSource())) {
-        continue;
-      }
-
-      taskCount++;
-      final KafkaIndexTask kafkaTask = (KafkaIndexTask) task;
-      final String taskId = task.getId();
-
-      // Determine which task group this task belongs to based on one of the partitions handled by this task. If we
-      // later determine that this task is actively reading, we will make sure that it matches our current partition
-      // allocation (getTaskGroupIdForPartition(partition) should return the same value for every partition being read
-      // by this task) and kill it if it is not compatible. If the task is instead found to be in the publishing
-      // state, we will permit it to complete even if it doesn't match our current partition allocation to support
-      // seamless schema migration.
-
-      Iterator<Integer> it = kafkaTask.getIOConfig().getStartPartitions().getPartitionOffsetMap().keySet().iterator();
-      final Integer taskGroupId = (it.hasNext() ? getTaskGroupIdForPartition(it.next()) : null);
-
-      if (taskGroupId != null) {
-        // check to see if we already know about this task, either in [taskGroups] or in [pendingCompletionTaskGroups]
-        // and if not add it to taskGroups or pendingCompletionTaskGroups (if status = PUBLISHING)
-        TaskGroup taskGroup = taskGroups.get(taskGroupId);
-        if (!isTaskInPendingCompletionGroups(taskId) && (taskGroup == null || !taskGroup.tasks.containsKey(taskId))) {
-
-          futureTaskIds.add(taskId);
-          futures.add(
-              Futures.transform(
-                  taskClient.getStatusAsync(taskId), new Function<KafkaIndexTask.Status, Boolean>()
-                  {
-                    @Override
-                    public Boolean apply(KafkaIndexTask.Status status)
-                    {
-                      try {
-                        log.debug("Task [%s], status [%s]", taskId, status);
-                        if (status == KafkaIndexTask.Status.PUBLISHING) {
-                          kafkaTask.getIOConfig().getStartPartitions().getPartitionOffsetMap().keySet().forEach(
-                              partition -> addDiscoveredTaskToPendingCompletionTaskGroups(
-                                  getTaskGroupIdForPartition(partition),
-                                  taskId,
-                                  kafkaTask.getIOConfig()
-                                           .getStartPartitions()
-                                           .getPartitionOffsetMap()
-                              )
-                          );
-
-                          // update partitionGroups with the publishing task's offsets (if they are greater than what is
-                          // existing) so that the next tasks will start reading from where this task left off
-                          Map<Integer, Long> publishingTaskEndOffsets = taskClient.getEndOffsets(taskId);
-
-                          for (Entry<Integer, Long> entry : publishingTaskEndOffsets.entrySet()) {
-                            Integer partition = entry.getKey();
-                            Long offset = entry.getValue();
-                            ConcurrentHashMap<Integer, Long> partitionOffsets = partitionGroups.get(
-                                getTaskGroupIdForPartition(partition)
-                            );
-
-                            boolean succeeded;
-                            do {
-                              succeeded = true;
-                              Long previousOffset = partitionOffsets.putIfAbsent(partition, offset);
-                              if (previousOffset != null && previousOffset < offset) {
-                                succeeded = partitionOffsets.replace(partition, previousOffset, offset);
-                              }
-                            } while (!succeeded);
-                          }
-                        } else {
-                          for (Integer partition : kafkaTask.getIOConfig()
-                                                            .getStartPartitions()
-                                                            .getPartitionOffsetMap()
-                                                            .keySet()) {
-                            if (!taskGroupId.equals(getTaskGroupIdForPartition(partition))) {
-                              log.warn(
-                                  "Stopping task [%s] which does not match the expected partition allocation",
-                                  taskId
-                              );
-                              try {
-                                stopTask(taskId, false).get(futureTimeoutInSeconds, TimeUnit.SECONDS);
-                              }
-                              catch (InterruptedException | ExecutionException | TimeoutException e) {
-                                log.warn(e, "Exception while stopping task");
-                              }
-                              return false;
-                            }
-                          }
-                          // make sure the task's io and tuning configs match with the supervisor config
-                          // if it is current then only create corresponding taskGroup if it does not exist
-                          if (!isTaskCurrent(taskGroupId, taskId)) {
-                            log.info(
-                                "Stopping task [%s] which does not match the expected parameters and ingestion spec",
-                                taskId
-                            );
-                            try {
-                              stopTask(taskId, false).get(futureTimeoutInSeconds, TimeUnit.SECONDS);
-                            }
-                            catch (InterruptedException | ExecutionException | TimeoutException e) {
-                              log.warn(e, "Exception while stopping task");
-                            }
-                            return false;
-                          } else {
-                            final TaskGroup taskGroup = taskGroups.computeIfAbsent(
-                                taskGroupId,
-                                k -> {
-                                  log.info("Creating a new task group for taskGroupId[%d]", taskGroupId);
-                                  return new TaskGroup(
-                                      taskGroupId,
-                                      ImmutableMap.copyOf(
-                                          kafkaTask.getIOConfig().getStartPartitions().getPartitionOffsetMap()
-                                      ),
-                                      kafkaTask.getIOConfig().getMinimumMessageTime(),
-                                      kafkaTask.getIOConfig().getMaximumMessageTime()
-                                  );
-                                }
-                            );
-                            taskGroupsToVerify.put(taskGroupId, taskGroup);
-                            final TaskData prevTaskData = taskGroup.tasks.putIfAbsent(taskId, new TaskData());
-                            if (prevTaskData != null) {
-                              throw new ISE(
-                                  "WTH? a taskData[%s] already exists for new task[%s]",
-                                  prevTaskData,
-                                  taskId
-                              );
-                            }
-                          }
-                        }
-                        return true;
-                      }
-                      catch (Throwable t) {
-                        log.error(t, "Something bad while discovering task [%s]", taskId);
-                        return null;
-                      }
-                    }
-                  }, workerExec
-              )
-          );
-        }
-      }
-    }
-
-    List<Boolean> results = Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS);
-    for (int i = 0; i < results.size(); i++) {
-      if (results.get(i) == null) {
-        String taskId = futureTaskIds.get(i);
-        killTask(taskId, "Task [%s] failed to return status, killing task", taskId);
-      }
-    }
-    log.debug("Found [%d] Kafka indexing tasks for dataSource [%s]", taskCount, dataSource);
-
-    // make sure the checkpoints are consistent with each other and with the metadata store
-    verifyAndMergeCheckpoints(taskGroupsToVerify.values());
-  }
-
-  private void verifyAndMergeCheckpoints(final Collection<TaskGroup> taskGroupsToVerify)
-  {
-    final List<ListenableFuture<?>> futures = new ArrayList<>();
-    for (TaskGroup taskGroup : taskGroupsToVerify) {
-      futures.add(workerExec.submit(() -> verifyAndMergeCheckpoints(taskGroup)));
-    }
-    try {
-      Futures.allAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS);
-    }
-    catch (InterruptedException | ExecutionException | TimeoutException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  /**
-   * This method does two things -
-   * 1. Makes sure the checkpoints information in the taskGroup is consistent with that of the tasks, if not kill
-   * inconsistent tasks.
-   * 2. truncates the checkpoints in the taskGroup corresponding to which segments have been published, so that any newly
-   * created tasks for the taskGroup start indexing from after the latest published offsets.
-   */
-  private void verifyAndMergeCheckpoints(final TaskGroup taskGroup)
-  {
-    final int groupId = taskGroup.groupId;
-    final List<Pair<String, TreeMap<Integer, Map<Integer, Long>>>> taskSequences = new ArrayList<>();
-    final List<ListenableFuture<TreeMap<Integer, Map<Integer, Long>>>> futures = new ArrayList<>();
-    final List<String> taskIds = new ArrayList<>();
-
-    for (String taskId : taskGroup.taskIds()) {
-      final ListenableFuture<TreeMap<Integer, Map<Integer, Long>>> checkpointsFuture = taskClient.getCheckpointsAsync(
-          taskId,
-          true
-      );
-      taskIds.add(taskId);
-      futures.add(checkpointsFuture);
-    }
-
-    try {
-      List<TreeMap<Integer, Map<Integer, Long>>> futuresResult = Futures.successfulAsList(futures)
-                                                                        .get(futureTimeoutInSeconds, TimeUnit.SECONDS);
-
-      for (int i = 0; i < futuresResult.size(); i++) {
-        final TreeMap<Integer, Map<Integer, Long>> checkpoints = futuresResult.get(i);
-        final String taskId = taskIds.get(i);
-        if (checkpoints == null) {
-          try {
-            // catch the exception in failed futures
-            futures.get(i).get();
-          }
-          catch (Exception e) {
-            log.error(e, "Problem while getting checkpoints for task [%s], killing the task", taskId);
-            killTask(taskId, "Exception[%s] while getting checkpoints", e.getClass());
-            taskGroup.tasks.remove(taskId);
-          }
-        } else if (checkpoints.isEmpty()) {
-          log.warn("Ignoring task [%s], as probably it is not started running yet", taskId);
-        } else {
-          taskSequences.add(new Pair<>(taskId, checkpoints));
-        }
-      }
-    }
-    catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-
-    final KafkaDataSourceMetadata latestDataSourceMetadata = (KafkaDataSourceMetadata) indexerMetadataStorageCoordinator
-        .getDataSourceMetadata(dataSource);
-    final boolean hasValidOffsetsFromDb = latestDataSourceMetadata != null &&
-                                          latestDataSourceMetadata.getKafkaPartitions() != null &&
-                                          ioConfig.getTopic().equals(
-                                              latestDataSourceMetadata.getKafkaPartitions().getTopic()
-                                          );
-    final Map<Integer, Long> latestOffsetsFromDb;
-    if (hasValidOffsetsFromDb) {
-      latestOffsetsFromDb = latestDataSourceMetadata.getKafkaPartitions().getPartitionOffsetMap();
-    } else {
-      latestOffsetsFromDb = null;
-    }
-
-    // order tasks of this taskGroup by the latest sequenceId
-    taskSequences.sort((o1, o2) -> o2.rhs.firstKey().compareTo(o1.rhs.firstKey()));
-
-    final Set<String> tasksToKill = new HashSet<>();
-    final AtomicInteger earliestConsistentSequenceId = new AtomicInteger(-1);
-    int taskIndex = 0;
-
-    while (taskIndex < taskSequences.size()) {
-      TreeMap<Integer, Map<Integer, Long>> taskCheckpoints = taskSequences.get(taskIndex).rhs;
-      String taskId = taskSequences.get(taskIndex).lhs;
-      if (earliestConsistentSequenceId.get() == -1) {
-        // find the first replica task with earliest sequenceId consistent with datasource metadata in the metadata
-        // store
-        if (taskCheckpoints.entrySet().stream().anyMatch(
-            sequenceCheckpoint -> sequenceCheckpoint.getValue().entrySet().stream().allMatch(
-                partitionOffset -> Longs.compare(
-                    partitionOffset.getValue(),
-                    latestOffsetsFromDb == null ?
-                    partitionOffset.getValue() :
-                    latestOffsetsFromDb.getOrDefault(partitionOffset.getKey(), partitionOffset.getValue())
-                ) == 0) && earliestConsistentSequenceId.compareAndSet(-1, sequenceCheckpoint.getKey())) || (
-                someTaskGroupsPendingCompletion(groupId)
-                && earliestConsistentSequenceId.compareAndSet(-1, taskCheckpoints.firstKey()))) {
-          final SortedMap<Integer, Map<Integer, Long>> latestCheckpoints = new TreeMap<>(
-              taskCheckpoints.tailMap(earliestConsistentSequenceId.get())
-          );
-          log.info("Setting taskGroup sequences to [%s] for group [%d]", latestCheckpoints, groupId);
-          taskGroup.sequenceOffsets.clear();
-          taskGroup.sequenceOffsets.putAll(latestCheckpoints);
-        } else {
-          log.debug(
-              "Adding task [%s] to kill list, checkpoints[%s], latestoffsets from DB [%s]",
-              taskId,
-              taskCheckpoints,
-              latestOffsetsFromDb
-          );
-          tasksToKill.add(taskId);
-        }
-      } else {
-        // check consistency with taskGroup sequences
-        if (taskCheckpoints.get(taskGroup.sequenceOffsets.firstKey()) == null
-            || !(taskCheckpoints.get(taskGroup.sequenceOffsets.firstKey())
-                                .equals(taskGroup.sequenceOffsets.firstEntry().getValue()))
-            || taskCheckpoints.tailMap(taskGroup.sequenceOffsets.firstKey()).size()
-               != taskGroup.sequenceOffsets.size()) {
-          log.debug(
-              "Adding task [%s] to kill list, checkpoints[%s], taskgroup checkpoints [%s]",
-              taskId,
-              taskCheckpoints,
-              taskGroup.sequenceOffsets
-          );
-          tasksToKill.add(taskId);
-        }
-      }
-      taskIndex++;
-    }
-
-    if ((tasksToKill.size() > 0 && tasksToKill.size() == taskGroup.tasks.size()) ||
-        (taskGroup.tasks.size() == 0 && !someTaskGroupsPendingCompletion(groupId))) {
-      // killing all tasks or no task left in the group ?
-      // clear state about the taskgroup so that get latest offset information is fetched from metadata store
-      log.warn("Clearing task group [%d] information as no valid tasks left the group", groupId);
-      taskGroups.remove(groupId);
-      partitionGroups.get(groupId).replaceAll((partition, offset) -> NOT_SET);
-    }
-
-    taskSequences.stream().filter(taskIdSequences -> tasksToKill.contains(taskIdSequences.lhs)).forEach(
-        sequenceCheckpoint -> {
-          killTask(
-              sequenceCheckpoint.lhs,
-              "Killing task [%s], as its checkpoints [%s] are not consistent with group checkpoints[%s] or latest "
-              + "persisted offsets in metadata store [%s]",
-              sequenceCheckpoint.lhs,
-              sequenceCheckpoint.rhs,
-              taskGroup.sequenceOffsets,
-              latestOffsetsFromDb
-          );
-          taskGroup.tasks.remove(sequenceCheckpoint.lhs);
-        }
-    );
-  }
-
-  private void addDiscoveredTaskToPendingCompletionTaskGroups(
-      int groupId,
-      String taskId,
-      Map<Integer, Long> startingPartitions
-  )
-  {
-    final CopyOnWriteArrayList<TaskGroup> taskGroupList = pendingCompletionTaskGroups.computeIfAbsent(
-        groupId,
-        k -> new CopyOnWriteArrayList<>()
-    );
-    for (TaskGroup taskGroup : taskGroupList) {
-      if (taskGroup.partitionOffsets.equals(startingPartitions)) {
-        if (taskGroup.tasks.putIfAbsent(taskId, new TaskData()) == null) {
-          log.info("Added discovered task [%s] to existing pending task group [%s]", taskId, groupId);
-        }
-        return;
-      }
-    }
-
-    log.info("Creating new pending completion task group [%s] for discovered task [%s]", groupId, taskId);
-
-    // reading the minimumMessageTime & maximumMessageTime from the publishing task and setting it here is not necessary as this task cannot
-    // change to a state where it will read any more events
-    TaskGroup newTaskGroup = new TaskGroup(
-        groupId,
-        ImmutableMap.copyOf(startingPartitions),
-        Optional.absent(),
-        Optional.absent()
-    );
-
-    newTaskGroup.tasks.put(taskId, new TaskData());
-    newTaskGroup.completionTimeout = DateTimes.nowUtc().plus(ioConfig.getCompletionTimeout());
-
-    taskGroupList.add(newTaskGroup);
-  }
-
-  private void updateTaskStatus() throws ExecutionException, InterruptedException, TimeoutException
-  {
-    final List<ListenableFuture<Boolean>> futures = new ArrayList<>();
-    final List<String> futureTaskIds = new ArrayList<>();
-
-    // update status (and startTime if unknown) of current tasks in taskGroups
-    for (TaskGroup group : taskGroups.values()) {
-      for (Entry<String, TaskData> entry : group.tasks.entrySet()) {
-        final String taskId = entry.getKey();
-        final TaskData taskData = entry.getValue();
-
-        if (taskData.startTime == null) {
-          futureTaskIds.add(taskId);
-          futures.add(
-              Futures.transform(
-                  taskClient.getStartTimeAsync(taskId), new Function<DateTime, Boolean>()
-                  {
-                    @Nullable
-                    @Override
-                    public Boolean apply(@Nullable DateTime startTime)
-                    {
-                      if (startTime == null) {
-                        return false;
-                      }
-
-                      taskData.startTime = startTime;
-                      long millisRemaining = ioConfig.getTaskDuration().getMillis() -
-                                             (System.currentTimeMillis() - taskData.startTime.getMillis());
-                      if (millisRemaining > 0) {
-                        scheduledExec.schedule(
-                            buildRunTask(),
-                            millisRemaining + MAX_RUN_FREQUENCY_MILLIS,
-                            TimeUnit.MILLISECONDS
-                        );
-                      }
-
-                      return true;
-                    }
-                  }, workerExec
-              )
-          );
-        }
-
-        taskData.status = taskStorage.getStatus(taskId).get();
-      }
-    }
-
-    // update status of pending completion tasks in pendingCompletionTaskGroups
-    for (List<TaskGroup> taskGroups : pendingCompletionTaskGroups.values()) {
-      for (TaskGroup group : taskGroups) {
-        for (Entry<String, TaskData> entry : group.tasks.entrySet()) {
-          entry.getValue().status = taskStorage.getStatus(entry.getKey()).get();
-        }
-      }
-    }
-
-    List<Boolean> results = Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS);
-    for (int i = 0; i < results.size(); i++) {
-      // false means the task hasn't started running yet and that's okay; null means it should be running but the HTTP
-      // request threw an exception so kill the task
-      if (results.get(i) == null) {
-        String taskId = futureTaskIds.get(i);
-        killTask(taskId, "Task [%s] failed to return start time, killing task", taskId);
-      }
-    }
-  }
-
-  private void checkTaskDuration() throws InterruptedException, ExecutionException, TimeoutException
-  {
-    final List<ListenableFuture<Map<Integer, Long>>> futures = new ArrayList<>();
-    final List<Integer> futureGroupIds = new ArrayList<>();
-
-    for (Entry<Integer, TaskGroup> entry : taskGroups.entrySet()) {
-      Integer groupId = entry.getKey();
-      TaskGroup group = entry.getValue();
-
-      // find the longest running task from this group
-      DateTime earliestTaskStart = DateTimes.nowUtc();
-      for (TaskData taskData : group.tasks.values()) {
-        // startTime can be null if kafkaSupervisor is stopped gracefully before processing any runNotice
-        if (taskData.startTime != null && earliestTaskStart.isAfter(taskData.startTime)) {
-          earliestTaskStart = taskData.startTime;
-        }
-      }
-
-      // if this task has run longer than the configured duration, signal all tasks in the group to persist
-      if (earliestTaskStart.plus(ioConfig.getTaskDuration()).isBeforeNow()) {
-        log.info("Task group [%d] has run for [%s]", groupId, ioConfig.getTaskDuration());
-        futureGroupIds.add(groupId);
-        futures.add(checkpointTaskGroup(group, true));
-      }
-    }
-
-    List<Map<Integer, Long>> results = Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS);
-    for (int j = 0; j < results.size(); j++) {
-      Integer groupId = futureGroupIds.get(j);
-      TaskGroup group = taskGroups.get(groupId);
-      Map<Integer, Long> endOffsets = results.get(j);
-
-      if (endOffsets != null) {
-        // set a timeout and put this group in pendingCompletionTaskGroups so that it can be monitored for completion
-        group.completionTimeout = DateTimes.nowUtc().plus(ioConfig.getCompletionTimeout());
-        pendingCompletionTaskGroups.computeIfAbsent(groupId, k -> new CopyOnWriteArrayList<>()).add(group);
-
-        // set endOffsets as the next startOffsets
-        for (Entry<Integer, Long> entry : endOffsets.entrySet()) {
-          partitionGroups.get(groupId).put(entry.getKey(), entry.getValue());
-        }
-      } else {
-        for (String id : group.taskIds()) {
-          killTask(
-              id,
-              "All tasks in group [%s] failed to transition to publishing state",
-              groupId
-          );
-        }
-        // clear partitionGroups, so that latest offsets from db is used as start offsets not the stale ones
-        // if tasks did some successful incremental handoffs
-        partitionGroups.get(groupId).replaceAll((partition, offset) -> NOT_SET);
-      }
-
-      // remove this task group from the list of current task groups now that it has been handled
-      taskGroups.remove(groupId);
-    }
-  }
-
-  private ListenableFuture<Map<Integer, Long>> checkpointTaskGroup(final TaskGroup taskGroup, final boolean finalize)
-  {
-    if (finalize) {
-      // 1) Check if any task completed (in which case we're done) and kill unassigned tasks
-      Iterator<Entry<String, TaskData>> i = taskGroup.tasks.entrySet().iterator();
-      while (i.hasNext()) {
-        Entry<String, TaskData> taskEntry = i.next();
-        String taskId = taskEntry.getKey();
-        TaskData task = taskEntry.getValue();
-
-        // task.status can be null if kafkaSupervisor is stopped gracefully before processing any runNotice.
-        if (task.status != null) {
-          if (task.status.isSuccess()) {
-            // If any task in this group has already completed, stop the rest of the tasks in the group and return.
-            // This will cause us to create a new set of tasks next cycle that will start from the offsets in
-            // metadata store (which will have advanced if we succeeded in publishing and will remain the same if
-            // publishing failed and we need to re-ingest)
-            return Futures.transform(
-                stopTasksInGroup(taskGroup, "task[%s] succeeded in the taskGroup", task.status.getId()),
-                new Function<Object, Map<Integer, Long>>()
-                {
-                  @Nullable
-                  @Override
-                  public Map<Integer, Long> apply(@Nullable Object input)
-                  {
-                    return null;
-                  }
-                }
-            );
-          }
-
-          if (task.status.isRunnable()) {
-            if (taskInfoProvider.getTaskLocation(taskId).equals(TaskLocation.unknown())) {
-              killTask(taskId, "Killing task [%s] which hasn't been assigned to a worker", taskId);
-              i.remove();
-            }
-          }
-        }
-      }
-    }
-
-    // 2) Pause running tasks
-    final List<ListenableFuture<Map<Integer, Long>>> pauseFutures = new ArrayList<>();
-    final List<String> pauseTaskIds = ImmutableList.copyOf(taskGroup.taskIds());
-    for (final String taskId : pauseTaskIds) {
-      pauseFutures.add(taskClient.pauseAsync(taskId));
-    }
-
-    return Futures.transform(
-        Futures.successfulAsList(pauseFutures), new Function<List<Map<Integer, Long>>, Map<Integer, Long>>()
-        {
-          @Nullable
-          @Override
-          public Map<Integer, Long> apply(List<Map<Integer, Long>> input)
-          {
-            // 3) Build a map of the highest offset read by any task in the group for each partition
-            final Map<Integer, Long> endOffsets = new HashMap<>();
-            for (int i = 0; i < input.size(); i++) {
-              final Map<Integer, Long> result = input.get(i);
-              final String taskId = pauseTaskIds.get(i);
-
-              if (result == null) {
-                // Get the exception
-                final Throwable pauseException;
-                try {
-                  // The below get should throw ExecutionException since result is null.
-                  final Map<Integer, Long> pauseResult = pauseFutures.get(i).get();
-                  throw new ISE(
-                      "WTH? The pause request for task [%s] is supposed to fail, but returned [%s]",
-                      taskId,
-                      pauseResult
-                  );
-                }
-                catch (InterruptedException e) {
-                  throw new RuntimeException(e);
-                }
-                catch (ExecutionException e) {
-                  pauseException = e.getCause() == null ? e : e.getCause();
-                }
-
-                killTask(
-                    taskId,
-                    "An exception occured while waiting for task [%s] to pause: [%s]",
-                    taskId,
-                    pauseException
-                );
-                taskGroup.tasks.remove(taskId);
-
-              } else if (result.isEmpty()) {
-                killTask(taskId, "Task [%s] returned empty offsets after pause", taskId);
-                taskGroup.tasks.remove(taskId);
-              } else { // otherwise build a map of the highest offsets seen
-                for (Entry<Integer, Long> offset : result.entrySet()) {
-                  if (!endOffsets.containsKey(offset.getKey())
-                      || endOffsets.get(offset.getKey()).compareTo(offset.getValue()) < 0) {
-                    endOffsets.put(offset.getKey(), offset.getValue());
-                  }
-                }
-              }
-            }
-
-            // 4) Set the end offsets for each task to the values from step 3 and resume the tasks. All the tasks should
-            //    finish reading and start publishing within a short period, depending on how in sync the tasks were.
-            final List<ListenableFuture<Boolean>> setEndOffsetFutures = new ArrayList<>();
-            final List<String> setEndOffsetTaskIds = ImmutableList.copyOf(taskGroup.taskIds());
-
-            if (setEndOffsetTaskIds.isEmpty()) {
-              log.info("All tasks in taskGroup [%d] have failed, tasks will be re-created", taskGroup.groupId);
-              return null;
-            }
-
-            try {
-
-              if (endOffsets.equals(taskGroup.sequenceOffsets.lastEntry().getValue())) {
-                log.warn(
-                    "Checkpoint [%s] is same as the start offsets [%s] of latest sequence for the task group [%d]",
-                    endOffsets,
-                    taskGroup.sequenceOffsets.lastEntry().getValue(),
-                    taskGroup.groupId
-                );
-              }
-
-              log.info(
-                  "Setting endOffsets for tasks in taskGroup [%d] to %s and resuming",
-                  taskGroup.groupId,
-                  endOffsets
-              );
-              for (final String taskId : setEndOffsetTaskIds) {
-                setEndOffsetFutures.add(taskClient.setEndOffsetsAsync(taskId, endOffsets, finalize));
-              }
-
-              List<Boolean> results = Futures.successfulAsList(setEndOffsetFutures)
-                                             .get(futureTimeoutInSeconds, TimeUnit.SECONDS);
-              for (int i = 0; i < results.size(); i++) {
-                if (results.get(i) == null || !results.get(i)) {
-                  String taskId = setEndOffsetTaskIds.get(i);
-                  killTask(
-                      taskId,
-                      "Task [%s] failed to respond to [set end offsets] in a timely manner, killing task",
-                      taskId
-                  );
-                  taskGroup.tasks.remove(taskId);
-                }
-              }
-            }
-            catch (Exception e) {
-              log.error("Something bad happened [%s]", e.getMessage());
-              Throwables.propagate(e);
-            }
-
-            if (taskGroup.tasks.isEmpty()) {
-              log.info("All tasks in taskGroup [%d] have failed, tasks will be re-created", taskGroup.groupId);
-              return null;
-            }
-
-            return endOffsets;
-          }
-        }, workerExec
-    );
-  }
-
-  /**
-   * Monitors [pendingCompletionTaskGroups] for tasks that have completed. If any task in a task group has completed, we
-   * can safely stop the rest of the tasks in that group. If a task group has exceeded its publishing timeout, then
-   * we need to stop all tasks in not only that task group but also 1) any subsequent task group that is also pending
-   * completion and 2) the current task group that is running, because the assumption that we have handled up to the
-   * starting offset for subsequent task groups is no longer valid, and subsequent tasks would fail as soon as they
-   * attempted to publish because of the contiguous range consistency check.
-   */
-  private void checkPendingCompletionTasks() throws ExecutionException, InterruptedException, TimeoutException
-  {
-    List<ListenableFuture<?>> futures = new ArrayList<>();
-
-    for (Entry<Integer, CopyOnWriteArrayList<TaskGroup>> pendingGroupList : pendingCompletionTaskGroups.entrySet()) {
-
-      boolean stopTasksInTaskGroup = false;
-      Integer groupId = pendingGroupList.getKey();
-      CopyOnWriteArrayList<TaskGroup> taskGroupList = pendingGroupList.getValue();
-      List<TaskGroup> toRemove = new ArrayList<>();
-
-      for (TaskGroup group : taskGroupList) {
-        boolean foundSuccess = false, entireTaskGroupFailed = false;
-
-        if (stopTasksInTaskGroup) {
-          // One of the earlier groups that was handling the same partition set timed out before the segments were
-          // published so stop any additional groups handling the same partition set that are pending completion.
-          futures.add(
-              stopTasksInGroup(
-                  group,
-                  "one of earlier groups that was handling the same partition set timed out before publishing segments"
-              )
-          );
-          toRemove.add(group);
-          continue;
-        }
-
-        Iterator<Entry<String, TaskData>> iTask = group.tasks.entrySet().iterator();
-        while (iTask.hasNext()) {
-          final Entry<String, TaskData> entry = iTask.next();
-          final String taskId = entry.getKey();
-          final TaskData taskData = entry.getValue();
-
-          Preconditions.checkNotNull(taskData.status, "WTH? task[%s] has a null status", taskId);
-
-          if (taskData.status.isFailure()) {
-            iTask.remove(); // remove failed task
-            if (group.tasks.isEmpty()) {
-              // if all tasks in the group have failed, just nuke all task groups with this partition set and restart
-              entireTaskGroupFailed = true;
-              break;
-            }
-          }
-
-          if (taskData.status.isSuccess()) {
-            // If one of the pending completion tasks was successful, stop the rest of the tasks in the group as
-            // we no longer need them to publish their segment.
-            futures.add(
-                stopTasksInGroup(group, "Task [%s] completed successfully, stopping tasks %s", taskId, group.taskIds())
-            );
-            foundSuccess = true;
-            toRemove.add(group); // remove the TaskGroup from the list of pending completion task groups
-            break; // skip iterating the rest of the tasks in this group as they've all been stopped now
-          }
-        }
-
-        if ((!foundSuccess && group.completionTimeout.isBeforeNow()) || entireTaskGroupFailed) {
-          if (entireTaskGroupFailed) {
-            log.warn("All tasks in group [%d] failed to publish, killing all tasks for these partitions", groupId);
-          } else {
-            log.makeAlert(
-                "No task in [%s] for taskGroup [%d] succeeded before the completion timeout elapsed [%s]!",
-                group.taskIds(),
-                groupId,
-                ioConfig.getCompletionTimeout()
-            ).emit();
-          }
-
-          // reset partitions offsets for this task group so that they will be re-read from metadata storage
-          partitionGroups.get(groupId).replaceAll((partition, offset) -> NOT_SET);
-          // kill all the tasks in this pending completion group
-          killTasksInGroup(
-              group,
-              "No task in pending completion taskGroup[%d] succeeded before completion timeout elapsed",
-              groupId
-          );
-          // set a flag so the other pending completion groups for this set of partitions will also stop
-          stopTasksInTaskGroup = true;
-
-          // kill all the tasks in the currently reading task group and remove the bad task group
-          killTasksInGroup(
-              taskGroups.remove(groupId),
-              "No task in the corresponding pending completion taskGroup[%d] succeeded before completion timeout elapsed",
-              groupId
-          );
-          toRemove.add(group);
-        }
-      }
-
-      taskGroupList.removeAll(toRemove);
-    }
-
-    // wait for all task shutdowns to complete before returning
-    Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS);
-  }
-
-  private void checkCurrentTaskState() throws ExecutionException, InterruptedException, TimeoutException
-  {
-    List<ListenableFuture<?>> futures = new ArrayList<>();
-    Iterator<Entry<Integer, TaskGroup>> iTaskGroups = taskGroups.entrySet().iterator();
-    while (iTaskGroups.hasNext()) {
-      Entry<Integer, TaskGroup> taskGroupEntry = iTaskGroups.next();
-      Integer groupId = taskGroupEntry.getKey();
-      TaskGroup taskGroup = taskGroupEntry.getValue();
-
-      // Iterate the list of known tasks in this group and:
-      //   1) Kill any tasks which are not "current" (have the partitions, starting offsets, and minimumMessageTime
-      //      & maximumMessageTime (if applicable) in [taskGroups])
-      //   2) Remove any tasks that have failed from the list
-      //   3) If any task completed successfully, stop all the tasks in this group and move to the next group
-
-      log.debug("Task group [%d] pre-pruning: %s", groupId, taskGroup.taskIds());
-
-      Iterator<Entry<String, TaskData>> iTasks = taskGroup.tasks.entrySet().iterator();
-      while (iTasks.hasNext()) {
-        Entry<String, TaskData> task = iTasks.next();
-        String taskId = task.getKey();
-        TaskData taskData = task.getValue();
-
-        // stop and remove bad tasks from the task group
-        if (!isTaskCurrent(groupId, taskId)) {
-          log.info("Stopping task [%s] which does not match the expected offset range and ingestion spec", taskId);
-          futures.add(stopTask(taskId, false));
-          iTasks.remove();
-          continue;
-        }
-
-        Preconditions.checkNotNull(taskData.status, "WTH? task[%s] has a null status", taskId);
-
-        // remove failed tasks
-        if (taskData.status.isFailure()) {
-          iTasks.remove();
-          continue;
-        }
-
-        // check for successful tasks, and if we find one, stop all tasks in the group and remove the group so it can
-        // be recreated with the next set of offsets
-        if (taskData.status.isSuccess()) {
-          futures.add(stopTasksInGroup(taskGroup, "task[%s] succeeded in the same taskGroup", taskData.status.getId()));
-          iTaskGroups.remove();
-          break;
-        }
-      }
-      log.debug("Task group [%d] post-pruning: %s", groupId, taskGroup.taskIds());
-    }
-
-    // wait for all task shutdowns to complete before returning
-    Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS);
-  }
-
-  void createNewTasks() throws JsonProcessingException
-  {
-    // update the checkpoints in the taskGroup to latest ones so that new tasks do not read what is already published
-    verifyAndMergeCheckpoints(
-        taskGroups.values()
-                  .stream()
-                  .filter(taskGroup -> taskGroup.tasks.size() < ioConfig.getReplicas())
-                  .collect(Collectors.toList())
-    );
-
-    // check that there is a current task group for each group of partitions in [partitionGroups]
-    for (Integer groupId : partitionGroups.keySet()) {
-      if (!taskGroups.containsKey(groupId)) {
-        log.info("Creating new task group [%d] for partitions %s", groupId, partitionGroups.get(groupId).keySet());
-
-        Optional<DateTime> minimumMessageTime = (ioConfig.getLateMessageRejectionPeriod().isPresent() ? Optional.of(
-            DateTimes.nowUtc().minus(ioConfig.getLateMessageRejectionPeriod().get())
-        ) : Optional.absent());
-
-        Optional<DateTime> maximumMessageTime = (ioConfig.getEarlyMessageRejectionPeriod().isPresent() ? Optional.of(
-            DateTimes.nowUtc().plus(ioConfig.getTaskDuration()).plus(ioConfig.getEarlyMessageRejectionPeriod().get())
-        ) : Optional.absent());
-
-        final TaskGroup taskGroup = new TaskGroup(
-            groupId,
-            generateStartingOffsetsForPartitionGroup(groupId),
-            minimumMessageTime,
-            maximumMessageTime
-        );
-        taskGroups.put(
-            groupId,
-            taskGroup
-        );
-      }
-    }
-
-    // iterate through all the current task groups and make sure each one has the desired number of replica tasks
-    boolean createdTask = false;
-    for (Entry<Integer, TaskGroup> entry : taskGroups.entrySet()) {
-      TaskGroup taskGroup = entry.getValue();
-      Integer groupId = entry.getKey();
+  private final ServiceEmitter emitter;
+  private final DruidMonitorSchedulerConfig monitorSchedulerConfig;
+  private volatile Map<Integer, Long> latestSequenceFromStream;
 
-      if (ioConfig.getReplicas() > taskGroup.tasks.size()) {
-        log.info(
-            "Number of tasks [%d] does not match configured numReplicas [%d] in task group [%d], creating more tasks",
-            taskGroup.tasks.size(), ioConfig.getReplicas(), groupId
-        );
-        createKafkaTasksForGroup(groupId, ioConfig.getReplicas() - taskGroup.tasks.size());
-        createdTask = true;
-      }
-    }
 
-    if (createdTask && firstRunTime.isBeforeNow()) {
-      // Schedule a run event after a short delay to update our internal data structures with the new tasks that were
-      // just created. This is mainly for the benefit of the status API in situations where the run period is lengthy.
-      scheduledExec.schedule(buildRunTask(), 5000, TimeUnit.MILLISECONDS);
-    }
-  }
+  private final KafkaSupervisorSpec spec;
 
-  private void createKafkaTasksForGroup(int groupId, int replicas) throws JsonProcessingException
+  public KafkaSupervisor(
+      final TaskStorage taskStorage,
+      final TaskMaster taskMaster,
+      final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator,
+      final KafkaIndexTaskClientFactory taskClientFactory,
+      final ObjectMapper mapper,
+      final KafkaSupervisorSpec spec,
+      final RowIngestionMetersFactory rowIngestionMetersFactory
+  )
   {
-    Map<Integer, Long> startPartitions = taskGroups.get(groupId).partitionOffsets;
-    Map<Integer, Long> endPartitions = new HashMap<>();
-
-    for (Integer partition : startPartitions.keySet()) {
-      endPartitions.put(partition, Long.MAX_VALUE);
-    }
-    TaskGroup group = taskGroups.get(groupId);
-
-    Map<String, Object> consumerProperties = new HashMap<>(ioConfig.getConsumerProperties());
-    DateTime minimumMessageTime = taskGroups.get(groupId).minimumMessageTime.orNull();
-    DateTime maximumMessageTime = taskGroups.get(groupId).maximumMessageTime.orNull();
-
-    KafkaIOConfig kafkaIOConfig = new KafkaIOConfig(
-        groupId,
-        group.baseSequenceName,
-        new KafkaPartitions(ioConfig.getTopic(), startPartitions),
-        new KafkaPartitions(ioConfig.getTopic(), endPartitions),
-        consumerProperties,
-        true,
-        minimumMessageTime,
-        maximumMessageTime,
-        ioConfig.isSkipOffsetGaps()
+    super(
+        StringUtils.format("KafkaSupervisor-%s", spec.getDataSchema().getDataSource()),
+        taskStorage,
+        taskMaster,
+        indexerMetadataStorageCoordinator,
+        taskClientFactory,
+        mapper,
+        spec,
+        rowIngestionMetersFactory,
+        false
     );
 
-    final String checkpoints = sortingMapper.writerWithType(new TypeReference<TreeMap<Integer, Map<Integer, Long>>>()
-    {
-    }).writeValueAsString(taskGroups.get(groupId).sequenceOffsets);
-    final Map<String, Object> context = spec.getContext() == null
-                                        ? ImmutableMap.of(
-        "checkpoints",
-        checkpoints,
-        IS_INCREMENTAL_HANDOFF_SUPPORTED,
-        true
-    )
-                                        : ImmutableMap.<String, Object>builder()
-                                            .put("checkpoints", checkpoints)
-                                            .put(IS_INCREMENTAL_HANDOFF_SUPPORTED, true)
-                                            .putAll(spec.getContext())
-                                            .build();
-    for (int i = 0; i < replicas; i++) {
-      String taskId = Joiner.on("_").join(group.baseSequenceName, RealtimeIndexTask.makeRandomId());
-      KafkaIndexTask indexTask = new KafkaIndexTask(
-          taskId,
-          new TaskResource(group.baseSequenceName, 1),
-          spec.getDataSchema(),
-          taskTuningConfig,
-          kafkaIOConfig,
-          context,
-          null,
-          null,
-          rowIngestionMetersFactory,
-          sortingMapper
-      );
-
-      Optional<TaskQueue> taskQueue = taskMaster.getTaskQueue();
-      if (taskQueue.isPresent()) {
-        try {
-          taskQueue.get().add(indexTask);
-        }
-        catch (EntryExistsException e) {
-          log.error("Tried to add task [%s] but it already exists", indexTask.getId());
-        }
-      } else {
-        log.error("Failed to get task queue because I'm not the leader!");
-      }
-    }
-  }
-
-  private ImmutableMap<Integer, Long> generateStartingOffsetsForPartitionGroup(int groupId)
-  {
-    ImmutableMap.Builder<Integer, Long> builder = ImmutableMap.builder();
-    for (Entry<Integer, Long> entry : partitionGroups.get(groupId).entrySet()) {
-      Integer partition = entry.getKey();
-      Long offset = entry.getValue();
-
-      if (offset != null && offset != NOT_SET) {
-        // if we are given a startingOffset (set by a previous task group which is pending completion) then use it
-        builder.put(partition, offset);
-      } else {
-        // if we don't have a startingOffset (first run or we had some previous failures and reset the offsets) then
-        // get the offset from metadata storage (if available) or Kafka (otherwise)
-        builder.put(partition, getOffsetFromStorageForPartition(partition));
-      }
-    }
-    return builder.build();
-  }
-
-  /**
-   * Queries the dataSource metadata table to see if there is a previous ending offset for this partition. If it doesn't
-   * find any data, it will retrieve the latest or earliest Kafka offset depending on the useEarliestOffset config.
-   */
-  private long getOffsetFromStorageForPartition(int partition)
-  {
-    long offset;
-    final Map<Integer, Long> metadataOffsets = getOffsetsFromMetadataStorage();
-    if (metadataOffsets.get(partition) != null) {
-      offset = metadataOffsets.get(partition);
-      log.debug("Getting offset [%,d] from metadata storage for partition [%d]", offset, partition);
-
-      long latestKafkaOffset = getOffsetFromKafkaForPartition(partition, false);
-      if (offset > latestKafkaOffset) {
-        throw new ISE(
-            "Offset in metadata storage [%,d] > latest Kafka offset [%,d] for partition[%d] dataSource[%s]. If these "
-            + "messages are no longer available (perhaps you deleted and re-created your Kafka topic) you can use the "
-            + "supervisor reset API to restart ingestion.",
-            offset,
-            latestKafkaOffset,
-            partition,
-            dataSource
-        );
-      }
-
-    } else {
-      offset = getOffsetFromKafkaForPartition(partition, ioConfig.isUseEarliestOffset());
-      log.debug("Getting offset [%,d] from Kafka for partition [%d]", offset, partition);
-    }
-
-    return offset;
-  }
-
-  private Map<Integer, Long> getOffsetsFromMetadataStorage()
-  {
-    final DataSourceMetadata dataSourceMetadata = indexerMetadataStorageCoordinator.getDataSourceMetadata(dataSource);
-    if (dataSourceMetadata instanceof KafkaDataSourceMetadata) {
-      KafkaPartitions partitions = ((KafkaDataSourceMetadata) dataSourceMetadata).getKafkaPartitions();
-      if (partitions != null) {
-        if (!ioConfig.getTopic().equals(partitions.getTopic())) {
-          log.warn(
-              "Topic in metadata storage [%s] doesn't match spec topic [%s], ignoring stored offsets",
-              partitions.getTopic(),
-              ioConfig.getTopic()
-          );
-          return Collections.emptyMap();
-        } else if (partitions.getPartitionOffsetMap() != null) {
-          return partitions.getPartitionOffsetMap();
-        }
-      }
-    }
-
-    return Collections.emptyMap();
+    this.spec = spec;
+    this.emitter = spec.getEmitter();
+    this.monitorSchedulerConfig = spec.getMonitorSchedulerConfig();
   }
 
-  private long getOffsetFromKafkaForPartition(int partition, boolean useEarliestOffset)
-  {
-    synchronized (consumerLock) {
-      TopicPartition topicPartition = new TopicPartition(ioConfig.getTopic(), partition);
-      if (!consumer.assignment().contains(topicPartition)) {
-        consumer.assign(Collections.singletonList(topicPartition));
-      }
-
-      if (useEarliestOffset) {
-        consumer.seekToBeginning(Collections.singletonList(topicPartition));
-      } else {
-        consumer.seekToEnd(Collections.singletonList(topicPartition));
-      }
-
-      return consumer.position(topicPartition);
-    }
-  }
 
-  /**
-   * Compares the sequence name from the task with one generated for the task's group ID and returns false if they do
-   * not match. The sequence name is generated from a hash of the dataSchema, tuningConfig, starting offsets, and the
-   * minimumMessageTime or maximumMessageTime if set.
-   */
-  private boolean isTaskCurrent(int taskGroupId, String taskId)
+  @Override
+  protected RecordSupplier<Integer, Long> setupRecordSupplier()
   {
-    Optional<Task> taskOptional = taskStorage.getTask(taskId);
-    if (!taskOptional.isPresent() || !(taskOptional.get() instanceof KafkaIndexTask)) {
-      return false;
-    }
-
-    String taskSequenceName = ((KafkaIndexTask) taskOptional.get()).getIOConfig().getBaseSequenceName();
-    if (taskGroups.get(taskGroupId) != null) {
-      return Preconditions
-          .checkNotNull(taskGroups.get(taskGroupId), "null taskGroup for taskId[%s]", taskGroupId)
-          .baseSequenceName
-          .equals(taskSequenceName);
-    } else {
-      return generateSequenceName(
-          ((KafkaIndexTask) taskOptional.get()).getIOConfig()
-                                               .getStartPartitions()
-                                               .getPartitionOffsetMap(),
-          ((KafkaIndexTask) taskOptional.get()).getIOConfig().getMinimumMessageTime(),
-          ((KafkaIndexTask) taskOptional.get()).getIOConfig().getMaximumMessageTime()
-      ).equals(taskSequenceName);
-    }
+    return new KafkaRecordSupplier(spec.getIoConfig().getConsumerProperties(), sortingMapper);
   }
 
-  private ListenableFuture<?> stopTasksInGroup(@Nullable TaskGroup taskGroup, String stopReasonFormat, Object... args)
-  {
-    if (taskGroup == null) {
-      return Futures.immediateFuture(null);
-    }
-
-    log.info(
-        "Stopping all tasks in taskGroup[%s] because: [%s]",
-        taskGroup.groupId,
-        StringUtils.format(stopReasonFormat, args)
+  @Override
+  protected void scheduleReporting(ScheduledExecutorService reportingExec)
+  {
+    KafkaSupervisorIOConfig ioConfig = spec.getIoConfig();
+    KafkaSupervisorTuningConfig tuningConfig = spec.getTuningConfig();
+    reportingExec.scheduleAtFixedRate(
+        updateCurrentAndLatestOffsets(),
+        ioConfig.getStartDelay().getMillis() + INITIAL_GET_OFFSET_DELAY_MILLIS, // wait for tasks to start up
+        Math.max(
+            tuningConfig.getOffsetFetchPeriod().getMillis(), MINIMUM_GET_OFFSET_PERIOD_MILLIS
+        ),
+        TimeUnit.MILLISECONDS
     );
 
-    final List<ListenableFuture<Void>> futures = new ArrayList<>();
-    for (Entry<String, TaskData> entry : taskGroup.tasks.entrySet()) {
-      final String taskId = entry.getKey();
-      final TaskData taskData = entry.getValue();
-      if (taskData.status == null) {
-        killTask(taskId, "Killing task since task status is not known to supervisor");
-      } else if (!taskData.status.isComplete()) {
-        futures.add(stopTask(taskId, false));
-      }
-    }
-
-    return Futures.successfulAsList(futures);
-  }
-
-  private ListenableFuture<Void> stopTask(final String id, final boolean publish)
-  {
-    return Futures.transform(
-        taskClient.stopAsync(id, publish), new Function<Boolean, Void>()
-        {
-          @Nullable
-          @Override
-          public Void apply(@Nullable Boolean result)
-          {
-            if (result == null || !result) {
-              killTask(id, "Task [%s] failed to stop in a timely manner, killing task", id);
-            }
-            return null;
-          }
-        }
+    reportingExec.scheduleAtFixedRate(
+        emitLag(),
+        ioConfig.getStartDelay().getMillis() + INITIAL_EMIT_LAG_METRIC_DELAY_MILLIS, // wait for tasks to start up
+        monitorSchedulerConfig.getEmitterPeriod().getMillis(),
+        TimeUnit.MILLISECONDS
     );
   }
 
-  private void killTask(final String id, String reasonFormat, Object... args)
+
+  @Override
+  protected int getTaskGroupIdForPartition(Integer partition)
   {
-    Optional<TaskQueue> taskQueue = taskMaster.getTaskQueue();
-    if (taskQueue.isPresent()) {
-      taskQueue.get().shutdown(id, reasonFormat, args);
-    } else {
-      log.error("Failed to get task queue because I'm not the leader!");
-    }
+    return partition % spec.getIoConfig().getTaskCount();
   }
 
-  protected int getTaskGroupIdForPartition(int partition)
+  @Override
+  protected boolean checkSourceMetadataMatch(DataSourceMetadata metadata)
   {
-    return partition % ioConfig.getTaskCount();
+    return metadata instanceof KafkaDataSourceMetadata;
   }
 
-  private boolean isTaskInPendingCompletionGroups(String taskId)
+  @Override
+  protected boolean doesTaskTypeMatchSupervisor(Task task)
   {
-    for (List<TaskGroup> taskGroups : pendingCompletionTaskGroups.values()) {
-      for (TaskGroup taskGroup : taskGroups) {
-        if (taskGroup.tasks.containsKey(taskId)) {
-          return true;
-        }
-      }
-    }
-    return false;
+    return task instanceof KafkaIndexTask;
   }
 
-  private SupervisorReport<KafkaSupervisorReportPayload> generateReport(boolean includeOffsets)
+  @Override
+  protected SeekableStreamSupervisorReportPayload<Integer, Long> createReportPayload(
+      int numPartitions,
+      boolean includeOffsets
+  )
   {
-    int numPartitions = partitionGroups.values().stream().mapToInt(Map::size).sum();
-
+    KafkaSupervisorIOConfig ioConfig = spec.getIoConfig();
     Map<Integer, Long> partitionLag = getLagPerPartition(getHighestCurrentOffsets());
-    final KafkaSupervisorReportPayload payload = new KafkaSupervisorReportPayload(
-        dataSource,
+    return new KafkaSupervisorReportPayload(
+        spec.getDataSchema().getDataSource(),
         ioConfig.getTopic(),
         numPartitions,
         ioConfig.getReplicas(),
         ioConfig.getTaskDuration().getMillis() / 1000,
-        includeOffsets ? latestOffsetsFromKafka : null,
+        includeOffsets ? latestSequenceFromStream : null,
         includeOffsets ? partitionLag : null,
         includeOffsets ? partitionLag.values().stream().mapToLong(x -> Math.max(x, 0)).sum() : null,
-        includeOffsets ? offsetsLastUpdated : null,
+        includeOffsets ? sequenceLastUpdated : null,
         spec.isSuspended()
     );
-    SupervisorReport<KafkaSupervisorReportPayload> report = new SupervisorReport<>(
-        dataSource,
-        DateTimes.nowUtc(),
-        payload
-    );
-
-    List<TaskReportData> taskReports = new ArrayList<>();
-
-    try {
-      for (TaskGroup taskGroup : taskGroups.values()) {
-        for (Entry<String, TaskData> entry : taskGroup.tasks.entrySet()) {
-          String taskId = entry.getKey();
-          @Nullable
-          DateTime startTime = entry.getValue().startTime;
-          Map<Integer, Long> currentOffsets = entry.getValue().currentOffsets;
-          Long remainingSeconds = null;
-          if (startTime != null) {
-            long elapsedMillis = System.currentTimeMillis() - startTime.getMillis();
-            long remainingMillis = Math.max(0, ioConfig.getTaskDuration().getMillis() - elapsedMillis);
-            remainingSeconds = TimeUnit.MILLISECONDS.toSeconds(remainingMillis);
-          }
-
-          taskReports.add(
-              new TaskReportData(
-                  taskId,
-                  includeOffsets ? taskGroup.partitionOffsets : null,
-                  includeOffsets ? currentOffsets : null,
-                  startTime,
-                  remainingSeconds,
-                  TaskReportData.TaskType.ACTIVE,
-                  includeOffsets ? getLagPerPartition(currentOffsets) : null
-              )
-          );
-        }
-      }
-
-      for (List<TaskGroup> taskGroups : pendingCompletionTaskGroups.values()) {
-        for (TaskGroup taskGroup : taskGroups) {
-          for (Entry<String, TaskData> entry : taskGroup.tasks.entrySet()) {
-            String taskId = entry.getKey();
-            @Nullable
-            DateTime startTime = entry.getValue().startTime;
-            Map<Integer, Long> currentOffsets = entry.getValue().currentOffsets;
-            Long remainingSeconds = null;
-            if (taskGroup.completionTimeout != null) {
-              remainingSeconds = Math.max(0, taskGroup.completionTimeout.getMillis() - System.currentTimeMillis())
-                                 / 1000;
-            }
-
-            taskReports.add(
-                new TaskReportData(
-                    taskId,
-                    includeOffsets ? taskGroup.partitionOffsets : null,
-                    includeOffsets ? currentOffsets : null,
-                    startTime,
-                    remainingSeconds,
-                    TaskReportData.TaskType.PUBLISHING,
-                    null
-                )
-            );
-          }
-        }
-      }
-
-      taskReports.forEach(payload::addTask);
-    }
-    catch (Exception e) {
-      log.warn(e, "Failed to generate status report");
-    }
-
-    return report;
   }
 
-  private Runnable buildRunTask()
+
+  @Override
+  protected SeekableStreamIndexTaskIOConfig createTaskIoConfig(
+      int groupId,
+      Map<Integer, Long> startPartitions,
+      Map<Integer, Long> endPartitions,
+      String baseSequenceName,
+      DateTime minimumMessageTime,
+      DateTime maximumMessageTime,
+      Set<Integer> exclusiveStartSequenceNumberPartitions,
+      SeekableStreamSupervisorIOConfig ioConfig
+  )
   {
-    return () -> notices.add(new RunNotice());
+    KafkaSupervisorIOConfig kafkaIoConfig = (KafkaSupervisorIOConfig) ioConfig;
+    return new KafkaIndexTaskIOConfig(
+        groupId,
+        baseSequenceName,
+        new SeekableStreamPartitions<>(kafkaIoConfig.getTopic(), startPartitions),
+        new SeekableStreamPartitions<>(kafkaIoConfig.getTopic(), endPartitions),
+        kafkaIoConfig.getConsumerProperties(),
+        true,
+        minimumMessageTime,
+        maximumMessageTime,
+        kafkaIoConfig.isSkipOffsetGaps()
+    );
   }
 
-  private void updateLatestOffsetsFromKafka()
+  @Override
+  protected List<SeekableStreamIndexTask<Integer, Long>> createIndexTasks(
+      int replicas,
+      String baseSequenceName,
+      ObjectMapper sortingMapper,
+      TreeMap<Integer, Map<Integer, Long>> sequenceOffsets,
+      SeekableStreamIndexTaskIOConfig taskIoConfig,
+      SeekableStreamIndexTaskTuningConfig taskTuningConfig,
+      RowIngestionMetersFactory rowIngestionMetersFactory
+  ) throws JsonProcessingException
   {
-    synchronized (consumerLock) {
-      final List<PartitionInfo> partitionInfoList = consumer.partitionsFor(ioConfig.getTopic());
-
-      if (partitionInfoList == null || partitionInfoList.size() == 0) {
-        throw new ISE("Could not retrieve partitions for topic [%s]", ioConfig.getTopic());
-      }
-
-      final Set<TopicPartition> topicPartitions = partitionInfoList
-          .stream()
-          .map(x -> new TopicPartition(x.topic(), x.partition()))
-          .collect(Collectors.toSet());
-
-      consumer.assign(topicPartitions);
-
-      final Map<TopicPartition, Long> endOffsets = consumer.endOffsets(topicPartitions);
+    final String checkpoints = sortingMapper.writerWithType(new TypeReference<TreeMap<Integer, Map<Integer, Long>>>()
+    {
+    }).writeValueAsString(sequenceOffsets);
+    final Map<String, Object> context = spec.getContext() == null
+                                        ? ImmutableMap.of(
+        "checkpoints",
+        checkpoints,
+        IS_INCREMENTAL_HANDOFF_SUPPORTED,
+        true
+    ) : ImmutableMap.<String, Object>builder()
+                                            .put("checkpoints", checkpoints)
+                                            .put(IS_INCREMENTAL_HANDOFF_SUPPORTED, true)
+                                            .putAll(spec.getContext())
+                                            .build();
 
-      latestOffsetsFromKafka = endOffsets
-          .entrySet()
-          .stream()
-          .collect(Collectors.toMap(e -> e.getKey().partition(), Map.Entry::getValue));
+    List<SeekableStreamIndexTask<Integer, Long>> taskList = new ArrayList<>();
+    for (int i = 0; i < replicas; i++) {
+      String taskId = Joiner.on("_").join(baseSequenceName, RandomIdUtils.getRandomId());
+      taskList.add(new KafkaIndexTask(
+          taskId,
+          new TaskResource(baseSequenceName, 1),
+          spec.getDataSchema(),
+          (KafkaIndexTaskTuningConfig) taskTuningConfig,
+          (KafkaIndexTaskIOConfig) taskIoConfig,
+          context,
+          null,
+          null,
+          rowIngestionMetersFactory,
+          sortingMapper
+      ));
     }
+    return taskList;
   }
 
-  private Map<Integer, Long> getHighestCurrentOffsets()
-  {
-    return taskGroups
-        .values()
-        .stream()
-        .flatMap(taskGroup -> taskGroup.tasks.entrySet().stream())
-        .flatMap(taskData -> taskData.getValue().currentOffsets.entrySet().stream())
-        .collect(Collectors.toMap(Entry::getKey, Entry::getValue, Long::max));
-  }
 
-  private Map<Integer, Long> getLagPerPartition(Map<Integer, Long> currentOffsets)
+  @Override
+  protected Map<Integer, Long> getLagPerPartition(Map<Integer, Long> currentOffsets)
   {
     return currentOffsets
         .entrySet()
@@ -2348,29 +271,42 @@ private void updateLatestOffsetsFromKafka()
         .collect(
             Collectors.toMap(
                 Entry::getKey,
-                e -> latestOffsetsFromKafka != null
-                     && latestOffsetsFromKafka.get(e.getKey()) != null
+                e -> latestSequenceFromStream != null
+                     && latestSequenceFromStream.get(e.getKey()) != null
                      && e.getValue() != null
-                     ? latestOffsetsFromKafka.get(e.getKey()) - e.getValue()
+                     ? latestSequenceFromStream.get(e.getKey()) - e.getValue()
                      : Integer.MIN_VALUE
             )
         );
   }
 
+  @Override
+  protected KafkaDataSourceMetadata createDataSourceMetaData(String topic, Map<Integer, Long> map)
+  {
+    return new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, map));
+  }
+
+  @Override
+  protected OrderedSequenceNumber<Long> makeSequenceNumber(Long seq, boolean isExclusive)
+  {
+    return KafkaSequenceNumber.of(seq);
+  }
+
   private Runnable emitLag()
   {
     return () -> {
       try {
         Map<Integer, Long> highestCurrentOffsets = getHighestCurrentOffsets();
+        String dataSource = spec.getDataSchema().getDataSource();
 
-        if (latestOffsetsFromKafka == null) {
+        if (latestSequenceFromStream == null) {
           throw new ISE("Latest offsets from Kafka have not been fetched");
         }
 
-        if (!latestOffsetsFromKafka.keySet().equals(highestCurrentOffsets.keySet())) {
+        if (!latestSequenceFromStream.keySet().equals(highestCurrentOffsets.keySet())) {
           log.warn(
               "Lag metric: Kafka partitions %s do not match task partitions %s",
-              latestOffsetsFromKafka.keySet(),
+              latestSequenceFromStream.keySet(),
               highestCurrentOffsets.keySet()
           );
         }
@@ -2401,239 +337,47 @@ private Runnable emitLag()
     };
   }
 
-  private void updateCurrentOffsets() throws InterruptedException, ExecutionException, TimeoutException
-  {
-    final List<ListenableFuture<Void>> futures = Stream.concat(
-        taskGroups.values().stream().flatMap(taskGroup -> taskGroup.tasks.entrySet().stream()),
-        pendingCompletionTaskGroups.values()
-                                   .stream()
-                                   .flatMap(List::stream)
-                                   .flatMap(taskGroup -> taskGroup.tasks.entrySet().stream())
-    ).map(
-        task -> Futures.transform(
-            taskClient.getCurrentOffsetsAsync(task.getKey(), false),
-            (Function<Map<Integer, Long>, Void>) (currentOffsets) -> {
-
-              if (currentOffsets != null && !currentOffsets.isEmpty()) {
-                task.getValue().currentOffsets = currentOffsets;
-              }
-
-              return null;
-            }
-        )
-    ).collect(Collectors.toList());
-
-    Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS);
-  }
-
-  @VisibleForTesting
-  Runnable updateCurrentAndLatestOffsets()
+  @Override
+  protected Long getNotSetMarker()
   {
-    return () -> {
-      try {
-        updateCurrentOffsets();
-        updateLatestOffsetsFromKafka();
-        offsetsLastUpdated = DateTimes.nowUtc();
-      }
-      catch (Exception e) {
-        log.warn(e, "Exception while getting current/latest offsets");
-      }
-    };
+    return NOT_SET;
   }
 
-  /**
-   * Collect row ingestion stats from all tasks managed by this supervisor.
-   *
-   * @return A map of groupId->taskId->task row stats
-   *
-   * @throws InterruptedException
-   * @throws ExecutionException
-   * @throws TimeoutException
-   */
-  private Map<String, Map<String, Object>> getCurrentTotalStats()
-      throws InterruptedException, ExecutionException, TimeoutException
+  @Override
+  protected Long getEndOfPartitionMarker()
   {
-    Map<String, Map<String, Object>> allStats = new HashMap<>();
-    final List<ListenableFuture<StatsFromTaskResult>> futures = new ArrayList<>();
-    final List<Pair<Integer, String>> groupAndTaskIds = new ArrayList<>();
-
-    for (int groupId : taskGroups.keySet()) {
-      TaskGroup group = taskGroups.get(groupId);
-      for (String taskId : group.taskIds()) {
-        futures.add(
-            Futures.transform(
-                taskClient.getMovingAveragesAsync(taskId),
-                (Function<Map<String, Object>, StatsFromTaskResult>) (currentStats) -> {
-                  return new StatsFromTaskResult(
-                      groupId,
-                      taskId,
-                      currentStats
-                  );
-                }
-            )
-        );
-        groupAndTaskIds.add(new Pair<>(groupId, taskId));
-      }
-    }
-
-    for (int groupId : pendingCompletionTaskGroups.keySet()) {
-      List<TaskGroup> pendingGroups = pendingCompletionTaskGroups.get(groupId);
-      for (TaskGroup pendingGroup : pendingGroups) {
-        for (String taskId : pendingGroup.taskIds()) {
-          futures.add(
-              Futures.transform(
-                  taskClient.getMovingAveragesAsync(taskId),
-                  (Function<Map<String, Object>, StatsFromTaskResult>) (currentStats) -> new StatsFromTaskResult(
-                      groupId,
-                      taskId,
-                      currentStats
-                  )
-              )
-          );
-          groupAndTaskIds.add(new Pair<>(groupId, taskId));
-        }
-      }
-    }
-
-    List<StatsFromTaskResult> results = Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS);
-    for (int i = 0; i < results.size(); i++) {
-      StatsFromTaskResult result = results.get(i);
-      if (result != null) {
-        Map<String, Object> groupMap = allStats.computeIfAbsent(result.getGroupId(), k -> new HashMap<>());
-        groupMap.put(result.getTaskId(), result.getStats());
-      } else {
-        Pair<Integer, String> groupAndTaskId = groupAndTaskIds.get(i);
-        log.error("Failed to get stats for group[%d]-task[%s]", groupAndTaskId.lhs, groupAndTaskId.rhs);
-      }
-    }
-
-    return allStats;
+    return END_OF_PARTITION;
   }
 
-  @VisibleForTesting
-  void addTaskGroupToActivelyReadingTaskGroup(
-      int taskGroupId,
-      ImmutableMap<Integer, Long> partitionOffsets,
-      Optional<DateTime> minMsgTime,
-      Optional<DateTime> maxMsgTime,
-      Set<String> tasks
-  )
+  @Override
+  protected boolean isEndOfShard(Long seqNum)
   {
-    TaskGroup group = new TaskGroup(
-        taskGroupId,
-        partitionOffsets,
-        minMsgTime,
-        maxMsgTime
-    );
-    group.tasks.putAll(tasks.stream().collect(Collectors.toMap(x -> x, x -> new TaskData())));
-    if (taskGroups.putIfAbsent(taskGroupId, group) != null) {
-      throw new ISE(
-          "trying to add taskGroup with ID [%s] to actively reading task groups, but group already exists.",
-          taskGroupId
-      );
-    }
+    return false;
   }
 
-  @VisibleForTesting
-  void addTaskGroupToPendingCompletionTaskGroup(
-      int taskGroupId,
-      ImmutableMap<Integer, Long> partitionOffsets,
-      Optional<DateTime> minMsgTime,
-      Optional<DateTime> maxMsgTime,
-      Set<String> tasks
+  @Override
+  protected void updateLatestSequenceFromStream(
+      RecordSupplier<Integer, Long> recordSupplier,
+      Set<StreamPartition<Integer>> partitions
   )
   {
-    TaskGroup group = new TaskGroup(
-        taskGroupId,
-        partitionOffsets,
-        minMsgTime,
-        maxMsgTime
-    );
-    group.tasks.putAll(tasks.stream().collect(Collectors.toMap(x -> x, x -> new TaskData())));
-    pendingCompletionTaskGroups.computeIfAbsent(taskGroupId, x -> new CopyOnWriteArrayList<>())
-                               .add(group);
-  }
-
-  @VisibleForTesting
-  @Nullable
-  TaskGroup removeTaskGroup(int taskGroupId)
-  {
-    return taskGroups.remove(taskGroupId);
-  }
-
-  @VisibleForTesting
-  void moveTaskGroupToPendingCompletion(int taskGroupId)
-  {
-    final TaskGroup taskGroup = taskGroups.remove(taskGroupId);
-    if (taskGroup != null) {
-      pendingCompletionTaskGroups.computeIfAbsent(taskGroupId, k -> new CopyOnWriteArrayList<>()).add(taskGroup);
-    }
-  }
-
-  @VisibleForTesting
-  int getNoticesQueueSize()
-  {
-    return notices.size();
-  }
-
-  private static class StatsFromTaskResult
-  {
-    private final String groupId;
-    private final String taskId;
-    private final Map<String, Object> stats;
-
-    public StatsFromTaskResult(
-        int groupId,
-        String taskId,
-        Map<String, Object> stats
-    )
-    {
-      this.groupId = String.valueOf(groupId);
-      this.taskId = taskId;
-      this.stats = stats;
-    }
-
-    public String getGroupId()
-    {
-      return groupId;
-    }
-
-    public String getTaskId()
-    {
-      return taskId;
-    }
-
-    public Map<String, Object> getStats()
-    {
-      return stats;
-    }
-  }
-
-  // exposed for testing for visibility into initialization state
-  @VisibleForTesting
-  public boolean isStarted()
-  {
-    return started;
-  }
-
-  // exposed for testing for visibility into initialization state
-  @VisibleForTesting
-  public boolean isLifecycleStarted()
-  {
-    return lifecycleStarted;
+    latestSequenceFromStream = partitions.stream()
+                                         .collect(Collectors.toMap(
+                                             StreamPartition::getPartitionId,
+                                             recordSupplier::getPosition
+                                         ));
   }
 
-  // exposed for testing for visibility into initialization state
-  @VisibleForTesting
-  public int getInitRetryCounter()
+  @Override
+  protected String baseTaskName()
   {
-    return initRetryCounter;
+    return "index_kafka";
   }
 
-  // exposed for testing to allow "bootstrap.servers" to be changed after supervisor is created
+  @Override
   @VisibleForTesting
   public KafkaSupervisorIOConfig getIoConfig()
   {
-    return ioConfig;
+    return spec.getIoConfig();
   }
 }
diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java
index 44c2bb2d6f7..80b842b8349 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java
@@ -21,32 +21,21 @@
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
+import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig;
 import org.apache.druid.java.util.common.StringUtils;
-import org.joda.time.Duration;
 import org.joda.time.Period;
 
 import java.util.Map;
 
-public class KafkaSupervisorIOConfig
+public class KafkaSupervisorIOConfig extends SeekableStreamSupervisorIOConfig
 {
   public static final String BOOTSTRAP_SERVERS_KEY = "bootstrap.servers";
   public static final String TRUST_STORE_PASSWORD_KEY = "ssl.truststore.password";
   public static final String KEY_STORE_PASSWORD_KEY = "ssl.keystore.password";
   public static final String KEY_PASSWORD_KEY = "ssl.key.password";
 
-  private final String topic;
-  private final Integer replicas;
-  private final Integer taskCount;
-  private final Duration taskDuration;
   private final Map<String, Object> consumerProperties;
-  private final Duration startDelay;
-  private final Duration period;
-  private final boolean useEarliestOffset;
-  private final Duration completionTimeout;
-  private final Optional<Duration> lateMessageRejectionPeriod;
-  private final Optional<Duration> earlyMessageRejectionPeriod;
   private final boolean skipOffsetGaps;
 
   @JsonCreator
@@ -65,51 +54,31 @@ public KafkaSupervisorIOConfig(
       @JsonProperty("skipOffsetGaps") Boolean skipOffsetGaps
   )
   {
-    this.topic = Preconditions.checkNotNull(topic, "topic");
+    super(
+        Preconditions.checkNotNull(topic, "topic"),
+        replicas,
+        taskCount,
+        taskDuration,
+        startDelay,
+        period,
+        useEarliestOffset,
+        completionTimeout,
+        lateMessageRejectionPeriod,
+        earlyMessageRejectionPeriod
+    );
+
     this.consumerProperties = Preconditions.checkNotNull(consumerProperties, "consumerProperties");
     Preconditions.checkNotNull(
         consumerProperties.get(BOOTSTRAP_SERVERS_KEY),
         StringUtils.format("consumerProperties must contain entry for [%s]", BOOTSTRAP_SERVERS_KEY)
     );
-
-    this.replicas = replicas != null ? replicas : 1;
-    this.taskCount = taskCount != null ? taskCount : 1;
-    this.taskDuration = defaultDuration(taskDuration, "PT1H");
-    this.startDelay = defaultDuration(startDelay, "PT5S");
-    this.period = defaultDuration(period, "PT30S");
-    this.useEarliestOffset = useEarliestOffset != null ? useEarliestOffset : false;
-    this.completionTimeout = defaultDuration(completionTimeout, "PT30M");
-    this.lateMessageRejectionPeriod = lateMessageRejectionPeriod == null
-                                      ? Optional.absent()
-                                      : Optional.of(lateMessageRejectionPeriod.toStandardDuration());
-    this.earlyMessageRejectionPeriod = earlyMessageRejectionPeriod == null
-                                       ? Optional.absent()
-                                       : Optional.of(earlyMessageRejectionPeriod.toStandardDuration());
     this.skipOffsetGaps = skipOffsetGaps != null ? skipOffsetGaps : false;
   }
 
   @JsonProperty
   public String getTopic()
   {
-    return topic;
-  }
-
-  @JsonProperty
-  public Integer getReplicas()
-  {
-    return replicas;
-  }
-
-  @JsonProperty
-  public Integer getTaskCount()
-  {
-    return taskCount;
-  }
-
-  @JsonProperty
-  public Duration getTaskDuration()
-  {
-    return taskDuration;
+    return getStream();
   }
 
   @JsonProperty
@@ -118,40 +87,10 @@ public Duration getTaskDuration()
     return consumerProperties;
   }
 
-  @JsonProperty
-  public Duration getStartDelay()
-  {
-    return startDelay;
-  }
-
-  @JsonProperty
-  public Duration getPeriod()
-  {
-    return period;
-  }
-
   @JsonProperty
   public boolean isUseEarliestOffset()
   {
-    return useEarliestOffset;
-  }
-
-  @JsonProperty
-  public Duration getCompletionTimeout()
-  {
-    return completionTimeout;
-  }
-
-  @JsonProperty
-  public Optional<Duration> getEarlyMessageRejectionPeriod()
-  {
-    return earlyMessageRejectionPeriod;
-  }
-
-  @JsonProperty
-  public Optional<Duration> getLateMessageRejectionPeriod()
-  {
-    return lateMessageRejectionPeriod;
+    return isUseEarliestSequenceNumber();
   }
 
   @JsonProperty
@@ -164,22 +103,19 @@ public boolean isSkipOffsetGaps()
   public String toString()
   {
     return "KafkaSupervisorIOConfig{" +
-           "topic='" + topic + '\'' +
-           ", replicas=" + replicas +
-           ", taskCount=" + taskCount +
-           ", taskDuration=" + taskDuration +
+           "topic='" + getTopic() + '\'' +
+           ", replicas=" + getReplicas() +
+           ", taskCount=" + getTaskCount() +
+           ", taskDuration=" + getTaskDuration() +
            ", consumerProperties=" + consumerProperties +
-           ", startDelay=" + startDelay +
-           ", period=" + period +
-           ", useEarliestOffset=" + useEarliestOffset +
-           ", completionTimeout=" + completionTimeout +
-           ", lateMessageRejectionPeriod=" + lateMessageRejectionPeriod +
+           ", startDelay=" + getStartDelay() +
+           ", period=" + getPeriod() +
+           ", useEarliestOffset=" + isUseEarliestOffset() +
+           ", completionTimeout=" + getCompletionTimeout() +
+           ", earlyMessageRejectionPeriod=" + getEarlyMessageRejectionPeriod() +
+           ", lateMessageRejectionPeriod=" + getLateMessageRejectionPeriod() +
            ", skipOffsetGaps=" + skipOffsetGaps +
            '}';
   }
 
-  private static Duration defaultDuration(final Period period, final String theDefault)
-  {
-    return (period == null ? new Period(theDefault) : period).toStandardDuration();
-  }
 }
diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java
index d9533a37fb2..d5f4efa02a2 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java
@@ -19,30 +19,14 @@
 
 package org.apache.druid.indexing.kafka.supervisor;
 
-import com.fasterxml.jackson.annotation.JsonInclude;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload;
 import org.joda.time.DateTime;
 
 import javax.annotation.Nullable;
-import java.util.ArrayList;
-import java.util.List;
 import java.util.Map;
 
-public class KafkaSupervisorReportPayload
+public class KafkaSupervisorReportPayload extends SeekableStreamSupervisorReportPayload<Integer, Long>
 {
-  private final String dataSource;
-  private final String topic;
-  private final int partitions;
-  private final int replicas;
-  private final long durationSeconds;
-  private final List<TaskReportData> activeTasks;
-  private final List<TaskReportData> publishingTasks;
-  private final Map<Integer, Long> latestOffsets;
-  private final Map<Integer, Long> minimumLag;
-  private final Long aggregateLag;
-  private final DateTime offsetsLastUpdated;
-  private final boolean suspended;
 
   public KafkaSupervisorReportPayload(
       String dataSource,
@@ -57,122 +41,38 @@ public KafkaSupervisorReportPayload(
       boolean suspended
   )
   {
-    this.dataSource = dataSource;
-    this.topic = topic;
-    this.partitions = partitions;
-    this.replicas = replicas;
-    this.durationSeconds = durationSeconds;
-    this.activeTasks = new ArrayList<>();
-    this.publishingTasks = new ArrayList<>();
-    this.latestOffsets = latestOffsets;
-    this.minimumLag = minimumLag;
-    this.aggregateLag = aggregateLag;
-    this.offsetsLastUpdated = offsetsLastUpdated;
-    this.suspended = suspended;
+    super(
+        dataSource,
+        topic,
+        partitions,
+        replicas,
+        durationSeconds,
+        latestOffsets,
+        minimumLag,
+        aggregateLag,
+        offsetsLastUpdated,
+        suspended
+    );
   }
 
-  public void addTask(TaskReportData data)
-  {
-    if (data.getType().equals(TaskReportData.TaskType.ACTIVE)) {
-      activeTasks.add(data);
-    } else if (data.getType().equals(TaskReportData.TaskType.PUBLISHING)) {
-      publishingTasks.add(data);
-    } else {
-      throw new IAE("Unknown task type [%s]", data.getType().name());
-    }
-  }
-
-  @JsonProperty
-  public String getDataSource()
-  {
-    return dataSource;
-  }
-
-  @JsonProperty
-  public String getTopic()
-  {
-    return topic;
-  }
-
-  @JsonProperty
-  public int getPartitions()
-  {
-    return partitions;
-  }
-
-  @JsonProperty
-  public int getReplicas()
-  {
-    return replicas;
-  }
-
-  @JsonProperty
-  public long getDurationSeconds()
-  {
-    return durationSeconds;
-  }
-
-  @JsonProperty
-  public List<TaskReportData> getActiveTasks()
-  {
-    return activeTasks;
-  }
-
-  @JsonProperty
-  public List<TaskReportData> getPublishingTasks()
-  {
-    return publishingTasks;
-  }
-
-  @JsonProperty
-  @JsonInclude(JsonInclude.Include.NON_NULL)
-  public Map<Integer, Long> getLatestOffsets()
-  {
-    return latestOffsets;
-  }
-
-  @JsonProperty
-  @JsonInclude(JsonInclude.Include.NON_NULL)
-  public Map<Integer, Long> getMinimumLag()
-  {
-    return minimumLag;
-  }
-
-  @JsonProperty
-  @JsonInclude(JsonInclude.Include.NON_NULL)
-  public Long getAggregateLag()
-  {
-    return aggregateLag;
-  }
-
-  @JsonProperty
-  public DateTime getOffsetsLastUpdated()
-  {
-    return offsetsLastUpdated;
-  }
-
-  @JsonProperty
-  public boolean getSuspended()
-  {
-    return suspended;
-  }
 
   @Override
   public String toString()
   {
-    return "{" +
-           "dataSource='" + dataSource + '\'' +
-           ", topic='" + topic + '\'' +
-           ", partitions=" + partitions +
-           ", replicas=" + replicas +
-           ", durationSeconds=" + durationSeconds +
-           ", active=" + activeTasks +
-           ", publishing=" + publishingTasks +
-           (latestOffsets != null ? ", latestOffsets=" + latestOffsets : "") +
-           (minimumLag != null ? ", minimumLag=" + minimumLag : "") +
-           (aggregateLag != null ? ", aggregateLag=" + aggregateLag : "") +
-           (offsetsLastUpdated != null ? ", offsetsLastUpdated=" + offsetsLastUpdated : "") +
-           ", suspended=" + suspended +
+    return "KafkaSupervisorReportPayload{" +
+           "dataSource='" + getDataSource() + '\'' +
+           ", topic='" + getStream() + '\'' +
+           ", partitions=" + getPartitions() +
+           ", replicas=" + getReplicas() +
+           ", durationSeconds=" + getDurationSeconds() +
+           ", active=" + getActiveTasks() +
+           ", publishing=" + getPublishingTasks() +
+           (getLatestOffsets() != null ? ", latestOffsets=" + getLatestOffsets() : "") +
+           (getMinimumLag() != null ? ", minimumLag=" + getMinimumLag() : "") +
+           (getAggregateLag() != null ? ", aggregateLag=" + getAggregateLag() : "") +
+           (getOffsetsLastUpdated() != null ? ", sequenceLastUpdated=" + getOffsetsLastUpdated() : "") +
+           ", suspended=" + getSuspended() +
            '}';
   }
+
 }
diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java
index 28dab02c56e..33ad3e11bee 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java
@@ -23,8 +23,6 @@
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableList;
 import org.apache.druid.guice.annotations.Json;
 import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory;
 import org.apache.druid.indexing.kafka.KafkaIndexTaskClientFactory;
@@ -32,30 +30,15 @@
 import org.apache.druid.indexing.overlord.TaskMaster;
 import org.apache.druid.indexing.overlord.TaskStorage;
 import org.apache.druid.indexing.overlord.supervisor.Supervisor;
-import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec;
+import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorSpec;
 import org.apache.druid.java.util.emitter.service.ServiceEmitter;
 import org.apache.druid.segment.indexing.DataSchema;
 import org.apache.druid.server.metrics.DruidMonitorSchedulerConfig;
 
-import java.util.List;
 import java.util.Map;
 
-public class KafkaSupervisorSpec implements SupervisorSpec
+public class KafkaSupervisorSpec extends SeekableStreamSupervisorSpec
 {
-  private final DataSchema dataSchema;
-  private final KafkaSupervisorTuningConfig tuningConfig;
-  private final KafkaSupervisorIOConfig ioConfig;
-  private final Map<String, Object> context;
-
-  private final TaskStorage taskStorage;
-  private final TaskMaster taskMaster;
-  private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator;
-  private final KafkaIndexTaskClientFactory kafkaIndexTaskClientFactory;
-  private final ObjectMapper mapper;
-  private final ServiceEmitter emitter;
-  private final DruidMonitorSchedulerConfig monitorSchedulerConfig;
-  private final RowIngestionMetersFactory rowIngestionMetersFactory;
-  private final boolean suspended;
 
   @JsonCreator
   public KafkaSupervisorSpec(
@@ -74,93 +57,47 @@ public KafkaSupervisorSpec(
       @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory
   )
   {
-    this.dataSchema = Preconditions.checkNotNull(dataSchema, "dataSchema");
-    this.tuningConfig = tuningConfig != null
-                        ? tuningConfig
-                        : new KafkaSupervisorTuningConfig(
-                            null,
-                            null,
-                            null,
-                            null,
-                            null,
-                            null,
-                            null,
-                            null,
-                            null,
-                            null,
-                            null,
-                            null,
-                            null,
-                            null,
-                            null,
-                            null,
-                            null,
-                            null,
-                            null,
-                            null,
-                            null,
-                            null,
-                            null
-                        );
-    this.ioConfig = Preconditions.checkNotNull(ioConfig, "ioConfig");
-    this.context = context;
-
-    this.taskStorage = taskStorage;
-    this.taskMaster = taskMaster;
-    this.indexerMetadataStorageCoordinator = indexerMetadataStorageCoordinator;
-    this.kafkaIndexTaskClientFactory = kafkaIndexTaskClientFactory;
-    this.mapper = mapper;
-    this.emitter = emitter;
-    this.monitorSchedulerConfig = monitorSchedulerConfig;
-    this.rowIngestionMetersFactory = rowIngestionMetersFactory;
-    this.suspended = suspended != null ? suspended : false;
-  }
-
-  @JsonProperty
-  public DataSchema getDataSchema()
-  {
-    return dataSchema;
-  }
-
-  @JsonProperty
-  public KafkaSupervisorTuningConfig getTuningConfig()
-  {
-    return tuningConfig;
-  }
-
-  @JsonProperty
-  public KafkaSupervisorIOConfig getIoConfig()
-  {
-    return ioConfig;
-  }
-
-  @JsonProperty
-  public Map<String, Object> getContext()
-  {
-    return context;
-  }
-
-  @Override
-  @JsonProperty("suspended")
-  public boolean isSuspended()
-  {
-    return suspended;
-  }
-
-  public ServiceEmitter getEmitter()
-  {
-    return emitter;
-  }
-
-  @Override
-  public String getId()
-  {
-    return dataSchema.getDataSource();
-  }
-
-  public DruidMonitorSchedulerConfig getMonitorSchedulerConfig()
-  {
-    return monitorSchedulerConfig;
+    super(
+        dataSchema,
+        tuningConfig != null
+        ? tuningConfig
+        : new KafkaSupervisorTuningConfig(
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null
+        ),
+        ioConfig,
+        context,
+        suspended,
+        taskStorage,
+        taskMaster,
+        indexerMetadataStorageCoordinator,
+        kafkaIndexTaskClientFactory,
+        mapper,
+        emitter,
+        monitorSchedulerConfig,
+        rowIngestionMetersFactory
+    );
   }
 
   @Override
@@ -170,7 +107,7 @@ public Supervisor createSupervisor()
         taskStorage,
         taskMaster,
         indexerMetadataStorageCoordinator,
-        kafkaIndexTaskClientFactory,
+        (KafkaIndexTaskClientFactory) indexTaskClientFactory,
         mapper,
         this,
         rowIngestionMetersFactory
@@ -178,49 +115,48 @@ public Supervisor createSupervisor()
   }
 
   @Override
-  public List<String> getDataSources()
-  {
-    return ImmutableList.of(getDataSchema().getDataSource());
-  }
-
-  @Override
-  public String toString()
+  @JsonProperty
+  public KafkaSupervisorTuningConfig getTuningConfig()
   {
-    return "KafkaSupervisorSpec{" +
-           "dataSchema=" + dataSchema +
-           ", tuningConfig=" + tuningConfig +
-           ", ioConfig=" + ioConfig +
-           '}';
+    return (KafkaSupervisorTuningConfig) super.getTuningConfig();
   }
 
   @Override
-  public KafkaSupervisorSpec createSuspendedSpec()
+  @JsonProperty
+  public KafkaSupervisorIOConfig getIoConfig()
   {
-    return toggleSuspend(true);
+    return (KafkaSupervisorIOConfig) super.getIoConfig();
   }
 
   @Override
-  public KafkaSupervisorSpec createRunningSpec()
-  {
-    return toggleSuspend(false);
-  }
-
-  private KafkaSupervisorSpec toggleSuspend(boolean suspend)
+  protected KafkaSupervisorSpec toggleSuspend(boolean suspend)
   {
     return new KafkaSupervisorSpec(
-        dataSchema,
-        tuningConfig,
-        ioConfig,
-        context,
+        getDataSchema(),
+        getTuningConfig(),
+        getIoConfig(),
+        getContext(),
         suspend,
         taskStorage,
         taskMaster,
         indexerMetadataStorageCoordinator,
-        kafkaIndexTaskClientFactory,
+        (KafkaIndexTaskClientFactory) indexTaskClientFactory,
         mapper,
         emitter,
         monitorSchedulerConfig,
         rowIngestionMetersFactory
     );
   }
+
+  @Override
+  public String toString()
+  {
+    return "KafkaSupervisorSpec{" +
+           "dataSchema=" + getDataSchema() +
+           ", tuningConfig=" + getTuningConfig() +
+           ", ioConfig=" + getIoConfig() +
+           ", context=" + getContext() +
+           ", suspend=" + isSuspended() +
+           '}';
+  }
 }
diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java
index 4715f336f75..8e4c6e9f913 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java
@@ -20,7 +20,9 @@
 package org.apache.druid.indexing.kafka.supervisor;
 
 import com.fasterxml.jackson.annotation.JsonProperty;
-import org.apache.druid.indexing.kafka.KafkaTuningConfig;
+import org.apache.druid.indexing.kafka.KafkaIndexTaskTuningConfig;
+import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig;
+import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorTuningConfig;
 import org.apache.druid.segment.IndexSpec;
 import org.apache.druid.segment.indexing.TuningConfigs;
 import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory;
@@ -30,8 +32,11 @@
 import javax.annotation.Nullable;
 import java.io.File;
 
-public class KafkaSupervisorTuningConfig extends KafkaTuningConfig
+public class KafkaSupervisorTuningConfig extends KafkaIndexTaskTuningConfig
+    implements SeekableStreamSupervisorTuningConfig
 {
+  private static final String DEFAULT_OFFSET_FETCH_PERIOD = "PT30S";
+
   private final Integer workerThreads;
   private final Integer chatThreads;
   private final Long chatRetries;
@@ -85,39 +90,49 @@ public KafkaSupervisorTuningConfig(
         maxParseExceptions,
         maxSavedParseExceptions
     );
-
     this.workerThreads = workerThreads;
     this.chatThreads = chatThreads;
-    this.chatRetries = (chatRetries != null ? chatRetries : 8);
-    this.httpTimeout = defaultDuration(httpTimeout, "PT10S");
-    this.shutdownTimeout = defaultDuration(shutdownTimeout, "PT80S");
-    this.offsetFetchPeriod = defaultDuration(offsetFetchPeriod, "PT30S");
+    this.chatRetries = (chatRetries != null ? chatRetries : DEFAULT_CHAT_RETRIES);
+    this.httpTimeout = SeekableStreamSupervisorTuningConfig.defaultDuration(httpTimeout, DEFAULT_HTTP_TIMEOUT);
+    this.shutdownTimeout = SeekableStreamSupervisorTuningConfig.defaultDuration(
+        shutdownTimeout,
+        DEFAULT_SHUTDOWN_TIMEOUT
+    );
+    this.offsetFetchPeriod = SeekableStreamSupervisorTuningConfig.defaultDuration(
+        offsetFetchPeriod,
+        DEFAULT_OFFSET_FETCH_PERIOD
+    );
   }
 
+  @Override
   @JsonProperty
   public Integer getWorkerThreads()
   {
     return workerThreads;
   }
 
+  @Override
   @JsonProperty
   public Integer getChatThreads()
   {
     return chatThreads;
   }
 
+  @Override
   @JsonProperty
   public Long getChatRetries()
   {
     return chatRetries;
   }
 
+  @Override
   @JsonProperty
   public Duration getHttpTimeout()
   {
     return httpTimeout;
   }
 
+  @Override
   @JsonProperty
   public Duration getShutdownTimeout()
   {
@@ -159,8 +174,27 @@ public String toString()
            '}';
   }
 
-  private static Duration defaultDuration(final Period period, final String theDefault)
+  @Override
+  public SeekableStreamIndexTaskTuningConfig convertToTaskTuningConfig()
   {
-    return (period == null ? new Period(theDefault) : period).toStandardDuration();
+    return new KafkaIndexTaskTuningConfig(
+        getMaxRowsInMemory(),
+        getMaxBytesInMemory(),
+        getMaxRowsPerSegment(),
+        getMaxTotalRows(),
+        getIntermediatePersistPeriod(),
+        getBasePersistDirectory(),
+        getMaxPendingPersists(),
+        getIndexSpec(),
+        true,
+        isReportParseExceptions(),
+        getHandoffConditionTimeout(),
+        isResetOffsetAutomatically(),
+        getSegmentWriteOutMediumFactory(),
+        getIntermediateHandoffPeriod(),
+        isLogParseExceptions(),
+        getMaxParseExceptions(),
+        getMaxSavedParseExceptions()
+    );
   }
 }
diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadataTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadataTest.java
index 51afdeb8ea4..89f5ce8b0e7 100644
--- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadataTest.java
+++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadataTest.java
@@ -20,6 +20,7 @@
 package org.apache.druid.indexing.kafka;
 
 import com.google.common.collect.ImmutableMap;
+import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -125,6 +126,6 @@ public void testMinus()
 
   private static KafkaDataSourceMetadata KM(String topic, Map<Integer, Long> offsets)
   {
-    return new KafkaDataSourceMetadata(new KafkaPartitions(topic, offsets));
+    return new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, offsets));
   }
 }
diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIOConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIOConfigTest.java
index 2321974ff38..7ce8df02444 100644
--- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIOConfigTest.java
+++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIOConfigTest.java
@@ -32,6 +32,8 @@
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
 
+import java.util.Collections;
+
 public class KafkaIOConfigTest
 {
   private final ObjectMapper mapper;
@@ -57,7 +59,7 @@ public void testSerdeWithDefaults() throws Exception
                      + "  \"consumerProperties\": {\"bootstrap.servers\":\"localhost:9092\"}\n"
                      + "}";
 
-    KafkaIOConfig config = (KafkaIOConfig) mapper.readValue(
+    KafkaIndexTaskIOConfig config = (KafkaIndexTaskIOConfig) mapper.readValue(
         mapper.writeValueAsString(
             mapper.readValue(
                 jsonStr,
@@ -67,15 +69,16 @@ public void testSerdeWithDefaults() throws Exception
     );
 
     Assert.assertEquals("my-sequence-name", config.getBaseSequenceName());
-    Assert.assertEquals("mytopic", config.getStartPartitions().getTopic());
-    Assert.assertEquals(ImmutableMap.of(0, 1L, 1, 10L), config.getStartPartitions().getPartitionOffsetMap());
-    Assert.assertEquals("mytopic", config.getEndPartitions().getTopic());
-    Assert.assertEquals(ImmutableMap.of(0, 15L, 1, 200L), config.getEndPartitions().getPartitionOffsetMap());
+    Assert.assertEquals("mytopic", config.getStartPartitions().getStream());
+    Assert.assertEquals(ImmutableMap.of(0, 1L, 1, 10L), config.getStartPartitions().getPartitionSequenceNumberMap());
+    Assert.assertEquals("mytopic", config.getEndPartitions().getStream());
+    Assert.assertEquals(ImmutableMap.of(0, 15L, 1, 200L), config.getEndPartitions().getPartitionSequenceNumberMap());
     Assert.assertEquals(ImmutableMap.of("bootstrap.servers", "localhost:9092"), config.getConsumerProperties());
     Assert.assertTrue(config.isUseTransaction());
     Assert.assertFalse("minimumMessageTime", config.getMinimumMessageTime().isPresent());
     Assert.assertFalse("maximumMessageTime", config.getMaximumMessageTime().isPresent());
     Assert.assertFalse("skipOffsetGaps", config.isSkipOffsetGaps());
+    Assert.assertEquals(Collections.EMPTY_SET, config.getExclusiveStartSequenceNumberPartitions());
   }
 
   @Test
@@ -94,7 +97,7 @@ public void testSerdeWithNonDefaults() throws Exception
                      + "  \"skipOffsetGaps\": true\n"
                      + "}";
 
-    KafkaIOConfig config = (KafkaIOConfig) mapper.readValue(
+    KafkaIndexTaskIOConfig config = (KafkaIndexTaskIOConfig) mapper.readValue(
         mapper.writeValueAsString(
             mapper.readValue(
                 jsonStr,
@@ -104,15 +107,17 @@ public void testSerdeWithNonDefaults() throws Exception
     );
 
     Assert.assertEquals("my-sequence-name", config.getBaseSequenceName());
-    Assert.assertEquals("mytopic", config.getStartPartitions().getTopic());
-    Assert.assertEquals(ImmutableMap.of(0, 1L, 1, 10L), config.getStartPartitions().getPartitionOffsetMap());
-    Assert.assertEquals("mytopic", config.getEndPartitions().getTopic());
-    Assert.assertEquals(ImmutableMap.of(0, 15L, 1, 200L), config.getEndPartitions().getPartitionOffsetMap());
+    Assert.assertEquals("mytopic", config.getStartPartitions().getStream());
+    Assert.assertEquals(ImmutableMap.of(0, 1L, 1, 10L), config.getStartPartitions().getPartitionSequenceNumberMap());
+    Assert.assertEquals("mytopic", config.getEndPartitions().getStream());
+    Assert.assertEquals(ImmutableMap.of(0, 15L, 1, 200L), config.getEndPartitions().getPartitionSequenceNumberMap());
     Assert.assertEquals(ImmutableMap.of("bootstrap.servers", "localhost:9092"), config.getConsumerProperties());
     Assert.assertFalse(config.isUseTransaction());
     Assert.assertEquals(DateTimes.of("2016-05-31T12:00Z"), config.getMinimumMessageTime().get());
     Assert.assertEquals(DateTimes.of("2016-05-31T14:00Z"), config.getMaximumMessageTime().get());
     Assert.assertTrue("skipOffsetGaps", config.isSkipOffsetGaps());
+    Assert.assertEquals(Collections.EMPTY_SET, config.getExclusiveStartSequenceNumberPartitions());
+
   }
 
   @Test
@@ -212,7 +217,7 @@ public void testStartAndEndTopicMatch() throws Exception
 
     exception.expect(JsonMappingException.class);
     exception.expectCause(CoreMatchers.isA(IllegalArgumentException.class));
-    exception.expectMessage(CoreMatchers.containsString("start topic and end topic must match"));
+    exception.expectMessage(CoreMatchers.containsString("start topic/stream and end topic/stream must match"));
     mapper.readValue(jsonStr, IOConfig.class);
   }
 
diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientTest.java
index 5f0d915e5c8..28cc0bbe9d1 100644
--- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientTest.java
+++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientTest.java
@@ -23,7 +23,6 @@
 import com.google.common.base.Optional;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
@@ -31,6 +30,7 @@
 import org.apache.druid.indexer.TaskStatus;
 import org.apache.druid.indexing.common.IndexTaskClient;
 import org.apache.druid.indexing.common.TaskInfoProvider;
+import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner.Status;
 import org.apache.druid.jackson.DefaultObjectMapper;
 import org.apache.druid.java.util.common.DateTimes;
 import org.apache.druid.java.util.common.IAE;
@@ -41,6 +41,7 @@
 import org.apache.druid.java.util.http.client.response.FullResponseHolder;
 import org.easymock.Capture;
 import org.easymock.CaptureType;
+import org.easymock.EasyMock;
 import org.easymock.EasyMockSupport;
 import org.jboss.netty.handler.codec.http.HttpHeaders;
 import org.jboss.netty.handler.codec.http.HttpMethod;
@@ -60,15 +61,12 @@
 import java.io.IOException;
 import java.net.URL;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
-import static org.easymock.EasyMock.anyObject;
-import static org.easymock.EasyMock.capture;
-import static org.easymock.EasyMock.eq;
 import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.reset;
 
 @RunWith(Parameterized.class)
 public class KafkaIndexTaskClientTest extends EasyMockSupport
@@ -78,7 +76,7 @@
 
   private static final ObjectMapper objectMapper = new DefaultObjectMapper();
   private static final String TEST_ID = "test-id";
-  private static final List<String> TEST_IDS = Lists.newArrayList("test-id1", "test-id2", "test-id3", "test-id4");
+  private static final List<String> TEST_IDS = Arrays.asList("test-id1", "test-id2", "test-id3", "test-id4");
   private static final String TEST_HOST = "test-host";
   private static final int TEST_PORT = 1234;
   private static final int TEST_TLS_PORT = -1;
@@ -116,9 +114,12 @@ public void setUp()
     headers = createMock(HttpHeaders.class);
 
     client = new TestableKafkaIndexTaskClient(httpClient, objectMapper, taskInfoProvider);
-    expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT))
-                                                     .anyTimes();
-    expect(taskInfoProvider.getTaskStatus(TEST_ID)).andReturn(Optional.of(TaskStatus.running(TEST_ID))).anyTimes();
+    expect(taskInfoProvider.getTaskLocation(TEST_ID))
+        .andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT))
+        .anyTimes();
+    expect(taskInfoProvider.getTaskStatus(TEST_ID))
+        .andReturn(Optional.of(TaskStatus.running(TEST_ID)))
+        .anyTimes();
 
     for (String testId : TEST_IDS) {
       expect(taskInfoProvider.getTaskLocation(testId))
@@ -139,16 +140,18 @@ public void tearDown()
   @Test
   public void testNoTaskLocation() throws IOException
   {
-    reset(taskInfoProvider);
+    EasyMock.reset(taskInfoProvider);
     expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(TaskLocation.unknown()).anyTimes();
-    expect(taskInfoProvider.getTaskStatus(TEST_ID)).andReturn(Optional.of(TaskStatus.running(TEST_ID))).anyTimes();
+    expect(taskInfoProvider.getTaskStatus(TEST_ID))
+        .andReturn(Optional.of(TaskStatus.running(TEST_ID)))
+        .anyTimes();
     replayAll();
 
     Assert.assertEquals(false, client.stop(TEST_ID, true));
     Assert.assertEquals(false, client.resume(TEST_ID));
     Assert.assertEquals(ImmutableMap.of(), client.pause(TEST_ID));
     Assert.assertEquals(ImmutableMap.of(), client.pause(TEST_ID));
-    Assert.assertEquals(KafkaIndexTask.Status.NOT_STARTED, client.getStatus(TEST_ID));
+    Assert.assertEquals(Status.NOT_STARTED, client.getStatus(TEST_ID));
     Assert.assertEquals(null, client.getStartTime(TEST_ID));
     Assert.assertEquals(ImmutableMap.of(), client.getCurrentOffsets(TEST_ID, true));
     Assert.assertEquals(ImmutableMap.of(), client.getEndOffsets(TEST_ID));
@@ -164,10 +167,13 @@ public void testTaskNotRunnableException()
     expectedException.expect(IndexTaskClient.TaskNotRunnableException.class);
     expectedException.expectMessage("Aborting request because task [test-id] is not runnable");
 
-    reset(taskInfoProvider);
-    expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT))
-                                                     .anyTimes();
-    expect(taskInfoProvider.getTaskStatus(TEST_ID)).andReturn(Optional.of(TaskStatus.failure(TEST_ID))).anyTimes();
+    EasyMock.reset(taskInfoProvider);
+    expect(taskInfoProvider.getTaskLocation(TEST_ID))
+        .andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT))
+        .anyTimes();
+    expect(taskInfoProvider.getTaskStatus(TEST_ID))
+        .andReturn(Optional.of(TaskStatus.failure(TEST_ID)))
+        .anyTimes();
     replayAll();
 
     client.getCurrentOffsets(TEST_ID, true);
@@ -183,9 +189,9 @@ public void testInternalServerError()
     expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.INTERNAL_SERVER_ERROR).times(2);
     expect(
         httpClient.go(
-            anyObject(Request.class),
-            anyObject(FullResponseHandler.class),
-            eq(TEST_HTTP_TIMEOUT)
+            EasyMock.anyObject(Request.class),
+            EasyMock.anyObject(FullResponseHandler.class),
+            EasyMock.eq(TEST_HTTP_TIMEOUT)
         )
     ).andReturn(
         Futures.immediateFuture(responseHolder)
@@ -206,9 +212,9 @@ public void testBadRequest()
     expect(responseHolder.getContent()).andReturn("");
     expect(
         httpClient.go(
-            anyObject(Request.class),
-            anyObject(FullResponseHandler.class),
-            eq(TEST_HTTP_TIMEOUT)
+            EasyMock.anyObject(Request.class),
+            EasyMock.anyObject(FullResponseHandler.class),
+            EasyMock.eq(TEST_HTTP_TIMEOUT)
         )
     ).andReturn(
         Futures.immediateFuture(responseHolder)
@@ -231,9 +237,9 @@ public void testTaskLocationMismatch()
     expect(headers.get("X-Druid-Task-Id")).andReturn("a-different-task-id");
     expect(
         httpClient.go(
-            anyObject(Request.class),
-            anyObject(FullResponseHandler.class),
-            eq(TEST_HTTP_TIMEOUT)
+            EasyMock.anyObject(Request.class),
+            EasyMock.anyObject(FullResponseHandler.class),
+            EasyMock.eq(TEST_HTTP_TIMEOUT)
         )
     ).andReturn(
         Futures.immediateFuture(responseHolder)
@@ -252,7 +258,11 @@ public void testGetCurrentOffsets() throws Exception
     Capture<Request> captured = Capture.newInstance();
     expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK);
     expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}");
-    expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn(
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
         Futures.immediateFuture(responseHolder)
     );
     replayAll();
@@ -287,7 +297,11 @@ public void testGetCurrentOffsetsWithRetry() throws Exception
     expect(response.headers()).andReturn(headers).times(2);
     expect(headers.get("X-Druid-Task-Id")).andReturn(TEST_ID).times(2);
 
-    expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn(
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
         Futures.immediateFuture(responseHolder)
     ).times(3);
 
@@ -327,9 +341,9 @@ public void testGetCurrentOffsetsWithExhaustedRetries()
 
     expect(
         httpClient.go(
-            anyObject(Request.class),
-            anyObject(FullResponseHandler.class),
-            eq(TEST_HTTP_TIMEOUT)
+            EasyMock.anyObject(Request.class),
+            EasyMock.anyObject(FullResponseHandler.class),
+            EasyMock.eq(TEST_HTTP_TIMEOUT)
         )
     ).andReturn(Futures.immediateFuture(responseHolder)).anyTimes();
     replayAll();
@@ -344,7 +358,11 @@ public void testGetEndOffsets() throws Exception
     Capture<Request> captured = Capture.newInstance();
     expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK);
     expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}");
-    expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn(
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
         Futures.immediateFuture(responseHolder)
     );
     replayAll();
@@ -378,7 +396,11 @@ public void testGetStartTime() throws Exception
     expect(response.headers()).andReturn(headers);
     expect(headers.get("X-Druid-Task-Id")).andReturn(null);
     expect(responseHolder.getContent()).andReturn(String.valueOf(now.getMillis())).anyTimes();
-    expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn(
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
         Futures.immediateFuture(responseHolder)
     ).times(2);
     replayAll();
@@ -400,17 +422,21 @@ public void testGetStartTime() throws Exception
   @Test
   public void testGetStatus() throws Exception
   {
-    KafkaIndexTask.Status status = KafkaIndexTask.Status.READING;
+    Status status = Status.READING;
 
     Capture<Request> captured = Capture.newInstance();
     expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK);
     expect(responseHolder.getContent()).andReturn(StringUtils.format("\"%s\"", status.toString())).anyTimes();
-    expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn(
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
         Futures.immediateFuture(responseHolder)
     );
     replayAll();
 
-    KafkaIndexTask.Status results = client.getStatus(TEST_ID);
+    Status results = client.getStatus(TEST_ID);
     verifyAll();
 
     Request request = captured.getValue();
@@ -430,7 +456,11 @@ public void testPause() throws Exception
     Capture<Request> captured = Capture.newInstance();
     expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).times(2);
     expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}").anyTimes();
-    expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn(
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
         Futures.immediateFuture(responseHolder)
     );
     replayAll();
@@ -462,13 +492,25 @@ public void testPauseWithSubsequentGetOffsets() throws Exception
                                       .andReturn(HttpResponseStatus.OK).anyTimes();
     expect(responseHolder.getContent()).andReturn("\"PAUSED\"").times(2)
                                        .andReturn("{\"0\":1, \"1\":10}").anyTimes();
-    expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn(
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
         Futures.immediateFuture(responseHolder)
     );
-    expect(httpClient.go(capture(captured2), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn(
+    expect(httpClient.go(
+        EasyMock.capture(captured2),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
         Futures.immediateFuture(responseHolder)
     );
-    expect(httpClient.go(capture(captured3), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn(
+    expect(httpClient.go(
+        EasyMock.capture(captured3),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
         Futures.immediateFuture(responseHolder)
     );
 
@@ -509,7 +551,11 @@ public void testResume() throws Exception
   {
     Capture<Request> captured = Capture.newInstance();
     expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes();
-    expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn(
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
         Futures.immediateFuture(responseHolder)
     );
     replayAll();
@@ -533,7 +579,11 @@ public void testSetEndOffsets() throws Exception
 
     Capture<Request> captured = Capture.newInstance();
     expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes();
-    expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn(
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
         Futures.immediateFuture(responseHolder)
     );
     replayAll();
@@ -558,7 +608,11 @@ public void testSetEndOffsetsAndResume() throws Exception
 
     Capture<Request> captured = Capture.newInstance();
     expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes();
-    expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn(
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
         Futures.immediateFuture(responseHolder)
     );
     replayAll();
@@ -581,7 +635,11 @@ public void testStop() throws Exception
   {
     Capture<Request> captured = Capture.newInstance();
     expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes();
-    expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn(
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
         Futures.immediateFuture(responseHolder)
     );
     replayAll();
@@ -603,7 +661,11 @@ public void testStopAndPublish() throws Exception
   {
     Capture<Request> captured = Capture.newInstance();
     expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes();
-    expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn(
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
         Futures.immediateFuture(responseHolder)
     );
     replayAll();
@@ -626,7 +688,11 @@ public void testStopAsync() throws Exception
     final int numRequests = TEST_IDS.size();
     Capture<Request> captured = Capture.newInstance(CaptureType.ALL);
     expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes();
-    expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn(
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
         Futures.immediateFuture(responseHolder)
     ).times(numRequests);
     replayAll();
@@ -658,7 +724,11 @@ public void testResumeAsync() throws Exception
     final int numRequests = TEST_IDS.size();
     Capture<Request> captured = Capture.newInstance(CaptureType.ALL);
     expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes();
-    expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn(
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
         Futures.immediateFuture(responseHolder)
     ).times(numRequests);
     replayAll();
@@ -691,7 +761,11 @@ public void testPauseAsync() throws Exception
     Capture<Request> captured = Capture.newInstance(CaptureType.ALL);
     expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes();
     expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes();
-    expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn(
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
         Futures.immediateFuture(responseHolder)
     ).times(numRequests);
     replayAll();
@@ -724,19 +798,23 @@ public void testGetStatusAsync() throws Exception
     Capture<Request> captured = Capture.newInstance(CaptureType.ALL);
     expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes();
     expect(responseHolder.getContent()).andReturn("\"READING\"").anyTimes();
-    expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn(
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
         Futures.immediateFuture(responseHolder)
     ).times(numRequests);
     replayAll();
 
     List<URL> expectedUrls = new ArrayList<>();
-    List<ListenableFuture<KafkaIndexTask.Status>> futures = new ArrayList<>();
+    List<ListenableFuture<Status>> futures = new ArrayList<>();
     for (String testId : TEST_IDS) {
       expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "status")));
       futures.add(client.getStatusAsync(testId));
     }
 
-    List<KafkaIndexTask.Status> responses = Futures.allAsList(futures).get();
+    List<Status> responses = Futures.allAsList(futures).get();
 
     verifyAll();
     List<Request> requests = captured.getValues();
@@ -746,7 +824,7 @@ public void testGetStatusAsync() throws Exception
     for (int i = 0; i < numRequests; i++) {
       Assert.assertEquals(HttpMethod.GET, requests.get(i).getMethod());
       Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl()));
-      Assert.assertEquals(KafkaIndexTask.Status.READING, responses.get(i));
+      Assert.assertEquals(Status.READING, responses.get(i));
     }
   }
 
@@ -758,7 +836,11 @@ public void testGetStartTimeAsync() throws Exception
     Capture<Request> captured = Capture.newInstance(CaptureType.ALL);
     expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes();
     expect(responseHolder.getContent()).andReturn(String.valueOf(now.getMillis())).anyTimes();
-    expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn(
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
         Futures.immediateFuture(responseHolder)
     ).times(numRequests);
     replayAll();
@@ -791,7 +873,11 @@ public void testGetCurrentOffsetsAsync() throws Exception
     Capture<Request> captured = Capture.newInstance(CaptureType.ALL);
     expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes();
     expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes();
-    expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn(
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
         Futures.immediateFuture(responseHolder)
     ).times(numRequests);
     replayAll();
@@ -824,7 +910,11 @@ public void testGetEndOffsetsAsync() throws Exception
     Capture<Request> captured = Capture.newInstance(CaptureType.ALL);
     expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes();
     expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes();
-    expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn(
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
         Futures.immediateFuture(responseHolder)
     ).times(numRequests);
     replayAll();
@@ -857,7 +947,11 @@ public void testSetEndOffsetsAsync() throws Exception
     final int numRequests = TEST_IDS.size();
     Capture<Request> captured = Capture.newInstance(CaptureType.ALL);
     expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes();
-    expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn(
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
         Futures.immediateFuture(responseHolder)
     ).times(numRequests);
     replayAll();
@@ -896,7 +990,11 @@ public void testSetEndOffsetsAsyncWithResume() throws Exception
     final int numRequests = TEST_IDS.size();
     Capture<Request> captured = Capture.newInstance(CaptureType.ALL);
     expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes();
-    expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn(
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
         Futures.immediateFuture(responseHolder)
     ).times(numRequests);
     replayAll();
diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java
index 45c8a93226d..72041eb947d 100644
--- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java
+++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java
@@ -69,7 +69,6 @@
 import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory;
 import org.apache.druid.indexing.common.task.IndexTaskTest;
 import org.apache.druid.indexing.common.task.Task;
-import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisor;
 import org.apache.druid.indexing.kafka.test.TestBroker;
 import org.apache.druid.indexing.overlord.DataSourceMetadata;
 import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
@@ -77,6 +76,9 @@
 import org.apache.druid.indexing.overlord.TaskLockbox;
 import org.apache.druid.indexing.overlord.TaskStorage;
 import org.apache.druid.indexing.overlord.supervisor.SupervisorManager;
+import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner.Status;
+import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
+import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor;
 import org.apache.druid.indexing.test.TestDataSegmentAnnouncer;
 import org.apache.druid.indexing.test.TestDataSegmentKiller;
 import org.apache.druid.java.util.common.CompressionUtils;
@@ -101,6 +103,7 @@
 import org.apache.druid.query.Druids;
 import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator;
 import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryPlus;
 import org.apache.druid.query.QueryRunner;
 import org.apache.druid.query.QueryRunnerFactoryConglomerate;
 import org.apache.druid.query.QueryToolChest;
@@ -174,8 +177,6 @@
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
-import static org.apache.druid.query.QueryPlus.wrap;
-
 @RunWith(Parameterized.class)
 public class KafkaIndexTaskTest
 {
@@ -379,11 +380,11 @@ public void testRunAfterDataInserted() throws Exception
 
     final KafkaIndexTask task = createTask(
         null,
-        new KafkaIOConfig(
+        new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
-            new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)),
             kafkaServer.consumerProperties(),
             true,
             null,
@@ -407,7 +408,7 @@ public void testRunAfterDataInserted() throws Exception
     SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
     Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
     Assert.assertEquals(
-        new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))),
+        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -421,11 +422,11 @@ public void testRunBeforeDataInserted() throws Exception
   {
     final KafkaIndexTask task = createTask(
         null,
-        new KafkaIOConfig(
+        new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
-            new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)),
             kafkaServer.consumerProperties(),
             true,
             null,
@@ -437,7 +438,7 @@ public void testRunBeforeDataInserted() throws Exception
     final ListenableFuture<TaskStatus> future = runTask(task);
 
     // Wait for the task to start reading
-    while (task.getRunner().getStatus() != KafkaIndexTask.Status.READING) {
+    while (task.getRunner().getStatus() != Status.READING) {
       Thread.sleep(10);
     }
 
@@ -461,7 +462,7 @@ public void testRunBeforeDataInserted() throws Exception
     SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
     Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
     Assert.assertEquals(
-        new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))),
+        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -489,15 +490,47 @@ public void testIncrementalHandOff() throws Exception
     Map<String, Object> consumerProps = kafkaServer.consumerProperties();
     consumerProps.put("max.poll.records", "1");
 
-    final KafkaPartitions startPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 0L, 1, 0L));
+    final SeekableStreamPartitions<Integer, Long> startPartitions = new SeekableStreamPartitions<>(
+        topic,
+        ImmutableMap.of(
+            0,
+            0L,
+            1,
+            0L
+        )
+    );
     // Checkpointing will happen at either checkpoint1 or checkpoint2 depending on ordering
     // of events fetched across two partitions from Kafka
-    final KafkaPartitions checkpoint1 = new KafkaPartitions(topic, ImmutableMap.of(0, 5L, 1, 0L));
-    final KafkaPartitions checkpoint2 = new KafkaPartitions(topic, ImmutableMap.of(0, 4L, 1, 2L));
-    final KafkaPartitions endPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 10L, 1, 2L));
+    final SeekableStreamPartitions<Integer, Long> checkpoint1 = new SeekableStreamPartitions<>(
+        topic,
+        ImmutableMap.of(
+            0,
+            5L,
+            1,
+            0L
+        )
+    );
+    final SeekableStreamPartitions<Integer, Long> checkpoint2 = new SeekableStreamPartitions<>(
+        topic,
+        ImmutableMap.of(
+            0,
+            4L,
+            1,
+            2L
+        )
+    );
+    final SeekableStreamPartitions<Integer, Long> endPartitions = new SeekableStreamPartitions<>(
+        topic,
+        ImmutableMap.of(
+            0,
+            10L,
+            1,
+            2L
+        )
+    );
     final KafkaIndexTask task = createTask(
         null,
-        new KafkaIOConfig(
+        new KafkaIndexTaskIOConfig(
             0,
             baseSequenceName,
             startPartitions,
@@ -510,12 +543,13 @@ public void testIncrementalHandOff() throws Exception
         )
     );
     final ListenableFuture<TaskStatus> future = runTask(task);
-    while (task.getRunner().getStatus() != KafkaIndexTask.Status.PAUSED) {
+    while (task.getRunner().getStatus() != Status.PAUSED) {
       Thread.sleep(10);
     }
     final Map<Integer, Long> currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets());
-    Assert.assertTrue(checkpoint1.getPartitionOffsetMap().equals(currentOffsets) || checkpoint2.getPartitionOffsetMap()
-                                                                                               .equals(currentOffsets));
+    Assert.assertTrue(checkpoint1.getPartitionSequenceNumberMap().equals(currentOffsets)
+                      || checkpoint2.getPartitionSequenceNumberMap()
+                                    .equals(currentOffsets));
     task.getRunner().setEndOffsets(currentOffsets, false);
     Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode());
 
@@ -526,7 +560,7 @@ public void testIncrementalHandOff() throws Exception
                 DATA_SCHEMA.getDataSource(),
                 0,
                 new KafkaDataSourceMetadata(startPartitions),
-                new KafkaDataSourceMetadata(new KafkaPartitions(topic, currentOffsets))
+                new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, currentOffsets))
             )
         )
     );
@@ -546,7 +580,7 @@ public void testIncrementalHandOff() throws Exception
     SegmentDescriptor desc7 = SD(task, "2013/P1D", 0);
     Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors());
     Assert.assertEquals(
-        new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 10L, 1, 2L))),
+        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 2L))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -584,14 +618,46 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception
       Map<String, Object> consumerProps = kafkaServer.consumerProperties();
       consumerProps.put("max.poll.records", "1");
 
-      final KafkaPartitions startPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 0L, 1, 0L));
-      final KafkaPartitions checkpoint1 = new KafkaPartitions(topic, ImmutableMap.of(0, 3L, 1, 0L));
-      final KafkaPartitions checkpoint2 = new KafkaPartitions(topic, ImmutableMap.of(0, 10L, 1, 0L));
+      final SeekableStreamPartitions<Integer, Long> startPartitions = new SeekableStreamPartitions<>(
+          topic,
+          ImmutableMap.of(
+              0,
+              0L,
+              1,
+              0L
+          )
+      );
+      final SeekableStreamPartitions<Integer, Long> checkpoint1 = new SeekableStreamPartitions<>(
+          topic,
+          ImmutableMap.of(
+              0,
+              3L,
+              1,
+              0L
+          )
+      );
+      final SeekableStreamPartitions<Integer, Long> checkpoint2 = new SeekableStreamPartitions<>(
+          topic,
+          ImmutableMap.of(
+              0,
+              10L,
+              1,
+              0L
+          )
+      );
 
-      final KafkaPartitions endPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 10L, 1, 2L));
+      final SeekableStreamPartitions<Integer, Long> endPartitions = new SeekableStreamPartitions<>(
+          topic,
+          ImmutableMap.of(
+              0,
+              10L,
+              1,
+              2L
+          )
+      );
       final KafkaIndexTask task = createTask(
           null,
-          new KafkaIOConfig(
+          new KafkaIndexTaskIOConfig(
               0,
               baseSequenceName,
               startPartitions,
@@ -604,15 +670,15 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception
           )
       );
       final ListenableFuture<TaskStatus> future = runTask(task);
-      while (task.getRunner().getStatus() != KafkaIndexTask.Status.PAUSED) {
+      while (task.getRunner().getStatus() != Status.PAUSED) {
         Thread.sleep(10);
       }
       final Map<Integer, Long> currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets());
 
-      Assert.assertTrue(checkpoint1.getPartitionOffsetMap().equals(currentOffsets));
+      Assert.assertTrue(checkpoint1.getPartitionSequenceNumberMap().equals(currentOffsets));
       task.getRunner().setEndOffsets(currentOffsets, false);
 
-      while (task.getRunner().getStatus() != KafkaIndexTask.Status.PAUSED) {
+      while (task.getRunner().getStatus() != Status.PAUSED) {
         Thread.sleep(10);
       }
 
@@ -622,7 +688,7 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception
       }
       final Map<Integer, Long> nextOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets());
 
-      Assert.assertTrue(checkpoint2.getPartitionOffsetMap().equals(nextOffsets));
+      Assert.assertTrue(checkpoint2.getPartitionSequenceNumberMap().equals(nextOffsets));
       task.getRunner().setEndOffsets(nextOffsets, false);
 
       Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode());
@@ -634,7 +700,7 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception
                   DATA_SCHEMA.getDataSource(),
                   0,
                   new KafkaDataSourceMetadata(startPartitions),
-                  new KafkaDataSourceMetadata(new KafkaPartitions(topic, currentOffsets))
+                  new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, currentOffsets))
               )
           )
       );
@@ -643,8 +709,8 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception
               Objects.hash(
                   DATA_SCHEMA.getDataSource(),
                   0,
-                  new KafkaDataSourceMetadata(new KafkaPartitions(topic, currentOffsets)),
-                  new KafkaDataSourceMetadata(new KafkaPartitions(topic, nextOffsets))
+                  new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, currentOffsets)),
+                  new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, nextOffsets))
               )
           )
       );
@@ -664,7 +730,7 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception
       SegmentDescriptor desc7 = SD(task, "2013/P1D", 0);
       Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors());
       Assert.assertEquals(
-          new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 10L, 1, 2L))),
+          new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 2L))),
           metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
       );
 
@@ -701,13 +767,37 @@ public void testTimeBasedIncrementalHandOff() throws Exception
     Map<String, Object> consumerProps = kafkaServer.consumerProperties();
     consumerProps.put("max.poll.records", "1");
 
-    final KafkaPartitions startPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 0L, 1, 0L));
+    final SeekableStreamPartitions<Integer, Long> startPartitions = new SeekableStreamPartitions<>(
+        topic,
+        ImmutableMap.of(
+            0,
+            0L,
+            1,
+            0L
+        )
+    );
     // Checkpointing will happen at checkpoint
-    final KafkaPartitions checkpoint = new KafkaPartitions(topic, ImmutableMap.of(0, 1L, 1, 0L));
-    final KafkaPartitions endPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 2L, 1, 0L));
+    final SeekableStreamPartitions<Integer, Long> checkpoint = new SeekableStreamPartitions<>(
+        topic,
+        ImmutableMap.of(
+            0,
+            1L,
+            1,
+            0L
+        )
+    );
+    final SeekableStreamPartitions<Integer, Long> endPartitions = new SeekableStreamPartitions<>(
+        topic,
+        ImmutableMap.of(
+            0,
+            2L,
+            1,
+            0L
+        )
+    );
     final KafkaIndexTask task = createTask(
         null,
-        new KafkaIOConfig(
+        new KafkaIndexTaskIOConfig(
             0,
             baseSequenceName,
             startPartitions,
@@ -722,11 +812,11 @@ public void testTimeBasedIncrementalHandOff() throws Exception
     final ListenableFuture<TaskStatus> future = runTask(task);
 
     // task will pause for checkpointing
-    while (task.getRunner().getStatus() != KafkaIndexTask.Status.PAUSED) {
+    while (task.getRunner().getStatus() != Status.PAUSED) {
       Thread.sleep(10);
     }
     final Map<Integer, Long> currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets());
-    Assert.assertTrue(checkpoint.getPartitionOffsetMap().equals(currentOffsets));
+    Assert.assertTrue(checkpoint.getPartitionSequenceNumberMap().equals(currentOffsets));
     task.getRunner().setEndOffsets(currentOffsets, false);
     Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode());
 
@@ -737,7 +827,10 @@ public void testTimeBasedIncrementalHandOff() throws Exception
                 DATA_SCHEMA.getDataSource(),
                 0,
                 new KafkaDataSourceMetadata(startPartitions),
-                new KafkaDataSourceMetadata(new KafkaPartitions(topic, checkpoint.getPartitionOffsetMap()))
+                new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(
+                    topic,
+                    checkpoint.getPartitionSequenceNumberMap()
+                ))
             )
         )
     );
@@ -752,7 +845,7 @@ public void testTimeBasedIncrementalHandOff() throws Exception
     SegmentDescriptor desc2 = SD(task, "2009/P1D", 0);
     Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
     Assert.assertEquals(
-        new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 2L, 1, 0L))),
+        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L, 1, 0L))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -791,13 +884,22 @@ public void testIncrementalHandOffReadsThroughEndOffsets() throws Exception
     Map<String, Object> consumerProps = kafkaServer.consumerProperties();
     consumerProps.put("max.poll.records", "1");
 
-    final KafkaPartitions startPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 0L));
-    final KafkaPartitions checkpoint1 = new KafkaPartitions(topic, ImmutableMap.of(0, 5L));
-    final KafkaPartitions endPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 7L));
+    final SeekableStreamPartitions<Integer, Long> startPartitions = new SeekableStreamPartitions<>(
+        topic,
+        ImmutableMap.of(0, 0L)
+    );
+    final SeekableStreamPartitions<Integer, Long> checkpoint1 = new SeekableStreamPartitions<>(
+        topic,
+        ImmutableMap.of(0, 5L)
+    );
+    final SeekableStreamPartitions<Integer, Long> endPartitions = new SeekableStreamPartitions<>(
+        topic,
+        ImmutableMap.of(0, 7L)
+    );
 
     final KafkaIndexTask task = createTask(
         null,
-        new KafkaIOConfig(
+        new KafkaIndexTaskIOConfig(
             0,
             baseSequenceName,
             startPartitions,
@@ -810,11 +912,11 @@ public void testIncrementalHandOffReadsThroughEndOffsets() throws Exception
         )
     );
     final ListenableFuture<TaskStatus> future = runTask(task);
-    while (task.getRunner().getStatus() != KafkaIndexTask.Status.PAUSED) {
+    while (task.getRunner().getStatus() != Status.PAUSED) {
       Thread.sleep(10);
     }
     final Map<Integer, Long> currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets());
-    Assert.assertTrue(checkpoint1.getPartitionOffsetMap().equals(currentOffsets));
+    Assert.assertTrue(checkpoint1.getPartitionSequenceNumberMap().equals(currentOffsets));
 
     // actual checkpoint offset is 5, but simulating behavior of publishing set end offset call, to ensure this task
     // will continue reading through the end offset of the checkpointed sequence
@@ -833,11 +935,11 @@ public void testRunWithMinimumMessageTime() throws Exception
   {
     final KafkaIndexTask task = createTask(
         null,
-        new KafkaIOConfig(
+        new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new KafkaPartitions(topic, ImmutableMap.of(0, 0L)),
-            new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)),
             kafkaServer.consumerProperties(),
             true,
             DateTimes.of("2010"),
@@ -849,7 +951,7 @@ public void testRunWithMinimumMessageTime() throws Exception
     final ListenableFuture<TaskStatus> future = runTask(task);
 
     // Wait for the task to start reading
-    while (task.getRunner().getStatus() != KafkaIndexTask.Status.READING) {
+    while (task.getRunner().getStatus() != Status.READING) {
       Thread.sleep(10);
     }
 
@@ -873,7 +975,7 @@ public void testRunWithMinimumMessageTime() throws Exception
     SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
     Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
     Assert.assertEquals(
-        new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))),
+        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -887,11 +989,11 @@ public void testRunWithMaximumMessageTime() throws Exception
   {
     final KafkaIndexTask task = createTask(
         null,
-        new KafkaIOConfig(
+        new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new KafkaPartitions(topic, ImmutableMap.of(0, 0L)),
-            new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)),
             kafkaServer.consumerProperties(),
             true,
             null,
@@ -903,7 +1005,7 @@ public void testRunWithMaximumMessageTime() throws Exception
     final ListenableFuture<TaskStatus> future = runTask(task);
 
     // Wait for the task to start reading
-    while (task.getRunner().getStatus() != KafkaIndexTask.Status.READING) {
+    while (task.getRunner().getStatus() != Status.READING) {
       Thread.sleep(10);
     }
 
@@ -928,7 +1030,7 @@ public void testRunWithMaximumMessageTime() throws Exception
     SegmentDescriptor desc3 = SD(task, "2010/P1D", 0);
     Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors());
     Assert.assertEquals(
-        new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))),
+        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -951,11 +1053,11 @@ public void testRunWithTransformSpec() throws Exception
                 )
             )
         ),
-        new KafkaIOConfig(
+        new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new KafkaPartitions(topic, ImmutableMap.of(0, 0L)),
-            new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)),
             kafkaServer.consumerProperties(),
             true,
             null,
@@ -967,7 +1069,7 @@ public void testRunWithTransformSpec() throws Exception
     final ListenableFuture<TaskStatus> future = runTask(task);
 
     // Wait for the task to start reading
-    while (task.getRunner().getStatus() != KafkaIndexTask.Status.READING) {
+    while (task.getRunner().getStatus() != Status.READING) {
       Thread.sleep(10);
     }
 
@@ -990,7 +1092,7 @@ public void testRunWithTransformSpec() throws Exception
     SegmentDescriptor desc1 = SD(task, "2009/P1D", 0);
     Assert.assertEquals(ImmutableSet.of(desc1), publishedDescriptors());
     Assert.assertEquals(
-        new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))),
+        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -1011,11 +1113,11 @@ public void testRunOnNothing() throws Exception
 
     final KafkaIndexTask task = createTask(
         null,
-        new KafkaIOConfig(
+        new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
-            new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)),
             kafkaServer.consumerProperties(),
             true,
             null,
@@ -1052,11 +1154,11 @@ public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception
 
     final KafkaIndexTask task = createTask(
         null,
-        new KafkaIOConfig(
+        new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
-            new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)),
             kafkaServer.consumerProperties(),
             true,
             null,
@@ -1080,7 +1182,7 @@ public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception
     SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
     Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
     Assert.assertEquals(
-        new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))),
+        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -1104,11 +1206,11 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio
 
     final KafkaIndexTask task = createTask(
         null,
-        new KafkaIOConfig(
+        new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
-            new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)),
             kafkaServer.consumerProperties(),
             true,
             null,
@@ -1132,7 +1234,7 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio
     SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
     Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
     Assert.assertEquals(
-        new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))),
+        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -1159,11 +1261,11 @@ public void testReportParseExceptions() throws Exception
 
     final KafkaIndexTask task = createTask(
         null,
-        new KafkaIOConfig(
+        new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
-            new KafkaPartitions(topic, ImmutableMap.of(0, 7L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 7L)),
             kafkaServer.consumerProperties(),
             true,
             null,
@@ -1203,11 +1305,11 @@ public void testMultipleParseExceptionsSuccess() throws Exception
 
     final KafkaIndexTask task = createTask(
         null,
-        new KafkaIOConfig(
+        new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
-            new KafkaPartitions(topic, ImmutableMap.of(0, 13L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 13L)),
             kafkaServer.consumerProperties(),
             true,
             null,
@@ -1237,7 +1339,7 @@ public void testMultipleParseExceptionsSuccess() throws Exception
     SegmentDescriptor desc4 = SD(task, "2049/P1D", 0);
     Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors());
     Assert.assertEquals(
-        new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 13L))),
+        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 13L))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -1285,11 +1387,11 @@ public void testMultipleParseExceptionsFailure() throws Exception
 
     final KafkaIndexTask task = createTask(
         null,
-        new KafkaIOConfig(
+        new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
-            new KafkaPartitions(topic, ImmutableMap.of(0, 10L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L)),
             kafkaServer.consumerProperties(),
             true,
             null,
@@ -1345,11 +1447,11 @@ public void testRunReplicas() throws Exception
   {
     final KafkaIndexTask task1 = createTask(
         null,
-        new KafkaIOConfig(
+        new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
-            new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)),
             kafkaServer.consumerProperties(),
             true,
             null,
@@ -1359,11 +1461,11 @@ public void testRunReplicas() throws Exception
     );
     final KafkaIndexTask task2 = createTask(
         null,
-        new KafkaIOConfig(
+        new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
-            new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)),
             kafkaServer.consumerProperties(),
             true,
             null,
@@ -1399,7 +1501,7 @@ public void testRunReplicas() throws Exception
     SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0);
     Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
     Assert.assertEquals(
-        new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))),
+        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -1413,11 +1515,11 @@ public void testRunConflicting() throws Exception
   {
     final KafkaIndexTask task1 = createTask(
         null,
-        new KafkaIOConfig(
+        new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
-            new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)),
             kafkaServer.consumerProperties(),
             true,
             null,
@@ -1427,11 +1529,11 @@ public void testRunConflicting() throws Exception
     );
     final KafkaIndexTask task2 = createTask(
         null,
-        new KafkaIOConfig(
+        new KafkaIndexTaskIOConfig(
             1,
             "sequence1",
-            new KafkaPartitions(topic, ImmutableMap.of(0, 3L)),
-            new KafkaPartitions(topic, ImmutableMap.of(0, 10L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 3L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L)),
             kafkaServer.consumerProperties(),
             true,
             null,
@@ -1468,7 +1570,7 @@ public void testRunConflicting() throws Exception
     SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0);
     Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
     Assert.assertEquals(
-        new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))),
+        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -1482,11 +1584,11 @@ public void testRunConflictingWithoutTransactions() throws Exception
   {
     final KafkaIndexTask task1 = createTask(
         null,
-        new KafkaIOConfig(
+        new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
-            new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)),
             kafkaServer.consumerProperties(),
             false,
             null,
@@ -1496,11 +1598,11 @@ public void testRunConflictingWithoutTransactions() throws Exception
     );
     final KafkaIndexTask task2 = createTask(
         null,
-        new KafkaIOConfig(
+        new KafkaIndexTaskIOConfig(
             1,
             "sequence1",
-            new KafkaPartitions(topic, ImmutableMap.of(0, 3L)),
-            new KafkaPartitions(topic, ImmutableMap.of(0, 10L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 3L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L)),
             kafkaServer.consumerProperties(),
             false,
             null,
@@ -1556,11 +1658,11 @@ public void testRunOneTaskTwoPartitions() throws Exception
   {
     final KafkaIndexTask task = createTask(
         null,
-        new KafkaIOConfig(
+        new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new KafkaPartitions(topic, ImmutableMap.of(0, 2L, 1, 0L)),
-            new KafkaPartitions(topic, ImmutableMap.of(0, 5L, 1, 2L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L, 1, 0L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L, 1, 2L)),
             kafkaServer.consumerProperties(),
             true,
             null,
@@ -1597,7 +1699,7 @@ public void testRunOneTaskTwoPartitions() throws Exception
                         ? ImmutableSet.of(desc1, desc2, desc4)
                         : ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors());
     Assert.assertEquals(
-        new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L, 1, 2L))),
+        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L, 1, 2L))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -1621,11 +1723,11 @@ public void testRunTwoTasksTwoPartitions() throws Exception
   {
     final KafkaIndexTask task1 = createTask(
         null,
-        new KafkaIOConfig(
+        new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
-            new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)),
             kafkaServer.consumerProperties(),
             true,
             null,
@@ -1635,11 +1737,11 @@ public void testRunTwoTasksTwoPartitions() throws Exception
     );
     final KafkaIndexTask task2 = createTask(
         null,
-        new KafkaIOConfig(
+        new KafkaIndexTaskIOConfig(
             1,
             "sequence1",
-            new KafkaPartitions(topic, ImmutableMap.of(1, 0L)),
-            new KafkaPartitions(topic, ImmutableMap.of(1, 1L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(1, 0L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(1, 1L)),
             kafkaServer.consumerProperties(),
             true,
             null,
@@ -1676,7 +1778,7 @@ public void testRunTwoTasksTwoPartitions() throws Exception
     SegmentDescriptor desc3 = SD(task2, "2012/P1D", 0);
     Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors());
     Assert.assertEquals(
-        new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L, 1, 1L))),
+        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L, 1, 1L))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -1691,11 +1793,11 @@ public void testRestore() throws Exception
   {
     final KafkaIndexTask task1 = createTask(
         null,
-        new KafkaIOConfig(
+        new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
-            new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)),
             kafkaServer.consumerProperties(),
             true,
             null,
@@ -1728,11 +1830,11 @@ public void testRestore() throws Exception
     // Start a new task
     final KafkaIndexTask task2 = createTask(
         task1.getId(),
-        new KafkaIOConfig(
+        new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
-            new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)),
             kafkaServer.consumerProperties(),
             true,
             null,
@@ -1766,7 +1868,7 @@ public void testRestore() throws Exception
     SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0);
     Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
     Assert.assertEquals(
-        new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))),
+        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -1780,11 +1882,11 @@ public void testRunWithPauseAndResume() throws Exception
   {
     final KafkaIndexTask task = createTask(
         null,
-        new KafkaIOConfig(
+        new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
-            new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)),
             kafkaServer.consumerProperties(),
             true,
             null,
@@ -1808,7 +1910,7 @@ public void testRunWithPauseAndResume() throws Exception
     }
 
     Assert.assertEquals(2, countEvents(task));
-    Assert.assertEquals(KafkaIndexTask.Status.READING, task.getRunner().getStatus());
+    Assert.assertEquals(Status.READING, task.getRunner().getStatus());
 
     Map<Integer, Long> currentOffsets = objectMapper.readValue(
         task.getRunner().pause().getEntity().toString(),
@@ -1816,7 +1918,7 @@ public void testRunWithPauseAndResume() throws Exception
         {
         }
     );
-    Assert.assertEquals(KafkaIndexTask.Status.PAUSED, task.getRunner().getStatus());
+    Assert.assertEquals(Status.PAUSED, task.getRunner().getStatus());
 
     // Insert remaining data
     try (final KafkaProducer<byte[], byte[]> kafkaProducer = kafkaServer.newProducer()) {
@@ -1850,7 +1952,7 @@ public void testRunWithPauseAndResume() throws Exception
     SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
     Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
     Assert.assertEquals(
-        new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))),
+        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -1864,11 +1966,11 @@ public void testRunWithOffsetOutOfRangeExceptionAndPause() throws Exception
   {
     final KafkaIndexTask task = createTask(
         null,
-        new KafkaIOConfig(
+        new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
-            new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)),
             kafkaServer.consumerProperties(),
             true,
             null,
@@ -1879,13 +1981,13 @@ public void testRunWithOffsetOutOfRangeExceptionAndPause() throws Exception
 
     runTask(task);
 
-    while (!task.getRunner().getStatus().equals(KafkaIndexTask.Status.READING)) {
+    while (!task.getRunner().getStatus().equals(Status.READING)) {
       Thread.sleep(2000);
     }
 
     task.getRunner().pause();
 
-    while (!task.getRunner().getStatus().equals(KafkaIndexTask.Status.PAUSED)) {
+    while (!task.getRunner().getStatus().equals(Status.PAUSED)) {
       Thread.sleep(25);
     }
   }
@@ -1903,11 +2005,11 @@ public void testRunWithOffsetOutOfRangeExceptionAndNextOffsetGreaterThanLeastAva
 
     final KafkaIndexTask task = createTask(
         null,
-        new KafkaIOConfig(
+        new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new KafkaPartitions(topic, ImmutableMap.of(0, 200L)),
-            new KafkaPartitions(topic, ImmutableMap.of(0, 500L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 200L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 500L)),
             kafkaServer.consumerProperties(),
             true,
             null,
@@ -1918,12 +2020,12 @@ public void testRunWithOffsetOutOfRangeExceptionAndNextOffsetGreaterThanLeastAva
 
     runTask(task);
 
-    while (!task.getRunner().getStatus().equals(KafkaIndexTask.Status.READING)) {
+    while (!task.getRunner().getStatus().equals(Status.READING)) {
       Thread.sleep(20);
     }
 
     for (int i = 0; i < 5; i++) {
-      Assert.assertEquals(task.getRunner().getStatus(), KafkaIndexTask.Status.READING);
+      Assert.assertEquals(task.getRunner().getStatus(), Status.READING);
       // Offset should not be reset
       Assert.assertTrue(task.getRunner().getCurrentOffsets().get(0) == 200L);
     }
@@ -1956,12 +2058,12 @@ public void testRunContextSequenceAheadOfStartingOffsets() throws Exception
 
     final KafkaIndexTask task = createTask(
         null,
-        new KafkaIOConfig(
+        new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
             // task should ignore these and use sequence info sent in the context
-            new KafkaPartitions(topic, ImmutableMap.of(0, 0L)),
-            new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L)),
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)),
             kafkaServer.consumerProperties(),
             true,
             null,
@@ -1986,7 +2088,7 @@ public void testRunContextSequenceAheadOfStartingOffsets() throws Exception
     SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
     Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
     Assert.assertEquals(
-        new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))),
+        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -2042,7 +2144,7 @@ public boolean apply(TaskLock lock)
 
   private KafkaIndexTask createTask(
       final String taskId,
-      final KafkaIOConfig ioConfig
+      final KafkaIndexTaskIOConfig ioConfig
   )
   {
     return createTask(taskId, DATA_SCHEMA, ioConfig);
@@ -2050,7 +2152,7 @@ private KafkaIndexTask createTask(
 
   private KafkaIndexTask createTask(
       final String taskId,
-      final KafkaIOConfig ioConfig,
+      final KafkaIndexTaskIOConfig ioConfig,
       final Map<String, Object> context
   )
   {
@@ -2060,10 +2162,10 @@ private KafkaIndexTask createTask(
   private KafkaIndexTask createTask(
       final String taskId,
       final DataSchema dataSchema,
-      final KafkaIOConfig ioConfig
+      final KafkaIndexTaskIOConfig ioConfig
   )
   {
-    final KafkaTuningConfig tuningConfig = new KafkaTuningConfig(
+    final KafkaIndexTaskTuningConfig tuningConfig = new KafkaIndexTaskTuningConfig(
         1000,
         null,
         maxRowsPerSegment,
@@ -2083,7 +2185,10 @@ private KafkaIndexTask createTask(
         maxSavedParseExceptions
     );
     final Map<String, Object> context = isIncrementalHandoffSupported
-                                        ? ImmutableMap.of(KafkaSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED, true)
+                                        ? ImmutableMap.of(
+        SeekableStreamSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED,
+        true
+    )
                                         : null;
     final KafkaIndexTask task = new KafkaIndexTask(
         taskId,
@@ -2105,11 +2210,11 @@ private KafkaIndexTask createTask(
   private KafkaIndexTask createTask(
       final String taskId,
       final DataSchema dataSchema,
-      final KafkaIOConfig ioConfig,
+      final KafkaIndexTaskIOConfig ioConfig,
       final Map<String, Object> context
   )
   {
-    final KafkaTuningConfig tuningConfig = new KafkaTuningConfig(
+    final KafkaIndexTaskTuningConfig tuningConfig = new KafkaIndexTaskTuningConfig(
         1000,
         null,
         maxRowsPerSegment,
@@ -2129,7 +2234,7 @@ private KafkaIndexTask createTask(
         maxSavedParseExceptions
     );
     if (isIncrementalHandoffSupported) {
-      context.put(KafkaSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED, true);
+      context.put(SeekableStreamSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED, true);
     }
 
     final KafkaIndexTask task = new KafkaIndexTask(
@@ -2401,7 +2506,8 @@ private File getSegmentDirectory()
     );
     IndexIO indexIO = new TestUtils().getTestIndexIO();
     QueryableIndex index = indexIO.loadIndex(outputLocation);
-    DictionaryEncodedColumn<String> theColumn = (DictionaryEncodedColumn<String>) index.getColumnHolder(column).getColumn();
+    DictionaryEncodedColumn<String> theColumn = (DictionaryEncodedColumn<String>) index.getColumnHolder(column)
+                                                                                       .getColumn();
     List<String> values = new ArrayList<>();
     for (int i = 0; i < theColumn.length(); i++) {
       int id = theColumn.getSingleValueRow(i);
@@ -2425,7 +2531,7 @@ public long countEvents(final Task task)
                                   .build();
 
     List<Result<TimeseriesResultValue>> results =
-        task.getQueryRunner(query).run(wrap(query), ImmutableMap.of()).toList();
+        task.getQueryRunner(query).run(QueryPlus.wrap(query), ImmutableMap.of()).toList();
 
     return results.isEmpty() ? 0L : DimensionHandlerUtils.nullToZero(results.get(0).getValue().getLongMetric("rows"));
   }
diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaTuningConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java
similarity index 74%
rename from extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaTuningConfigTest.java
rename to extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java
index 69cf186a3c3..abcf07f6c10 100644
--- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaTuningConfigTest.java
+++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java
@@ -21,6 +21,7 @@
 
 import com.fasterxml.jackson.databind.Module;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorTuningConfig;
 import org.apache.druid.jackson.DefaultObjectMapper;
 import org.apache.druid.segment.IndexSpec;
 import org.apache.druid.segment.indexing.TuningConfig;
@@ -30,11 +31,11 @@
 
 import java.io.File;
 
-public class KafkaTuningConfigTest
+public class KafkaIndexTaskTuningConfigTest
 {
   private final ObjectMapper mapper;
 
-  public KafkaTuningConfigTest()
+  public KafkaIndexTaskTuningConfigTest()
   {
     mapper = new DefaultObjectMapper();
     mapper.registerModules((Iterable<Module>) new KafkaIndexTaskModule().getJacksonModules());
@@ -45,7 +46,7 @@ public void testSerdeWithDefaults() throws Exception
   {
     String jsonStr = "{\"type\": \"kafka\"}";
 
-    KafkaTuningConfig config = (KafkaTuningConfig) mapper.readValue(
+    KafkaIndexTaskTuningConfig config = (KafkaIndexTaskTuningConfig) mapper.readValue(
         mapper.writeValueAsString(
             mapper.readValue(
                 jsonStr,
@@ -81,7 +82,7 @@ public void testSerdeWithNonDefaults() throws Exception
                      + "  \"handoffConditionTimeout\": 100\n"
                      + "}";
 
-    KafkaTuningConfig config = (KafkaTuningConfig) mapper.readValue(
+    KafkaIndexTaskTuningConfig config = (KafkaIndexTaskTuningConfig) mapper.readValue(
         mapper.writeValueAsString(
             mapper.readValue(
                 jsonStr,
@@ -97,15 +98,15 @@ public void testSerdeWithNonDefaults() throws Exception
     Assert.assertNotEquals(null, config.getMaxTotalRows());
     Assert.assertEquals(1000, config.getMaxTotalRows().longValue());
     Assert.assertEquals(new Period("PT1H"), config.getIntermediatePersistPeriod());
-    Assert.assertEquals(0, config.getMaxPendingPersists());
+    Assert.assertEquals(100, config.getMaxPendingPersists());
     Assert.assertEquals(true, config.isReportParseExceptions());
     Assert.assertEquals(100, config.getHandoffConditionTimeout());
   }
 
   @Test
-  public void testCopyOf()
+  public void testConvert()
   {
-    KafkaTuningConfig original = new KafkaTuningConfig(
+    KafkaSupervisorTuningConfig original = new KafkaSupervisorTuningConfig(
         1,
         null,
         2,
@@ -122,9 +123,15 @@ public void testCopyOf()
         null,
         null,
         null,
+        null,
+        null,
+        null,
+        null,
+        null,
+        null,
         null
     );
-    KafkaTuningConfig copy = KafkaTuningConfig.copyOf(original);
+    KafkaIndexTaskTuningConfig copy = (KafkaIndexTaskTuningConfig) original.convertToTaskTuningConfig();
 
     Assert.assertEquals(1, copy.getMaxRowsInMemory());
     Assert.assertEquals(2, copy.getMaxRowsPerSegment());
@@ -132,9 +139,32 @@ public void testCopyOf()
     Assert.assertEquals(10L, copy.getMaxTotalRows().longValue());
     Assert.assertEquals(new Period("PT3S"), copy.getIntermediatePersistPeriod());
     Assert.assertEquals(new File("/tmp/xxx"), copy.getBasePersistDirectory());
-    Assert.assertEquals(0, copy.getMaxPendingPersists());
+    Assert.assertEquals(4, copy.getMaxPendingPersists());
     Assert.assertEquals(new IndexSpec(), copy.getIndexSpec());
     Assert.assertEquals(true, copy.isReportParseExceptions());
     Assert.assertEquals(5L, copy.getHandoffConditionTimeout());
   }
+
+  private static KafkaIndexTaskTuningConfig copy(KafkaIndexTaskTuningConfig config)
+  {
+    return new KafkaIndexTaskTuningConfig(
+        config.getMaxRowsInMemory(),
+        config.getMaxBytesInMemory(),
+        config.getMaxRowsPerSegment(),
+        config.getMaxTotalRows(),
+        config.getIntermediatePersistPeriod(),
+        config.getBasePersistDirectory(),
+        0,
+        config.getIndexSpec(),
+        true,
+        config.isReportParseExceptions(),
+        config.getHandoffConditionTimeout(),
+        config.isResetOffsetAutomatically(),
+        config.getSegmentWriteOutMediumFactory(),
+        config.getIntermediateHandoffPeriod(),
+        config.isLogParseExceptions(),
+        config.getMaxParseExceptions(),
+        config.getMaxSavedParseExceptions()
+    );
+  }
 }
diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaRecordSupplierTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaRecordSupplierTest.java
new file mode 100644
index 00000000000..2f445aa9943
--- /dev/null
+++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaRecordSupplierTest.java
@@ -0,0 +1,437 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.indexing.kafka;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Throwables;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import org.apache.curator.test.TestingCluster;
+import org.apache.druid.indexing.kafka.test.TestBroker;
+import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord;
+import org.apache.druid.indexing.seekablestream.common.StreamPartition;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.segment.TestHelper;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+
+public class KafkaRecordSupplierTest
+{
+  private static final Logger log = new Logger(KafkaRecordSupplierTest.class);
+  private static String topic = "topic";
+  private static long poll_timeout_millis = 1000;
+  private static int pollRetry = 5;
+  private static int topicPosFix = 0;
+  private static final ObjectMapper objectMapper = TestHelper.makeJsonMapper();
+
+  private static TestingCluster zkServer;
+  private static TestBroker kafkaServer;
+
+  private List<ProducerRecord<byte[], byte[]>> records;
+
+
+  private static List<ProducerRecord<byte[], byte[]>> generateRecords(String topic)
+  {
+    return ImmutableList.of(
+        new ProducerRecord<>(topic, 0, null, JB("2008", "a", "y", "10", "20.0", "1.0")),
+        new ProducerRecord<>(topic, 0, null, JB("2009", "b", "y", "10", "20.0", "1.0")),
+        new ProducerRecord<>(topic, 0, null, JB("2010", "c", "y", "10", "20.0", "1.0")),
+        new ProducerRecord<>(topic, 0, null, JB("2011", "d", "y", "10", "20.0", "1.0")),
+        new ProducerRecord<>(topic, 0, null, JB("2011", "e", "y", "10", "20.0", "1.0")),
+        new ProducerRecord<>(topic, 0, null, JB("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")),
+        new ProducerRecord<>(topic, 0, null, StringUtils.toUtf8("unparseable")),
+        new ProducerRecord<>(topic, 0, null, StringUtils.toUtf8("unparseable2")),
+        new ProducerRecord<>(topic, 0, null, null),
+        new ProducerRecord<>(topic, 0, null, JB("2013", "f", "y", "10", "20.0", "1.0")),
+        new ProducerRecord<>(topic, 0, null, JB("2049", "f", "y", "notanumber", "20.0", "1.0")),
+        new ProducerRecord<>(topic, 1, null, JB("2049", "f", "y", "10", "notanumber", "1.0")),
+        new ProducerRecord<>(topic, 1, null, JB("2049", "f", "y", "10", "20.0", "notanumber")),
+        new ProducerRecord<>(topic, 1, null, JB("2012", "g", "y", "10", "20.0", "1.0")),
+        new ProducerRecord<>(topic, 1, null, JB("2011", "h", "y", "10", "20.0", "1.0"))
+    );
+  }
+
+  private static byte[] JB(String timestamp, String dim1, String dim2, String dimLong, String dimFloat, String met1)
+  {
+    try {
+      return new ObjectMapper().writeValueAsBytes(
+          ImmutableMap.builder()
+                      .put("timestamp", timestamp)
+                      .put("dim1", dim1)
+                      .put("dim2", dim2)
+                      .put("dimLong", dimLong)
+                      .put("dimFloat", dimFloat)
+                      .put("met1", met1)
+                      .build()
+      );
+    }
+    catch (Exception e) {
+      throw Throwables.propagate(e);
+    }
+  }
+
+  private static String getTopicName()
+  {
+    return "topic-" + topicPosFix++;
+  }
+
+  private List<OrderedPartitionableRecord<Integer, Long>> createOrderedPartitionableRecords()
+  {
+    Map<Integer, Long> partitionToOffset = new HashMap<>();
+    return records.stream().map(r -> {
+      long offset = 0;
+      if (partitionToOffset.containsKey(r.partition())) {
+        offset = partitionToOffset.get(r.partition());
+        partitionToOffset.put(r.partition(), offset + 1);
+      } else {
+        partitionToOffset.put(r.partition(), 1L);
+      }
+      return new OrderedPartitionableRecord<>(
+          topic,
+          r.partition(),
+          offset,
+          r.value() == null ? null : Collections.singletonList(r.value())
+      );
+    }).collect(Collectors.toList());
+  }
+
+  @BeforeClass
+  public static void setupClass() throws Exception
+  {
+    zkServer = new TestingCluster(1);
+    zkServer.start();
+
+    kafkaServer = new TestBroker(
+        zkServer.getConnectString(),
+        null,
+        1,
+        ImmutableMap.of("num.partitions", "2")
+    );
+    kafkaServer.start();
+
+  }
+
+  @Before
+  public void setupTest()
+  {
+    topic = getTopicName();
+    records = generateRecords(topic);
+  }
+
+  @AfterClass
+  public static void tearDownClass() throws Exception
+  {
+    kafkaServer.close();
+    kafkaServer = null;
+
+    zkServer.stop();
+    zkServer = null;
+  }
+
+  @Test
+  public void testSupplierSetup() throws ExecutionException, InterruptedException
+  {
+
+    // Insert data
+    try (final KafkaProducer<byte[], byte[]> kafkaProducer = kafkaServer.newProducer()) {
+      for (ProducerRecord<byte[], byte[]> record : records) {
+        kafkaProducer.send(record).get();
+      }
+    }
+
+    Set<StreamPartition<Integer>> partitions = ImmutableSet.of(
+        StreamPartition.of(topic, 0),
+        StreamPartition.of(topic, 1)
+    );
+
+    KafkaRecordSupplier recordSupplier = new KafkaRecordSupplier(
+        kafkaServer.consumerProperties(), objectMapper);
+
+    Assert.assertTrue(recordSupplier.getAssignment().isEmpty());
+
+    recordSupplier.assign(partitions);
+
+    Assert.assertEquals(partitions, recordSupplier.getAssignment());
+    Assert.assertEquals(ImmutableSet.of(0, 1), recordSupplier.getPartitionIds(topic));
+
+    recordSupplier.close();
+  }
+
+  @Test
+  public void testPoll() throws InterruptedException, ExecutionException
+  {
+
+    // Insert data
+    try (final KafkaProducer<byte[], byte[]> kafkaProducer = kafkaServer.newProducer()) {
+      for (ProducerRecord<byte[], byte[]> record : records) {
+        kafkaProducer.send(record).get();
+      }
+    }
+
+    Set<StreamPartition<Integer>> partitions = ImmutableSet.of(
+        StreamPartition.of(topic, 0),
+        StreamPartition.of(topic, 1)
+    );
+
+    KafkaRecordSupplier recordSupplier = new KafkaRecordSupplier(
+        kafkaServer.consumerProperties(), objectMapper);
+
+    recordSupplier.assign(partitions);
+    recordSupplier.seekToEarliest(partitions);
+
+    List<OrderedPartitionableRecord<Integer, Long>> initialRecords = new ArrayList<>(createOrderedPartitionableRecords());
+
+    List<OrderedPartitionableRecord<Integer, Long>> polledRecords = recordSupplier.poll(poll_timeout_millis);
+    for (int i = 0; polledRecords.size() != initialRecords.size() && i < pollRetry; i++) {
+      polledRecords.addAll(recordSupplier.poll(poll_timeout_millis));
+      Thread.sleep(200);
+    }
+
+    Assert.assertEquals(partitions, recordSupplier.getAssignment());
+    Assert.assertEquals(initialRecords.size(), polledRecords.size());
+    Assert.assertTrue(initialRecords.containsAll(polledRecords));
+
+    recordSupplier.close();
+  }
+
+
+  @Test
+  public void testPollAfterMoreDataAdded() throws InterruptedException, ExecutionException
+  {
+    // Insert data
+
+    KafkaProducer<byte[], byte[]> producer = kafkaServer.newProducer();
+    for (ProducerRecord<byte[], byte[]> record : records.subList(0, 13)) {
+      producer.send(record).get();
+    }
+
+    Set<StreamPartition<Integer>> partitions = ImmutableSet.of(
+        StreamPartition.of(topic, 0),
+        StreamPartition.of(topic, 1)
+    );
+
+
+    KafkaRecordSupplier recordSupplier = new KafkaRecordSupplier(
+        kafkaServer.consumerProperties(), objectMapper);
+
+    recordSupplier.assign(partitions);
+    recordSupplier.seekToEarliest(partitions);
+
+    List<OrderedPartitionableRecord<Integer, Long>> polledRecords = recordSupplier.poll(poll_timeout_millis);
+    for (int i = 0; polledRecords.size() != 13 && i < pollRetry; i++) {
+      polledRecords.addAll(recordSupplier.poll(poll_timeout_millis));
+      Thread.sleep(200);
+    }
+
+    // Insert data
+    for (ProducerRecord<byte[], byte[]> rec : records.subList(13, 15)) {
+      producer.send(rec).get();
+    }
+
+
+    for (int i = 0; polledRecords.size() != records.size() && i < pollRetry; i++) {
+      polledRecords.addAll(recordSupplier.poll(poll_timeout_millis));
+      Thread.sleep(200);
+    }
+
+    List<OrderedPartitionableRecord<Integer, Long>> initialRecords = createOrderedPartitionableRecords();
+
+    Assert.assertEquals(records.size(), polledRecords.size());
+    Assert.assertTrue(initialRecords.containsAll(polledRecords));
+
+
+    recordSupplier.close();
+  }
+
+  @Test
+  public void testSeek() throws InterruptedException, ExecutionException
+  {
+    // Insert data
+    try (final KafkaProducer<byte[], byte[]> kafkaProducer = kafkaServer.newProducer()) {
+      for (ProducerRecord<byte[], byte[]> record : records) {
+        kafkaProducer.send(record).get();
+      }
+    }
+
+    StreamPartition<Integer> partition0 = StreamPartition.of(topic, 0);
+    StreamPartition<Integer> partition1 = StreamPartition.of(topic, 1);
+
+    Set<StreamPartition<Integer>> partitions = ImmutableSet.of(
+        StreamPartition.of(topic, 0),
+        StreamPartition.of(topic, 1)
+    );
+
+    KafkaRecordSupplier recordSupplier = new KafkaRecordSupplier(
+        kafkaServer.consumerProperties(), objectMapper);
+
+    recordSupplier.assign(partitions);
+
+    Assert.assertEquals(0L, (long) recordSupplier.getEarliestSequenceNumber(partition0));
+    Assert.assertEquals(0L, (long) recordSupplier.getEarliestSequenceNumber(partition1));
+
+    recordSupplier.seek(partition0, 2L);
+    recordSupplier.seek(partition1, 2L);
+
+    List<OrderedPartitionableRecord<Integer, Long>> initialRecords = createOrderedPartitionableRecords();
+
+    List<OrderedPartitionableRecord<Integer, Long>> polledRecords = recordSupplier.poll(poll_timeout_millis);
+    for (int i = 0; polledRecords.size() != 11 && i < pollRetry; i++) {
+      polledRecords.addAll(recordSupplier.poll(poll_timeout_millis));
+      Thread.sleep(200);
+    }
+
+
+    Assert.assertEquals(11, polledRecords.size());
+    Assert.assertTrue(initialRecords.containsAll(polledRecords));
+
+
+    recordSupplier.close();
+
+  }
+
+  @Test
+  public void testSeekToLatest() throws InterruptedException, ExecutionException
+  {
+    // Insert data
+    try (final KafkaProducer<byte[], byte[]> kafkaProducer = kafkaServer.newProducer()) {
+      for (ProducerRecord<byte[], byte[]> record : records) {
+        kafkaProducer.send(record).get();
+      }
+    }
+
+    StreamPartition<Integer> partition0 = StreamPartition.of(topic, 0);
+    StreamPartition<Integer> partition1 = StreamPartition.of(topic, 1);
+
+    Set<StreamPartition<Integer>> partitions = ImmutableSet.of(
+        StreamPartition.of(topic, 0),
+        StreamPartition.of(topic, 1)
+    );
+
+    KafkaRecordSupplier recordSupplier = new KafkaRecordSupplier(
+        kafkaServer.consumerProperties(), objectMapper);
+
+    recordSupplier.assign(partitions);
+
+    Assert.assertEquals(0L, (long) recordSupplier.getEarliestSequenceNumber(partition0));
+    Assert.assertEquals(0L, (long) recordSupplier.getEarliestSequenceNumber(partition1));
+
+    recordSupplier.seekToLatest(partitions);
+    List<OrderedPartitionableRecord<Integer, Long>> polledRecords = recordSupplier.poll(poll_timeout_millis);
+
+    Assert.assertEquals(Collections.emptyList(), polledRecords);
+    recordSupplier.close();
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testSeekUnassigned() throws InterruptedException, ExecutionException
+  {
+    // Insert data
+    try (final KafkaProducer<byte[], byte[]> kafkaProducer = kafkaServer.newProducer()) {
+      for (ProducerRecord<byte[], byte[]> record : records) {
+        kafkaProducer.send(record).get();
+      }
+    }
+
+    StreamPartition<Integer> partition0 = StreamPartition.of(topic, 0);
+    StreamPartition<Integer> partition1 = StreamPartition.of(topic, 1);
+
+    Set<StreamPartition<Integer>> partitions = ImmutableSet.of(
+        StreamPartition.of(topic, 0)
+    );
+
+    KafkaRecordSupplier recordSupplier = new KafkaRecordSupplier(
+        kafkaServer.consumerProperties(), objectMapper);
+
+    recordSupplier.assign(partitions);
+
+    Assert.assertEquals(0, (long) recordSupplier.getEarliestSequenceNumber(partition0));
+
+    recordSupplier.seekToEarliest(Collections.singleton(partition1));
+
+    recordSupplier.close();
+  }
+
+  @Test
+  public void testPosition() throws ExecutionException, InterruptedException
+  {
+    // Insert data
+    try (final KafkaProducer<byte[], byte[]> kafkaProducer = kafkaServer.newProducer()) {
+      for (ProducerRecord<byte[], byte[]> record : records) {
+        kafkaProducer.send(record).get();
+      }
+    }
+
+    StreamPartition<Integer> partition0 = StreamPartition.of(topic, 0);
+    StreamPartition<Integer> partition1 = StreamPartition.of(topic, 1);
+
+    Set<StreamPartition<Integer>> partitions = ImmutableSet.of(
+        StreamPartition.of(topic, 0),
+        StreamPartition.of(topic, 1)
+    );
+
+    KafkaRecordSupplier recordSupplier = new KafkaRecordSupplier(
+        kafkaServer.consumerProperties(), objectMapper);
+
+    recordSupplier.assign(partitions);
+
+    Assert.assertEquals(0L, (long) recordSupplier.getPosition(partition0));
+    Assert.assertEquals(0L, (long) recordSupplier.getPosition(partition1));
+
+    recordSupplier.seek(partition0, 4L);
+    recordSupplier.seek(partition1, 5L);
+
+    Assert.assertEquals(4L, (long) recordSupplier.getPosition(partition0));
+    Assert.assertEquals(5L, (long) recordSupplier.getPosition(partition1));
+
+    recordSupplier.seekToEarliest(Collections.singleton(partition0));
+    Assert.assertEquals(0L, (long) recordSupplier.getPosition(partition0));
+
+    recordSupplier.seekToLatest(Collections.singleton(partition0));
+    Assert.assertEquals(11L, (long) recordSupplier.getPosition(partition0));
+
+    long prevPos = recordSupplier.getPosition(partition0);
+    recordSupplier.getEarliestSequenceNumber(partition0);
+    Assert.assertEquals(prevPos, (long) recordSupplier.getPosition(partition0));
+
+    recordSupplier.getLatestSequenceNumber(partition0);
+    Assert.assertEquals(prevPos, (long) recordSupplier.getPosition(partition0));
+
+
+    recordSupplier.close();
+  }
+
+
+}
diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfigTest.java
index e64015adba5..cbf25def6b4 100644
--- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfigTest.java
+++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfigTest.java
@@ -23,8 +23,8 @@
 import com.fasterxml.jackson.databind.Module;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.collect.ImmutableMap;
-import org.apache.druid.indexing.kafka.KafkaIndexTask;
 import org.apache.druid.indexing.kafka.KafkaIndexTaskModule;
+import org.apache.druid.indexing.kafka.KafkaRecordSupplier;
 import org.apache.druid.jackson.DefaultObjectMapper;
 import org.hamcrest.CoreMatchers;
 import org.joda.time.Duration;
@@ -136,7 +136,7 @@ public void testSerdeForConsumerPropertiesWithPasswords() throws Exception
 
     KafkaSupervisorIOConfig config = mapper.readValue(jsonStr, KafkaSupervisorIOConfig.class);
     Properties props = new Properties();
-    KafkaIndexTask.addConsumerPropertiesFromConfig(props, mapper, config.getConsumerProperties());
+    KafkaRecordSupplier.addConsumerPropertiesFromConfig(props, mapper, config.getConsumerProperties());
 
     Assert.assertEquals("my-topic", config.getTopic());
     Assert.assertEquals("localhost:9092", props.getProperty("bootstrap.servers"));
@@ -187,4 +187,5 @@ public void testBootstrapServersRequired() throws Exception
     exception.expectMessage(CoreMatchers.containsString("bootstrap.servers"));
     mapper.readValue(jsonStr, KafkaSupervisorIOConfig.class);
   }
+
 }
diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java
index f08212aa309..427b9d1742d 100644
--- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java
+++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java
@@ -45,12 +45,10 @@
 import org.apache.druid.indexing.common.task.RealtimeIndexTask;
 import org.apache.druid.indexing.common.task.Task;
 import org.apache.druid.indexing.kafka.KafkaDataSourceMetadata;
-import org.apache.druid.indexing.kafka.KafkaIOConfig;
 import org.apache.druid.indexing.kafka.KafkaIndexTask;
 import org.apache.druid.indexing.kafka.KafkaIndexTaskClient;
 import org.apache.druid.indexing.kafka.KafkaIndexTaskClientFactory;
-import org.apache.druid.indexing.kafka.KafkaPartitions;
-import org.apache.druid.indexing.kafka.KafkaTuningConfig;
+import org.apache.druid.indexing.kafka.KafkaIndexTaskIOConfig;
 import org.apache.druid.indexing.kafka.test.TestBroker;
 import org.apache.druid.indexing.overlord.DataSourceMetadata;
 import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
@@ -61,6 +59,9 @@
 import org.apache.druid.indexing.overlord.TaskRunnerWorkItem;
 import org.apache.druid.indexing.overlord.TaskStorage;
 import org.apache.druid.indexing.overlord.supervisor.SupervisorReport;
+import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner.Status;
+import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
+import org.apache.druid.indexing.seekablestream.supervisor.TaskReportData;
 import org.apache.druid.java.util.common.DateTimes;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.StringUtils;
@@ -112,15 +113,11 @@
 import java.util.concurrent.Executor;
 import java.util.concurrent.TimeoutException;
 
-import static org.easymock.EasyMock.anyBoolean;
 import static org.easymock.EasyMock.anyObject;
-import static org.easymock.EasyMock.anyString;
 import static org.easymock.EasyMock.capture;
-import static org.easymock.EasyMock.eq;
 import static org.easymock.EasyMock.expect;
 import static org.easymock.EasyMock.expectLastCall;
 import static org.easymock.EasyMock.replay;
-import static org.easymock.EasyMock.reset;
 
 @RunWith(Parameterized.class)
 public class KafkaSupervisorTest extends EasyMockSupport
@@ -141,6 +138,7 @@
   private static int topicPostfix;
   private static ZkUtils zkUtils;
 
+
   private final int numThreads;
 
   private KafkaSupervisor supervisor;
@@ -282,9 +280,9 @@ public void testNoInitialState() throws Exception
 
     KafkaIndexTask task = captured.getValue();
     Assert.assertEquals(dataSchema, task.getDataSchema());
-    Assert.assertEquals(KafkaTuningConfig.copyOf(tuningConfig), task.getTuningConfig());
+    Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), task.getTuningConfig());
 
-    KafkaIOConfig taskConfig = task.getIOConfig();
+    KafkaIndexTaskIOConfig taskConfig = task.getIOConfig();
     Assert.assertEquals(kafkaHost, taskConfig.getConsumerProperties().get("bootstrap.servers"));
     Assert.assertEquals("myCustomValue", taskConfig.getConsumerProperties().get("myCustomKey"));
     Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName());
@@ -293,15 +291,15 @@ public void testNoInitialState() throws Exception
     Assert.assertFalse("maximumMessageTime", taskConfig.getMaximumMessageTime().isPresent());
     Assert.assertFalse("skipOffsetGaps", taskConfig.isSkipOffsetGaps());
 
-    Assert.assertEquals(topic, taskConfig.getStartPartitions().getTopic());
-    Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(0));
-    Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(1));
-    Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(2));
+    Assert.assertEquals(topic, taskConfig.getStartPartitions().getStream());
+    Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0));
+    Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(1));
+    Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2));
 
-    Assert.assertEquals(topic, taskConfig.getEndPartitions().getTopic());
-    Assert.assertEquals(Long.MAX_VALUE, (long) taskConfig.getEndPartitions().getPartitionOffsetMap().get(0));
-    Assert.assertEquals(Long.MAX_VALUE, (long) taskConfig.getEndPartitions().getPartitionOffsetMap().get(1));
-    Assert.assertEquals(Long.MAX_VALUE, (long) taskConfig.getEndPartitions().getPartitionOffsetMap().get(2));
+    Assert.assertEquals(topic, taskConfig.getEndPartitions().getStream());
+    Assert.assertEquals(Long.MAX_VALUE, (long) taskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(0));
+    Assert.assertEquals(Long.MAX_VALUE, (long) taskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(1));
+    Assert.assertEquals(Long.MAX_VALUE, (long) taskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(2));
   }
 
   @Test
@@ -328,7 +326,7 @@ public void testSkipOffsetGaps() throws Exception
     verifyAll();
 
     KafkaIndexTask task = captured.getValue();
-    KafkaIOConfig taskConfig = task.getIOConfig();
+    KafkaIndexTaskIOConfig taskConfig = task.getIOConfig();
 
     Assert.assertTrue("skipOffsetGaps", taskConfig.isSkipOffsetGaps());
   }
@@ -356,18 +354,27 @@ public void testMultiTask() throws Exception
     verifyAll();
 
     KafkaIndexTask task1 = captured.getValues().get(0);
-    Assert.assertEquals(2, task1.getIOConfig().getStartPartitions().getPartitionOffsetMap().size());
-    Assert.assertEquals(2, task1.getIOConfig().getEndPartitions().getPartitionOffsetMap().size());
-    Assert.assertEquals(0L, (long) task1.getIOConfig().getStartPartitions().getPartitionOffsetMap().get(0));
-    Assert.assertEquals(Long.MAX_VALUE, (long) task1.getIOConfig().getEndPartitions().getPartitionOffsetMap().get(0));
-    Assert.assertEquals(0L, (long) task1.getIOConfig().getStartPartitions().getPartitionOffsetMap().get(2));
-    Assert.assertEquals(Long.MAX_VALUE, (long) task1.getIOConfig().getEndPartitions().getPartitionOffsetMap().get(2));
+    Assert.assertEquals(2, task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().size());
+    Assert.assertEquals(2, task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().size());
+    Assert.assertEquals(0L, (long) task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(0));
+    Assert.assertEquals(
+        Long.MAX_VALUE,
+        (long) task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(0)
+    );
+    Assert.assertEquals(0L, (long) task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(2));
+    Assert.assertEquals(
+        Long.MAX_VALUE,
+        (long) task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(2)
+    );
 
     KafkaIndexTask task2 = captured.getValues().get(1);
-    Assert.assertEquals(1, task2.getIOConfig().getStartPartitions().getPartitionOffsetMap().size());
-    Assert.assertEquals(1, task2.getIOConfig().getEndPartitions().getPartitionOffsetMap().size());
-    Assert.assertEquals(0L, (long) task2.getIOConfig().getStartPartitions().getPartitionOffsetMap().get(1));
-    Assert.assertEquals(Long.MAX_VALUE, (long) task2.getIOConfig().getEndPartitions().getPartitionOffsetMap().get(1));
+    Assert.assertEquals(1, task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().size());
+    Assert.assertEquals(1, task2.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().size());
+    Assert.assertEquals(0L, (long) task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(1));
+    Assert.assertEquals(
+        Long.MAX_VALUE,
+        (long) task2.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(1)
+    );
   }
 
   @Test
@@ -393,18 +400,18 @@ public void testReplicas() throws Exception
     verifyAll();
 
     KafkaIndexTask task1 = captured.getValues().get(0);
-    Assert.assertEquals(3, task1.getIOConfig().getStartPartitions().getPartitionOffsetMap().size());
-    Assert.assertEquals(3, task1.getIOConfig().getEndPartitions().getPartitionOffsetMap().size());
-    Assert.assertEquals(0L, (long) task1.getIOConfig().getStartPartitions().getPartitionOffsetMap().get(0));
-    Assert.assertEquals(0L, (long) task1.getIOConfig().getStartPartitions().getPartitionOffsetMap().get(1));
-    Assert.assertEquals(0L, (long) task1.getIOConfig().getStartPartitions().getPartitionOffsetMap().get(2));
+    Assert.assertEquals(3, task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().size());
+    Assert.assertEquals(3, task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().size());
+    Assert.assertEquals(0L, (long) task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(0));
+    Assert.assertEquals(0L, (long) task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(1));
+    Assert.assertEquals(0L, (long) task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(2));
 
     KafkaIndexTask task2 = captured.getValues().get(1);
-    Assert.assertEquals(3, task2.getIOConfig().getStartPartitions().getPartitionOffsetMap().size());
-    Assert.assertEquals(3, task2.getIOConfig().getEndPartitions().getPartitionOffsetMap().size());
-    Assert.assertEquals(0L, (long) task2.getIOConfig().getStartPartitions().getPartitionOffsetMap().get(0));
-    Assert.assertEquals(0L, (long) task2.getIOConfig().getStartPartitions().getPartitionOffsetMap().get(1));
-    Assert.assertEquals(0L, (long) task2.getIOConfig().getStartPartitions().getPartitionOffsetMap().get(2));
+    Assert.assertEquals(3, task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().size());
+    Assert.assertEquals(3, task2.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().size());
+    Assert.assertEquals(0L, (long) task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(0));
+    Assert.assertEquals(0L, (long) task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(1));
+    Assert.assertEquals(0L, (long) task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(2));
   }
 
   @Test
@@ -511,9 +518,9 @@ public void testLatestOffset() throws Exception
     verifyAll();
 
     KafkaIndexTask task = captured.getValue();
-    Assert.assertEquals(1100L, (long) task.getIOConfig().getStartPartitions().getPartitionOffsetMap().get(0));
-    Assert.assertEquals(1100L, (long) task.getIOConfig().getStartPartitions().getPartitionOffsetMap().get(1));
-    Assert.assertEquals(1100L, (long) task.getIOConfig().getStartPartitions().getPartitionOffsetMap().get(2));
+    Assert.assertEquals(1100L, (long) task.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(0));
+    Assert.assertEquals(1100L, (long) task.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(1));
+    Assert.assertEquals(1100L, (long) task.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(2));
   }
 
   @Test
@@ -532,7 +539,7 @@ public void testDatasourceMetadata() throws Exception
     expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes();
     expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
         new KafkaDataSourceMetadata(
-            new KafkaPartitions(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))
         )
     ).anyTimes();
     expect(taskQueue.add(capture(captured))).andReturn(true);
@@ -543,11 +550,11 @@ public void testDatasourceMetadata() throws Exception
     verifyAll();
 
     KafkaIndexTask task = captured.getValue();
-    KafkaIOConfig taskConfig = task.getIOConfig();
+    KafkaIndexTaskIOConfig taskConfig = task.getIOConfig();
     Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName());
-    Assert.assertEquals(10L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(0));
-    Assert.assertEquals(20L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(1));
-    Assert.assertEquals(30L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(2));
+    Assert.assertEquals(10L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0));
+    Assert.assertEquals(20L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(1));
+    Assert.assertEquals(30L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2));
   }
 
   @Test(expected = ISE.class)
@@ -560,7 +567,7 @@ public void testBadMetadataOffsets() throws Exception
     expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes();
     expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
         new KafkaDataSourceMetadata(
-            new KafkaPartitions(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))
+            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))
         )
     ).anyTimes();
     replayAll();
@@ -580,8 +587,8 @@ public void testKillIncompatibleTasks() throws Exception
         "id1",
         DATASOURCE,
         1,
-        new KafkaPartitions("topic", ImmutableMap.of(0, 0L)),
-        new KafkaPartitions("topic", ImmutableMap.of(0, 10L)),
+        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L)),
+        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 10L)),
         null,
         null
     );
@@ -591,8 +598,8 @@ public void testKillIncompatibleTasks() throws Exception
         "id2",
         DATASOURCE,
         0,
-        new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
-        new KafkaPartitions("topic", ImmutableMap.of(0, 333L, 1, 333L, 2, 333L)),
+        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
+        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 333L, 1, 333L, 2, 333L)),
         null,
         null
     );
@@ -602,8 +609,8 @@ public void testKillIncompatibleTasks() throws Exception
         "id3",
         DATASOURCE,
         1,
-        new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 1L)),
-        new KafkaPartitions("topic", ImmutableMap.of(0, 333L, 1, 333L, 2, 330L)),
+        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 1L)),
+        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 333L, 1, 333L, 2, 330L)),
         null,
         null
     );
@@ -613,8 +620,8 @@ public void testKillIncompatibleTasks() throws Exception
         "id4",
         "other-datasource",
         2,
-        new KafkaPartitions("topic", ImmutableMap.of(0, 0L)),
-        new KafkaPartitions("topic", ImmutableMap.of(0, 10L)),
+        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L)),
+        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 10L)),
         null,
         null
     );
@@ -643,9 +650,12 @@ public void testKillIncompatibleTasks() throws Exception
     expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes();
     expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes();
     expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes();
-    expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.NOT_STARTED))
-                                                  .anyTimes();
-    expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes();
+    expect(taskClient.getStatusAsync(EasyMock.anyString()))
+        .andReturn(Futures.immediateFuture(Status.NOT_STARTED))
+        .anyTimes();
+    expect(taskClient.getStartTimeAsync(EasyMock.anyString()))
+        .andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
+        .anyTimes();
     expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
         new KafkaDataSourceMetadata(
             null
@@ -660,7 +670,7 @@ public void testKillIncompatibleTasks() throws Exception
 
     TreeMap<Integer, Map<Integer, Long>> checkpoints = new TreeMap<>();
     checkpoints.put(0, ImmutableMap.of(0, 0L, 1, 0L, 2, 0L));
-    expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean()))
+    expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean()))
         .andReturn(Futures.immediateFuture(checkpoints))
         .times(2);
 
@@ -681,8 +691,8 @@ public void testKillBadPartitionAssignment() throws Exception
         "id1",
         DATASOURCE,
         0,
-        new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 2, 0L)),
-        new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
+        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 2, 0L)),
+        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
         null,
         null
     );
@@ -690,8 +700,8 @@ public void testKillBadPartitionAssignment() throws Exception
         "id2",
         DATASOURCE,
         1,
-        new KafkaPartitions("topic", ImmutableMap.of(1, 0L)),
-        new KafkaPartitions("topic", ImmutableMap.of(1, Long.MAX_VALUE)),
+        new SeekableStreamPartitions<>("topic", ImmutableMap.of(1, 0L)),
+        new SeekableStreamPartitions<>("topic", ImmutableMap.of(1, Long.MAX_VALUE)),
         null,
         null
     );
@@ -699,8 +709,11 @@ public void testKillBadPartitionAssignment() throws Exception
         "id3",
         DATASOURCE,
         0,
-        new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
-        new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
+        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
+        new SeekableStreamPartitions<>(
+            "topic",
+            ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
+        ),
         null,
         null
     );
@@ -708,8 +721,8 @@ public void testKillBadPartitionAssignment() throws Exception
         "id4",
         DATASOURCE,
         0,
-        new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L)),
-        new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE)),
+        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 1, 0L)),
+        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE)),
         null,
         null
     );
@@ -717,8 +730,8 @@ public void testKillBadPartitionAssignment() throws Exception
         "id5",
         DATASOURCE,
         0,
-        new KafkaPartitions("topic", ImmutableMap.of(1, 0L, 2, 0L)),
-        new KafkaPartitions("topic", ImmutableMap.of(1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
+        new SeekableStreamPartitions<>("topic", ImmutableMap.of(1, 0L, 2, 0L)),
+        new SeekableStreamPartitions<>("topic", ImmutableMap.of(1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
         null,
         null
     );
@@ -739,9 +752,12 @@ public void testKillBadPartitionAssignment() throws Exception
     expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes();
     expect(taskStorage.getTask("id4")).andReturn(Optional.of(id4)).anyTimes();
     expect(taskStorage.getTask("id5")).andReturn(Optional.of(id5)).anyTimes();
-    expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.NOT_STARTED))
-                                                  .anyTimes();
-    expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes();
+    expect(taskClient.getStatusAsync(EasyMock.anyString()))
+        .andReturn(Futures.immediateFuture(Status.NOT_STARTED))
+        .anyTimes();
+    expect(taskClient.getStartTimeAsync(EasyMock.anyString()))
+        .andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
+        .anyTimes();
     expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
         new KafkaDataSourceMetadata(
             null
@@ -755,10 +771,10 @@ public void testKillBadPartitionAssignment() throws Exception
     checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L));
     TreeMap<Integer, Map<Integer, Long>> checkpoints2 = new TreeMap<>();
     checkpoints2.put(0, ImmutableMap.of(1, 0L));
-    expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), anyBoolean()))
+    expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean()))
         .andReturn(Futures.immediateFuture(checkpoints1))
         .times(1);
-    expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean()))
+    expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean()))
         .andReturn(Futures.immediateFuture(checkpoints2))
         .times(1);
 
@@ -783,9 +799,12 @@ public void testRequeueTaskWhenFailed() throws Exception
     expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes();
     expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes();
     expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes();
-    expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.NOT_STARTED))
-                                                  .anyTimes();
-    expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes();
+    expect(taskClient.getStatusAsync(EasyMock.anyString()))
+        .andReturn(Futures.immediateFuture(Status.NOT_STARTED))
+        .anyTimes();
+    expect(taskClient.getStartTimeAsync(EasyMock.anyString()))
+        .andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
+        .anyTimes();
     expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
         new KafkaDataSourceMetadata(
             null
@@ -797,10 +816,10 @@ public void testRequeueTaskWhenFailed() throws Exception
     checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L));
     TreeMap<Integer, Map<Integer, Long>> checkpoints2 = new TreeMap<>();
     checkpoints2.put(0, ImmutableMap.of(1, 0L));
-    expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean()))
+    expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean()))
         .andReturn(Futures.immediateFuture(checkpoints1))
         .anyTimes();
-    expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean()))
+    expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean()))
         .andReturn(Futures.immediateFuture(checkpoints2))
         .anyTimes();
 
@@ -815,10 +834,12 @@ public void testRequeueTaskWhenFailed() throws Exception
 
     // test that running the main loop again checks the status of the tasks that were created and does nothing if they
     // are all still running
-    reset(taskStorage);
+    EasyMock.reset(taskStorage);
     expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes();
     for (Task task : tasks) {
-      expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes();
+      expect(taskStorage.getStatus(task.getId()))
+          .andReturn(Optional.of(TaskStatus.running(task.getId())))
+          .anyTimes();
       expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes();
     }
     replay(taskStorage);
@@ -830,14 +851,17 @@ public void testRequeueTaskWhenFailed() throws Exception
     Capture<Task> aNewTaskCapture = Capture.newInstance();
     List<Task> imStillAlive = tasks.subList(0, 3);
     KafkaIndexTask iHaveFailed = (KafkaIndexTask) tasks.get(3);
-    reset(taskStorage);
-    reset(taskQueue);
+    EasyMock.reset(taskStorage);
+    EasyMock.reset(taskQueue);
     expect(taskStorage.getActiveTasks()).andReturn(imStillAlive).anyTimes();
     for (Task task : imStillAlive) {
-      expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes();
+      expect(taskStorage.getStatus(task.getId()))
+          .andReturn(Optional.of(TaskStatus.running(task.getId())))
+          .anyTimes();
       expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes();
     }
-    expect(taskStorage.getStatus(iHaveFailed.getId())).andReturn(Optional.of(TaskStatus.failure(iHaveFailed.getId())));
+    expect(taskStorage.getStatus(iHaveFailed.getId()))
+        .andReturn(Optional.of(TaskStatus.failure(iHaveFailed.getId())));
     expect(taskStorage.getTask(iHaveFailed.getId())).andReturn(Optional.of((Task) iHaveFailed)).anyTimes();
     expect(taskQueue.add(capture(aNewTaskCapture))).andReturn(true);
     replay(taskStorage);
@@ -865,8 +889,8 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception
         "id1",
         DATASOURCE,
         0,
-        new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 2, 0L)),
-        new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
+        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 2, 0L)),
+        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
         now,
         maxi
     );
@@ -880,7 +904,7 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception
     expect(taskStorage.getActiveTasks()).andReturn(existingTasks).anyTimes();
     expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes();
     expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes();
-    expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING));
+    expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.READING));
     expect(taskClient.getStartTimeAsync("id1")).andReturn(Futures.immediateFuture(now)).anyTimes();
     expect(taskQueue.add(capture(captured))).andReturn(true);
     expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
@@ -891,7 +915,7 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception
 
     TreeMap<Integer, Map<Integer, Long>> checkpoints = new TreeMap<>();
     checkpoints.put(0, ImmutableMap.of(0, 0L, 2, 0L));
-    expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), anyBoolean()))
+    expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean()))
         .andReturn(Futures.immediateFuture(checkpoints))
         .times(2);
 
@@ -909,24 +933,27 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception
     String runningTaskId = captured.getValue().getId();
     Capture<Task> aNewTaskCapture = Capture.newInstance();
     KafkaIndexTask iHaveFailed = (KafkaIndexTask) existingTasks.get(0);
-    reset(taskStorage);
-    reset(taskQueue);
-    reset(taskClient);
+    EasyMock.reset(taskStorage);
+    EasyMock.reset(taskQueue);
+    EasyMock.reset(taskClient);
 
     // for the newly created replica task
-    expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean()))
+    expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean()))
         .andReturn(Futures.immediateFuture(checkpoints))
         .times(2);
-    expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), anyBoolean()))
+    expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean()))
         .andReturn(Futures.immediateFuture(checkpoints))
         .times(1);
 
     expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(captured.getValue())).anyTimes();
-    expect(taskStorage.getStatus(iHaveFailed.getId())).andReturn(Optional.of(TaskStatus.failure(iHaveFailed.getId())));
-    expect(taskStorage.getStatus(runningTaskId)).andReturn(Optional.of(TaskStatus.running(runningTaskId))).anyTimes();
+    expect(taskStorage.getStatus(iHaveFailed.getId()))
+        .andReturn(Optional.of(TaskStatus.failure(iHaveFailed.getId())));
+    expect(taskStorage.getStatus(runningTaskId))
+        .andReturn(Optional.of(TaskStatus.running(runningTaskId)))
+        .anyTimes();
     expect(taskStorage.getTask(iHaveFailed.getId())).andReturn(Optional.of((Task) iHaveFailed)).anyTimes();
     expect(taskStorage.getTask(runningTaskId)).andReturn(Optional.of(captured.getValue())).anyTimes();
-    expect(taskClient.getStatusAsync(runningTaskId)).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING));
+    expect(taskClient.getStatusAsync(runningTaskId)).andReturn(Futures.immediateFuture(Status.READING));
     expect(taskClient.getStartTimeAsync(runningTaskId)).andReturn(Futures.immediateFuture(now)).anyTimes();
     expect(taskQueue.add(capture(aNewTaskCapture))).andReturn(true);
     replay(taskStorage);
@@ -962,9 +989,12 @@ public void testQueueNextTasksOnSuccess() throws Exception
     expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes();
     expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes();
     expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes();
-    expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.NOT_STARTED))
-                                                  .anyTimes();
-    expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes();
+    expect(taskClient.getStatusAsync(EasyMock.anyString()))
+        .andReturn(Futures.immediateFuture(Status.NOT_STARTED))
+        .anyTimes();
+    expect(taskClient.getStartTimeAsync(EasyMock.anyString()))
+        .andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
+        .anyTimes();
     expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
         new KafkaDataSourceMetadata(
             null
@@ -981,27 +1011,32 @@ public void testQueueNextTasksOnSuccess() throws Exception
 
     List<Task> tasks = captured.getValues();
 
-    reset(taskStorage);
-    reset(taskClient);
+    EasyMock.reset(taskStorage);
+    EasyMock.reset(taskClient);
 
-    expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.NOT_STARTED))
-                                                  .anyTimes();
-    expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes();
+    expect(taskClient.getStatusAsync(EasyMock.anyString()))
+        .andReturn(Futures.immediateFuture(Status.NOT_STARTED))
+        .anyTimes();
+    expect(taskClient.getStartTimeAsync(EasyMock.anyString()))
+        .andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
+        .anyTimes();
     TreeMap<Integer, Map<Integer, Long>> checkpoints1 = new TreeMap<>();
     checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L));
     TreeMap<Integer, Map<Integer, Long>> checkpoints2 = new TreeMap<>();
     checkpoints2.put(0, ImmutableMap.of(1, 0L));
     // there would be 4 tasks, 2 for each task group
-    expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean()))
+    expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean()))
         .andReturn(Futures.immediateFuture(checkpoints1))
         .times(2);
-    expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean()))
+    expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean()))
         .andReturn(Futures.immediateFuture(checkpoints2))
         .times(2);
 
     expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes();
     for (Task task : tasks) {
-      expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes();
+      expect(taskStorage.getStatus(task.getId()))
+          .andReturn(Optional.of(TaskStatus.running(task.getId())))
+          .anyTimes();
       expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes();
     }
     replay(taskStorage);
@@ -1016,18 +1051,22 @@ public void testQueueNextTasksOnSuccess() throws Exception
     Capture<String> shutdownTaskIdCapture = Capture.newInstance();
     List<Task> imStillRunning = tasks.subList(1, 4);
     KafkaIndexTask iAmSuccess = (KafkaIndexTask) tasks.get(0);
-    reset(taskStorage);
-    reset(taskQueue);
-    reset(taskClient);
+    EasyMock.reset(taskStorage);
+    EasyMock.reset(taskQueue);
+    EasyMock.reset(taskClient);
     expect(taskStorage.getActiveTasks()).andReturn(imStillRunning).anyTimes();
     for (Task task : imStillRunning) {
-      expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes();
+      expect(taskStorage.getStatus(task.getId()))
+          .andReturn(Optional.of(TaskStatus.running(task.getId())))
+          .anyTimes();
       expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes();
     }
-    expect(taskStorage.getStatus(iAmSuccess.getId())).andReturn(Optional.of(TaskStatus.success(iAmSuccess.getId())));
+    expect(taskStorage.getStatus(iAmSuccess.getId()))
+        .andReturn(Optional.of(TaskStatus.success(iAmSuccess.getId())));
     expect(taskStorage.getTask(iAmSuccess.getId())).andReturn(Optional.of((Task) iAmSuccess)).anyTimes();
     expect(taskQueue.add(capture(newTasksCapture))).andReturn(true).times(2);
-    expect(taskClient.stopAsync(capture(shutdownTaskIdCapture), eq(false))).andReturn(Futures.immediateFuture(true));
+    expect(taskClient.stopAsync(capture(shutdownTaskIdCapture), EasyMock.eq(false)))
+        .andReturn(Futures.immediateFuture(true));
     replay(taskStorage);
     replay(taskQueue);
     replay(taskClient);
@@ -1071,16 +1110,18 @@ public void testBeginPublishAndQueueNextTasks() throws Exception
       workItems.add(new TestTaskRunnerWorkItem(task, null, location));
     }
 
-    reset(taskStorage, taskRunner, taskClient, taskQueue);
+    EasyMock.reset(taskStorage, taskRunner, taskClient, taskQueue);
     captured = Capture.newInstance(CaptureType.ALL);
     expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes();
     for (Task task : tasks) {
-      expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes();
+      expect(taskStorage.getStatus(task.getId()))
+          .andReturn(Optional.of(TaskStatus.running(task.getId())))
+          .anyTimes();
       expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes();
     }
     expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes();
-    expect(taskClient.getStatusAsync(anyString()))
-        .andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING))
+    expect(taskClient.getStatusAsync(EasyMock.anyString()))
+        .andReturn(Futures.immediateFuture(Status.READING))
         .anyTimes();
     expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0")))
         .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2)))
@@ -1104,10 +1145,10 @@ public void testBeginPublishAndQueueNextTasks() throws Exception
     checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L));
     TreeMap<Integer, Map<Integer, Long>> checkpoints2 = new TreeMap<>();
     checkpoints2.put(0, ImmutableMap.of(1, 0L));
-    expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean()))
+    expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean()))
         .andReturn(Futures.immediateFuture(checkpoints1))
         .times(2);
-    expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean()))
+    expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean()))
         .andReturn(Futures.immediateFuture(checkpoints2))
         .times(2);
 
@@ -1119,16 +1160,16 @@ public void testBeginPublishAndQueueNextTasks() throws Exception
     for (Task task : captured.getValues()) {
       KafkaIndexTask kafkaIndexTask = (KafkaIndexTask) task;
       Assert.assertEquals(dataSchema, kafkaIndexTask.getDataSchema());
-      Assert.assertEquals(KafkaTuningConfig.copyOf(tuningConfig), kafkaIndexTask.getTuningConfig());
+      Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), kafkaIndexTask.getTuningConfig());
 
-      KafkaIOConfig taskConfig = kafkaIndexTask.getIOConfig();
+      KafkaIndexTaskIOConfig taskConfig = kafkaIndexTask.getIOConfig();
       Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName());
       Assert.assertTrue("isUseTransaction", taskConfig.isUseTransaction());
 
-      Assert.assertEquals(topic, taskConfig.getStartPartitions().getTopic());
-      Assert.assertEquals(10L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(0));
-      Assert.assertEquals(20L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(1));
-      Assert.assertEquals(35L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(2));
+      Assert.assertEquals(topic, taskConfig.getStartPartitions().getStream());
+      Assert.assertEquals(10L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0));
+      Assert.assertEquals(20L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(1));
+      Assert.assertEquals(35L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2));
     }
   }
 
@@ -1144,8 +1185,11 @@ public void testDiscoverExistingPublishingTask() throws Exception
         "id1",
         DATASOURCE,
         0,
-        new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
-        new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
+        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
+        new SeekableStreamPartitions<>(
+            "topic",
+            ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
+        ),
         null,
         null
     );
@@ -1165,7 +1209,7 @@ public void testDiscoverExistingPublishingTask() throws Exception
             null
         )
     ).anyTimes();
-    expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.PUBLISHING));
+    expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING));
     expect(taskClient.getCurrentOffsetsAsync("id1", false))
         .andReturn(Futures.immediateFuture((Map<Integer, Long>) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)));
     expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L));
@@ -1173,7 +1217,9 @@ public void testDiscoverExistingPublishingTask() throws Exception
 
     TreeMap<Integer, Map<Integer, Long>> checkpoints = new TreeMap<>();
     checkpoints.put(0, ImmutableMap.of(0, 0L, 1, 0L, 2, 0L));
-    expect(taskClient.getCheckpoints(anyString(), anyBoolean())).andReturn(checkpoints).anyTimes();
+    expect(taskClient.getCheckpoints(EasyMock.anyString(), EasyMock.anyBoolean()))
+        .andReturn(checkpoints)
+        .anyTimes();
 
     taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class));
     replayAll();
@@ -1192,7 +1238,7 @@ public void testDiscoverExistingPublishingTask() throws Exception
     Assert.assertEquals(3600L, (long) payload.getDurationSeconds());
     Assert.assertEquals(NUM_PARTITIONS, (int) payload.getPartitions());
     Assert.assertEquals(1, (int) payload.getReplicas());
-    Assert.assertEquals(topic, payload.getTopic());
+    Assert.assertEquals(topic, payload.getStream());
     Assert.assertEquals(0, payload.getActiveTasks().size());
     Assert.assertEquals(1, payload.getPublishingTasks().size());
 
@@ -1204,24 +1250,33 @@ public void testDiscoverExistingPublishingTask() throws Exception
 
     KafkaIndexTask capturedTask = captured.getValue();
     Assert.assertEquals(dataSchema, capturedTask.getDataSchema());
-    Assert.assertEquals(KafkaTuningConfig.copyOf(tuningConfig), capturedTask.getTuningConfig());
+    Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), capturedTask.getTuningConfig());
 
-    KafkaIOConfig capturedTaskConfig = capturedTask.getIOConfig();
+    KafkaIndexTaskIOConfig capturedTaskConfig = capturedTask.getIOConfig();
     Assert.assertEquals(kafkaHost, capturedTaskConfig.getConsumerProperties().get("bootstrap.servers"));
     Assert.assertEquals("myCustomValue", capturedTaskConfig.getConsumerProperties().get("myCustomKey"));
     Assert.assertEquals("sequenceName-0", capturedTaskConfig.getBaseSequenceName());
     Assert.assertTrue("isUseTransaction", capturedTaskConfig.isUseTransaction());
 
     // check that the new task was created with starting offsets matching where the publishing task finished
-    Assert.assertEquals(topic, capturedTaskConfig.getStartPartitions().getTopic());
-    Assert.assertEquals(10L, (long) capturedTaskConfig.getStartPartitions().getPartitionOffsetMap().get(0));
-    Assert.assertEquals(20L, (long) capturedTaskConfig.getStartPartitions().getPartitionOffsetMap().get(1));
-    Assert.assertEquals(30L, (long) capturedTaskConfig.getStartPartitions().getPartitionOffsetMap().get(2));
-
-    Assert.assertEquals(topic, capturedTaskConfig.getEndPartitions().getTopic());
-    Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionOffsetMap().get(0));
-    Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionOffsetMap().get(1));
-    Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionOffsetMap().get(2));
+    Assert.assertEquals(topic, capturedTaskConfig.getStartPartitions().getStream());
+    Assert.assertEquals(10L, (long) capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0));
+    Assert.assertEquals(20L, (long) capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(1));
+    Assert.assertEquals(30L, (long) capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2));
+
+    Assert.assertEquals(topic, capturedTaskConfig.getEndPartitions().getStream());
+    Assert.assertEquals(
+        Long.MAX_VALUE,
+        (long) capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(0)
+    );
+    Assert.assertEquals(
+        Long.MAX_VALUE,
+        (long) capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(1)
+    );
+    Assert.assertEquals(
+        Long.MAX_VALUE,
+        (long) capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(2)
+    );
   }
 
   @Test
@@ -1236,8 +1291,8 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation()
         "id1",
         DATASOURCE,
         0,
-        new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 2, 0L)),
-        new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
+        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 2, 0L)),
+        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
         null,
         null
     );
@@ -1257,7 +1312,7 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation()
             null
         )
     ).anyTimes();
-    expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.PUBLISHING));
+    expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING));
     expect(taskClient.getCurrentOffsetsAsync("id1", false))
         .andReturn(Futures.immediateFuture((Map<Integer, Long>) ImmutableMap.of(0, 10L, 2, 30L)));
     expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 2, 30L));
@@ -1280,7 +1335,7 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation()
     Assert.assertEquals(3600L, (long) payload.getDurationSeconds());
     Assert.assertEquals(NUM_PARTITIONS, (int) payload.getPartitions());
     Assert.assertEquals(1, (int) payload.getReplicas());
-    Assert.assertEquals(topic, payload.getTopic());
+    Assert.assertEquals(topic, payload.getStream());
     Assert.assertEquals(0, payload.getActiveTasks().size());
     Assert.assertEquals(1, payload.getPublishingTasks().size());
 
@@ -1292,24 +1347,33 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation()
 
     KafkaIndexTask capturedTask = captured.getValue();
     Assert.assertEquals(dataSchema, capturedTask.getDataSchema());
-    Assert.assertEquals(KafkaTuningConfig.copyOf(tuningConfig), capturedTask.getTuningConfig());
+    Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), capturedTask.getTuningConfig());
 
-    KafkaIOConfig capturedTaskConfig = capturedTask.getIOConfig();
+    KafkaIndexTaskIOConfig capturedTaskConfig = capturedTask.getIOConfig();
     Assert.assertEquals(kafkaHost, capturedTaskConfig.getConsumerProperties().get("bootstrap.servers"));
     Assert.assertEquals("myCustomValue", capturedTaskConfig.getConsumerProperties().get("myCustomKey"));
     Assert.assertEquals("sequenceName-0", capturedTaskConfig.getBaseSequenceName());
     Assert.assertTrue("isUseTransaction", capturedTaskConfig.isUseTransaction());
 
     // check that the new task was created with starting offsets matching where the publishing task finished
-    Assert.assertEquals(topic, capturedTaskConfig.getStartPartitions().getTopic());
-    Assert.assertEquals(10L, (long) capturedTaskConfig.getStartPartitions().getPartitionOffsetMap().get(0));
-    Assert.assertEquals(0L, (long) capturedTaskConfig.getStartPartitions().getPartitionOffsetMap().get(1));
-    Assert.assertEquals(30L, (long) capturedTaskConfig.getStartPartitions().getPartitionOffsetMap().get(2));
-
-    Assert.assertEquals(topic, capturedTaskConfig.getEndPartitions().getTopic());
-    Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionOffsetMap().get(0));
-    Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionOffsetMap().get(1));
-    Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionOffsetMap().get(2));
+    Assert.assertEquals(topic, capturedTaskConfig.getStartPartitions().getStream());
+    Assert.assertEquals(10L, (long) capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0));
+    Assert.assertEquals(0L, (long) capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(1));
+    Assert.assertEquals(30L, (long) capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2));
+
+    Assert.assertEquals(topic, capturedTaskConfig.getEndPartitions().getStream());
+    Assert.assertEquals(
+        Long.MAX_VALUE,
+        (long) capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(0)
+    );
+    Assert.assertEquals(
+        Long.MAX_VALUE,
+        (long) capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(1)
+    );
+    Assert.assertEquals(
+        Long.MAX_VALUE,
+        (long) capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(2)
+    );
   }
 
   @Test
@@ -1326,8 +1390,11 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception
         "id1",
         DATASOURCE,
         0,
-        new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
-        new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
+        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
+        new SeekableStreamPartitions<>(
+            "topic",
+            ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
+        ),
         null,
         null
     );
@@ -1336,8 +1403,11 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception
         "id2",
         DATASOURCE,
         0,
-        new KafkaPartitions("topic", ImmutableMap.of(0, 1L, 1, 2L, 2, 3L)),
-        new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
+        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 1L, 1, 2L, 2, 3L)),
+        new SeekableStreamPartitions<>(
+            "topic",
+            ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
+        ),
         null,
         null
     );
@@ -1359,8 +1429,8 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception
             null
         )
     ).anyTimes();
-    expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.PUBLISHING));
-    expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING));
+    expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING));
+    expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING));
     expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime));
     expect(taskClient.getCurrentOffsetsAsync("id1", false))
         .andReturn(Futures.immediateFuture((Map<Integer, Long>) ImmutableMap.of(0, 1L, 1, 2L, 2, 3L)));
@@ -1373,7 +1443,7 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception
     // since id1 is publishing, so getCheckpoints wouldn't be called for it
     TreeMap<Integer, Map<Integer, Long>> checkpoints = new TreeMap<>();
     checkpoints.put(0, ImmutableMap.of(0, 1L, 1, 2L, 2, 3L));
-    expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean()))
+    expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean()))
         .andReturn(Futures.immediateFuture(checkpoints))
         .times(1);
 
@@ -1393,7 +1463,7 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception
     Assert.assertEquals(3600L, (long) payload.getDurationSeconds());
     Assert.assertEquals(NUM_PARTITIONS, (int) payload.getPartitions());
     Assert.assertEquals(1, (int) payload.getReplicas());
-    Assert.assertEquals(topic, payload.getTopic());
+    Assert.assertEquals(topic, payload.getStream());
     Assert.assertEquals(1, payload.getActiveTasks().size());
     Assert.assertEquals(1, payload.getPublishingTasks().size());
 
@@ -1443,25 +1513,27 @@ public void testKillUnresponsiveTasksWhileGettingStartTime() throws Exception
 
     List<Task> tasks = captured.getValues();
 
-    reset(taskStorage, taskClient, taskQueue);
+    EasyMock.reset(taskStorage, taskClient, taskQueue);
 
     TreeMap<Integer, Map<Integer, Long>> checkpoints1 = new TreeMap<>();
     checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L));
     TreeMap<Integer, Map<Integer, Long>> checkpoints2 = new TreeMap<>();
     checkpoints2.put(0, ImmutableMap.of(1, 0L));
-    expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean()))
+    expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean()))
         .andReturn(Futures.immediateFuture(checkpoints1))
         .times(2);
-    expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean()))
+    expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean()))
         .andReturn(Futures.immediateFuture(checkpoints2))
         .times(2);
 
     expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes();
     for (Task task : tasks) {
-      expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes();
+      expect(taskStorage.getStatus(task.getId()))
+          .andReturn(Optional.of(TaskStatus.running(task.getId())))
+          .anyTimes();
       expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes();
       expect(taskClient.getStatusAsync(task.getId()))
-          .andReturn(Futures.immediateFuture(KafkaIndexTask.Status.NOT_STARTED));
+          .andReturn(Futures.immediateFuture(Status.NOT_STARTED));
       expect(taskClient.getStartTimeAsync(task.getId()))
           .andReturn(Futures.immediateFailedFuture(new RuntimeException()));
       taskQueue.shutdown(task.getId(), "Task [%s] failed to return start time, killing task", task.getId());
@@ -1504,28 +1576,30 @@ public void testKillUnresponsiveTasksWhilePausing() throws Exception
       workItems.add(new TestTaskRunnerWorkItem(task, null, location));
     }
 
-    reset(taskStorage, taskRunner, taskClient, taskQueue);
+    EasyMock.reset(taskStorage, taskRunner, taskClient, taskQueue);
 
     TreeMap<Integer, Map<Integer, Long>> checkpoints1 = new TreeMap<>();
     checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L));
     TreeMap<Integer, Map<Integer, Long>> checkpoints2 = new TreeMap<>();
     checkpoints2.put(0, ImmutableMap.of(1, 0L));
-    expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean()))
+    expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean()))
         .andReturn(Futures.immediateFuture(checkpoints1))
         .times(2);
-    expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean()))
+    expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean()))
         .andReturn(Futures.immediateFuture(checkpoints2))
         .times(2);
 
     captured = Capture.newInstance(CaptureType.ALL);
     expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes();
     for (Task task : tasks) {
-      expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes();
+      expect(taskStorage.getStatus(task.getId()))
+          .andReturn(Optional.of(TaskStatus.running(task.getId())))
+          .anyTimes();
       expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes();
     }
     expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes();
-    expect(taskClient.getStatusAsync(anyString()))
-        .andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING))
+    expect(taskClient.getStatusAsync(EasyMock.anyString()))
+        .andReturn(Futures.immediateFuture(Status.READING))
         .anyTimes();
     expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0")))
         .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2)))
@@ -1550,9 +1624,9 @@ public void testKillUnresponsiveTasksWhilePausing() throws Exception
     verifyAll();
 
     for (Task task : captured.getValues()) {
-      KafkaIOConfig taskConfig = ((KafkaIndexTask) task).getIOConfig();
-      Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(0));
-      Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(2));
+      KafkaIndexTaskIOConfig taskConfig = ((KafkaIndexTask) task).getIOConfig();
+      Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0));
+      Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2));
     }
   }
 
@@ -1588,28 +1662,30 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception
       workItems.add(new TestTaskRunnerWorkItem(task, null, location));
     }
 
-    reset(taskStorage, taskRunner, taskClient, taskQueue);
+    EasyMock.reset(taskStorage, taskRunner, taskClient, taskQueue);
 
     TreeMap<Integer, Map<Integer, Long>> checkpoints1 = new TreeMap<>();
     checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L));
     TreeMap<Integer, Map<Integer, Long>> checkpoints2 = new TreeMap<>();
     checkpoints2.put(0, ImmutableMap.of(1, 0L));
-    expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean()))
+    expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean()))
         .andReturn(Futures.immediateFuture(checkpoints1))
         .times(2);
-    expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean()))
+    expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean()))
         .andReturn(Futures.immediateFuture(checkpoints2))
         .times(2);
 
     captured = Capture.newInstance(CaptureType.ALL);
     expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes();
     for (Task task : tasks) {
-      expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes();
+      expect(taskStorage.getStatus(task.getId()))
+          .andReturn(Optional.of(TaskStatus.running(task.getId())))
+          .anyTimes();
       expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes();
     }
     expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes();
-    expect(taskClient.getStatusAsync(anyString()))
-        .andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING))
+    expect(taskClient.getStatusAsync(EasyMock.anyString()))
+        .andReturn(Futures.immediateFuture(Status.READING))
         .anyTimes();
     expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0")))
         .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2)))
@@ -1641,9 +1717,9 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception
     verifyAll();
 
     for (Task task : captured.getValues()) {
-      KafkaIOConfig taskConfig = ((KafkaIndexTask) task).getIOConfig();
-      Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(0));
-      Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(2));
+      KafkaIndexTaskIOConfig taskConfig = ((KafkaIndexTask) task).getIOConfig();
+      Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0));
+      Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2));
     }
   }
 
@@ -1683,8 +1759,11 @@ public void testStopGracefully() throws Exception
         "id1",
         DATASOURCE,
         0,
-        new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
-        new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
+        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
+        new SeekableStreamPartitions<>(
+            "topic",
+            ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
+        ),
         null,
         null
     );
@@ -1693,8 +1772,11 @@ public void testStopGracefully() throws Exception
         "id2",
         DATASOURCE,
         0,
-        new KafkaPartitions("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
-        new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
+        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
+        new SeekableStreamPartitions<>(
+            "topic",
+            ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
+        ),
         null,
         null
     );
@@ -1703,8 +1785,11 @@ public void testStopGracefully() throws Exception
         "id3",
         DATASOURCE,
         0,
-        new KafkaPartitions("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
-        new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
+        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
+        new SeekableStreamPartitions<>(
+            "topic",
+            ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
+        ),
         null,
         null
     );
@@ -1728,9 +1813,9 @@ public void testStopGracefully() throws Exception
             null
         )
     ).anyTimes();
-    expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.PUBLISHING));
-    expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING));
-    expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING));
+    expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING));
+    expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING));
+    expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(Status.READING));
     expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime));
     expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime));
     expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L));
@@ -1738,10 +1823,10 @@ public void testStopGracefully() throws Exception
     // getCheckpoints will not be called for id1 as it is in publishing state
     TreeMap<Integer, Map<Integer, Long>> checkpoints = new TreeMap<>();
     checkpoints.put(0, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L));
-    expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean()))
+    expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean()))
         .andReturn(Futures.immediateFuture(checkpoints))
         .times(1);
-    expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), anyBoolean()))
+    expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean()))
         .andReturn(Futures.immediateFuture(checkpoints))
         .times(1);
 
@@ -1752,7 +1837,7 @@ public void testStopGracefully() throws Exception
     supervisor.runInternal();
     verifyAll();
 
-    reset(taskRunner, taskClient, taskQueue);
+    EasyMock.reset(taskRunner, taskClient, taskQueue);
     expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes();
     expect(taskClient.pauseAsync("id2"))
         .andReturn(Futures.immediateFuture((Map<Integer, Long>) ImmutableMap.of(0, 15L, 1, 25L, 2, 30L)));
@@ -1784,7 +1869,7 @@ public void testResetNoTasks() throws Exception
     supervisor.runInternal();
     verifyAll();
 
-    reset(indexerMetadataStorageCoordinator);
+    EasyMock.reset(indexerMetadataStorageCoordinator);
     expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true);
     replay(indexerMetadataStorageCoordinator);
 
@@ -1811,26 +1896,27 @@ public void testResetDataSourceMetadata() throws Exception
     Capture<String> captureDataSource = EasyMock.newCapture();
     Capture<DataSourceMetadata> captureDataSourceMetadata = EasyMock.newCapture();
 
-    KafkaDataSourceMetadata kafkaDataSourceMetadata = new KafkaDataSourceMetadata(new KafkaPartitions(
+    KafkaDataSourceMetadata kafkaDataSourceMetadata = new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(
         topic,
         ImmutableMap.of(0, 1000L, 1, 1000L, 2, 1000L)
     ));
 
-    KafkaDataSourceMetadata resetMetadata = new KafkaDataSourceMetadata(new KafkaPartitions(
+    KafkaDataSourceMetadata resetMetadata = new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(
         topic,
         ImmutableMap.of(1, 1000L, 2, 1000L)
     ));
 
-    KafkaDataSourceMetadata expectedMetadata = new KafkaDataSourceMetadata(new KafkaPartitions(
+    KafkaDataSourceMetadata expectedMetadata = new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(
         topic,
         ImmutableMap.of(0, 1000L)
     ));
 
-    reset(indexerMetadataStorageCoordinator);
-    expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(kafkaDataSourceMetadata);
+    EasyMock.reset(indexerMetadataStorageCoordinator);
+    expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE))
+        .andReturn(kafkaDataSourceMetadata);
     expect(indexerMetadataStorageCoordinator.resetDataSourceMetadata(
-        EasyMock.capture(captureDataSource),
-        EasyMock.capture(captureDataSourceMetadata)
+        capture(captureDataSource),
+        capture(captureDataSourceMetadata)
     )).andReturn(true);
     replay(indexerMetadataStorageCoordinator);
 
@@ -1838,7 +1924,7 @@ public void testResetDataSourceMetadata() throws Exception
       supervisor.resetInternal(resetMetadata);
     }
     catch (NullPointerException npe) {
-      // Expected as there will be an attempt to reset partitionGroups offsets to NOT_SET
+      // Expected as there will be an attempt to EasyMock.reset partitionGroups offsets to NOT_SET
       // however there would be no entries in the map as we have not put nay data in kafka
       Assert.assertTrue(npe.getCause() == null);
     }
@@ -1863,12 +1949,12 @@ public void testResetNoDataSourceMetadata() throws Exception
     supervisor.runInternal();
     verifyAll();
 
-    KafkaDataSourceMetadata resetMetadata = new KafkaDataSourceMetadata(new KafkaPartitions(
+    KafkaDataSourceMetadata resetMetadata = new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(
         topic,
         ImmutableMap.of(1, 1000L, 2, 1000L)
     ));
 
-    reset(indexerMetadataStorageCoordinator);
+    EasyMock.reset(indexerMetadataStorageCoordinator);
     // no DataSourceMetadata in metadata store
     expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(null);
     replay(indexerMetadataStorageCoordinator);
@@ -1891,8 +1977,11 @@ public void testResetRunningTasks() throws Exception
         "id1",
         DATASOURCE,
         0,
-        new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
-        new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
+        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
+        new SeekableStreamPartitions<>(
+            "topic",
+            ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
+        ),
         null,
         null
     );
@@ -1901,8 +1990,11 @@ public void testResetRunningTasks() throws Exception
         "id2",
         DATASOURCE,
         0,
-        new KafkaPartitions("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
-        new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
+        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
+        new SeekableStreamPartitions<>(
+            "topic",
+            ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
+        ),
         null,
         null
     );
@@ -1911,8 +2003,11 @@ public void testResetRunningTasks() throws Exception
         "id3",
         DATASOURCE,
         0,
-        new KafkaPartitions("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
-        new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
+        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
+        new SeekableStreamPartitions<>(
+            "topic",
+            ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
+        ),
         null,
         null
     );
@@ -1936,19 +2031,19 @@ public void testResetRunningTasks() throws Exception
             null
         )
     ).anyTimes();
-    expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.PUBLISHING));
-    expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING));
-    expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING));
+    expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING));
+    expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING));
+    expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(Status.READING));
     expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime));
     expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime));
     expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L));
 
     TreeMap<Integer, Map<Integer, Long>> checkpoints = new TreeMap<>();
     checkpoints.put(0, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L));
-    expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean()))
+    expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean()))
         .andReturn(Futures.immediateFuture(checkpoints))
         .times(1);
-    expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), anyBoolean()))
+    expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean()))
         .andReturn(Futures.immediateFuture(checkpoints))
         .times(1);
 
@@ -1959,7 +2054,7 @@ public void testResetRunningTasks() throws Exception
     supervisor.runInternal();
     verifyAll();
 
-    reset(taskQueue, indexerMetadataStorageCoordinator);
+    EasyMock.reset(taskQueue, indexerMetadataStorageCoordinator);
     expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true);
     taskQueue.shutdown("id2", "DataSourceMetadata is not found while reset");
     taskQueue.shutdown("id3", "DataSourceMetadata is not found while reset");
@@ -1979,8 +2074,11 @@ public void testNoDataIngestionTasks() throws Exception
         "id1",
         DATASOURCE,
         0,
-        new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
-        new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
+        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
+        new SeekableStreamPartitions<>(
+            "topic",
+            ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
+        ),
         null,
         null
     );
@@ -1989,8 +2087,11 @@ public void testNoDataIngestionTasks() throws Exception
         "id2",
         DATASOURCE,
         0,
-        new KafkaPartitions("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
-        new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
+        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
+        new SeekableStreamPartitions<>(
+            "topic",
+            ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
+        ),
         null,
         null
     );
@@ -1999,8 +2100,11 @@ public void testNoDataIngestionTasks() throws Exception
         "id3",
         DATASOURCE,
         0,
-        new KafkaPartitions("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
-        new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
+        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
+        new SeekableStreamPartitions<>(
+            "topic",
+            ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
+        ),
         null,
         null
     );
@@ -2019,22 +2123,22 @@ public void testNoDataIngestionTasks() throws Exception
             null
         )
     ).anyTimes();
-    expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING));
-    expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING));
-    expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING));
+    expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.READING));
+    expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING));
+    expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(Status.READING));
     expect(taskClient.getStartTimeAsync("id1")).andReturn(Futures.immediateFuture(startTime));
     expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime));
     expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime));
 
     TreeMap<Integer, Map<Integer, Long>> checkpoints = new TreeMap<>();
     checkpoints.put(0, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L));
-    expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), anyBoolean()))
+    expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean()))
         .andReturn(Futures.immediateFuture(checkpoints))
         .times(1);
-    expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean()))
+    expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean()))
         .andReturn(Futures.immediateFuture(checkpoints))
         .times(1);
-    expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), anyBoolean()))
+    expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean()))
         .andReturn(Futures.immediateFuture(checkpoints))
         .times(1);
 
@@ -2045,7 +2149,7 @@ public void testNoDataIngestionTasks() throws Exception
     supervisor.runInternal();
     verifyAll();
 
-    reset(taskQueue, indexerMetadataStorageCoordinator);
+    EasyMock.reset(taskQueue, indexerMetadataStorageCoordinator);
     expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true);
     taskQueue.shutdown("id1", "DataSourceMetadata is not found while reset");
     taskQueue.shutdown("id2", "DataSourceMetadata is not found while reset");
@@ -2066,8 +2170,8 @@ public void testCheckpointForInactiveTaskGroup()
         "id1",
         DATASOURCE,
         0,
-        new KafkaPartitions(topic, ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
-        new KafkaPartitions(topic, ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
+        new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
+        new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
         null,
         null
     );
@@ -2076,8 +2180,8 @@ public void testCheckpointForInactiveTaskGroup()
         "id2",
         DATASOURCE,
         0,
-        new KafkaPartitions(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
-        new KafkaPartitions(topic, ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
+        new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
+        new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
         null,
         null
     );
@@ -2086,8 +2190,8 @@ public void testCheckpointForInactiveTaskGroup()
         "id3",
         DATASOURCE,
         0,
-        new KafkaPartitions(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
-        new KafkaPartitions(topic, ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
+        new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
+        new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
         null,
         null
     );
@@ -2109,12 +2213,12 @@ public void testCheckpointForInactiveTaskGroup()
     expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes();
     expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes();
     expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes();
-    expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE))
-        .andReturn(new KafkaDataSourceMetadata(null))
-        .anyTimes();
-    expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING));
-    expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING));
-    expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING));
+    expect(
+        indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(new KafkaDataSourceMetadata(null)
+    ).anyTimes();
+    expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.READING));
+    expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING));
+    expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(Status.READING));
 
     final DateTime startTime = DateTimes.nowUtc();
     expect(taskClient.getStartTimeAsync("id1")).andReturn(Futures.immediateFuture(startTime));
@@ -2123,13 +2227,13 @@ public void testCheckpointForInactiveTaskGroup()
 
     final TreeMap<Integer, Map<Integer, Long>> checkpoints = new TreeMap<>();
     checkpoints.put(0, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L));
-    expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), anyBoolean()))
+    expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean()))
         .andReturn(Futures.immediateFuture(checkpoints))
         .times(1);
-    expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean()))
+    expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean()))
         .andReturn(Futures.immediateFuture(checkpoints))
         .times(1);
-    expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), anyBoolean()))
+    expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean()))
         .andReturn(Futures.immediateFuture(checkpoints))
         .times(1);
 
@@ -2144,8 +2248,8 @@ public void testCheckpointForInactiveTaskGroup()
     supervisor.checkpoint(
         0,
         ((KafkaIndexTask) id1).getIOConfig().getBaseSequenceName(),
-        new KafkaDataSourceMetadata(new KafkaPartitions(topic, checkpoints.get(0))),
-        new KafkaDataSourceMetadata(new KafkaPartitions(topic, fakeCheckpoints))
+        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, checkpoints.get(0))),
+        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, fakeCheckpoints))
     );
 
     while (supervisor.getNoticesQueueSize() > 0) {
@@ -2160,7 +2264,8 @@ public void testCheckpointForInactiveTaskGroup()
   }
 
   @Test(timeout = 60_000L)
-  public void testCheckpointForUnknownTaskGroup() throws InterruptedException
+  public void testCheckpointForUnknownTaskGroup()
+      throws InterruptedException
   {
     supervisor = getSupervisor(2, 1, true, "PT1S", null, null, false);
     //not adding any events
@@ -2168,8 +2273,8 @@ public void testCheckpointForUnknownTaskGroup() throws InterruptedException
         "id1",
         DATASOURCE,
         0,
-        new KafkaPartitions(topic, ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
-        new KafkaPartitions(topic, ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
+        new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
+        new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
         null,
         null
     );
@@ -2178,8 +2283,8 @@ public void testCheckpointForUnknownTaskGroup() throws InterruptedException
         "id2",
         DATASOURCE,
         0,
-        new KafkaPartitions(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
-        new KafkaPartitions(topic, ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
+        new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
+        new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
         null,
         null
     );
@@ -2188,8 +2293,8 @@ public void testCheckpointForUnknownTaskGroup() throws InterruptedException
         "id3",
         DATASOURCE,
         0,
-        new KafkaPartitions(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
-        new KafkaPartitions(topic, ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
+        new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
+        new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
         null,
         null
     );
@@ -2214,8 +2319,8 @@ public void testCheckpointForUnknownTaskGroup() throws InterruptedException
     supervisor.checkpoint(
         0,
         ((KafkaIndexTask) id1).getIOConfig().getBaseSequenceName(),
-        new KafkaDataSourceMetadata(new KafkaPartitions(topic, Collections.emptyMap())),
-        new KafkaDataSourceMetadata(new KafkaPartitions(topic, Collections.emptyMap()))
+        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, Collections.emptyMap())),
+        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, Collections.emptyMap()))
     );
 
     while (supervisor.getNoticesQueueSize() > 0) {
@@ -2231,7 +2336,7 @@ public void testCheckpointForUnknownTaskGroup() throws InterruptedException
     Assert.assertTrue(serviceEmitter.getStackTrace()
                                     .startsWith("org.apache.druid.java.util.common.ISE: WTH?! cannot find"));
     Assert.assertEquals(
-        "WTH?! cannot find taskGroup [0] among all taskGroups [{}]",
+        "WTH?! cannot find taskGroup [0] among all activelyReadingTaskGroups [{}]",
         serviceEmitter.getExceptionMessage()
     );
     Assert.assertEquals(ISE.class, serviceEmitter.getExceptionClass());
@@ -2247,8 +2352,8 @@ public void testCheckpointWithNullTaskGroupId()
         "id1",
         DATASOURCE,
         0,
-        new KafkaPartitions(topic, ImmutableMap.of(0, 0L)),
-        new KafkaPartitions(topic, ImmutableMap.of(0, Long.MAX_VALUE)),
+        new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L)),
+        new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, Long.MAX_VALUE)),
         null,
         null
     );
@@ -2257,8 +2362,8 @@ public void testCheckpointWithNullTaskGroupId()
         "id2",
         DATASOURCE,
         0,
-        new KafkaPartitions(topic, ImmutableMap.of(0, 0L)),
-        new KafkaPartitions(topic, ImmutableMap.of(0, Long.MAX_VALUE)),
+        new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L)),
+        new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, Long.MAX_VALUE)),
         null,
         null
     );
@@ -2267,8 +2372,8 @@ public void testCheckpointWithNullTaskGroupId()
         "id3",
         DATASOURCE,
         0,
-        new KafkaPartitions(topic, ImmutableMap.of(0, 0L)),
-        new KafkaPartitions(topic, ImmutableMap.of(0, Long.MAX_VALUE)),
+        new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L)),
+        new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, Long.MAX_VALUE)),
         null,
         null
     );
@@ -2286,19 +2391,25 @@ public void testCheckpointWithNullTaskGroupId()
         indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(new KafkaDataSourceMetadata(null)
     ).anyTimes();
     taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class));
-    expect(taskClient.getStatusAsync(anyString()))
-        .andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING))
+    expect(taskClient.getStatusAsync(EasyMock.anyString()))
+        .andReturn(Futures.immediateFuture(Status.READING))
         .anyTimes();
     final TreeMap<Integer, Map<Integer, Long>> checkpoints = new TreeMap<>();
     checkpoints.put(0, ImmutableMap.of(0, 0L));
-    expect(taskClient.getCheckpointsAsync(anyString(), anyBoolean()))
+    expect(taskClient.getCheckpointsAsync(EasyMock.anyString(), EasyMock.anyBoolean()))
         .andReturn(Futures.immediateFuture(checkpoints))
         .times(3);
-    expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes();
-    expect(taskClient.pauseAsync(anyString()))
+    expect(taskClient.getStartTimeAsync(EasyMock.anyString()))
+        .andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
+        .anyTimes();
+    expect(taskClient.pauseAsync(EasyMock.anyString()))
         .andReturn(Futures.immediateFuture(ImmutableMap.of(0, 10L)))
         .anyTimes();
-    expect(taskClient.setEndOffsetsAsync(anyString(), EasyMock.eq(ImmutableMap.of(0, 10L)), anyBoolean()))
+    expect(taskClient.setEndOffsetsAsync(
+        EasyMock.anyString(),
+        EasyMock.eq(ImmutableMap.of(0, 10L)),
+        EasyMock.anyBoolean()
+    ))
         .andReturn(Futures.immediateFuture(true))
         .anyTimes();
 
@@ -2313,8 +2424,8 @@ public void testCheckpointWithNullTaskGroupId()
     supervisor.checkpoint(
         null,
         ((KafkaIndexTask) id1).getIOConfig().getBaseSequenceName(),
-        new KafkaDataSourceMetadata(new KafkaPartitions(topic, checkpoints.get(0))),
-        new KafkaDataSourceMetadata(new KafkaPartitions(topic, newCheckpoints.get(0)))
+        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, checkpoints.get(0))),
+        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, newCheckpoints.get(0)))
     );
 
     while (supervisor.getNoticesQueueSize() > 0) {
@@ -2367,8 +2478,11 @@ public void testSuspendedRunningTasks() throws Exception
         "id1",
         DATASOURCE,
         0,
-        new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
-        new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
+        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
+        new SeekableStreamPartitions<>(
+            "topic",
+            ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
+        ),
         null,
         null
     );
@@ -2377,8 +2491,11 @@ public void testSuspendedRunningTasks() throws Exception
         "id2",
         DATASOURCE,
         0,
-        new KafkaPartitions("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
-        new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
+        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
+        new SeekableStreamPartitions<>(
+            "topic",
+            ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
+        ),
         null,
         null
     );
@@ -2387,8 +2504,11 @@ public void testSuspendedRunningTasks() throws Exception
         "id3",
         DATASOURCE,
         0,
-        new KafkaPartitions("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
-        new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
+        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
+        new SeekableStreamPartitions<>(
+            "topic",
+            ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
+        ),
         null,
         null
     );
@@ -2412,9 +2532,12 @@ public void testSuspendedRunningTasks() throws Exception
             null
         )
     ).anyTimes();
-    expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.PUBLISHING));
-    expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING));
-    expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING));
+    expect(taskClient.getStatusAsync("id1"))
+        .andReturn(Futures.immediateFuture(Status.PUBLISHING));
+    expect(taskClient.getStatusAsync("id2"))
+        .andReturn(Futures.immediateFuture(Status.READING));
+    expect(taskClient.getStatusAsync("id3"))
+        .andReturn(Futures.immediateFuture(Status.READING));
     expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime));
     expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime));
     expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L));
@@ -2422,10 +2545,10 @@ public void testSuspendedRunningTasks() throws Exception
     // getCheckpoints will not be called for id1 as it is in publishing state
     TreeMap<Integer, Map<Integer, Long>> checkpoints = new TreeMap<>();
     checkpoints.put(0, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L));
-    expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean()))
+    expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean()))
         .andReturn(Futures.immediateFuture(checkpoints))
         .times(1);
-    expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), anyBoolean()))
+    expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean()))
         .andReturn(Futures.immediateFuture(checkpoints))
         .times(1);
 
@@ -2461,7 +2584,7 @@ public void testResetSuspended() throws Exception
     supervisor.runInternal();
     verifyAll();
 
-    reset(indexerMetadataStorageCoordinator);
+    EasyMock.reset(indexerMetadataStorageCoordinator);
     expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true);
     replay(indexerMetadataStorageCoordinator);
 
@@ -2538,9 +2661,9 @@ public void testFailedInitializationAndRecovery() throws Exception
 
     KafkaIndexTask task = captured.getValue();
     Assert.assertEquals(dataSchema, task.getDataSchema());
-    Assert.assertEquals(KafkaTuningConfig.copyOf(tuningConfig), task.getTuningConfig());
+    Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), task.getTuningConfig());
 
-    KafkaIOConfig taskConfig = task.getIOConfig();
+    KafkaIndexTaskIOConfig taskConfig = task.getIOConfig();
     Assert.assertEquals(kafkaHost, taskConfig.getConsumerProperties().get("bootstrap.servers"));
     Assert.assertEquals("myCustomValue", taskConfig.getConsumerProperties().get("myCustomKey"));
     Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName());
@@ -2549,19 +2672,19 @@ public void testFailedInitializationAndRecovery() throws Exception
     Assert.assertFalse("maximumMessageTime", taskConfig.getMaximumMessageTime().isPresent());
     Assert.assertFalse("skipOffsetGaps", taskConfig.isSkipOffsetGaps());
 
-    Assert.assertEquals(topic, taskConfig.getStartPartitions().getTopic());
-    Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(0));
-    Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(1));
-    Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(2));
+    Assert.assertEquals(topic, taskConfig.getStartPartitions().getStream());
+    Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0));
+    Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(1));
+    Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2));
 
-    Assert.assertEquals(topic, taskConfig.getEndPartitions().getTopic());
-    Assert.assertEquals(Long.MAX_VALUE, (long) taskConfig.getEndPartitions().getPartitionOffsetMap().get(0));
-    Assert.assertEquals(Long.MAX_VALUE, (long) taskConfig.getEndPartitions().getPartitionOffsetMap().get(1));
-    Assert.assertEquals(Long.MAX_VALUE, (long) taskConfig.getEndPartitions().getPartitionOffsetMap().get(2));
+    Assert.assertEquals(topic, taskConfig.getEndPartitions().getStream());
+    Assert.assertEquals(Long.MAX_VALUE, (long) taskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(0));
+    Assert.assertEquals(Long.MAX_VALUE, (long) taskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(1));
+    Assert.assertEquals(Long.MAX_VALUE, (long) taskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(2));
   }
 
   @Test
-  public void testGetCurrentTotalStats() throws Exception
+  public void testGetCurrentTotalStats()
   {
     supervisor = getSupervisor(1, 2, true, "PT1H", null, null, false);
     supervisor.addTaskGroupToActivelyReadingTaskGroup(
@@ -2569,7 +2692,8 @@ public void testGetCurrentTotalStats() throws Exception
         ImmutableMap.of(0, 0L),
         Optional.absent(),
         Optional.absent(),
-        ImmutableSet.of("task1")
+        ImmutableSet.of("task1"),
+        ImmutableSet.of()
     );
 
     supervisor.addTaskGroupToPendingCompletionTaskGroup(
@@ -2577,7 +2701,8 @@ public void testGetCurrentTotalStats() throws Exception
         ImmutableMap.of(0, 0L),
         Optional.absent(),
         Optional.absent(),
-        ImmutableSet.of("task2")
+        ImmutableSet.of("task2"),
+        ImmutableSet.of()
     );
 
     expect(taskClient.getMovingAveragesAsync("task1")).andReturn(Futures.immediateFuture(ImmutableMap.of(
@@ -2762,8 +2887,8 @@ private KafkaIndexTask createKafkaIndexTask(
       String id,
       String dataSource,
       int taskGroupId,
-      KafkaPartitions startPartitions,
-      KafkaPartitions endPartitions,
+      SeekableStreamPartitions<Integer, Long> startPartitions,
+      SeekableStreamPartitions<Integer, Long> endPartitions,
       DateTime minimumMessageTime,
       DateTime maximumMessageTime
   )
@@ -2773,12 +2898,12 @@ private KafkaIndexTask createKafkaIndexTask(
         null,
         getDataSchema(dataSource),
         tuningConfig,
-        new KafkaIOConfig(
+        new KafkaIndexTaskIOConfig(
             taskGroupId,
             "sequenceName-" + taskGroupId,
             startPartitions,
             endPartitions,
-            ImmutableMap.of(),
+            ImmutableMap.of("bootstrap.servers", kafkaHost),
             true,
             minimumMessageTime,
             maximumMessageTime,
@@ -2860,4 +2985,6 @@ protected String generateSequenceName(
       return StringUtils.format("sequenceName-%d", groupId);
     }
   }
+
+
 }
diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfigTest.java
index 0a3ea95fea5..45470ffe10d 100644
--- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfigTest.java
+++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfigTest.java
@@ -107,7 +107,7 @@ public void testSerdeWithNonDefaults() throws Exception
     Assert.assertEquals(100, config.getMaxRowsInMemory());
     Assert.assertEquals(100, config.getMaxRowsPerSegment());
     Assert.assertEquals(new Period("PT1H"), config.getIntermediatePersistPeriod());
-    Assert.assertEquals(0, config.getMaxPendingPersists());
+    Assert.assertEquals(100, config.getMaxPendingPersists());
     Assert.assertEquals(true, config.isReportParseExceptions());
     Assert.assertEquals(100, config.getHandoffConditionTimeout());
     Assert.assertEquals(12, (int) config.getWorkerThreads());
@@ -117,4 +117,5 @@ public void testSerdeWithNonDefaults() throws Exception
     Assert.assertEquals(Duration.standardSeconds(95), config.getShutdownTimeout());
     Assert.assertEquals(Duration.standardSeconds(20), config.getOffsetFetchPeriod());
   }
+
 }
diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestBroker.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestBroker.java
index a57d22b1446..6659f92ebbf 100644
--- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestBroker.java
+++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestBroker.java
@@ -49,7 +49,7 @@
   private final File directory;
   private final boolean directoryCleanup;
   private final int id;
-  private final Map<String, String> brokerProps;
+  private final Map<String, Object> brokerProps;
 
   private volatile KafkaServer server;
 
@@ -57,7 +57,7 @@ public TestBroker(
       String zookeeperConnect,
       @Nullable File directory,
       int id,
-      Map<String, String> brokerProps
+      Map<String, Object> brokerProps
   )
   {
     this.zookeeperConnect = zookeeperConnect;
@@ -81,7 +81,12 @@ public void start()
 
     final KafkaConfig config = new KafkaConfig(props);
 
-    server = new KafkaServer(config, Time.SYSTEM, Some.apply(StringUtils.format("TestingBroker[%d]-", id)), List$.MODULE$.empty());
+    server = new KafkaServer(
+        config,
+        Time.SYSTEM,
+        Some.apply(StringUtils.format("TestingBroker[%d]-", id)),
+        List$.MODULE$.empty()
+    );
     server.startup();
   }
 
diff --git a/extensions-core/kinesis-indexing-service/pom.xml b/extensions-core/kinesis-indexing-service/pom.xml
new file mode 100644
index 00000000000..db79d856671
--- /dev/null
+++ b/extensions-core/kinesis-indexing-service/pom.xml
@@ -0,0 +1,109 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one
+  ~ or more contributor license agreements.  See the NOTICE file
+  ~ distributed with this work for additional information
+  ~ regarding copyright ownership.  The ASF licenses this file
+  ~ to you under the Apache License, Version 2.0 (the
+  ~ "License"); you may not use this file except in compliance
+  ~ with the License.  You may obtain a copy of the License at
+  ~
+  ~   http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing,
+  ~ software distributed under the License is distributed on an
+  ~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+  ~ KIND, either express or implied.  See the License for the
+  ~ specific language governing permissions and limitations
+  ~ under the License.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <groupId>org.apache.druid.extensions</groupId>
+    <artifactId>druid-kinesis-indexing-service</artifactId>
+    <name>druid-kinesis-indexing-service</name>
+    <description>druid-kinesis-indexing-service</description>
+
+    <parent>
+        <groupId>org.apache.druid</groupId>
+        <artifactId>druid</artifactId>
+        <version>0.13.0-incubating-SNAPSHOT</version>
+        <relativePath>../../pom.xml</relativePath>
+    </parent>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.druid</groupId>
+            <artifactId>druid-core</artifactId>
+            <version>${project.parent.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.druid</groupId>
+            <artifactId>druid-indexing-service</artifactId>
+            <version>${project.parent.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.druid</groupId>
+            <artifactId>druid-server</artifactId>
+            <version>${project.parent.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-kinesis</artifactId>
+            <version>${aws.sdk.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-sts</artifactId>
+            <version>${aws.sdk.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>amazon-kinesis-client</artifactId>
+            <version>1.9.2</version>
+            <scope>provided</scope>
+        </dependency>
+
+        <!-- Tests -->
+        <dependency>
+            <groupId>org.easymock</groupId>
+            <artifactId>easymock</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.druid</groupId>
+            <artifactId>druid-server</artifactId>
+            <version>${project.parent.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.druid</groupId>
+            <artifactId>druid-processing</artifactId>
+            <version>${project.parent.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.druid</groupId>
+            <artifactId>druid-indexing-service</artifactId>
+            <version>${project.parent.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+
+</project>
diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadata.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadata.java
new file mode 100644
index 00000000000..4592c1b6774
--- /dev/null
+++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadata.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.indexing.kinesis;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata;
+import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
+
+import java.util.Map;
+
+public class KinesisDataSourceMetadata extends SeekableStreamDataSourceMetadata<String, String>
+{
+  @JsonCreator
+  public KinesisDataSourceMetadata(
+      @JsonProperty("partitions") SeekableStreamPartitions<String, String> kinesisPartitions
+  )
+  {
+    super(kinesisPartitions);
+  }
+
+  @Override
+  protected KinesisDataSourceMetadata createConcreteDataSourceMetaData(String streamName, Map<String, String> newMap)
+  {
+    return new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(streamName, newMap));
+  }
+}
diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java
new file mode 100644
index 00000000000..3467481762a
--- /dev/null
+++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.indexing.kinesis;
+
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.druid.common.aws.AWSCredentialsConfig;
+import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory;
+import org.apache.druid.indexing.common.task.TaskResource;
+import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask;
+import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner;
+import org.apache.druid.segment.indexing.DataSchema;
+import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider;
+import org.apache.druid.server.security.AuthorizerMapper;
+
+import java.util.Map;
+
+public class KinesisIndexTask extends SeekableStreamIndexTask<String, String>
+{
+  private static final String TYPE = "index_kinesis";
+
+  private final AWSCredentialsConfig awsCredentialsConfig;
+
+  @JsonCreator
+  public KinesisIndexTask(
+      @JsonProperty("id") String id,
+      @JsonProperty("resource") TaskResource taskResource,
+      @JsonProperty("dataSchema") DataSchema dataSchema,
+      @JsonProperty("tuningConfig") KinesisIndexTaskTuningConfig tuningConfig,
+      @JsonProperty("ioConfig") KinesisIndexTaskIOConfig ioConfig,
+      @JsonProperty("context") Map<String, Object> context,
+      @JacksonInject ChatHandlerProvider chatHandlerProvider,
+      @JacksonInject AuthorizerMapper authorizerMapper,
+      @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory,
+      @JacksonInject AWSCredentialsConfig awsCredentialsConfig
+  )
+  {
+    super(
+        id == null ? getFormattedId(dataSchema.getDataSource(), TYPE) : id,
+        taskResource,
+        dataSchema,
+        tuningConfig,
+        ioConfig,
+        context,
+        chatHandlerProvider,
+        authorizerMapper,
+        rowIngestionMetersFactory,
+        getFormattedGroupId(dataSchema.getDataSource(), TYPE)
+    );
+    this.awsCredentialsConfig = awsCredentialsConfig;
+  }
+
+
+  @Override
+  protected SeekableStreamIndexTaskRunner<String, String> createTaskRunner()
+  {
+    return new KinesisIndexTaskRunner(
+        this,
+        parser,
+        authorizerMapper,
+        chatHandlerProvider,
+        savedParseExceptions,
+        rowIngestionMetersFactory
+    );
+  }
+
+  @Override
+  protected KinesisRecordSupplier newTaskRecordSupplier()
+      throws RuntimeException
+  {
+    KinesisIndexTaskIOConfig ioConfig = ((KinesisIndexTaskIOConfig) super.ioConfig);
+    KinesisIndexTaskTuningConfig tuningConfig = ((KinesisIndexTaskTuningConfig) super.tuningConfig);
+    int fetchThreads = tuningConfig.getFetchThreads() != null
+                       ? tuningConfig.getFetchThreads()
+                       : Math.max(1, ioConfig.getStartPartitions().getPartitionSequenceNumberMap().size());
+
+    return new KinesisRecordSupplier(
+        KinesisRecordSupplier.getAmazonKinesisClient(
+            ioConfig.getEndpoint(),
+            awsCredentialsConfig,
+            ioConfig.getAwsAssumedRoleArn(),
+            ioConfig.getAwsExternalId()
+        ),
+        ioConfig.getRecordsPerFetch(),
+        ioConfig.getFetchDelayMillis(),
+        fetchThreads,
+        ioConfig.isDeaggregate(),
+        tuningConfig.getRecordBufferSize(),
+        tuningConfig.getRecordBufferOfferTimeout(),
+        tuningConfig.getRecordBufferFullWait(),
+        tuningConfig.getFetchSequenceNumberTimeout(),
+        tuningConfig.getMaxRecordsPerPoll()
+    );
+  }
+
+  @Override
+  @JsonProperty("ioConfig")
+  public KinesisIndexTaskIOConfig getIOConfig()
+  {
+    return (KinesisIndexTaskIOConfig) super.getIOConfig();
+  }
+
+  @Override
+  public String getType()
+  {
+    return TYPE;
+  }
+}
diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClient.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClient.java
new file mode 100644
index 00000000000..8f1c66a1acd
--- /dev/null
+++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClient.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.indexing.kinesis;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.druid.indexing.common.TaskInfoProvider;
+import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClient;
+import org.apache.druid.java.util.http.client.HttpClient;
+import org.joda.time.Duration;
+
+public class KinesisIndexTaskClient extends SeekableStreamIndexTaskClient<String, String>
+{
+  KinesisIndexTaskClient(
+      HttpClient httpClient,
+      ObjectMapper jsonMapper,
+      TaskInfoProvider taskInfoProvider,
+      String dataSource,
+      int numThreads,
+      Duration httpTimeout,
+      long numRetries
+  )
+  {
+    super(
+        httpClient,
+        jsonMapper,
+        taskInfoProvider,
+        dataSource,
+        numThreads,
+        httpTimeout,
+        numRetries
+    );
+  }
+
+  @Override
+  protected Class<String> getPartitionType()
+  {
+    return String.class;
+  }
+
+  @Override
+  protected Class<String> getSequenceType()
+  {
+    return String.class;
+  }
+
+}
+
diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientFactory.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientFactory.java
new file mode 100644
index 00000000000..6d443464da6
--- /dev/null
+++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientFactory.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.indexing.kinesis;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.inject.Inject;
+import org.apache.druid.guice.annotations.EscalatedGlobal;
+import org.apache.druid.guice.annotations.Json;
+import org.apache.druid.indexing.common.TaskInfoProvider;
+import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory;
+import org.apache.druid.java.util.http.client.HttpClient;
+import org.joda.time.Duration;
+
+public class KinesisIndexTaskClientFactory extends SeekableStreamIndexTaskClientFactory<KinesisIndexTaskClient>
+{
+  @Inject
+  public KinesisIndexTaskClientFactory(
+      @EscalatedGlobal HttpClient httpClient,
+      @Json ObjectMapper mapper
+  )
+  {
+    super(httpClient, mapper);
+  }
+
+  @Override
+  public KinesisIndexTaskClient build(
+      TaskInfoProvider taskInfoProvider,
+      String dataSource,
+      int numThreads,
+      Duration httpTimeout,
+      long numRetries
+  )
+  {
+    return new KinesisIndexTaskClient(
+        getHttpClient(),
+        getMapper(),
+        taskInfoProvider,
+        dataSource,
+        numThreads,
+        httpTimeout,
+        numRetries
+    );
+  }
+}
diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskIOConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskIOConfig.java
new file mode 100644
index 00000000000..307e971bcfd
--- /dev/null
+++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskIOConfig.java
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.indexing.kinesis;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig;
+import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
+import org.joda.time.DateTime;
+
+import javax.annotation.Nullable;
+import java.util.Set;
+
+public class KinesisIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig<String, String>
+{
+  public static final int DEFAULT_RECORDS_PER_FETCH = 4000;
+  public static final int DEFAULT_FETCH_DELAY_MILLIS = 0;
+
+  private final String endpoint;
+  private final Integer recordsPerFetch;
+  private final Integer fetchDelayMillis;
+
+  private final String awsAssumedRoleArn;
+  private final String awsExternalId;
+  private final boolean deaggregate;
+
+  @JsonCreator
+  public KinesisIndexTaskIOConfig(
+      @JsonProperty("taskGroupId") @Nullable Integer taskGroupId,
+      @JsonProperty("baseSequenceName") String baseSequenceName,
+      @JsonProperty("startPartitions") SeekableStreamPartitions<String, String> startPartitions,
+      @JsonProperty("endPartitions") SeekableStreamPartitions<String, String> endPartitions,
+      @JsonProperty("useTransaction") Boolean useTransaction,
+      @JsonProperty("minimumMessageTime") DateTime minimumMessageTime,
+      @JsonProperty("maximumMessageTime") DateTime maximumMessageTime,
+      @JsonProperty("endpoint") String endpoint,
+      @JsonProperty("recordsPerFetch") Integer recordsPerFetch,
+      @JsonProperty("fetchDelayMillis") Integer fetchDelayMillis,
+      @JsonProperty("exclusiveStartSequenceNumberPartitions") Set<String> exclusiveStartSequenceNumberPartitions,
+      @JsonProperty("awsAssumedRoleArn") String awsAssumedRoleArn,
+      @JsonProperty("awsExternalId") String awsExternalId,
+      @JsonProperty("deaggregate") boolean deaggregate
+  )
+  {
+    super(
+        taskGroupId,
+        baseSequenceName,
+        startPartitions,
+        endPartitions,
+        useTransaction,
+        minimumMessageTime,
+        maximumMessageTime,
+        true,
+        exclusiveStartSequenceNumberPartitions
+    );
+    Preconditions.checkArgument(endPartitions.getPartitionSequenceNumberMap()
+                                             .values()
+                                             .stream()
+                                             .noneMatch(x -> x.equals(KinesisSequenceNumber.END_OF_SHARD_MARKER)));
+
+    this.endpoint = Preconditions.checkNotNull(endpoint, "endpoint");
+    this.recordsPerFetch = recordsPerFetch != null ? recordsPerFetch : DEFAULT_RECORDS_PER_FETCH;
+    this.fetchDelayMillis = fetchDelayMillis != null ? fetchDelayMillis : DEFAULT_FETCH_DELAY_MILLIS;
+    this.awsAssumedRoleArn = awsAssumedRoleArn;
+    this.awsExternalId = awsExternalId;
+    this.deaggregate = deaggregate;
+  }
+
+  @JsonProperty
+  public String getEndpoint()
+  {
+    return endpoint;
+  }
+
+  @JsonProperty
+  public int getRecordsPerFetch()
+  {
+    return recordsPerFetch;
+  }
+
+  @JsonProperty
+  public int getFetchDelayMillis()
+  {
+    return fetchDelayMillis;
+  }
+
+  @JsonProperty
+  public String getAwsAssumedRoleArn()
+  {
+    return awsAssumedRoleArn;
+  }
+
+  @JsonProperty
+  public String getAwsExternalId()
+  {
+    return awsExternalId;
+  }
+
+  @JsonProperty
+  public boolean isDeaggregate()
+  {
+    return deaggregate;
+  }
+
+  @Override
+  public String toString()
+  {
+    return "KinesisIndexTaskIOConfig{" +
+           "baseSequenceName='" + getBaseSequenceName() + '\'' +
+           ", startPartitions=" + getStartPartitions() +
+           ", endPartitions=" + getEndPartitions() +
+           ", useTransaction=" + isUseTransaction() +
+           ", minimumMessageTime=" + getMinimumMessageTime() +
+           ", maximumMessageTime=" + getMaximumMessageTime() +
+           ", endpoint='" + endpoint + '\'' +
+           ", recordsPerFetch=" + recordsPerFetch +
+           ", fetchDelayMillis=" + fetchDelayMillis +
+           ", exclusiveStartSequenceNumberPartitions=" + getExclusiveStartSequenceNumberPartitions() +
+           ", awsAssumedRoleArn='" + awsAssumedRoleArn + '\'' +
+           ", awsExternalId='" + awsExternalId + '\'' +
+           ", deaggregate=" + deaggregate +
+           '}';
+  }
+}
diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java
new file mode 100644
index 00000000000..3e7e5e7aedd
--- /dev/null
+++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java
@@ -0,0 +1,199 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.indexing.kinesis;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Optional;
+import org.apache.druid.data.input.impl.InputRowParser;
+import org.apache.druid.indexing.common.TaskToolbox;
+import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory;
+import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata;
+import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner;
+import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
+import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord;
+import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber;
+import org.apache.druid.indexing.seekablestream.common.RecordSupplier;
+import org.apache.druid.indexing.seekablestream.common.StreamPartition;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider;
+import org.apache.druid.server.security.AuthorizerMapper;
+import org.apache.druid.utils.CircularBuffer;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.stream.Collectors;
+
+public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner<String, String>
+{
+  private static final EmittingLogger log = new EmittingLogger(KinesisIndexTaskRunner.class);
+  private static final long POLL_TIMEOUT = 100;
+
+  private final KinesisIndexTask task;
+
+  KinesisIndexTaskRunner(
+      KinesisIndexTask task,
+      InputRowParser<ByteBuffer> parser,
+      AuthorizerMapper authorizerMapper,
+      Optional<ChatHandlerProvider> chatHandlerProvider,
+      CircularBuffer<Throwable> savedParseExceptions,
+      RowIngestionMetersFactory rowIngestionMetersFactory
+  )
+  {
+    super(
+        task,
+        parser,
+        authorizerMapper,
+        chatHandlerProvider,
+        savedParseExceptions,
+        rowIngestionMetersFactory
+    );
+    this.task = task;
+  }
+
+
+  @Override
+  protected String getSequenceNumberToStoreAfterRead(String sequenceNumber)
+  {
+    return sequenceNumber;
+  }
+
+  @Nonnull
+  @Override
+  protected List<OrderedPartitionableRecord<String, String>> getRecords(
+      RecordSupplier<String, String> recordSupplier, TaskToolbox toolbox
+  )
+  {
+    return recordSupplier.poll(POLL_TIMEOUT);
+  }
+
+  @Override
+  protected SeekableStreamPartitions<String, String> deserializeSeekableStreamPartitionsFromMetadata(
+      ObjectMapper mapper,
+      Object object
+  )
+  {
+    return mapper.convertValue(object, mapper.getTypeFactory().constructParametrizedType(
+        SeekableStreamPartitions.class,
+        SeekableStreamPartitions.class,
+        String.class,
+        String.class
+    ));
+  }
+
+  @Override
+  protected SeekableStreamDataSourceMetadata<String, String> createDataSourceMetadata(
+      SeekableStreamPartitions<String, String> partitions
+  )
+  {
+    return new KinesisDataSourceMetadata(partitions);
+  }
+
+  @Override
+  protected OrderedSequenceNumber<String> createSequenceNumber(String sequenceNumber)
+  {
+    return KinesisSequenceNumber.of(sequenceNumber);
+  }
+
+  @Override
+  protected void possiblyResetDataSourceMetadata(
+      TaskToolbox toolbox,
+      RecordSupplier<String, String> recordSupplier,
+      Set<StreamPartition<String>> assignment,
+      Map<String, String> currOffsets
+  )
+  {
+    if (!task.getTuningConfig().isSkipSequenceNumberAvailabilityCheck()) {
+      for (final StreamPartition<String> streamPartition : assignment) {
+        String sequence = currOffsets.get(streamPartition.getPartitionId());
+        String earliestSequenceNumber = recordSupplier.getEarliestSequenceNumber(streamPartition);
+        if (earliestSequenceNumber == null
+            || createSequenceNumber(earliestSequenceNumber).compareTo(createSequenceNumber(sequence)) > 0) {
+          if (task.getTuningConfig().isResetOffsetAutomatically()) {
+            log.info("Attempting to reset sequences automatically for all partitions");
+            try {
+              sendResetRequestAndWait(
+                  assignment.stream()
+                            .collect(Collectors.toMap(x -> x, x -> currOffsets.get(x.getPartitionId()))),
+                  toolbox
+              );
+            }
+            catch (IOException e) {
+              throw new ISE(e, "Exception while attempting to automatically reset sequences");
+            }
+          } else {
+            throw new ISE(
+                "Starting sequenceNumber [%s] is no longer available for partition [%s] (earliest: [%s]) and resetOffsetAutomatically is not enabled",
+                sequence,
+                streamPartition.getPartitionId(),
+                earliestSequenceNumber
+            );
+          }
+        }
+      }
+    }
+  }
+
+  @Override
+  protected boolean isEndSequenceOffsetsExclusive()
+  {
+    return false;
+  }
+
+  @Override
+  protected boolean isStartingSequenceOffsetsExclusive()
+  {
+    return true;
+  }
+
+  @Override
+  protected boolean isEndOfShard(String seqNum)
+  {
+    return KinesisSequenceNumber.END_OF_SHARD_MARKER.equals(seqNum);
+  }
+
+  @Nullable
+  @Override
+  protected TreeMap<Integer, Map<String, String>> getCheckPointsFromContext(
+      TaskToolbox toolbox,
+      String checkpointsString
+  ) throws IOException
+  {
+    if (checkpointsString != null) {
+      log.info("Checkpoints [%s]", checkpointsString);
+      return toolbox.getObjectMapper().readValue(
+          checkpointsString,
+          new TypeReference<TreeMap<Integer, Map<String, String>>>()
+          {
+          }
+      );
+    } else {
+      return null;
+    }
+  }
+
+}
diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java
new file mode 100644
index 00000000000..95ee278985f
--- /dev/null
+++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java
@@ -0,0 +1,278 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.indexing.kinesis;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.google.common.base.Preconditions;
+import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig;
+import org.apache.druid.segment.IndexSpec;
+import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory;
+import org.joda.time.Period;
+
+import javax.annotation.Nullable;
+import java.io.File;
+import java.util.Objects;
+
+@JsonTypeName("KinesisTuningConfig")
+public class KinesisIndexTaskTuningConfig extends SeekableStreamIndexTaskTuningConfig
+{
+  private static final int DEFAULT_RECORD_BUFFER_SIZE = 10000;
+  private static final int DEFAULT_RECORD_BUFFER_OFFER_TIMEOUT = 5000;
+  private static final int DEFAULT_RECORD_BUFFER_FULL_WAIT = 5000;
+  private static final int DEFAULT_FETCH_SEQUENCE_NUMBER_TIMEOUT = 20000;
+  private static final int DEFAULT_MAX_RECORDS_PER_POLL = 100;
+
+  private final int recordBufferSize;
+  private final int recordBufferOfferTimeout;
+  private final int recordBufferFullWait;
+  private final int fetchSequenceNumberTimeout;
+  private final Integer fetchThreads;
+  private final int maxRecordsPerPoll;
+
+  @JsonCreator
+  public KinesisIndexTaskTuningConfig(
+      @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory,
+      @JsonProperty("maxBytesInMemory") Long maxBytesInMemory,
+      @JsonProperty("maxRowsPerSegment") Integer maxRowsPerSegment,
+      @JsonProperty("maxTotalRows") Long maxTotalRows,
+      @JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod,
+      @JsonProperty("basePersistDirectory") File basePersistDirectory,
+      @JsonProperty("maxPendingPersists") Integer maxPendingPersists,
+      @JsonProperty("indexSpec") IndexSpec indexSpec,
+      @JsonProperty("buildV9Directly") Boolean buildV9Directly,
+      @JsonProperty("reportParseExceptions") Boolean reportParseExceptions,
+      @JsonProperty("handoffConditionTimeout") Long handoffConditionTimeout,
+      @JsonProperty("resetOffsetAutomatically") Boolean resetOffsetAutomatically,
+      @JsonProperty("skipSequenceNumberAvailabilityCheck") Boolean skipSequenceNumberAvailabilityCheck,
+      @JsonProperty("recordBufferSize") Integer recordBufferSize,
+      @JsonProperty("recordBufferOfferTimeout") Integer recordBufferOfferTimeout,
+      @JsonProperty("recordBufferFullWait") Integer recordBufferFullWait,
+      @JsonProperty("fetchSequenceNumberTimeout") Integer fetchSequenceNumberTimeout,
+      @JsonProperty("fetchThreads") Integer fetchThreads,
+      @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory,
+      @JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions,
+      @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions,
+      @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions,
+      @JsonProperty("maxRecordsPerPoll") @Nullable Integer maxRecordsPerPoll,
+      @JsonProperty("intermediateHandoffPeriod") @Nullable Period intermediateHandoffPeriod
+  )
+  {
+    super(
+        maxRowsInMemory,
+        maxBytesInMemory,
+        maxRowsPerSegment,
+        maxTotalRows,
+        intermediatePersistPeriod,
+        basePersistDirectory,
+        maxPendingPersists,
+        indexSpec,
+        true,
+        reportParseExceptions,
+        handoffConditionTimeout,
+        resetOffsetAutomatically,
+        skipSequenceNumberAvailabilityCheck,
+        segmentWriteOutMediumFactory,
+        intermediateHandoffPeriod,
+        logParseExceptions,
+        maxParseExceptions,
+        maxSavedParseExceptions
+    );
+    this.recordBufferSize = recordBufferSize == null ? DEFAULT_RECORD_BUFFER_SIZE : recordBufferSize;
+    this.recordBufferOfferTimeout = recordBufferOfferTimeout == null
+                                    ? DEFAULT_RECORD_BUFFER_OFFER_TIMEOUT
+                                    : recordBufferOfferTimeout;
+    this.recordBufferFullWait = recordBufferFullWait == null ? DEFAULT_RECORD_BUFFER_FULL_WAIT : recordBufferFullWait;
+    this.fetchSequenceNumberTimeout = fetchSequenceNumberTimeout
+                                      == null ? DEFAULT_FETCH_SEQUENCE_NUMBER_TIMEOUT : fetchSequenceNumberTimeout;
+    this.fetchThreads = fetchThreads; // we handle this being null later
+    this.maxRecordsPerPoll = maxRecordsPerPoll == null ? DEFAULT_MAX_RECORDS_PER_POLL : maxRecordsPerPoll;
+
+    Preconditions.checkArgument(
+        !(super.isResetOffsetAutomatically() && super.isSkipSequenceNumberAvailabilityCheck()),
+        "resetOffsetAutomatically cannot be used if skipSequenceNumberAvailabilityCheck=true"
+    );
+  }
+
+  @JsonProperty
+  public int getRecordBufferSize()
+  {
+    return recordBufferSize;
+  }
+
+  @JsonProperty
+  public int getRecordBufferOfferTimeout()
+  {
+    return recordBufferOfferTimeout;
+  }
+
+  @JsonProperty
+  public int getRecordBufferFullWait()
+  {
+    return recordBufferFullWait;
+  }
+
+  @JsonProperty
+  public int getFetchSequenceNumberTimeout()
+  {
+    return fetchSequenceNumberTimeout;
+  }
+
+  @JsonProperty
+  public Integer getFetchThreads()
+  {
+    return fetchThreads;
+  }
+
+  @JsonProperty
+  public int getMaxRecordsPerPoll()
+  {
+    return maxRecordsPerPoll;
+  }
+
+  @Override
+  public KinesisIndexTaskTuningConfig withBasePersistDirectory(File dir)
+  {
+    return new KinesisIndexTaskTuningConfig(
+        getMaxRowsInMemory(),
+        getMaxBytesInMemory(),
+        getMaxRowsPerSegment(),
+        getMaxTotalRows(),
+        getIntermediatePersistPeriod(),
+        dir,
+        getMaxPendingPersists(),
+        getIndexSpec(),
+        true,
+        isReportParseExceptions(),
+        getHandoffConditionTimeout(),
+        isResetOffsetAutomatically(),
+        isSkipSequenceNumberAvailabilityCheck(),
+        getRecordBufferSize(),
+        getRecordBufferOfferTimeout(),
+        getRecordBufferFullWait(),
+        getFetchSequenceNumberTimeout(),
+        getFetchThreads(),
+        getSegmentWriteOutMediumFactory(),
+        isLogParseExceptions(),
+        getMaxParseExceptions(),
+        getMaxSavedParseExceptions(),
+        getMaxRecordsPerPoll(),
+        getIntermediateHandoffPeriod()
+    );
+  }
+
+  @Override
+  public boolean equals(Object o)
+  {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    KinesisIndexTaskTuningConfig that = (KinesisIndexTaskTuningConfig) o;
+    return getMaxRowsInMemory() == that.getMaxRowsInMemory() &&
+           getMaxBytesInMemory() == that.getMaxBytesInMemory() &&
+           getMaxRowsPerSegment() == that.getMaxRowsPerSegment() &&
+           getMaxPendingPersists() == that.getMaxPendingPersists() &&
+           getBuildV9Directly() == that.getBuildV9Directly() &&
+           isReportParseExceptions() == that.isReportParseExceptions() &&
+           getHandoffConditionTimeout() == that.getHandoffConditionTimeout() &&
+           isResetOffsetAutomatically() == that.isResetOffsetAutomatically() &&
+           isSkipSequenceNumberAvailabilityCheck() == that.isSkipSequenceNumberAvailabilityCheck() &&
+           getRecordBufferSize() == that.getRecordBufferSize() &&
+           getRecordBufferOfferTimeout() == that.getRecordBufferOfferTimeout() &&
+           getRecordBufferFullWait() == that.getRecordBufferFullWait() &&
+           getFetchSequenceNumberTimeout() == that.getFetchSequenceNumberTimeout() &&
+           isLogParseExceptions() == that.isLogParseExceptions() &&
+           getMaxParseExceptions() == that.getMaxParseExceptions() &&
+           getMaxSavedParseExceptions() == that.getMaxSavedParseExceptions() &&
+           getMaxRecordsPerPoll() == that.getMaxRecordsPerPoll() &&
+           Objects.equals(getIntermediatePersistPeriod(), that.getIntermediatePersistPeriod()) &&
+           Objects.equals(getBasePersistDirectory(), that.getBasePersistDirectory()) &&
+           Objects.equals(getIndexSpec(), that.getIndexSpec()) &&
+           Objects.equals(getFetchThreads(), that.getFetchThreads()) &&
+           Objects.equals(getSegmentWriteOutMediumFactory(), that.getSegmentWriteOutMediumFactory()) &&
+           Objects.equals(getMaxTotalRows(), that.getMaxTotalRows()) &&
+           Objects.equals(getIntermediateHandoffPeriod(), that.getIntermediateHandoffPeriod());
+  }
+
+  @Override
+  public int hashCode()
+  {
+    return Objects.hash(
+        getMaxRowsInMemory(),
+        getMaxBytesInMemory(),
+        getMaxRowsPerSegment(),
+        getMaxTotalRows(),
+        getIntermediatePersistPeriod(),
+        getBasePersistDirectory(),
+        getMaxPendingPersists(),
+        getIndexSpec(),
+        true,
+        isReportParseExceptions(),
+        getHandoffConditionTimeout(),
+        isResetOffsetAutomatically(),
+        isSkipSequenceNumberAvailabilityCheck(),
+        getRecordBufferSize(),
+        getRecordBufferOfferTimeout(),
+        getRecordBufferFullWait(),
+        getFetchSequenceNumberTimeout(),
+        getFetchThreads(),
+        getSegmentWriteOutMediumFactory(),
+        isLogParseExceptions(),
+        getMaxParseExceptions(),
+        getMaxSavedParseExceptions(),
+        getMaxRecordsPerPoll(),
+        getIntermediateHandoffPeriod()
+    );
+  }
+
+  @Override
+  public String toString()
+  {
+    return "KinesisIndexTaskTuningConfig{" +
+           "maxRowsInMemory=" + getMaxRowsInMemory() +
+           ", maxBytesInMemory=" + getMaxBytesInMemory() +
+           ", maxRowsPerSegment=" + getMaxRowsPerSegment() +
+           ", maxTotalRows=" + getMaxTotalRows() +
+           ", intermediatePersistPeriod=" + getIntermediatePersistPeriod() +
+           ", basePersistDirectory=" + getBasePersistDirectory() +
+           ", maxPendingPersists=" + getMaxPendingPersists() +
+           ", indexSpec=" + getIndexSpec() +
+           ", reportParseExceptions=" + isReportParseExceptions() +
+           ", handoffConditionTimeout=" + getHandoffConditionTimeout() +
+           ", resetOffsetAutomatically=" + isResetOffsetAutomatically() +
+           ", skipSequenceNumberAvailabilityCheck=" + isSkipSequenceNumberAvailabilityCheck() +
+           ", recordBufferSize=" + recordBufferSize +
+           ", recordBufferOfferTimeout=" + recordBufferOfferTimeout +
+           ", recordBufferFullWait=" + recordBufferFullWait +
+           ", fetchSequenceNumberTimeout=" + fetchSequenceNumberTimeout +
+           ", fetchThreads=" + fetchThreads +
+           ", segmentWriteOutMediumFactory=" + getSegmentWriteOutMediumFactory() +
+           ", logParseExceptions=" + isLogParseExceptions() +
+           ", maxParseExceptions=" + getMaxParseExceptions() +
+           ", maxSavedParseExceptions=" + getMaxSavedParseExceptions() +
+           ", maxRecordsPerPoll=" + maxRecordsPerPoll +
+           ", intermediateHandoffPeriod=" + getIntermediateHandoffPeriod() +
+           '}';
+  }
+}
diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java
new file mode 100644
index 00000000000..cba5166f268
--- /dev/null
+++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.indexing.kinesis;
+
+import com.fasterxml.jackson.databind.Module;
+import com.fasterxml.jackson.databind.jsontype.NamedType;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import com.google.common.collect.ImmutableList;
+import com.google.inject.Binder;
+import com.google.inject.TypeLiteral;
+import com.google.inject.name.Names;
+import org.apache.druid.common.aws.AWSCredentialsConfig;
+import org.apache.druid.guice.JsonConfigProvider;
+import org.apache.druid.guice.LazySingleton;
+import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorSpec;
+import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorTuningConfig;
+import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory;
+import org.apache.druid.initialization.DruidModule;
+
+import java.util.List;
+
+public class KinesisIndexingServiceModule implements DruidModule
+{
+  @Override
+  public List<? extends Module> getJacksonModules()
+  {
+    return ImmutableList.of(
+        new SimpleModule(getClass().getSimpleName())
+            .registerSubtypes(
+                new NamedType(KinesisIndexTask.class, "index_kinesis"),
+                new NamedType(KinesisDataSourceMetadata.class, "kinesis"),
+                new NamedType(KinesisIndexTaskIOConfig.class, "kinesis"),
+                new NamedType(KinesisSupervisorTuningConfig.class, "kinesis"),
+                new NamedType(KinesisSupervisorSpec.class, "kinesis")
+            )
+    );
+  }
+
+  @Override
+  public void configure(Binder binder)
+  {
+    binder.bind(
+        new TypeLiteral<SeekableStreamIndexTaskClientFactory<KinesisIndexTaskClient>>()
+        {
+        }
+    ).to(KinesisIndexTaskClientFactory.class).in(LazySingleton.class);
+
+    JsonConfigProvider.bind(binder, "druid.kinesis", AWSCredentialsConfig.class, Names.named("kinesis"));
+  }
+}
diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java
new file mode 100644
index 00000000000..622c1f37bbc
--- /dev/null
+++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java
@@ -0,0 +1,780 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.indexing.kinesis;
+
+import com.amazonaws.AmazonServiceException;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider;
+import com.amazonaws.client.builder.AwsClientBuilder;
+import com.amazonaws.services.kinesis.AmazonKinesis;
+import com.amazonaws.services.kinesis.AmazonKinesisClientBuilder;
+import com.amazonaws.services.kinesis.model.ExpiredIteratorException;
+import com.amazonaws.services.kinesis.model.GetRecordsRequest;
+import com.amazonaws.services.kinesis.model.GetRecordsResult;
+import com.amazonaws.services.kinesis.model.InvalidArgumentException;
+import com.amazonaws.services.kinesis.model.ProvisionedThroughputExceededException;
+import com.amazonaws.services.kinesis.model.Record;
+import com.amazonaws.services.kinesis.model.ResourceNotFoundException;
+import com.amazonaws.services.kinesis.model.Shard;
+import com.amazonaws.services.kinesis.model.ShardIteratorType;
+import com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClientBuilder;
+import com.amazonaws.util.AwsHostNameUtils;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Queues;
+import org.apache.druid.common.aws.AWSCredentialsConfig;
+import org.apache.druid.common.aws.AWSCredentialsUtils;
+import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord;
+import org.apache.druid.indexing.seekablestream.common.RecordSupplier;
+import org.apache.druid.indexing.seekablestream.common.StreamPartition;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.lang.invoke.MethodHandle;
+import java.lang.invoke.MethodHandles;
+import java.lang.reflect.Method;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+/**
+ * This class implements a local buffer for storing fetched Kinesis records. Fetching is done
+ * in background threads.
+ */
+public class KinesisRecordSupplier implements RecordSupplier<String, String>
+{
+  private static final EmittingLogger log = new EmittingLogger(KinesisRecordSupplier.class);
+  private static final long PROVISIONED_THROUGHPUT_EXCEEDED_BACKOFF_MS = 3000;
+  private static final long EXCEPTION_RETRY_DELAY_MS = 10000;
+
+  private static boolean isServiceExceptionRecoverable(AmazonServiceException ex)
+  {
+    final boolean isIOException = ex.getCause() instanceof IOException;
+    final boolean isTimeout = "RequestTimeout".equals(ex.getErrorCode());
+    return isIOException || isTimeout;
+  }
+
+  private class PartitionResource
+  {
+    private final StreamPartition<String> streamPartition;
+
+    // shardIterator points to the record that will be polled next by recordRunnable
+    // can be null when shard is closed due to the user shard splitting or changing the number
+    // of shards in the stream, in which case a 'EOS' marker is used by the KinesisRecordSupplier
+    // to indicate that this shard has no more records to read
+    @Nullable
+    private volatile String shardIterator;
+    private volatile boolean started;
+    private volatile boolean stopRequested;
+
+    PartitionResource(
+        StreamPartition<String> streamPartition
+    )
+    {
+      this.streamPartition = streamPartition;
+    }
+
+    void startBackgroundFetch()
+    {
+      if (started) {
+        return;
+      }
+
+      log.info(
+          "Starting scheduled fetch runnable for stream[%s] partition[%s]",
+          streamPartition.getStream(),
+          streamPartition.getPartitionId()
+      );
+
+      stopRequested = false;
+      started = true;
+
+      rescheduleRunnable(fetchDelayMillis);
+    }
+
+    void stopBackgroundFetch()
+    {
+      log.info(
+          "Stopping scheduled fetch runnable for stream[%s] partition[%s]",
+          streamPartition.getStream(),
+          streamPartition.getPartitionId()
+      );
+      stopRequested = true;
+    }
+
+
+    private Runnable getRecordRunnable()
+    {
+      return () -> {
+
+        if (stopRequested) {
+          started = false;
+          stopRequested = false;
+
+          log.info("Worker for partition[%s] has been stopped", streamPartition.getPartitionId());
+          return;
+        }
+
+        // used for retrying on InterruptedException
+        GetRecordsResult recordsResult = null;
+        OrderedPartitionableRecord<String, String> currRecord;
+
+        try {
+
+          if (shardIterator == null) {
+            log.info("shardIterator[%s] has been closed and has no more records", streamPartition.getPartitionId());
+
+            // add an end-of-shard marker so caller knows this shard is closed
+            currRecord = new OrderedPartitionableRecord<>(
+                streamPartition.getStream(),
+                streamPartition.getPartitionId(),
+                KinesisSequenceNumber.END_OF_SHARD_MARKER,
+                null
+            );
+
+            recordsResult = null;
+
+            if (!records.offer(currRecord, recordBufferOfferTimeout, TimeUnit.MILLISECONDS)) {
+              log.warn("OrderedPartitionableRecord buffer full, retrying in [%,dms]", recordBufferFullWait);
+              rescheduleRunnable(recordBufferFullWait);
+            }
+
+            return;
+          }
+
+          recordsResult = kinesis.getRecords(new GetRecordsRequest().withShardIterator(
+              shardIterator).withLimit(recordsPerFetch));
+
+          // list will come back empty if there are no records
+          for (Record kinesisRecord : recordsResult.getRecords()) {
+
+            final List<byte[]> data;
+
+            if (deaggregate) {
+              if (deaggregateHandle == null || getDataHandle == null) {
+                throw new ISE("deaggregateHandle or getDataHandle is null!");
+              }
+
+              data = new ArrayList<>();
+
+              final List userRecords = (List) deaggregateHandle.invokeExact(
+                  Collections.singletonList(kinesisRecord)
+              );
+
+              for (Object userRecord : userRecords) {
+                data.add(toByteArray((ByteBuffer) getDataHandle.invoke(userRecord)));
+              }
+            } else {
+              data = Collections.singletonList(toByteArray(kinesisRecord.getData()));
+            }
+
+            currRecord = new OrderedPartitionableRecord<>(
+                streamPartition.getStream(),
+                streamPartition.getPartitionId(),
+                kinesisRecord.getSequenceNumber(),
+                data
+            );
+
+
+            log.trace(
+                "Stream[%s] / partition[%s] / sequenceNum[%s] / bufferRemainingCapacity[%d]: %s",
+                currRecord.getStream(),
+                currRecord.getPartitionId(),
+                currRecord.getSequenceNumber(),
+                records.remainingCapacity(),
+                currRecord.getData().stream().map(StringUtils::fromUtf8).collect(Collectors.toList())
+            );
+
+            // If the buffer was full and we weren't able to add the message, grab a new stream iterator starting
+            // from this message and back off for a bit to let the buffer drain before retrying.
+            if (!records.offer(currRecord, recordBufferOfferTimeout, TimeUnit.MILLISECONDS)) {
+              log.warn(
+                  "OrderedPartitionableRecord buffer full, storing iterator and retrying in [%,dms]",
+                  recordBufferFullWait
+              );
+
+              shardIterator = kinesis.getShardIterator(
+                  currRecord.getStream(),
+                  currRecord.getPartitionId(),
+                  ShardIteratorType.AT_SEQUENCE_NUMBER.toString(),
+                  currRecord.getSequenceNumber()
+              ).getShardIterator();
+
+              rescheduleRunnable(recordBufferFullWait);
+              return;
+            }
+          }
+
+          shardIterator = recordsResult.getNextShardIterator(); // will be null if the shard has been closed
+
+          rescheduleRunnable(fetchDelayMillis);
+        }
+        catch (ProvisionedThroughputExceededException e) {
+          log.warn(
+              e,
+              "encounted ProvisionedThroughputExceededException while fetching records, this means "
+              + "that the request rate for the stream is too high, or the requested data is too large for "
+              + "the available throughput. Reduce the frequency or size of your requests."
+          );
+          long retryMs = Math.max(PROVISIONED_THROUGHPUT_EXCEEDED_BACKOFF_MS, fetchDelayMillis);
+          rescheduleRunnable(retryMs);
+        }
+        catch (InterruptedException e) {
+          // may happen if interrupted while BlockingQueue.offer() is waiting
+          log.warn(
+              e,
+              "Interrupted while waiting to add record to buffer, retrying in [%,dms]",
+              EXCEPTION_RETRY_DELAY_MS
+          );
+          rescheduleRunnable(EXCEPTION_RETRY_DELAY_MS);
+        }
+        catch (ExpiredIteratorException e) {
+          log.warn(
+              e,
+              "ShardIterator expired while trying to fetch records, retrying in [%,dms]",
+              fetchDelayMillis
+          );
+          if (recordsResult != null) {
+            shardIterator = recordsResult.getNextShardIterator(); // will be null if the shard has been closed
+            rescheduleRunnable(fetchDelayMillis);
+          } else {
+            throw new ISE("can't reschedule fetch records runnable, recordsResult is null??");
+          }
+        }
+        catch (ResourceNotFoundException | InvalidArgumentException e) {
+          // aws errors
+          log.error(e, "encounted AWS error while attempting to fetch records, will not retry");
+          throw e;
+        }
+        catch (AmazonServiceException e) {
+          if (isServiceExceptionRecoverable(e)) {
+            log.warn(e, "encounted unknown recoverable AWS exception, retrying in [%,dms]", EXCEPTION_RETRY_DELAY_MS);
+            rescheduleRunnable(EXCEPTION_RETRY_DELAY_MS);
+          } else {
+            log.warn(e, "encounted unknown unrecoverable AWS exception, will not retry");
+            throw new RuntimeException(e);
+          }
+        }
+        catch (Throwable e) {
+          // non transient errors
+          log.error(e, "unknown getRecordRunnable exception, will not retry");
+          throw new RuntimeException(e);
+        }
+
+      };
+    }
+
+    private void rescheduleRunnable(long delayMillis)
+    {
+      if (started && !stopRequested) {
+        try {
+          scheduledExec.schedule(getRecordRunnable(), delayMillis, TimeUnit.MILLISECONDS);
+        }
+        catch (RejectedExecutionException e) {
+          log.warn(
+              e,
+              "Caught RejectedExecutionException, KinesisRecordSupplier for partition[%s] has likely temporarily shutdown the ExecutorService."
+              + "This is expected behavior after calling seek(), seekToEarliest() and seekToLatest()",
+              streamPartition.getPartitionId()
+          );
+
+        }
+      } else {
+        log.info("Worker for partition[%s] has been stopped", streamPartition.getPartitionId());
+      }
+    }
+  }
+
+  // used for deaggregate
+  private final MethodHandle deaggregateHandle;
+  private final MethodHandle getDataHandle;
+
+  private final AmazonKinesis kinesis;
+
+  private final int recordsPerFetch;
+  private final int fetchDelayMillis;
+  private final boolean deaggregate;
+  private final int recordBufferOfferTimeout;
+  private final int recordBufferFullWait;
+  private final int fetchSequenceNumberTimeout;
+  private final int maxRecordsPerPoll;
+  private final int fetchThreads;
+  private final int recordBufferSize;
+
+  private ScheduledExecutorService scheduledExec;
+
+  private final Map<StreamPartition<String>, PartitionResource> partitionResources = new ConcurrentHashMap<>();
+  private BlockingQueue<OrderedPartitionableRecord<String, String>> records;
+
+  private volatile boolean checkPartitionsStarted = false;
+  private volatile boolean closed = false;
+
+  public KinesisRecordSupplier(
+      AmazonKinesis amazonKinesis,
+      int recordsPerFetch,
+      int fetchDelayMillis,
+      int fetchThreads,
+      boolean deaggregate,
+      int recordBufferSize,
+      int recordBufferOfferTimeout,
+      int recordBufferFullWait,
+      int fetchSequenceNumberTimeout,
+      int maxRecordsPerPoll
+  )
+  {
+    Preconditions.checkNotNull(amazonKinesis);
+    this.kinesis = amazonKinesis;
+    this.recordsPerFetch = recordsPerFetch;
+    this.fetchDelayMillis = fetchDelayMillis;
+    this.deaggregate = deaggregate;
+    this.recordBufferOfferTimeout = recordBufferOfferTimeout;
+    this.recordBufferFullWait = recordBufferFullWait;
+    this.fetchSequenceNumberTimeout = fetchSequenceNumberTimeout;
+    this.maxRecordsPerPoll = maxRecordsPerPoll;
+    this.fetchThreads = fetchThreads;
+    this.recordBufferSize = recordBufferSize;
+
+    // the deaggregate function is implemented by the amazon-kinesis-client, whose license is not compatible with Apache.
+    // The work around here is to use reflection to find the deaggregate function in the classpath. See details on the
+    // docs page for more information on how to use deaggregation
+    if (deaggregate) {
+      try {
+        Class<?> kclUserRecordclass = Class.forName("com.amazonaws.services.kinesis.clientlibrary.types.UserRecord");
+        MethodHandles.Lookup lookup = MethodHandles.publicLookup();
+
+        Method deaggregateMethod = kclUserRecordclass.getMethod("deaggregate", List.class);
+        Method getDataMethod = kclUserRecordclass.getMethod("getData");
+
+        deaggregateHandle = lookup.unreflect(deaggregateMethod);
+        getDataHandle = lookup.unreflect(getDataMethod);
+      }
+      catch (ClassNotFoundException e) {
+        throw new ISE(e, "cannot find class[com.amazonaws.services.kinesis.clientlibrary.types.UserRecord], "
+                         + "note that when using deaggregate=true, you must provide the Kinesis Client Library jar in the classpath");
+      }
+      catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    } else {
+      deaggregateHandle = null;
+      getDataHandle = null;
+    }
+
+    log.info(
+        "Creating fetch thread pool of size [%d] (Runtime.availableProcessors=%d)",
+        fetchThreads,
+        Runtime.getRuntime().availableProcessors()
+    );
+
+    scheduledExec = Executors.newScheduledThreadPool(
+        fetchThreads,
+        Execs.makeThreadFactory("KinesisRecordSupplier-Worker-%d")
+    );
+
+    records = new LinkedBlockingQueue<>(recordBufferSize);
+  }
+
+  public static AmazonKinesis getAmazonKinesisClient(
+      String endpoint,
+      AWSCredentialsConfig awsCredentialsConfig,
+      String awsAssumedRoleArn,
+      String awsExternalId
+  )
+  {
+    AWSCredentialsProvider awsCredentialsProvider = AWSCredentialsUtils.defaultAWSCredentialsProviderChain(
+        awsCredentialsConfig
+    );
+
+    if (awsAssumedRoleArn != null) {
+      log.info("Assuming role [%s] with externalId [%s]", awsAssumedRoleArn, awsExternalId);
+
+      STSAssumeRoleSessionCredentialsProvider.Builder builder = new STSAssumeRoleSessionCredentialsProvider
+          .Builder(awsAssumedRoleArn, StringUtils.format("druid-kinesis-%s", UUID.randomUUID().toString()))
+          .withStsClient(AWSSecurityTokenServiceClientBuilder.standard()
+                                                             .withCredentials(awsCredentialsProvider)
+                                                             .build());
+
+      if (awsExternalId != null) {
+        builder.withExternalId(awsExternalId);
+      }
+
+      awsCredentialsProvider = builder.build();
+    }
+
+    return AmazonKinesisClientBuilder.standard()
+                                     .withCredentials(awsCredentialsProvider)
+                                     .withClientConfiguration(new ClientConfiguration())
+                                     .withEndpointConfiguration(new AwsClientBuilder.EndpointConfiguration(
+                                         endpoint,
+                                         AwsHostNameUtils.parseRegion(
+                                             endpoint,
+                                             null
+                                         )
+                                     )).build();
+  }
+
+
+  @VisibleForTesting
+  public void start()
+  {
+    checkIfClosed();
+    if (checkPartitionsStarted) {
+      partitionResources.values().forEach(PartitionResource::startBackgroundFetch);
+      checkPartitionsStarted = false;
+    }
+  }
+
+  @Override
+  public void assign(Set<StreamPartition<String>> collection)
+  {
+    checkIfClosed();
+
+    collection.forEach(
+        streamPartition -> partitionResources.putIfAbsent(
+            streamPartition,
+            new PartitionResource(streamPartition)
+        )
+    );
+
+    for (Iterator<Map.Entry<StreamPartition<String>, PartitionResource>> i = partitionResources.entrySet()
+                                                                                               .iterator(); i.hasNext(); ) {
+      Map.Entry<StreamPartition<String>, PartitionResource> entry = i.next();
+      if (!collection.contains(entry.getKey())) {
+        i.remove();
+        entry.getValue().stopBackgroundFetch();
+      }
+    }
+
+  }
+
+  @Override
+  public void seek(StreamPartition<String> partition, String sequenceNumber) throws InterruptedException
+  {
+    checkIfClosed();
+    filterBufferAndResetFetchRunnable(ImmutableSet.of(partition));
+    seekInternal(partition, sequenceNumber, ShardIteratorType.AT_SEQUENCE_NUMBER);
+  }
+
+  @Override
+  public void seekToEarliest(Set<StreamPartition<String>> partitions) throws InterruptedException
+  {
+    checkIfClosed();
+    filterBufferAndResetFetchRunnable(partitions);
+    partitions.forEach(partition -> seekInternal(partition, null, ShardIteratorType.TRIM_HORIZON));
+  }
+
+  @Override
+  public void seekToLatest(Set<StreamPartition<String>> partitions) throws InterruptedException
+  {
+    checkIfClosed();
+    filterBufferAndResetFetchRunnable(partitions);
+    partitions.forEach(partition -> seekInternal(partition, null, ShardIteratorType.LATEST));
+  }
+
+  @Override
+  public Collection<StreamPartition<String>> getAssignment()
+  {
+    return partitionResources.keySet();
+  }
+
+  @Nonnull
+  @Override
+  public List<OrderedPartitionableRecord<String, String>> poll(long timeout)
+  {
+    checkIfClosed();
+    if (checkPartitionsStarted) {
+      partitionResources.values().forEach(PartitionResource::startBackgroundFetch);
+      checkPartitionsStarted = false;
+    }
+
+    try {
+      int expectedSize = Math.min(Math.max(records.size(), 1), maxRecordsPerPoll);
+
+      List<OrderedPartitionableRecord<String, String>> polledRecords = new ArrayList<>(expectedSize);
+
+      Queues.drain(
+          records,
+          polledRecords,
+          expectedSize,
+          timeout,
+          TimeUnit.MILLISECONDS
+      );
+
+      polledRecords = polledRecords.stream()
+                                   .filter(x -> partitionResources.containsKey(x.getStreamPartition()))
+                                   .collect(Collectors.toList());
+
+      return polledRecords;
+    }
+    catch (InterruptedException e) {
+      log.warn(e, "Interrupted while polling");
+      return Collections.emptyList();
+    }
+
+  }
+
+  @Nullable
+  @Override
+  public String getLatestSequenceNumber(StreamPartition<String> partition)
+  {
+    checkIfClosed();
+    return getSequenceNumberInternal(partition, ShardIteratorType.LATEST);
+  }
+
+  @Nullable
+  @Override
+  public String getEarliestSequenceNumber(StreamPartition<String> partition)
+  {
+    checkIfClosed();
+    return getSequenceNumberInternal(partition, ShardIteratorType.TRIM_HORIZON);
+  }
+
+  @Nullable
+  @Override
+  public String getPosition(StreamPartition<String> partition)
+  {
+    throw new UnsupportedOperationException("getPosition() is not supported in Kinesis");
+  }
+
+  @Override
+  public Set<String> getPartitionIds(String stream)
+  {
+    checkIfClosed();
+    return kinesis.describeStream(stream)
+                  .getStreamDescription()
+                  .getShards()
+                  .stream()
+                  .map(Shard::getShardId).collect(Collectors.toSet());
+  }
+
+  @Override
+  public void close()
+  {
+    if (this.closed) {
+      return;
+    }
+
+    assign(ImmutableSet.of());
+
+    scheduledExec.shutdown();
+
+    try {
+      if (!scheduledExec.awaitTermination(EXCEPTION_RETRY_DELAY_MS, TimeUnit.MILLISECONDS)) {
+        scheduledExec.shutdownNow();
+      }
+    }
+    catch (InterruptedException e) {
+      log.warn(e, "InterruptedException while shutting down");
+      throw new RuntimeException(e);
+    }
+
+    this.closed = true;
+  }
+
+  private void seekInternal(StreamPartition<String> partition, String sequenceNumber, ShardIteratorType iteratorEnum)
+  {
+    PartitionResource resource = partitionResources.get(partition);
+    if (resource == null) {
+      throw new ISE("Partition [%s] has not been assigned", partition);
+    }
+
+    log.debug(
+        "Seeking partition [%s] to [%s]",
+        partition.getPartitionId(),
+        sequenceNumber != null ? sequenceNumber : iteratorEnum.toString()
+    );
+
+    resource.shardIterator = kinesis.getShardIterator(
+        partition.getStream(),
+        partition.getPartitionId(),
+        iteratorEnum.toString(),
+        sequenceNumber
+    ).getShardIterator();
+
+    checkPartitionsStarted = true;
+  }
+
+  private void filterBufferAndResetFetchRunnable(Set<StreamPartition<String>> partitions) throws InterruptedException
+  {
+    scheduledExec.shutdown();
+
+    try {
+      if (!scheduledExec.awaitTermination(EXCEPTION_RETRY_DELAY_MS, TimeUnit.MILLISECONDS)) {
+        scheduledExec.shutdownNow();
+      }
+    }
+    catch (InterruptedException e) {
+      log.warn(e, "InterruptedException while shutting down");
+      throw e;
+    }
+
+    scheduledExec = Executors.newScheduledThreadPool(
+        fetchThreads,
+        Execs.makeThreadFactory("KinesisRecordSupplier-Worker-%d")
+    );
+
+    // filter records in buffer and only retain ones whose partition was not seeked
+    BlockingQueue<OrderedPartitionableRecord<String, String>> newQ = new LinkedBlockingQueue<>(recordBufferSize);
+    records
+        .stream()
+        .filter(x -> !partitions.contains(x.getStreamPartition()))
+        .forEachOrdered(newQ::offer);
+
+    records = newQ;
+
+    // restart fetching threads
+    partitionResources.values().forEach(x -> x.started = false);
+    checkPartitionsStarted = true;
+  }
+
+  @Nullable
+  private String getSequenceNumberInternal(StreamPartition<String> partition, ShardIteratorType iteratorEnum)
+  {
+
+    String shardIterator = null;
+    try {
+      shardIterator = kinesis.getShardIterator(
+          partition.getStream(),
+          partition.getPartitionId(),
+          iteratorEnum.toString()
+      ).getShardIterator();
+    }
+    catch (ResourceNotFoundException e) {
+      log.warn(e, "Caught ResourceNotFoundException while getting shardIterator");
+    }
+
+    return getSequenceNumberInternal(partition, shardIterator);
+  }
+
+  @Nullable
+  private String getSequenceNumberInternal(StreamPartition<String> partition, String shardIterator)
+  {
+    long timeoutMillis = System.currentTimeMillis() + fetchSequenceNumberTimeout;
+
+    while (shardIterator != null && System.currentTimeMillis() < timeoutMillis) {
+
+      if (closed) {
+        log.info("KinesisRecordSupplier closed while fetching sequenceNumber");
+        return null;
+      }
+
+      GetRecordsResult recordsResult;
+      try {
+        // we call getRecords with limit 1000 to make sure that we can find the first (earliest) record in the shard.
+        // In the case where the shard is constantly removing records that are past their retention period, it is possible
+        // that we never find the first record in the shard if we use a limit of 1.
+        recordsResult = kinesis.getRecords(new GetRecordsRequest().withShardIterator(shardIterator).withLimit(1000));
+      }
+      catch (ProvisionedThroughputExceededException e) {
+        log.warn(
+            e,
+            "encountered ProvisionedThroughputExceededException while fetching records, this means "
+            + "that the request rate for the stream is too high, or the requested data is too large for "
+            + "the available throughput. Reduce the frequency or size of your requests. Consider increasing "
+            + "the number of shards to increase throughput."
+        );
+        try {
+          Thread.sleep(PROVISIONED_THROUGHPUT_EXCEEDED_BACKOFF_MS);
+          continue;
+        }
+        catch (InterruptedException e1) {
+          log.warn(e1, "Thread interrupted!");
+          Thread.currentThread().interrupt();
+          break;
+        }
+      }
+
+      List<Record> records = recordsResult.getRecords();
+
+      if (!records.isEmpty()) {
+        return records.get(0).getSequenceNumber();
+      }
+
+      shardIterator = recordsResult.getNextShardIterator();
+    }
+
+    if (shardIterator == null) {
+      log.info("Partition[%s] returned a null shard iterator, is the shard closed?", partition.getPartitionId());
+      return KinesisSequenceNumber.END_OF_SHARD_MARKER;
+    }
+
+
+    // if we reach here, it usually means either the shard has no more records, or records have not been
+    // added to this shard
+    log.warn(
+        "timed out while trying to fetch position for shard[%s], likely no more records in shard",
+        partition.getPartitionId()
+    );
+    return null;
+
+  }
+
+  private void checkIfClosed()
+  {
+    if (closed) {
+      throw new ISE("Invalid operation - KinesisRecordSupplier has already been closed");
+    }
+  }
+
+  /**
+   * Returns an array with the content between the position and limit of "buffer". This may be the buffer's backing
+   * array itself. Does not modify position or limit of the buffer.
+   */
+  private static byte[] toByteArray(final ByteBuffer buffer)
+  {
+    if (buffer.hasArray()
+        && buffer.arrayOffset() == 0
+        && buffer.position() == 0
+        && buffer.array().length == buffer.limit()) {
+      return buffer.array();
+    } else {
+      final byte[] retVal = new byte[buffer.remaining()];
+      buffer.duplicate().get(retVal);
+      return retVal;
+    }
+  }
+
+  @VisibleForTesting
+  public int bufferSize()
+  {
+    return records.size();
+  }
+}
diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRegion.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRegion.java
new file mode 100644
index 00000000000..a5e1195616f
--- /dev/null
+++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRegion.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.indexing.kinesis;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonValue;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.StringUtils;
+
+import java.util.EnumSet;
+import java.util.List;
+import java.util.stream.Collectors;
+
+public enum KinesisRegion
+{
+  US_EAST_2,
+  US_EAST_1,
+  US_WEST_1,
+  US_WEST_2,
+  AP_NORTHEAST_1,
+  AP_NORTHEAST_2,
+  AP_NORTHEAST_3,
+  AP_SOUTH_1,
+  AP_SOUTHEAST_1,
+  AP_SOUTHEAST_2,
+  CA_CENTRAL_1,
+  CN_NORTH_1,
+  CN_NORTHWEST_1,
+  EU_CENTRAL_1,
+  EU_WEST_1,
+  EU_WEST_2,
+  EU_WEST_3,
+  SA_EAST_1,
+  US_GOV_WEST_1;
+
+  @JsonCreator
+  public static KinesisRegion fromString(String value)
+  {
+    return EnumSet.allOf(KinesisRegion.class)
+                  .stream()
+                  .filter(x -> x.serialize().equals(value))
+                  .findFirst()
+                  .orElseThrow(() -> new IAE("Invalid region %s, region must be one of: %s", value, getNames()));
+  }
+
+  private static List<String> getNames()
+  {
+    return EnumSet.allOf(KinesisRegion.class).stream().map(KinesisRegion::serialize).collect(Collectors.toList());
+  }
+
+  public String getEndpoint()
+  {
+    return StringUtils.format("kinesis.%s.amazonaws.com%s", serialize(), serialize().startsWith("cn-") ? ".cn" : "");
+  }
+
+  @JsonValue
+  public String serialize()
+  {
+    return StringUtils.toLowerCase(name()).replace('_', '-');
+  }
+}
diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java
new file mode 100644
index 00000000000..9a96e4ed598
--- /dev/null
+++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.indexing.kinesis;
+
+
+import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
+import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber;
+
+import javax.validation.constraints.NotNull;
+import java.math.BigInteger;
+
+public class KinesisSequenceNumber extends OrderedSequenceNumber<String>
+{
+
+  /**
+   * In Kinesis, when a shard is closed due to shard splitting, a null ShardIterator is returned.
+   * The EOS marker is placed at the end of the Kinesis Record Supplier buffer, such that when
+   * an indexing task pulls the record 'EOS', it knows the shard has been closed and should stop
+   * reading and start publishing
+   */
+  public static final String END_OF_SHARD_MARKER = "EOS";
+  // this flag is used to indicate either END_OF_SHARD_MARKER
+  // or NO_END_SEQUENCE_NUMBER so that they can be properly compared
+  // with other sequence numbers
+  private final boolean isMaxSequenceNumber;
+  private final BigInteger intSequence;
+
+  private KinesisSequenceNumber(@NotNull String sequenceNumber, boolean isExclusive)
+  {
+    super(sequenceNumber, isExclusive);
+    if (END_OF_SHARD_MARKER.equals(sequenceNumber)
+        || SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER.equals(sequenceNumber)) {
+      isMaxSequenceNumber = true;
+      this.intSequence = null;
+    } else {
+      isMaxSequenceNumber = false;
+      this.intSequence = new BigInteger(sequenceNumber);
+    }
+  }
+
+  public static KinesisSequenceNumber of(String sequenceNumber)
+  {
+    return new KinesisSequenceNumber(sequenceNumber, false);
+  }
+
+  public static KinesisSequenceNumber of(String sequenceNumber, boolean isExclusive)
+  {
+    return new KinesisSequenceNumber(sequenceNumber, isExclusive);
+  }
+
+  @Override
+  public int compareTo(@NotNull OrderedSequenceNumber<String> o)
+  {
+    KinesisSequenceNumber num = (KinesisSequenceNumber) o;
+    if (isMaxSequenceNumber && num.isMaxSequenceNumber) {
+      return 0;
+    } else if (isMaxSequenceNumber) {
+      return 1;
+    } else if (num.isMaxSequenceNumber) {
+      return -1;
+    } else {
+      return this.intSequence.compareTo(new BigInteger(o.get()));
+    }
+  }
+
+}
diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java
new file mode 100644
index 00000000000..f53bdf2ad12
--- /dev/null
+++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java
@@ -0,0 +1,311 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.indexing.kinesis.supervisor;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.common.aws.AWSCredentialsConfig;
+import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory;
+import org.apache.druid.indexing.common.task.Task;
+import org.apache.druid.indexing.common.task.TaskResource;
+import org.apache.druid.indexing.kinesis.KinesisDataSourceMetadata;
+import org.apache.druid.indexing.kinesis.KinesisIndexTask;
+import org.apache.druid.indexing.kinesis.KinesisIndexTaskClientFactory;
+import org.apache.druid.indexing.kinesis.KinesisIndexTaskIOConfig;
+import org.apache.druid.indexing.kinesis.KinesisIndexTaskTuningConfig;
+import org.apache.druid.indexing.kinesis.KinesisRecordSupplier;
+import org.apache.druid.indexing.kinesis.KinesisSequenceNumber;
+import org.apache.druid.indexing.overlord.DataSourceMetadata;
+import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
+import org.apache.druid.indexing.overlord.TaskMaster;
+import org.apache.druid.indexing.overlord.TaskStorage;
+import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata;
+import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask;
+import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig;
+import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig;
+import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
+import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber;
+import org.apache.druid.indexing.seekablestream.common.RecordSupplier;
+import org.apache.druid.indexing.seekablestream.common.StreamPartition;
+import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor;
+import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig;
+import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload;
+import org.apache.druid.indexing.seekablestream.utils.RandomIdUtils;
+import org.apache.druid.java.util.common.StringUtils;
+import org.joda.time.DateTime;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ScheduledExecutorService;
+
+/**
+ * Supervisor responsible for managing the KinesisIndexTask for a single dataSource. At a high level, the class accepts a
+ * {@link KinesisSupervisorSpec} which includes the Kinesis stream and configuration as well as an ingestion spec which will
+ * be used to generate the indexing tasks. The run loop periodically refreshes its view of the Kinesis stream's partitions
+ * and the list of running indexing tasks and ensures that all partitions are being read from and that there are enough
+ * tasks to satisfy the desired number of replicas. As tasks complete, new tasks are queued to process the next range of
+ * Kinesis sequences.
+ * <p>
+ * the Kinesis supervisor does not yet support lag calculations
+ */
+public class KinesisSupervisor extends SeekableStreamSupervisor<String, String>
+{
+  private static final String NOT_SET = "-1";
+  private final KinesisSupervisorSpec spec;
+  private final AWSCredentialsConfig awsCredentialsConfig;
+
+  public KinesisSupervisor(
+      final TaskStorage taskStorage,
+      final TaskMaster taskMaster,
+      final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator,
+      final KinesisIndexTaskClientFactory taskClientFactory,
+      final ObjectMapper mapper,
+      final KinesisSupervisorSpec spec,
+      final RowIngestionMetersFactory rowIngestionMetersFactory,
+      final AWSCredentialsConfig awsCredentialsConfig
+  )
+  {
+    super(
+        StringUtils.format("KinesisSupervisor-%s", spec.getDataSchema().getDataSource()),
+        taskStorage,
+        taskMaster,
+        indexerMetadataStorageCoordinator,
+        taskClientFactory,
+        mapper,
+        spec,
+        rowIngestionMetersFactory,
+        true
+    );
+
+    this.spec = spec;
+    this.awsCredentialsConfig = awsCredentialsConfig;
+  }
+
+  @Override
+  protected SeekableStreamIndexTaskIOConfig createTaskIoConfig(
+      int groupId,
+      Map<String, String> startPartitions,
+      Map<String, String> endPartitions,
+      String baseSequenceName,
+      DateTime minimumMessageTime,
+      DateTime maximumMessageTime,
+      Set<String> exclusiveStartSequenceNumberPartitions,
+      SeekableStreamSupervisorIOConfig ioConfigg
+  )
+  {
+    KinesisSupervisorIOConfig ioConfig = (KinesisSupervisorIOConfig) ioConfigg;
+    return new KinesisIndexTaskIOConfig(
+        groupId,
+        baseSequenceName,
+        new SeekableStreamPartitions<>(ioConfig.getStream(), startPartitions),
+        new SeekableStreamPartitions<>(ioConfig.getStream(), endPartitions),
+        true,
+        minimumMessageTime,
+        maximumMessageTime,
+        ioConfig.getEndpoint(),
+        ioConfig.getRecordsPerFetch(),
+        ioConfig.getFetchDelayMillis(),
+        exclusiveStartSequenceNumberPartitions,
+        ioConfig.getAwsAssumedRoleArn(),
+        ioConfig.getAwsExternalId(),
+        ioConfig.isDeaggregate()
+    );
+  }
+
+  @Override
+  protected List<SeekableStreamIndexTask<String, String>> createIndexTasks(
+      int replicas,
+      String baseSequenceName,
+      ObjectMapper sortingMapper,
+      TreeMap<Integer, Map<String, String>> sequenceOffsets,
+      SeekableStreamIndexTaskIOConfig taskIoConfig,
+      SeekableStreamIndexTaskTuningConfig taskTuningConfig,
+      RowIngestionMetersFactory rowIngestionMetersFactory
+  ) throws JsonProcessingException
+  {
+    final String checkpoints = sortingMapper.writerFor(new TypeReference<TreeMap<Integer, Map<String, String>>>()
+    {
+    }).writeValueAsString(sequenceOffsets);
+    final Map<String, Object> context = spec.getContext() == null
+                                        ? ImmutableMap.of(
+        "checkpoints",
+        checkpoints,
+        IS_INCREMENTAL_HANDOFF_SUPPORTED,
+        true
+    ) : ImmutableMap.<String, Object>builder()
+                                            .put("checkpoints", checkpoints)
+                                            .put(IS_INCREMENTAL_HANDOFF_SUPPORTED, true)
+                                            .putAll(spec.getContext())
+                                            .build();
+    List<SeekableStreamIndexTask<String, String>> taskList = new ArrayList<>();
+    for (int i = 0; i < replicas; i++) {
+      String taskId = Joiner.on("_").join(baseSequenceName, RandomIdUtils.getRandomId());
+      taskList.add(new KinesisIndexTask(
+          taskId,
+          new TaskResource(baseSequenceName, 1),
+          spec.getDataSchema(),
+          (KinesisIndexTaskTuningConfig) taskTuningConfig,
+          (KinesisIndexTaskIOConfig) taskIoConfig,
+          context,
+          null,
+          null,
+          rowIngestionMetersFactory,
+          awsCredentialsConfig
+      ));
+    }
+    return taskList;
+  }
+
+
+  @Override
+  protected RecordSupplier<String, String> setupRecordSupplier()
+      throws RuntimeException
+  {
+    KinesisSupervisorIOConfig ioConfig = spec.getIoConfig();
+    KinesisIndexTaskTuningConfig taskTuningConfig = spec.getTuningConfig();
+
+    return new KinesisRecordSupplier(
+        KinesisRecordSupplier.getAmazonKinesisClient(
+            ioConfig.getEndpoint(),
+            awsCredentialsConfig,
+            ioConfig.getAwsAssumedRoleArn(),
+            ioConfig.getAwsExternalId()
+        ),
+        ioConfig.getRecordsPerFetch(),
+        ioConfig.getFetchDelayMillis(),
+        1,
+        ioConfig.isDeaggregate(),
+        taskTuningConfig.getRecordBufferSize(),
+        taskTuningConfig.getRecordBufferOfferTimeout(),
+        taskTuningConfig.getRecordBufferFullWait(),
+        taskTuningConfig.getFetchSequenceNumberTimeout(),
+        taskTuningConfig.getMaxRecordsPerPoll()
+    );
+
+  }
+
+
+  @Override
+  protected void scheduleReporting(ScheduledExecutorService reportingExec)
+  {
+    // not yet implemented, see issue #6739
+  }
+
+  @Override
+  protected int getTaskGroupIdForPartition(String partitionId)
+  {
+    if (!partitionIds.contains(partitionId)) {
+      partitionIds.add(partitionId);
+    }
+
+    return partitionIds.indexOf(partitionId) % spec.getIoConfig().getTaskCount();
+  }
+
+  @Override
+  protected boolean checkSourceMetadataMatch(DataSourceMetadata metadata)
+  {
+    return metadata instanceof KinesisDataSourceMetadata;
+  }
+
+  @Override
+  protected boolean doesTaskTypeMatchSupervisor(Task task)
+  {
+    return task instanceof KinesisIndexTask;
+  }
+
+  @Override
+  protected SeekableStreamSupervisorReportPayload<String, String> createReportPayload(
+      int numPartitions,
+      boolean includeOffsets
+  )
+  {
+    KinesisSupervisorIOConfig ioConfig = spec.getIoConfig();
+    return new KinesisSupervisorReportPayload(
+        spec.getDataSchema().getDataSource(),
+        ioConfig.getStream(),
+        numPartitions,
+        ioConfig.getReplicas(),
+        ioConfig.getTaskDuration().getMillis() / 1000,
+        spec.isSuspended()
+    );
+  }
+
+  // not yet supported, will be implemented in the future
+  @Override
+  protected Map<String, String> getLagPerPartition(Map<String, String> currentOffsets)
+  {
+    return ImmutableMap.of();
+  }
+
+  @Override
+  protected SeekableStreamDataSourceMetadata<String, String> createDataSourceMetaData(
+      String stream,
+      Map<String, String> map
+  )
+  {
+    return new KinesisDataSourceMetadata(
+        new SeekableStreamPartitions<>(stream, map)
+    );
+  }
+
+  @Override
+  protected OrderedSequenceNumber<String> makeSequenceNumber(String seq, boolean isExclusive)
+  {
+    return KinesisSequenceNumber.of(seq, isExclusive);
+  }
+
+  @Override
+  protected void updateLatestSequenceFromStream(
+      RecordSupplier<String, String> recordSupplier, Set<StreamPartition<String>> streamPartitions
+  )
+  {
+    // do nothing
+  }
+
+  @Override
+  protected String baseTaskName()
+  {
+    return "index_kinesis";
+  }
+
+  @Override
+  protected String getNotSetMarker()
+  {
+    return NOT_SET;
+  }
+
+  @Override
+  protected String getEndOfPartitionMarker()
+  {
+    return SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER;
+  }
+
+  @Override
+  protected boolean isEndOfShard(String seqNum)
+  {
+    return KinesisSequenceNumber.END_OF_SHARD_MARKER.equals(seqNum);
+  }
+}
diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java
new file mode 100644
index 00000000000..2fb43d31517
--- /dev/null
+++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.indexing.kinesis.supervisor;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.druid.indexing.kinesis.KinesisIndexTaskIOConfig;
+import org.apache.druid.indexing.kinesis.KinesisRegion;
+import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig;
+import org.joda.time.Period;
+
+public class KinesisSupervisorIOConfig extends SeekableStreamSupervisorIOConfig
+{
+  private final String endpoint;
+
+  // In determining a suitable value for recordsPerFetch:
+  //   - Each data record can be up to 1 MB in size
+  //   - Each shard can read up to 2 MB per second
+  //   - The maximum size of data that GetRecords can return is 10 MB. If a call returns this amount of data,
+  //     subsequent calls made within the next 5 seconds throw ProvisionedThroughputExceededException.
+  //
+  // If there is insufficient provisioned throughput on the shard, subsequent calls made within the next 1 second
+  // throw ProvisionedThroughputExceededException. Note that GetRecords won't return any data when it throws an
+  // exception. For this reason, we recommend that you wait one second between calls to GetRecords; however, it's
+  // possible that the application will get exceptions for longer than 1 second.
+  private final Integer recordsPerFetch;
+  private final Integer fetchDelayMillis;
+
+  private final String awsAssumedRoleArn;
+  private final String awsExternalId;
+  private final boolean deaggregate;
+
+  @JsonCreator
+  public KinesisSupervisorIOConfig(
+      @JsonProperty("stream") String stream,
+      @JsonProperty("endpoint") String endpoint,
+      @JsonProperty("region") KinesisRegion region,
+      @JsonProperty("replicas") Integer replicas,
+      @JsonProperty("taskCount") Integer taskCount,
+      @JsonProperty("taskDuration") Period taskDuration,
+      @JsonProperty("startDelay") Period startDelay,
+      @JsonProperty("period") Period period,
+      @JsonProperty("useEarliestSequenceNumber") Boolean useEarliestSequenceNumber,
+      @JsonProperty("completionTimeout") Period completionTimeout,
+      @JsonProperty("lateMessageRejectionPeriod") Period lateMessageRejectionPeriod,
+      @JsonProperty("earlyMessageRejectionPeriod") Period earlyMessageRejectionPeriod,
+      @JsonProperty("recordsPerFetch") Integer recordsPerFetch,
+      @JsonProperty("fetchDelayMillis") Integer fetchDelayMillis,
+      @JsonProperty("awsAssumedRoleArn") String awsAssumedRoleArn,
+      @JsonProperty("awsExternalId") String awsExternalId,
+      @JsonProperty("deaggregate") boolean deaggregate
+  )
+  {
+    super(
+        Preconditions.checkNotNull(stream, "stream"),
+        replicas,
+        taskCount,
+        taskDuration,
+        startDelay,
+        period,
+        useEarliestSequenceNumber,
+        completionTimeout,
+        lateMessageRejectionPeriod,
+        earlyMessageRejectionPeriod
+    );
+    this.endpoint = endpoint != null
+                    ? endpoint
+                    : (region != null ? region.getEndpoint() : KinesisRegion.US_EAST_1.getEndpoint());
+    this.recordsPerFetch = recordsPerFetch != null
+                           ? recordsPerFetch
+                           : KinesisIndexTaskIOConfig.DEFAULT_RECORDS_PER_FETCH;
+    this.fetchDelayMillis = fetchDelayMillis != null
+                            ? fetchDelayMillis
+                            : KinesisIndexTaskIOConfig.DEFAULT_FETCH_DELAY_MILLIS;
+    this.awsAssumedRoleArn = awsAssumedRoleArn;
+    this.awsExternalId = awsExternalId;
+    this.deaggregate = deaggregate;
+  }
+
+  @JsonProperty
+  public String getEndpoint()
+  {
+    return endpoint;
+  }
+
+  @JsonProperty
+  public Integer getRecordsPerFetch()
+  {
+    return recordsPerFetch;
+  }
+
+  @JsonProperty
+  public Integer getFetchDelayMillis()
+  {
+    return fetchDelayMillis;
+  }
+
+  @JsonProperty
+  public String getAwsAssumedRoleArn()
+  {
+    return awsAssumedRoleArn;
+  }
+
+  @JsonProperty
+  public String getAwsExternalId()
+  {
+    return awsExternalId;
+  }
+
+  @JsonProperty
+  public boolean isDeaggregate()
+  {
+    return deaggregate;
+  }
+
+  @Override
+  public String toString()
+  {
+    return "KinesisSupervisorIOConfig{" +
+           "stream='" + getStream() + '\'' +
+           ", endpoint='" + endpoint + '\'' +
+           ", replicas=" + getReplicas() +
+           ", taskCount=" + getTaskCount() +
+           ", taskDuration=" + getTaskDuration() +
+           ", startDelay=" + getStartDelay() +
+           ", period=" + getPeriod() +
+           ", useEarliestSequenceNumber=" + isUseEarliestSequenceNumber() +
+           ", completionTimeout=" + getCompletionTimeout() +
+           ", lateMessageRejectionPeriod=" + getLateMessageRejectionPeriod() +
+           ", earlyMessageRejectionPeriod=" + getEarlyMessageRejectionPeriod() +
+           ", recordsPerFetch=" + recordsPerFetch +
+           ", fetchDelayMillis=" + fetchDelayMillis +
+           ", awsAssumedRoleArn='" + awsAssumedRoleArn + '\'' +
+           ", awsExternalId='" + awsExternalId + '\'' +
+           ", deaggregate=" + deaggregate +
+           '}';
+  }
+}
diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java
new file mode 100644
index 00000000000..fb08337b464
--- /dev/null
+++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.indexing.kinesis.supervisor;
+
+import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload;
+
+import java.util.Collections;
+
+public class KinesisSupervisorReportPayload extends SeekableStreamSupervisorReportPayload<String, String>
+{
+  public KinesisSupervisorReportPayload(
+      String dataSource,
+      String stream,
+      Integer partitions,
+      Integer replicas,
+      Long durationSeconds,
+      boolean suspended
+  )
+  {
+    super(
+        dataSource,
+        stream,
+        partitions,
+        replicas,
+        durationSeconds,
+        Collections.emptyMap(),
+        Collections.emptyMap(),
+        null,
+        null,
+        suspended
+    );
+  }
+
+  @Override
+  public String toString()
+  {
+    return "KinesisSupervisorReportPayload{" +
+           "dataSource='" + getDataSource() + '\'' +
+           ", stream='" + getStream() + '\'' +
+           ", partitions=" + getPartitions() +
+           ", replicas=" + getReplicas() +
+           ", durationSeconds=" + getDurationSeconds() +
+           ", active=" + getActiveTasks() +
+           ", publishing=" + getPublishingTasks() +
+           ", suspended=" + getSuspended() +
+           '}';
+  }
+
+}
diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java
new file mode 100644
index 00000000000..479851f8eb8
--- /dev/null
+++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.indexing.kinesis.supervisor;
+
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.inject.name.Named;
+import org.apache.druid.common.aws.AWSCredentialsConfig;
+import org.apache.druid.guice.annotations.Json;
+import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory;
+import org.apache.druid.indexing.kinesis.KinesisIndexTaskClientFactory;
+import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
+import org.apache.druid.indexing.overlord.TaskMaster;
+import org.apache.druid.indexing.overlord.TaskStorage;
+import org.apache.druid.indexing.overlord.supervisor.Supervisor;
+import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorSpec;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.segment.indexing.DataSchema;
+import org.apache.druid.server.metrics.DruidMonitorSchedulerConfig;
+
+import java.util.Map;
+
+public class KinesisSupervisorSpec extends SeekableStreamSupervisorSpec
+{
+  private final AWSCredentialsConfig awsCredentialsConfig;
+
+  @JsonCreator
+  public KinesisSupervisorSpec(
+      @JsonProperty("dataSchema") DataSchema dataSchema,
+      @JsonProperty("tuningConfig") KinesisSupervisorTuningConfig tuningConfig,
+      @JsonProperty("ioConfig") KinesisSupervisorIOConfig ioConfig,
+      @JsonProperty("context") Map<String, Object> context,
+      @JsonProperty("suspended") Boolean suspended,
+      @JacksonInject TaskStorage taskStorage,
+      @JacksonInject TaskMaster taskMaster,
+      @JacksonInject IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator,
+      @JacksonInject KinesisIndexTaskClientFactory kinesisIndexTaskClientFactory,
+      @JacksonInject @Json ObjectMapper mapper,
+      @JacksonInject ServiceEmitter emitter,
+      @JacksonInject DruidMonitorSchedulerConfig monitorSchedulerConfig,
+      @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory,
+      @JacksonInject @Named("kinesis") AWSCredentialsConfig awsCredentialsConfig
+  )
+  {
+    super(
+        dataSchema,
+        tuningConfig != null
+        ? tuningConfig
+        : new KinesisSupervisorTuningConfig(
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null
+        ),
+        ioConfig,
+        context,
+        suspended,
+        taskStorage,
+        taskMaster,
+        indexerMetadataStorageCoordinator,
+        kinesisIndexTaskClientFactory,
+        mapper,
+        emitter,
+        monitorSchedulerConfig,
+        rowIngestionMetersFactory
+    );
+    this.awsCredentialsConfig = awsCredentialsConfig;
+  }
+
+
+  @Override
+  public Supervisor createSupervisor()
+  {
+    return new KinesisSupervisor(
+        taskStorage,
+        taskMaster,
+        indexerMetadataStorageCoordinator,
+        (KinesisIndexTaskClientFactory) indexTaskClientFactory,
+        mapper,
+        this,
+        rowIngestionMetersFactory,
+        awsCredentialsConfig
+    );
+  }
+
+  @Override
+  public String toString()
+  {
+    return "KinesisSupervisorSpec{" +
+           "dataSchema=" + getDataSchema() +
+           ", tuningConfig=" + getTuningConfig() +
+           ", ioConfig=" + getIoConfig() +
+           ", suspended=" + isSuspended() +
+           ", context=" + getContext() +
+           '}';
+  }
+
+  @Override
+  @JsonProperty
+  public KinesisSupervisorTuningConfig getTuningConfig()
+  {
+    return (KinesisSupervisorTuningConfig) super.getTuningConfig();
+  }
+
+  @Override
+  @JsonProperty
+  public KinesisSupervisorIOConfig getIoConfig()
+  {
+    return (KinesisSupervisorIOConfig) super.getIoConfig();
+  }
+
+  @Override
+  protected KinesisSupervisorSpec toggleSuspend(boolean suspend)
+  {
+    return new KinesisSupervisorSpec(
+        getDataSchema(),
+        getTuningConfig(),
+        getIoConfig(),
+        getContext(),
+        suspend,
+        taskStorage,
+        taskMaster,
+        indexerMetadataStorageCoordinator,
+        (KinesisIndexTaskClientFactory) indexTaskClientFactory,
+        mapper,
+        emitter,
+        monitorSchedulerConfig,
+        rowIngestionMetersFactory,
+        awsCredentialsConfig
+    );
+  }
+}
diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java
new file mode 100644
index 00000000000..3c749be5cff
--- /dev/null
+++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java
@@ -0,0 +1,213 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.indexing.kinesis.supervisor;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.druid.indexing.kinesis.KinesisIndexTaskTuningConfig;
+import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig;
+import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorTuningConfig;
+import org.apache.druid.segment.IndexSpec;
+import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory;
+import org.joda.time.Duration;
+import org.joda.time.Period;
+
+import javax.annotation.Nullable;
+import java.io.File;
+
+public class KinesisSupervisorTuningConfig extends KinesisIndexTaskTuningConfig
+    implements SeekableStreamSupervisorTuningConfig
+{
+  private final Integer workerThreads;
+  private final Integer chatThreads;
+  private final Long chatRetries;
+  private final Duration httpTimeout;
+  private final Duration shutdownTimeout;
+
+  public KinesisSupervisorTuningConfig(
+      @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory,
+      @JsonProperty("maxBytesInMemory") Long maxBytesInMemory,
+      @JsonProperty("maxRowsPerSegment") Integer maxRowsPerSegment,
+      @JsonProperty("maxTotalRows") Long maxTotalRows,
+      @JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod,
+      @JsonProperty("basePersistDirectory") File basePersistDirectory,
+      @JsonProperty("maxPendingPersists") Integer maxPendingPersists,
+      @JsonProperty("indexSpec") IndexSpec indexSpec,
+      @JsonProperty("buildV9Directly") Boolean buildV9Directly,
+      @JsonProperty("reportParseExceptions") Boolean reportParseExceptions,
+      @JsonProperty("handoffConditionTimeout") Long handoffConditionTimeout,
+      @JsonProperty("resetOffsetAutomatically") Boolean resetOffsetAutomatically,
+      @JsonProperty("skipSequenceNumberAvailabilityCheck") Boolean skipSequenceNumberAvailabilityCheck,
+      @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory,
+      @JsonProperty("workerThreads") Integer workerThreads,
+      @JsonProperty("chatThreads") Integer chatThreads,
+      @JsonProperty("chatRetries") Long chatRetries,
+      @JsonProperty("httpTimeout") Period httpTimeout,
+      @JsonProperty("shutdownTimeout") Period shutdownTimeout,
+      @JsonProperty("recordBufferSize") Integer recordBufferSize,
+      @JsonProperty("recordBufferOfferTimeout") Integer recordBufferOfferTimeout,
+      @JsonProperty("recordBufferFullWait") Integer recordBufferFullWait,
+      @JsonProperty("fetchSequenceNumberTimeout") Integer fetchSequenceNumberTimeout,
+      @JsonProperty("fetchThreads") Integer fetchThreads,
+      @JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions,
+      @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions,
+      @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions,
+      @JsonProperty("maxRecordsPerPoll") @Nullable Integer maxRecordsPerPoll,
+      @JsonProperty("intermediateHandoffPeriod") Period intermediateHandoffPeriod
+  )
+  {
+    super(
+        maxRowsInMemory,
+        maxBytesInMemory,
+        maxRowsPerSegment,
+        maxTotalRows,
+        intermediatePersistPeriod,
+        basePersistDirectory,
+        maxPendingPersists,
+        indexSpec,
+        buildV9Directly,
+        reportParseExceptions,
+        handoffConditionTimeout,
+        resetOffsetAutomatically,
+        skipSequenceNumberAvailabilityCheck,
+        recordBufferSize,
+        recordBufferOfferTimeout,
+        recordBufferFullWait,
+        fetchSequenceNumberTimeout,
+        fetchThreads,
+        segmentWriteOutMediumFactory,
+        logParseExceptions,
+        maxParseExceptions,
+        maxSavedParseExceptions,
+        maxRecordsPerPoll,
+        intermediateHandoffPeriod
+    );
+
+    this.workerThreads = workerThreads;
+    this.chatThreads = chatThreads;
+    this.chatRetries = (chatRetries != null ? chatRetries : DEFAULT_CHAT_RETRIES);
+    this.httpTimeout = SeekableStreamSupervisorTuningConfig.defaultDuration(httpTimeout, DEFAULT_HTTP_TIMEOUT);
+    this.shutdownTimeout = SeekableStreamSupervisorTuningConfig.defaultDuration(
+        shutdownTimeout,
+        DEFAULT_SHUTDOWN_TIMEOUT
+    );
+  }
+
+  @Override
+  @JsonProperty
+  public Integer getWorkerThreads()
+  {
+    return workerThreads;
+  }
+
+  @Override
+  @JsonProperty
+  public Integer getChatThreads()
+  {
+    return chatThreads;
+  }
+
+  @Override
+  @JsonProperty
+  public Long getChatRetries()
+  {
+    return chatRetries;
+  }
+
+  @Override
+  @JsonProperty
+  public Duration getHttpTimeout()
+  {
+    return httpTimeout;
+  }
+
+  @Override
+  @JsonProperty
+  public Duration getShutdownTimeout()
+  {
+    return shutdownTimeout;
+  }
+
+  @Override
+  public String toString()
+  {
+    return "KinesisSupervisorTuningConfig{" +
+           "maxRowsInMemory=" + getMaxRowsInMemory() +
+           ", maxBytesInMemory=" + getMaxBytesInMemory() +
+           ", maxRowsPerSegment=" + getMaxRowsPerSegment() +
+           ", maxTotalRows=" + getMaxTotalRows() +
+           ", intermediatePersistPeriod=" + getIntermediatePersistPeriod() +
+           ", basePersistDirectory=" + getBasePersistDirectory() +
+           ", maxPendingPersists=" + getMaxPendingPersists() +
+           ", indexSpec=" + getIndexSpec() +
+           ", reportParseExceptions=" + isReportParseExceptions() +
+           ", handoffConditionTimeout=" + getHandoffConditionTimeout() +
+           ", resetOffsetAutomatically=" + isResetOffsetAutomatically() +
+           ", skipSequenceNumberAvailabilityCheck=" + isSkipSequenceNumberAvailabilityCheck() +
+           ", workerThreads=" + workerThreads +
+           ", chatThreads=" + chatThreads +
+           ", chatRetries=" + chatRetries +
+           ", httpTimeout=" + httpTimeout +
+           ", shutdownTimeout=" + shutdownTimeout +
+           ", recordBufferSize=" + getRecordBufferSize() +
+           ", recordBufferOfferTimeout=" + getRecordBufferOfferTimeout() +
+           ", recordBufferFullWait=" + getRecordBufferFullWait() +
+           ", fetchSequenceNumberTimeout=" + getFetchSequenceNumberTimeout() +
+           ", fetchThreads=" + getFetchThreads() +
+           ", segmentWriteOutMediumFactory=" + getSegmentWriteOutMediumFactory() +
+           ", logParseExceptions=" + isLogParseExceptions() +
+           ", maxParseExceptions=" + getMaxParseExceptions() +
+           ", maxSavedParseExceptions=" + getMaxSavedParseExceptions() +
+           ", maxRecordsPerPoll=" + getMaxRecordsPerPoll() +
+           ", intermediateHandoffPeriod=" + getIntermediateHandoffPeriod() +
+           '}';
+  }
+
+  @Override
+  public SeekableStreamIndexTaskTuningConfig convertToTaskTuningConfig()
+  {
+    return new KinesisIndexTaskTuningConfig(
+        getMaxRowsInMemory(),
+        getMaxBytesInMemory(),
+        getMaxRowsPerSegment(),
+        getMaxTotalRows(),
+        getIntermediatePersistPeriod(),
+        getBasePersistDirectory(),
+        getMaxPendingPersists(),
+        getIndexSpec(),
+        true,
+        isReportParseExceptions(),
+        getHandoffConditionTimeout(),
+        isResetOffsetAutomatically(),
+        isSkipSequenceNumberAvailabilityCheck(),
+        getRecordBufferSize(),
+        getRecordBufferOfferTimeout(),
+        getRecordBufferFullWait(),
+        getFetchSequenceNumberTimeout(),
+        getFetchThreads(),
+        getSegmentWriteOutMediumFactory(),
+        isLogParseExceptions(),
+        getMaxParseExceptions(),
+        getMaxSavedParseExceptions(),
+        getMaxRecordsPerPoll(),
+        getIntermediateHandoffPeriod()
+    );
+  }
+
+}
diff --git a/extensions-core/kinesis-indexing-service/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-core/kinesis-indexing-service/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule
new file mode 100644
index 00000000000..4c1d6cb0a28
--- /dev/null
+++ b/extensions-core/kinesis-indexing-service/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule
@@ -0,0 +1,16 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+org.apache.druid.indexing.kinesis.KinesisIndexingServiceModule
diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadataTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadataTest.java
new file mode 100644
index 00000000000..f1e3b0fca65
--- /dev/null
+++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadataTest.java
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+package org.apache.druid.indexing.kinesis;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Map;
+
+public class KinesisDataSourceMetadataTest
+{
+  private static final KinesisDataSourceMetadata KM0 = KM("foo", ImmutableMap.of());
+  private static final KinesisDataSourceMetadata KM1 = KM("foo", ImmutableMap.of("0", "2L", "1", "3L"));
+  private static final KinesisDataSourceMetadata KM2 = KM("foo", ImmutableMap.of("0", "2L", "1", "4L", "2", "5L"));
+  private static final KinesisDataSourceMetadata KM3 = KM("foo", ImmutableMap.of("0", "2L", "2", "5L"));
+
+  @Test
+  public void testMatches()
+  {
+    Assert.assertTrue(KM0.matches(KM0));
+    Assert.assertTrue(KM0.matches(KM1));
+    Assert.assertTrue(KM0.matches(KM2));
+    Assert.assertTrue(KM0.matches(KM3));
+
+    Assert.assertTrue(KM1.matches(KM0));
+    Assert.assertTrue(KM1.matches(KM1));
+    Assert.assertFalse(KM1.matches(KM2));
+    Assert.assertTrue(KM1.matches(KM3));
+
+    Assert.assertTrue(KM2.matches(KM0));
+    Assert.assertFalse(KM2.matches(KM1));
+    Assert.assertTrue(KM2.matches(KM2));
+    Assert.assertTrue(KM2.matches(KM3));
+
+    Assert.assertTrue(KM3.matches(KM0));
+    Assert.assertTrue(KM3.matches(KM1));
+    Assert.assertTrue(KM3.matches(KM2));
+    Assert.assertTrue(KM3.matches(KM3));
+  }
+
+  @Test
+  public void testIsValidStart()
+  {
+    Assert.assertTrue(KM0.isValidStart());
+    Assert.assertTrue(KM1.isValidStart());
+    Assert.assertTrue(KM2.isValidStart());
+    Assert.assertTrue(KM3.isValidStart());
+  }
+
+  @Test
+  public void testPlus()
+  {
+    Assert.assertEquals(
+        KM("foo", ImmutableMap.of("0", "2L", "1", "3L", "2", "5L")),
+        KM1.plus(KM3)
+    );
+
+    Assert.assertEquals(
+        KM("foo", ImmutableMap.of("0", "2L", "1", "4L", "2", "5L")),
+        KM0.plus(KM2)
+    );
+
+    Assert.assertEquals(
+        KM("foo", ImmutableMap.of("0", "2L", "1", "4L", "2", "5L")),
+        KM1.plus(KM2)
+    );
+
+    Assert.assertEquals(
+        KM("foo", ImmutableMap.of("0", "2L", "1", "3L", "2", "5L")),
+        KM2.plus(KM1)
+    );
+
+    Assert.assertEquals(
+        KM("foo", ImmutableMap.of("0", "2L", "1", "4L", "2", "5L")),
+        KM2.plus(KM2)
+    );
+  }
+
+  @Test
+  public void testMinus()
+  {
+    Assert.assertEquals(
+        KM("foo", ImmutableMap.of("1", "3L")),
+        KM1.minus(KM3)
+    );
+
+    Assert.assertEquals(
+        KM("foo", ImmutableMap.of()),
+        KM0.minus(KM2)
+    );
+
+    Assert.assertEquals(
+        KM("foo", ImmutableMap.of()),
+        KM1.minus(KM2)
+    );
+
+    Assert.assertEquals(
+        KM("foo", ImmutableMap.of("2", "5L")),
+        KM2.minus(KM1)
+    );
+
+    Assert.assertEquals(
+        KM("foo", ImmutableMap.of()),
+        KM2.minus(KM2)
+    );
+  }
+
+  private static KinesisDataSourceMetadata KM(String stream, Map<String, String> sequences)
+  {
+    return new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, sequences));
+  }
+}
diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java
new file mode 100644
index 00000000000..fff34f946b2
--- /dev/null
+++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java
@@ -0,0 +1,249 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.indexing.kinesis;
+
+import com.fasterxml.jackson.databind.JsonMappingException;
+import com.fasterxml.jackson.databind.Module;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import org.apache.druid.jackson.DefaultObjectMapper;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.segment.indexing.IOConfig;
+import org.hamcrest.CoreMatchers;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.util.Collections;
+
+public class KinesisIOConfigTest
+{
+  private final ObjectMapper mapper;
+
+  public KinesisIOConfigTest()
+  {
+    mapper = new DefaultObjectMapper();
+    mapper.registerModules((Iterable<Module>) new KinesisIndexingServiceModule().getJacksonModules());
+  }
+
+  @Rule
+  public final ExpectedException exception = ExpectedException.none();
+
+  @Test
+  public void testSerdeWithDefaults() throws Exception
+  {
+    String jsonStr = "{\n"
+                     + "  \"type\": \"kinesis\",\n"
+                     + "  \"baseSequenceName\": \"my-sequence-name\",\n"
+                     + "  \"endpoint\": \"kinesis.us-east-1.amazonaws.com\",\n"
+                     + "  \"startPartitions\": {\"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"1\", \"1\":\"10\"}},\n"
+                     + "  \"endPartitions\": {\"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"15\", \"1\":\"200\"}}\n"
+                     + "}";
+
+    KinesisIndexTaskIOConfig config = (KinesisIndexTaskIOConfig) mapper.readValue(
+        mapper.writeValueAsString(
+            mapper.readValue(
+                jsonStr,
+                IOConfig.class
+            )
+        ), IOConfig.class
+    );
+
+    Assert.assertNull(config.getTaskGroupId());
+    Assert.assertEquals("my-sequence-name", config.getBaseSequenceName());
+    Assert.assertEquals("mystream", config.getStartPartitions().getStream());
+    Assert.assertEquals(
+        ImmutableMap.of("0", "1", "1", "10"),
+        config.getStartPartitions().getPartitionSequenceNumberMap()
+    );
+    Assert.assertEquals("mystream", config.getEndPartitions().getStream());
+    Assert.assertEquals(
+        ImmutableMap.of("0", "15", "1", "200"),
+        config.getEndPartitions().getPartitionSequenceNumberMap()
+    );
+    Assert.assertTrue(config.isUseTransaction());
+    Assert.assertFalse("minimumMessageTime", config.getMinimumMessageTime().isPresent());
+    Assert.assertEquals(config.getEndpoint(), "kinesis.us-east-1.amazonaws.com");
+    Assert.assertEquals(config.getRecordsPerFetch(), 4000);
+    Assert.assertEquals(config.getFetchDelayMillis(), 0);
+    Assert.assertEquals(Collections.emptySet(), config.getExclusiveStartSequenceNumberPartitions());
+    Assert.assertNull(config.getAwsAssumedRoleArn());
+    Assert.assertNull(config.getAwsExternalId());
+    Assert.assertFalse(config.isDeaggregate());
+    Assert.assertTrue(config.isSkipOffsetGaps());
+  }
+
+  @Test
+  public void testSerdeWithNonDefaults() throws Exception
+  {
+    String jsonStr = "{\n"
+                     + "  \"taskGroupId\": 0,\n"
+                     + "  \"type\": \"kinesis\",\n"
+                     + "  \"baseSequenceName\": \"my-sequence-name\",\n"
+                     + "  \"startPartitions\": {\"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"1\", \"1\":\"10\"}},\n"
+                     + "  \"endPartitions\": {\"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"15\", \"1\":\"200\"}},\n"
+                     + "  \"useTransaction\": false,\n"
+                     + "  \"minimumMessageTime\": \"2016-05-31T12:00Z\",\n"
+                     + "  \"maximumMessageTime\": \"2016-05-31T14:00Z\",\n"
+                     + "  \"endpoint\": \"kinesis.us-east-2.amazonaws.com\",\n"
+                     + "  \"recordsPerFetch\": 1000,\n"
+                     + "  \"fetchDelayMillis\": 1000,\n"
+                     + "  \"exclusiveStartSequenceNumberPartitions\": [\"0\"],\n"
+                     + "  \"awsAssumedRoleArn\": \"role\",\n"
+                     + "  \"awsExternalId\": \"awsexternalid\",\n"
+                     + "  \"deaggregate\": true\n"
+                     + "}";
+
+    KinesisIndexTaskIOConfig config = (KinesisIndexTaskIOConfig) mapper.readValue(
+        mapper.writeValueAsString(
+            mapper.readValue(
+                jsonStr,
+                IOConfig.class
+            )
+        ), IOConfig.class
+    );
+
+    Assert.assertEquals((Integer) 0, config.getTaskGroupId());
+    Assert.assertEquals("my-sequence-name", config.getBaseSequenceName());
+    Assert.assertEquals("mystream", config.getStartPartitions().getStream());
+    Assert.assertEquals(
+        ImmutableMap.of("0", "1", "1", "10"),
+        config.getStartPartitions().getPartitionSequenceNumberMap()
+    );
+    Assert.assertEquals("mystream", config.getEndPartitions().getStream());
+    Assert.assertEquals(
+        ImmutableMap.of("0", "15", "1", "200"),
+        config.getEndPartitions().getPartitionSequenceNumberMap()
+    );
+    Assert.assertFalse(config.isUseTransaction());
+    Assert.assertTrue("maximumMessageTime", config.getMaximumMessageTime().isPresent());
+    Assert.assertTrue("minimumMessageTime", config.getMinimumMessageTime().isPresent());
+    Assert.assertEquals(DateTimes.of("2016-05-31T12:00Z"), config.getMinimumMessageTime().get());
+    Assert.assertEquals(DateTimes.of("2016-05-31T14:00Z"), config.getMaximumMessageTime().get());
+    Assert.assertEquals(config.getEndpoint(), "kinesis.us-east-2.amazonaws.com");
+    Assert.assertEquals(config.getExclusiveStartSequenceNumberPartitions(), ImmutableSet.of("0"));
+    Assert.assertEquals(1000, config.getRecordsPerFetch());
+    Assert.assertEquals(1000, config.getFetchDelayMillis());
+    Assert.assertEquals("role", config.getAwsAssumedRoleArn());
+    Assert.assertEquals("awsexternalid", config.getAwsExternalId());
+    Assert.assertTrue(config.isDeaggregate());
+    Assert.assertTrue(config.isSkipOffsetGaps());
+  }
+
+  @Test
+  public void testBaseSequenceNameRequired() throws Exception
+  {
+    String jsonStr = "{\n"
+                     + "  \"type\": \"kinesis\",\n"
+                     + "  \"endpoint\": \"kinesis.us-east-1.amazonaws.com\",\n"
+                     + "  \"startPartitions\": {\"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"1\", \"1\":\"10\"}},\n"
+                     + "  \"endPartitions\": {\"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"15\", \"1\":\"200\"}}\n"
+                     + "}";
+
+    exception.expect(JsonMappingException.class);
+    exception.expectCause(CoreMatchers.isA(NullPointerException.class));
+    exception.expectMessage(CoreMatchers.containsString("baseSequenceName"));
+    mapper.readValue(jsonStr, IOConfig.class);
+  }
+
+  @Test
+  public void testStartPartitionsRequired() throws Exception
+  {
+    String jsonStr = "{\n"
+                     + "  \"type\": \"kinesis\",\n"
+                     + "  \"baseSequenceName\": \"my-sequence-name\",\n"
+                     + "  \"endpoint\": \"kinesis.us-east-1.amazonaws.com\",\n"
+                     + "  \"endPartitions\": {\"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"15\", \"1\":\"200\"}}\n"
+                     + "}";
+
+    exception.expect(JsonMappingException.class);
+    exception.expectCause(CoreMatchers.isA(NullPointerException.class));
+    exception.expectMessage(CoreMatchers.containsString("startPartitions"));
+    mapper.readValue(jsonStr, IOConfig.class);
+  }
+
+  @Test
+  public void testEndPartitionsRequired() throws Exception
+  {
+    String jsonStr = "{\n"
+                     + "  \"type\": \"kinesis\",\n"
+                     + "  \"baseSequenceName\": \"my-sequence-name\",\n"
+                     + "  \"endpoint\": \"kinesis.us-east-1.amazonaws.com\",\n"
+                     + "  \"startPartitions\": {\"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"1\", \"1\":\"10\"}}\n"
+                     + "}";
+
+    exception.expect(JsonMappingException.class);
+    exception.expectCause(CoreMatchers.isA(NullPointerException.class));
+    exception.expectMessage(CoreMatchers.containsString("endPartitions"));
+    mapper.readValue(jsonStr, IOConfig.class);
+  }
+
+  @Test
+  public void testStartAndEndstreamMatch() throws Exception
+  {
+    String jsonStr = "{\n"
+                     + "  \"type\": \"kinesis\",\n"
+                     + "  \"baseSequenceName\": \"my-sequence-name\",\n"
+                     + "  \"endpoint\": \"kinesis.us-east-1.amazonaws.com\",\n"
+                     + "  \"startPartitions\": {\"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"1\", \"1\":\"10\"}},\n"
+                     + "  \"endPartitions\": {\"stream\":\"notmystream\", \"partitionSequenceNumberMap\" : {\"0\":\"15\", \"1\":\"200\"}}\n"
+                     + "}";
+
+    exception.expect(JsonMappingException.class);
+    exception.expectCause(CoreMatchers.isA(IllegalArgumentException.class));
+    exception.expectMessage(CoreMatchers.containsString("must match"));
+    mapper.readValue(jsonStr, IOConfig.class);
+  }
+
+  @Test
+  public void testStartAndEndPartitionSetMatch() throws Exception
+  {
+    String jsonStr = "{\n"
+                     + "  \"type\": \"kinesis\",\n"
+                     + "  \"baseSequenceName\": \"my-sequence-name\",\n"
+                     + "  \"endpoint\": \"kinesis.us-east-1.amazonaws.com\",\n"
+                     + "  \"startPartitions\": {\"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"1\", \"1\":\"10\"}},\n"
+                     + "  \"endPartitions\": {\"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"15\", \"2\":\"200\"}}\n"
+                     + "}";
+
+    exception.expect(JsonMappingException.class);
+    exception.expectCause(CoreMatchers.isA(IllegalArgumentException.class));
+    exception.expectMessage(CoreMatchers.containsString("start partition set and end partition set must match"));
+    mapper.readValue(jsonStr, IOConfig.class);
+  }
+
+  @Test
+  public void testEndPointRequired() throws Exception
+  {
+    String jsonStr = "{\n"
+                     + "  \"type\": \"kinesis\",\n"
+                     + "  \"baseSequenceName\": \"my-sequence-name\",\n"
+                     + "  \"startPartitions\": {\"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"1\", \"1\":\"10\"}},\n"
+                     + "  \"endPartitions\": {\"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"15\", \"1\":\"200\"}}\n"
+                     + "}";
+
+    exception.expect(JsonMappingException.class);
+    exception.expectCause(CoreMatchers.isA(NullPointerException.class));
+    exception.expectMessage(CoreMatchers.containsString("endpoint"));
+    mapper.readValue(jsonStr, IOConfig.class);
+  }
+}
diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java
new file mode 100644
index 00000000000..9d7fafbb4c7
--- /dev/null
+++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java
@@ -0,0 +1,1059 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.indexing.kinesis;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Optional;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.apache.druid.indexer.TaskLocation;
+import org.apache.druid.indexer.TaskStatus;
+import org.apache.druid.indexing.common.IndexTaskClient;
+import org.apache.druid.indexing.common.TaskInfoProvider;
+import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner.Status;
+import org.apache.druid.jackson.DefaultObjectMapper;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.http.client.HttpClient;
+import org.apache.druid.java.util.http.client.Request;
+import org.apache.druid.java.util.http.client.response.FullResponseHandler;
+import org.apache.druid.java.util.http.client.response.FullResponseHolder;
+import org.easymock.Capture;
+import org.easymock.CaptureType;
+import org.easymock.EasyMock;
+import org.easymock.EasyMockSupport;
+import org.jboss.netty.handler.codec.http.HttpHeaders;
+import org.jboss.netty.handler.codec.http.HttpMethod;
+import org.jboss.netty.handler.codec.http.HttpResponse;
+import org.jboss.netty.handler.codec.http.HttpResponseStatus;
+import org.joda.time.DateTime;
+import org.joda.time.Duration;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.IOException;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import static org.easymock.EasyMock.expect;
+
+
+@RunWith(Parameterized.class)
+public class KinesisIndexTaskClientTest extends EasyMockSupport
+{
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();
+
+  private static final ObjectMapper objectMapper = new DefaultObjectMapper();
+  private static final String TEST_ID = "test-id";
+  private static final List<String> TEST_IDS = Arrays.asList("test-id1", "test-id2", "test-id3", "test-id4");
+  private static final String TEST_HOST = "test-host";
+  private static final int TEST_PORT = 1234;
+  private static final int TEST_TLS_PORT = -1;
+  private static final String TEST_DATASOURCE = "test-datasource";
+  private static final Duration TEST_HTTP_TIMEOUT = new Duration(5000);
+  private static final long TEST_NUM_RETRIES = 0;
+  private static final String URL_FORMATTER = "http://%s:%d/druid/worker/v1/chat/%s/%s";
+
+  private int numThreads;
+  private HttpClient httpClient;
+  private TaskInfoProvider taskInfoProvider;
+  private FullResponseHolder responseHolder;
+  private HttpResponse response;
+  private HttpHeaders headers;
+  private KinesisIndexTaskClient client;
+
+  @Parameterized.Parameters(name = "numThreads = {0}")
+  public static Iterable<Object[]> constructorFeeder()
+  {
+    return ImmutableList.of(new Object[]{1}, new Object[]{8});
+  }
+
+  public KinesisIndexTaskClientTest(int numThreads)
+  {
+    this.numThreads = numThreads;
+  }
+
+  @Before
+  public void setUp()
+  {
+    httpClient = createMock(HttpClient.class);
+    taskInfoProvider = createMock(TaskInfoProvider.class);
+    responseHolder = createMock(FullResponseHolder.class);
+    response = createMock(HttpResponse.class);
+    headers = createMock(HttpHeaders.class);
+
+    client = new TestableKinesisIndexTaskClient(httpClient, objectMapper, taskInfoProvider);
+    expect(taskInfoProvider.getTaskLocation(TEST_ID))
+        .andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT))
+        .anyTimes();
+    expect(taskInfoProvider.getTaskStatus(TEST_ID))
+        .andReturn(Optional.of(TaskStatus.running(TEST_ID)))
+        .anyTimes();
+
+    for (String testId : TEST_IDS) {
+      expect(taskInfoProvider.getTaskLocation(testId))
+          .andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT))
+          .anyTimes();
+      expect(taskInfoProvider.getTaskStatus(testId))
+          .andReturn(Optional.of(TaskStatus.running(testId)))
+          .anyTimes();
+    }
+  }
+
+  @After
+  public void tearDown()
+  {
+    client.close();
+  }
+
+  @Test
+  public void testNoTaskLocation() throws IOException
+  {
+    EasyMock.reset(taskInfoProvider);
+    expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(TaskLocation.unknown()).anyTimes();
+    expect(taskInfoProvider.getTaskStatus(TEST_ID))
+        .andReturn(Optional.of(TaskStatus.running(TEST_ID)))
+        .anyTimes();
+    replayAll();
+
+    Assert.assertFalse(client.stop(TEST_ID, true));
+    Assert.assertFalse(client.resume(TEST_ID));
+    Assert.assertEquals(ImmutableMap.of(), client.pause(TEST_ID));
+    Assert.assertEquals(ImmutableMap.of(), client.pause(TEST_ID));
+    Assert.assertEquals(Status.NOT_STARTED, client.getStatus(TEST_ID));
+    Assert.assertNull(client.getStartTime(TEST_ID));
+    Assert.assertEquals(ImmutableMap.of(), client.getCurrentOffsets(TEST_ID, true));
+    Assert.assertEquals(ImmutableMap.of(), client.getEndOffsets(TEST_ID));
+    Assert.assertFalse(client.setEndOffsets(TEST_ID, Collections.emptyMap(), true));
+    Assert.assertFalse(client.setEndOffsets(TEST_ID, Collections.emptyMap(), true));
+
+    verifyAll();
+  }
+
+  @Test
+  public void testTaskNotRunnableException()
+  {
+    expectedException.expect(IndexTaskClient.TaskNotRunnableException.class);
+    expectedException.expectMessage("Aborting request because task [test-id] is not runnable");
+
+    EasyMock.reset(taskInfoProvider);
+    expect(taskInfoProvider.getTaskLocation(TEST_ID))
+        .andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT))
+        .anyTimes();
+    expect(taskInfoProvider.getTaskStatus(TEST_ID))
+        .andReturn(Optional.of(TaskStatus.failure(TEST_ID)))
+        .anyTimes();
+    replayAll();
+
+    client.getCurrentOffsets(TEST_ID, true);
+    verifyAll();
+  }
+
+  @Test
+  public void testInternalServerError()
+  {
+    expectedException.expect(RuntimeException.class);
+    expectedException.expectMessage("org.apache.druid.java.util.common.IOE: Received status [500]");
+
+    expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.INTERNAL_SERVER_ERROR).times(2);
+    expect(
+        httpClient.go(
+            EasyMock.anyObject(Request.class),
+            EasyMock.anyObject(FullResponseHandler.class),
+            EasyMock.eq(TEST_HTTP_TIMEOUT)
+        )
+    ).andReturn(
+        Futures.immediateFuture(responseHolder)
+    );
+    replayAll();
+
+    client.getCurrentOffsets(TEST_ID, true);
+    verifyAll();
+  }
+
+  @Test
+  public void testBadRequest()
+  {
+    expectedException.expect(IAE.class);
+    expectedException.expectMessage("Received 400 Bad Request with body:");
+
+    expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.BAD_REQUEST).times(2);
+    expect(responseHolder.getContent()).andReturn("");
+    expect(
+        httpClient.go(
+            EasyMock.anyObject(Request.class),
+            EasyMock.anyObject(FullResponseHandler.class),
+            EasyMock.eq(TEST_HTTP_TIMEOUT)
+        )
+    ).andReturn(
+        Futures.immediateFuture(responseHolder)
+    );
+    replayAll();
+
+    client.getCurrentOffsets(TEST_ID, true);
+    verifyAll();
+  }
+
+  @Test
+  public void testTaskLocationMismatch()
+  {
+    expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(3)
+                                      .andReturn(HttpResponseStatus.OK);
+    expect(responseHolder.getResponse()).andReturn(response);
+    expect(responseHolder.getContent()).andReturn("")
+                                       .andReturn("{}");
+    expect(response.headers()).andReturn(headers);
+    expect(headers.get("X-Druid-Task-Id")).andReturn("a-different-task-id");
+    expect(
+        httpClient.go(
+            EasyMock.anyObject(Request.class),
+            EasyMock.anyObject(FullResponseHandler.class),
+            EasyMock.eq(TEST_HTTP_TIMEOUT)
+        )
+    ).andReturn(
+        Futures.immediateFuture(responseHolder)
+    ).times(2);
+    replayAll();
+
+    Map<String, String> results = client.getCurrentOffsets(TEST_ID, true);
+    verifyAll();
+
+    Assert.assertEquals(0, results.size());
+  }
+
+  @Test
+  public void testGetCurrentOffsets() throws Exception
+  {
+    Capture<Request> captured = Capture.newInstance();
+    expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK);
+    expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}");
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
+        Futures.immediateFuture(responseHolder)
+    );
+    replayAll();
+
+    Map<String, String> results = client.getCurrentOffsets(TEST_ID, true);
+    verifyAll();
+
+    Request request = captured.getValue();
+    Assert.assertEquals(HttpMethod.GET, request.getMethod());
+    Assert.assertEquals(
+        new URL("http://test-host:1234/druid/worker/v1/chat/test-id/offsets/current"),
+        request.getUrl()
+    );
+    Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id"));
+
+    Assert.assertEquals(2, results.size());
+    Assert.assertEquals("1", results.get("0"));
+    Assert.assertEquals("10", results.get("1"));
+  }
+
+  @Test
+  public void testGetCurrentOffsetsWithRetry() throws Exception
+  {
+    client = new TestableKinesisIndexTaskClient(httpClient, objectMapper, taskInfoProvider, 3);
+
+    Capture<Request> captured = Capture.newInstance(CaptureType.ALL);
+    expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(6)
+                                      .andReturn(HttpResponseStatus.OK).times(1);
+    expect(responseHolder.getContent()).andReturn("").times(2)
+                                       .andReturn("{\"0\":1, \"1\":10}");
+    expect(responseHolder.getResponse()).andReturn(response).times(2);
+    expect(response.headers()).andReturn(headers).times(2);
+    expect(headers.get("X-Druid-Task-Id")).andReturn(TEST_ID).times(2);
+
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
+        Futures.immediateFuture(responseHolder)
+    ).times(3);
+
+    replayAll();
+
+    Map<String, String> results = client.getCurrentOffsets(TEST_ID, true);
+    verifyAll();
+
+    Assert.assertEquals(3, captured.getValues().size());
+    for (Request request : captured.getValues()) {
+      Assert.assertEquals(HttpMethod.GET, request.getMethod());
+      Assert.assertEquals(
+          new URL("http://test-host:1234/druid/worker/v1/chat/test-id/offsets/current"),
+          request.getUrl()
+      );
+      Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id"));
+    }
+
+    Assert.assertEquals(2, results.size());
+    Assert.assertEquals("1", results.get("0"));
+    Assert.assertEquals("10", results.get("1"));
+  }
+
+  @Test
+  public void testGetCurrentOffsetsWithExhaustedRetries()
+  {
+    expectedException.expect(RuntimeException.class);
+    expectedException.expectMessage("org.apache.druid.java.util.common.IOE: Received status [404]");
+
+    client = new TestableKinesisIndexTaskClient(httpClient, objectMapper, taskInfoProvider, 2);
+
+    expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).anyTimes();
+    expect(responseHolder.getContent()).andReturn("").anyTimes();
+    expect(responseHolder.getResponse()).andReturn(response).anyTimes();
+    expect(response.headers()).andReturn(headers).anyTimes();
+    expect(headers.get("X-Druid-Task-Id")).andReturn(TEST_ID).anyTimes();
+
+    expect(
+        httpClient.go(
+            EasyMock.anyObject(Request.class),
+            EasyMock.anyObject(FullResponseHandler.class),
+            EasyMock.eq(TEST_HTTP_TIMEOUT)
+        )
+    ).andReturn(Futures.immediateFuture(responseHolder)).anyTimes();
+    replayAll();
+
+    client.getCurrentOffsets(TEST_ID, true);
+    verifyAll();
+  }
+
+  @Test
+  public void testGetEndOffsets() throws Exception
+  {
+    Capture<Request> captured = Capture.newInstance();
+    expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK);
+    expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}");
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
+        Futures.immediateFuture(responseHolder)
+    );
+    replayAll();
+
+    Map<String, String> results = client.getEndOffsets(TEST_ID);
+    verifyAll();
+
+    Request request = captured.getValue();
+    Assert.assertEquals(HttpMethod.GET, request.getMethod());
+    Assert.assertEquals(
+        new URL("http://test-host:1234/druid/worker/v1/chat/test-id/offsets/end"),
+        request.getUrl()
+    );
+    Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id"));
+
+    Assert.assertEquals(2, results.size());
+    Assert.assertEquals("1", results.get("0"));
+    Assert.assertEquals("10", results.get("1"));
+  }
+
+  @Test
+  public void testGetStartTime() throws Exception
+  {
+    client = new TestableKinesisIndexTaskClient(httpClient, objectMapper, taskInfoProvider, 2);
+    DateTime now = DateTimes.nowUtc();
+
+    Capture<Request> captured = Capture.newInstance();
+    expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(3)
+                                      .andReturn(HttpResponseStatus.OK);
+    expect(responseHolder.getResponse()).andReturn(response);
+    expect(response.headers()).andReturn(headers);
+    expect(headers.get("X-Druid-Task-Id")).andReturn(null);
+    expect(responseHolder.getContent()).andReturn(String.valueOf(now.getMillis())).anyTimes();
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
+        Futures.immediateFuture(responseHolder)
+    ).times(2);
+    replayAll();
+
+    DateTime results = client.getStartTime(TEST_ID);
+    verifyAll();
+
+    Request request = captured.getValue();
+    Assert.assertEquals(HttpMethod.GET, request.getMethod());
+    Assert.assertEquals(
+        new URL("http://test-host:1234/druid/worker/v1/chat/test-id/time/start"),
+        request.getUrl()
+    );
+    Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id"));
+
+    Assert.assertEquals(now, results);
+  }
+
+  @Test
+  public void testGetStatus() throws Exception
+  {
+    Status status = Status.READING;
+
+    Capture<Request> captured = Capture.newInstance();
+    expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK);
+    expect(responseHolder.getContent()).andReturn(StringUtils.format("\"%s\"", status.toString())).anyTimes();
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
+        Futures.immediateFuture(responseHolder)
+    );
+    replayAll();
+
+    Status results = client.getStatus(TEST_ID);
+    verifyAll();
+
+    Request request = captured.getValue();
+    Assert.assertEquals(HttpMethod.GET, request.getMethod());
+    Assert.assertEquals(
+        new URL("http://test-host:1234/druid/worker/v1/chat/test-id/status"),
+        request.getUrl()
+    );
+    Assert.assertTrue(null, request.getHeaders().get("X-Druid-Task-Id").contains("test-id"));
+
+    Assert.assertEquals(status, results);
+  }
+
+  @Test
+  public void testPause() throws Exception
+  {
+    Capture<Request> captured = Capture.newInstance();
+    expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).times(2);
+    expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}").anyTimes();
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
+        Futures.immediateFuture(responseHolder)
+    );
+    replayAll();
+
+    Map<String, String> results = client.pause(TEST_ID);
+    verifyAll();
+
+    Request request = captured.getValue();
+    Assert.assertEquals(HttpMethod.POST, request.getMethod());
+    Assert.assertEquals(
+        new URL("http://test-host:1234/druid/worker/v1/chat/test-id/pause"),
+        request.getUrl()
+    );
+    Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id"));
+
+    Assert.assertEquals(2, results.size());
+    Assert.assertEquals("1", results.get("0"));
+    Assert.assertEquals("10", results.get("1"));
+  }
+
+  @Test
+  public void testPauseWithSubsequentGetOffsets() throws Exception
+  {
+    Capture<Request> captured = Capture.newInstance();
+    Capture<Request> captured2 = Capture.newInstance();
+    Capture<Request> captured3 = Capture.newInstance();
+    expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.ACCEPTED).times(2)
+                                      .andReturn(HttpResponseStatus.OK).anyTimes();
+    expect(responseHolder.getContent()).andReturn("\"PAUSED\"").times(2)
+                                       .andReturn("{\"0\":1, \"1\":10}").anyTimes();
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
+        Futures.immediateFuture(responseHolder)
+    );
+    expect(httpClient.go(
+        EasyMock.capture(captured2),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
+        Futures.immediateFuture(responseHolder)
+    );
+    expect(httpClient.go(
+        EasyMock.capture(captured3),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
+        Futures.immediateFuture(responseHolder)
+    );
+
+    replayAll();
+
+    Map<String, String> results = client.pause(TEST_ID);
+    verifyAll();
+
+    Request request = captured.getValue();
+    Assert.assertEquals(HttpMethod.POST, request.getMethod());
+    Assert.assertEquals(
+        new URL("http://test-host:1234/druid/worker/v1/chat/test-id/pause"),
+        request.getUrl()
+    );
+    Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id"));
+
+    request = captured2.getValue();
+    Assert.assertEquals(HttpMethod.GET, request.getMethod());
+    Assert.assertEquals(
+        new URL("http://test-host:1234/druid/worker/v1/chat/test-id/status"),
+        request.getUrl()
+    );
+
+    request = captured3.getValue();
+    Assert.assertEquals(HttpMethod.GET, request.getMethod());
+    Assert.assertEquals(
+        new URL("http://test-host:1234/druid/worker/v1/chat/test-id/offsets/current"),
+        request.getUrl()
+    );
+
+    Assert.assertEquals(2, results.size());
+    Assert.assertEquals("1", results.get("0"));
+    Assert.assertEquals("10", results.get("1"));
+  }
+
+  @Test
+  public void testResume() throws Exception
+  {
+    Capture<Request> captured = Capture.newInstance();
+    expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes();
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
+        Futures.immediateFuture(responseHolder)
+    );
+    replayAll();
+
+    client.resume(TEST_ID);
+    verifyAll();
+
+    Request request = captured.getValue();
+    Assert.assertEquals(HttpMethod.POST, request.getMethod());
+    Assert.assertEquals(
+        new URL("http://test-host:1234/druid/worker/v1/chat/test-id/resume"),
+        request.getUrl()
+    );
+    Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id"));
+  }
+
+  @Test
+  public void testSetEndOffsets() throws Exception
+  {
+    Map<String, String> endOffsets = ImmutableMap.of("0", "15", "1", "120");
+
+    Capture<Request> captured = Capture.newInstance();
+    expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes();
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
+        Futures.immediateFuture(responseHolder)
+    );
+    replayAll();
+
+    client.setEndOffsets(TEST_ID, endOffsets, true);
+    verifyAll();
+
+    Request request = captured.getValue();
+    Assert.assertEquals(HttpMethod.POST, request.getMethod());
+    Assert.assertEquals(
+        new URL("http://test-host:1234/druid/worker/v1/chat/test-id/offsets/end?finish=true"),
+        request.getUrl()
+    );
+    Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id"));
+    Assert.assertEquals("{\"0\":\"15\",\"1\":\"120\"}", StringUtils.fromUtf8(request.getContent().array()));
+  }
+
+  @Test
+  public void testSetEndOffsetsAndResume() throws Exception
+  {
+    Map<String, String> endOffsets = ImmutableMap.of("0", "15", "1", "120");
+
+    Capture<Request> captured = Capture.newInstance();
+    expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes();
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
+        Futures.immediateFuture(responseHolder)
+    );
+    replayAll();
+
+    client.setEndOffsets(TEST_ID, endOffsets, true);
+    verifyAll();
+
+    Request request = captured.getValue();
+    Assert.assertEquals(HttpMethod.POST, request.getMethod());
+    Assert.assertEquals(
+        new URL("http://test-host:1234/druid/worker/v1/chat/test-id/offsets/end?finish=true"),
+        request.getUrl()
+    );
+    Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id"));
+    Assert.assertEquals("{\"0\":\"15\",\"1\":\"120\"}", StringUtils.fromUtf8(request.getContent().array()));
+  }
+
+  @Test
+  public void testStop() throws Exception
+  {
+    Capture<Request> captured = Capture.newInstance();
+    expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes();
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
+        Futures.immediateFuture(responseHolder)
+    );
+    replayAll();
+
+    client.stop(TEST_ID, false);
+    verifyAll();
+
+    Request request = captured.getValue();
+    Assert.assertEquals(HttpMethod.POST, request.getMethod());
+    Assert.assertEquals(
+        new URL("http://test-host:1234/druid/worker/v1/chat/test-id/stop"),
+        request.getUrl()
+    );
+    Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id"));
+  }
+
+  @Test
+  public void testStopAndPublish() throws Exception
+  {
+    Capture<Request> captured = Capture.newInstance();
+    expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes();
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
+        Futures.immediateFuture(responseHolder)
+    );
+    replayAll();
+
+    client.stop(TEST_ID, true);
+    verifyAll();
+
+    Request request = captured.getValue();
+    Assert.assertEquals(HttpMethod.POST, request.getMethod());
+    Assert.assertEquals(
+        new URL("http://test-host:1234/druid/worker/v1/chat/test-id/stop?publish=true"),
+        request.getUrl()
+    );
+    Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id"));
+  }
+
+  @Test
+  public void testStopAsync() throws Exception
+  {
+    final int numRequests = TEST_IDS.size();
+    Capture<Request> captured = Capture.newInstance(CaptureType.ALL);
+    expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes();
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
+        Futures.immediateFuture(responseHolder)
+    ).times(numRequests);
+    replayAll();
+
+    List<URL> expectedUrls = new ArrayList<>();
+    List<ListenableFuture<Boolean>> futures = new ArrayList<>();
+    for (String testId : TEST_IDS) {
+      expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "stop")));
+      futures.add(client.stopAsync(testId, false));
+    }
+
+    List<Boolean> responses = Futures.allAsList(futures).get();
+
+    verifyAll();
+    List<Request> requests = captured.getValues();
+
+    Assert.assertEquals(numRequests, requests.size());
+    Assert.assertEquals(numRequests, responses.size());
+    for (int i = 0; i < numRequests; i++) {
+      Assert.assertEquals(HttpMethod.POST, requests.get(i).getMethod());
+      Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl()));
+      Assert.assertTrue(responses.get(i));
+    }
+  }
+
+  @Test
+  public void testResumeAsync() throws Exception
+  {
+    final int numRequests = TEST_IDS.size();
+    Capture<Request> captured = Capture.newInstance(CaptureType.ALL);
+    expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes();
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
+        Futures.immediateFuture(responseHolder)
+    ).times(numRequests);
+    replayAll();
+
+    List<URL> expectedUrls = new ArrayList<>();
+    List<ListenableFuture<Boolean>> futures = new ArrayList<>();
+    for (String testId : TEST_IDS) {
+      expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "resume")));
+      futures.add(client.resumeAsync(testId));
+    }
+
+    List<Boolean> responses = Futures.allAsList(futures).get();
+
+    verifyAll();
+    List<Request> requests = captured.getValues();
+
+    Assert.assertEquals(numRequests, requests.size());
+    Assert.assertEquals(numRequests, responses.size());
+    for (int i = 0; i < numRequests; i++) {
+      Assert.assertEquals(HttpMethod.POST, requests.get(i).getMethod());
+      Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl()));
+      Assert.assertTrue(responses.get(i));
+    }
+  }
+
+  @Test
+  public void testPauseAsync() throws Exception
+  {
+    final int numRequests = TEST_IDS.size();
+    Capture<Request> captured = Capture.newInstance(CaptureType.ALL);
+    expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes();
+    expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes();
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
+        Futures.immediateFuture(responseHolder)
+    ).times(numRequests);
+    replayAll();
+
+    List<URL> expectedUrls = new ArrayList<>();
+    List<ListenableFuture<Map<String, String>>> futures = new ArrayList<>();
+    for (String testId : TEST_IDS) {
+      expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "pause")));
+      futures.add(client.pauseAsync(testId));
+    }
+
+    List<Map<String, String>> responses = Futures.allAsList(futures).get();
+
+    verifyAll();
+    List<Request> requests = captured.getValues();
+
+    Assert.assertEquals(numRequests, requests.size());
+    Assert.assertEquals(numRequests, responses.size());
+    for (int i = 0; i < numRequests; i++) {
+      Assert.assertEquals(HttpMethod.POST, requests.get(i).getMethod());
+      Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl()));
+      Assert.assertEquals(Maps.newLinkedHashMap(ImmutableMap.of("0", "1")), responses.get(i));
+    }
+  }
+
+  @Test
+  public void testGetStatusAsync() throws Exception
+  {
+    final int numRequests = TEST_IDS.size();
+    Capture<Request> captured = Capture.newInstance(CaptureType.ALL);
+    expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes();
+    expect(responseHolder.getContent()).andReturn("\"READING\"").anyTimes();
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
+        Futures.immediateFuture(responseHolder)
+    ).times(numRequests);
+    replayAll();
+
+    List<URL> expectedUrls = new ArrayList<>();
+    List<ListenableFuture<Status>> futures = new ArrayList<>();
+    for (String testId : TEST_IDS) {
+      expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "status")));
+      futures.add(client.getStatusAsync(testId));
+    }
+
+    List<Status> responses = Futures.allAsList(futures).get();
+
+    verifyAll();
+    List<Request> requests = captured.getValues();
+
+    Assert.assertEquals(numRequests, requests.size());
+    Assert.assertEquals(numRequests, responses.size());
+    for (int i = 0; i < numRequests; i++) {
+      Assert.assertEquals(HttpMethod.GET, requests.get(i).getMethod());
+      Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl()));
+      Assert.assertEquals(Status.READING, responses.get(i));
+    }
+  }
+
+  @Test
+  public void testGetStartTimeAsync() throws Exception
+  {
+    final DateTime now = DateTimes.nowUtc();
+    final int numRequests = TEST_IDS.size();
+    Capture<Request> captured = Capture.newInstance(CaptureType.ALL);
+    expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes();
+    expect(responseHolder.getContent()).andReturn(String.valueOf(now.getMillis())).anyTimes();
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
+        Futures.immediateFuture(responseHolder)
+    ).times(numRequests);
+    replayAll();
+
+    List<URL> expectedUrls = new ArrayList<>();
+    List<ListenableFuture<DateTime>> futures = new ArrayList<>();
+    for (String testId : TEST_IDS) {
+      expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "time/start")));
+      futures.add(client.getStartTimeAsync(testId));
+    }
+
+    List<DateTime> responses = Futures.allAsList(futures).get();
+
+    verifyAll();
+    List<Request> requests = captured.getValues();
+
+    Assert.assertEquals(numRequests, requests.size());
+    Assert.assertEquals(numRequests, responses.size());
+    for (int i = 0; i < numRequests; i++) {
+      Assert.assertEquals(HttpMethod.GET, requests.get(i).getMethod());
+      Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl()));
+      Assert.assertEquals(now, responses.get(i));
+    }
+  }
+
+  @Test
+  public void testGetCurrentOffsetsAsync() throws Exception
+  {
+    final int numRequests = TEST_IDS.size();
+    Capture<Request> captured = Capture.newInstance(CaptureType.ALL);
+    expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes();
+    expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes();
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
+        Futures.immediateFuture(responseHolder)
+    ).times(numRequests);
+    replayAll();
+
+    List<URL> expectedUrls = new ArrayList<>();
+    List<ListenableFuture<Map<String, String>>> futures = new ArrayList<>();
+    for (String testId : TEST_IDS) {
+      expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "offsets/current")));
+      futures.add(client.getCurrentOffsetsAsync(testId, false));
+    }
+
+    List<Map<String, String>> responses = Futures.allAsList(futures).get();
+
+    verifyAll();
+    List<Request> requests = captured.getValues();
+
+    Assert.assertEquals(numRequests, requests.size());
+    Assert.assertEquals(numRequests, responses.size());
+    for (int i = 0; i < numRequests; i++) {
+      Assert.assertEquals(HttpMethod.GET, requests.get(i).getMethod());
+      Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl()));
+      Assert.assertEquals(Maps.newLinkedHashMap(ImmutableMap.of("0", "1")), responses.get(i));
+    }
+  }
+
+  @Test
+  public void testGetEndOffsetsAsync() throws Exception
+  {
+    final int numRequests = TEST_IDS.size();
+    Capture<Request> captured = Capture.newInstance(CaptureType.ALL);
+    expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes();
+    expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes();
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
+        Futures.immediateFuture(responseHolder)
+    ).times(numRequests);
+    replayAll();
+
+    List<URL> expectedUrls = new ArrayList<>();
+    List<ListenableFuture<Map<String, String>>> futures = new ArrayList<>();
+    for (String testId : TEST_IDS) {
+      expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "offsets/end")));
+      futures.add(client.getEndOffsetsAsync(testId));
+    }
+
+    List<Map<String, String>> responses = Futures.allAsList(futures).get();
+
+    verifyAll();
+    List<Request> requests = captured.getValues();
+
+    Assert.assertEquals(numRequests, requests.size());
+    Assert.assertEquals(numRequests, responses.size());
+    for (int i = 0; i < numRequests; i++) {
+      Assert.assertEquals(HttpMethod.GET, requests.get(i).getMethod());
+      Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl()));
+      Assert.assertEquals(Maps.newLinkedHashMap(ImmutableMap.of("0", "1")), responses.get(i));
+    }
+  }
+
+  @Test
+  public void testSetEndOffsetsAsync() throws Exception
+  {
+    Map<String, String> endOffsets = ImmutableMap.of("0", "15L", "1", "120L");
+    final int numRequests = TEST_IDS.size();
+    Capture<Request> captured = Capture.newInstance(CaptureType.ALL);
+    expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes();
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
+        Futures.immediateFuture(responseHolder)
+    ).times(numRequests);
+    replayAll();
+
+    List<URL> expectedUrls = new ArrayList<>();
+    List<ListenableFuture<Boolean>> futures = new ArrayList<>();
+    for (String testId : TEST_IDS) {
+      expectedUrls.add(new URL(StringUtils.format(
+          URL_FORMATTER,
+          TEST_HOST,
+          TEST_PORT,
+          testId,
+          StringUtils.format("offsets/end?finish=%s", true)
+      )));
+      futures.add(client.setEndOffsetsAsync(testId, endOffsets, true));
+    }
+
+    List<Boolean> responses = Futures.allAsList(futures).get();
+
+    verifyAll();
+    List<Request> requests = captured.getValues();
+
+    Assert.assertEquals(numRequests, requests.size());
+    Assert.assertEquals(numRequests, responses.size());
+    for (int i = 0; i < numRequests; i++) {
+      Assert.assertEquals(HttpMethod.POST, requests.get(i).getMethod());
+      Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl()));
+      Assert.assertTrue(responses.get(i));
+    }
+  }
+
+  @Test
+  public void testSetEndOffsetsAsyncWithResume() throws Exception
+  {
+    Map<String, String> endOffsets = ImmutableMap.of("0", "15L", "1", "120L");
+    final int numRequests = TEST_IDS.size();
+    Capture<Request> captured = Capture.newInstance(CaptureType.ALL);
+    expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes();
+    expect(httpClient.go(
+        EasyMock.capture(captured),
+        EasyMock.anyObject(FullResponseHandler.class),
+        EasyMock.eq(TEST_HTTP_TIMEOUT)
+    )).andReturn(
+        Futures.immediateFuture(responseHolder)
+    ).times(numRequests);
+    replayAll();
+
+    List<URL> expectedUrls = new ArrayList<>();
+    List<ListenableFuture<Boolean>> futures = new ArrayList<>();
+    for (String testId : TEST_IDS) {
+      expectedUrls.add(
+          new URL(
+              StringUtils.format(
+                  URL_FORMATTER,
+                  TEST_HOST,
+                  TEST_PORT,
+                  testId,
+                  "offsets/end?finish=true"
+              )
+          )
+      );
+      futures.add(client.setEndOffsetsAsync(testId, endOffsets, true));
+    }
+
+    List<Boolean> responses = Futures.allAsList(futures).get();
+
+    verifyAll();
+    List<Request> requests = captured.getValues();
+
+    Assert.assertEquals(numRequests, requests.size());
+    Assert.assertEquals(numRequests, responses.size());
+    for (int i = 0; i < numRequests; i++) {
+      Assert.assertEquals(HttpMethod.POST, requests.get(i).getMethod());
+      Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl()));
+      Assert.assertTrue(responses.get(i));
+    }
+  }
+
+  private class TestableKinesisIndexTaskClient extends KinesisIndexTaskClient
+  {
+    TestableKinesisIndexTaskClient(
+        HttpClient httpClient,
+        ObjectMapper jsonMapper,
+        TaskInfoProvider taskInfoProvider
+    )
+    {
+      this(httpClient, jsonMapper, taskInfoProvider, TEST_NUM_RETRIES);
+    }
+
+    TestableKinesisIndexTaskClient(
+        HttpClient httpClient,
+        ObjectMapper jsonMapper,
+        TaskInfoProvider taskInfoProvider,
+        long numRetries
+    )
+    {
+      super(httpClient, jsonMapper, taskInfoProvider, TEST_DATASOURCE, numThreads, TEST_HTTP_TIMEOUT, numRetries);
+    }
+
+    @Override
+    protected void checkConnection(String host, int port)
+    {
+    }
+  }
+}
diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java
new file mode 100644
index 00000000000..cef97955db2
--- /dev/null
+++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java
@@ -0,0 +1,2968 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.indexing.kinesis;
+
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.InjectableValues;
+import com.fasterxml.jackson.databind.Module;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Predicate;
+import com.google.common.base.Predicates;
+import com.google.common.base.Throwables;
+import com.google.common.collect.FluentIterable;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+import com.google.common.io.Files;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import org.apache.druid.client.cache.CacheConfig;
+import org.apache.druid.client.cache.CachePopulatorStats;
+import org.apache.druid.client.cache.MapCache;
+import org.apache.druid.common.aws.AWSCredentialsConfig;
+import org.apache.druid.data.input.impl.DimensionsSpec;
+import org.apache.druid.data.input.impl.FloatDimensionSchema;
+import org.apache.druid.data.input.impl.JSONParseSpec;
+import org.apache.druid.data.input.impl.LongDimensionSchema;
+import org.apache.druid.data.input.impl.StringDimensionSchema;
+import org.apache.druid.data.input.impl.StringInputRowParser;
+import org.apache.druid.data.input.impl.TimestampSpec;
+import org.apache.druid.discovery.DataNodeService;
+import org.apache.druid.discovery.DruidNodeAnnouncer;
+import org.apache.druid.discovery.LookupNodeService;
+import org.apache.druid.indexer.TaskState;
+import org.apache.druid.indexer.TaskStatus;
+import org.apache.druid.indexing.common.Counters;
+import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData;
+import org.apache.druid.indexing.common.SegmentLoaderFactory;
+import org.apache.druid.indexing.common.TaskLock;
+import org.apache.druid.indexing.common.TaskReport;
+import org.apache.druid.indexing.common.TaskReportFileWriter;
+import org.apache.druid.indexing.common.TaskToolbox;
+import org.apache.druid.indexing.common.TaskToolboxFactory;
+import org.apache.druid.indexing.common.TestUtils;
+import org.apache.druid.indexing.common.actions.LocalTaskActionClientFactory;
+import org.apache.druid.indexing.common.actions.TaskActionClientFactory;
+import org.apache.druid.indexing.common.actions.TaskActionToolbox;
+import org.apache.druid.indexing.common.actions.TaskAuditLogConfig;
+import org.apache.druid.indexing.common.config.TaskConfig;
+import org.apache.druid.indexing.common.config.TaskStorageConfig;
+import org.apache.druid.indexing.common.stats.RowIngestionMeters;
+import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory;
+import org.apache.druid.indexing.common.task.IndexTaskTest;
+import org.apache.druid.indexing.common.task.Task;
+import org.apache.druid.indexing.common.task.TaskResource;
+import org.apache.druid.indexing.overlord.DataSourceMetadata;
+import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
+import org.apache.druid.indexing.overlord.MetadataTaskStorage;
+import org.apache.druid.indexing.overlord.TaskLockbox;
+import org.apache.druid.indexing.overlord.TaskStorage;
+import org.apache.druid.indexing.overlord.supervisor.SupervisorManager;
+import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner;
+import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
+import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord;
+import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor;
+import org.apache.druid.indexing.test.TestDataSegmentAnnouncer;
+import org.apache.druid.indexing.test.TestDataSegmentKiller;
+import org.apache.druid.java.util.common.CompressionUtils;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.common.parsers.JSONPathSpec;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.java.util.emitter.core.NoopEmitter;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.java.util.metrics.MonitorScheduler;
+import org.apache.druid.math.expr.ExprMacroTable;
+import org.apache.druid.metadata.DerbyMetadataStorageActionHandlerFactory;
+import org.apache.druid.metadata.EntryExistsException;
+import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator;
+import org.apache.druid.metadata.TestDerbyConnector;
+import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate;
+import org.apache.druid.query.Druids;
+import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.query.QueryRunner;
+import org.apache.druid.query.QueryRunnerFactoryConglomerate;
+import org.apache.druid.query.QueryToolChest;
+import org.apache.druid.query.Result;
+import org.apache.druid.query.SegmentDescriptor;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
+import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
+import org.apache.druid.query.filter.SelectorDimFilter;
+import org.apache.druid.query.timeseries.TimeseriesQuery;
+import org.apache.druid.query.timeseries.TimeseriesQueryEngine;
+import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest;
+import org.apache.druid.query.timeseries.TimeseriesQueryRunnerFactory;
+import org.apache.druid.query.timeseries.TimeseriesResultValue;
+import org.apache.druid.segment.DimensionHandlerUtils;
+import org.apache.druid.segment.IndexIO;
+import org.apache.druid.segment.QueryableIndex;
+import org.apache.druid.segment.TestHelper;
+import org.apache.druid.segment.column.DictionaryEncodedColumn;
+import org.apache.druid.segment.indexing.DataSchema;
+import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
+import org.apache.druid.segment.loading.DataSegmentPusher;
+import org.apache.druid.segment.loading.LocalDataSegmentPusher;
+import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig;
+import org.apache.druid.segment.loading.SegmentLoaderConfig;
+import org.apache.druid.segment.loading.SegmentLoaderLocalCacheManager;
+import org.apache.druid.segment.loading.StorageLocationConfig;
+import org.apache.druid.segment.realtime.appenderator.AppenderatorImpl;
+import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider;
+import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifier;
+import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory;
+import org.apache.druid.segment.transform.ExpressionTransform;
+import org.apache.druid.segment.transform.TransformSpec;
+import org.apache.druid.server.DruidNode;
+import org.apache.druid.server.coordination.DataSegmentServerAnnouncer;
+import org.apache.druid.server.coordination.ServerType;
+import org.apache.druid.server.security.AuthorizerMapper;
+import org.apache.druid.timeline.DataSegment;
+import org.easymock.EasyMock;
+import org.easymock.EasyMockSupport;
+import org.joda.time.Interval;
+import org.joda.time.Period;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import javax.annotation.Nullable;
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner.Status;
+import static org.easymock.EasyMock.anyLong;
+import static org.easymock.EasyMock.anyObject;
+import static org.easymock.EasyMock.anyString;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.expectLastCall;
+import static org.easymock.EasyMock.reset;
+
+public class KinesisIndexTaskTest extends EasyMockSupport
+{
+  private static final Logger log = new Logger(KinesisIndexTaskTest.class);
+  private static final ObjectMapper objectMapper = TestHelper.makeJsonMapper();
+  private static String stream = "stream";
+  private static String shardId1 = "1";
+  private static String shardId0 = "0";
+  private static KinesisRecordSupplier recordSupplier;
+  private static List<OrderedPartitionableRecord<String, String>> records = ImmutableList.of(
+      new OrderedPartitionableRecord<>(stream, "1", "0", JB("2008", "a", "y", "10", "20.0", "1.0")),
+      new OrderedPartitionableRecord<>(stream, "1", "1", JB("2009", "b", "y", "10", "20.0", "1.0")),
+      new OrderedPartitionableRecord<>(stream, "1", "2", JB("2010", "c", "y", "10", "20.0", "1.0")),
+      new OrderedPartitionableRecord<>(stream, "1", "3", JB("2011", "d", "y", "10", "20.0", "1.0")),
+      new OrderedPartitionableRecord<>(stream, "1", "4", JB("2011", "e", "y", "10", "20.0", "1.0")),
+      new OrderedPartitionableRecord<>(
+          stream,
+          "1",
+          "5",
+          JB("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")
+      ),
+      new OrderedPartitionableRecord<>(stream, "1", "6", Collections.singletonList(StringUtils.toUtf8("unparseable"))),
+      new OrderedPartitionableRecord<>(stream, "1", "7", Collections.singletonList(StringUtils.toUtf8("unparseable2"))),
+      new OrderedPartitionableRecord<>(stream, "1", "8", Collections.singletonList(StringUtils.toUtf8("{}"))),
+      new OrderedPartitionableRecord<>(stream, "1", "9", JB("2013", "f", "y", "10", "20.0", "1.0")),
+      new OrderedPartitionableRecord<>(stream, "1", "10", JB("2049", "f", "y", "notanumber", "20.0", "1.0")),
+      new OrderedPartitionableRecord<>(stream, "1", "11", JB("2049", "f", "y", "10", "notanumber", "1.0")),
+      new OrderedPartitionableRecord<>(stream, "1", "12", JB("2049", "f", "y", "10", "20.0", "notanumber")),
+      new OrderedPartitionableRecord<>(stream, "0", "0", JB("2012", "g", "y", "10", "20.0", "1.0")),
+      new OrderedPartitionableRecord<>(stream, "0", "1", JB("2011", "h", "y", "10", "20.0", "1.0"))
+  );
+
+  private static ServiceEmitter emitter;
+  private static ListeningExecutorService taskExec;
+
+  private final List<Task> runningTasks = new ArrayList<>();
+
+  private long handoffConditionTimeout = 0;
+  private boolean reportParseExceptions = false;
+  private boolean logParseExceptions = true;
+  private Integer maxParseExceptions = null;
+  private Integer maxSavedParseExceptions = null;
+  private boolean resetOffsetAutomatically = false;
+  private boolean doHandoff = true;
+  private int maxRowsInMemory = 1000;
+  private Integer maxRowsPerSegment = null;
+  private Long maxTotalRows = null;
+  private Period intermediateHandoffPeriod = null;
+  private int maxRecordsPerPoll;
+  private boolean skipAvailabilityCheck = false;
+
+  private TaskToolboxFactory toolboxFactory;
+  private IndexerMetadataStorageCoordinator metadataStorageCoordinator;
+  private TaskStorage taskStorage;
+  private TaskLockbox taskLockbox;
+  private File directory;
+  private final Set<Integer> checkpointRequestsHash = new HashSet<>();
+  private File reportsFile;
+  private RowIngestionMetersFactory rowIngestionMetersFactory;
+
+  private static final DataSchema DATA_SCHEMA = new DataSchema(
+      "test_ds",
+      objectMapper.convertValue(
+          new StringInputRowParser(
+              new JSONParseSpec(
+                  new TimestampSpec("timestamp", "iso", null),
+                  new DimensionsSpec(
+                      Arrays.asList(
+                          new StringDimensionSchema("dim1"),
+                          new StringDimensionSchema("dim1t"),
+                          new StringDimensionSchema("dim2"),
+                          new LongDimensionSchema("dimLong"),
+                          new FloatDimensionSchema("dimFloat")
+                      ),
+                      null,
+                      null
+                  ),
+                  new JSONPathSpec(true, ImmutableList.of()),
+                  ImmutableMap.of()
+              ),
+              StandardCharsets.UTF_8.name()
+          ),
+          Map.class
+      ),
+      new AggregatorFactory[]{
+          new DoubleSumAggregatorFactory("met1sum", "met1"),
+          new CountAggregatorFactory("rows")
+      },
+      new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null),
+      null,
+      objectMapper
+  );
+
+  @Rule
+  public final TemporaryFolder tempFolder = new TemporaryFolder();
+
+  @Rule
+  public final TestDerbyConnector.DerbyConnectorRule derby = new TestDerbyConnector.DerbyConnectorRule();
+
+  @BeforeClass
+  public static void setupClass()
+  {
+    emitter = new ServiceEmitter(
+        "service",
+        "host",
+        new NoopEmitter()
+    );
+    emitter.start();
+    EmittingLogger.registerEmitter(emitter);
+    taskExec = MoreExecutors.listeningDecorator(
+        Executors.newCachedThreadPool(
+            Execs.makeThreadFactory("kinesis-task-test-%d")
+        )
+    );
+  }
+
+  @Before
+  public void setupTest() throws IOException, InterruptedException
+  {
+    handoffConditionTimeout = 0;
+    reportParseExceptions = false;
+    logParseExceptions = true;
+    maxParseExceptions = null;
+    maxSavedParseExceptions = null;
+    skipAvailabilityCheck = false;
+    doHandoff = true;
+    reportsFile = File.createTempFile("KinesisIndexTaskTestReports-" + System.currentTimeMillis(), "json");
+    maxRecordsPerPoll = 1;
+
+    recordSupplier = mock(KinesisRecordSupplier.class);
+
+    // sleep required because of kinesalite
+    Thread.sleep(500);
+    makeToolboxFactory();
+  }
+
+  @After
+  public void tearDownTest()
+  {
+    synchronized (runningTasks) {
+      for (Task task : runningTasks) {
+        task.stopGracefully();
+      }
+
+      runningTasks.clear();
+    }
+    reportsFile.delete();
+    destroyToolboxFactory();
+  }
+
+  @AfterClass
+  public static void tearDownClass() throws Exception
+  {
+    taskExec.shutdown();
+    taskExec.awaitTermination(9999, TimeUnit.DAYS);
+    emitter.close();
+  }
+
+  @Test(timeout = 120_000L)
+  public void testRunAfterDataInserted() throws Exception
+  {
+    recordSupplier.assign(anyObject());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes();
+
+    recordSupplier.seek(anyObject(), anyString());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.poll(anyLong())).andReturn(records.subList(2, 5)).once();
+
+    recordSupplier.close();
+    expectLastCall().once();
+
+    replayAll();
+
+    final KinesisIndexTask task = createTask(
+        null,
+        new KinesisIndexTaskIOConfig(
+            null,
+            "sequence0",
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "2"
+            )),
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "4"
+            )),
+            true,
+            null,
+            null,
+            "awsEndpoint",
+            null,
+            null,
+            null,
+            null,
+            null,
+            false
+        )
+
+    );
+
+    final ListenableFuture<TaskStatus> future = runTask(task);
+
+    // Wait for task to exit
+    Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode());
+
+    verifyAll();
+
+    // Check metrics
+    Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed());
+    Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable());
+    Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
+
+    // Check published metadata
+    SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
+    SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
+    Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
+    Assert.assertEquals(
+        new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(
+            stream,
+            ImmutableMap.of(
+                shardId1,
+                "4"
+            )
+        )),
+        metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
+    );
+
+    // Check segments in deep storage
+    Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
+    Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2));
+  }
+
+  @Test(timeout = 120_000L)
+  public void testRunBeforeDataInserted() throws Exception
+  {
+
+    recordSupplier.assign(anyObject());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes();
+
+    recordSupplier.seek(anyObject(), anyString());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.poll(anyLong())).andReturn(Collections.emptyList())
+                                          .times(5)
+                                          .andReturn(records.subList(13, 15))
+                                          .once();
+
+    recordSupplier.close();
+    expectLastCall().once();
+
+    replayAll();
+
+    final KinesisIndexTask task = createTask(
+        null,
+        new KinesisIndexTaskIOConfig(
+            null,
+            "sequence0",
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId0,
+                "0"
+            )),
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId0,
+                "1"
+            )),
+            true,
+            null,
+            null,
+            "awsEndpoint",
+            null,
+            null,
+            null,
+            null,
+            null,
+            false
+        )
+
+    );
+
+    final ListenableFuture<TaskStatus> future = runTask(task);
+
+    // Wait for task to exit
+    Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode());
+
+    verifyAll();
+    // Check metrics
+    Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getProcessed());
+    Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable());
+    Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
+
+    // Check published metadata
+    SegmentDescriptor desc1 = SD(task, "2011/P1D", 0);
+    SegmentDescriptor desc2 = SD(task, "2012/P1D", 0);
+    Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
+    Assert.assertEquals(
+        new KinesisDataSourceMetadata(
+            new SeekableStreamPartitions<>(
+                stream,
+                ImmutableMap.of(
+                    shardId0,
+                    "1"
+                )
+            )),
+        metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
+    );
+
+    // Check segments in deep storage
+    Assert.assertEquals(ImmutableList.of("h"), readSegmentColumn("dim1", desc1));
+    Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc2));
+  }
+
+
+  @Test(timeout = 120_000L)
+  public void testIncrementalHandOff() throws Exception
+  {
+    final String baseSequenceName = "sequence0";
+    // as soon as any segment has more than one record, incremental publishing should happen
+    maxRowsPerSegment = 2;
+    maxRecordsPerPoll = 1;
+
+    recordSupplier.assign(anyObject());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes();
+
+    recordSupplier.seek(anyObject(), anyString());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.poll(anyLong())).andReturn(records.subList(0, 5))
+                                          .once()
+                                          .andReturn(records.subList(4, records.size()))
+                                          .once();
+
+    recordSupplier.close();
+    expectLastCall().once();
+
+    replayAll();
+
+    final SeekableStreamPartitions<String, String> startPartitions = new SeekableStreamPartitions<>(
+        stream,
+        ImmutableMap.of(
+            shardId1,
+            "0",
+            shardId0,
+            "0"
+        )
+    );
+
+    final SeekableStreamPartitions<String, String> checkpoint1 = new SeekableStreamPartitions<>(
+        stream,
+        ImmutableMap.of(
+            shardId1,
+            "4",
+            shardId0,
+            "0"
+        )
+    );
+
+    final SeekableStreamPartitions<String, String> endPartitions = new SeekableStreamPartitions<>(
+        stream,
+        ImmutableMap.of(
+            shardId1,
+            "9",
+            shardId0,
+            "1"
+        )
+    );
+    final KinesisIndexTask task = createTask(
+        null,
+        new KinesisIndexTaskIOConfig(
+            null,
+            baseSequenceName,
+            startPartitions,
+            endPartitions,
+            true,
+            null,
+            null,
+            "awsEndpoint",
+            null,
+            null,
+            null,
+            null,
+            null,
+            false
+        )
+    );
+    final ListenableFuture<TaskStatus> future = runTask(task);
+    while (task.getRunner().getStatus() != SeekableStreamIndexTaskRunner.Status.PAUSED) {
+      Thread.sleep(10);
+    }
+    final Map<String, String> currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets());
+    Assert.assertTrue(checkpoint1.getPartitionSequenceNumberMap().equals(currentOffsets));
+    task.getRunner().setEndOffsets(currentOffsets, false);
+
+    Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode());
+
+    verifyAll();
+
+    Assert.assertEquals(1, checkpointRequestsHash.size());
+    Assert.assertTrue(
+        checkpointRequestsHash.contains(
+            Objects.hash(
+                DATA_SCHEMA.getDataSource(),
+                0,
+                new KinesisDataSourceMetadata(startPartitions),
+                new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, currentOffsets))
+            )
+        )
+    );
+
+    // Check metrics
+    Assert.assertEquals(8, task.getRunner().getRowIngestionMeters().getProcessed());
+    Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getUnparseable());
+    Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
+
+    // Check published metadata
+    SegmentDescriptor desc1 = SD(task, "2008/P1D", 0);
+    SegmentDescriptor desc2 = SD(task, "2009/P1D", 0);
+    SegmentDescriptor desc3 = SD(task, "2010/P1D", 0);
+    SegmentDescriptor desc4 = SD(task, "2011/P1D", 0);
+    SegmentDescriptor desc5 = SD(task, "2011/P1D", 1);
+    SegmentDescriptor desc6 = SD(task, "2012/P1D", 0);
+    SegmentDescriptor desc7 = SD(task, "2013/P1D", 0);
+    Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors());
+    Assert.assertEquals(
+        new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+            shardId1,
+            "9",
+            shardId0,
+            "1"
+        ))),
+        metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
+    );
+
+    // Check segments in deep storage
+    Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1));
+    Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2));
+    Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc3));
+    Assert.assertTrue((ImmutableList.of("d", "e").equals(readSegmentColumn("dim1", desc4))
+                       && ImmutableList.of("h").equals(readSegmentColumn("dim1", desc5))) ||
+                      (ImmutableList.of("d", "h").equals(readSegmentColumn("dim1", desc4))
+                       && ImmutableList.of("e").equals(readSegmentColumn("dim1", desc5))));
+    Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc6));
+    Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc7));
+  }
+
+
+  @Test(timeout = 120_000L)
+  public void testIncrementalHandOffMaxTotalRows() throws Exception
+  {
+    final String baseSequenceName = "sequence0";
+    // incremental publish should happen every 3 records
+    maxRowsPerSegment = Integer.MAX_VALUE;
+    maxTotalRows = 3L;
+
+    recordSupplier.assign(anyObject());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes();
+
+    recordSupplier.seek(anyObject(), anyString());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.poll(anyLong())).andReturn(records.subList(0, 3))
+                                          .once()
+                                          .andReturn(records.subList(2, 10))
+                                          .once()
+                                          .andReturn(records.subList(9, 11));
+
+    recordSupplier.close();
+    expectLastCall().once();
+
+    replayAll();
+
+    // Insert data
+    final SeekableStreamPartitions<String, String> startPartitions = new SeekableStreamPartitions<>(
+        stream,
+        ImmutableMap.of(
+            shardId1,
+            "0"
+        )
+    );
+    // Checkpointing will happen at either checkpoint1 or checkpoint2 depending on ordering
+    // of events fetched across two partitions from Kafka
+    final SeekableStreamPartitions<String, String> checkpoint1 = new SeekableStreamPartitions<>(
+        stream,
+        ImmutableMap.of(
+            shardId1,
+            "2"
+        )
+    );
+    final SeekableStreamPartitions<String, String> checkpoint2 = new SeekableStreamPartitions<>(
+        stream,
+        ImmutableMap.of(
+            shardId1,
+            "9"
+        )
+    );
+    final SeekableStreamPartitions<String, String> endPartitions = new SeekableStreamPartitions<>(
+        stream,
+        ImmutableMap.of(
+            shardId1,
+            "10"
+        )
+    );
+
+    final KinesisIndexTask task = createTask(
+        null,
+        new KinesisIndexTaskIOConfig(
+            null,
+            baseSequenceName,
+            startPartitions,
+            endPartitions,
+            true,
+            null,
+            null,
+            "awsEndpoint",
+            null,
+            null,
+            null,
+            null,
+            null,
+            false
+        )
+    );
+
+    final ListenableFuture<TaskStatus> future = runTask(task);
+    while (task.getRunner().getStatus() != Status.PAUSED) {
+      Thread.sleep(10);
+    }
+    final Map<String, String> currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets());
+
+    Assert.assertEquals(checkpoint1.getPartitionSequenceNumberMap(), currentOffsets);
+    task.getRunner().setEndOffsets(currentOffsets, false);
+
+    while (task.getRunner().getStatus() != Status.PAUSED) {
+      Thread.sleep(10);
+    }
+
+    final Map<String, String> nextOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets());
+
+    Assert.assertEquals(checkpoint2.getPartitionSequenceNumberMap(), nextOffsets);
+
+    task.getRunner().setEndOffsets(nextOffsets, false);
+
+    Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode());
+
+    verifyAll();
+
+    Assert.assertEquals(2, checkpointRequestsHash.size());
+    Assert.assertTrue(
+        checkpointRequestsHash.contains(
+            Objects.hash(
+                DATA_SCHEMA.getDataSource(),
+                0,
+                new KinesisDataSourceMetadata(startPartitions),
+                new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, currentOffsets))
+            )
+        )
+    );
+    Assert.assertTrue(
+        checkpointRequestsHash.contains(
+            Objects.hash(
+                DATA_SCHEMA.getDataSource(),
+                0,
+                new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, currentOffsets)),
+                new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, nextOffsets))
+            )
+        )
+    );
+
+    // Check metrics
+    Assert.assertEquals(6, task.getRunner().getRowIngestionMeters().getProcessed());
+    Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getUnparseable());
+    Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
+
+    // Check published metadata
+    SegmentDescriptor desc1 = SD(task, "2008/P1D", 0);
+    SegmentDescriptor desc2 = SD(task, "2009/P1D", 0);
+    SegmentDescriptor desc3 = SD(task, "2010/P1D", 0);
+    SegmentDescriptor desc4 = SD(task, "2011/P1D", 0);
+    SegmentDescriptor desc5 = SD(task, "2049/P1D", 0);
+    SegmentDescriptor desc7 = SD(task, "2013/P1D", 0);
+    Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc7), publishedDescriptors());
+    Assert.assertEquals(
+        new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+            shardId1,
+            "10"
+        ))),
+        metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
+    );
+
+    // Check segments in deep storage
+    Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1));
+    Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2));
+    Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc3));
+    Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc4));
+    Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc5));
+    Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc7));
+  }
+
+
+  @Test(timeout = 120_000L)
+  public void testRunWithMinimumMessageTime() throws Exception
+  {
+    recordSupplier.assign(anyObject());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes();
+
+    recordSupplier.seek(anyObject(), anyString());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.poll(anyLong())).andReturn(records.subList(0, 13)).once();
+
+    recordSupplier.close();
+    expectLastCall().once();
+
+    replayAll();
+
+    final KinesisIndexTask task = createTask(
+        null,
+        new KinesisIndexTaskIOConfig(
+            null,
+            "sequence0",
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "0"
+            )),
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "4"
+            )),
+            true,
+            DateTimes.of("2010"),
+            null,
+            "awsEndpoint",
+            null,
+            null,
+            null,
+            null,
+            null,
+            false
+        )
+    );
+
+    final ListenableFuture<TaskStatus> future = runTask(task);
+
+    // Wait for the task to start reading
+    while (task.getRunner().getStatus() != Status.READING) {
+      Thread.sleep(10);
+    }
+
+    // Wait for task to exit
+    Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode());
+
+    verifyAll();
+
+    // Check metrics
+    Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed());
+    Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable());
+    Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway());
+
+    // Check published metadata
+    SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
+    SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
+    Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
+    Assert.assertEquals(
+        new KinesisDataSourceMetadata(
+            new SeekableStreamPartitions<>(
+                stream,
+                ImmutableMap.of(
+                    shardId1,
+                    "4"
+                )
+            )),
+        metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
+    );
+
+    // Check segments in deep storage
+    Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
+    Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2));
+  }
+
+
+  @Test(timeout = 120_000L)
+  public void testRunWithMaximumMessageTime() throws Exception
+  {
+    recordSupplier.assign(anyObject());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes();
+
+    recordSupplier.seek(anyObject(), anyString());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.poll(anyLong())).andReturn(records.subList(0, 13)).once();
+
+    recordSupplier.close();
+    expectLastCall().once();
+
+    replayAll();
+
+    final KinesisIndexTask task = createTask(
+        null,
+        new KinesisIndexTaskIOConfig(
+            null,
+            "sequence0",
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "0"
+            )),
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "4"
+            )),
+            true,
+            null,
+            DateTimes.of("2010"),
+            "awsEndpoint",
+            null,
+            null,
+            null,
+            null,
+            null,
+            false
+        )
+    );
+
+    final ListenableFuture<TaskStatus> future = runTask(task);
+
+    // Wait for the task to start reading
+    while (task.getRunner().getStatus() != Status.READING) {
+      Thread.sleep(10);
+    }
+
+    // Wait for task to exit
+    Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode());
+
+    verifyAll();
+
+    // Check metrics
+    Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed());
+    Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable());
+    Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway());
+
+    // Check published metadata
+    SegmentDescriptor desc1 = SD(task, "2008/P1D", 0);
+    SegmentDescriptor desc2 = SD(task, "2009/P1D", 0);
+    SegmentDescriptor desc3 = SD(task, "2010/P1D", 0);
+    Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors());
+    Assert.assertEquals(
+        new KinesisDataSourceMetadata(
+            new SeekableStreamPartitions<>(
+                stream,
+                ImmutableMap.of(
+                    shardId1,
+                    "4"
+                )
+            )),
+        metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
+    );
+
+    // Check segments in deep storage
+    Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1));
+    Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2));
+    Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc3));
+  }
+
+
+  @Test(timeout = 120_000L)
+  public void testRunWithTransformSpec() throws Exception
+  {
+    recordSupplier.assign(anyObject());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes();
+
+    recordSupplier.seek(anyObject(), anyString());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.poll(anyLong())).andReturn(records.subList(0, 13)).once();
+
+    recordSupplier.close();
+    expectLastCall().once();
+
+    replayAll();
+
+    final KinesisIndexTask task = createTask(
+        null,
+        DATA_SCHEMA.withTransformSpec(
+            new TransformSpec(
+                new SelectorDimFilter("dim1", "b", null),
+                ImmutableList.of(
+                    new ExpressionTransform("dim1t", "concat(dim1,dim1)", ExprMacroTable.nil())
+                )
+            )
+        ),
+        new KinesisIndexTaskIOConfig(
+            null,
+            "sequence0",
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "0"
+            )),
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "4"
+            )),
+            true,
+            null,
+            null,
+            "awsEndpoint",
+            null,
+            null,
+            null,
+            null,
+            null,
+            false
+        )
+    );
+
+    final ListenableFuture<TaskStatus> future = runTask(task);
+
+    // Wait for the task to start reading
+    while (task.getRunner().getStatus() != Status.READING) {
+      Thread.sleep(10);
+    }
+
+    // Wait for task to exit
+    Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode());
+
+    verifyAll();
+
+    // Check metrics
+    Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getProcessed());
+    Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable());
+    Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getThrownAway());
+
+    // Check published metadata
+    SegmentDescriptor desc1 = SD(task, "2009/P1D", 0);
+    Assert.assertEquals(ImmutableSet.of(desc1), publishedDescriptors());
+    Assert.assertEquals(
+        new KinesisDataSourceMetadata(
+            new SeekableStreamPartitions<>(
+                stream,
+                ImmutableMap.of(
+                    shardId1,
+                    "4"
+                )
+            )),
+        metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
+    );
+
+    // Check segments in deep storage
+    Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc1));
+    Assert.assertEquals(ImmutableList.of("bb"), readSegmentColumn("dim1t", desc1));
+  }
+
+
+  @Test(timeout = 120_000L)
+  public void testRunOnNothing() throws Exception
+  {
+    recordSupplier.assign(anyObject());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes();
+
+    recordSupplier.seek(anyObject(), anyString());
+    expectLastCall().anyTimes();
+
+    recordSupplier.close();
+    expectLastCall().once();
+
+    replayAll();
+
+    final KinesisIndexTask task = createTask(
+        null,
+        new KinesisIndexTaskIOConfig(
+            null,
+            "sequence0",
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "2"
+            )),
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "2"
+            )),
+            true,
+            null,
+            null,
+            "awsEndpoint",
+            null,
+            null,
+            null,
+            null,
+            null,
+            false
+        )
+    );
+
+    final ListenableFuture<TaskStatus> future = runTask(task);
+
+    // Wait for task to exit
+    Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode());
+
+    verifyAll();
+
+    // Check metrics
+    Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getProcessed());
+    Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable());
+    Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
+
+    // Check published metadata
+    Assert.assertEquals(ImmutableSet.of(), publishedDescriptors());
+  }
+
+
+  @Test(timeout = 60_000L)
+  public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception
+  {
+    handoffConditionTimeout = 5_000;
+
+    recordSupplier.assign(anyObject());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes();
+
+    recordSupplier.seek(anyObject(), anyString());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.poll(anyLong())).andReturn(records.subList(2, 13)).once();
+
+    recordSupplier.close();
+    expectLastCall().once();
+
+    replayAll();
+
+    final KinesisIndexTask task = createTask(
+        null,
+        new KinesisIndexTaskIOConfig(
+            null,
+            "sequence0",
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "2"
+            )),
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "4"
+            )),
+            true,
+            null,
+            null,
+            "awsEndpoint",
+            null,
+            null,
+            null,
+            null,
+            null,
+            false
+        )
+    );
+
+    final ListenableFuture<TaskStatus> future = runTask(task);
+
+    // Wait for task to exit
+    Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode());
+
+    verifyAll();
+
+    // Check metrics
+    Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed());
+    Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable());
+    Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
+
+    // Check published metadata
+    SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
+    SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
+    Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
+    Assert.assertEquals(
+        new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+            shardId1,
+            "4"
+        ))),
+        metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
+    );
+
+    // Check segments in deep storage
+    Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
+    Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2));
+  }
+
+
+  @Test(timeout = 60_000L)
+  public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exception
+  {
+    doHandoff = false;
+    handoffConditionTimeout = 100;
+
+    recordSupplier.assign(anyObject());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes();
+
+    recordSupplier.seek(anyObject(), anyString());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.poll(anyLong())).andReturn(records.subList(2, 13)).once();
+
+    recordSupplier.close();
+    expectLastCall().once();
+
+    replayAll();
+
+    final KinesisIndexTask task = createTask(
+        null,
+        new KinesisIndexTaskIOConfig(
+            null,
+            "sequence0",
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "2"
+            )),
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "4"
+            )),
+            true,
+            null,
+            null,
+            "awsEndpoint",
+            null,
+            null,
+            null,
+            null,
+            null,
+            false
+        )
+    );
+
+    final ListenableFuture<TaskStatus> future = runTask(task);
+
+    // Wait for task to exit
+    Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode());
+
+    verifyAll();
+
+    // Check metrics
+    Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed());
+    Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable());
+    Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
+
+    // Check published metadata
+    SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
+    SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
+    Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
+    Assert.assertEquals(
+        new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+            shardId1,
+            "4"
+        ))),
+        metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
+    );
+
+    // Check segments in deep storage
+    Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
+    Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2));
+  }
+
+
+  @Test(timeout = 120_000L)
+  public void testReportParseExceptions() throws Exception
+  {
+    reportParseExceptions = true;
+
+    // these will be ignored because reportParseExceptions is true
+    maxParseExceptions = 1000;
+    maxSavedParseExceptions = 2;
+
+    recordSupplier.assign(anyObject());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes();
+
+    recordSupplier.seek(anyObject(), anyString());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.poll(anyLong())).andReturn(records.subList(2, 13)).once();
+
+    replayAll();
+
+    final KinesisIndexTask task = createTask(
+        null,
+        new KinesisIndexTaskIOConfig(
+            null,
+            "sequence0",
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "2"
+            )),
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "5"
+            )),
+            true,
+            null,
+            null,
+            "awsEndpoint",
+            null,
+            null,
+            null,
+            null,
+            null,
+            false
+        )
+    );
+
+    final ListenableFuture<TaskStatus> future = runTask(task);
+
+    // Wait for task to exit
+    Assert.assertEquals(TaskState.FAILED, future.get().getStatusCode());
+
+    verifyAll();
+
+    // Check metrics
+    Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed());
+    Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getUnparseable());
+    Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
+
+    // Check published metadata
+    Assert.assertEquals(ImmutableSet.of(), publishedDescriptors());
+    Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()));
+  }
+
+
+  @Test(timeout = 120_000L)
+  public void testMultipleParseExceptionsSuccess() throws Exception
+  {
+    reportParseExceptions = false;
+    maxParseExceptions = 7;
+    maxSavedParseExceptions = 7;
+
+    recordSupplier.assign(anyObject());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes();
+
+    recordSupplier.seek(anyObject(), anyString());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.poll(anyLong())).andReturn(records.subList(2, 13)).once();
+
+    recordSupplier.close();
+    expectLastCall().once();
+
+    replayAll();
+
+    final KinesisIndexTask task = createTask(
+        null,
+        new KinesisIndexTaskIOConfig(
+            null,
+            "sequence0",
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "2"
+            )),
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "12"
+            )),
+            true,
+            null,
+            null,
+            "awsEndpoint",
+            null,
+            null,
+            null,
+            null,
+            null,
+            false
+        )
+    );
+
+    final ListenableFuture<TaskStatus> future = runTask(task);
+
+    TaskStatus status = future.get();
+
+    // Wait for task to exit
+    Assert.assertEquals(TaskState.SUCCESS, status.getStatusCode());
+
+    verifyAll();
+
+    Assert.assertNull(status.getErrorMsg());
+
+    // Check metrics
+    Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getProcessed());
+    Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessedWithError());
+    Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getUnparseable());
+
+    // Check published metadata
+    SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
+    SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
+    SegmentDescriptor desc3 = SD(task, "2013/P1D", 0);
+    SegmentDescriptor desc4 = SD(task, "2049/P1D", 0);
+    Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors());
+    Assert.assertEquals(
+        new KinesisDataSourceMetadata(
+            new SeekableStreamPartitions<>(
+                stream,
+                ImmutableMap.of(
+                    shardId1,
+                    "12"
+                )
+            )),
+        metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
+    );
+
+    IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData();
+
+    Map<String, Object> expectedMetrics = ImmutableMap.of(
+        RowIngestionMeters.BUILD_SEGMENTS,
+        ImmutableMap.of(
+            RowIngestionMeters.PROCESSED, 4,
+            RowIngestionMeters.PROCESSED_WITH_ERROR, 3,
+            RowIngestionMeters.UNPARSEABLE, 4,
+            RowIngestionMeters.THROWN_AWAY, 0
+        )
+    );
+    Assert.assertEquals(expectedMetrics, reportData.getRowStats());
+
+    Map<String, Object> unparseableEvents = ImmutableMap.of(
+        RowIngestionMeters.BUILD_SEGMENTS,
+        Arrays.asList(
+            "Found unparseable columns in row: [MapBasedInputRow{timestamp=2049-01-01T00:00:00.000Z, event={timestamp=2049, dim1=f, dim2=y, dimLong=10, dimFloat=20.0, met1=notanumber}, dimensions=[dim1, dim1t, dim2, dimLong, dimFloat]}], exceptions: [Unable to parse value[notanumber] for field[met1],]",
+            "Found unparseable columns in row: [MapBasedInputRow{timestamp=2049-01-01T00:00:00.000Z, event={timestamp=2049, dim1=f, dim2=y, dimLong=10, dimFloat=notanumber, met1=1.0}, dimensions=[dim1, dim1t, dim2, dimLong, dimFloat]}], exceptions: [could not convert value [notanumber] to float,]",
+            "Found unparseable columns in row: [MapBasedInputRow{timestamp=2049-01-01T00:00:00.000Z, event={timestamp=2049, dim1=f, dim2=y, dimLong=notanumber, dimFloat=20.0, met1=1.0}, dimensions=[dim1, dim1t, dim2, dimLong, dimFloat]}], exceptions: [could not convert value [notanumber] to long,]",
+            "Unparseable timestamp found! Event: {}",
+            "Unable to parse row [unparseable2]",
+            "Unable to parse row [unparseable]",
+            "Encountered row with timestamp that cannot be represented as a long: [MapBasedInputRow{timestamp=246140482-04-24T15:36:27.903Z, event={timestamp=246140482-04-24T15:36:27.903Z, dim1=x, dim2=z, dimLong=10, dimFloat=20.0, met1=1.0}, dimensions=[dim1, dim1t, dim2, dimLong, dimFloat]}]"
+        )
+    );
+
+    Assert.assertEquals(unparseableEvents, reportData.getUnparseableEvents());
+  }
+
+
+  @Test(timeout = 120_000L)
+  public void testMultipleParseExceptionsFailure() throws Exception
+  {
+    reportParseExceptions = false;
+    maxParseExceptions = 2;
+    maxSavedParseExceptions = 2;
+
+    recordSupplier.assign(anyObject());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes();
+
+    recordSupplier.seek(anyObject(), anyString());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.poll(anyLong())).andReturn(records.subList(2, 13)).once();
+
+
+    replayAll();
+
+    final KinesisIndexTask task = createTask(
+        null,
+        new KinesisIndexTaskIOConfig(
+            null,
+            "sequence0",
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "2"
+            )),
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "9"
+            )),
+            true,
+            null,
+            null,
+            "awsEndpoint",
+            null,
+            null,
+            null,
+            null,
+            null,
+            false
+        )
+    );
+
+    final ListenableFuture<TaskStatus> future = runTask(task);
+
+    TaskStatus status = future.get();
+
+    // Wait for task to exit
+    Assert.assertEquals(TaskState.FAILED, status.getStatusCode());
+    verifyAll();
+    IndexTaskTest.checkTaskStatusErrorMsgForParseExceptionsExceeded(status);
+
+    // Check metrics
+    Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed());
+    Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getProcessedWithError());
+    Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getUnparseable());
+    Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
+
+    // Check published metadata
+    Assert.assertEquals(ImmutableSet.of(), publishedDescriptors());
+    Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()));
+
+    IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData();
+
+    Map<String, Object> expectedMetrics = ImmutableMap.of(
+        RowIngestionMeters.BUILD_SEGMENTS,
+        ImmutableMap.of(
+            RowIngestionMeters.PROCESSED, 3,
+            RowIngestionMeters.PROCESSED_WITH_ERROR, 0,
+            RowIngestionMeters.UNPARSEABLE, 3,
+            RowIngestionMeters.THROWN_AWAY, 0
+        )
+    );
+    Assert.assertEquals(expectedMetrics, reportData.getRowStats());
+
+    Map<String, Object> unparseableEvents = ImmutableMap.of(
+        RowIngestionMeters.BUILD_SEGMENTS,
+        Arrays.asList(
+            "Unable to parse row [unparseable2]",
+            "Unable to parse row [unparseable]"
+        )
+    );
+
+    Assert.assertEquals(unparseableEvents, reportData.getUnparseableEvents());
+  }
+
+
+  @Test(timeout = 120_000L)
+  public void testRunReplicas() throws Exception
+  {
+    // Insert data
+    recordSupplier.assign(anyObject());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes();
+
+    recordSupplier.seek(anyObject(), anyString());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.poll(anyLong())).andReturn(records.subList(2, 13)).times(2);
+
+    recordSupplier.close();
+    expectLastCall().times(2);
+
+    replayAll();
+
+    final KinesisIndexTask task1 = createTask(
+        null,
+        new KinesisIndexTaskIOConfig(
+            null,
+            "sequence0",
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "2"
+            )),
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "4"
+            )),
+            true,
+            null,
+            null,
+            "awsEndpoint",
+            null,
+            null,
+            null,
+            null,
+            null,
+            false
+        )
+    );
+    final KinesisIndexTask task2 = createTask(
+        null,
+        new KinesisIndexTaskIOConfig(
+            null,
+            "sequence0",
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "2"
+            )),
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "4"
+            )),
+            true,
+            null,
+            null,
+            "awsEndpoint",
+            null,
+            null,
+            null,
+            null,
+            null,
+            false
+        )
+    );
+
+    final ListenableFuture<TaskStatus> future1 = runTask(task1);
+    final ListenableFuture<TaskStatus> future2 = runTask(task2);
+
+    // Wait for tasks to exit
+    Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode());
+    Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode());
+
+    verifyAll();
+
+    // Check metrics
+    Assert.assertEquals(3, task1.getRunner().getRowIngestionMeters().getProcessed());
+    Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable());
+    Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway());
+    Assert.assertEquals(3, task2.getRunner().getRowIngestionMeters().getProcessed());
+    Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getUnparseable());
+    Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
+
+    // Check published segments & metadata
+    SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0);
+    SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0);
+    Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
+    Assert.assertEquals(
+        new KinesisDataSourceMetadata(
+            new SeekableStreamPartitions<>(
+                stream,
+                ImmutableMap.of(
+                    shardId1,
+                    "4"
+                )
+            )),
+        metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
+    );
+
+    // Check segments in deep storage
+    Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
+    Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2));
+  }
+
+
+  @Test(timeout = 120_000L)
+  public void testRunConflicting() throws Exception
+  {
+    recordSupplier.assign(anyObject());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes();
+
+    recordSupplier.seek(anyObject(), anyString());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.poll(anyLong())).andReturn(records.subList(2, 13))
+                                          .once()
+                                          .andReturn(records.subList(3, 13))
+                                          .once();
+
+    recordSupplier.close();
+    expectLastCall().once();
+
+    replayAll();
+
+    final KinesisIndexTask task1 = createTask(
+        null,
+        new KinesisIndexTaskIOConfig(
+            null,
+            "sequence0",
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "2"
+            )),
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "4"
+            )),
+            true,
+            null,
+            null,
+            "awsEndpoint",
+            null,
+            null,
+            null,
+            null,
+            null,
+            false
+        )
+    );
+    final KinesisIndexTask task2 = createTask(
+        null,
+        new KinesisIndexTaskIOConfig(
+            null,
+            "sequence1",
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "3"
+            )),
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "9"
+            )),
+            true,
+            null,
+            null,
+            "awsEndpoint",
+            null,
+            null,
+            null,
+            null,
+            null,
+            false
+        )
+    );
+
+    // Run first task
+    final ListenableFuture<TaskStatus> future1 = runTask(task1);
+    Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode());
+
+    // Run second task
+    final ListenableFuture<TaskStatus> future2 = runTask(task2);
+    Assert.assertEquals(TaskState.FAILED, future2.get().getStatusCode());
+
+    verifyAll();
+    // Check metrics
+    Assert.assertEquals(3, task1.getRunner().getRowIngestionMeters().getProcessed());
+    Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable());
+    Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway());
+    Assert.assertEquals(3, task2.getRunner().getRowIngestionMeters().getProcessed());
+    Assert.assertEquals(4, task2.getRunner().getRowIngestionMeters().getUnparseable());
+    Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
+
+    // Check published segments & metadata, should all be from the first task
+    SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0);
+    SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0);
+    Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
+    Assert.assertEquals(
+        new KinesisDataSourceMetadata(
+            new SeekableStreamPartitions<>(
+                stream,
+                ImmutableMap.of(
+                    shardId1,
+                    "4"
+                )
+            )),
+        metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
+    );
+
+    // Check segments in deep storage
+    Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
+    Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2));
+  }
+
+
+  @Test(timeout = 120_000L)
+  public void testRunConflictingWithoutTransactions() throws Exception
+  {
+    recordSupplier.assign(anyObject());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes();
+
+    recordSupplier.seek(anyObject(), anyString());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.poll(anyLong())).andReturn(records.subList(2, 13))
+                                          .once()
+                                          .andReturn(records.subList(3, 13))
+                                          .once();
+
+    recordSupplier.close();
+    expectLastCall().times(2);
+
+    replayAll();
+
+    final KinesisIndexTask task1 = createTask(
+        null,
+        new KinesisIndexTaskIOConfig(
+            null,
+            "sequence0",
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "2"
+            )),
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "4"
+            )),
+            false,
+            null,
+            null,
+            "awsEndpoint",
+            null,
+            null,
+            null,
+            null,
+            null,
+            false
+        )
+    );
+    final KinesisIndexTask task2 = createTask(
+        null,
+        new KinesisIndexTaskIOConfig(
+            null,
+            "sequence1",
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "3"
+            )),
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "9"
+            )),
+            false,
+            null,
+            null,
+            "awsEndpoint",
+            null,
+            null,
+            null,
+            null,
+            null,
+            false
+        )
+    );
+
+    // Run first task
+    final ListenableFuture<TaskStatus> future1 = runTask(task1);
+    Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode());
+
+    // Check published segments & metadata
+    SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0);
+    SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0);
+    Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
+    Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()));
+
+    // Run second task
+    final ListenableFuture<TaskStatus> future2 = runTask(task2);
+    Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode());
+
+    verifyAll();
+
+    // Check metrics
+    Assert.assertEquals(3, task1.getRunner().getRowIngestionMeters().getProcessed());
+    Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable());
+    Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway());
+    Assert.assertEquals(3, task2.getRunner().getRowIngestionMeters().getProcessed());
+    Assert.assertEquals(4, task2.getRunner().getRowIngestionMeters().getUnparseable());
+    Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
+
+    // Check published segments & metadata
+    SegmentDescriptor desc3 = SD(task2, "2011/P1D", 1);
+    SegmentDescriptor desc4 = SD(task2, "2013/P1D", 0);
+    Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors());
+    Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()));
+
+    // Check segments in deep storage
+    Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
+    Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2));
+    Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc3));
+    Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc4));
+  }
+
+
+  @Test(timeout = 120_000L)
+  public void testRunOneTaskTwoPartitions() throws Exception
+  {
+    // Insert data
+    recordSupplier.assign(anyObject());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes();
+
+    recordSupplier.seek(anyObject(), anyString());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.poll(anyLong())).andReturn(records.subList(2, records.size())).once();
+
+    recordSupplier.close();
+    expectLastCall().once();
+
+    replayAll();
+
+    final KinesisIndexTask task = createTask(
+        null,
+        new KinesisIndexTaskIOConfig(
+            null,
+            "sequence1",
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "2",
+                shardId0,
+                "0"
+            )),
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "4",
+                shardId0,
+                "1"
+            )),
+            true,
+            null,
+            null,
+            "awsEndpoint",
+            null,
+            null,
+            null,
+            null,
+            null,
+            false
+        )
+    );
+
+    final ListenableFuture<TaskStatus> future = runTask(task);
+
+    while (countEvents(task) < 5) {
+      Thread.sleep(10);
+    }
+
+    // Wait for tasks to exit
+    Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode());
+
+    verifyAll();
+
+    // Check metrics
+    Assert.assertEquals(5, task.getRunner().getRowIngestionMeters().getProcessed());
+    Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable());
+    Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
+
+    // Check published segments & metadata
+    SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
+    SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
+    SegmentDescriptor desc4 = SD(task, "2012/P1D", 0);
+    Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc4), publishedDescriptors());
+    Assert.assertEquals(
+        new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+            shardId1,
+            "4",
+            shardId0,
+            "1"
+        ))),
+        metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
+    );
+
+    // Check segments in deep storage
+    Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
+    Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc4));
+
+    // Check desc2/desc3 without strong ordering because two partitions are interleaved nondeterministically
+    Assert.assertEquals(
+        ImmutableSet.of(ImmutableList.of("d", "e", "h")),
+        ImmutableSet.of(readSegmentColumn("dim1", desc2))
+    );
+  }
+
+
+  @Test(timeout = 120_000L)
+  public void testRunTwoTasksTwoPartitions() throws Exception
+  {
+    recordSupplier.assign(anyObject());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes();
+
+    recordSupplier.seek(anyObject(), anyString());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.poll(anyLong())).andReturn(records.subList(2, 13))
+                                          .once()
+                                          .andReturn(records.subList(13, 15))
+                                          .once();
+
+    recordSupplier.close();
+    expectLastCall().times(2);
+
+    replayAll();
+
+    final KinesisIndexTask task1 = createTask(
+        null,
+        new KinesisIndexTaskIOConfig(
+            null,
+            "sequence0",
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "2"
+            )),
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "4"
+            )),
+            true,
+            null,
+            null,
+            "awsEndpoint",
+            null,
+            null,
+            null,
+            null,
+            null,
+            false
+        )
+    );
+    final KinesisIndexTask task2 = createTask(
+        null,
+        new KinesisIndexTaskIOConfig(
+            null,
+            "sequence1",
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId0,
+                "0"
+            )),
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId0,
+                "1"
+            )),
+            true,
+            null,
+            null,
+            "awsEndpoint",
+            null,
+            null,
+            null,
+            null,
+            null,
+            false
+        )
+    );
+
+    final ListenableFuture<TaskStatus> future1 = runTask(task1);
+    Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode());
+
+    final ListenableFuture<TaskStatus> future2 = runTask(task2);
+    Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode());
+
+    verifyAll();
+
+    // Check metrics
+    Assert.assertEquals(3, task1.getRunner().getRowIngestionMeters().getProcessed());
+    Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable());
+    Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway());
+    Assert.assertEquals(2, task2.getRunner().getRowIngestionMeters().getProcessed());
+    Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getUnparseable());
+    Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
+
+    // Check published segments & metadata
+    SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0);
+    SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0);
+    SegmentDescriptor desc3 = SD(task2, "2011/P1D", 1);
+    SegmentDescriptor desc4 = SD(task2, "2012/P1D", 0);
+
+    Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors());
+    Assert.assertEquals(
+        new KinesisDataSourceMetadata(
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "4",
+                shardId0,
+                "1"
+            ))),
+        metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
+    );
+
+    // Check segments in deep storage
+    Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
+    // Check desc2/desc3 without strong ordering because two partitions are interleaved nondeterministically
+    Assert.assertEquals(
+        ImmutableSet.of(ImmutableList.of("d", "e"), ImmutableList.of("h")),
+        ImmutableSet.of(readSegmentColumn("dim1", desc2), readSegmentColumn("dim1", desc3))
+    );
+    Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc4));
+  }
+
+
+  @Test(timeout = 120_000L)
+  public void testRestore() throws Exception
+  {
+    recordSupplier.assign(anyObject());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes();
+
+    recordSupplier.seek(anyObject(), anyString());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.poll(anyLong())).andReturn(records.subList(2, 4))
+                                          .once()
+                                          .andReturn(Collections.emptyList())
+                                          .anyTimes();
+
+    replayAll();
+
+    final KinesisIndexTask task1 = createTask(
+        "task1",
+        new KinesisIndexTaskIOConfig(
+            null,
+            "sequence0",
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "2"
+            )),
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "5"
+            )),
+            true,
+            null,
+            null,
+            "awsEndpoint",
+            null,
+            null,
+            null,
+            null,
+            null,
+            false
+        )
+    );
+
+    final ListenableFuture<TaskStatus> future1 = runTask(task1);
+
+    while (countEvents(task1) != 2) {
+      Thread.sleep(25);
+    }
+
+    Assert.assertEquals(2, countEvents(task1));
+
+    // Stop without publishing segment
+    task1.stopGracefully();
+    unlockAppenderatorBasePersistDirForTask(task1);
+
+    Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode());
+
+    verifyAll();
+    reset(recordSupplier);
+
+    recordSupplier.assign(anyObject());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes();
+
+    recordSupplier.seek(anyObject(), anyString());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.poll(anyLong())).andReturn(records.subList(3, 6)).once();
+
+    recordSupplier.close();
+    expectLastCall();
+
+    replayAll();
+
+    // Start a new task
+    final KinesisIndexTask task2 = createTask(
+        task1.getId(),
+        new KinesisIndexTaskIOConfig(
+            null,
+            "sequence0",
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "2"
+            )),
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "5"
+            )),
+            true,
+            null,
+            null,
+            "awsEndpoint",
+            null,
+            null,
+            ImmutableSet.of(shardId1),
+            null,
+            null,
+            false
+        )
+    );
+
+    final ListenableFuture<TaskStatus> future2 = runTask(task2);
+
+    while (countEvents(task2) < 3) {
+      Thread.sleep(25);
+    }
+
+    Assert.assertEquals(3, countEvents(task2));
+
+    // Wait for task to exit
+    Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode());
+
+    verifyAll();
+
+    // Check metrics
+    Assert.assertEquals(2, task1.getRunner().getRowIngestionMeters().getProcessed());
+    Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable());
+    Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway());
+    Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getProcessed());
+    Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getUnparseable());
+    Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
+
+    // Check published segments & metadata
+    SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0);
+    SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0);
+    Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
+    Assert.assertEquals(
+        new KinesisDataSourceMetadata(
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "5"
+            ))),
+        metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
+    );
+
+    // Check segments in deep storage
+    Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
+    Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2));
+  }
+
+
+  @Test(timeout = 120_000L)
+  public void testRunWithPauseAndResume() throws Exception
+  {
+    recordSupplier.assign(anyObject());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes();
+
+    recordSupplier.seek(anyObject(), anyString());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.poll(anyLong())).andReturn(records.subList(2, 5))
+                                          .once()
+                                          .andReturn(Collections.emptyList())
+                                          .anyTimes();
+
+    replayAll();
+
+    final KinesisIndexTask task = createTask(
+        "task1",
+        new KinesisIndexTaskIOConfig(
+            null,
+            "sequence0",
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "2"
+            )),
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "13"
+            )),
+            true,
+            null,
+            null,
+            "awsEndpoint",
+            null,
+            null,
+            null,
+            null,
+            null,
+            false
+        )
+    );
+
+    final ListenableFuture<TaskStatus> future = runTask(task);
+
+
+    while (countEvents(task) != 3) {
+      Thread.sleep(25);
+    }
+
+    Assert.assertEquals(3, countEvents(task));
+    Assert.assertEquals(Status.READING, task.getRunner().getStatus());
+
+    task.getRunner().pause();
+
+    while (task.getRunner().getStatus() != Status.PAUSED) {
+      Thread.sleep(10);
+    }
+    Assert.assertEquals(Status.PAUSED, task.getRunner().getStatus());
+
+    verifyAll();
+
+    Map<String, String> currentOffsets = task.getRunner().getCurrentOffsets();
+
+    try {
+      future.get(10, TimeUnit.SECONDS);
+      Assert.fail("Task completed when it should have been paused");
+    }
+    catch (TimeoutException e) {
+      // carry on..
+    }
+
+    Assert.assertEquals(currentOffsets, task.getRunner().getCurrentOffsets());
+
+    reset(recordSupplier);
+
+    recordSupplier.assign(anyObject());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes();
+
+    recordSupplier.seek(anyObject(), anyString());
+    expectLastCall().anyTimes();
+
+    recordSupplier.close();
+    expectLastCall().once();
+
+    replayAll();
+
+    task.getRunner().setEndOffsets(currentOffsets, true);
+
+    Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode());
+
+    verifyAll();
+    Assert.assertEquals(task.getRunner().getEndOffsets(), task.getRunner().getCurrentOffsets());
+
+    // Check metrics
+    Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed());
+    Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable());
+    Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
+
+    // Check published metadata
+    SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
+    SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
+    Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
+    Assert.assertEquals(
+        new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(
+            stream,
+            ImmutableMap.of(
+                shardId1,
+                currentOffsets.get(shardId1)
+            )
+        )),
+        metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
+    );
+
+    // Check segments in deep storage
+    Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
+    Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2));
+  }
+
+
+  @Test(timeout = 60_000L)
+  public void testRunContextSequenceAheadOfStartingOffsets() throws Exception
+  {
+    // This tests the case when a replacement task is created in place of a failed test
+    // which has done some incremental handoffs, thus the context will contain starting
+    // sequence sequences from which the task should start reading and ignore the start sequences
+    // Insert data
+    recordSupplier.assign(anyObject());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes();
+
+    recordSupplier.seek(anyObject(), anyString());
+    expectLastCall().anyTimes();
+
+    expect(recordSupplier.poll(anyLong())).andReturn(records.subList(2, 13))
+                                          .once();
+
+    recordSupplier.close();
+    expectLastCall();
+
+    replayAll();
+
+    final TreeMap<Integer, Map<String, String>> sequences = new TreeMap<>();
+    // Here the sequence number is 1 meaning that one incremental handoff was done by the failed task
+    // and this task should start reading from stream 2 for partition 0
+    sequences.put(1, ImmutableMap.of(shardId1, "2"));
+    final Map<String, Object> context = new HashMap<>();
+    context.put("checkpoints", objectMapper.writerWithType(new TypeReference<TreeMap<Integer, Map<String, String>>>()
+    {
+    }).writeValueAsString(sequences));
+
+
+    final KinesisIndexTask task = createTask(
+        "task1",
+        new KinesisIndexTaskIOConfig(
+            null,
+            "sequence0",
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "0"
+            )),
+            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+                shardId1,
+                "4"
+            )),
+            true,
+            null,
+            null,
+            "awsEndpoint",
+            null,
+            null,
+            null,
+            null,
+            null,
+            false
+        ),
+        context
+    );
+    final ListenableFuture<TaskStatus> future = runTask(task);
+
+    // Wait for task to exit
+    Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode());
+
+    // Check metrics
+    Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed());
+    Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable());
+    Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
+
+    // Check published metadata
+    SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
+    SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
+    Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
+    Assert.assertEquals(
+        new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of(
+            shardId1,
+            "4"
+        ))),
+        metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
+    );
+
+    // Check segments in deep storage
+    Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
+    Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2));
+  }
+
+  private ListenableFuture<TaskStatus> runTask(final Task task)
+  {
+    try {
+      taskStorage.insert(task, TaskStatus.running(task.getId()));
+    }
+    catch (EntryExistsException e) {
+      // suppress
+    }
+    taskLockbox.syncFromStorage();
+    final TaskToolbox toolbox = toolboxFactory.build(task);
+    synchronized (runningTasks) {
+      runningTasks.add(task);
+    }
+    return taskExec.submit(
+        () -> {
+          try {
+            if (task.isReady(toolbox.getTaskActionClient())) {
+              return task.run(toolbox);
+            } else {
+              throw new ISE("Task is not ready");
+            }
+          }
+          catch (Exception e) {
+            log.warn(e, "Task failed");
+            return TaskStatus.failure(task.getId(), Throwables.getStackTraceAsString(e));
+          }
+        }
+    );
+  }
+
+
+  private TaskLock getLock(final Task task, final Interval interval)
+  {
+    return Iterables.find(
+        taskLockbox.findLocksForTask(task),
+        new Predicate<TaskLock>()
+        {
+          @Override
+          public boolean apply(TaskLock lock)
+          {
+            return lock.getInterval().contains(interval);
+          }
+        }
+    );
+  }
+
+  private KinesisIndexTask createTask(
+      final String taskId,
+      final KinesisIndexTaskIOConfig ioConfig
+  )
+  {
+    return createTask(taskId, DATA_SCHEMA, ioConfig);
+  }
+
+  private KinesisIndexTask createTask(
+      final String taskId,
+      final KinesisIndexTaskIOConfig ioConfig,
+      final Map<String, Object> context
+  )
+  {
+    return createTask(taskId, DATA_SCHEMA, ioConfig, context);
+  }
+
+  private KinesisIndexTask createTask(
+      final String taskId,
+      final DataSchema dataSchema,
+      final KinesisIndexTaskIOConfig ioConfig
+  )
+  {
+    final KinesisIndexTaskTuningConfig tuningConfig = new KinesisIndexTaskTuningConfig(
+        1000,
+        null,
+        maxRowsPerSegment,
+        maxTotalRows,
+        new Period("P1Y"),
+        null,
+        null,
+        null,
+        true,
+        reportParseExceptions,
+        handoffConditionTimeout,
+        resetOffsetAutomatically,
+        skipAvailabilityCheck,
+        null,
+        null,
+        null,
+        5000,
+        null,
+        null,
+        logParseExceptions,
+        maxParseExceptions,
+        maxSavedParseExceptions,
+        maxRecordsPerPoll,
+        intermediateHandoffPeriod
+    );
+    final Map<String, Object> context = null;
+    final KinesisIndexTask task = new TestableKinesisIndexTask(
+        taskId,
+        null,
+        cloneDataSchema(dataSchema),
+        tuningConfig,
+        ioConfig,
+        context,
+        null,
+        null,
+        rowIngestionMetersFactory,
+        null
+    );
+
+    return task;
+  }
+
+
+  private KinesisIndexTask createTask(
+      final String taskId,
+      final DataSchema dataSchema,
+      final KinesisIndexTaskIOConfig ioConfig,
+      final Map<String, Object> context
+  )
+  {
+    final KinesisIndexTaskTuningConfig tuningConfig = new KinesisIndexTaskTuningConfig(
+        maxRowsInMemory,
+        null,
+        maxRowsPerSegment,
+        maxTotalRows,
+        new Period("P1Y"),
+        null,
+        null,
+        null,
+        true,
+        reportParseExceptions,
+        handoffConditionTimeout,
+        resetOffsetAutomatically,
+        true,
+        null,
+        null,
+        null,
+        null,
+        null,
+        null,
+        logParseExceptions,
+        maxParseExceptions,
+        maxSavedParseExceptions,
+        maxRecordsPerPoll,
+        intermediateHandoffPeriod
+    );
+    context.put(SeekableStreamSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED, true);
+
+    final KinesisIndexTask task = new TestableKinesisIndexTask(
+        taskId,
+        null,
+        cloneDataSchema(dataSchema),
+        tuningConfig,
+        ioConfig,
+        context,
+        null,
+        null,
+        rowIngestionMetersFactory,
+        null
+    );
+
+    return task;
+  }
+
+  private static DataSchema cloneDataSchema(final DataSchema dataSchema)
+  {
+    return new DataSchema(
+        dataSchema.getDataSource(),
+        dataSchema.getParserMap(),
+        dataSchema.getAggregators(),
+        dataSchema.getGranularitySpec(),
+        dataSchema.getTransformSpec(),
+        objectMapper
+    );
+  }
+
+  private QueryRunnerFactoryConglomerate makeTimeseriesOnlyConglomerate()
+  {
+    IntervalChunkingQueryRunnerDecorator queryRunnerDecorator = new IntervalChunkingQueryRunnerDecorator(
+        null,
+        null,
+        null
+    )
+    {
+      @Override
+      public <T> QueryRunner<T> decorate(
+          QueryRunner<T> delegate,
+          QueryToolChest<T, ? extends Query<T>> toolChest
+      )
+      {
+        return delegate;
+      }
+    };
+    return new DefaultQueryRunnerFactoryConglomerate(
+        ImmutableMap.of(
+            TimeseriesQuery.class,
+            new TimeseriesQueryRunnerFactory(
+                new TimeseriesQueryQueryToolChest(queryRunnerDecorator),
+                new TimeseriesQueryEngine(),
+                (query, future) -> {
+                  // do nothing
+                }
+            )
+        )
+    );
+  }
+
+  private void makeToolboxFactory() throws IOException
+  {
+    directory = tempFolder.newFolder();
+    final TestUtils testUtils = new TestUtils();
+    rowIngestionMetersFactory = testUtils.getRowIngestionMetersFactory();
+    final ObjectMapper objectMapper = testUtils.getTestObjectMapper();
+    objectMapper.setInjectableValues(((InjectableValues.Std) objectMapper.getInjectableValues()).addValue(
+        AWSCredentialsConfig.class,
+        new AWSCredentialsConfig()
+    ));
+    for (Module module : new KinesisIndexingServiceModule().getJacksonModules()) {
+      objectMapper.registerModule(module);
+    }
+    final TaskConfig taskConfig = new TaskConfig(
+        new File(directory, "taskBaseDir").getPath(),
+        null,
+        null,
+        50000,
+        null,
+        false,
+        null,
+        null
+    );
+    final TestDerbyConnector derbyConnector = derby.getConnector();
+    derbyConnector.createDataSourceTable();
+    derbyConnector.createPendingSegmentsTable();
+    derbyConnector.createSegmentTable();
+    derbyConnector.createRulesTable();
+    derbyConnector.createConfigTable();
+    derbyConnector.createTaskTables();
+    derbyConnector.createAuditTable();
+    taskStorage = new MetadataTaskStorage(
+        derbyConnector,
+        new TaskStorageConfig(null),
+        new DerbyMetadataStorageActionHandlerFactory(
+            derbyConnector,
+            derby.metadataTablesConfigSupplier().get(),
+            objectMapper
+        )
+    );
+    metadataStorageCoordinator = new IndexerSQLMetadataStorageCoordinator(
+        testUtils.getTestObjectMapper(),
+        derby.metadataTablesConfigSupplier().get(),
+        derbyConnector
+    );
+    taskLockbox = new TaskLockbox(taskStorage);
+    final TaskActionToolbox taskActionToolbox = new TaskActionToolbox(
+        taskLockbox,
+        taskStorage,
+        metadataStorageCoordinator,
+        emitter,
+        new SupervisorManager(null)
+        {
+          @Override
+          public boolean checkPointDataSourceMetadata(
+              String supervisorId,
+              @Nullable Integer taskGroupId,
+              String baseSequenceName,
+              @Nullable DataSourceMetadata previousDataSourceMetadata,
+              @Nullable DataSourceMetadata currentDataSourceMetadata
+          )
+          {
+            log.info("Adding checkpoint hash to the set");
+            checkpointRequestsHash.add(
+                Objects.hash(
+                    supervisorId,
+                    taskGroupId,
+                    previousDataSourceMetadata,
+                    currentDataSourceMetadata
+                )
+            );
+            return true;
+          }
+        },
+        new Counters()
+    );
+    final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory(
+        taskStorage,
+        taskActionToolbox,
+        new TaskAuditLogConfig(false)
+    );
+
+    final SegmentHandoffNotifierFactory handoffNotifierFactory = dataSource -> new SegmentHandoffNotifier()
+    {
+      @Override
+      public boolean registerSegmentHandoffCallback(
+          SegmentDescriptor descriptor,
+          Executor exec,
+          Runnable handOffRunnable
+      )
+      {
+        if (doHandoff) {
+          // Simulate immediate handoff
+          exec.execute(handOffRunnable);
+        }
+        return true;
+      }
+
+      @Override
+      public void start()
+      {
+        //Noop
+      }
+
+      @Override
+      public void close()
+      {
+        //Noop
+      }
+    };
+    final LocalDataSegmentPusherConfig dataSegmentPusherConfig = new LocalDataSegmentPusherConfig();
+    dataSegmentPusherConfig.storageDirectory = getSegmentDirectory();
+    final DataSegmentPusher dataSegmentPusher = new LocalDataSegmentPusher(dataSegmentPusherConfig, objectMapper);
+    SegmentLoaderConfig segmentLoaderConfig = new SegmentLoaderConfig()
+    {
+      @Override
+      public List<StorageLocationConfig> getLocations()
+      {
+        return new ArrayList<>();
+      }
+    };
+    toolboxFactory = new TaskToolboxFactory(
+        taskConfig,
+        taskActionClientFactory,
+        emitter,
+        dataSegmentPusher,
+        new TestDataSegmentKiller(),
+        null, // DataSegmentMover
+        null, // DataSegmentArchiver
+        new TestDataSegmentAnnouncer(),
+        EasyMock.createNiceMock(DataSegmentServerAnnouncer.class),
+        handoffNotifierFactory,
+        this::makeTimeseriesOnlyConglomerate,
+        MoreExecutors.sameThreadExecutor(), // queryExecutorService
+        EasyMock.createMock(MonitorScheduler.class),
+        new SegmentLoaderFactory(
+            new SegmentLoaderLocalCacheManager(null, segmentLoaderConfig, testUtils.getTestObjectMapper())
+        ),
+        testUtils.getTestObjectMapper(),
+        testUtils.getTestIndexIO(),
+        MapCache.create(1024),
+        new CacheConfig(),
+        new CachePopulatorStats(),
+        testUtils.getTestIndexMergerV9(),
+        EasyMock.createNiceMock(DruidNodeAnnouncer.class),
+        EasyMock.createNiceMock(DruidNode.class),
+        new LookupNodeService("tier"),
+        new DataNodeService("tier", 1, ServerType.INDEXER_EXECUTOR, 0),
+        new TaskReportFileWriter(reportsFile)
+    );
+  }
+
+  private void destroyToolboxFactory()
+  {
+    toolboxFactory = null;
+    taskStorage = null;
+    taskLockbox = null;
+    metadataStorageCoordinator = null;
+  }
+
+
+  private Set<SegmentDescriptor> publishedDescriptors()
+  {
+    return FluentIterable.from(
+        metadataStorageCoordinator.getUsedSegmentsForInterval(
+            DATA_SCHEMA.getDataSource(),
+            Intervals.of("0000/3000")
+        )
+    ).transform(DataSegment::toDescriptor).toSet();
+  }
+
+  private void unlockAppenderatorBasePersistDirForTask(KinesisIndexTask task)
+      throws NoSuchMethodException, InvocationTargetException, IllegalAccessException
+  {
+    Method unlockBasePersistDir = ((AppenderatorImpl) task.getAppenderator()).getClass()
+                                                                             .getDeclaredMethod(
+                                                                                 "unlockBasePersistDirectory");
+    unlockBasePersistDir.setAccessible(true);
+    unlockBasePersistDir.invoke(task.getAppenderator());
+  }
+
+  private File getSegmentDirectory()
+  {
+    return new File(directory, "segments");
+  }
+
+
+  private List<String> readSegmentColumn(final String column, final SegmentDescriptor descriptor) throws IOException
+  {
+    File indexBasePath = new File(
+        StringUtils.format(
+            "%s/%s/%s_%s/%s/%d",
+            getSegmentDirectory(),
+            DATA_SCHEMA.getDataSource(),
+            descriptor.getInterval().getStart(),
+            descriptor.getInterval().getEnd(),
+            descriptor.getVersion(),
+            descriptor.getPartitionNumber()
+        )
+    );
+
+    File outputLocation = new File(
+        directory,
+        StringUtils.format(
+            "%s_%s_%s_%s",
+            descriptor.getInterval().getStart(),
+            descriptor.getInterval().getEnd(),
+            descriptor.getVersion(),
+            descriptor.getPartitionNumber()
+        )
+    );
+    outputLocation.mkdir();
+    CompressionUtils.unzip(
+        Files.asByteSource(new File(indexBasePath.listFiles()[0], "index.zip")),
+        outputLocation,
+        Predicates.alwaysFalse(),
+        false
+    );
+    IndexIO indexIO = new TestUtils().getTestIndexIO();
+    QueryableIndex index = indexIO.loadIndex(outputLocation);
+    DictionaryEncodedColumn<String> theColumn = (DictionaryEncodedColumn<String>) index.getColumnHolder(column)
+                                                                                       .getColumn();
+    List<String> values = new ArrayList<>();
+    for (int i = 0; i < theColumn.length(); i++) {
+      int id = theColumn.getSingleValueRow(i);
+      String value = theColumn.lookupName(id);
+      values.add(value);
+    }
+    return values;
+  }
+
+  public long countEvents(final Task task)
+  {
+    // Do a query.
+    TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
+                                  .dataSource(DATA_SCHEMA.getDataSource())
+                                  .aggregators(
+                                      ImmutableList.of(
+                                          new LongSumAggregatorFactory("rows", "rows")
+                                      )
+                                  ).granularity(Granularities.ALL)
+                                  .intervals("0000/3000")
+                                  .build();
+
+    List<Result<TimeseriesResultValue>> results =
+        task.getQueryRunner(query).run(QueryPlus.wrap(query), ImmutableMap.of()).toList();
+
+    return results.isEmpty() ? 0L : DimensionHandlerUtils.nullToZero(results.get(0).getValue().getLongMetric("rows"));
+  }
+
+  private static List<byte[]> JB(
+      String timestamp,
+      String dim1,
+      String dim2,
+      String dimLong,
+      String dimFloat,
+      String met1
+  )
+  {
+    try {
+      return Collections.singletonList(new ObjectMapper().writeValueAsBytes(
+          ImmutableMap.builder()
+                      .put("timestamp", timestamp)
+                      .put("dim1", dim1)
+                      .put("dim2", dim2)
+                      .put("dimLong", dimLong)
+                      .put("dimFloat", dimFloat)
+                      .put("met1", met1)
+                      .build()
+      ));
+    }
+    catch (Exception e) {
+      throw Throwables.propagate(e);
+    }
+  }
+
+  private SegmentDescriptor SD(final Task task, final String intervalString, final int partitionNum)
+  {
+    final Interval interval = Intervals.of(intervalString);
+    return new SegmentDescriptor(interval, getLock(task, interval).getVersion(), partitionNum);
+  }
+
+  private IngestionStatsAndErrorsTaskReportData getTaskReportData() throws IOException
+  {
+    Map<String, TaskReport> taskReports = objectMapper.readValue(
+        reportsFile,
+        new TypeReference<Map<String, TaskReport>>()
+        {
+        }
+    );
+    return IngestionStatsAndErrorsTaskReportData.getPayloadFromTaskReports(
+        taskReports
+    );
+  }
+
+  @JsonTypeName("index_kinesis")
+  private static class TestableKinesisIndexTask extends KinesisIndexTask
+  {
+    @JsonCreator
+    public TestableKinesisIndexTask(
+        @JsonProperty("id") String id,
+        @JsonProperty("resource") TaskResource taskResource,
+        @JsonProperty("dataSchema") DataSchema dataSchema,
+        @JsonProperty("tuningConfig") KinesisIndexTaskTuningConfig tuningConfig,
+        @JsonProperty("ioConfig") KinesisIndexTaskIOConfig ioConfig,
+        @JsonProperty("context") Map<String, Object> context,
+        @JacksonInject ChatHandlerProvider chatHandlerProvider,
+        @JacksonInject AuthorizerMapper authorizerMapper,
+        @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory,
+        @JacksonInject AWSCredentialsConfig awsCredentialsConfig
+    )
+    {
+      super(
+          id,
+          taskResource,
+          dataSchema,
+          tuningConfig,
+          ioConfig,
+          context,
+          chatHandlerProvider,
+          authorizerMapper,
+          rowIngestionMetersFactory,
+          awsCredentialsConfig
+      );
+    }
+
+    @Override
+    protected KinesisRecordSupplier newTaskRecordSupplier()
+    {
+      return recordSupplier;
+    }
+  }
+
+}
diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java
new file mode 100644
index 00000000000..4e967c414a9
--- /dev/null
+++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.indexing.kinesis;
+
+import com.fasterxml.jackson.databind.JsonMappingException;
+import com.fasterxml.jackson.databind.Module;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorTuningConfig;
+import org.apache.druid.jackson.DefaultObjectMapper;
+import org.apache.druid.segment.IndexSpec;
+import org.apache.druid.segment.indexing.TuningConfig;
+import org.hamcrest.CoreMatchers;
+import org.joda.time.Period;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.io.File;
+
+public class KinesisIndexTaskTuningConfigTest
+{
+  private final ObjectMapper mapper;
+
+  public KinesisIndexTaskTuningConfigTest()
+  {
+    mapper = new DefaultObjectMapper();
+    mapper.registerModules((Iterable<Module>) new KinesisIndexingServiceModule().getJacksonModules());
+  }
+
+  @Rule
+  public final ExpectedException exception = ExpectedException.none();
+
+  @Test
+  public void testSerdeWithDefaults() throws Exception
+  {
+    String jsonStr = "{\"type\": \"kinesis\"}";
+
+    KinesisIndexTaskTuningConfig config = (KinesisIndexTaskTuningConfig) mapper.readValue(
+        mapper.writeValueAsString(
+            mapper.readValue(
+                jsonStr,
+                TuningConfig.class
+            )
+        ),
+        TuningConfig.class
+    );
+
+    Assert.assertNotNull(config.getBasePersistDirectory());
+    Assert.assertEquals(1000000, config.getMaxRowsInMemory());
+    Assert.assertEquals(5_000_000, config.getMaxRowsPerSegment());
+    Assert.assertEquals(new Period("PT10M"), config.getIntermediatePersistPeriod());
+    Assert.assertEquals(0, config.getMaxPendingPersists());
+    Assert.assertEquals(new IndexSpec(), config.getIndexSpec());
+    Assert.assertTrue(config.getBuildV9Directly());
+    Assert.assertFalse(config.isReportParseExceptions());
+    Assert.assertEquals(0, config.getHandoffConditionTimeout());
+    Assert.assertEquals(10000, config.getRecordBufferSize());
+    Assert.assertEquals(5000, config.getRecordBufferOfferTimeout());
+    Assert.assertEquals(5000, config.getRecordBufferFullWait());
+    Assert.assertEquals(20000, config.getFetchSequenceNumberTimeout());
+    Assert.assertNull(config.getFetchThreads());
+    Assert.assertFalse(config.isSkipSequenceNumberAvailabilityCheck());
+    Assert.assertFalse(config.isResetOffsetAutomatically());
+  }
+
+  @Test
+  public void testSerdeWithNonDefaults() throws Exception
+  {
+    String jsonStr = "{\n"
+                     + "  \"type\": \"kinesis\",\n"
+                     + "  \"basePersistDirectory\": \"/tmp/xxx\",\n"
+                     + "  \"maxRowsInMemory\": 100,\n"
+                     + "  \"maxRowsPerSegment\": 100,\n"
+                     + "  \"intermediatePersistPeriod\": \"PT1H\",\n"
+                     + "  \"maxPendingPersists\": 100,\n"
+                     + "  \"buildV9Directly\": true,\n"
+                     + "  \"reportParseExceptions\": true,\n"
+                     + "  \"handoffConditionTimeout\": 100,\n"
+                     + "  \"recordBufferSize\": 1000,\n"
+                     + "  \"recordBufferOfferTimeout\": 500,\n"
+                     + "  \"recordBufferFullWait\": 500,\n"
+                     + "  \"fetchSequenceNumberTimeout\": 6000,\n"
+                     + "  \"resetOffsetAutomatically\": false,\n"
+                     + "  \"skipSequenceNumberAvailabilityCheck\": true,\n"
+                     + "  \"fetchThreads\": 2\n"
+                     + "}";
+
+    KinesisIndexTaskTuningConfig config = (KinesisIndexTaskTuningConfig) mapper.readValue(
+        mapper.writeValueAsString(
+            mapper.readValue(
+                jsonStr,
+                TuningConfig.class
+            )
+        ),
+        TuningConfig.class
+    );
+
+    Assert.assertEquals(new File("/tmp/xxx"), config.getBasePersistDirectory());
+    Assert.assertEquals(100, config.getMaxRowsInMemory());
+    Assert.assertEquals(100, config.getMaxRowsPerSegment());
+    Assert.assertEquals(new Period("PT1H"), config.getIntermediatePersistPeriod());
+    Assert.assertEquals(100, config.getMaxPendingPersists());
+    Assert.assertTrue(config.getBuildV9Directly());
+    Assert.assertTrue(config.isReportParseExceptions());
+    Assert.assertEquals(100, config.getHandoffConditionTimeout());
+    Assert.assertEquals(1000, config.getRecordBufferSize());
+    Assert.assertEquals(500, config.getRecordBufferOfferTimeout());
+    Assert.assertEquals(500, config.getRecordBufferFullWait());
+    Assert.assertEquals(6000, config.getFetchSequenceNumberTimeout());
+    Assert.assertEquals(2, (int) config.getFetchThreads());
+    Assert.assertTrue(config.isSkipSequenceNumberAvailabilityCheck());
+    Assert.assertFalse(config.isResetOffsetAutomatically());
+  }
+
+  @Test
+  public void testResetOffsetAndSkipSequenceNotBothTrue() throws Exception
+  {
+    String jsonStr = "{\n"
+                     + "  \"type\": \"kinesis\",\n"
+                     + "  \"basePersistDirectory\": \"/tmp/xxx\",\n"
+                     + "  \"maxRowsInMemory\": 100,\n"
+                     + "  \"maxRowsPerSegment\": 100,\n"
+                     + "  \"intermediatePersistPeriod\": \"PT1H\",\n"
+                     + "  \"maxPendingPersists\": 100,\n"
+                     + "  \"buildV9Directly\": true,\n"
+                     + "  \"reportParseExceptions\": true,\n"
+                     + "  \"handoffConditionTimeout\": 100,\n"
+                     + "  \"recordBufferSize\": 1000,\n"
+                     + "  \"recordBufferOfferTimeout\": 500,\n"
+                     + "  \"recordBufferFullWait\": 500,\n"
+                     + "  \"fetchSequenceNumberTimeout\": 6000,\n"
+                     + "  \"resetOffsetAutomatically\": true,\n"
+                     + "  \"skipSequenceNumberAvailabilityCheck\": true,\n"
+                     + "  \"fetchThreads\": 2\n"
+                     + "}";
+
+    exception.expect(JsonMappingException.class);
+    exception.expectCause(CoreMatchers.isA(IllegalArgumentException.class));
+    exception.expectMessage(CoreMatchers.containsString(
+        "resetOffsetAutomatically cannot be used if skipSequenceNumberAvailabilityCheck=true"));
+    mapper.readValue(jsonStr, TuningConfig.class);
+  }
+
+  @Test
+  public void testConvert()
+  {
+    KinesisSupervisorTuningConfig original = new KinesisSupervisorTuningConfig(
+        1,
+        (long) 3,
+        2,
+        100L,
+        new Period("PT3S"),
+        new File("/tmp/xxx"),
+        4,
+        new IndexSpec(),
+        true,
+        true,
+        5L,
+        true,
+        false,
+        null,
+        null,
+        null,
+        null,
+        null,
+        null,
+        1000,
+        500,
+        500,
+        6000,
+        2,
+        null,
+        null,
+        null,
+        null,
+        null
+    );
+    KinesisIndexTaskTuningConfig copy = (KinesisIndexTaskTuningConfig) original.convertToTaskTuningConfig();
+
+    Assert.assertEquals(1, copy.getMaxRowsInMemory());
+    Assert.assertEquals(3, copy.getMaxBytesInMemory());
+    Assert.assertEquals(2, copy.getMaxRowsPerSegment());
+    Assert.assertEquals(100L, (long) copy.getMaxTotalRows());
+    Assert.assertEquals(new Period("PT3S"), copy.getIntermediatePersistPeriod());
+    Assert.assertEquals(new File("/tmp/xxx"), copy.getBasePersistDirectory());
+    Assert.assertEquals(4, copy.getMaxPendingPersists());
+    Assert.assertEquals(new IndexSpec(), copy.getIndexSpec());
+    Assert.assertTrue(copy.getBuildV9Directly());
+    Assert.assertTrue(copy.isReportParseExceptions());
+    Assert.assertEquals(5L, copy.getHandoffConditionTimeout());
+    Assert.assertEquals(1000, copy.getRecordBufferSize());
+    Assert.assertEquals(500, copy.getRecordBufferOfferTimeout());
+    Assert.assertEquals(500, copy.getRecordBufferFullWait());
+    Assert.assertEquals(6000, copy.getFetchSequenceNumberTimeout());
+    Assert.assertEquals(2, (int) copy.getFetchThreads());
+    Assert.assertFalse(copy.isSkipSequenceNumberAvailabilityCheck());
+    Assert.assertTrue(copy.isResetOffsetAutomatically());
+    Assert.assertEquals(100, copy.getMaxRecordsPerPoll());
+    Assert.assertEquals(new Period().withDays(Integer.MAX_VALUE), copy.getIntermediateHandoffPeriod());
+  }
+}
diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java
new file mode 100644
index 00000000000..166678cb7eb
--- /dev/null
+++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java
@@ -0,0 +1,615 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.indexing.kinesis;
+
+import com.amazonaws.services.kinesis.AmazonKinesis;
+import com.amazonaws.services.kinesis.AmazonKinesisClient;
+import com.amazonaws.services.kinesis.model.DescribeStreamResult;
+import com.amazonaws.services.kinesis.model.GetRecordsRequest;
+import com.amazonaws.services.kinesis.model.GetRecordsResult;
+import com.amazonaws.services.kinesis.model.GetShardIteratorResult;
+import com.amazonaws.services.kinesis.model.Record;
+import com.amazonaws.services.kinesis.model.Shard;
+import com.amazonaws.services.kinesis.model.StreamDescription;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Throwables;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord;
+import org.apache.druid.indexing.seekablestream.common.StreamPartition;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.easymock.Capture;
+import org.easymock.EasyMockSupport;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.easymock.EasyMock.anyObject;
+import static org.easymock.EasyMock.anyString;
+import static org.easymock.EasyMock.capture;
+import static org.easymock.EasyMock.eq;
+import static org.easymock.EasyMock.expect;
+
+public class KinesisRecordSupplierTest extends EasyMockSupport
+{
+  private static final String stream = "stream";
+  private static long poll_timeout_millis = 2000;
+  private static int recordsPerFetch;
+  private static String shardId1 = "1";
+  private static String shardId0 = "0";
+  private static String shard1Iterator = "1";
+  private static String shard0Iterator = "0";
+  private static AmazonKinesis kinesis;
+  private static DescribeStreamResult describeStreamResult;
+  private static GetShardIteratorResult getShardIteratorResult0;
+  private static GetShardIteratorResult getShardIteratorResult1;
+  private static GetRecordsResult getRecordsResult0;
+  private static GetRecordsResult getRecordsResult1;
+  private static StreamDescription streamDescription;
+  private static Shard shard0;
+  private static Shard shard1;
+  private static KinesisRecordSupplier recordSupplier;
+  private static List<Record> shard1Records = ImmutableList.of(
+      new Record().withData(JB("2011", "d", "y", "10", "20.0", "1.0")).withSequenceNumber("0"),
+      new Record().withData(JB("2011", "e", "y", "10", "20.0", "1.0")).withSequenceNumber("1"),
+      new Record().withData(JB("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")).withSequenceNumber("2"),
+      new Record().withData(ByteBuffer.wrap(StringUtils.toUtf8("unparseable"))).withSequenceNumber("3"),
+      new Record().withData(ByteBuffer.wrap(StringUtils.toUtf8("unparseable2"))).withSequenceNumber("4"),
+      new Record().withData(ByteBuffer.wrap(StringUtils.toUtf8("{}"))).withSequenceNumber("5"),
+      new Record().withData(JB("2013", "f", "y", "10", "20.0", "1.0")).withSequenceNumber("6"),
+      new Record().withData(JB("2049", "f", "y", "notanumber", "20.0", "1.0")).withSequenceNumber("7"),
+      new Record().withData(JB("2012", "g", "y", "10", "20.0", "1.0")).withSequenceNumber("8"),
+      new Record().withData(JB("2011", "h", "y", "10", "20.0", "1.0")).withSequenceNumber("9")
+  );
+  private static List<Record> shard0Records = ImmutableList.of(
+      new Record().withData(JB("2008", "a", "y", "10", "20.0", "1.0")).withSequenceNumber("0"),
+      new Record().withData(JB("2009", "b", "y", "10", "20.0", "1.0")).withSequenceNumber("1")
+  );
+  private static List<Object> allRecords = ImmutableList.builder()
+                                                        .addAll(shard0Records.stream()
+                                                                             .map(x -> new OrderedPartitionableRecord<>(
+                                                                                 stream,
+                                                                                 shardId0,
+                                                                                 x.getSequenceNumber(),
+                                                                                 Collections
+                                                                                     .singletonList(
+                                                                                         toByteArray(
+                                                                                             x.getData()))
+                                                                             ))
+                                                                             .collect(
+                                                                                 Collectors
+                                                                                     .toList()))
+                                                        .addAll(shard1Records.stream()
+                                                                             .map(x -> new OrderedPartitionableRecord<>(
+                                                                                 stream,
+                                                                                 shardId1,
+                                                                                 x.getSequenceNumber(),
+                                                                                 Collections
+                                                                                     .singletonList(
+                                                                                         toByteArray(
+                                                                                             x.getData()))
+                                                                             ))
+                                                                             .collect(
+                                                                                 Collectors
+                                                                                     .toList()))
+                                                        .build();
+
+  private static ByteBuffer JB(String timestamp, String dim1, String dim2, String dimLong, String dimFloat, String met1)
+  {
+    try {
+      return ByteBuffer.wrap(new ObjectMapper().writeValueAsBytes(
+          ImmutableMap.builder()
+                      .put("timestamp", timestamp)
+                      .put("dim1", dim1)
+                      .put("dim2", dim2)
+                      .put("dimLong", dimLong)
+                      .put("dimFloat", dimFloat)
+                      .put("met1", met1)
+                      .build()
+      ));
+    }
+    catch (Exception e) {
+      throw Throwables.propagate(e);
+    }
+  }
+
+  @Before
+  public void setupTest()
+  {
+    kinesis = createMock(AmazonKinesisClient.class);
+    describeStreamResult = createMock(DescribeStreamResult.class);
+    getShardIteratorResult0 = createMock(GetShardIteratorResult.class);
+    getShardIteratorResult1 = createMock(GetShardIteratorResult.class);
+    getRecordsResult0 = createMock(GetRecordsResult.class);
+    getRecordsResult1 = createMock(GetRecordsResult.class);
+    streamDescription = createMock(StreamDescription.class);
+    shard0 = createMock(Shard.class);
+    shard1 = createMock(Shard.class);
+    recordsPerFetch = 1;
+  }
+
+  @After
+  public void tearDownTest()
+  {
+    recordSupplier.close();
+    recordSupplier = null;
+  }
+
+  @Test
+  public void testSupplierSetup()
+  {
+    Capture<String> captured = Capture.newInstance();
+    expect(kinesis.describeStream(capture(captured))).andReturn(describeStreamResult).once();
+    expect(describeStreamResult.getStreamDescription()).andReturn(streamDescription).once();
+    expect(streamDescription.getShards()).andReturn(ImmutableList.of(shard0, shard1)).once();
+    expect(shard0.getShardId()).andReturn(shardId0).once();
+    expect(shard1.getShardId()).andReturn(shardId1).once();
+
+    replayAll();
+
+    Set<StreamPartition<String>> partitions = ImmutableSet.of(
+        StreamPartition.of(stream, shardId0),
+        StreamPartition.of(stream, shardId1)
+    );
+
+    recordSupplier = new KinesisRecordSupplier(
+        kinesis,
+        recordsPerFetch,
+        0,
+        2,
+        false,
+        100,
+        5000,
+        5000,
+        60000,
+        5
+    );
+
+    Assert.assertTrue(recordSupplier.getAssignment().isEmpty());
+
+    recordSupplier.assign(partitions);
+
+    Assert.assertEquals(partitions, recordSupplier.getAssignment());
+    Assert.assertEquals(ImmutableSet.of(shardId1, shardId0), recordSupplier.getPartitionIds(stream));
+    Assert.assertEquals(Collections.emptyList(), recordSupplier.poll(100));
+
+    verifyAll();
+    Assert.assertEquals(stream, captured.getValue());
+  }
+
+  private static GetRecordsRequest generateGetRecordsReq(String shardIterator, int limit)
+  {
+    return new GetRecordsRequest().withShardIterator(shardIterator).withLimit(limit);
+  }
+
+  // filter out EOS markers
+  private static List<OrderedPartitionableRecord<String, String>> cleanRecords(List<OrderedPartitionableRecord<String, String>> records)
+  {
+    return records.stream()
+                  .filter(x -> !x.getSequenceNumber()
+                                 .equals(KinesisSequenceNumber.END_OF_SHARD_MARKER))
+                  .collect(Collectors.toList());
+  }
+
+  @Test
+  public void testPoll() throws InterruptedException
+  {
+    recordsPerFetch = 100;
+
+    expect(kinesis.getShardIterator(
+        anyObject(),
+        eq(shardId0),
+        anyString(),
+        anyString()
+    )).andReturn(
+        getShardIteratorResult0).anyTimes();
+
+    expect(kinesis.getShardIterator(
+        anyObject(),
+        eq(shardId1),
+        anyString(),
+        anyString()
+    )).andReturn(
+        getShardIteratorResult1).anyTimes();
+
+    expect(getShardIteratorResult0.getShardIterator()).andReturn(shard0Iterator).anyTimes();
+    expect(getShardIteratorResult1.getShardIterator()).andReturn(shard1Iterator).anyTimes();
+    expect(kinesis.getRecords(generateGetRecordsReq(shard0Iterator, recordsPerFetch))).andReturn(getRecordsResult0)
+                                                                                      .anyTimes();
+    expect(kinesis.getRecords(generateGetRecordsReq(shard1Iterator, recordsPerFetch))).andReturn(getRecordsResult1)
+                                                                                      .anyTimes();
+    expect(getRecordsResult0.getRecords()).andReturn(shard0Records).once();
+    expect(getRecordsResult1.getRecords()).andReturn(shard1Records).once();
+    expect(getRecordsResult0.getNextShardIterator()).andReturn(null).anyTimes();
+    expect(getRecordsResult1.getNextShardIterator()).andReturn(null).anyTimes();
+
+    replayAll();
+
+    Set<StreamPartition<String>> partitions = ImmutableSet.of(
+        StreamPartition.of(stream, shardId0),
+        StreamPartition.of(stream, shardId1)
+    );
+
+
+    recordSupplier = new KinesisRecordSupplier(
+        kinesis,
+        recordsPerFetch,
+        0,
+        2,
+        false,
+        100,
+        5000,
+        5000,
+        60000,
+        100

  (This diff was longer than 20,000 lines, and has been truncated...)


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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