You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by ji...@apache.org on 2019/03/22 00:19:30 UTC

[incubator-druid] branch 0.14.0-incubating updated: Fix exclusivity for start offset in kinesis indexing service & check exclusivity properly in IndexerSQLMetadataStorageCoordinator (#7291) (#7318)

This is an automated email from the ASF dual-hosted git repository.

jihoonson pushed a commit to branch 0.14.0-incubating
in repository https://gitbox.apache.org/repos/asf/incubator-druid.git


The following commit(s) were added to refs/heads/0.14.0-incubating by this push:
     new 61b59e5  Fix exclusivity for start offset in kinesis indexing service & check exclusivity properly in IndexerSQLMetadataStorageCoordinator (#7291) (#7318)
61b59e5 is described below

commit 61b59e547b407c9efd65f8279a5e188673669264
Author: Jihoon Son <ji...@apache.org>
AuthorDate: Thu Mar 21 17:19:23 2019 -0700

    Fix exclusivity for start offset in kinesis indexing service & check exclusivity properly in IndexerSQLMetadataStorageCoordinator (#7291) (#7318)
    
    * Fix exclusivity for start offset in kinesis indexing service
    
    * some adjustment
    
    * Fix SeekableStreamDataSourceMetadata
    
    * Add missing javadocs
    
    * Add missing comments and unit test
    
    * fix SeekableStreamStartSequenceNumbers.plus and add comments
    
    * remove extra exclusivePartitions in KafkaIOConfig and fix downgrade issue
    
    * Add javadocs
    
    * fix compilation
    
    * fix test
    
    * remove unused variable
---
 .../DerivativeDataSourceMetadata.java              |   6 +
 .../IncrementalPublishingKafkaIndexTaskRunner.java |  14 +-
 .../indexing/kafka/KafkaDataSourceMetadata.java    |  26 +-
 .../indexing/kafka/KafkaIndexTaskIOConfig.java     |  88 +-
 .../druid/indexing/kafka/KafkaSequenceNumber.java  |   2 +
 .../indexing/kafka/LegacyKafkaIndexTaskRunner.java |  56 +-
 .../indexing/kafka/supervisor/KafkaSupervisor.java |  18 +-
 .../kafka/KafkaDataSourceMetadataTest.java         | 142 +--
 .../druid/indexing/kafka/KafkaIOConfigTest.java    | 222 ++++-
 .../druid/indexing/kafka/KafkaIndexTaskTest.java   | 342 ++++---
 .../supervisor/KafkaSupervisorIOConfigTest.java    |   1 -
 .../kafka/supervisor/KafkaSupervisorTest.java      | 444 ++++++----
 .../kinesis/KinesisDataSourceMetadata.java         |  27 +-
 .../druid/indexing/kinesis/KinesisIndexTask.java   |   2 +-
 .../indexing/kinesis/KinesisIndexTaskIOConfig.java |  32 +-
 .../indexing/kinesis/KinesisIndexTaskRunner.java   |  16 +-
 .../indexing/kinesis/KinesisSequenceNumber.java    |  19 +-
 .../kinesis/supervisor/KinesisSupervisor.java      |  27 +-
 .../kinesis/KinesisDataSourceMetadataTest.java     | 204 +++--
 .../indexing/kinesis/KinesisIOConfigTest.java      |  59 +-
 .../indexing/kinesis/KinesisIndexTaskTest.java     | 535 +++--------
 .../kinesis/supervisor/KinesisSupervisorTest.java  | 984 ++++++++++-----------
 .../SeekableStreamDataSourceMetadata.java          |  76 +-
 .../SeekableStreamEndSequenceNumbers.java          | 221 +++++
 .../SeekableStreamIndexTaskIOConfig.java           |  44 +-
 .../SeekableStreamIndexTaskRunner.java             |  87 +-
 .../seekablestream/SeekableStreamPartitions.java   | 151 ----
 .../SeekableStreamSequenceNumbers.java             |  64 ++
 .../SeekableStreamStartSequenceNumbers.java        | 208 +++++
 .../indexing/seekablestream/SequenceMetadata.java  |  15 +-
 .../supervisor/SeekableStreamSupervisor.java       |  74 +-
 ...a => SeekableStreamEndSequenceNumbersTest.java} |  34 +-
 .../indexing/overlord/DataSourceMetadata.java      |   9 +
 .../druid/indexing/overlord/ObjectMetadata.java    |   6 +
 .../IndexerSQLMetadataStorageCoordinator.java      |  17 +-
 35 files changed, 2421 insertions(+), 1851 deletions(-)

diff --git a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/DerivativeDataSourceMetadata.java b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/DerivativeDataSourceMetadata.java
index 89d3c8c..0c38b2f 100644
--- a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/DerivativeDataSourceMetadata.java
+++ b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/DerivativeDataSourceMetadata.java
@@ -74,6 +74,12 @@ public class DerivativeDataSourceMetadata implements DataSourceMetadata
   }
 
   @Override
+  public DataSourceMetadata asStartMetadata()
+  {
+    return this;
+  }
+
+  @Override
   public boolean matches(DataSourceMetadata other) 
   {
     return equals(other);
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 9e38a97..bf0580cd 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
@@ -26,8 +26,9 @@ 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.SeekableStreamEndSequenceNumbers;
 import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner;
-import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
+import org.apache.druid.indexing.seekablestream.SeekableStreamSequenceNumbers;
 import org.apache.druid.indexing.seekablestream.SequenceMetadata;
 import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord;
 import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber;
@@ -112,14 +113,14 @@ public class IncrementalPublishingKafkaIndexTaskRunner extends SeekableStreamInd
   }
 
   @Override
-  protected SeekableStreamPartitions<Integer, Long> deserializePartitionsFromMetadata(
+  protected SeekableStreamEndSequenceNumbers<Integer, Long> deserializePartitionsFromMetadata(
       ObjectMapper mapper,
       Object object
   )
   {
     return mapper.convertValue(object, mapper.getTypeFactory().constructParametrizedType(
-        SeekableStreamPartitions.class,
-        SeekableStreamPartitions.class,
+        SeekableStreamEndSequenceNumbers.class,
+        SeekableStreamEndSequenceNumbers.class,
         Integer.class,
         Long.class
     ));
@@ -185,7 +186,7 @@ public class IncrementalPublishingKafkaIndexTaskRunner extends SeekableStreamInd
 
   @Override
   protected SeekableStreamDataSourceMetadata<Integer, Long> createDataSourceMetadata(
-      SeekableStreamPartitions<Integer, Long> partitions
+      SeekableStreamSequenceNumbers<Integer, Long> partitions
   )
   {
     return new KafkaDataSourceMetadata(partitions);
@@ -201,8 +202,7 @@ public class IncrementalPublishingKafkaIndexTaskRunner extends SeekableStreamInd
   protected void possiblyResetDataSourceMetadata(
       TaskToolbox toolbox,
       RecordSupplier<Integer, Long> recordSupplier,
-      Set<StreamPartition<Integer>> assignment,
-      Map<Integer, Long> currOffsets
+      Set<StreamPartition<Integer>> assignment
   )
   {
     // do nothing
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 55b03c0..81ea6de 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,28 +21,40 @@ package org.apache.druid.indexing.kafka;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.druid.indexing.overlord.DataSourceMetadata;
 import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata;
-import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
-
-import java.util.Map;
+import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers;
+import org.apache.druid.indexing.seekablestream.SeekableStreamSequenceNumbers;
 
 public class KafkaDataSourceMetadata extends SeekableStreamDataSourceMetadata<Integer, Long>
 {
 
   @JsonCreator
   public KafkaDataSourceMetadata(
-      @JsonProperty("partitions") SeekableStreamPartitions<Integer, Long> kafkaPartitions
+      @JsonProperty("partitions") SeekableStreamSequenceNumbers<Integer, Long> kafkaPartitions
   )
   {
     super(kafkaPartitions);
   }
 
   @Override
+  public DataSourceMetadata asStartMetadata()
+  {
+    final SeekableStreamSequenceNumbers<Integer, Long> sequenceNumbers = getSeekableStreamSequenceNumbers();
+    if (sequenceNumbers instanceof SeekableStreamEndSequenceNumbers) {
+      return createConcreteDataSourceMetaData(
+          ((SeekableStreamEndSequenceNumbers<Integer, Long>) sequenceNumbers).asStartPartitions(true)
+      );
+    } else {
+      return this;
+    }
+  }
+
+  @Override
   protected SeekableStreamDataSourceMetadata<Integer, Long> createConcreteDataSourceMetaData(
-      String streamId,
-      Map<Integer, Long> newMap
+      SeekableStreamSequenceNumbers<Integer, Long> seekableStreamSequenceNumbers
   )
   {
-    return new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(streamId, newMap));
+    return new KafkaDataSourceMetadata(seekableStreamSequenceNumbers);
   }
 }
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
index af84bfc..d42dadc 100644
--- 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
@@ -23,8 +23,9 @@ import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.google.common.base.Preconditions;
 import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig;
+import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers;
 import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig;
-import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
+import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers;
 import org.joda.time.DateTime;
 
 import javax.annotation.Nullable;
@@ -39,8 +40,14 @@ public class KafkaIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig<Inte
   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,
+      // startPartitions and endPartitions exist to be able to read old ioConfigs in metadata store
+      @JsonProperty("startPartitions") @Nullable SeekableStreamEndSequenceNumbers<Integer, Long> startPartitions,
+      @JsonProperty("endPartitions") @Nullable SeekableStreamEndSequenceNumbers<Integer, Long> endPartitions,
+      // startSequenceNumbers and endSequenceNumbers must be set for new versions
+      @JsonProperty("startSequenceNumbers")
+      @Nullable SeekableStreamStartSequenceNumbers<Integer, Long> startSequenceNumbers,
+      @JsonProperty("endSequenceNumbers")
+      @Nullable SeekableStreamEndSequenceNumbers<Integer, Long> endSequenceNumbers,
       @JsonProperty("consumerProperties") Map<String, Object> consumerProperties,
       @JsonProperty("pollTimeout") Long pollTimeout,
       @JsonProperty("useTransaction") Boolean useTransaction,
@@ -52,29 +59,86 @@ public class KafkaIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig<Inte
     super(
         taskGroupId,
         baseSequenceName,
-        startPartitions,
-        endPartitions,
+        startSequenceNumbers == null
+        ? Preconditions.checkNotNull(startPartitions, "startPartitions").asStartPartitions(true)
+        : startSequenceNumbers,
+        endSequenceNumbers == null ? endPartitions : endSequenceNumbers,
         useTransaction,
         minimumMessageTime,
         maximumMessageTime,
-        skipOffsetGaps,
-        null
+        skipOffsetGaps
     );
 
     this.consumerProperties = Preconditions.checkNotNull(consumerProperties, "consumerProperties");
     this.pollTimeout = pollTimeout != null ? pollTimeout : KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS;
 
-    for (int partition : endPartitions.getPartitionSequenceNumberMap().keySet()) {
+    final SeekableStreamEndSequenceNumbers<Integer, Long> myEndSequenceNumbers = getEndSequenceNumbers();
+    for (int partition : myEndSequenceNumbers.getPartitionSequenceNumberMap().keySet()) {
       Preconditions.checkArgument(
-          endPartitions.getPartitionSequenceNumberMap()
+          myEndSequenceNumbers.getPartitionSequenceNumberMap()
                        .get(partition)
-                       .compareTo(startPartitions.getPartitionSequenceNumberMap().get(partition)) >= 0,
+                       .compareTo(getStartSequenceNumbers().getPartitionSequenceNumberMap().get(partition)) >= 0,
           "end offset must be >= start offset for partition[%s]",
           partition
       );
     }
   }
 
+  public KafkaIndexTaskIOConfig(
+      int taskGroupId,
+      String baseSequenceName,
+      SeekableStreamStartSequenceNumbers<Integer, Long> startSequenceNumbers,
+      SeekableStreamEndSequenceNumbers<Integer, Long> endSequenceNumbers,
+      Map<String, Object> consumerProperties,
+      Long pollTimeout,
+      Boolean useTransaction,
+      DateTime minimumMessageTime,
+      DateTime maximumMessageTime,
+      Boolean skipOffsetGaps
+  )
+  {
+    this(
+        taskGroupId,
+        baseSequenceName,
+        null,
+        null,
+        startSequenceNumbers,
+        endSequenceNumbers,
+        consumerProperties,
+        pollTimeout,
+        useTransaction,
+        minimumMessageTime,
+        maximumMessageTime,
+        skipOffsetGaps
+    );
+  }
+
+  /**
+   * This method is for compatibilty so that newer version of KafkaIndexTaskIOConfig can be read by
+   * old version of Druid. Note that this method returns end sequence numbers instead of start. This is because
+   * {@link SeekableStreamStartSequenceNumbers} didn't exist before.
+   */
+  @JsonProperty
+  public SeekableStreamEndSequenceNumbers<Integer, Long> getStartPartitions()
+  {
+    // Converting to start sequence numbers. This is allowed for Kafka because the start offset is always inclusive.
+    final SeekableStreamStartSequenceNumbers<Integer, Long> startSequenceNumbers = getStartSequenceNumbers();
+    return new SeekableStreamEndSequenceNumbers<>(
+        startSequenceNumbers.getStream(),
+        startSequenceNumbers.getPartitionSequenceNumberMap()
+    );
+  }
+
+  /**
+   * This method is for compatibilty so that newer version of KafkaIndexTaskIOConfig can be read by
+   * old version of Druid.
+   */
+  @JsonProperty
+  public SeekableStreamEndSequenceNumbers<Integer, Long> getEndPartitions()
+  {
+    return getEndSequenceNumbers();
+  }
+
   @JsonProperty
   public Map<String, Object> getConsumerProperties()
   {
@@ -93,8 +157,8 @@ public class KafkaIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig<Inte
     return "KafkaIndexTaskIOConfig{" +
            "taskGroupId=" + getTaskGroupId() +
            ", baseSequenceName='" + getBaseSequenceName() + '\'' +
-           ", startPartitions=" + getStartPartitions() +
-           ", endPartitions=" + getEndPartitions() +
+           ", startSequenceNumbers=" + getStartSequenceNumbers() +
+           ", endSequenceNumbers=" + getEndSequenceNumbers() +
            ", consumerProperties=" + consumerProperties +
            ", pollTimeout=" + pollTimeout +
            ", useTransaction=" + isUseTransaction() +
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
index e903352..d727e5a 100644
--- 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
@@ -23,6 +23,8 @@ import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber;
 
 import javax.validation.constraints.NotNull;
 
+// OrderedSequenceNumber.equals() should be used instead.
+@SuppressWarnings("ComparableImplementedButEqualsNotOverridden")
 public class KafkaSequenceNumber extends OrderedSequenceNumber<Long>
 {
   private KafkaSequenceNumber(Long sequenceNumber)
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 528780d..cc58061 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
@@ -48,9 +48,11 @@ 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.seekablestream.SeekableStreamDataSourceMetadata;
+import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers;
 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.SeekableStreamSequenceNumbers;
+import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers;
 import org.apache.druid.indexing.seekablestream.SequenceMetadata;
 import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord;
 import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber;
@@ -210,7 +212,7 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner<In
     this.savedParseExceptions = savedParseExceptions;
     this.rowIngestionMeters = rowIngestionMetersFactory.createRowIngestionMeters();
 
-    this.endOffsets.putAll(ioConfig.getEndPartitions().getPartitionSequenceNumberMap());
+    this.endOffsets.putAll(ioConfig.getEndSequenceNumbers().getPartitionSequenceNumberMap());
     this.ingestionState = IngestionState.NOT_STARTED;
   }
 
@@ -294,19 +296,19 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner<In
 
       appenderator = appenderator0;
 
-      final String topic = ioConfig.getStartPartitions().getStream();
+      final String topic = ioConfig.getStartSequenceNumbers().getStream();
 
       // Start up, set up initial offsets.
       final Object restoredMetadata = driver.startJob();
       if (restoredMetadata == null) {
-        nextOffsets.putAll(ioConfig.getStartPartitions().getPartitionSequenceNumberMap());
+        nextOffsets.putAll(ioConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap());
       } else {
         final Map<String, Object> restoredMetadataMap = (Map) restoredMetadata;
-        final SeekableStreamPartitions<Integer, Long> restoredNextPartitions = toolbox.getObjectMapper().convertValue(
+        final SeekableStreamEndSequenceNumbers<Integer, Long> restoredNextPartitions = toolbox.getObjectMapper().convertValue(
             restoredMetadataMap.get(METADATA_NEXT_PARTITIONS),
             toolbox.getObjectMapper().getTypeFactory().constructParametrizedType(
-                SeekableStreamPartitions.class,
-                SeekableStreamPartitions.class,
+                SeekableStreamStartSequenceNumbers.class,
+                SeekableStreamStartSequenceNumbers.class,
                 Integer.class,
                 Long.class
             )
@@ -314,19 +316,19 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner<In
         nextOffsets.putAll(restoredNextPartitions.getPartitionSequenceNumberMap());
 
         // Sanity checks.
-        if (!restoredNextPartitions.getStream().equals(ioConfig.getStartPartitions().getStream())) {
+        if (!restoredNextPartitions.getStream().equals(ioConfig.getStartSequenceNumbers().getStream())) {
           throw new ISE(
               "WTF?! Restored topic[%s] but expected topic[%s]",
               restoredNextPartitions.getStream(),
-              ioConfig.getStartPartitions().getStream()
+              ioConfig.getStartSequenceNumbers().getStream()
           );
         }
 
-        if (!nextOffsets.keySet().equals(ioConfig.getStartPartitions().getPartitionSequenceNumberMap().keySet())) {
+        if (!nextOffsets.keySet().equals(ioConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().keySet())) {
           throw new ISE(
               "WTF?! Restored partitions[%s] but expected partitions[%s]",
               nextOffsets.keySet(),
-              ioConfig.getStartPartitions().getPartitionSequenceNumberMap().keySet()
+              ioConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().keySet()
           );
         }
       }
@@ -351,8 +353,9 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner<In
             public Object getMetadata()
             {
               return ImmutableMap.of(
-                  METADATA_NEXT_PARTITIONS, new SeekableStreamPartitions<>(
-                      ioConfig.getStartPartitions().getStream(),
+                  METADATA_NEXT_PARTITIONS,
+                  new SeekableStreamEndSequenceNumbers<>(
+                      ioConfig.getStartSequenceNumbers().getStream(),
                       snapshot
                   )
               );
@@ -518,13 +521,13 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner<In
       }
 
       final TransactionalSegmentPublisher publisher = (segments, commitMetadata) -> {
-        final SeekableStreamPartitions<Integer, Long> finalPartitions = toolbox.getObjectMapper().convertValue(
+        final SeekableStreamEndSequenceNumbers<Integer, Long> finalPartitions = toolbox.getObjectMapper().convertValue(
             ((Map) Preconditions.checkNotNull(commitMetadata, "commitMetadata")).get(METADATA_NEXT_PARTITIONS),
             toolbox.getObjectMapper()
                    .getTypeFactory()
                    .constructParametrizedType(
-                       SeekableStreamPartitions.class,
-                       SeekableStreamPartitions.class,
+                       SeekableStreamEndSequenceNumbers.class,
+                       SeekableStreamEndSequenceNumbers.class,
                        Integer.class,
                        Long.class
                    )
@@ -540,7 +543,7 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner<In
         if (ioConfig.isUseTransaction()) {
           action = new SegmentTransactionalInsertAction(
               segments,
-              new KafkaDataSourceMetadata(ioConfig.getStartPartitions()),
+              new KafkaDataSourceMetadata(ioConfig.getStartSequenceNumbers()),
               new KafkaDataSourceMetadata(finalPartitions)
           );
         } else {
@@ -715,8 +718,7 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner<In
   protected void possiblyResetDataSourceMetadata(
       TaskToolbox toolbox,
       RecordSupplier<Integer, Long> recordSupplier,
-      Set<StreamPartition<Integer>> assignment,
-      Map<Integer, Long> currOffsets
+      Set<StreamPartition<Integer>> assignment
   )
   {
     throw new UnsupportedOperationException();
@@ -729,7 +731,7 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner<In
   }
 
   @Override
-  protected SeekableStreamPartitions<Integer, Long> deserializePartitionsFromMetadata(
+  protected SeekableStreamEndSequenceNumbers<Integer, Long> deserializePartitionsFromMetadata(
       ObjectMapper mapper,
       Object object
   )
@@ -791,11 +793,13 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner<In
     boolean result = taskToolbox.getTaskActionClient()
                                 .submit(new ResetDataSourceMetadataAction(
                                     task.getDataSource(),
-                                    new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(
-                                        ioConfig.getStartPartitions()
-                                                .getStream(),
-                                        partitionOffsetMap
-                                    ))
+                                    new KafkaDataSourceMetadata(
+                                        new SeekableStreamStartSequenceNumbers<>(
+                                            ioConfig.getStartSequenceNumbers().getStream(),
+                                            partitionOffsetMap,
+                                            Collections.emptySet()
+                                        )
+                                    )
                                 ));
 
     if (result) {
@@ -1217,7 +1221,7 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner<In
 
   @Override
   protected SeekableStreamDataSourceMetadata<Integer, Long> createDataSourceMetadata(
-      SeekableStreamPartitions<Integer, Long> partitions
+      SeekableStreamSequenceNumbers<Integer, Long> partitions
   )
   {
     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 0e9dcac..86639fe 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
@@ -39,10 +39,11 @@ 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.SeekableStreamEndSequenceNumbers;
 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.SeekableStreamStartSequenceNumbers;
 import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber;
 import org.apache.druid.indexing.seekablestream.common.RecordSupplier;
 import org.apache.druid.indexing.seekablestream.common.StreamPartition;
@@ -59,6 +60,7 @@ import org.apache.druid.server.metrics.DruidMonitorSchedulerConfig;
 import org.joda.time.DateTime;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -206,8 +208,8 @@ public class KafkaSupervisor extends SeekableStreamSupervisor<Integer, Long>
     return new KafkaIndexTaskIOConfig(
         groupId,
         baseSequenceName,
-        new SeekableStreamPartitions<>(kafkaIoConfig.getTopic(), startPartitions),
-        new SeekableStreamPartitions<>(kafkaIoConfig.getTopic(), endPartitions),
+        new SeekableStreamStartSequenceNumbers<>(kafkaIoConfig.getTopic(), startPartitions, Collections.emptySet()),
+        new SeekableStreamEndSequenceNumbers<>(kafkaIoConfig.getTopic(), endPartitions),
         kafkaIoConfig.getConsumerProperties(),
         kafkaIoConfig.getPollTimeout(),
         true,
@@ -282,9 +284,9 @@ public class KafkaSupervisor extends SeekableStreamSupervisor<Integer, Long>
   }
 
   @Override
-  protected KafkaDataSourceMetadata createDataSourceMetaData(String topic, Map<Integer, Long> map)
+  protected KafkaDataSourceMetadata createDataSourceMetaDataForReset(String topic, Map<Integer, Long> map)
   {
-    return new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, map));
+    return new KafkaDataSourceMetadata(new SeekableStreamStartSequenceNumbers<>(topic, map, Collections.emptySet()));
   }
 
   @Override
@@ -357,6 +359,12 @@ public class KafkaSupervisor extends SeekableStreamSupervisor<Integer, Long>
   }
 
   @Override
+  protected boolean useExclusiveStartSequenceNumberForStartSequence()
+  {
+    return false;
+  }
+
+  @Override
   protected void updateLatestSequenceFromStream(
       RecordSupplier<Integer, Long> recordSupplier,
       Set<StreamPartition<Integer>> partitions
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 89f5ce8..1933618 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,7 +20,9 @@
 package org.apache.druid.indexing.kafka;
 
 import com.google.common.collect.ImmutableMap;
-import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
+import com.google.common.collect.ImmutableSet;
+import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers;
+import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -28,70 +30,95 @@ import java.util.Map;
 
 public class KafkaDataSourceMetadataTest
 {
-  private static final KafkaDataSourceMetadata KM0 = KM("foo", ImmutableMap.of());
-  private static final KafkaDataSourceMetadata KM1 = KM("foo", ImmutableMap.of(0, 2L, 1, 3L));
-  private static final KafkaDataSourceMetadata KM2 = KM("foo", ImmutableMap.of(0, 2L, 1, 4L, 2, 5L));
-  private static final KafkaDataSourceMetadata KM3 = KM("foo", ImmutableMap.of(0, 2L, 2, 5L));
+  private static final KafkaDataSourceMetadata START0 = startMetadata(ImmutableMap.of());
+  private static final KafkaDataSourceMetadata START1 = startMetadata(ImmutableMap.of(0, 2L, 1, 3L));
+  private static final KafkaDataSourceMetadata START2 = startMetadata(ImmutableMap.of(0, 2L, 1, 4L, 2, 5L));
+  private static final KafkaDataSourceMetadata START3 = startMetadata(ImmutableMap.of(0, 2L, 2, 5L));
+  private static final KafkaDataSourceMetadata END0 = endMetadata(ImmutableMap.of());
+  private static final KafkaDataSourceMetadata END1 = endMetadata(ImmutableMap.of(0, 2L, 2, 5L));
+  private static final KafkaDataSourceMetadata END2 = endMetadata(ImmutableMap.of(0, 2L, 1, 4L));
 
   @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));
+    Assert.assertTrue(START0.matches(START0));
+    Assert.assertTrue(START0.matches(START1));
+    Assert.assertTrue(START0.matches(START2));
+    Assert.assertTrue(START0.matches(START3));
+
+    Assert.assertTrue(START1.matches(START0));
+    Assert.assertTrue(START1.matches(START1));
+    Assert.assertFalse(START1.matches(START2));
+    Assert.assertTrue(START1.matches(START3));
+
+    Assert.assertTrue(START2.matches(START0));
+    Assert.assertFalse(START2.matches(START1));
+    Assert.assertTrue(START2.matches(START2));
+    Assert.assertTrue(START2.matches(START3));
+
+    Assert.assertTrue(START3.matches(START0));
+    Assert.assertTrue(START3.matches(START1));
+    Assert.assertTrue(START3.matches(START2));
+    Assert.assertTrue(START3.matches(START3));
+
+    Assert.assertTrue(END0.matches(END0));
+    Assert.assertTrue(END0.matches(END1));
+    Assert.assertTrue(END0.matches(END2));
+
+    Assert.assertTrue(END1.matches(END0));
+    Assert.assertTrue(END1.matches(END1));
+    Assert.assertTrue(END1.matches(END2));
+
+    Assert.assertTrue(END2.matches(END0));
+    Assert.assertTrue(END2.matches(END1));
+    Assert.assertTrue(END2.matches(END2));
   }
 
   @Test
   public void testIsValidStart()
   {
-    Assert.assertTrue(KM0.isValidStart());
-    Assert.assertTrue(KM1.isValidStart());
-    Assert.assertTrue(KM2.isValidStart());
-    Assert.assertTrue(KM3.isValidStart());
+    Assert.assertTrue(START0.isValidStart());
+    Assert.assertTrue(START1.isValidStart());
+    Assert.assertTrue(START2.isValidStart());
+    Assert.assertTrue(START3.isValidStart());
   }
 
   @Test
   public void testPlus()
   {
     Assert.assertEquals(
-        KM("foo", ImmutableMap.of(0, 2L, 1, 3L, 2, 5L)),
-        KM1.plus(KM3)
+        startMetadata(ImmutableMap.of(0, 2L, 1, 3L, 2, 5L)),
+        START1.plus(START3)
     );
 
     Assert.assertEquals(
-        KM("foo", ImmutableMap.of(0, 2L, 1, 4L, 2, 5L)),
-        KM0.plus(KM2)
+        startMetadata(ImmutableMap.of(0, 2L, 1, 4L, 2, 5L)),
+        START0.plus(START2)
     );
 
     Assert.assertEquals(
-        KM("foo", ImmutableMap.of(0, 2L, 1, 4L, 2, 5L)),
-        KM1.plus(KM2)
+        startMetadata(ImmutableMap.of(0, 2L, 1, 4L, 2, 5L)),
+        START1.plus(START2)
     );
 
     Assert.assertEquals(
-        KM("foo", ImmutableMap.of(0, 2L, 1, 3L, 2, 5L)),
-        KM2.plus(KM1)
+        startMetadata(ImmutableMap.of(0, 2L, 1, 3L, 2, 5L)),
+        START2.plus(START1)
     );
 
     Assert.assertEquals(
-        KM("foo", ImmutableMap.of(0, 2L, 1, 4L, 2, 5L)),
-        KM2.plus(KM2)
+        startMetadata(ImmutableMap.of(0, 2L, 1, 4L, 2, 5L)),
+        START2.plus(START2)
+    );
+
+    Assert.assertEquals(
+        endMetadata(ImmutableMap.of(0, 2L, 2, 5L)),
+        END0.plus(END1)
+    );
+
+    Assert.assertEquals(
+        endMetadata(ImmutableMap.of(0, 2L, 1, 4L, 2, 5L)),
+        END1.plus(END2)
     );
   }
 
@@ -99,33 +126,48 @@ public class KafkaDataSourceMetadataTest
   public void testMinus()
   {
     Assert.assertEquals(
-        KM("foo", ImmutableMap.of(1, 3L)),
-        KM1.minus(KM3)
+        startMetadata(ImmutableMap.of(1, 3L)),
+        START1.minus(START3)
     );
 
     Assert.assertEquals(
-        KM("foo", ImmutableMap.of()),
-        KM0.minus(KM2)
+        startMetadata(ImmutableMap.of()),
+        START0.minus(START2)
     );
 
     Assert.assertEquals(
-        KM("foo", ImmutableMap.of()),
-        KM1.minus(KM2)
+        startMetadata(ImmutableMap.of()),
+        START1.minus(START2)
     );
 
     Assert.assertEquals(
-        KM("foo", ImmutableMap.of(2, 5L)),
-        KM2.minus(KM1)
+        startMetadata(ImmutableMap.of(2, 5L)),
+        START2.minus(START1)
     );
 
     Assert.assertEquals(
-        KM("foo", ImmutableMap.of()),
-        KM2.minus(KM2)
+        startMetadata(ImmutableMap.of()),
+        START2.minus(START2)
     );
+
+    Assert.assertEquals(
+        endMetadata(ImmutableMap.of(1, 4L)),
+        END2.minus(END1)
+    );
+
+    Assert.assertEquals(
+        endMetadata(ImmutableMap.of(2, 5L)),
+        END1.minus(END2)
+    );
+  }
+
+  private static KafkaDataSourceMetadata startMetadata(Map<Integer, Long> offsets)
+  {
+    return new KafkaDataSourceMetadata(new SeekableStreamStartSequenceNumbers<>("foo", offsets, ImmutableSet.of()));
   }
 
-  private static KafkaDataSourceMetadata KM(String topic, Map<Integer, Long> offsets)
+  private static KafkaDataSourceMetadata endMetadata(Map<Integer, Long> offsets)
   {
-    return new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, offsets));
+    return new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>("foo", 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 7ce8df0..0f4fc05 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
@@ -19,20 +19,30 @@
 
 package org.apache.druid.indexing.kafka;
 
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.databind.JsonMappingException;
 import com.fasterxml.jackson.databind.Module;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.jsontype.NamedType;
+import com.google.common.base.Optional;
 import com.google.common.collect.ImmutableMap;
+import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers;
+import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers;
 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.joda.time.DateTime;
 import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
 
+import javax.annotation.Nullable;
+import java.io.IOException;
 import java.util.Collections;
+import java.util.Map;
 
 public class KafkaIOConfigTest
 {
@@ -69,16 +79,49 @@ public class KafkaIOConfigTest
     );
 
     Assert.assertEquals("my-sequence-name", config.getBaseSequenceName());
-    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("mytopic", config.getStartSequenceNumbers().getStream());
+    Assert.assertEquals(ImmutableMap.of(0, 1L, 1, 10L), config.getStartSequenceNumbers().getPartitionSequenceNumberMap());
+    Assert.assertEquals("mytopic", config.getEndSequenceNumbers().getStream());
+    Assert.assertEquals(ImmutableMap.of(0, 15L, 1, 200L), config.getEndSequenceNumbers().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.assertEquals(Collections.emptySet(), config.getStartSequenceNumbers().getExclusivePartitions());
+  }
+
+  @Test
+  public void testSerdeWithDefaultsAndSequenceNumbers() throws Exception
+  {
+    String jsonStr = "{\n"
+                     + "  \"type\": \"kafka\",\n"
+                     + "  \"taskGroupId\": 0,\n"
+                     + "  \"baseSequenceName\": \"my-sequence-name\",\n"
+                     + "  \"startSequenceNumbers\": {\"type\":\"start\", \"stream\":\"mytopic\", \"partitionSequenceNumberMap\" : {\"0\":1, \"1\":10}},\n"
+                     + "  \"endSequenceNumbers\": {\"type\":\"end\", \"stream\":\"mytopic\", \"partitionSequenceNumberMap\" : {\"0\":15, \"1\":200}},\n"
+                     + "  \"consumerProperties\": {\"bootstrap.servers\":\"localhost:9092\"}\n"
+                     + "}";
+
+    KafkaIndexTaskIOConfig config = (KafkaIndexTaskIOConfig) mapper.readValue(
+        mapper.writeValueAsString(
+            mapper.readValue(
+                jsonStr,
+                IOConfig.class
+            )
+        ), IOConfig.class
+    );
+
+    Assert.assertEquals("my-sequence-name", config.getBaseSequenceName());
+    Assert.assertEquals("mytopic", config.getStartSequenceNumbers().getStream());
+    Assert.assertEquals(ImmutableMap.of(0, 1L, 1, 10L), config.getStartSequenceNumbers().getPartitionSequenceNumberMap());
+    Assert.assertEquals("mytopic", config.getEndSequenceNumbers().getStream());
+    Assert.assertEquals(ImmutableMap.of(0, 15L, 1, 200L), config.getEndSequenceNumbers().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());
+    Assert.assertEquals(Collections.emptySet(), config.getStartSequenceNumbers().getExclusivePartitions());
   }
 
   @Test
@@ -107,17 +150,16 @@ public class KafkaIOConfigTest
     );
 
     Assert.assertEquals("my-sequence-name", config.getBaseSequenceName());
-    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("mytopic", config.getStartSequenceNumbers().getStream());
+    Assert.assertEquals(ImmutableMap.of(0, 1L, 1, 10L), config.getStartSequenceNumbers().getPartitionSequenceNumberMap());
+    Assert.assertEquals("mytopic", config.getEndSequenceNumbers().getStream());
+    Assert.assertEquals(ImmutableMap.of(0, 15L, 1, 200L), config.getEndSequenceNumbers().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());
-
+    Assert.assertEquals(Collections.emptySet(), config.getStartSequenceNumbers().getExclusivePartitions());
   }
 
   @Test
@@ -176,7 +218,7 @@ public class KafkaIOConfigTest
 
     exception.expect(JsonMappingException.class);
     exception.expectCause(CoreMatchers.isA(NullPointerException.class));
-    exception.expectMessage(CoreMatchers.containsString("endPartitions"));
+    exception.expectMessage(CoreMatchers.containsString("endSequenceNumbers"));
     mapper.readValue(jsonStr, IOConfig.class);
   }
 
@@ -262,4 +304,160 @@ public class KafkaIOConfigTest
     exception.expectMessage(CoreMatchers.containsString("end offset must be >= start offset"));
     mapper.readValue(jsonStr, IOConfig.class);
   }
+
+  @Test
+  public void testDeserializeToOldIoConfig() throws IOException
+  {
+    final KafkaIndexTaskIOConfig currentConfig = new KafkaIndexTaskIOConfig(
+        0,
+        "baseSequenceNamee",
+        null,
+        null,
+        new SeekableStreamStartSequenceNumbers<>("stream", ImmutableMap.of(1, 10L, 2, 5L), null),
+        new SeekableStreamEndSequenceNumbers<>("stream", ImmutableMap.of(1, 20L, 2, 30L)),
+        ImmutableMap.of("consumer", "properties"),
+        100L,
+        true,
+        DateTimes.nowUtc(),
+        DateTimes.nowUtc(),
+        true
+    );
+    final byte[] json = mapper.writeValueAsBytes(currentConfig);
+
+    final ObjectMapper oldMapper = new DefaultObjectMapper();
+    oldMapper.registerSubtypes(new NamedType(OldKafkaIndexTaskIoConfig.class, "kafka"));
+    final OldKafkaIndexTaskIoConfig oldConfig = (OldKafkaIndexTaskIoConfig) oldMapper.readValue(json, IOConfig.class);
+
+    Assert.assertEquals(currentConfig.getTaskGroupId().intValue(), oldConfig.taskGroupId);
+    Assert.assertEquals(currentConfig.getBaseSequenceName(), oldConfig.baseSequenceName);
+    Assert.assertEquals(currentConfig.getStartSequenceNumbers(), oldConfig.startPartitions.asStartPartitions(true));
+    Assert.assertEquals(currentConfig.getEndSequenceNumbers(), oldConfig.getEndPartitions());
+    Assert.assertEquals(currentConfig.getConsumerProperties(), oldConfig.getConsumerProperties());
+    Assert.assertEquals(currentConfig.getPollTimeout(), oldConfig.getPollTimeout());
+    Assert.assertEquals(currentConfig.isUseTransaction(), oldConfig.isUseTransaction());
+    Assert.assertEquals(currentConfig.getMinimumMessageTime(), oldConfig.getMinimumMessageTime());
+    Assert.assertEquals(currentConfig.getMaximumMessageTime(), oldConfig.getMaximumMessageTime());
+  }
+
+  @Test
+  public void testDeserializeFromOldIoConfig() throws IOException
+  {
+    final ObjectMapper oldMapper = new DefaultObjectMapper();
+    oldMapper.registerSubtypes(new NamedType(OldKafkaIndexTaskIoConfig.class, "kafka"));
+
+    final OldKafkaIndexTaskIoConfig oldConfig = new OldKafkaIndexTaskIoConfig(
+        0,
+        "baseSequenceNamee",
+        new SeekableStreamEndSequenceNumbers<>("stream", ImmutableMap.of(1, 10L, 2, 5L)),
+        new SeekableStreamEndSequenceNumbers<>("stream", ImmutableMap.of(1, 20L, 2, 30L)),
+        ImmutableMap.of("consumer", "properties"),
+        100L,
+        true,
+        DateTimes.nowUtc(),
+        DateTimes.nowUtc()
+    );
+    final byte[] json = oldMapper.writeValueAsBytes(oldConfig);
+
+    final KafkaIndexTaskIOConfig currentConfig = (KafkaIndexTaskIOConfig) mapper.readValue(json, IOConfig.class);
+    Assert.assertEquals(oldConfig.getTaskGroupId(), currentConfig.getTaskGroupId().intValue());
+    Assert.assertEquals(oldConfig.getBaseSequenceName(), currentConfig.getBaseSequenceName());
+    Assert.assertEquals(oldConfig.getStartPartitions().asStartPartitions(true), currentConfig.getStartSequenceNumbers());
+    Assert.assertEquals(oldConfig.getEndPartitions(), currentConfig.getEndSequenceNumbers());
+    Assert.assertEquals(oldConfig.getConsumerProperties(), currentConfig.getConsumerProperties());
+    Assert.assertEquals(oldConfig.getPollTimeout(), currentConfig.getPollTimeout());
+    Assert.assertEquals(oldConfig.isUseTransaction(), currentConfig.isUseTransaction());
+    Assert.assertEquals(oldConfig.getMinimumMessageTime(), currentConfig.getMinimumMessageTime());
+    Assert.assertEquals(oldConfig.getMaximumMessageTime(), currentConfig.getMaximumMessageTime());
+  }
+
+  private static class OldKafkaIndexTaskIoConfig implements IOConfig
+  {
+    private final int taskGroupId;
+    private final String baseSequenceName;
+    private final SeekableStreamEndSequenceNumbers<Integer, Long> startPartitions;
+    private final SeekableStreamEndSequenceNumbers<Integer, Long> endPartitions;
+    private final Map<String, Object> consumerProperties;
+    private final long pollTimeout;
+    private final boolean useTransaction;
+    private final Optional<DateTime> minimumMessageTime;
+    private final Optional<DateTime> maximumMessageTime;
+
+    @JsonCreator
+    private OldKafkaIndexTaskIoConfig(
+        @JsonProperty("taskGroupId") int taskGroupId,
+        @JsonProperty("baseSequenceName") String baseSequenceName,
+        @JsonProperty("startPartitions") @Nullable SeekableStreamEndSequenceNumbers<Integer, Long> startPartitions,
+        @JsonProperty("endPartitions") @Nullable SeekableStreamEndSequenceNumbers<Integer, Long> endPartitions,
+        @JsonProperty("consumerProperties") Map<String, Object> consumerProperties,
+        @JsonProperty("pollTimeout") Long pollTimeout,
+        @JsonProperty("useTransaction") Boolean useTransaction,
+        @JsonProperty("minimumMessageTime") DateTime minimumMessageTime,
+        @JsonProperty("maximumMessageTime") DateTime maximumMessageTime
+    )
+    {
+      this.taskGroupId = taskGroupId;
+      this.baseSequenceName = baseSequenceName;
+      this.startPartitions = startPartitions;
+      this.endPartitions = endPartitions;
+      this.consumerProperties = consumerProperties;
+      this.pollTimeout = pollTimeout;
+      this.useTransaction = useTransaction;
+      this.minimumMessageTime = Optional.fromNullable(minimumMessageTime);
+      this.maximumMessageTime = Optional.fromNullable(maximumMessageTime);
+    }
+
+    @JsonProperty
+    public int getTaskGroupId()
+    {
+      return taskGroupId;
+    }
+
+    @JsonProperty
+    public String getBaseSequenceName()
+    {
+      return baseSequenceName;
+    }
+
+    @JsonProperty
+    public SeekableStreamEndSequenceNumbers<Integer, Long> getStartPartitions()
+    {
+      return startPartitions;
+    }
+
+    @JsonProperty
+    public SeekableStreamEndSequenceNumbers<Integer, Long> getEndPartitions()
+    {
+      return endPartitions;
+    }
+
+    @JsonProperty
+    public Map<String, Object> getConsumerProperties()
+    {
+      return consumerProperties;
+    }
+
+    @JsonProperty
+    public long getPollTimeout()
+    {
+      return pollTimeout;
+    }
+
+    @JsonProperty
+    public boolean isUseTransaction()
+    {
+      return useTransaction;
+    }
+
+    @JsonProperty
+    public Optional<DateTime> getMinimumMessageTime()
+    {
+      return minimumMessageTime;
+    }
+
+    @JsonProperty
+    public Optional<DateTime> getMaximumMessageTime()
+    {
+      return maximumMessageTime;
+    }
+  }
 }
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 a52dd80..5a3f7dd 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
@@ -76,8 +76,9 @@ 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.SeekableStreamEndSequenceNumbers;
 import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner.Status;
-import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
+import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers;
 import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor;
 import org.apache.druid.indexing.test.TestDataSegmentAnnouncer;
 import org.apache.druid.indexing.test.TestDataSegmentKiller;
@@ -402,8 +403,8 @@ public class KafkaIndexTaskTest
         new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)),
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)),
+            new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)),
             kafkaServer.consumerProperties(),
             KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
             true,
@@ -428,7 +429,7 @@ public class KafkaIndexTaskTest
     SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
     Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
     Assert.assertEquals(
-        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
+        new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -445,8 +446,8 @@ public class KafkaIndexTaskTest
         new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)),
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)),
+            new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)),
             kafkaServer.consumerProperties(),
             KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
             true,
@@ -483,7 +484,7 @@ public class KafkaIndexTaskTest
     SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
     Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
     Assert.assertEquals(
-        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
+        new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -511,43 +512,24 @@ public class KafkaIndexTaskTest
     Map<String, Object> consumerProps = kafkaServer.consumerProperties();
     consumerProps.put("max.poll.records", "1");
 
-    final SeekableStreamPartitions<Integer, Long> startPartitions = new SeekableStreamPartitions<>(
+    final SeekableStreamStartSequenceNumbers<Integer, Long> startPartitions = new SeekableStreamStartSequenceNumbers<>(
         topic,
-        ImmutableMap.of(
-            0,
-            0L,
-            1,
-            0L
-        )
+        ImmutableMap.of(0, 0L, 1, 0L),
+        ImmutableSet.of()
     );
     // Checkpointing will happen at either checkpoint1 or checkpoint2 depending on ordering
     // of events fetched across two partitions from Kafka
-    final SeekableStreamPartitions<Integer, Long> checkpoint1 = new SeekableStreamPartitions<>(
+    final SeekableStreamEndSequenceNumbers<Integer, Long> checkpoint1 = new SeekableStreamEndSequenceNumbers<>(
         topic,
-        ImmutableMap.of(
-            0,
-            5L,
-            1,
-            0L
-        )
+        ImmutableMap.of(0, 5L, 1, 0L)
     );
-    final SeekableStreamPartitions<Integer, Long> checkpoint2 = new SeekableStreamPartitions<>(
+    final SeekableStreamEndSequenceNumbers<Integer, Long> checkpoint2 = new SeekableStreamEndSequenceNumbers<>(
         topic,
-        ImmutableMap.of(
-            0,
-            4L,
-            1,
-            2L
-        )
+        ImmutableMap.of(0, 4L, 1, 2L)
     );
-    final SeekableStreamPartitions<Integer, Long> endPartitions = new SeekableStreamPartitions<>(
+    final SeekableStreamEndSequenceNumbers<Integer, Long> endPartitions = new SeekableStreamEndSequenceNumbers<>(
         topic,
-        ImmutableMap.of(
-            0,
-            10L,
-            1,
-            2L
-        )
+        ImmutableMap.of(0, 10L, 1, 2L)
     );
     final KafkaIndexTask task = createTask(
         null,
@@ -582,7 +564,9 @@ public class KafkaIndexTaskTest
                 DATA_SCHEMA.getDataSource(),
                 0,
                 new KafkaDataSourceMetadata(startPartitions),
-                new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, currentOffsets))
+                new KafkaDataSourceMetadata(
+                    new SeekableStreamEndSequenceNumbers<>(topic, currentOffsets)
+                )
             )
         )
     );
@@ -602,7 +586,9 @@ public class KafkaIndexTaskTest
     SegmentDescriptor desc7 = SD(task, "2013/P1D", 0);
     Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors());
     Assert.assertEquals(
-        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 2L))),
+        new KafkaDataSourceMetadata(
+            new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L, 1, 2L))
+        ),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -640,42 +626,23 @@ public class KafkaIndexTaskTest
       Map<String, Object> consumerProps = kafkaServer.consumerProperties();
       consumerProps.put("max.poll.records", "1");
 
-      final SeekableStreamPartitions<Integer, Long> startPartitions = new SeekableStreamPartitions<>(
+      final SeekableStreamStartSequenceNumbers<Integer, Long> startPartitions = new SeekableStreamStartSequenceNumbers<>(
           topic,
-          ImmutableMap.of(
-              0,
-              0L,
-              1,
-              0L
-          )
+          ImmutableMap.of(0, 0L, 1, 0L),
+          ImmutableSet.of()
       );
-      final SeekableStreamPartitions<Integer, Long> checkpoint1 = new SeekableStreamPartitions<>(
+      final SeekableStreamEndSequenceNumbers<Integer, Long> checkpoint1 = new SeekableStreamEndSequenceNumbers<>(
           topic,
-          ImmutableMap.of(
-              0,
-              3L,
-              1,
-              0L
-          )
+          ImmutableMap.of(0, 3L, 1, 0L)
       );
-      final SeekableStreamPartitions<Integer, Long> checkpoint2 = new SeekableStreamPartitions<>(
+      final SeekableStreamEndSequenceNumbers<Integer, Long> checkpoint2 = new SeekableStreamEndSequenceNumbers<>(
           topic,
-          ImmutableMap.of(
-              0,
-              10L,
-              1,
-              0L
-          )
+          ImmutableMap.of(0, 10L, 1, 0L)
       );
 
-      final SeekableStreamPartitions<Integer, Long> endPartitions = new SeekableStreamPartitions<>(
+      final SeekableStreamEndSequenceNumbers<Integer, Long> endPartitions = new SeekableStreamEndSequenceNumbers<>(
           topic,
-          ImmutableMap.of(
-              0,
-              10L,
-              1,
-              2L
-          )
+          ImmutableMap.of(0, 10L, 1, 2L)
       );
       final KafkaIndexTask task = createTask(
           null,
@@ -692,13 +659,14 @@ public class KafkaIndexTaskTest
               false
           )
       );
+
       final ListenableFuture<TaskStatus> future = runTask(task);
       while (task.getRunner().getStatus() != Status.PAUSED) {
         Thread.sleep(10);
       }
       final Map<Integer, Long> currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets());
 
-      Assert.assertTrue(checkpoint1.getPartitionSequenceNumberMap().equals(currentOffsets));
+      Assert.assertEquals(checkpoint1.getPartitionSequenceNumberMap(), currentOffsets);
       task.getRunner().setEndOffsets(currentOffsets, false);
 
       while (task.getRunner().getStatus() != Status.PAUSED) {
@@ -723,7 +691,9 @@ public class KafkaIndexTaskTest
                   DATA_SCHEMA.getDataSource(),
                   0,
                   new KafkaDataSourceMetadata(startPartitions),
-                  new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, currentOffsets))
+                  new KafkaDataSourceMetadata(
+                      new SeekableStreamEndSequenceNumbers<>(topic, currentOffsets)
+                  )
               )
           )
       );
@@ -732,8 +702,10 @@ public class KafkaIndexTaskTest
               Objects.hash(
                   DATA_SCHEMA.getDataSource(),
                   0,
-                  new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, currentOffsets)),
-                  new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, nextOffsets))
+                  new KafkaDataSourceMetadata(
+                      new SeekableStreamStartSequenceNumbers<>(topic, currentOffsets, ImmutableSet.of())
+                  ),
+                  new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, nextOffsets))
               )
           )
       );
@@ -753,7 +725,16 @@ public class KafkaIndexTaskTest
       SegmentDescriptor desc7 = SD(task, "2013/P1D", 0);
       Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors());
       Assert.assertEquals(
-          new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 2L))),
+          new KafkaDataSourceMetadata(
+              new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L, 1, 2L))
+          ),
+          metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()));
+
+      Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors());
+      Assert.assertEquals(
+          new KafkaDataSourceMetadata(
+              new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L, 1, 2L))
+          ),
           metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
       );
 
@@ -790,33 +771,19 @@ public class KafkaIndexTaskTest
     Map<String, Object> consumerProps = kafkaServer.consumerProperties();
     consumerProps.put("max.poll.records", "1");
 
-    final SeekableStreamPartitions<Integer, Long> startPartitions = new SeekableStreamPartitions<>(
+    final SeekableStreamStartSequenceNumbers<Integer, Long> startPartitions = new SeekableStreamStartSequenceNumbers<>(
         topic,
-        ImmutableMap.of(
-            0,
-            0L,
-            1,
-            0L
-        )
+        ImmutableMap.of(0, 0L, 1, 0L),
+        ImmutableSet.of()
     );
     // Checkpointing will happen at checkpoint
-    final SeekableStreamPartitions<Integer, Long> checkpoint = new SeekableStreamPartitions<>(
+    final SeekableStreamEndSequenceNumbers<Integer, Long> checkpoint = new SeekableStreamEndSequenceNumbers<>(
         topic,
-        ImmutableMap.of(
-            0,
-            1L,
-            1,
-            0L
-        )
+        ImmutableMap.of(0, 1L, 1, 0L)
     );
-    final SeekableStreamPartitions<Integer, Long> endPartitions = new SeekableStreamPartitions<>(
+    final SeekableStreamEndSequenceNumbers<Integer, Long> endPartitions = new SeekableStreamEndSequenceNumbers<>(
         topic,
-        ImmutableMap.of(
-            0,
-            2L,
-            1,
-            0L
-        )
+        ImmutableMap.of(0, 2L, 1, 0L)
     );
     final KafkaIndexTask task = createTask(
         null,
@@ -851,10 +818,9 @@ public class KafkaIndexTaskTest
                 DATA_SCHEMA.getDataSource(),
                 0,
                 new KafkaDataSourceMetadata(startPartitions),
-                new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(
-                    topic,
-                    checkpoint.getPartitionSequenceNumberMap()
-                ))
+                new KafkaDataSourceMetadata(
+                    new SeekableStreamEndSequenceNumbers<>(topic, checkpoint.getPartitionSequenceNumberMap())
+                )
             )
         )
     );
@@ -869,7 +835,9 @@ public class KafkaIndexTaskTest
     SegmentDescriptor desc2 = SD(task, "2009/P1D", 0);
     Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
     Assert.assertEquals(
-        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L, 1, 0L))),
+        new KafkaDataSourceMetadata(
+            new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 2L, 1, 0L))
+        ),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -899,14 +867,14 @@ public class KafkaIndexTaskTest
     Map<String, Object> consumerProps = kafkaServer.consumerProperties();
     consumerProps.put("max.poll.records", "1");
 
-    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> checkpoint2 =
-        new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 9L));
-    final SeekableStreamPartitions<Integer, Long> endPartitions =
-        new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, Long.MAX_VALUE));
+    final SeekableStreamStartSequenceNumbers<Integer, Long> startPartitions =
+        new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of());
+    final SeekableStreamEndSequenceNumbers<Integer, Long> checkpoint1 =
+        new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L));
+    final SeekableStreamEndSequenceNumbers<Integer, Long> checkpoint2 =
+        new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 9L));
+    final SeekableStreamEndSequenceNumbers<Integer, Long> endPartitions =
+        new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, Long.MAX_VALUE));
 
     final KafkaIndexTask normalReplica = createTask(
         null,
@@ -996,8 +964,8 @@ public class KafkaIndexTaskTest
         new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L)),
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)),
+            new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)),
             kafkaServer.consumerProperties(),
             KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
             true,
@@ -1034,7 +1002,7 @@ public class KafkaIndexTaskTest
     SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
     Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
     Assert.assertEquals(
-        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
+        new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -1051,8 +1019,8 @@ public class KafkaIndexTaskTest
         new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L)),
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)),
+            new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)),
             kafkaServer.consumerProperties(),
             KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
             true,
@@ -1090,7 +1058,7 @@ public class KafkaIndexTaskTest
     SegmentDescriptor desc3 = SD(task, "2010/P1D", 0);
     Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors());
     Assert.assertEquals(
-        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
+        new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -1116,8 +1084,8 @@ public class KafkaIndexTaskTest
         new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L)),
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)),
+            new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)),
             kafkaServer.consumerProperties(),
             KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
             true,
@@ -1153,7 +1121,7 @@ public class KafkaIndexTaskTest
     SegmentDescriptor desc1 = SD(task, "2009/P1D", 0);
     Assert.assertEquals(ImmutableSet.of(desc1), publishedDescriptors());
     Assert.assertEquals(
-        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
+        new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -1177,8 +1145,8 @@ public class KafkaIndexTaskTest
         new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)),
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)),
+            new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 2L)),
             kafkaServer.consumerProperties(),
             KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
             true,
@@ -1219,8 +1187,8 @@ public class KafkaIndexTaskTest
         new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)),
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)),
+            new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)),
             kafkaServer.consumerProperties(),
             KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
             true,
@@ -1245,7 +1213,7 @@ public class KafkaIndexTaskTest
     SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
     Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
     Assert.assertEquals(
-        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
+        new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -1272,8 +1240,8 @@ public class KafkaIndexTaskTest
         new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)),
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)),
+            new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)),
             kafkaServer.consumerProperties(),
             KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
             true,
@@ -1298,7 +1266,9 @@ public class KafkaIndexTaskTest
     SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
     Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
     Assert.assertEquals(
-        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
+        new KafkaDataSourceMetadata(
+            new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))
+        ),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -1328,8 +1298,8 @@ public class KafkaIndexTaskTest
         new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)),
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 7L)),
+            new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 7L)),
             kafkaServer.consumerProperties(),
             KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
             true,
@@ -1373,8 +1343,8 @@ public class KafkaIndexTaskTest
         new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)),
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 13L)),
+            new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 13L)),
             kafkaServer.consumerProperties(),
             KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
             true,
@@ -1405,7 +1375,7 @@ public class KafkaIndexTaskTest
     SegmentDescriptor desc4 = SD(task, "2049/P1D", 0);
     Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors());
     Assert.assertEquals(
-        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 13L))),
+        new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 13L))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -1456,8 +1426,8 @@ public class KafkaIndexTaskTest
         new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)),
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L)),
+            new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L)),
             kafkaServer.consumerProperties(),
             KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
             true,
@@ -1517,8 +1487,8 @@ public class KafkaIndexTaskTest
         new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)),
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)),
+            new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)),
             kafkaServer.consumerProperties(),
             KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
             true,
@@ -1532,8 +1502,8 @@ public class KafkaIndexTaskTest
         new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)),
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)),
+            new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)),
             kafkaServer.consumerProperties(),
             KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
             true,
@@ -1570,7 +1540,7 @@ public class KafkaIndexTaskTest
     SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0);
     Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
     Assert.assertEquals(
-        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
+        new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -1587,8 +1557,8 @@ public class KafkaIndexTaskTest
         new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)),
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)),
+            new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)),
             kafkaServer.consumerProperties(),
             KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
             true,
@@ -1602,8 +1572,8 @@ public class KafkaIndexTaskTest
         new KafkaIndexTaskIOConfig(
             1,
             "sequence1",
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 3L)),
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L)),
+            new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 3L), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L)),
             kafkaServer.consumerProperties(),
             KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
             true,
@@ -1641,7 +1611,7 @@ public class KafkaIndexTaskTest
     SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0);
     Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
     Assert.assertEquals(
-        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
+        new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -1658,8 +1628,8 @@ public class KafkaIndexTaskTest
         new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)),
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)),
+            new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)),
             kafkaServer.consumerProperties(),
             KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
             false,
@@ -1673,8 +1643,8 @@ public class KafkaIndexTaskTest
         new KafkaIndexTaskIOConfig(
             1,
             "sequence1",
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 3L)),
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L)),
+            new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 3L), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L)),
             kafkaServer.consumerProperties(),
             KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
             false,
@@ -1734,8 +1704,8 @@ public class KafkaIndexTaskTest
         new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L, 1, 0L)),
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L, 1, 2L)),
+            new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L, 1, 0L), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L, 1, 2L)),
             kafkaServer.consumerProperties(),
             KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
             true,
@@ -1773,7 +1743,9 @@ public class KafkaIndexTaskTest
                         ? ImmutableSet.of(desc1, desc2, desc4)
                         : ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors());
     Assert.assertEquals(
-        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L, 1, 2L))),
+        new KafkaDataSourceMetadata(
+            new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L, 1, 2L))
+        ),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -1800,8 +1772,8 @@ public class KafkaIndexTaskTest
         new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)),
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)),
+            new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)),
             kafkaServer.consumerProperties(),
             KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
             true,
@@ -1815,8 +1787,8 @@ public class KafkaIndexTaskTest
         new KafkaIndexTaskIOConfig(
             1,
             "sequence1",
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(1, 0L)),
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(1, 1L)),
+            new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(1, 0L), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(1, 1L)),
             kafkaServer.consumerProperties(),
             KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
             true,
@@ -1854,7 +1826,9 @@ public class KafkaIndexTaskTest
     SegmentDescriptor desc3 = SD(task2, "2012/P1D", 0);
     Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors());
     Assert.assertEquals(
-        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L, 1, 1L))),
+        new KafkaDataSourceMetadata(
+            new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L, 1, 1L))
+        ),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -1872,8 +1846,8 @@ public class KafkaIndexTaskTest
         new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)),
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)),
+            new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)),
             kafkaServer.consumerProperties(),
             KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
             true,
@@ -1910,8 +1884,8 @@ public class KafkaIndexTaskTest
         new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)),
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)),
+            new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)),
             kafkaServer.consumerProperties(),
             KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
             true,
@@ -1946,7 +1920,7 @@ public class KafkaIndexTaskTest
     SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0);
     Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
     Assert.assertEquals(
-        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
+        new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -1972,8 +1946,8 @@ public class KafkaIndexTaskTest
         new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L)),
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 9L)),
+            new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 9L)),
             consumerProps,
             KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
             true,
@@ -1983,9 +1957,10 @@ public class KafkaIndexTaskTest
         )
     );
 
-    final SeekableStreamPartitions<Integer, Long> checkpoint = new SeekableStreamPartitions<>(
+    final SeekableStreamStartSequenceNumbers<Integer, Long> checkpoint = new SeekableStreamStartSequenceNumbers<>(
         topic,
-        ImmutableMap.of(0, 5L)
+        ImmutableMap.of(0, 5L),
+        ImmutableSet.of()
     );
 
     final ListenableFuture<TaskStatus> future1 = runTask(task1);
@@ -2017,8 +1992,8 @@ public class KafkaIndexTaskTest
         new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L)),
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 9L)),
+            new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 9L)),
             consumerProps,
             KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
             true,
@@ -2059,7 +2034,7 @@ public class KafkaIndexTaskTest
     SegmentDescriptor desc7 = SD(task1, "2012/P1D", 0);
     Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors());
     Assert.assertEquals(
-        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 9L))),
+        new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 9L))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
   }
@@ -2072,8 +2047,8 @@ public class KafkaIndexTaskTest
         new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)),
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)),
+            new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)),
             kafkaServer.consumerProperties(),
             KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
             true,
@@ -2140,7 +2115,7 @@ public class KafkaIndexTaskTest
     SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
     Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
     Assert.assertEquals(
-        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
+        new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -2157,8 +2132,8 @@ public class KafkaIndexTaskTest
         new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)),
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)),
+            new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)),
             kafkaServer.consumerProperties(),
             KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
             true,
@@ -2197,8 +2172,8 @@ public class KafkaIndexTaskTest
         new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 200L)),
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 500L)),
+            new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 200L), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 500L)),
             kafkaServer.consumerProperties(),
             KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
             true,
@@ -2252,8 +2227,8 @@ public class KafkaIndexTaskTest
             0,
             "sequence0",
             // task should ignore these and use sequence info sent in the context
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L)),
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)),
+            new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)),
             kafkaServer.consumerProperties(),
             KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
             true,
@@ -2279,7 +2254,7 @@ public class KafkaIndexTaskTest
     SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
     Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
     Assert.assertEquals(
-        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
+        new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -2303,8 +2278,8 @@ public class KafkaIndexTaskTest
         new KafkaIndexTaskIOConfig(
             0,
             "sequence0",
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 200L)),
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 500L)),
+            new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 200L), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 500L)),
             kafkaServer.consumerProperties(),
             KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
             true,
@@ -2351,24 +2326,15 @@ public class KafkaIndexTaskTest
     Map<String, Object> consumerProps = kafkaServer.consumerProperties();
     consumerProps.put("max.poll.records", "1");
 
-    final SeekableStreamPartitions<Integer, Long> startPartitions = new SeekableStreamPartitions<>(
+    final SeekableStreamStartSequenceNumbers<Integer, Long> startPartitions = new SeekableStreamStartSequenceNumbers<>(
         topic,
-        ImmutableMap.of(
-            0,
-            0L,
-            1,
-            1L
-        )
+        ImmutableMap.of(0, 0L, 1, 1L),
+        ImmutableSet.of()
     );
 
-    final SeekableStreamPartitions<Integer, Long> endPartitions = new SeekableStreamPartitions<>(
+    final SeekableStreamEndSequenceNumbers<Integer, Long> endPartitions = new SeekableStreamEndSequenceNumbers<>(
         topic,
-        ImmutableMap.of(
-            0,
-            10L,
-            1,
-            2L
-        )
+        ImmutableMap.of(0, 10L, 1, 2L)
     );
 
     final KafkaIndexTask task = createTask(
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 3337faa..e4deb8e 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
@@ -190,5 +190,4 @@ public class KafkaSupervisorIOConfigTest
     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 a08806a..bb8c6a2 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
@@ -59,8 +59,9 @@ 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.SupervisorReport;
+import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers;
 import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner.Status;
-import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
+import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers;
 import org.apache.druid.indexing.seekablestream.supervisor.TaskReportData;
 import org.apache.druid.java.util.common.DateTimes;
 import org.apache.druid.java.util.common.ISE;
@@ -291,15 +292,24 @@ public class KafkaSupervisorTest extends EasyMockSupport
     Assert.assertFalse("maximumMessageTime", taskConfig.getMaximumMessageTime().isPresent());
     Assert.assertFalse("skipOffsetGaps", taskConfig.isSkipOffsetGaps());
 
-    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.getStartSequenceNumbers().getStream());
+    Assert.assertEquals(0L, (long) taskConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(0));
+    Assert.assertEquals(0L, (long) taskConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(1));
+    Assert.assertEquals(0L, (long) taskConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().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));
+    Assert.assertEquals(topic, taskConfig.getEndSequenceNumbers().getStream());
+    Assert.assertEquals(
+        Long.MAX_VALUE,
+        (long) taskConfig.getEndSequenceNumbers().getPartitionSequenceNumberMap().get(0)
+    );
+    Assert.assertEquals(
+        Long.MAX_VALUE,
+        (long) taskConfig.getEndSequenceNumbers().getPartitionSequenceNumberMap().get(1)
+    );
+    Assert.assertEquals(
+        Long.MAX_VALUE,
+        (long) taskConfig.getEndSequenceNumbers().getPartitionSequenceNumberMap().get(2)
+    );
   }
 
   @Test
@@ -354,26 +364,35 @@ public class KafkaSupervisorTest extends EasyMockSupport
     verifyAll();
 
     KafkaIndexTask task1 = captured.getValues().get(0);
-    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(2, task1.getIOConfig().getStartSequenceNumbers().getPartitionSequenceNumberMap().size());
+    Assert.assertEquals(2, task1.getIOConfig().getEndSequenceNumbers().getPartitionSequenceNumberMap().size());
+    Assert.assertEquals(
+        0L,
+        task1.getIOConfig().getStartSequenceNumbers().getPartitionSequenceNumberMap().get(0).longValue()
+    );
     Assert.assertEquals(
         Long.MAX_VALUE,
-        (long) task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(0)
+        task1.getIOConfig().getEndSequenceNumbers().getPartitionSequenceNumberMap().get(0).longValue()
+    );
+    Assert.assertEquals(
+        0L,
+        task1.getIOConfig().getStartSequenceNumbers().getPartitionSequenceNumberMap().get(2).longValue()
     );
-    Assert.assertEquals(0L, (long) task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(2));
     Assert.assertEquals(
         Long.MAX_VALUE,
-        (long) task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(2)
+        task1.getIOConfig().getEndSequenceNumbers().getPartitionSequenceNumberMap().get(2).longValue()
     );
 
     KafkaIndexTask task2 = captured.getValues().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(1, task2.getIOConfig().getStartSequenceNumbers().getPartitionSequenceNumberMap().size());
+    Assert.assertEquals(1, task2.getIOConfig().getEndSequenceNumbers().getPartitionSequenceNumberMap().size());
+    Assert.assertEquals(
+        0L,
+        task2.getIOConfig().getStartSequenceNumbers().getPartitionSequenceNumberMap().get(1).longValue()
+    );
     Assert.assertEquals(
         Long.MAX_VALUE,
-        (long) task2.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(1)
+        task2.getIOConfig().getEndSequenceNumbers().getPartitionSequenceNumberMap().get(1).longValue()
     );
   }
 
@@ -400,18 +419,36 @@ public class KafkaSupervisorTest extends EasyMockSupport
     verifyAll();
 
     KafkaIndexTask task1 = captured.getValues().get(0);
-    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));
+    Assert.assertEquals(3, task1.getIOConfig().getStartSequenceNumbers().getPartitionSequenceNumberMap().size());
+    Assert.assertEquals(3, task1.getIOConfig().getEndSequenceNumbers().getPartitionSequenceNumberMap().size());
+    Assert.assertEquals(
+        0L,
+        task1.getIOConfig().getStartSequenceNumbers().getPartitionSequenceNumberMap().get(0).longValue()
+    );
+    Assert.assertEquals(
+        0L,
+        task1.getIOConfig().getStartSequenceNumbers().getPartitionSequenceNumberMap().get(1).longValue()
+    );
+    Assert.assertEquals(
+        0L,
+        task1.getIOConfig().getStartSequenceNumbers().getPartitionSequenceNumberMap().get(2).longValue()
+    );
 
     KafkaIndexTask task2 = captured.getValues().get(1);
-    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));
+    Assert.assertEquals(3, task2.getIOConfig().getStartSequenceNumbers().getPartitionSequenceNumberMap().size());
+    Assert.assertEquals(3, task2.getIOConfig().getEndSequenceNumbers().getPartitionSequenceNumberMap().size());
+    Assert.assertEquals(
+        0L,
+        task2.getIOConfig().getStartSequenceNumbers().getPartitionSequenceNumberMap().get(0).longValue()
+    );
+    Assert.assertEquals(
+        0L,
+        task2.getIOConfig().getStartSequenceNumbers().getPartitionSequenceNumberMap().get(1).longValue()
+    );
+    Assert.assertEquals(
+        0L,
+        task2.getIOConfig().getStartSequenceNumbers().getPartitionSequenceNumberMap().get(2).longValue()
+    );
   }
 
   @Test
@@ -518,9 +555,18 @@ public class KafkaSupervisorTest extends EasyMockSupport
     verifyAll();
 
     KafkaIndexTask task = captured.getValue();
-    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));
+    Assert.assertEquals(
+        1100L,
+        task.getIOConfig().getStartSequenceNumbers().getPartitionSequenceNumberMap().get(0).longValue()
+    );
+    Assert.assertEquals(
+        1100L,
+        task.getIOConfig().getStartSequenceNumbers().getPartitionSequenceNumberMap().get(1).longValue()
+    );
+    Assert.assertEquals(
+        1100L,
+        task.getIOConfig().getStartSequenceNumbers().getPartitionSequenceNumberMap().get(2).longValue()
+    );
   }
 
   @Test
@@ -539,7 +585,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
     expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes();
     expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
         new KafkaDataSourceMetadata(
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))
+            new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L), ImmutableSet.of())
         )
     ).anyTimes();
     expect(taskQueue.add(capture(captured))).andReturn(true);
@@ -552,9 +598,18 @@ public class KafkaSupervisorTest extends EasyMockSupport
     KafkaIndexTask task = captured.getValue();
     KafkaIndexTaskIOConfig taskConfig = task.getIOConfig();
     Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName());
-    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));
+    Assert.assertEquals(
+        10L,
+        taskConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(0).longValue()
+    );
+    Assert.assertEquals(
+        20L,
+        taskConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(1).longValue()
+    );
+    Assert.assertEquals(
+        30L,
+        taskConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(2).longValue()
+    );
   }
 
   @Test(expected = ISE.class)
@@ -567,7 +622,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
     expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes();
     expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
         new KafkaDataSourceMetadata(
-            new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))
+            new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L), ImmutableSet.of())
         )
     ).anyTimes();
     replayAll();
@@ -587,8 +642,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
         "id1",
         DATASOURCE,
         1,
-        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L)),
-        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 10L)),
+        new SeekableStreamStartSequenceNumbers<>("topic", ImmutableMap.of(0, 0L), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>("topic", ImmutableMap.of(0, 10L)),
         null,
         null
     );
@@ -598,8 +653,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
         "id2",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
-        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 333L, 1, 333L, 2, 333L)),
+        new SeekableStreamStartSequenceNumbers<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>("topic", ImmutableMap.of(0, 333L, 1, 333L, 2, 333L)),
         null,
         null
     );
@@ -609,8 +664,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
         "id3",
         DATASOURCE,
         1,
-        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 1L)),
-        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 333L, 1, 333L, 2, 330L)),
+        new SeekableStreamStartSequenceNumbers<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 1L), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>("topic", ImmutableMap.of(0, 333L, 1, 333L, 2, 330L)),
         null,
         null
     );
@@ -620,8 +675,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
         "id4",
         "other-datasource",
         2,
-        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L)),
-        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 10L)),
+        new SeekableStreamStartSequenceNumbers<>("topic", ImmutableMap.of(0, 0L), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>("topic", ImmutableMap.of(0, 10L)),
         null,
         null
     );
@@ -691,8 +746,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
         "id1",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 2, 0L)),
-        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
+        new SeekableStreamStartSequenceNumbers<>("topic", ImmutableMap.of(0, 0L, 2, 0L), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
         null,
         null
     );
@@ -700,8 +755,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
         "id2",
         DATASOURCE,
         1,
-        new SeekableStreamPartitions<>("topic", ImmutableMap.of(1, 0L)),
-        new SeekableStreamPartitions<>("topic", ImmutableMap.of(1, Long.MAX_VALUE)),
+        new SeekableStreamStartSequenceNumbers<>("topic", ImmutableMap.of(1, 0L), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>("topic", ImmutableMap.of(1, Long.MAX_VALUE)),
         null,
         null
     );
@@ -709,8 +764,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
         "id3",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
-        new SeekableStreamPartitions<>(
+        new SeekableStreamStartSequenceNumbers<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>(
             "topic",
             ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
         ),
@@ -721,8 +776,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
         "id4",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 1, 0L)),
-        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE)),
+        new SeekableStreamStartSequenceNumbers<>("topic", ImmutableMap.of(0, 0L, 1, 0L), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE)),
         null,
         null
     );
@@ -730,8 +785,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
         "id5",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("topic", ImmutableMap.of(1, 0L, 2, 0L)),
-        new SeekableStreamPartitions<>("topic", ImmutableMap.of(1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
+        new SeekableStreamStartSequenceNumbers<>("topic", ImmutableMap.of(1, 0L, 2, 0L), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>("topic", ImmutableMap.of(1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
         null,
         null
     );
@@ -740,7 +795,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
 
     expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
     expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes();
-    expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes();
+    expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes();
     expect(taskStorage.getActiveTasks()).andReturn(existingTasks).anyTimes();
     expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes();
     expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes();
@@ -889,8 +944,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
         "id1",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 2, 0L)),
-        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
+        new SeekableStreamStartSequenceNumbers<>("topic", ImmutableMap.of(0, 0L, 2, 0L), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
         now,
         maxi
     );
@@ -1166,10 +1221,10 @@ public class KafkaSupervisorTest extends EasyMockSupport
       Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName());
       Assert.assertTrue("isUseTransaction", taskConfig.isUseTransaction());
 
-      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));
+      Assert.assertEquals(topic, taskConfig.getStartSequenceNumbers().getStream());
+      Assert.assertEquals(10L, (long) taskConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(0));
+      Assert.assertEquals(20L, (long) taskConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(1));
+      Assert.assertEquals(35L, (long) taskConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(2));
     }
   }
 
@@ -1185,8 +1240,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
         "id1",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
-        new SeekableStreamPartitions<>(
+        new SeekableStreamStartSequenceNumbers<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>(
             "topic",
             ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
         ),
@@ -1259,23 +1314,32 @@ public class KafkaSupervisorTest extends EasyMockSupport
     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().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.getStartSequenceNumbers().getStream());
+    Assert.assertEquals(
+        10L,
+        capturedTaskConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(0).longValue()
+    );
+    Assert.assertEquals(
+        20L,
+        capturedTaskConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(1).longValue()
+    );
+    Assert.assertEquals(
+        30L,
+        capturedTaskConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(2).longValue()
+    );
 
-    Assert.assertEquals(topic, capturedTaskConfig.getEndPartitions().getStream());
+    Assert.assertEquals(topic, capturedTaskConfig.getEndSequenceNumbers().getStream());
     Assert.assertEquals(
         Long.MAX_VALUE,
-        (long) capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(0)
+        capturedTaskConfig.getEndSequenceNumbers().getPartitionSequenceNumberMap().get(0).longValue()
     );
     Assert.assertEquals(
         Long.MAX_VALUE,
-        (long) capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(1)
+        capturedTaskConfig.getEndSequenceNumbers().getPartitionSequenceNumberMap().get(1).longValue()
     );
     Assert.assertEquals(
         Long.MAX_VALUE,
-        (long) capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(2)
+        capturedTaskConfig.getEndSequenceNumbers().getPartitionSequenceNumberMap().get(2).longValue()
     );
   }
 
@@ -1291,8 +1355,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
         "id1",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 2, 0L)),
-        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
+        new SeekableStreamStartSequenceNumbers<>("topic", ImmutableMap.of(0, 0L, 2, 0L), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
         null,
         null
     );
@@ -1356,23 +1420,32 @@ public class KafkaSupervisorTest extends EasyMockSupport
     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().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.getStartSequenceNumbers().getStream());
+    Assert.assertEquals(
+        10L,
+        capturedTaskConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(0).longValue()
+    );
+    Assert.assertEquals(
+        0L,
+        capturedTaskConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(1).longValue()
+    );
+    Assert.assertEquals(
+        30L,
+        capturedTaskConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(2).longValue()
+    );
 
-    Assert.assertEquals(topic, capturedTaskConfig.getEndPartitions().getStream());
+    Assert.assertEquals(topic, capturedTaskConfig.getEndSequenceNumbers().getStream());
     Assert.assertEquals(
         Long.MAX_VALUE,
-        (long) capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(0)
+        capturedTaskConfig.getEndSequenceNumbers().getPartitionSequenceNumberMap().get(0).longValue()
     );
     Assert.assertEquals(
         Long.MAX_VALUE,
-        (long) capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(1)
+        capturedTaskConfig.getEndSequenceNumbers().getPartitionSequenceNumberMap().get(1).longValue()
     );
     Assert.assertEquals(
         Long.MAX_VALUE,
-        (long) capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(2)
+        capturedTaskConfig.getEndSequenceNumbers().getPartitionSequenceNumberMap().get(2).longValue()
     );
   }
 
@@ -1390,8 +1463,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
         "id1",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
-        new SeekableStreamPartitions<>(
+        new SeekableStreamStartSequenceNumbers<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>(
             "topic",
             ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
         ),
@@ -1403,8 +1476,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
         "id2",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 1L, 1, 2L, 2, 3L)),
-        new SeekableStreamPartitions<>(
+        new SeekableStreamStartSequenceNumbers<>("topic", ImmutableMap.of(0, 1L, 1, 2L, 2, 3L), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>(
             "topic",
             ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
         ),
@@ -1625,8 +1698,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
 
     for (Task task : captured.getValues()) {
       KafkaIndexTaskIOConfig taskConfig = ((KafkaIndexTask) task).getIOConfig();
-      Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0));
-      Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2));
+      Assert.assertEquals(0L, (long) taskConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(0));
+      Assert.assertEquals(0L, (long) taskConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(2));
     }
   }
 
@@ -1718,8 +1791,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
 
     for (Task task : captured.getValues()) {
       KafkaIndexTaskIOConfig taskConfig = ((KafkaIndexTask) task).getIOConfig();
-      Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0));
-      Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2));
+      Assert.assertEquals(0L, (long) taskConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(0));
+      Assert.assertEquals(0L, (long) taskConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(2));
     }
   }
 
@@ -1759,8 +1832,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
         "id1",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
-        new SeekableStreamPartitions<>(
+        new SeekableStreamStartSequenceNumbers<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>(
             "topic",
             ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
         ),
@@ -1772,8 +1845,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
         "id2",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
-        new SeekableStreamPartitions<>(
+        new SeekableStreamStartSequenceNumbers<>("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>(
             "topic",
             ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
         ),
@@ -1785,8 +1858,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
         "id3",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
-        new SeekableStreamPartitions<>(
+        new SeekableStreamStartSequenceNumbers<>("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>(
             "topic",
             ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
         ),
@@ -1896,20 +1969,16 @@ public class KafkaSupervisorTest extends EasyMockSupport
     Capture<String> captureDataSource = EasyMock.newCapture();
     Capture<DataSourceMetadata> captureDataSourceMetadata = EasyMock.newCapture();
 
-    KafkaDataSourceMetadata kafkaDataSourceMetadata = new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(
-        topic,
-        ImmutableMap.of(0, 1000L, 1, 1000L, 2, 1000L)
-    ));
+    KafkaDataSourceMetadata kafkaDataSourceMetadata = new KafkaDataSourceMetadata(
+        new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 1000L, 1, 1000L, 2, 1000L), ImmutableSet.of())
+    );
 
-    KafkaDataSourceMetadata resetMetadata = new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(
-        topic,
-        ImmutableMap.of(1, 1000L, 2, 1000L)
-    ));
+    KafkaDataSourceMetadata resetMetadata = new KafkaDataSourceMetadata(
+        new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(1, 1000L, 2, 1000L), ImmutableSet.of())
+    );
 
-    KafkaDataSourceMetadata expectedMetadata = new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(
-        topic,
-        ImmutableMap.of(0, 1000L)
-    ));
+    KafkaDataSourceMetadata expectedMetadata = new KafkaDataSourceMetadata(
+        new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 1000L), ImmutableSet.of()));
 
     EasyMock.reset(indexerMetadataStorageCoordinator);
     expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE))
@@ -1949,10 +2018,13 @@ public class KafkaSupervisorTest extends EasyMockSupport
     supervisor.runInternal();
     verifyAll();
 
-    KafkaDataSourceMetadata resetMetadata = new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(
-        topic,
-        ImmutableMap.of(1, 1000L, 2, 1000L)
-    ));
+    KafkaDataSourceMetadata resetMetadata = new KafkaDataSourceMetadata(
+        new SeekableStreamStartSequenceNumbers<>(
+            topic,
+            ImmutableMap.of(1, 1000L, 2, 1000L),
+            ImmutableSet.of()
+        )
+    );
 
     EasyMock.reset(indexerMetadataStorageCoordinator);
     // no DataSourceMetadata in metadata store
@@ -1977,8 +2049,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
         "id1",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
-        new SeekableStreamPartitions<>(
+        new SeekableStreamStartSequenceNumbers<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>(
             "topic",
             ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
         ),
@@ -1990,8 +2062,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
         "id2",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
-        new SeekableStreamPartitions<>(
+        new SeekableStreamStartSequenceNumbers<>("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>(
             "topic",
             ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
         ),
@@ -2003,8 +2075,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
         "id3",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
-        new SeekableStreamPartitions<>(
+        new SeekableStreamStartSequenceNumbers<>("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>(
             "topic",
             ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
         ),
@@ -2074,8 +2146,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
         "id1",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
-        new SeekableStreamPartitions<>(
+        new SeekableStreamStartSequenceNumbers<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>(
             "topic",
             ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
         ),
@@ -2087,8 +2159,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
         "id2",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
-        new SeekableStreamPartitions<>(
+        new SeekableStreamStartSequenceNumbers<>("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>(
             "topic",
             ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
         ),
@@ -2100,8 +2172,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
         "id3",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
-        new SeekableStreamPartitions<>(
+        new SeekableStreamStartSequenceNumbers<>("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>(
             "topic",
             ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
         ),
@@ -2170,8 +2242,11 @@ public class KafkaSupervisorTest extends EasyMockSupport
         "id1",
         DATASOURCE,
         0,
-        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)),
+        new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L, 1, 0L, 2, 0L), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>(
+            topic,
+            ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
+        ),
         null,
         null
     );
@@ -2180,8 +2255,11 @@ public class KafkaSupervisorTest extends EasyMockSupport
         "id2",
         DATASOURCE,
         0,
-        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)),
+        new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>(
+            topic,
+            ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
+        ),
         null,
         null
     );
@@ -2190,8 +2268,11 @@ public class KafkaSupervisorTest extends EasyMockSupport
         "id3",
         DATASOURCE,
         0,
-        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)),
+        new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>(
+            topic,
+            ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
+        ),
         null,
         null
     );
@@ -2248,8 +2329,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
     supervisor.checkpoint(
         0,
         ((KafkaIndexTask) id1).getIOConfig().getBaseSequenceName(),
-        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, checkpoints.get(0))),
-        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, fakeCheckpoints))
+        new KafkaDataSourceMetadata(new SeekableStreamStartSequenceNumbers<>(topic, checkpoints.get(0), ImmutableSet.of())),
+        new KafkaDataSourceMetadata(new SeekableStreamStartSequenceNumbers<>(topic, fakeCheckpoints, fakeCheckpoints.keySet()))
     );
 
     while (supervisor.getNoticesQueueSize() > 0) {
@@ -2273,8 +2354,11 @@ public class KafkaSupervisorTest extends EasyMockSupport
         "id1",
         DATASOURCE,
         0,
-        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)),
+        new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L, 1, 0L, 2, 0L), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>(
+            topic,
+            ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
+        ),
         null,
         null
     );
@@ -2283,8 +2367,11 @@ public class KafkaSupervisorTest extends EasyMockSupport
         "id2",
         DATASOURCE,
         0,
-        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)),
+        new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>(
+            topic,
+            ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
+        ),
         null,
         null
     );
@@ -2293,8 +2380,11 @@ public class KafkaSupervisorTest extends EasyMockSupport
         "id3",
         DATASOURCE,
         0,
-        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)),
+        new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>(
+            topic,
+            ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
+        ),
         null,
         null
     );
@@ -2319,8 +2409,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
     supervisor.checkpoint(
         0,
         ((KafkaIndexTask) id1).getIOConfig().getBaseSequenceName(),
-        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, Collections.emptyMap())),
-        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, Collections.emptyMap()))
+        new KafkaDataSourceMetadata(new SeekableStreamStartSequenceNumbers<>(topic, Collections.emptyMap(), ImmutableSet.of())),
+        new KafkaDataSourceMetadata(new SeekableStreamStartSequenceNumbers<>(topic, Collections.emptyMap(), ImmutableSet.of()))
     );
 
     while (supervisor.getNoticesQueueSize() > 0) {
@@ -2333,8 +2423,9 @@ public class KafkaSupervisorTest extends EasyMockSupport
       Thread.sleep(100);
     }
 
-    Assert.assertTrue(serviceEmitter.getStackTrace()
-                                    .startsWith("org.apache.druid.java.util.common.ISE: WTH?! cannot find"));
+    Assert.assertTrue(
+        serviceEmitter.getStackTrace().startsWith("org.apache.druid.java.util.common.ISE: WTH?! cannot find")
+    );
     Assert.assertEquals(
         "WTH?! cannot find taskGroup [0] among all activelyReadingTaskGroups [{}]",
         serviceEmitter.getExceptionMessage()
@@ -2352,8 +2443,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
         "id1",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L)),
-        new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, Long.MAX_VALUE)),
+        new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, Long.MAX_VALUE)),
         null,
         null
     );
@@ -2362,8 +2453,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
         "id2",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L)),
-        new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, Long.MAX_VALUE)),
+        new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, Long.MAX_VALUE)),
         null,
         null
     );
@@ -2372,8 +2463,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
         "id3",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L)),
-        new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, Long.MAX_VALUE)),
+        new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, Long.MAX_VALUE)),
         null,
         null
     );
@@ -2424,8 +2515,10 @@ public class KafkaSupervisorTest extends EasyMockSupport
     supervisor.checkpoint(
         null,
         ((KafkaIndexTask) id1).getIOConfig().getBaseSequenceName(),
-        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, checkpoints.get(0))),
-        new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, newCheckpoints.get(0)))
+        new KafkaDataSourceMetadata(new SeekableStreamStartSequenceNumbers<>(topic, checkpoints.get(0), ImmutableSet.of())),
+        new KafkaDataSourceMetadata(
+            new SeekableStreamStartSequenceNumbers<>(topic, newCheckpoints.get(0), newCheckpoints.get(0).keySet())
+        )
     );
 
     while (supervisor.getNoticesQueueSize() > 0) {
@@ -2478,8 +2571,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
         "id1",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
-        new SeekableStreamPartitions<>(
+        new SeekableStreamStartSequenceNumbers<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>(
             "topic",
             ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
         ),
@@ -2491,8 +2584,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
         "id2",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
-        new SeekableStreamPartitions<>(
+        new SeekableStreamStartSequenceNumbers<>("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>(
             "topic",
             ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
         ),
@@ -2504,8 +2597,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
         "id3",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
-        new SeekableStreamPartitions<>(
+        new SeekableStreamStartSequenceNumbers<>("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>(
             "topic",
             ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)
         ),
@@ -2672,15 +2765,33 @@ public class KafkaSupervisorTest extends EasyMockSupport
     Assert.assertFalse("maximumMessageTime", taskConfig.getMaximumMessageTime().isPresent());
     Assert.assertFalse("skipOffsetGaps", taskConfig.isSkipOffsetGaps());
 
-    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.getStartSequenceNumbers().getStream());
+    Assert.assertEquals(
+        0L,
+        taskConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(0).longValue()
+    );
+    Assert.assertEquals(
+        0L,
+        taskConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(1).longValue()
+    );
+    Assert.assertEquals(
+        0L,
+        taskConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(2).longValue()
+    );
 
-    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));
+    Assert.assertEquals(topic, taskConfig.getEndSequenceNumbers().getStream());
+    Assert.assertEquals(
+        Long.MAX_VALUE,
+        taskConfig.getEndSequenceNumbers().getPartitionSequenceNumberMap().get(0).longValue()
+    );
+    Assert.assertEquals(
+        Long.MAX_VALUE,
+        taskConfig.getEndSequenceNumbers().getPartitionSequenceNumberMap().get(1).longValue()
+    );
+    Assert.assertEquals(
+        Long.MAX_VALUE,
+        taskConfig.getEndSequenceNumbers().getPartitionSequenceNumberMap().get(2).longValue()
+    );
   }
 
   @Test
@@ -2727,7 +2838,6 @@ public class KafkaSupervisorTest extends EasyMockSupport
     Assert.assertEquals(ImmutableMap.of("task2", ImmutableMap.of("prop2", "val2")), stats.get("1"));
   }
 
-
   private void addSomeEvents(int numEventsPerPartition) throws Exception
   {
     //create topic manually
@@ -2888,8 +2998,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
       String id,
       String dataSource,
       int taskGroupId,
-      SeekableStreamPartitions<Integer, Long> startPartitions,
-      SeekableStreamPartitions<Integer, Long> endPartitions,
+      SeekableStreamStartSequenceNumbers<Integer, Long> startPartitions,
+      SeekableStreamEndSequenceNumbers<Integer, Long> endPartitions,
       DateTime minimumMessageTime,
       DateTime maximumMessageTime
   )
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
index 4592c1b..d3c1630 100644
--- 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
@@ -21,24 +21,39 @@ package org.apache.druid.indexing.kinesis;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.druid.indexing.overlord.DataSourceMetadata;
 import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata;
-import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
-
-import java.util.Map;
+import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers;
+import org.apache.druid.indexing.seekablestream.SeekableStreamSequenceNumbers;
 
 public class KinesisDataSourceMetadata extends SeekableStreamDataSourceMetadata<String, String>
 {
   @JsonCreator
   public KinesisDataSourceMetadata(
-      @JsonProperty("partitions") SeekableStreamPartitions<String, String> kinesisPartitions
+      @JsonProperty("partitions") SeekableStreamSequenceNumbers<String, String> kinesisPartitions
   )
   {
     super(kinesisPartitions);
   }
 
   @Override
-  protected KinesisDataSourceMetadata createConcreteDataSourceMetaData(String streamName, Map<String, String> newMap)
+  public DataSourceMetadata asStartMetadata()
+  {
+    final SeekableStreamSequenceNumbers<String, String> sequenceNumbers = getSeekableStreamSequenceNumbers();
+    if (sequenceNumbers instanceof SeekableStreamEndSequenceNumbers) {
+      return createConcreteDataSourceMetaData(
+          ((SeekableStreamEndSequenceNumbers<String, String>) sequenceNumbers).asStartPartitions(false)
+      );
+    } else {
+      return this;
+    }
+  }
+
+  @Override
+  protected KinesisDataSourceMetadata createConcreteDataSourceMetaData(
+      SeekableStreamSequenceNumbers<String, String> seekableStreamSequenceNumbers
+  )
   {
-    return new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(streamName, newMap));
+    return new KinesisDataSourceMetadata(seekableStreamSequenceNumbers);
   }
 }
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
index 3467481..ff1847a 100644
--- 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
@@ -90,7 +90,7 @@ public class KinesisIndexTask extends SeekableStreamIndexTask<String, String>
     KinesisIndexTaskTuningConfig tuningConfig = ((KinesisIndexTaskTuningConfig) super.tuningConfig);
     int fetchThreads = tuningConfig.getFetchThreads() != null
                        ? tuningConfig.getFetchThreads()
-                       : Math.max(1, ioConfig.getStartPartitions().getPartitionSequenceNumberMap().size());
+                       : Math.max(1, ioConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().size());
 
     return new KinesisRecordSupplier(
         KinesisRecordSupplier.getAmazonKinesisClient(
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
index 307e971..1fdab0b 100644
--- 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
@@ -22,12 +22,12 @@ 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.SeekableStreamEndSequenceNumbers;
 import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig;
-import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
+import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers;
 import org.joda.time.DateTime;
 
 import javax.annotation.Nullable;
-import java.util.Set;
 
 public class KinesisIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig<String, String>
 {
@@ -46,15 +46,14 @@ public class KinesisIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig<St
   public KinesisIndexTaskIOConfig(
       @JsonProperty("taskGroupId") @Nullable Integer taskGroupId,
       @JsonProperty("baseSequenceName") String baseSequenceName,
-      @JsonProperty("startPartitions") SeekableStreamPartitions<String, String> startPartitions,
-      @JsonProperty("endPartitions") SeekableStreamPartitions<String, String> endPartitions,
+      @JsonProperty("startSequenceNumbers") SeekableStreamStartSequenceNumbers<String, String> startSequenceNumbers,
+      @JsonProperty("endSequenceNumbers") SeekableStreamEndSequenceNumbers<String, String> endSequenceNumbers,
       @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
@@ -63,18 +62,20 @@ public class KinesisIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig<St
     super(
         taskGroupId,
         baseSequenceName,
-        startPartitions,
-        endPartitions,
+        startSequenceNumbers,
+        endSequenceNumbers,
         useTransaction,
         minimumMessageTime,
         maximumMessageTime,
-        true,
-        exclusiveStartSequenceNumberPartitions
+        true
+    );
+    Preconditions.checkArgument(
+        endSequenceNumbers.getPartitionSequenceNumberMap()
+                          .values()
+                          .stream()
+                          .noneMatch(x -> x.equals(KinesisSequenceNumber.END_OF_SHARD_MARKER)),
+        "End sequenceNumbers must not have the end of shard marker (EOS)"
     );
-    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;
@@ -125,15 +126,14 @@ public class KinesisIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig<St
   {
     return "KinesisIndexTaskIOConfig{" +
            "baseSequenceName='" + getBaseSequenceName() + '\'' +
-           ", startPartitions=" + getStartPartitions() +
-           ", endPartitions=" + getEndPartitions() +
+           ", startPartitions=" + getStartSequenceNumbers() +
+           ", endPartitions=" + getEndSequenceNumbers() +
            ", 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
index 247f6d7..335119a 100644
--- 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
@@ -26,8 +26,9 @@ 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.SeekableStreamEndSequenceNumbers;
 import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner;
-import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
+import org.apache.druid.indexing.seekablestream.SeekableStreamSequenceNumbers;
 import org.apache.druid.indexing.seekablestream.SequenceMetadata;
 import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord;
 import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber;
@@ -47,6 +48,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeMap;
+import java.util.concurrent.ConcurrentMap;
 import java.util.stream.Collectors;
 
 public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner<String, String>
@@ -93,14 +95,14 @@ public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner<String
   }
 
   @Override
-  protected SeekableStreamPartitions<String, String> deserializePartitionsFromMetadata(
+  protected SeekableStreamEndSequenceNumbers<String, String> deserializePartitionsFromMetadata(
       ObjectMapper mapper,
       Object object
   )
   {
     return mapper.convertValue(object, mapper.getTypeFactory().constructParametrizedType(
-        SeekableStreamPartitions.class,
-        SeekableStreamPartitions.class,
+        SeekableStreamEndSequenceNumbers.class,
+        SeekableStreamEndSequenceNumbers.class,
         String.class,
         String.class
     ));
@@ -108,7 +110,7 @@ public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner<String
 
   @Override
   protected SeekableStreamDataSourceMetadata<String, String> createDataSourceMetadata(
-      SeekableStreamPartitions<String, String> partitions
+      SeekableStreamSequenceNumbers<String, String> partitions
   )
   {
     return new KinesisDataSourceMetadata(partitions);
@@ -124,11 +126,11 @@ public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner<String
   protected void possiblyResetDataSourceMetadata(
       TaskToolbox toolbox,
       RecordSupplier<String, String> recordSupplier,
-      Set<StreamPartition<String>> assignment,
-      Map<String, String> currOffsets
+      Set<StreamPartition<String>> assignment
   )
   {
     if (!task.getTuningConfig().isSkipSequenceNumberAvailabilityCheck()) {
+      final ConcurrentMap<String, String> currOffsets = getCurrentOffsets();
       for (final StreamPartition<String> streamPartition : assignment) {
         String sequence = currOffsets.get(streamPartition.getPartitionId());
         String earliestSequenceNumber = recordSupplier.getEarliestSequenceNumber(streamPartition);
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
index 9a96e4e..6cce144 100644
--- 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
@@ -20,15 +20,14 @@
 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;
 
+// OrderedSequenceNumber.equals() should be used instead.
+@SuppressWarnings("ComparableImplementedButEqualsNotOverridden")
 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
@@ -36,17 +35,24 @@ public class KinesisSequenceNumber extends OrderedSequenceNumber<String>
    * reading and start publishing
    */
   public static final String END_OF_SHARD_MARKER = "EOS";
+
+  // this special marker is used by the KinesisSupervisor to set the endOffsets
+  // of newly created indexing tasks. This is necessary because streaming tasks do not
+  // have endPartitionOffsets. This marker signals to the task that it should continue
+  // to ingest data until taskDuration has elapsed or the task was stopped or paused or killed
+  public static final String NO_END_SEQUENCE_NUMBER = "NO_END_SEQUENCE_NUMBER";
+
   // 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)
+  private KinesisSequenceNumber(String sequenceNumber, boolean isExclusive)
   {
     super(sequenceNumber, isExclusive);
     if (END_OF_SHARD_MARKER.equals(sequenceNumber)
-        || SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER.equals(sequenceNumber)) {
+        || NO_END_SEQUENCE_NUMBER.equals(sequenceNumber)) {
       isMaxSequenceNumber = true;
       this.intSequence = null;
     } else {
@@ -66,7 +72,7 @@ public class KinesisSequenceNumber extends OrderedSequenceNumber<String>
   }
 
   @Override
-  public int compareTo(@NotNull OrderedSequenceNumber<String> o)
+  public int compareTo(OrderedSequenceNumber<String> o)
   {
     KinesisSequenceNumber num = (KinesisSequenceNumber) o;
     if (isMaxSequenceNumber && num.isMaxSequenceNumber) {
@@ -79,5 +85,4 @@ public class KinesisSequenceNumber extends OrderedSequenceNumber<String>
       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
index f53bdf2..332e693 100644
--- 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
@@ -40,10 +40,11 @@ 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.SeekableStreamEndSequenceNumbers;
 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.SeekableStreamStartSequenceNumbers;
 import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber;
 import org.apache.druid.indexing.seekablestream.common.RecordSupplier;
 import org.apache.druid.indexing.seekablestream.common.StreamPartition;
@@ -55,6 +56,7 @@ import org.apache.druid.java.util.common.StringUtils;
 import org.joda.time.DateTime;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -120,15 +122,18 @@ public class KinesisSupervisor extends SeekableStreamSupervisor<String, String>
     return new KinesisIndexTaskIOConfig(
         groupId,
         baseSequenceName,
-        new SeekableStreamPartitions<>(ioConfig.getStream(), startPartitions),
-        new SeekableStreamPartitions<>(ioConfig.getStream(), endPartitions),
+        new SeekableStreamStartSequenceNumbers<>(
+            ioConfig.getStream(),
+            startPartitions,
+            exclusiveStartSequenceNumberPartitions
+        ),
+        new SeekableStreamEndSequenceNumbers<>(ioConfig.getStream(), endPartitions),
         true,
         minimumMessageTime,
         maximumMessageTime,
         ioConfig.getEndpoint(),
         ioConfig.getRecordsPerFetch(),
         ioConfig.getFetchDelayMillis(),
-        exclusiveStartSequenceNumberPartitions,
         ioConfig.getAwsAssumedRoleArn(),
         ioConfig.getAwsExternalId(),
         ioConfig.isDeaggregate()
@@ -204,10 +209,8 @@ public class KinesisSupervisor extends SeekableStreamSupervisor<String, String>
         taskTuningConfig.getFetchSequenceNumberTimeout(),
         taskTuningConfig.getMaxRecordsPerPoll()
     );
-
   }
 
-
   @Override
   protected void scheduleReporting(ScheduledExecutorService reportingExec)
   {
@@ -261,13 +264,13 @@ public class KinesisSupervisor extends SeekableStreamSupervisor<String, String>
   }
 
   @Override
-  protected SeekableStreamDataSourceMetadata<String, String> createDataSourceMetaData(
+  protected SeekableStreamDataSourceMetadata<String, String> createDataSourceMetaDataForReset(
       String stream,
       Map<String, String> map
   )
   {
     return new KinesisDataSourceMetadata(
-        new SeekableStreamPartitions<>(stream, map)
+        new SeekableStreamStartSequenceNumbers<>(stream, map, Collections.emptySet())
     );
   }
 
@@ -300,7 +303,7 @@ public class KinesisSupervisor extends SeekableStreamSupervisor<String, String>
   @Override
   protected String getEndOfPartitionMarker()
   {
-    return SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER;
+    return KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER;
   }
 
   @Override
@@ -308,4 +311,10 @@ public class KinesisSupervisor extends SeekableStreamSupervisor<String, String>
   {
     return KinesisSequenceNumber.END_OF_SHARD_MARKER.equals(seqNum);
   }
+
+  @Override
+  protected boolean useExclusiveStartSequenceNumberForStartSequence()
+  {
+    return true;
+  }
 }
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
index f1e3b0f..fbb3d64 100644
--- 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
@@ -21,78 +21,150 @@
 package org.apache.druid.indexing.kinesis;
 
 import com.google.common.collect.ImmutableMap;
-import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
+import com.google.common.collect.ImmutableSet;
+import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers;
+import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers;
 import org.junit.Assert;
 import org.junit.Test;
 
 import java.util.Map;
+import java.util.Set;
 
 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"));
+  private static final KinesisDataSourceMetadata START0 = simpleStartMetadata(ImmutableMap.of());
+  private static final KinesisDataSourceMetadata START1 = simpleStartMetadata(ImmutableMap.of("0", "2L", "1", "3L"));
+  private static final KinesisDataSourceMetadata START2 = simpleStartMetadata(
+      ImmutableMap.of("0", "2L", "1", "4L", "2", "5L")
+  );
+  private static final KinesisDataSourceMetadata START3 = simpleStartMetadata(ImmutableMap.of("0", "2L", "2", "5L"));
+  private static final KinesisDataSourceMetadata START4 = startMetadata(
+      ImmutableMap.of("0", "2L", "2", "5L"),
+      ImmutableSet.of()
+  );
+  private static final KinesisDataSourceMetadata START5 = startMetadata(
+      ImmutableMap.of("0", "2L", "1", "4L", "2", "5L"),
+      ImmutableSet.of("0", "1")
+  );
+  private static final KinesisDataSourceMetadata END0 = endMetadata(ImmutableMap.of());
+  private static final KinesisDataSourceMetadata END1 = endMetadata(ImmutableMap.of("0", "2L", "2", "5L"));
+  private static final KinesisDataSourceMetadata END2 = endMetadata(ImmutableMap.of("0", "2L", "1", "4L"));
 
   @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));
+    Assert.assertTrue(START0.matches(START0));
+    Assert.assertTrue(START0.matches(START1));
+    Assert.assertTrue(START0.matches(START2));
+    Assert.assertTrue(START0.matches(START3));
+    Assert.assertTrue(START0.matches(START4));
+    Assert.assertTrue(START0.matches(START5));
+
+    Assert.assertTrue(START1.matches(START0));
+    Assert.assertTrue(START1.matches(START1));
+    Assert.assertFalse(START1.matches(START2));
+    Assert.assertTrue(START1.matches(START3));
+    Assert.assertFalse(START1.matches(START4));
+    Assert.assertFalse(START1.matches(START5));
+
+    Assert.assertTrue(START2.matches(START0));
+    Assert.assertFalse(START2.matches(START1));
+    Assert.assertTrue(START2.matches(START2));
+    Assert.assertTrue(START2.matches(START3));
+    Assert.assertFalse(START2.matches(START4));
+    Assert.assertFalse(START2.matches(START5));
+
+    Assert.assertTrue(START3.matches(START0));
+    Assert.assertTrue(START3.matches(START1));
+    Assert.assertTrue(START3.matches(START2));
+    Assert.assertTrue(START3.matches(START3));
+    Assert.assertFalse(START3.matches(START4));
+    Assert.assertFalse(START3.matches(START5));
+
+    Assert.assertTrue(START4.matches(START0));
+    Assert.assertFalse(START4.matches(START1));
+    Assert.assertFalse(START4.matches(START2));
+    Assert.assertFalse(START4.matches(START3));
+    Assert.assertTrue(START4.matches(START4));
+    Assert.assertFalse(START4.matches(START5));
+
+    Assert.assertTrue(START5.matches(START0));
+    Assert.assertFalse(START5.matches(START1));
+    Assert.assertFalse(START5.matches(START2));
+    Assert.assertFalse(START5.matches(START3));
+    Assert.assertFalse(START5.matches(START4));
+    Assert.assertTrue(START5.matches(START5));
+
+    Assert.assertTrue(END0.matches(END0));
+    Assert.assertTrue(END0.matches(END1));
+    Assert.assertTrue(END0.matches(END2));
+
+    Assert.assertTrue(END1.matches(END0));
+    Assert.assertTrue(END1.matches(END1));
+    Assert.assertTrue(END1.matches(END2));
+
+    Assert.assertTrue(END2.matches(END0));
+    Assert.assertTrue(END2.matches(END1));
+    Assert.assertTrue(END2.matches(END2));
   }
 
   @Test
   public void testIsValidStart()
   {
-    Assert.assertTrue(KM0.isValidStart());
-    Assert.assertTrue(KM1.isValidStart());
-    Assert.assertTrue(KM2.isValidStart());
-    Assert.assertTrue(KM3.isValidStart());
+    Assert.assertTrue(START0.isValidStart());
+    Assert.assertTrue(START1.isValidStart());
+    Assert.assertTrue(START2.isValidStart());
+    Assert.assertTrue(START3.isValidStart());
+    Assert.assertTrue(START4.isValidStart());
+    Assert.assertTrue(START5.isValidStart());
   }
 
   @Test
   public void testPlus()
   {
     Assert.assertEquals(
-        KM("foo", ImmutableMap.of("0", "2L", "1", "3L", "2", "5L")),
-        KM1.plus(KM3)
+        simpleStartMetadata(ImmutableMap.of("0", "2L", "1", "3L", "2", "5L")),
+        START1.plus(START3)
     );
 
     Assert.assertEquals(
-        KM("foo", ImmutableMap.of("0", "2L", "1", "4L", "2", "5L")),
-        KM0.plus(KM2)
+        simpleStartMetadata(ImmutableMap.of("0", "2L", "1", "4L", "2", "5L")),
+        START0.plus(START2)
     );
 
     Assert.assertEquals(
-        KM("foo", ImmutableMap.of("0", "2L", "1", "4L", "2", "5L")),
-        KM1.plus(KM2)
+        simpleStartMetadata(ImmutableMap.of("0", "2L", "1", "4L", "2", "5L")),
+        START1.plus(START2)
     );
 
     Assert.assertEquals(
-        KM("foo", ImmutableMap.of("0", "2L", "1", "3L", "2", "5L")),
-        KM2.plus(KM1)
+        simpleStartMetadata(ImmutableMap.of("0", "2L", "1", "3L", "2", "5L")),
+        START2.plus(START1)
     );
 
     Assert.assertEquals(
-        KM("foo", ImmutableMap.of("0", "2L", "1", "4L", "2", "5L")),
-        KM2.plus(KM2)
+        simpleStartMetadata(ImmutableMap.of("0", "2L", "1", "4L", "2", "5L")),
+        START2.plus(START2)
+    );
+
+    Assert.assertEquals(
+        startMetadata(ImmutableMap.of("0", "2L", "1", "4L", "2", "5L"), ImmutableSet.of("1")),
+        START2.plus(START4)
+    );
+
+    Assert.assertEquals(
+        startMetadata(ImmutableMap.of("0", "2L", "1", "4L", "2", "5L"), ImmutableSet.of("0", "1")),
+        START2.plus(START5)
+    );
+
+    Assert.assertEquals(
+        endMetadata(ImmutableMap.of("0", "2L", "2", "5L")),
+        END0.plus(END1)
+    );
+
+    Assert.assertEquals(
+        endMetadata(ImmutableMap.of("0", "2L", "1", "4L", "2", "5L")),
+        END1.plus(END2)
     );
   }
 
@@ -100,33 +172,65 @@ public class KinesisDataSourceMetadataTest
   public void testMinus()
   {
     Assert.assertEquals(
-        KM("foo", ImmutableMap.of("1", "3L")),
-        KM1.minus(KM3)
+        simpleStartMetadata(ImmutableMap.of("1", "3L")),
+        START1.minus(START3)
     );
 
     Assert.assertEquals(
-        KM("foo", ImmutableMap.of()),
-        KM0.minus(KM2)
+        simpleStartMetadata(ImmutableMap.of()),
+        START0.minus(START2)
     );
 
     Assert.assertEquals(
-        KM("foo", ImmutableMap.of()),
-        KM1.minus(KM2)
+        simpleStartMetadata(ImmutableMap.of()),
+        START1.minus(START2)
     );
 
     Assert.assertEquals(
-        KM("foo", ImmutableMap.of("2", "5L")),
-        KM2.minus(KM1)
+        simpleStartMetadata(ImmutableMap.of("2", "5L")),
+        START2.minus(START1)
     );
 
     Assert.assertEquals(
-        KM("foo", ImmutableMap.of()),
-        KM2.minus(KM2)
+        simpleStartMetadata(ImmutableMap.of()),
+        START2.minus(START2)
+    );
+
+    Assert.assertEquals(
+        startMetadata(ImmutableMap.of(), ImmutableSet.of()),
+        START4.minus(START2)
+    );
+
+    Assert.assertEquals(
+        startMetadata(ImmutableMap.of("1", "4L"), ImmutableSet.of("1")),
+        START5.minus(START4)
+    );
+
+    Assert.assertEquals(
+        endMetadata(ImmutableMap.of("1", "4L")),
+        END2.minus(END1)
+    );
+
+    Assert.assertEquals(
+        endMetadata(ImmutableMap.of("2", "5L")),
+        END1.minus(END2)
+    );
+  }
+
+  private static KinesisDataSourceMetadata simpleStartMetadata(Map<String, String> sequences)
+  {
+    return startMetadata(sequences, sequences.keySet());
+  }
+
+  private static KinesisDataSourceMetadata startMetadata(Map<String, String> sequences, Set<String> exclusivePartitions)
+  {
+    return new KinesisDataSourceMetadata(
+        new SeekableStreamStartSequenceNumbers<>("foo", sequences, exclusivePartitions)
     );
   }
 
-  private static KinesisDataSourceMetadata KM(String stream, Map<String, String> sequences)
+  private static KinesisDataSourceMetadata endMetadata(Map<String, String> sequences)
   {
-    return new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, sequences));
+    return new KinesisDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>("foo", 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
index fff34f9..7f15fc9 100644
--- 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
@@ -55,8 +55,8 @@ public class KinesisIOConfigTest
                      + "  \"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"
+                     + "  \"startSequenceNumbers\": {\"type\":\"start\", \"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"1\", \"1\":\"10\"}},\n"
+                     + "  \"endSequenceNumbers\": {\"type\":\"end\", \"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"15\", \"1\":\"200\"}}\n"
                      + "}";
 
     KinesisIndexTaskIOConfig config = (KinesisIndexTaskIOConfig) mapper.readValue(
@@ -70,22 +70,22 @@ public class KinesisIOConfigTest
 
     Assert.assertNull(config.getTaskGroupId());
     Assert.assertEquals("my-sequence-name", config.getBaseSequenceName());
-    Assert.assertEquals("mystream", config.getStartPartitions().getStream());
+    Assert.assertEquals("mystream", config.getStartSequenceNumbers().getStream());
     Assert.assertEquals(
         ImmutableMap.of("0", "1", "1", "10"),
-        config.getStartPartitions().getPartitionSequenceNumberMap()
+        config.getStartSequenceNumbers().getPartitionSequenceNumberMap()
     );
-    Assert.assertEquals("mystream", config.getEndPartitions().getStream());
+    Assert.assertEquals("mystream", config.getEndSequenceNumbers().getStream());
     Assert.assertEquals(
         ImmutableMap.of("0", "15", "1", "200"),
-        config.getEndPartitions().getPartitionSequenceNumberMap()
+        config.getEndSequenceNumbers().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.assertEquals(Collections.emptySet(), config.getStartSequenceNumbers().getExclusivePartitions());
     Assert.assertNull(config.getAwsAssumedRoleArn());
     Assert.assertNull(config.getAwsExternalId());
     Assert.assertFalse(config.isDeaggregate());
@@ -99,15 +99,14 @@ public class KinesisIOConfigTest
                      + "  \"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"
+                     + "  \"startSequenceNumbers\": {\"type\":\"start\", \"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"1\", \"1\":\"10\"}, \"exclusivePartitions\" : [\"0\"] },\n"
+                     + "  \"endSequenceNumbers\": {\"type\":\"end\", \"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"
@@ -124,15 +123,15 @@ public class KinesisIOConfigTest
 
     Assert.assertEquals((Integer) 0, config.getTaskGroupId());
     Assert.assertEquals("my-sequence-name", config.getBaseSequenceName());
-    Assert.assertEquals("mystream", config.getStartPartitions().getStream());
+    Assert.assertEquals("mystream", config.getStartSequenceNumbers().getStream());
     Assert.assertEquals(
         ImmutableMap.of("0", "1", "1", "10"),
-        config.getStartPartitions().getPartitionSequenceNumberMap()
+        config.getStartSequenceNumbers().getPartitionSequenceNumberMap()
     );
-    Assert.assertEquals("mystream", config.getEndPartitions().getStream());
+    Assert.assertEquals("mystream", config.getEndSequenceNumbers().getStream());
     Assert.assertEquals(
         ImmutableMap.of("0", "15", "1", "200"),
-        config.getEndPartitions().getPartitionSequenceNumberMap()
+        config.getEndSequenceNumbers().getPartitionSequenceNumberMap()
     );
     Assert.assertFalse(config.isUseTransaction());
     Assert.assertTrue("maximumMessageTime", config.getMaximumMessageTime().isPresent());
@@ -140,7 +139,7 @@ public class KinesisIOConfigTest
     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(config.getStartSequenceNumbers().getExclusivePartitions(), ImmutableSet.of("0"));
     Assert.assertEquals(1000, config.getRecordsPerFetch());
     Assert.assertEquals(1000, config.getFetchDelayMillis());
     Assert.assertEquals("role", config.getAwsAssumedRoleArn());
@@ -155,8 +154,8 @@ public class KinesisIOConfigTest
     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"
+                     + "  \"startSequenceNumbers\": {\"type\":\"start\", \"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"1\", \"1\":\"10\"}},\n"
+                     + "  \"endSequenceNumbers\": {\"type\":\"end\", \"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"15\", \"1\":\"200\"}}\n"
                      + "}";
 
     exception.expect(JsonMappingException.class);
@@ -166,34 +165,34 @@ public class KinesisIOConfigTest
   }
 
   @Test
-  public void testStartPartitionsRequired() throws Exception
+  public void teststartSequenceNumbersRequired() 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"
+                     + "  \"endSequenceNumbers\": {\"type\":\"end\", \"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"15\", \"1\":\"200\"}}\n"
                      + "}";
 
     exception.expect(JsonMappingException.class);
     exception.expectCause(CoreMatchers.isA(NullPointerException.class));
-    exception.expectMessage(CoreMatchers.containsString("startPartitions"));
+    exception.expectMessage(CoreMatchers.containsString("startSequenceNumbers"));
     mapper.readValue(jsonStr, IOConfig.class);
   }
 
   @Test
-  public void testEndPartitionsRequired() throws Exception
+  public void testendSequenceNumbersRequired() 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"
+                     + "  \"startSequenceNumbers\": {\"type\":\"start\", \"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"1\", \"1\":\"10\"}}\n"
                      + "}";
 
     exception.expect(JsonMappingException.class);
     exception.expectCause(CoreMatchers.isA(NullPointerException.class));
-    exception.expectMessage(CoreMatchers.containsString("endPartitions"));
+    exception.expectMessage(CoreMatchers.containsString("endSequenceNumbers"));
     mapper.readValue(jsonStr, IOConfig.class);
   }
 
@@ -204,8 +203,8 @@ public class KinesisIOConfigTest
                      + "  \"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"
+                     + "  \"startSequenceNumbers\": {\"type\":\"start\", \"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"1\", \"1\":\"10\"}},\n"
+                     + "  \"endSequenceNumbers\": {\"type\":\"end\", \"stream\":\"notmystream\", \"partitionSequenceNumberMap\" : {\"0\":\"15\", \"1\":\"200\"}}\n"
                      + "}";
 
     exception.expect(JsonMappingException.class);
@@ -215,14 +214,14 @@ public class KinesisIOConfigTest
   }
 
   @Test
-  public void testStartAndEndPartitionSetMatch() throws Exception
+  public void testStartAndendSequenceNumbersetMatch() 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"
+                     + "  \"startSequenceNumbers\": {\"type\":\"start\", \"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"1\", \"1\":\"10\"}},\n"
+                     + "  \"endSequenceNumbers\": {\"type\":\"end\", \"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"15\", \"2\":\"200\"}}\n"
                      + "}";
 
     exception.expect(JsonMappingException.class);
@@ -237,8 +236,8 @@ public class KinesisIOConfigTest
     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"
+                     + "  \"startSequenceNumbers\": {\"type\":\"start\", \"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"1\", \"1\":\"10\"}},\n"
+                     + "  \"endSequenceNumbers\": {\"type\":\"end\", \"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"15\", \"1\":\"200\"}}\n"
                      + "}";
 
     exception.expect(JsonMappingException.class);
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
index 1823d93..127b3d9 100644
--- 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
@@ -80,8 +80,9 @@ 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.SeekableStreamEndSequenceNumbers;
 import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner;
-import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
+import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers;
 import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord;
 import org.apache.druid.indexing.seekablestream.common.StreamPartition;
 import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor;
@@ -399,14 +400,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
         new KinesisIndexTaskIOConfig(
             null,
             "sequence0",
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "2"
-            )),
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "4"
-            )),
+            new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "2"), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "4")),
             true,
             null,
             null,
@@ -415,7 +410,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport
             null,
             null,
             null,
-            null,
             false
         )
 
@@ -438,13 +432,9 @@ public class KinesisIndexTaskTest extends EasyMockSupport
     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"
-            )
-        )),
+        new KinesisDataSourceMetadata(
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "4"))
+        ),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -480,14 +470,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
         new KinesisIndexTaskIOConfig(
             null,
             "sequence0",
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId0,
-                "0"
-            )),
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId0,
-                "1"
-            )),
+            new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId0, "0"), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId0, "1")),
             true,
             null,
             null,
@@ -496,7 +480,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport
             null,
             null,
             null,
-            null,
             false
         )
 
@@ -519,13 +502,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
     Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
     Assert.assertEquals(
         new KinesisDataSourceMetadata(
-            new SeekableStreamPartitions<>(
-                stream,
-                ImmutableMap.of(
-                    shardId0,
-                    "1"
-                )
-            )),
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId0, "1"))
+        ),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -561,34 +539,20 @@ public class KinesisIndexTaskTest extends EasyMockSupport
 
     replayAll();
 
-    final SeekableStreamPartitions<String, String> startPartitions = new SeekableStreamPartitions<>(
+    final SeekableStreamStartSequenceNumbers<String, String> startPartitions = new SeekableStreamStartSequenceNumbers<>(
         stream,
-        ImmutableMap.of(
-            shardId1,
-            "0",
-            shardId0,
-            "0"
-        )
+        ImmutableMap.of(shardId1, "0", shardId0, "0"),
+        ImmutableSet.of()
     );
 
-    final SeekableStreamPartitions<String, String> checkpoint1 = new SeekableStreamPartitions<>(
+    final SeekableStreamEndSequenceNumbers<String, String> checkpoint1 = new SeekableStreamEndSequenceNumbers<>(
         stream,
-        ImmutableMap.of(
-            shardId1,
-            "4",
-            shardId0,
-            "0"
-        )
+        ImmutableMap.of(shardId1, "4", shardId0, "0")
     );
 
-    final SeekableStreamPartitions<String, String> endPartitions = new SeekableStreamPartitions<>(
+    final SeekableStreamEndSequenceNumbers<String, String> endPartitions = new SeekableStreamEndSequenceNumbers<>(
         stream,
-        ImmutableMap.of(
-            shardId1,
-            "9",
-            shardId0,
-            "1"
-        )
+        ImmutableMap.of(shardId1, "9", shardId0, "1")
     );
     final KinesisIndexTask task = createTask(
         null,
@@ -605,7 +569,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport
             null,
             null,
             null,
-            null,
             false
         )
     );
@@ -614,7 +577,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
       Thread.sleep(10);
     }
     final Map<String, String> currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets());
-    Assert.assertTrue(checkpoint1.getPartitionSequenceNumberMap().equals(currentOffsets));
+    Assert.assertEquals(checkpoint1.getPartitionSequenceNumberMap(), currentOffsets);
     task.getRunner().setEndOffsets(currentOffsets, false);
 
     Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode());
@@ -628,7 +591,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
                 DATA_SCHEMA.getDataSource(),
                 0,
                 new KinesisDataSourceMetadata(startPartitions),
-                new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, currentOffsets))
+                new KinesisDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(stream, currentOffsets))
             )
         )
     );
@@ -648,12 +611,12 @@ public class KinesisIndexTaskTest extends EasyMockSupport
     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"
-        ))),
+        new KinesisDataSourceMetadata(
+            new SeekableStreamEndSequenceNumbers<>(
+                stream,
+                ImmutableMap.of(shardId1, "9", shardId0, "1")
+            )
+        ),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -697,35 +660,24 @@ public class KinesisIndexTaskTest extends EasyMockSupport
     replayAll();
 
     // Insert data
-    final SeekableStreamPartitions<String, String> startPartitions = new SeekableStreamPartitions<>(
+    final SeekableStreamStartSequenceNumbers<String, String> startPartitions = new SeekableStreamStartSequenceNumbers<>(
         stream,
-        ImmutableMap.of(
-            shardId1,
-            "0"
-        )
+        ImmutableMap.of(shardId1, "0"),
+        ImmutableSet.of()
     );
     // 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<>(
+    final SeekableStreamEndSequenceNumbers<String, String> checkpoint1 = new SeekableStreamEndSequenceNumbers<>(
         stream,
-        ImmutableMap.of(
-            shardId1,
-            "2"
-        )
+        ImmutableMap.of(shardId1, "2")
     );
-    final SeekableStreamPartitions<String, String> checkpoint2 = new SeekableStreamPartitions<>(
+    final SeekableStreamEndSequenceNumbers<String, String> checkpoint2 = new SeekableStreamEndSequenceNumbers<>(
         stream,
-        ImmutableMap.of(
-            shardId1,
-            "9"
-        )
+        ImmutableMap.of(shardId1, "9")
     );
-    final SeekableStreamPartitions<String, String> endPartitions = new SeekableStreamPartitions<>(
+    final SeekableStreamEndSequenceNumbers<String, String> endPartitions = new SeekableStreamEndSequenceNumbers<>(
         stream,
-        ImmutableMap.of(
-            shardId1,
-            "10"
-        )
+        ImmutableMap.of(shardId1, "10")
     );
 
     final KinesisIndexTask task = createTask(
@@ -743,7 +695,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport
             null,
             null,
             null,
-            null,
             false
         )
     );
@@ -778,7 +729,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
                 DATA_SCHEMA.getDataSource(),
                 0,
                 new KinesisDataSourceMetadata(startPartitions),
-                new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, currentOffsets))
+                new KinesisDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(stream, currentOffsets))
             )
         )
     );
@@ -787,8 +738,10 @@ public class KinesisIndexTaskTest extends EasyMockSupport
             Objects.hash(
                 DATA_SCHEMA.getDataSource(),
                 0,
-                new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, currentOffsets)),
-                new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, nextOffsets))
+                new KinesisDataSourceMetadata(
+                    new SeekableStreamStartSequenceNumbers<>(stream, currentOffsets, ImmutableSet.of())
+                ),
+                new KinesisDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(stream, nextOffsets))
             )
         )
     );
@@ -807,10 +760,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
     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"
-        ))),
+        new KinesisDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "10"))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -847,14 +797,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
         new KinesisIndexTaskIOConfig(
             null,
             "sequence0",
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "0"
-            )),
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "4"
-            )),
+            new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "0"), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "4")),
             true,
             DateTimes.of("2010"),
             null,
@@ -863,7 +807,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport
             null,
             null,
             null,
-            null,
             false
         )
     );
@@ -890,14 +833,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
     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"
-                )
-            )),
+        new KinesisDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "4"))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -930,14 +866,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
         new KinesisIndexTaskIOConfig(
             null,
             "sequence0",
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "0"
-            )),
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "4"
-            )),
+            new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "0"), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "4")),
             true,
             null,
             DateTimes.of("2010"),
@@ -946,7 +876,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport
             null,
             null,
             null,
-            null,
             false
         )
     );
@@ -975,13 +904,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
     Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors());
     Assert.assertEquals(
         new KinesisDataSourceMetadata(
-            new SeekableStreamPartitions<>(
-                stream,
-                ImmutableMap.of(
-                    shardId1,
-                    "4"
-                )
-            )),
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "4"))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -1023,14 +946,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
         new KinesisIndexTaskIOConfig(
             null,
             "sequence0",
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "0"
-            )),
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "4"
-            )),
+            new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "0"), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "4")),
             true,
             null,
             null,
@@ -1039,7 +956,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport
             null,
             null,
             null,
-            null,
             false
         )
     );
@@ -1066,13 +982,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
     Assert.assertEquals(ImmutableSet.of(desc1), publishedDescriptors());
     Assert.assertEquals(
         new KinesisDataSourceMetadata(
-            new SeekableStreamPartitions<>(
-                stream,
-                ImmutableMap.of(
-                    shardId1,
-                    "4"
-                )
-            )),
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "4"))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -1107,14 +1017,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
         new KinesisIndexTaskIOConfig(
             null,
             "sequence0",
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "2"
-            )),
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "2"
-            )),
+            new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "2"), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "2")),
             true,
             null,
             null,
@@ -1123,7 +1027,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport
             null,
             null,
             null,
-            null,
             false
         )
     );
@@ -1170,14 +1073,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
         new KinesisIndexTaskIOConfig(
             null,
             "sequence0",
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "2"
-            )),
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "4"
-            )),
+            new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "2"), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "4")),
             true,
             null,
             null,
@@ -1186,7 +1083,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport
             null,
             null,
             null,
-            null,
             false
         )
     );
@@ -1208,10 +1104,9 @@ public class KinesisIndexTaskTest extends EasyMockSupport
     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"
-        ))),
+        new KinesisDataSourceMetadata(
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "4"))
+        ),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -1247,14 +1142,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
         new KinesisIndexTaskIOConfig(
             null,
             "sequence0",
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "2"
-            )),
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "4"
-            )),
+            new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "2"), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "4")),
             true,
             null,
             null,
@@ -1263,7 +1152,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport
             null,
             null,
             null,
-            null,
             false
         )
     );
@@ -1285,10 +1173,9 @@ public class KinesisIndexTaskTest extends EasyMockSupport
     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"
-        ))),
+        new KinesisDataSourceMetadata(
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "4"))
+        ),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -1324,14 +1211,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
         new KinesisIndexTaskIOConfig(
             null,
             "sequence0",
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "2"
-            )),
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "5"
-            )),
+            new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "2"), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "5")),
             true,
             null,
             null,
@@ -1340,7 +1221,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport
             null,
             null,
             null,
-            null,
             false
         )
     );
@@ -1390,14 +1270,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
         new KinesisIndexTaskIOConfig(
             null,
             "sequence0",
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "2"
-            )),
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "12"
-            )),
+            new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "2"), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "12")),
             true,
             null,
             null,
@@ -1406,7 +1280,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport
             null,
             null,
             null,
-            null,
             false
         )
     );
@@ -1435,13 +1308,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
     Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors());
     Assert.assertEquals(
         new KinesisDataSourceMetadata(
-            new SeekableStreamPartitions<>(
-                stream,
-                ImmutableMap.of(
-                    shardId1,
-                    "12"
-                )
-            )),
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "12"))
+        ),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -1500,14 +1368,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
         new KinesisIndexTaskIOConfig(
             null,
             "sequence0",
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "2"
-            )),
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "9"
-            )),
+            new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "2"), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "9")),
             true,
             null,
             null,
@@ -1516,7 +1378,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport
             null,
             null,
             null,
-            null,
             false
         )
     );
@@ -1589,14 +1450,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
         new KinesisIndexTaskIOConfig(
             null,
             "sequence0",
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "2"
-            )),
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "4"
-            )),
+            new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "2"), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "4")),
             true,
             null,
             null,
@@ -1605,7 +1460,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport
             null,
             null,
             null,
-            null,
             false
         )
     );
@@ -1614,14 +1468,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
         new KinesisIndexTaskIOConfig(
             null,
             "sequence0",
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "2"
-            )),
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "4"
-            )),
+            new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "2"), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "4")),
             true,
             null,
             null,
@@ -1630,7 +1478,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport
             null,
             null,
             null,
-            null,
             false
         )
     );
@@ -1658,13 +1505,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
     Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
     Assert.assertEquals(
         new KinesisDataSourceMetadata(
-            new SeekableStreamPartitions<>(
-                stream,
-                ImmutableMap.of(
-                    shardId1,
-                    "4"
-                )
-            )),
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "4"))
+        ),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -1700,14 +1542,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
         new KinesisIndexTaskIOConfig(
             null,
             "sequence0",
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "2"
-            )),
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "4"
-            )),
+            new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "2"), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "4")),
             true,
             null,
             null,
@@ -1716,7 +1552,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport
             null,
             null,
             null,
-            null,
             false
         )
     );
@@ -1725,14 +1560,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
         new KinesisIndexTaskIOConfig(
             null,
             "sequence1",
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "3"
-            )),
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "9"
-            )),
+            new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "3"), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "9")),
             true,
             null,
             null,
@@ -1741,7 +1570,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport
             null,
             null,
             null,
-            null,
             false
         )
     );
@@ -1768,14 +1596,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
     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"
-                )
-            )),
+        new KinesisDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "4"))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -1811,14 +1632,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
         new KinesisIndexTaskIOConfig(
             null,
             "sequence0",
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "2"
-            )),
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "4"
-            )),
+            new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "2"), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "4")),
             false,
             null,
             null,
@@ -1827,7 +1642,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport
             null,
             null,
             null,
-            null,
             false
         )
     );
@@ -1836,14 +1650,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
         new KinesisIndexTaskIOConfig(
             null,
             "sequence1",
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "3"
-            )),
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "9"
-            )),
+            new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "3"), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "9")),
             false,
             null,
             null,
@@ -1852,7 +1660,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport
             null,
             null,
             null,
-            null,
             false
         )
     );
@@ -1919,18 +1726,12 @@ public class KinesisIndexTaskTest extends EasyMockSupport
         new KinesisIndexTaskIOConfig(
             null,
             "sequence1",
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "2",
-                shardId0,
-                "0"
-            )),
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "4",
-                shardId0,
-                "1"
-            )),
+            new SeekableStreamStartSequenceNumbers<>(
+                stream,
+                ImmutableMap.of(shardId1, "2", shardId0, "0"),
+                ImmutableSet.of()
+            ),
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "4", shardId0, "1")),
             true,
             null,
             null,
@@ -1939,7 +1740,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport
             null,
             null,
             null,
-            null,
             false
         )
     );
@@ -1966,12 +1766,9 @@ public class KinesisIndexTaskTest extends EasyMockSupport
     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"
-        ))),
+        new KinesisDataSourceMetadata(
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "4", shardId0, "1"))
+        ),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -2013,14 +1810,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
         new KinesisIndexTaskIOConfig(
             null,
             "sequence0",
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "2"
-            )),
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "4"
-            )),
+            new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "2"), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "4")),
             true,
             null,
             null,
@@ -2029,7 +1820,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport
             null,
             null,
             null,
-            null,
             false
         )
     );
@@ -2038,14 +1828,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
         new KinesisIndexTaskIOConfig(
             null,
             "sequence1",
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId0,
-                "0"
-            )),
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId0,
-                "1"
-            )),
+            new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId0, "0"), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId0, "1")),
             true,
             null,
             null,
@@ -2054,7 +1838,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport
             null,
             null,
             null,
-            null,
             false
         )
     );
@@ -2084,12 +1867,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
     Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors());
     Assert.assertEquals(
         new KinesisDataSourceMetadata(
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "4",
-                shardId0,
-                "1"
-            ))),
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "4", shardId0, "1"))
+        ),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -2124,14 +1903,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
         new KinesisIndexTaskIOConfig(
             null,
             "sequence0",
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "2"
-            )),
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "5"
-            )),
+            new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "2"), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "5")),
             true,
             null,
             null,
@@ -2140,7 +1913,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport
             null,
             null,
             null,
-            null,
             false
         )
     );
@@ -2180,21 +1952,14 @@ public class KinesisIndexTaskTest extends EasyMockSupport
         new KinesisIndexTaskIOConfig(
             null,
             "sequence0",
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "2"
-            )),
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "5"
-            )),
+            new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "2"), ImmutableSet.of(shardId1)),
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "5")),
             true,
             null,
             null,
             "awsEndpoint",
             null,
             null,
-            ImmutableSet.of(shardId1),
             null,
             null,
             false
@@ -2228,10 +1993,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
     Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
     Assert.assertEquals(
         new KinesisDataSourceMetadata(
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "5"
-            ))),
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "5"))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -2274,14 +2036,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
         new KinesisIndexTaskIOConfig(
             null,
             "sequence0",
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "0"
-            )),
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "6"
-            )),
+            new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "0"), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "6")),
             true,
             null,
             null,
@@ -2290,14 +2046,14 @@ public class KinesisIndexTaskTest extends EasyMockSupport
             null,
             null,
             null,
-            null,
             false
         )
     );
 
-    final SeekableStreamPartitions<String, String> checkpoint1 = new SeekableStreamPartitions<>(
+    final SeekableStreamStartSequenceNumbers<String, String> checkpoint1 = new SeekableStreamStartSequenceNumbers<>(
         stream,
-        ImmutableMap.of(shardId1, "4")
+        ImmutableMap.of(shardId1, "4"),
+        ImmutableSet.of()
     );
 
     final ListenableFuture<TaskStatus> future1 = runTask(task1);
@@ -2342,21 +2098,14 @@ public class KinesisIndexTaskTest extends EasyMockSupport
         new KinesisIndexTaskIOConfig(
             null,
             "sequence0",
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "0"
-            )),
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "6"
-            )),
+            new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "0"), ImmutableSet.of(shardId1)),
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "6")),
             true,
             null,
             null,
             "awsEndpoint",
             null,
             null,
-            ImmutableSet.of(shardId1),
             null,
             null,
             false
@@ -2388,10 +2137,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
     Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6), publishedDescriptors());
     Assert.assertEquals(
         new KinesisDataSourceMetadata(
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "6"
-            ))),
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "6"))
+        ),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
   }
@@ -2416,14 +2163,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
         new KinesisIndexTaskIOConfig(
             null,
             "sequence0",
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "2"
-            )),
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "13"
-            )),
+            new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "2"), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "13")),
             true,
             null,
             null,
@@ -2432,7 +2173,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport
             null,
             null,
             null,
-            null,
             false
         )
     );
@@ -2494,12 +2234,9 @@ public class KinesisIndexTaskTest extends EasyMockSupport
     SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
     Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
     Assert.assertEquals(
-        new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(
+        new KinesisDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(
             stream,
-            ImmutableMap.of(
-                shardId1,
-                currentOffsets.get(shardId1)
-            )
+            ImmutableMap.of(shardId1, currentOffsets.get(shardId1))
         )),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
@@ -2509,7 +2246,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport
     Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2));
   }
 
-
   @Test(timeout = 60_000L)
   public void testRunContextSequenceAheadOfStartingOffsets() throws Exception
   {
@@ -2549,14 +2285,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
         new KinesisIndexTaskIOConfig(
             null,
             "sequence0",
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "0"
-            )),
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "4"
-            )),
+            new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "0"), ImmutableSet.of()),
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "4")),
             true,
             null,
             null,
@@ -2565,7 +2295,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport
             null,
             null,
             null,
-            null,
             false
         ),
         context
@@ -2585,10 +2314,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
     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"
-        ))),
+        new KinesisDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "4"))),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
 
@@ -2633,22 +2359,23 @@ public class KinesisIndexTaskTest extends EasyMockSupport
 
     replayAll();
 
-    final SeekableStreamPartitions<String, String> startPartitions = new SeekableStreamPartitions<>(
+    final SeekableStreamStartSequenceNumbers<String, String> startPartitions = new SeekableStreamStartSequenceNumbers<>(
         stream,
-        ImmutableMap.of(shardId1, "0")
+        ImmutableMap.of(shardId1, "0"),
+        ImmutableSet.of()
     );
 
-    final SeekableStreamPartitions<String, String> checkpoint1 = new SeekableStreamPartitions<>(
+    final SeekableStreamEndSequenceNumbers<String, String> checkpoint1 = new SeekableStreamEndSequenceNumbers<>(
         stream,
         ImmutableMap.of(shardId1, "4")
     );
 
-    final SeekableStreamPartitions<String, String> checkpoint2 = new SeekableStreamPartitions<>(
+    final SeekableStreamEndSequenceNumbers<String, String> checkpoint2 = new SeekableStreamEndSequenceNumbers<>(
         stream,
         ImmutableMap.of(shardId1, "9")
     );
 
-    final SeekableStreamPartitions<String, String> endPartitions = new SeekableStreamPartitions<>(
+    final SeekableStreamEndSequenceNumbers<String, String> endPartitions = new SeekableStreamEndSequenceNumbers<>(
         stream,
         ImmutableMap.of(shardId1, "100") // simulating unlimited
     );
@@ -2665,7 +2392,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport
         null,
         null,
         null,
-        null,
         false
     );
     final KinesisIndexTask normalReplica = createTask(
@@ -2743,10 +2469,9 @@ public class KinesisIndexTaskTest extends EasyMockSupport
     descriptors.add(SD(normalReplica, "2013/P1D", 0));
     Assert.assertEquals(descriptors, publishedDescriptors());
     Assert.assertEquals(
-        new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-            shardId1,
-            "9"
-        ))),
+        new KinesisDataSourceMetadata(
+            new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "9"))
+        ),
         metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
     );
   }
diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java
index 750b924..b42e356 100644
--- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java
+++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java
@@ -47,6 +47,7 @@ import org.apache.druid.indexing.kinesis.KinesisIndexTaskClient;
 import org.apache.druid.indexing.kinesis.KinesisIndexTaskClientFactory;
 import org.apache.druid.indexing.kinesis.KinesisIndexTaskIOConfig;
 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;
@@ -56,7 +57,8 @@ 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.SupervisorReport;
-import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
+import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers;
+import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers;
 import org.apache.druid.indexing.seekablestream.common.RecordSupplier;
 import org.apache.druid.indexing.seekablestream.common.StreamPartition;
 import org.apache.druid.indexing.seekablestream.supervisor.TaskReportData;
@@ -247,24 +249,24 @@ public class KinesisSupervisorTest extends EasyMockSupport
     Assert.assertFalse("minimumMessageTime", taskConfig.getMinimumMessageTime().isPresent());
     Assert.assertFalse("maximumMessageTime", taskConfig.getMaximumMessageTime().isPresent());
 
-    Assert.assertEquals(stream, taskConfig.getStartPartitions().getStream());
+    Assert.assertEquals(stream, taskConfig.getStartSequenceNumbers().getStream());
     Assert.assertEquals(
         "0",
-        taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1)
+        taskConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(shardId1)
     );
     Assert.assertEquals(
         "0",
-        taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId0)
+        taskConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(shardId0)
     );
 
-    Assert.assertEquals(stream, taskConfig.getEndPartitions().getStream());
+    Assert.assertEquals(stream, taskConfig.getEndSequenceNumbers().getStream());
     Assert.assertEquals(
-        SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER,
-        taskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(shardId1)
+        KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER,
+        taskConfig.getEndSequenceNumbers().getPartitionSequenceNumberMap().get(shardId1)
     );
     Assert.assertEquals(
-        SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER,
-        taskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(shardId0)
+        KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER,
+        taskConfig.getEndSequenceNumbers().getPartitionSequenceNumberMap().get(shardId0)
     );
   }
 
@@ -302,27 +304,27 @@ public class KinesisSupervisorTest extends EasyMockSupport
     verifyAll();
 
     KinesisIndexTask task1 = captured.getValues().get(0);
-    Assert.assertEquals(1, task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().size());
-    Assert.assertEquals(1, task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().size());
+    Assert.assertEquals(1, task1.getIOConfig().getStartSequenceNumbers().getPartitionSequenceNumberMap().size());
+    Assert.assertEquals(1, task1.getIOConfig().getEndSequenceNumbers().getPartitionSequenceNumberMap().size());
     Assert.assertEquals(
         "0",
-        task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(shardId1)
+        task1.getIOConfig().getStartSequenceNumbers().getPartitionSequenceNumberMap().get(shardId1)
     );
     Assert.assertEquals(
-        SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER,
-        task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(shardId1)
+        KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER,
+        task1.getIOConfig().getEndSequenceNumbers().getPartitionSequenceNumberMap().get(shardId1)
     );
 
     KinesisIndexTask task2 = captured.getValues().get(1);
-    Assert.assertEquals(1, task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().size());
-    Assert.assertEquals(1, task2.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().size());
+    Assert.assertEquals(1, task2.getIOConfig().getStartSequenceNumbers().getPartitionSequenceNumberMap().size());
+    Assert.assertEquals(1, task2.getIOConfig().getEndSequenceNumbers().getPartitionSequenceNumberMap().size());
     Assert.assertEquals(
         "0",
-        task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(shardId0)
+        task2.getIOConfig().getStartSequenceNumbers().getPartitionSequenceNumberMap().get(shardId0)
     );
     Assert.assertEquals(
-        SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER,
-        task2.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(shardId0)
+        KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER,
+        task2.getIOConfig().getEndSequenceNumbers().getPartitionSequenceNumberMap().get(shardId0)
     );
   }
 
@@ -359,43 +361,43 @@ public class KinesisSupervisorTest extends EasyMockSupport
     verifyAll();
 
     KinesisIndexTask task1 = captured.getValues().get(0);
-    Assert.assertEquals(2, task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().size());
-    Assert.assertEquals(2, task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().size());
+    Assert.assertEquals(2, task1.getIOConfig().getStartSequenceNumbers().getPartitionSequenceNumberMap().size());
+    Assert.assertEquals(2, task1.getIOConfig().getEndSequenceNumbers().getPartitionSequenceNumberMap().size());
     Assert.assertEquals(
         "0",
-        task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(shardId0)
+        task1.getIOConfig().getStartSequenceNumbers().getPartitionSequenceNumberMap().get(shardId0)
     );
     Assert.assertEquals(
-        SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER,
-        task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(shardId0)
+        KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER,
+        task1.getIOConfig().getEndSequenceNumbers().getPartitionSequenceNumberMap().get(shardId0)
     );
     Assert.assertEquals(
         "0",
-        task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(shardId1)
+        task1.getIOConfig().getStartSequenceNumbers().getPartitionSequenceNumberMap().get(shardId1)
     );
     Assert.assertEquals(
-        SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER,
-        task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(shardId1)
+        KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER,
+        task1.getIOConfig().getEndSequenceNumbers().getPartitionSequenceNumberMap().get(shardId1)
     );
 
     KinesisIndexTask task2 = captured.getValues().get(1);
-    Assert.assertEquals(2, task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().size());
-    Assert.assertEquals(2, task2.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().size());
+    Assert.assertEquals(2, task2.getIOConfig().getStartSequenceNumbers().getPartitionSequenceNumberMap().size());
+    Assert.assertEquals(2, task2.getIOConfig().getEndSequenceNumbers().getPartitionSequenceNumberMap().size());
     Assert.assertEquals(
         "0",
-        task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(shardId0)
+        task2.getIOConfig().getStartSequenceNumbers().getPartitionSequenceNumberMap().get(shardId0)
     );
     Assert.assertEquals(
-        SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER,
-        task2.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(shardId0)
+        KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER,
+        task2.getIOConfig().getEndSequenceNumbers().getPartitionSequenceNumberMap().get(shardId0)
     );
     Assert.assertEquals(
         "0",
-        task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(shardId1)
+        task2.getIOConfig().getStartSequenceNumbers().getPartitionSequenceNumberMap().get(shardId1)
     );
     Assert.assertEquals(
-        SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER,
-        task2.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(shardId1)
+        KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER,
+        task2.getIOConfig().getEndSequenceNumbers().getPartitionSequenceNumberMap().get(shardId1)
     );
 
   }
@@ -527,12 +529,7 @@ public class KinesisSupervisorTest extends EasyMockSupport
     EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes();
     EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
         new KinesisDataSourceMetadata(
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "2",
-                shardId0,
-                "1"
-            ))
+            new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "2", shardId0, "1"), ImmutableSet.of())
         )
     ).anyTimes();
 
@@ -548,11 +545,11 @@ public class KinesisSupervisorTest extends EasyMockSupport
     Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName());
     Assert.assertEquals(
         "2",
-        taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1)
+        taskConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(shardId1)
     );
     Assert.assertEquals(
         "1",
-        taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId0)
+        taskConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(shardId0)
     );
   }
 
@@ -578,12 +575,7 @@ public class KinesisSupervisorTest extends EasyMockSupport
     EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes();
     EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
         new KinesisDataSourceMetadata(
-            new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-                shardId1,
-                "101",
-                shardId0,
-                "-1"
-            ))
+            new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "101", shardId0, "-1"), ImmutableSet.of())
         )
     ).anyTimes();
     replayAll();
@@ -614,14 +606,8 @@ public class KinesisSupervisorTest extends EasyMockSupport
         "id1",
         DATASOURCE,
         1,
-        new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-            shardId0,
-            "0"
-        )),
-        new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-            shardId0,
-            "1"
-        )),
+        new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId0, "0"), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId0, "1")),
         null,
         null
     );
@@ -631,18 +617,11 @@ public class KinesisSupervisorTest extends EasyMockSupport
         "id2",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-            shardId0,
-            "0",
-            shardId1,
-            "0"
-        )),
-        new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-            shardId0,
-            "1",
-            shardId1,
-            "12"
-        )),
+        new SeekableStreamStartSequenceNumbers<>(
+            stream,
+            ImmutableMap.of(shardId0, "0", shardId1, "0"), ImmutableSet.of(shardId0, shardId1)
+        ),
+        new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId0, "1", shardId1, "12")),
         null,
         null
     );
@@ -652,18 +631,11 @@ public class KinesisSupervisorTest extends EasyMockSupport
         "id3",
         DATASOURCE,
         1,
-        new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-            shardId0,
-            "0",
-            shardId1,
-            "1"
-        )),
-        new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-            shardId0,
-            "1",
-            shardId1,
-            "11"
-        )),
+        new SeekableStreamStartSequenceNumbers<>(
+            stream,
+            ImmutableMap.of(shardId0, "0", shardId1, "1"), ImmutableSet.of(shardId0, shardId1)
+        ),
+        new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId0, "1", shardId1, "11")),
         null,
         null
     );
@@ -673,18 +645,12 @@ public class KinesisSupervisorTest extends EasyMockSupport
         "id4",
         "other-datasource",
         2,
-        new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-            shardId0,
-            "0",
-            shardId1,
-            "0"
-        )),
-        new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-            shardId0,
-            "1",
-            shardId1,
-            "12"
-        )),
+        new SeekableStreamStartSequenceNumbers<>(
+            stream,
+            ImmutableMap.of(shardId0, "0", shardId1, "0"),
+            ImmutableSet.of(shardId0, shardId1)
+        ),
+        new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId0, "1", shardId1, "12")),
         null,
         null
     );
@@ -770,14 +736,8 @@ public class KinesisSupervisorTest extends EasyMockSupport
         "id1",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-            shardId1,
-            "0"
-        )),
-        new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-            shardId1,
-            "12"
-        )),
+        new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "0"), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "12")),
         null,
         null
     );
@@ -785,14 +745,8 @@ public class KinesisSupervisorTest extends EasyMockSupport
         "id2",
         DATASOURCE,
         1,
-        new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-            shardId0,
-            "0"
-        )),
-        new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-            shardId0,
-            "1"
-        )),
+        new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId0, "0"), ImmutableSet.of(shardId0)),
+        new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId0, "1")),
         null,
         null
     );
@@ -800,18 +754,11 @@ public class KinesisSupervisorTest extends EasyMockSupport
         "id3",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-            shardId0,
-            "0",
-            shardId1,
-            "0"
-        )),
-        new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-            shardId0,
-            "1",
-            shardId1,
-            "12"
-        )),
+        new SeekableStreamStartSequenceNumbers<>(
+            stream,
+            ImmutableMap.of(shardId0, "0", shardId1, "0"), ImmutableSet.of(shardId0, shardId1)
+        ),
+        new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId0, "1", shardId1, "12")),
         null,
         null
     );
@@ -819,14 +766,8 @@ public class KinesisSupervisorTest extends EasyMockSupport
         "id4",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-            shardId0,
-            "0"
-        )),
-        new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-            shardId0,
-            "1"
-        )),
+        new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId0, "0"), ImmutableSet.of(shardId0)),
+        new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId0, "1")),
         null,
         null
     );
@@ -834,14 +775,8 @@ public class KinesisSupervisorTest extends EasyMockSupport
         "id5",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-            shardId0,
-            "0"
-        )),
-        new SeekableStreamPartitions<>(stream, ImmutableMap.of(
-            shardId0,
-            "1"
-        )),
+        new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId0, "0"), ImmutableSet.of(shardId0)),
+        new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId0, "1")),
         null,
         null
     );
@@ -1030,18 +965,16 @@ public class KinesisSupervisorTest extends EasyMockSupport
         "id1",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            "0",
-            shardId0,
-            "0"
-        )),
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER,
-            shardId0,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER
-        )),
+        new SeekableStreamStartSequenceNumbers<>("stream", ImmutableMap.of(shardId1, "0", shardId0, "0"), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(
+                shardId1,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER,
+                shardId0,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER
+            )
+        ),
         now,
         maxi
     );
@@ -1051,7 +984,7 @@ public class KinesisSupervisorTest extends EasyMockSupport
     Capture<Task> captured = Capture.newInstance();
     EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
     EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes();
-    EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes();
+    EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes();
     EasyMock.expect(taskStorage.getActiveTasks()).andReturn(existingTasks).anyTimes();
     EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes();
     EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes();
@@ -1279,17 +1212,15 @@ public class KinesisSupervisorTest extends EasyMockSupport
     supervisorRecordSupplier.seek(anyObject(), anyString());
     expectLastCall().anyTimes();
 
-    Capture<Task> captured = Capture.newInstance(CaptureType.ALL);
+    final Capture<Task> firstTasks = Capture.newInstance(CaptureType.ALL);
     EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
     EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes();
     EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes();
     EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes();
     EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
-        new KinesisDataSourceMetadata(
-            null
-        )
+        new KinesisDataSourceMetadata(null)
     ).anyTimes();
-    EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(4);
+    EasyMock.expect(taskQueue.add(EasyMock.capture(firstTasks))).andReturn(true).times(4);
     taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class));
     replayAll();
 
@@ -1297,14 +1228,14 @@ public class KinesisSupervisorTest extends EasyMockSupport
     supervisor.runInternal();
     verifyAll();
 
-    List<Task> tasks = captured.getValues();
+    final List<Task> tasks = firstTasks.getValues();
     Collection workItems = new ArrayList<>();
     for (Task task : tasks) {
       workItems.add(new TestTaskRunnerWorkItem(task, null, location));
     }
 
     EasyMock.reset(taskStorage, taskRunner, taskClient, taskQueue);
-    captured = Capture.newInstance(CaptureType.ALL);
+    final Capture<Task> secondTasks = Capture.newInstance(CaptureType.ALL);
     EasyMock.expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes();
     for (Task task : tasks) {
       EasyMock.expect(taskStorage.getStatus(task.getId()))
@@ -1347,7 +1278,7 @@ public class KinesisSupervisorTest extends EasyMockSupport
             EasyMock.eq(true)
         )
     ).andReturn(Futures.immediateFuture(true)).times(2);
-    EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(2);
+    EasyMock.expect(taskQueue.add(EasyMock.capture(secondTasks))).andReturn(true).times(2);
 
     TreeMap<Integer, Map<String, String>> checkpoints1 = new TreeMap<>();
     checkpoints1.put(0, ImmutableMap.of(
@@ -1371,7 +1302,7 @@ public class KinesisSupervisorTest extends EasyMockSupport
     supervisor.runInternal();
     verifyAll();
 
-    for (Task task : captured.getValues()) {
+    for (Task task : secondTasks.getValues()) {
       KinesisIndexTask KinesisIndexTask = (KinesisIndexTask) task;
       Assert.assertEquals(dataSchema, KinesisIndexTask.getDataSchema());
       Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), KinesisIndexTask.getTuningConfig());
@@ -1380,14 +1311,19 @@ public class KinesisSupervisorTest extends EasyMockSupport
       Assert.assertEquals("sequenceName-1", taskConfig.getBaseSequenceName());
       Assert.assertTrue("isUseTransaction", taskConfig.isUseTransaction());
 
-      Assert.assertEquals(stream, taskConfig.getStartPartitions().getStream());
+      Assert.assertEquals(stream, taskConfig.getStartSequenceNumbers().getStream());
       Assert.assertEquals(
           "3",
-          taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1)
+          taskConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(shardId1)
       );
       Assert.assertEquals(
           "1",
-          taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId0)
+          taskConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(shardId0)
+      );
+      // start sequenceNumbers should be exclusive for the second batch of tasks
+      Assert.assertEquals(
+          ImmutableSet.of("0", "1"),
+          ((KinesisIndexTask) task).getIOConfig().getStartSequenceNumbers().getExclusivePartitions()
       );
     }
   }
@@ -1416,18 +1352,16 @@ public class KinesisSupervisorTest extends EasyMockSupport
         "id1",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            "0",
-            shardId0,
-            "0"
-        )),
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER,
-            shardId0,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER
-        )),
+        new SeekableStreamStartSequenceNumbers<>("stream", ImmutableMap.of(shardId1, "0", shardId0, "0"), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(
+                shardId1,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER,
+                shardId0,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER
+            )
+        ),
         null,
         null
     );
@@ -1522,24 +1456,24 @@ public class KinesisSupervisorTest extends EasyMockSupport
     Assert.assertTrue("isUseTransaction", capturedTaskConfig.isUseTransaction());
 
     // check that the new task was created with starting sequences matching where the publishing task finished
-    Assert.assertEquals(stream, capturedTaskConfig.getStartPartitions().getStream());
+    Assert.assertEquals(stream, capturedTaskConfig.getStartSequenceNumbers().getStream());
     Assert.assertEquals(
         "2",
-        capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1)
+        capturedTaskConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(shardId1)
     );
     Assert.assertEquals(
         "1",
-        capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId0)
+        capturedTaskConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(shardId0)
     );
 
-    Assert.assertEquals(stream, capturedTaskConfig.getEndPartitions().getStream());
+    Assert.assertEquals(stream, capturedTaskConfig.getEndSequenceNumbers().getStream());
     Assert.assertEquals(
-        SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER,
-        capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(shardId1)
+        KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER,
+        capturedTaskConfig.getEndSequenceNumbers().getPartitionSequenceNumberMap().get(shardId1)
     );
     Assert.assertEquals(
-        SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER,
-        capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(shardId0)
+        KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER,
+        capturedTaskConfig.getEndSequenceNumbers().getPartitionSequenceNumberMap().get(shardId0)
     );
   }
 
@@ -1566,18 +1500,16 @@ public class KinesisSupervisorTest extends EasyMockSupport
         "id1",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            "0",
-            shardId0,
-            "0"
-        )),
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER,
-            shardId0,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER
-        )),
+        new SeekableStreamStartSequenceNumbers<>("stream", ImmutableMap.of(shardId1, "0", shardId0, "0"), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(
+                shardId1,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER,
+                shardId0,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER
+            )
+        ),
         null,
         null
     );
@@ -1661,24 +1593,24 @@ public class KinesisSupervisorTest extends EasyMockSupport
     Assert.assertTrue("isUseTransaction", capturedTaskConfig.isUseTransaction());
 
     // check that the new task was created with starting sequences matching where the publishing task finished
-    Assert.assertEquals(stream, capturedTaskConfig.getStartPartitions().getStream());
+    Assert.assertEquals(stream, capturedTaskConfig.getStartSequenceNumbers().getStream());
     Assert.assertEquals(
         "2",
-        capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1)
+        capturedTaskConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(shardId1)
     );
     Assert.assertEquals(
         "1",
-        capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId0)
+        capturedTaskConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(shardId0)
     );
 
-    Assert.assertEquals(stream, capturedTaskConfig.getEndPartitions().getStream());
+    Assert.assertEquals(stream, capturedTaskConfig.getEndSequenceNumbers().getStream());
     Assert.assertEquals(
-        SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER,
-        capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(shardId1)
+        KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER,
+        capturedTaskConfig.getEndSequenceNumbers().getPartitionSequenceNumberMap().get(shardId1)
     );
     Assert.assertEquals(
-        SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER,
-        capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(shardId0)
+        KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER,
+        capturedTaskConfig.getEndSequenceNumbers().getPartitionSequenceNumberMap().get(shardId0)
     );
   }
 
@@ -1707,21 +1639,20 @@ public class KinesisSupervisorTest extends EasyMockSupport
         "id1",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>(
+        new SeekableStreamStartSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(shardId1, "0", shardId0, "0"),
+            ImmutableSet.of()
+        ),
+        new SeekableStreamEndSequenceNumbers<>(
             "stream",
             ImmutableMap.of(
                 shardId1,
-                "0",
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER,
                 shardId0,
-                "0"
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER
             )
         ),
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER,
-            shardId0,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER
-        )),
         null,
         null
     );
@@ -1730,18 +1661,19 @@ public class KinesisSupervisorTest extends EasyMockSupport
         "id2",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            "2",
-            shardId0,
-            "1"
-        )),
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER,
-            shardId0,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER
-        )),
+        new SeekableStreamStartSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(shardId1, "2", shardId0, "1"), ImmutableSet.of(shardId0, shardId1)
+        ),
+        new SeekableStreamEndSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(
+                shardId1,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER,
+                shardId0,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER
+            )
+        ),
         null,
         null
     );
@@ -2030,10 +1962,10 @@ public class KinesisSupervisorTest extends EasyMockSupport
       KinesisIndexTaskIOConfig taskConfig = ((KinesisIndexTask) task).getIOConfig();
       Assert.assertEquals(
           "0",
-          taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1)
+          taskConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(shardId1)
       );
       Assert.assertNull(
-          taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId0)
+          taskConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(shardId0)
       );
     }
   }
@@ -2151,7 +2083,7 @@ public class KinesisSupervisorTest extends EasyMockSupport
       KinesisIndexTaskIOConfig taskConfig = ((KinesisIndexTask) task).getIOConfig();
       Assert.assertEquals(
           "0",
-          taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1)
+          taskConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(shardId1)
       );
     }
   }
@@ -2206,18 +2138,16 @@ public class KinesisSupervisorTest extends EasyMockSupport
         "id1",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            "0",
-            shardId0,
-            "0"
-        )),
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER,
-            shardId0,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER
-        )),
+        new SeekableStreamStartSequenceNumbers<>("stream", ImmutableMap.of(shardId1, "0", shardId0, "0"), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(
+                shardId1,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER,
+                shardId0,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER
+            )
+        ),
         null,
         null
     );
@@ -2226,18 +2156,25 @@ public class KinesisSupervisorTest extends EasyMockSupport
         "id2",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            "3",
-            shardId0,
-            "1"
-        )),
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER,
-            shardId0,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER
-        )),
+        new SeekableStreamStartSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(
+                shardId1,
+                "3",
+                shardId0,
+                "1"
+            ),
+            ImmutableSet.of(shardId0, shardId1)
+        ),
+        new SeekableStreamEndSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(
+                shardId1,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER,
+                shardId0,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER
+            )
+        ),
         null,
         null
     );
@@ -2246,18 +2183,20 @@ public class KinesisSupervisorTest extends EasyMockSupport
         "id3",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            "3",
-            shardId0,
-            "1"
-        )),
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER,
-            shardId0,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER
-        )),
+        new SeekableStreamStartSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(shardId1, "3", shardId0, "1"),
+            ImmutableSet.of(shardId0, shardId1)
+        ),
+        new SeekableStreamEndSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(
+                shardId1,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER,
+                shardId0,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER
+            )
+        ),
         null,
         null
     );
@@ -2392,35 +2331,44 @@ public class KinesisSupervisorTest extends EasyMockSupport
     Capture<String> captureDataSource = EasyMock.newCapture();
     Capture<DataSourceMetadata> captureDataSourceMetadata = EasyMock.newCapture();
 
-    KinesisDataSourceMetadata KinesisDataSourceMetadata = new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(
-        stream,
-        ImmutableMap.of(
-            shardId1,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER,
-            shardId0,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER
+    KinesisDataSourceMetadata kinesisDataSourceMetadata = new KinesisDataSourceMetadata(
+        new SeekableStreamStartSequenceNumbers<>(
+            stream,
+            ImmutableMap.of(
+                shardId1,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER,
+                shardId0,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER
+            ),
+            ImmutableSet.of()
         )
-    ));
+    );
 
-    KinesisDataSourceMetadata resetMetadata = new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(
-        stream,
-        ImmutableMap.of(
-            shardId0,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER
+    KinesisDataSourceMetadata resetMetadata = new KinesisDataSourceMetadata(
+        new SeekableStreamStartSequenceNumbers<>(
+            stream,
+            ImmutableMap.of(
+                shardId0,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER
+            ),
+            ImmutableSet.of()
         )
-    ));
+    );
 
-    KinesisDataSourceMetadata expectedMetadata = new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(
-        stream,
-        ImmutableMap.of(
-            shardId1,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER
+    KinesisDataSourceMetadata expectedMetadata = new KinesisDataSourceMetadata(
+        new SeekableStreamStartSequenceNumbers<>(
+            stream,
+            ImmutableMap.of(
+                shardId1,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER
+            ),
+            ImmutableSet.of()
         )
-    ));
+    );
 
     EasyMock.reset(indexerMetadataStorageCoordinator);
     EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE))
-            .andReturn(KinesisDataSourceMetadata);
+            .andReturn(kinesisDataSourceMetadata);
     EasyMock.expect(indexerMetadataStorageCoordinator.resetDataSourceMetadata(
         EasyMock.capture(captureDataSource),
         EasyMock.capture(captureDataSourceMetadata)
@@ -2448,7 +2396,7 @@ public class KinesisSupervisorTest extends EasyMockSupport
     supervisor = getSupervisor(1, 1, true, "PT1H", null, null);
     EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
     EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes();
-    EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes();
+    EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes();
     EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes();
     taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class));
     replayAll();
@@ -2457,13 +2405,13 @@ public class KinesisSupervisorTest extends EasyMockSupport
     supervisor.runInternal();
     verifyAll();
 
-    KinesisDataSourceMetadata resetMetadata = new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(
-        stream,
-        ImmutableMap.of(
-            shardId0,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER
+    KinesisDataSourceMetadata resetMetadata = new KinesisDataSourceMetadata(
+        new SeekableStreamStartSequenceNumbers<>(
+            stream,
+            ImmutableMap.of(shardId0, KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER),
+            ImmutableSet.of()
         )
-    ));
+    );
 
     EasyMock.reset(indexerMetadataStorageCoordinator);
     // no DataSourceMetadata in metadata store
@@ -2499,21 +2447,16 @@ public class KinesisSupervisorTest extends EasyMockSupport
         "id1",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-
-            "0",
-
-            shardId0,
-
-            "0"
-        )),
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER,
-            shardId0,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER
-        )),
+        new SeekableStreamStartSequenceNumbers<>("stream", ImmutableMap.of(shardId1, "0", shardId0, "0"), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(
+                shardId1,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER,
+                shardId0,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER
+            )
+        ),
         null,
         null
     );
@@ -2522,21 +2465,20 @@ public class KinesisSupervisorTest extends EasyMockSupport
         "id2",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-
-            "3",
-
-            shardId0,
-
-            "1"
-        )),
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER,
-            shardId0,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER
-        )),
+        new SeekableStreamStartSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(shardId1, "3", shardId0, "1"),
+            ImmutableSet.of(shardId0, shardId1)
+        ),
+        new SeekableStreamEndSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(
+                shardId1,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER,
+                shardId0,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER
+            )
+        ),
         null,
         null
     );
@@ -2545,21 +2487,20 @@ public class KinesisSupervisorTest extends EasyMockSupport
         "id3",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-
-            "3",
-
-            shardId0,
-
-            "1"
-        )),
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER,
-            shardId0,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER
-        )),
+        new SeekableStreamStartSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(shardId1, "3", shardId0, "1"),
+            ImmutableSet.of(shardId0, shardId1)
+        ),
+        new SeekableStreamEndSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(
+                shardId1,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER,
+                shardId0,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER
+            )
+        ),
         null,
         null
     );
@@ -2646,18 +2587,20 @@ public class KinesisSupervisorTest extends EasyMockSupport
         "id1",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            "0",
-            shardId0,
-            "0"
-        )),
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER,
-            shardId0,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER
-        )),
+        new SeekableStreamStartSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(shardId1, "0", shardId0, "0"),
+            ImmutableSet.of()
+        ),
+        new SeekableStreamEndSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(
+                shardId1,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER,
+                shardId0,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER
+            )
+        ),
         null,
         null
     );
@@ -2666,18 +2609,20 @@ public class KinesisSupervisorTest extends EasyMockSupport
         "id2",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            "10",
-            shardId0,
-            "20"
-        )),
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER,
-            shardId0,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER
-        )),
+        new SeekableStreamStartSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(shardId1, "10", shardId0, "20"),
+            ImmutableSet.of(shardId0, shardId1)
+        ),
+        new SeekableStreamEndSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(
+                shardId1,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER,
+                shardId0,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER
+            )
+        ),
         null,
         null
     );
@@ -2686,18 +2631,20 @@ public class KinesisSupervisorTest extends EasyMockSupport
         "id3",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            "10",
-            shardId0,
-            "20"
-        )),
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER,
-            shardId0,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER
-        )),
+        new SeekableStreamStartSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(shardId1, "10", shardId0, "20"),
+            ImmutableSet.of(shardId0, shardId1)
+        ),
+        new SeekableStreamEndSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(
+                shardId1,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER,
+                shardId0,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER
+            )
+        ),
         null,
         null
     );
@@ -2785,18 +2732,20 @@ public class KinesisSupervisorTest extends EasyMockSupport
         "id1",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            "0",
-            shardId0,
-            "0"
-        )),
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER,
-            shardId0,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER
-        )),
+        new SeekableStreamStartSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(shardId1, "0", shardId0, "0"),
+            ImmutableSet.of()
+        ),
+        new SeekableStreamEndSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(
+                shardId1,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER,
+                shardId0,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER
+            )
+        ),
         null,
         null
     );
@@ -2805,18 +2754,20 @@ public class KinesisSupervisorTest extends EasyMockSupport
         "id2",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            "10",
-            shardId0,
-            "20"
-        )),
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER,
-            shardId0,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER
-        )),
+        new SeekableStreamStartSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(shardId1, "10", shardId0, "20"),
+            ImmutableSet.of(shardId0, shardId1)
+        ),
+        new SeekableStreamEndSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(
+                shardId1,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER,
+                shardId0,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER
+            )
+        ),
         null,
         null
     );
@@ -2825,18 +2776,20 @@ public class KinesisSupervisorTest extends EasyMockSupport
         "id3",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            "10",
-            shardId0,
-            "20"
-        )),
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER,
-            shardId0,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER
-        )),
+        new SeekableStreamStartSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(shardId1, "10", shardId0, "20"),
+            ImmutableSet.of(shardId0, shardId1)
+        ),
+        new SeekableStreamEndSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(
+                shardId1,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER,
+                shardId0,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER
+            )
+        ),
         null,
         null
     );
@@ -2885,12 +2838,7 @@ public class KinesisSupervisorTest extends EasyMockSupport
     expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime));
 
     final TreeMap<Integer, Map<String, String>> checkpoints = new TreeMap<>();
-    checkpoints.put(0, ImmutableMap.of(
-        shardId1,
-        "10",
-        shardId0,
-        "20"
-    ));
+    checkpoints.put(0, ImmutableMap.of(shardId1, "10", shardId0, "20"));
     expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean()))
         .andReturn(Futures.immediateFuture(checkpoints))
         .times(1);
@@ -2912,8 +2860,10 @@ public class KinesisSupervisorTest extends EasyMockSupport
     supervisor.checkpoint(
         0,
         ((KinesisIndexTask) id1).getIOConfig().getBaseSequenceName(),
-        new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, checkpoints.get(0))),
-        new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, fakeCheckpoints))
+        new KinesisDataSourceMetadata(
+            new SeekableStreamStartSequenceNumbers<>(stream, checkpoints.get(0), checkpoints.get(0).keySet())
+        ),
+        new KinesisDataSourceMetadata(new SeekableStreamStartSequenceNumbers<>(stream, fakeCheckpoints, ImmutableSet.of()))
     );
 
     while (supervisor.getNoticesQueueSize() > 0) {
@@ -2952,18 +2902,20 @@ public class KinesisSupervisorTest extends EasyMockSupport
         "id1",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            "0",
-            shardId0,
-            "0"
-        )),
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER,
-            shardId0,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER
-        )),
+        new SeekableStreamStartSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(shardId1, "0", shardId0, "0"),
+            ImmutableSet.of()
+        ),
+        new SeekableStreamEndSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(
+                shardId1,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER,
+                shardId0,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER
+            )
+        ),
         null,
         null
     );
@@ -2972,18 +2924,20 @@ public class KinesisSupervisorTest extends EasyMockSupport
         "id2",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            "10",
-            shardId0,
-            "20"
-        )),
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER,
-            shardId0,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER
-        )),
+        new SeekableStreamStartSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(shardId1, "10", shardId0, "20"),
+            ImmutableSet.of(shardId0, shardId1)
+        ),
+        new SeekableStreamEndSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(
+                shardId1,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER,
+                shardId0,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER
+            )
+        ),
         null,
         null
     );
@@ -2992,18 +2946,20 @@ public class KinesisSupervisorTest extends EasyMockSupport
         "id3",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            "10",
-            shardId0,
-            "20"
-        )),
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER,
-            shardId0,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER
-        )),
+        new SeekableStreamStartSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(shardId1, "10", shardId0, "20"),
+            ImmutableSet.of(shardId0, shardId1)
+        ),
+        new SeekableStreamEndSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(
+                shardId1,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER,
+                shardId0,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER
+            )
+        ),
         null,
         null
     );
@@ -3029,8 +2985,8 @@ public class KinesisSupervisorTest extends EasyMockSupport
     supervisor.checkpoint(
         0,
         ((KinesisIndexTask) id1).getIOConfig().getBaseSequenceName(),
-        new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, Collections.emptyMap())),
-        new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, Collections.emptyMap()))
+        new KinesisDataSourceMetadata(new SeekableStreamStartSequenceNumbers<>(stream, Collections.emptyMap(), ImmutableSet.of())),
+        new KinesisDataSourceMetadata(new SeekableStreamStartSequenceNumbers<>(stream, Collections.emptyMap(), ImmutableSet.of()))
     );
 
     while (supervisor.getNoticesQueueSize() > 0) {
@@ -3062,14 +3018,11 @@ public class KinesisSupervisorTest extends EasyMockSupport
         "id1",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            "0"
-        )),
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER
-        )),
+        new SeekableStreamStartSequenceNumbers<>("stream", ImmutableMap.of(shardId1, "0"), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(shardId1, KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER)
+        ),
         null,
         null
     );
@@ -3078,14 +3031,11 @@ public class KinesisSupervisorTest extends EasyMockSupport
         "id2",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            "0"
-        )),
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER
-        )),
+        new SeekableStreamStartSequenceNumbers<>("stream", ImmutableMap.of(shardId1, "0"), ImmutableSet.of(shardId1)),
+        new SeekableStreamEndSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(shardId1, KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER)
+        ),
         null,
         null
     );
@@ -3094,14 +3044,11 @@ public class KinesisSupervisorTest extends EasyMockSupport
         "id3",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            "0"
-        )),
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER
-        )),
+        new SeekableStreamStartSequenceNumbers<>("stream", ImmutableMap.of(shardId1, "0"), ImmutableSet.of(shardId1)),
+        new SeekableStreamEndSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(shardId1, KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER)
+        ),
         null,
         null
     );
@@ -3154,8 +3101,8 @@ public class KinesisSupervisorTest extends EasyMockSupport
     supervisor.checkpoint(
         null,
         ((KinesisIndexTask) id1).getIOConfig().getBaseSequenceName(),
-        new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, checkpoints.get(0))),
-        new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, newCheckpoints.get(0)))
+        new KinesisDataSourceMetadata(new SeekableStreamStartSequenceNumbers<>(stream, checkpoints.get(0), ImmutableSet.of())),
+        new KinesisDataSourceMetadata(new SeekableStreamStartSequenceNumbers<>(stream, newCheckpoints.get(0), ImmutableSet.of()))
     );
 
     while (supervisor.getNoticesQueueSize() > 0) {
@@ -3222,18 +3169,16 @@ public class KinesisSupervisorTest extends EasyMockSupport
         "id1",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            "0",
-            shardId0,
-            "0"
-        )),
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER,
-            shardId0,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER
-        )),
+        new SeekableStreamStartSequenceNumbers<>("stream", ImmutableMap.of(shardId1, "0", shardId0, "0"), ImmutableSet.of()),
+        new SeekableStreamEndSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(
+                shardId1,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER,
+                shardId0,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER
+            )
+        ),
         null,
         null
     );
@@ -3242,18 +3187,20 @@ public class KinesisSupervisorTest extends EasyMockSupport
         "id2",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            "3",
-            shardId0,
-            "1"
-        )),
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER,
-            shardId0,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER
-        )),
+        new SeekableStreamStartSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(shardId1, "3", shardId0, "1"),
+            ImmutableSet.of(shardId0, shardId1)
+        ),
+        new SeekableStreamEndSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(
+                shardId1,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER,
+                shardId0,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER
+            )
+        ),
         null,
         null
     );
@@ -3262,18 +3209,20 @@ public class KinesisSupervisorTest extends EasyMockSupport
         "id3",
         DATASOURCE,
         0,
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            "3",
-            shardId0,
-            "1"
-        )),
-        new SeekableStreamPartitions<>("stream", ImmutableMap.of(
-            shardId1,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER,
-            shardId0,
-            SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER
-        )),
+        new SeekableStreamStartSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(shardId1, "3", shardId0, "1"),
+            ImmutableSet.of(shardId0, shardId1)
+        ),
+        new SeekableStreamEndSequenceNumbers<>(
+            "stream",
+            ImmutableMap.of(
+                shardId1,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER,
+                shardId0,
+                KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER
+            )
+        ),
         null,
         null
     );
@@ -3590,8 +3539,8 @@ public class KinesisSupervisorTest extends EasyMockSupport
       String id,
       String dataSource,
       int taskGroupId,
-      SeekableStreamPartitions<String, String> startPartitions,
-      SeekableStreamPartitions<String, String> endPartitions,
+      SeekableStreamStartSequenceNumbers<String, String> startPartitions,
+      SeekableStreamEndSequenceNumbers<String, String> endPartitions,
       DateTime minimumMessageTime,
       DateTime maximumMessageTime
   )
@@ -3614,7 +3563,6 @@ public class KinesisSupervisorTest extends EasyMockSupport
             null,
             null,
             null,
-            null,
             false
         ),
         Collections.emptyMap(),
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java
index b9e8d9a..d0c4f98 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java
@@ -23,26 +23,24 @@ import com.fasterxml.jackson.annotation.JsonProperty;
 import org.apache.druid.indexing.overlord.DataSourceMetadata;
 import org.apache.druid.java.util.common.IAE;
 
-import java.util.HashMap;
-import java.util.Map;
 import java.util.Objects;
 
 public abstract class SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType>
     implements DataSourceMetadata
 {
-  private final SeekableStreamPartitions<PartitionIdType, SequenceOffsetType> seekableStreamPartitions;
+  private final SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> seekableStreamSequenceNumbers;
 
   public SeekableStreamDataSourceMetadata(
-      SeekableStreamPartitions<PartitionIdType, SequenceOffsetType> seekableStreamPartitions
+      SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> seekableStreamSequenceNumbers
   )
   {
-    this.seekableStreamPartitions = seekableStreamPartitions;
+    this.seekableStreamSequenceNumbers = seekableStreamSequenceNumbers;
   }
 
   @JsonProperty("partitions")
-  public SeekableStreamPartitions<PartitionIdType, SequenceOffsetType> getSeekableStreamPartitions()
+  public SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> getSeekableStreamSequenceNumbers()
   {
-    return seekableStreamPartitions;
+    return seekableStreamSequenceNumbers;
   }
 
   @Override
@@ -64,7 +62,7 @@ public abstract class SeekableStreamDataSourceMetadata<PartitionIdType, Sequence
   @Override
   public DataSourceMetadata plus(DataSourceMetadata other)
   {
-    if (!(this.getClass().isInstance(other))) {
+    if (this.getClass() != other.getClass()) {
       throw new IAE(
           "Expected instance of %s, got %s",
           this.getClass().getCanonicalName(),
@@ -72,36 +70,17 @@ public abstract class SeekableStreamDataSourceMetadata<PartitionIdType, Sequence
       );
     }
 
-    @SuppressWarnings("unchecked")
-    final SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType> that = (SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType>) other;
+    //noinspection unchecked
+    final SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType> that =
+        (SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType>) other;
 
-    if (that.getSeekableStreamPartitions().getStream().equals(seekableStreamPartitions.getStream())) {
-      // Same stream, merge sequences.
-      final Map<PartitionIdType, SequenceOffsetType> newMap = new HashMap<>();
-
-      for (Map.Entry<PartitionIdType, SequenceOffsetType> entry : seekableStreamPartitions.getPartitionSequenceNumberMap()
-                                                                                          .entrySet()) {
-        newMap.put(entry.getKey(), entry.getValue());
-      }
-
-      for (Map.Entry<PartitionIdType, SequenceOffsetType> entry : that.getSeekableStreamPartitions()
-                                                                      .getPartitionSequenceNumberMap()
-                                                                      .entrySet()) {
-        newMap.put(entry.getKey(), entry.getValue());
-      }
-
-      return createConcreteDataSourceMetaData(seekableStreamPartitions.getStream(), newMap);
-    } else {
-      // Different stream, prefer "other".
-      return other;
-    }
+    return createConcreteDataSourceMetaData(seekableStreamSequenceNumbers.plus(that.seekableStreamSequenceNumbers));
   }
 
-
   @Override
   public DataSourceMetadata minus(DataSourceMetadata other)
   {
-    if (!(this.getClass().isInstance(other))) {
+    if (this.getClass() != other.getClass()) {
       throw new IAE(
           "Expected instance of %s, got %s",
           this.getClass().getCanonicalName(),
@@ -109,25 +88,11 @@ public abstract class SeekableStreamDataSourceMetadata<PartitionIdType, Sequence
       );
     }
 
-    @SuppressWarnings("unchecked")
-    final SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType> that = (SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType>) other;
-
-    if (that.getSeekableStreamPartitions().getStream().equals(seekableStreamPartitions.getStream())) {
-      // Same stream, remove partitions present in "that" from "this"
-      final Map<PartitionIdType, SequenceOffsetType> newMap = new HashMap<>();
+    //noinspection unchecked
+    final SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType> that =
+        (SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType>) other;
 
-      for (Map.Entry<PartitionIdType, SequenceOffsetType> entry : seekableStreamPartitions.getPartitionSequenceNumberMap()
-                                                                                          .entrySet()) {
-        if (!that.getSeekableStreamPartitions().getPartitionSequenceNumberMap().containsKey(entry.getKey())) {
-          newMap.put(entry.getKey(), entry.getValue());
-        }
-      }
-
-      return createConcreteDataSourceMetaData(seekableStreamPartitions.getStream(), newMap);
-    } else {
-      // Different stream, prefer "this".
-      return this;
-    }
+    return createConcreteDataSourceMetaData(seekableStreamSequenceNumbers.minus(that.seekableStreamSequenceNumbers));
   }
 
   @Override
@@ -140,25 +105,24 @@ public abstract class SeekableStreamDataSourceMetadata<PartitionIdType, Sequence
       return false;
     }
     SeekableStreamDataSourceMetadata that = (SeekableStreamDataSourceMetadata) o;
-    return Objects.equals(getSeekableStreamPartitions(), that.getSeekableStreamPartitions());
+    return Objects.equals(getSeekableStreamSequenceNumbers(), that.getSeekableStreamSequenceNumbers());
   }
 
   @Override
   public int hashCode()
   {
-    return Objects.hash(getSeekableStreamPartitions());
+    return seekableStreamSequenceNumbers.hashCode();
   }
 
   @Override
   public String toString()
   {
-    return "SeekableStreamDataSourceMetadata{" +
-           "SeekableStreamPartitions=" + getSeekableStreamPartitions() +
+    return getClass().getSimpleName() + "{" +
+           "SeekableStreamStartSequenceNumbers=" + getSeekableStreamSequenceNumbers() +
            '}';
   }
 
   protected abstract SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType> createConcreteDataSourceMetaData(
-      String streamId,
-      Map<PartitionIdType, SequenceOffsetType> newMap
+      SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> seekableStreamSequenceNumbers
   );
 }
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamEndSequenceNumbers.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamEndSequenceNumbers.java
new file mode 100644
index 0000000..3f87b27
--- /dev/null
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamEndSequenceNumbers.java
@@ -0,0 +1,221 @@
+/*
+ * 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.seekablestream;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.IAE;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+
+/**
+ * Represents the end sequenceNumber per partition of a sequence. Note that end sequenceNumbers are always
+ * exclusive/inclusive in Kafka/Kinesis indexing service, respectively.
+ *
+ * To be backward compatible with both Kafka and Kinesis datasource metadata when
+ * serializing and deserializing json, redundant constructor fields stream, topic,
+ * partitionSequenceNumberMap and partitionOffsetMap are created. Only one of topic, stream
+ * should have a non-null value and only one of partitionOffsetMap and partitionSequenceNumberMap
+ * should have a non-null value.
+ *
+ * Redundant getters are used for proper Jackson serialization/deserialization when processing terminologies
+ * used by Kafka and Kinesis (i.e. topic vs. stream)
+ */
+public class SeekableStreamEndSequenceNumbers<PartitionIdType, SequenceOffsetType> implements
+    SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType>
+{
+  // stream/topic
+  private final String stream;
+  // partitionId -> sequence number
+  private final Map<PartitionIdType, SequenceOffsetType> partitionSequenceNumberMap;
+
+  @JsonCreator
+  public SeekableStreamEndSequenceNumbers(
+      @JsonProperty("stream") final String stream,
+      // kept for backward compatibility
+      @JsonProperty("topic") final String topic,
+      @JsonProperty("partitionSequenceNumberMap")
+      final Map<PartitionIdType, SequenceOffsetType> partitionSequenceNumberMap,
+      // kept for backward compatibility
+      @JsonProperty("partitionOffsetMap") final Map<PartitionIdType, SequenceOffsetType> partitionOffsetMap
+  )
+  {
+    this.stream = stream == null ? topic : stream;
+    this.partitionSequenceNumberMap = partitionOffsetMap == null ? partitionSequenceNumberMap : partitionOffsetMap;
+
+    Preconditions.checkNotNull(this.stream, "stream");
+    Preconditions.checkNotNull(this.partitionSequenceNumberMap, "partitionIdToSequenceNumberMap");
+  }
+
+  public SeekableStreamEndSequenceNumbers(
+      final String stream,
+      final Map<PartitionIdType, SequenceOffsetType> partitionSequenceNumberMap
+  )
+  {
+    this(stream, null, partitionSequenceNumberMap, null);
+  }
+
+  /**
+   * Converts this end sequence numbers into start sequence numbers. This conversion is required when checking two
+   * sequence numbers are "matched" in {@code IndexerSQLMetadataStorageCoordinator#updateDataSourceMetadataWithHandle}
+   * because only sequences numbers of the same type can be compared.
+   *
+   * @param isExclusiveEndOffset flag that end offsets are exclusive. Should be true for Kafka and false for Kinesis.
+   */
+  public SeekableStreamStartSequenceNumbers<PartitionIdType, SequenceOffsetType> asStartPartitions(
+      boolean isExclusiveEndOffset
+  )
+  {
+    return new SeekableStreamStartSequenceNumbers<>(
+        stream,
+        partitionSequenceNumberMap,
+        // All start offsets are supposed to be opposite
+        isExclusiveEndOffset ? Collections.emptySet() : partitionSequenceNumberMap.keySet()
+    );
+  }
+
+  @Override
+  @JsonProperty
+  public String getStream()
+  {
+    return stream;
+  }
+
+  /**
+   * Identical to {@link #getStream()}. Here for backwards compatibility, so a serialized
+   * SeekableStreamStartSequenceNumbers can be read by older Druid versions as a KafkaPartitions object.
+   */
+  @JsonProperty
+  public String getTopic()
+  {
+    return stream;
+  }
+
+  @Override
+  @JsonProperty
+  public Map<PartitionIdType, SequenceOffsetType> getPartitionSequenceNumberMap()
+  {
+    return partitionSequenceNumberMap;
+  }
+
+  @Override
+  public SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> plus(
+      SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> other
+  )
+  {
+    if (this.getClass() != other.getClass()) {
+      throw new IAE(
+          "Expected instance of %s, got %s",
+          this.getClass().getCanonicalName(),
+          other.getClass().getCanonicalName()
+      );
+    }
+
+    final SeekableStreamEndSequenceNumbers<PartitionIdType, SequenceOffsetType> otherEnd =
+        (SeekableStreamEndSequenceNumbers<PartitionIdType, SequenceOffsetType>) other;
+
+    if (stream.equals(otherEnd.stream)) {
+      // Same stream, merge sequences.
+      final Map<PartitionIdType, SequenceOffsetType> newMap = new HashMap<>(partitionSequenceNumberMap);
+      newMap.putAll(otherEnd.partitionSequenceNumberMap);
+      return new SeekableStreamEndSequenceNumbers<>(stream, newMap);
+    } else {
+      // Different stream, prefer "other".
+      return other;
+    }
+  }
+
+  @Override
+  public SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> minus(
+      SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> other
+  )
+  {
+    if (this.getClass() != other.getClass()) {
+      throw new IAE(
+          "Expected instance of %s, got %s",
+          this.getClass().getCanonicalName(),
+          other.getClass().getCanonicalName()
+      );
+    }
+
+    final SeekableStreamEndSequenceNumbers<PartitionIdType, SequenceOffsetType> otherEnd =
+        (SeekableStreamEndSequenceNumbers<PartitionIdType, SequenceOffsetType>) other;
+
+    if (stream.equals(otherEnd.stream)) {
+      // Same stream, remove partitions present in "that" from "this"
+      final Map<PartitionIdType, SequenceOffsetType> newMap = new HashMap<>();
+
+      for (Entry<PartitionIdType, SequenceOffsetType> entry : partitionSequenceNumberMap.entrySet()) {
+        if (!otherEnd.partitionSequenceNumberMap.containsKey(entry.getKey())) {
+          newMap.put(entry.getKey(), entry.getValue());
+        }
+      }
+
+      return new SeekableStreamEndSequenceNumbers<>(stream, newMap);
+    } else {
+      // Different stream, prefer "this".
+      return this;
+    }
+  }
+
+  /**
+   * Identical to {@link #getPartitionSequenceNumberMap()} ()}. Here for backwards compatibility, so a serialized
+   * SeekableStreamStartSequenceNumbers can be read by older Druid versions as a KafkaPartitions object.
+   */
+  @JsonProperty
+  public Map<PartitionIdType, SequenceOffsetType> getPartitionOffsetMap()
+  {
+    return partitionSequenceNumberMap;
+  }
+
+  @Override
+  public boolean equals(Object o)
+  {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    SeekableStreamEndSequenceNumbers<?, ?> that = (SeekableStreamEndSequenceNumbers<?, ?>) o;
+    return Objects.equals(stream, that.stream) &&
+           Objects.equals(partitionSequenceNumberMap, that.partitionSequenceNumberMap);
+  }
+
+  @Override
+  public int hashCode()
+  {
+    return Objects.hash(stream, partitionSequenceNumberMap);
+  }
+
+  @Override
+  public String toString()
+  {
+    return "SeekableStreamEndSequenceNumbers{" +
+           "stream='" + stream + '\'' +
+           ", partitionSequenceNumberMap=" + partitionSequenceNumberMap +
+           '}';
+  }
+}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskIOConfig.java
index 286d882..76cb5ad 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskIOConfig.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskIOConfig.java
@@ -26,8 +26,6 @@ import org.apache.druid.segment.indexing.IOConfig;
 import org.joda.time.DateTime;
 
 import javax.annotation.Nullable;
-import java.util.Collections;
-import java.util.Set;
 
 public abstract class SeekableStreamIndexTaskIOConfig<PartitionIdType, SequenceOffsetType> implements IOConfig
 {
@@ -37,60 +35,46 @@ public abstract class SeekableStreamIndexTaskIOConfig<PartitionIdType, SequenceO
   @Nullable
   private final Integer taskGroupId;
   private final String baseSequenceName;
-  private final SeekableStreamPartitions<PartitionIdType, SequenceOffsetType> startPartitions;
-  private final SeekableStreamPartitions<PartitionIdType, SequenceOffsetType> endPartitions;
+  private final SeekableStreamStartSequenceNumbers<PartitionIdType, SequenceOffsetType> startSequenceNumbers;
+  private final SeekableStreamEndSequenceNumbers<PartitionIdType, SequenceOffsetType> endSequenceNumbers;
   private final boolean useTransaction;
   private final Optional<DateTime> minimumMessageTime;
   private final Optional<DateTime> maximumMessageTime;
   private final boolean skipOffsetGaps;
-  private final Set<PartitionIdType> exclusiveStartSequenceNumberPartitions;
 
   public SeekableStreamIndexTaskIOConfig(
       final @Nullable Integer taskGroupId, // can be null for backward compabitility
       final String baseSequenceName,
-      final SeekableStreamPartitions<PartitionIdType, SequenceOffsetType> startPartitions,
-      final SeekableStreamPartitions<PartitionIdType, SequenceOffsetType> endPartitions,
+      final SeekableStreamStartSequenceNumbers<PartitionIdType, SequenceOffsetType> startSequenceNumbers,
+      final SeekableStreamEndSequenceNumbers<PartitionIdType, SequenceOffsetType> endSequenceNumbers,
       final Boolean useTransaction,
       final DateTime minimumMessageTime,
       final DateTime maximumMessageTime,
-      final Boolean skipOffsetGaps,
-      final Set<PartitionIdType> exclusiveStartSequenceNumberPartitions
+      final Boolean skipOffsetGaps
   )
   {
     this.taskGroupId = taskGroupId;
     this.baseSequenceName = Preconditions.checkNotNull(baseSequenceName, "baseSequenceName");
-    this.startPartitions = Preconditions.checkNotNull(startPartitions, "startPartitions");
-    this.endPartitions = Preconditions.checkNotNull(endPartitions, "endPartitions");
+    this.startSequenceNumbers = Preconditions.checkNotNull(startSequenceNumbers, "startSequenceNumbers");
+    this.endSequenceNumbers = Preconditions.checkNotNull(endSequenceNumbers, "endSequenceNumbers");
     this.useTransaction = useTransaction != null ? useTransaction : DEFAULT_USE_TRANSACTION;
     this.minimumMessageTime = Optional.fromNullable(minimumMessageTime);
     this.maximumMessageTime = Optional.fromNullable(maximumMessageTime);
     this.skipOffsetGaps = skipOffsetGaps != null ? skipOffsetGaps : DEFAULT_SKIP_OFFSET_GAPS;
-    this.exclusiveStartSequenceNumberPartitions = exclusiveStartSequenceNumberPartitions == null
-                                                  ? Collections.emptySet()
-                                                  : exclusiveStartSequenceNumberPartitions;
 
     Preconditions.checkArgument(
-        startPartitions.getStream().equals(endPartitions.getStream()),
+        startSequenceNumbers.getStream().equals(endSequenceNumbers.getStream()),
         "start topic/stream and end topic/stream must match"
     );
 
     Preconditions.checkArgument(
-        startPartitions.getPartitionSequenceNumberMap()
+        startSequenceNumbers.getPartitionSequenceNumberMap()
                        .keySet()
-                       .equals(endPartitions.getPartitionSequenceNumberMap().keySet()),
+                       .equals(endSequenceNumbers.getPartitionSequenceNumberMap().keySet()),
         "start partition set and end partition set must match"
     );
   }
 
-  // exclusive starting sequence partitions are used only for kinesis where the starting
-  // sequence number for certain partitions are discarded because they've already been
-  // read by a previous task
-  @JsonProperty
-  public Set<PartitionIdType> getExclusiveStartSequenceNumberPartitions()
-  {
-    return exclusiveStartSequenceNumberPartitions;
-  }
-
   @Nullable
   @JsonProperty
   public Integer getTaskGroupId()
@@ -105,15 +89,15 @@ public abstract class SeekableStreamIndexTaskIOConfig<PartitionIdType, SequenceO
   }
 
   @JsonProperty
-  public SeekableStreamPartitions<PartitionIdType, SequenceOffsetType> getStartPartitions()
+  public SeekableStreamStartSequenceNumbers<PartitionIdType, SequenceOffsetType> getStartSequenceNumbers()
   {
-    return startPartitions;
+    return startSequenceNumbers;
   }
 
   @JsonProperty
-  public SeekableStreamPartitions<PartitionIdType, SequenceOffsetType> getEndPartitions()
+  public SeekableStreamEndSequenceNumbers<PartitionIdType, SequenceOffsetType> getEndSequenceNumbers()
   {
-    return endPartitions;
+    return endSequenceNumbers;
   }
 
   @JsonProperty
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java
index ee9a7e2..3907d54 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java
@@ -230,9 +230,9 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
     this.authorizerMapper = authorizerMapper;
     this.chatHandlerProvider = chatHandlerProvider;
     this.savedParseExceptions = savedParseExceptions;
-    this.stream = ioConfig.getStartPartitions().getStream();
+    this.stream = ioConfig.getStartSequenceNumbers().getStream();
     this.rowIngestionMeters = rowIngestionMetersFactory.createRowIngestionMeters();
-    this.endOffsets = new ConcurrentHashMap<>(ioConfig.getEndPartitions().getPartitionSequenceNumberMap());
+    this.endOffsets = new ConcurrentHashMap<>(ioConfig.getEndSequenceNumbers().getPartitionSequenceNumberMap());
     this.sequences = new CopyOnWriteArrayList<>();
     this.ingestionState = IngestionState.NOT_STARTED;
 
@@ -263,7 +263,6 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
     status = Status.STARTING;
     this.toolbox = toolbox;
 
-
     if (!restoreSequences()) {
       final TreeMap<Integer, Map<PartitionIdType, SequenceOffsetType>> checkpoints = getCheckPointsFromContext(
           toolbox,
@@ -299,7 +298,7 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
         addSequence(new SequenceMetadata<>(
             0,
             StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), 0),
-            ioConfig.getStartPartitions().getPartitionSequenceNumberMap(),
+            ioConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap(),
             endOffsets,
             false,
             null
@@ -350,8 +349,6 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
       appenderator = task.newAppenderator(fireDepartmentMetrics, toolbox);
       driver = task.newDriver(appenderator, toolbox, fireDepartmentMetrics);
 
-      final String stream = ioConfig.getStartPartitions().getStream();
-
       // Start up, set up initial sequences.
       final Object restoredMetadata = driver.startJob();
       if (restoredMetadata == null) {
@@ -360,7 +357,7 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
         Preconditions.checkState(sequences.get(0).startOffsets.entrySet().stream().allMatch(
             partitionOffsetEntry ->
                 createSequenceNumber(partitionOffsetEntry.getValue()).compareTo(
-                    createSequenceNumber(ioConfig.getStartPartitions()
+                    createSequenceNumber(ioConfig.getStartSequenceNumbers()
                                                  .getPartitionSequenceNumberMap()
                                                  .get(partitionOffsetEntry.getKey())
                     )) >= 0
@@ -369,27 +366,28 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
       } else {
         @SuppressWarnings("unchecked")
         final Map<String, Object> restoredMetadataMap = (Map) restoredMetadata;
-        final SeekableStreamPartitions<PartitionIdType, SequenceOffsetType> restoredNextPartitions = deserializePartitionsFromMetadata(
-            toolbox.getObjectMapper(),
-            restoredMetadataMap.get(METADATA_NEXT_PARTITIONS)
-        );
+        final SeekableStreamEndSequenceNumbers<PartitionIdType, SequenceOffsetType> restoredNextPartitions =
+            deserializePartitionsFromMetadata(
+                toolbox.getObjectMapper(),
+                restoredMetadataMap.get(METADATA_NEXT_PARTITIONS)
+            );
 
         currOffsets.putAll(restoredNextPartitions.getPartitionSequenceNumberMap());
 
         // Sanity checks.
-        if (!restoredNextPartitions.getStream().equals(ioConfig.getStartPartitions().getStream())) {
+        if (!restoredNextPartitions.getStream().equals(ioConfig.getStartSequenceNumbers().getStream())) {
           throw new ISE(
               "WTF?! Restored stream[%s] but expected stream[%s]",
               restoredNextPartitions.getStream(),
-              ioConfig.getStartPartitions().getStream()
+              ioConfig.getStartSequenceNumbers().getStream()
           );
         }
 
-        if (!currOffsets.keySet().equals(ioConfig.getStartPartitions().getPartitionSequenceNumberMap().keySet())) {
+        if (!currOffsets.keySet().equals(ioConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().keySet())) {
           throw new ISE(
               "WTF?! Restored partitions[%s] but expected partitions[%s]",
               currOffsets.keySet(),
-              ioConfig.getStartPartitions().getPartitionSequenceNumberMap().keySet()
+              ioConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().keySet()
           );
         }
         // sequences size can be 0 only when all sequences got published and task stopped before it could finish
@@ -418,10 +416,10 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
       if (!isEndOffsetExclusive()) {
         for (Map.Entry<PartitionIdType, SequenceOffsetType> entry : currOffsets.entrySet()) {
           final boolean isAtStart = entry.getValue().equals(
-              ioConfig.getStartPartitions().getPartitionSequenceNumberMap().get(entry.getKey())
+              ioConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(entry.getKey())
           );
 
-          if (!isAtStart || ioConfig.getExclusiveStartSequenceNumberPartitions().contains(entry.getKey())) {
+          if (!isAtStart || ioConfig.getStartSequenceNumbers().getExclusivePartitions().contains(entry.getKey())) {
             lastReadOffsets.put(entry.getKey(), entry.getValue());
           }
         }
@@ -438,12 +436,7 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
           @Override
           public Object getMetadata()
           {
-            return ImmutableMap.of(
-                METADATA_NEXT_PARTITIONS, new SeekableStreamPartitions<>(
-                    ioConfig.getStartPartitions().getStream(),
-                    snapshot
-                )
-            );
+            return ImmutableMap.of(METADATA_NEXT_PARTITIONS, new SeekableStreamEndSequenceNumbers<>(stream, snapshot));
           }
 
           @Override
@@ -458,7 +451,7 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
       maybePersistAndPublishSequences(committerSupplier);
 
       Set<StreamPartition<PartitionIdType>> assignment = assignPartitions(recordSupplier);
-      possiblyResetDataSourceMetadata(toolbox, recordSupplier, assignment, currOffsets);
+      possiblyResetDataSourceMetadata(toolbox, recordSupplier, assignment);
       seekToStartingSequence(recordSupplier, assignment);
 
       ingestionState = IngestionState.BUILD_SEGMENTS;
@@ -476,7 +469,7 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
             // partitions upon resuming. Don't call "seekToStartingSequence" after "assignPartitions", because there's
             // no need to re-seek here. All we're going to be doing is dropping partitions.
             assignment = assignPartitions(recordSupplier);
-            possiblyResetDataSourceMetadata(toolbox, recordSupplier, assignment, currOffsets);
+            possiblyResetDataSourceMetadata(toolbox, recordSupplier, assignment);
 
             if (assignment.isEmpty()) {
               log.info("All partitions have been fully read");
@@ -512,7 +505,7 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
           // note: getRecords() also updates assignment
           stillReading = !assignment.isEmpty();
 
-          SequenceMetadata sequenceToCheckpoint = null;
+          SequenceMetadata<PartitionIdType, SequenceOffsetType> sequenceToCheckpoint = null;
           for (OrderedPartitionableRecord<PartitionIdType, SequenceOffsetType> record : records) {
             final boolean shouldProcess = verifyRecordInRange(record.getPartitionId(), record.getSequenceNumber());
 
@@ -652,11 +645,19 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
                 task.getDataSource(),
                 ioConfig.getTaskGroupId(),
                 task.getIOConfig().getBaseSequenceName(),
-                createDataSourceMetadata(new SeekableStreamPartitions<>(
-                    stream,
-                    sequenceToCheckpoint.getStartOffsets()
-                )),
-                createDataSourceMetadata(new SeekableStreamPartitions<>(stream, currOffsets))
+                createDataSourceMetadata(
+                    new SeekableStreamStartSequenceNumbers<>(
+                        stream,
+                        sequenceToCheckpoint.getStartOffsets(),
+                        ioConfig.getStartSequenceNumbers().getExclusivePartitions()
+                    )
+                ),
+                createDataSourceMetadata(
+                    new SeekableStreamEndSequenceNumbers<>(
+                        stream,
+                        currOffsets
+                    )
+                )
             );
             if (!toolbox.getTaskActionClient().submit(checkpointAction)) {
               throw new ISE("Checkpoint request with sequences [%s] failed, dying", currOffsets);
@@ -1232,9 +1233,11 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
             new ResetDataSourceMetadataAction(
                 task.getDataSource(),
                 createDataSourceMetadata(
-                    new SeekableStreamPartitions<>(
-                        ioConfig.getStartPartitions().getStream(),
-                        partitionOffsetMap
+                    new SeekableStreamStartSequenceNumbers<>(
+                        ioConfig.getStartSequenceNumbers().getStream(),
+                        partitionOffsetMap,
+                        // Clear all exclusive start offsets for automatic reset
+                        Collections.emptySet()
                     )
                 )
             )
@@ -1743,14 +1746,14 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
   protected abstract SequenceOffsetType getNextStartOffset(SequenceOffsetType sequenceNumber);
 
   /**
-   * deserializes stored metadata into SeekableStreamPartitions
+   * deserializes stored metadata into SeekableStreamStartSequenceNumbers
    *
    * @param mapper json objectMapper
    * @param object metadata
    *
-   * @return SeekableStreamPartitions
+   * @return SeekableStreamEndSequenceNumbers
    */
-  protected abstract SeekableStreamPartitions<PartitionIdType, SequenceOffsetType> deserializePartitionsFromMetadata(
+  protected abstract SeekableStreamEndSequenceNumbers<PartitionIdType, SequenceOffsetType> deserializePartitionsFromMetadata(
       ObjectMapper mapper,
       Object object
   );
@@ -1780,7 +1783,7 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
    * @return datasource metadata
    */
   protected abstract SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType> createDataSourceMetadata(
-      SeekableStreamPartitions<PartitionIdType, SequenceOffsetType> partitions
+      SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> partitions
   );
 
   /**
@@ -1795,17 +1798,11 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
   /**
    * check if the sequence offsets stored in currOffsets are still valid sequence offsets compared to
    * earliest sequence offsets fetched from stream
-   *
-   * @param toolbox
-   * @param recordSupplier
-   * @param assignment
-   * @param currOffsets
    */
   protected abstract void possiblyResetDataSourceMetadata(
       TaskToolbox toolbox,
       RecordSupplier<PartitionIdType, SequenceOffsetType> recordSupplier,
-      Set<StreamPartition<PartitionIdType>> assignment,
-      Map<PartitionIdType, SequenceOffsetType> currOffsets
+      Set<StreamPartition<PartitionIdType>> assignment
   );
 
   /**
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java
deleted file mode 100644
index dc3ff87..0000000
--- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java
+++ /dev/null
@@ -1,151 +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.seekablestream;
-
-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 javax.validation.constraints.NotNull;
-import java.util.Map;
-import java.util.Objects;
-
-/**
- * class that encapsulates a partitionIdToSequenceNumberMap of partitionId -> sequenceNumber.
- * To be backward compatible with both Kafka and Kinesis datasource metadata when
- * serializing and deserializing json, redundant constructor fields stream, topic,
- * partitionSequenceNumberMap and partitionOffsetMap are created. Only one of topic, stream
- * should have a non-null value and only one of partitionOffsetMap and partitionSequenceNumberMap
- * should have a non-null value.
- *
- * Redundant getters are used for proper Jackson serialization/deserialization when processing terminologies
- * used by Kafka and Kinesis (i.e. topic vs. stream)
- *
- * @param <PartitionIdType>    partition id type
- * @param <SequenceOffsetType> sequence number type
- */
-public class SeekableStreamPartitions<PartitionIdType, SequenceOffsetType>
-{
-  // this special marker is used by the KinesisSupervisor to set the endOffsets
-  // of newly created indexing tasks. This is necessary because streaming tasks do not
-  // have endPartitionOffsets. This marker signals to the task that it should continue
-  // to ingest data until taskDuration has elapsed or the task was stopped or paused or killed
-  public static final String NO_END_SEQUENCE_NUMBER = "NO_END_SEQUENCE_NUMBER";
-
-  // stream/topic
-  private final String stream;
-  // partitionId -> sequence number
-  private final Map<PartitionIdType, SequenceOffsetType> partitionIdToSequenceNumberMap;
-
-  @JsonCreator
-  public SeekableStreamPartitions(
-      @JsonProperty("stream") final String stream,
-      // kept for backward compatibility
-      @JsonProperty("topic") final String topic,
-      @JsonProperty("partitionSequenceNumberMap")
-      final Map<PartitionIdType, SequenceOffsetType> partitionSequenceNumberMap,
-      // kept for backward compatibility
-      @JsonProperty("partitionOffsetMap") final Map<PartitionIdType, SequenceOffsetType> partitionOffsetMap
-  )
-  {
-    this.stream = stream == null ? topic : stream;
-    this.partitionIdToSequenceNumberMap = ImmutableMap.copyOf(partitionOffsetMap == null
-                                                              ? partitionSequenceNumberMap
-                                                              : partitionOffsetMap);
-    Preconditions.checkArgument(this.stream != null);
-    Preconditions.checkArgument(partitionIdToSequenceNumberMap != null);
-  }
-
-  // constructor for backward compatibility
-  public SeekableStreamPartitions(
-      @NotNull final String stream,
-      final Map<PartitionIdType, SequenceOffsetType> partitionOffsetMap
-  )
-  {
-    this(
-        Preconditions.checkNotNull(stream, "stream"),
-        null,
-        Preconditions.checkNotNull(partitionOffsetMap, "partitionOffsetMap"),
-        null
-    );
-  }
-
-  @JsonProperty
-  public String getStream()
-  {
-    return stream;
-  }
-
-  /**
-   * Identical to {@link #getStream()}. Here for backwards compatibility, so a serialized SeekableStreamPartitions can
-   * be read by older Druid versions as a KafkaPartitions object.
-   */
-  @JsonProperty
-  public String getTopic()
-  {
-    return stream;
-  }
-
-  @JsonProperty
-  public Map<PartitionIdType, SequenceOffsetType> getPartitionSequenceNumberMap()
-  {
-    return partitionIdToSequenceNumberMap;
-  }
-
-  /**
-   * Identical to {@link #getPartitionSequenceNumberMap()} ()}. Here for backwards compatibility, so a serialized
-   * SeekableStreamPartitions can be read by older Druid versions as a KafkaPartitions object.
-   */
-  @JsonProperty
-  public Map<PartitionIdType, SequenceOffsetType> getPartitionOffsetMap()
-  {
-    return partitionIdToSequenceNumberMap;
-  }
-
-  @Override
-  public boolean equals(Object o)
-  {
-    if (this == o) {
-      return true;
-    }
-    if (o == null || getClass() != o.getClass()) {
-      return false;
-    }
-    SeekableStreamPartitions that = (SeekableStreamPartitions) o;
-    return Objects.equals(stream, that.stream) &&
-           Objects.equals(partitionIdToSequenceNumberMap, that.partitionIdToSequenceNumberMap);
-  }
-
-  @Override
-  public int hashCode()
-  {
-    return Objects.hash(stream, partitionIdToSequenceNumberMap);
-  }
-
-  @Override
-  public String toString()
-  {
-    return getClass().getSimpleName() + "{" +
-           "stream='" + stream + '\'' +
-           ", partitionSequenceNumberMap=" + partitionIdToSequenceNumberMap +
-           '}';
-  }
-}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSequenceNumbers.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSequenceNumbers.java
new file mode 100644
index 0000000..a790974
--- /dev/null
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSequenceNumbers.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.seekablestream;
+
+import com.fasterxml.jackson.annotation.JsonSubTypes;
+import com.fasterxml.jackson.annotation.JsonSubTypes.Type;
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import com.fasterxml.jackson.annotation.JsonTypeInfo.Id;
+import org.apache.druid.indexing.overlord.DataSourceMetadata;
+
+import java.util.Map;
+
+@JsonTypeInfo(use = Id.NAME, property = "type", defaultImpl = SeekableStreamEndSequenceNumbers.class)
+@JsonSubTypes({
+    @Type(name = "start", value = SeekableStreamStartSequenceNumbers.class),
+    @Type(name = "end", value = SeekableStreamEndSequenceNumbers.class)
+})
+public interface SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType>
+{
+  /**
+   * Returns the stream/topic name.
+   */
+  String getStream();
+
+  /**
+   * Returns a map of partitionId -> sequenceNumber.
+   */
+  Map<PartitionIdType, SequenceOffsetType> getPartitionSequenceNumberMap();
+
+  /**
+   * Merges this and the given other and returns the merged result.
+   *
+   * @see DataSourceMetadata#plus
+   */
+  SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> plus(
+      SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> other
+  );
+
+  /**
+   * Subtracts the given other from this and returns the result.
+   *
+   * @see DataSourceMetadata#minus
+   */
+  SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> minus(
+      SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> other
+  );
+}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamStartSequenceNumbers.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamStartSequenceNumbers.java
new file mode 100644
index 0000000..f737292
--- /dev/null
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamStartSequenceNumbers.java
@@ -0,0 +1,208 @@
+/*
+ * 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.seekablestream;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.IAE;
+
+import javax.annotation.Nullable;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Set;
+
+/**
+ * Represents the start sequenceNumber per partition of a sequence. This class keeps an additional set of
+ * {@link #exclusivePartitions} for Kinesis indexing service in where each start offset can be either inclusive
+ * or exclusive.
+ */
+public class SeekableStreamStartSequenceNumbers<PartitionIdType, SequenceOffsetType> implements
+    SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType>
+{
+  // stream/topic
+  private final String stream;
+  // partitionId -> sequence number
+  private final Map<PartitionIdType, SequenceOffsetType> partitionSequenceNumberMap;
+  private final Set<PartitionIdType> exclusivePartitions;
+
+  @JsonCreator
+  public SeekableStreamStartSequenceNumbers(
+      @JsonProperty("stream") final String stream,
+      @JsonProperty("partitionSequenceNumberMap")
+      final Map<PartitionIdType, SequenceOffsetType> partitionSequenceNumberMap,
+      @JsonProperty("exclusivePartitions") @Nullable final Set<PartitionIdType> exclusivePartitions
+  )
+  {
+    this.stream = Preconditions.checkNotNull(stream, "stream");
+    this.partitionSequenceNumberMap = Preconditions.checkNotNull(
+        partitionSequenceNumberMap,
+        "partitionIdToSequenceNumberMap"
+    );
+    // exclusiveOffset can be null if this class is deserialized from metadata store. Note that only end offsets are
+    // stored in metadata store.
+    // The default is true because there was only Kafka indexing service before in which the end offset is always
+    // exclusive.
+    this.exclusivePartitions = exclusivePartitions == null ? Collections.emptySet() : exclusivePartitions;
+  }
+
+  @Override
+  @JsonProperty
+  public String getStream()
+  {
+    return stream;
+  }
+
+  @Override
+  @JsonProperty
+  public Map<PartitionIdType, SequenceOffsetType> getPartitionSequenceNumberMap()
+  {
+    return partitionSequenceNumberMap;
+  }
+
+  @Override
+  public SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> plus(
+      SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> other
+  )
+  {
+    if (this.getClass() != other.getClass()) {
+      throw new IAE(
+          "Expected instance of %s, got %s",
+          this.getClass().getCanonicalName(),
+          other.getClass().getCanonicalName()
+      );
+    }
+
+    final SeekableStreamStartSequenceNumbers<PartitionIdType, SequenceOffsetType> otherStart =
+        (SeekableStreamStartSequenceNumbers<PartitionIdType, SequenceOffsetType>) other;
+
+    if (stream.equals(otherStart.stream)) {
+      // Same stream, merge sequences.
+      final Map<PartitionIdType, SequenceOffsetType> newMap = new HashMap<>(partitionSequenceNumberMap);
+      newMap.putAll(otherStart.partitionSequenceNumberMap);
+
+      // A partition is exclusive if it's
+      // 1) exclusive in "this" and it's not in "other"'s partitionSequenceNumberMap or
+      // 2) exclusive in "other"
+      final Set<PartitionIdType> newExclusivePartitions = new HashSet<>();
+      partitionSequenceNumberMap.forEach(
+          (partitionId, sequenceOffset) -> {
+            if (exclusivePartitions.contains(partitionId)
+                && !otherStart.partitionSequenceNumberMap.containsKey(partitionId)) {
+              newExclusivePartitions.add(partitionId);
+            }
+          }
+      );
+      newExclusivePartitions.addAll(otherStart.exclusivePartitions);
+
+      return new SeekableStreamStartSequenceNumbers<>(
+          stream,
+          newMap,
+          newExclusivePartitions
+      );
+    } else {
+      // Different stream, prefer "other".
+      return other;
+    }
+  }
+
+  @Override
+  public SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> minus(
+      SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> other
+  )
+  {
+    if (this.getClass() != other.getClass()) {
+      throw new IAE(
+          "Expected instance of %s, got %s",
+          this.getClass().getCanonicalName(),
+          other.getClass().getCanonicalName()
+      );
+    }
+
+    final SeekableStreamStartSequenceNumbers<PartitionIdType, SequenceOffsetType> otherStart =
+        (SeekableStreamStartSequenceNumbers<PartitionIdType, SequenceOffsetType>) other;
+
+    if (stream.equals(otherStart.stream)) {
+      // Same stream, remove partitions present in "that" from "this"
+      final Map<PartitionIdType, SequenceOffsetType> newMap = new HashMap<>();
+      final Set<PartitionIdType> newExclusivePartitions = new HashSet<>();
+
+      for (Entry<PartitionIdType, SequenceOffsetType> entry : partitionSequenceNumberMap.entrySet()) {
+        if (!otherStart.partitionSequenceNumberMap.containsKey(entry.getKey())) {
+          newMap.put(entry.getKey(), entry.getValue());
+          // A partition is exclusive if it's exclusive in "this" and not in "other"'s partitionSequenceNumberMap
+          if (exclusivePartitions.contains(entry.getKey())) {
+            newExclusivePartitions.add(entry.getKey());
+          }
+        }
+      }
+
+      return new SeekableStreamStartSequenceNumbers<>(
+          stream,
+          newMap,
+          newExclusivePartitions
+      );
+    } else {
+      // Different stream, prefer "this".
+      return this;
+    }
+  }
+
+  @JsonProperty
+  public Set<PartitionIdType> getExclusivePartitions()
+  {
+    return exclusivePartitions;
+  }
+
+  @Override
+  public boolean equals(Object o)
+  {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    SeekableStreamStartSequenceNumbers<?, ?> that = (SeekableStreamStartSequenceNumbers<?, ?>) o;
+    return Objects.equals(stream, that.stream) &&
+           Objects.equals(partitionSequenceNumberMap, that.partitionSequenceNumberMap) &&
+           Objects.equals(exclusivePartitions, that.exclusivePartitions);
+  }
+
+  @Override
+  public int hashCode()
+  {
+    return Objects.hash(stream, partitionSequenceNumberMap, exclusivePartitions);
+  }
+
+  @Override
+  public String toString()
+  {
+    return "SeekableStreamStartSequenceNumbers{" +
+           "stream='" + stream + '\'' +
+           ", partitionSequenceNumberMap=" + partitionSequenceNumberMap +
+           ", exclusivePartitions=" + exclusivePartitions +
+           '}';
+  }
+}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SequenceMetadata.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SequenceMetadata.java
index 61bb35a..9cbafd0 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SequenceMetadata.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SequenceMetadata.java
@@ -47,7 +47,6 @@ public class SequenceMetadata<PartitionIdType, SequenceOffsetType>
   private final Set<PartitionIdType> exclusiveStartPartitions;
   private final Set<PartitionIdType> assignments;
   private final boolean sentinel;
-  private boolean checkpointed;
   /**
    * 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.
@@ -57,6 +56,8 @@ public class SequenceMetadata<PartitionIdType, SequenceOffsetType>
   final Map<PartitionIdType, SequenceOffsetType> startOffsets;
   final Map<PartitionIdType, SequenceOffsetType> endOffsets;
 
+  private boolean checkpointed;
+
   @JsonCreator
   public SequenceMetadata(
       @JsonProperty("sequenceId") int sequenceId,
@@ -274,9 +275,9 @@ public class SequenceMetadata<PartitionIdType, SequenceOffsetType>
               // subset of segments
               return ImmutableMap.of(
                   SeekableStreamIndexTaskRunner.METADATA_NEXT_PARTITIONS,
-                  new SeekableStreamPartitions<>(stream, lastPersistedOffsets),
+                  new SeekableStreamStartSequenceNumbers<>(stream, lastPersistedOffsets, exclusiveStartPartitions),
                   SeekableStreamIndexTaskRunner.METADATA_PUBLISH_PARTITIONS,
-                  new SeekableStreamPartitions<>(stream, endOffsets)
+                  new SeekableStreamEndSequenceNumbers<>(stream, endOffsets)
               );
             }
             finally {
@@ -301,7 +302,7 @@ public class SequenceMetadata<PartitionIdType, SequenceOffsetType>
   {
     return (segments, commitMetadata) -> {
       final Map commitMetaMap = (Map) Preconditions.checkNotNull(commitMetadata, "commitMetadata");
-      final SeekableStreamPartitions<PartitionIdType, SequenceOffsetType> finalPartitions =
+      final SeekableStreamEndSequenceNumbers<PartitionIdType, SequenceOffsetType> finalPartitions =
           runner.deserializePartitionsFromMetadata(
               toolbox.getObjectMapper(),
               commitMetaMap.get(SeekableStreamIndexTaskRunner.METADATA_PUBLISH_PARTITIONS)
@@ -322,7 +323,11 @@ public class SequenceMetadata<PartitionIdType, SequenceOffsetType>
         action = new SegmentTransactionalInsertAction(
             segments,
             runner.createDataSourceMetadata(
-                new SeekableStreamPartitions<>(finalPartitions.getStream(), getStartOffsets())
+                new SeekableStreamStartSequenceNumbers<>(
+                    finalPartitions.getStream(),
+                    getStartOffsets(),
+                    exclusiveStartPartitions
+                )
             ),
             runner.createDataSourceMetadata(finalPartitions)
         );
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java
index b93d15f..0b7a2c2 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java
@@ -61,7 +61,7 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFac
 import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig;
 import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner;
 import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig;
-import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
+import org.apache.druid.indexing.seekablestream.SeekableStreamSequenceNumbers;
 import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber;
 import org.apache.druid.indexing.seekablestream.common.RecordSupplier;
 import org.apache.druid.indexing.seekablestream.common.StreamPartition;
@@ -376,7 +376,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
           Map<PartitionIdType, SequenceOffsetType> checkpoint = checkpoints.get(sequenceId);
           // We have already verified the stream of the current checkpoint is same with that in ioConfig.
           // See checkpoint().
-          if (checkpoint.equals(previousCheckpoint.getSeekableStreamPartitions()
+          if (checkpoint.equals(previousCheckpoint.getSeekableStreamSequenceNumbers()
                                                   .getPartitionSequenceNumberMap()
           )) {
             break;
@@ -1098,7 +1098,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
       @SuppressWarnings("unchecked")
       final SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType> resetMetadata = (SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType>) dataSourceMetadata;
 
-      if (resetMetadata.getSeekableStreamPartitions().getStream().equals(ioConfig.getStream())) {
+      if (resetMetadata.getSeekableStreamSequenceNumbers().getStream().equals(ioConfig.getStream())) {
         // metadata can be null
         final DataSourceMetadata metadata = indexerMetadataStorageCoordinator.getDataSourceMetadata(dataSource);
         if (metadata != null && !checkSourceMetadataMatch(metadata)) {
@@ -1114,12 +1114,12 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
         // 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<PartitionIdType, SequenceOffsetType> resetPartitionOffset : resetMetadata.getSeekableStreamPartitions()
+        for (Entry<PartitionIdType, SequenceOffsetType> resetPartitionOffset : resetMetadata.getSeekableStreamSequenceNumbers()
                                                                                             .getPartitionSequenceNumberMap()
                                                                                             .entrySet()) {
           final SequenceOffsetType partitionOffsetInMetadataStore = currentMetadata == null
                                                                     ? null
-                                                                    : currentMetadata.getSeekableStreamPartitions()
+                                                                    : currentMetadata.getSeekableStreamSequenceNumbers()
                                                                                      .getPartitionSequenceNumberMap()
                                                                                      .get(resetPartitionOffset.getKey());
           final TaskGroup partitionTaskGroup = activelyReadingTaskGroups.get(
@@ -1153,7 +1153,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
           }
         }
         if (metadataUpdateSuccess) {
-          resetMetadata.getSeekableStreamPartitions().getPartitionSequenceNumberMap().keySet().forEach(partition -> {
+          resetMetadata.getSeekableStreamSequenceNumbers().getPartitionSequenceNumberMap().keySet().forEach(partition -> {
             final int groupId = getTaskGroupIdForPartition(partition);
             killTaskGroupForPartitions(ImmutableSet.of(partition), "DataSourceMetadata is updated while reset");
             activelyReadingTaskGroups.remove(groupId);
@@ -1165,7 +1165,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
       } else {
         log.warn(
             "Reset metadata stream [%s] and supervisor's stream name [%s] do not match",
-            resetMetadata.getSeekableStreamPartitions().getStream(),
+            resetMetadata.getSeekableStreamSequenceNumbers().getStream(),
             ioConfig.getStream()
         );
       }
@@ -1241,7 +1241,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
       // seamless schema migration.
 
       Iterator<PartitionIdType> it = seekableStreamIndexTask.getIOConfig()
-                                                            .getStartPartitions()
+                                                            .getStartSequenceNumbers()
                                                             .getPartitionSequenceNumberMap()
                                                             .keySet()
                                                             .iterator();
@@ -1265,7 +1265,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
                         log.debug("Task [%s], status [%s]", taskId, status);
                         if (status == SeekableStreamIndexTaskRunner.Status.PUBLISHING) {
                           seekableStreamIndexTask.getIOConfig()
-                                                 .getStartPartitions()
+                                                 .getStartSequenceNumbers()
                                                  .getPartitionSequenceNumberMap()
                                                  .keySet()
                                                  .forEach(
@@ -1273,7 +1273,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
                                                          getTaskGroupIdForPartition(partition),
                                                          taskId,
                                                          seekableStreamIndexTask.getIOConfig()
-                                                                                .getStartPartitions()
+                                                                                .getStartSequenceNumbers()
                                                                                 .getPartitionSequenceNumberMap()
                                                      ));
 
@@ -1301,7 +1301,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
                           }
                         } else {
                           for (PartitionIdType partition : seekableStreamIndexTask.getIOConfig()
-                                                                                  .getStartPartitions()
+                                                                                  .getStartSequenceNumbers()
                                                                                   .getPartitionSequenceNumberMap()
                                                                                   .keySet()) {
                             if (!taskGroupId.equals(getTaskGroupIdForPartition(partition))) {
@@ -1341,12 +1341,14 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
                                       taskGroupId,
                                       ImmutableMap.copyOf(
                                           seekableStreamIndexTask.getIOConfig()
-                                                                 .getStartPartitions()
+                                                                 .getStartSequenceNumbers()
                                                                  .getPartitionSequenceNumberMap()
                                       ),
                                       seekableStreamIndexTask.getIOConfig().getMinimumMessageTime(),
                                       seekableStreamIndexTask.getIOConfig().getMaximumMessageTime(),
-                                      seekableStreamIndexTask.getIOConfig().getExclusiveStartSequenceNumberPartitions()
+                                      seekableStreamIndexTask.getIOConfig()
+                                                             .getStartSequenceNumbers()
+                                                             .getExclusivePartitions()
                                   );
                                 }
                             );
@@ -1469,13 +1471,13 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
     final SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType> latestDataSourceMetadata = (SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType>) rawDataSourceMetadata;
 
     final boolean hasValidOffsetsFromDb = latestDataSourceMetadata != null &&
-                                          latestDataSourceMetadata.getSeekableStreamPartitions() != null &&
+                                          latestDataSourceMetadata.getSeekableStreamSequenceNumbers() != null &&
                                           ioConfig.getStream().equals(
-                                              latestDataSourceMetadata.getSeekableStreamPartitions().getStream()
+                                              latestDataSourceMetadata.getSeekableStreamSequenceNumbers().getStream()
                                           );
     final Map<PartitionIdType, SequenceOffsetType> latestOffsetsFromDb;
     if (hasValidOffsetsFromDb) {
-      latestOffsetsFromDb = latestDataSourceMetadata.getSeekableStreamPartitions().getPartitionSequenceNumberMap();
+      latestOffsetsFromDb = latestDataSourceMetadata.getSeekableStreamSequenceNumbers().getPartitionSequenceNumberMap();
     } else {
       latestOffsetsFromDb = null;
     }
@@ -1648,7 +1650,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
     } else {
       return generateSequenceName(
           task.getIOConfig()
-              .getStartPartitions()
+              .getStartSequenceNumbers()
               .getPartitionSequenceNumberMap(),
           task.getIOConfig().getMinimumMessageTime(),
           task.getIOConfig().getMaximumMessageTime()
@@ -2247,17 +2249,18 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
         ) : Optional.absent());
 
 
-        Map<PartitionIdType, OrderedSequenceNumber<SequenceOffsetType>> startingOffsets = generateStartingSequencesForPartitionGroup(
-            groupId);
+        final Map<PartitionIdType, OrderedSequenceNumber<SequenceOffsetType>> startingOffsets =
+            generateStartingSequencesForPartitionGroup(groupId);
 
         ImmutableMap<PartitionIdType, SequenceOffsetType> simpleStartingOffsets = startingOffsets
             .entrySet()
             .stream()
             .filter(x -> x.getValue().get() != null)
-            .collect(Collectors.collectingAndThen(
-                Collectors.toMap(Entry::getKey, x -> x.getValue().get()),
-                ImmutableMap::copyOf
-            ));
+            .collect(
+                Collectors.collectingAndThen(
+                    Collectors.toMap(Entry::getKey, x -> x.getValue().get()), ImmutableMap::copyOf
+                )
+            );
 
         Set<PartitionIdType> exclusiveStartSequenceNumberPartitions = !useExclusiveStartingSequence
                                                                       ? Collections.emptySet()
@@ -2345,7 +2348,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
       if (!getNotSetMarker().equals(sequence)) {
         // if we are given a startingOffset (set by a previous task group which is pending completion) then use it
         if (!isEndOfShard(sequence)) {
-          builder.put(partition, makeSequenceNumber(sequence, false));
+          builder.put(partition, makeSequenceNumber(sequence, useExclusiveStartSequenceNumberForStartSequence()));
         }
       } else {
         // if we don't have a startingOffset (first run or we had some previous failures and reset the sequences) then
@@ -2374,7 +2377,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
         if (!checkSequenceAvailability(partition, sequence)) {
           if (taskTuningConfig.isResetOffsetAutomatically()) {
             resetInternal(
-                createDataSourceMetaData(ioConfig.getStream(), ImmutableMap.of(partition, sequence))
+                createDataSourceMetaDataForReset(ioConfig.getStream(), ImmutableMap.of(partition, sequence))
             );
             throw new ISE(
                 "Previous sequenceNumber [%s] is no longer available for partition [%s] - automatically resetting sequence",
@@ -2391,7 +2394,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
           }
         }
       }
-      return makeSequenceNumber(sequence, true);
+      return makeSequenceNumber(sequence, useExclusiveStartSequenceNumberForStartSequence());
     } else {
       boolean useEarliestSequenceNumber = ioConfig.isUseEarliestSequenceNumber();
       if (subsequentlyDiscoveredPartitions.contains(partition)) {
@@ -2417,8 +2420,8 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
     if (dataSourceMetadata instanceof SeekableStreamDataSourceMetadata
         && checkSourceMetadataMatch(dataSourceMetadata)) {
       @SuppressWarnings("unchecked")
-      SeekableStreamPartitions<PartitionIdType, SequenceOffsetType> partitions = ((SeekableStreamDataSourceMetadata) dataSourceMetadata)
-          .getSeekableStreamPartitions();
+      SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> partitions = ((SeekableStreamDataSourceMetadata) dataSourceMetadata)
+          .getSeekableStreamSequenceNumbers();
       if (partitions != null) {
         if (!ioConfig.getStream().equals(partitions.getStream())) {
           log.warn(
@@ -2633,11 +2636,11 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
     Preconditions.checkArgument(
         spec.getIoConfig()
             .getStream()
-            .equals(((SeekableStreamDataSourceMetadata) currentCheckPoint).getSeekableStreamPartitions()
+            .equals(((SeekableStreamDataSourceMetadata) currentCheckPoint).getSeekableStreamSequenceNumbers()
                                                                           .getStream()),
         "Supervisor stream [%s] and stream in checkpoint [%s] does not match",
         spec.getIoConfig().getStream(),
-        ((SeekableStreamDataSourceMetadata) currentCheckPoint).getSeekableStreamPartitions().getStream()
+        ((SeekableStreamDataSourceMetadata) currentCheckPoint).getSeekableStreamSequenceNumbers().getStream()
     );
 
     log.info("Checkpointing [%s] for taskGroup [%s]", currentCheckPoint, taskGroupId);
@@ -2716,14 +2719,14 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
   protected abstract boolean doesTaskTypeMatchSupervisor(Task task);
 
   /**
-   * creates a specific instance of kafka/kinesis datasource metadata
+   * creates a specific instance of kafka/kinesis datasource metadata. Only used for reset.
    *
    * @param stream stream name
    * @param map    partitionId -> sequence
    *
    * @return specific instance of datasource metadata
    */
-  protected abstract SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType> createDataSourceMetaData(
+  protected abstract SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType> createDataSourceMetaDataForReset(
       String stream,
       Map<PartitionIdType, SequenceOffsetType> map
   );
@@ -2812,4 +2815,11 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
    * checks if seqNum marks the end of a Kinesis shard. Used by Kinesis only.
    */
   protected abstract boolean isEndOfShard(SequenceOffsetType seqNum);
+
+  /**
+   * Returns true if the start sequence number should be exclusive for the non-first sequences for the whole partition.
+   * For example, in Kinesis, the start offsets are inclusive for the first sequence, but exclusive for following
+   * sequences. In Kafka, start offsets are always inclusive.
+   */
+  protected abstract boolean useExclusiveStartSequenceNumberForStartSequence();
 }
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitionsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamEndSequenceNumbersTest.java
similarity index 68%
rename from indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitionsTest.java
rename to indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamEndSequenceNumbersTest.java
index da5c3ec..691f579 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitionsTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamEndSequenceNumbersTest.java
@@ -22,6 +22,7 @@ package org.apache.druid.indexing.seekablestream;
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
 import org.apache.druid.java.util.common.jackson.JacksonUtils;
 import org.apache.druid.segment.TestHelper;
 import org.junit.Assert;
@@ -29,7 +30,7 @@ import org.junit.Test;
 
 import java.util.Map;
 
-public class SeekableStreamPartitionsTest
+public class SeekableStreamEndSequenceNumbersTest
 {
   private static final ObjectMapper OBJECT_MAPPER = TestHelper.makeJsonMapper();
 
@@ -39,13 +40,16 @@ public class SeekableStreamPartitionsTest
     final String stream = "theStream";
     final Map<Integer, Long> offsetMap = ImmutableMap.of(1, 2L, 3, 4L);
 
-    final SeekableStreamPartitions<Integer, Long> partitions = new SeekableStreamPartitions<>(stream, offsetMap);
+    final SeekableStreamEndSequenceNumbers<Integer, Long> partitions = new SeekableStreamEndSequenceNumbers<>(
+        stream,
+        offsetMap
+    );
     final String serializedString = OBJECT_MAPPER.writeValueAsString(partitions);
 
     // Check round-trip.
-    final SeekableStreamPartitions<Integer, Long> partitions2 = OBJECT_MAPPER.readValue(
+    final SeekableStreamEndSequenceNumbers<Integer, Long> partitions2 = OBJECT_MAPPER.readValue(
         serializedString,
-        new TypeReference<SeekableStreamPartitions<Integer, Long>>() {}
+        new TypeReference<SeekableStreamEndSequenceNumbers<Integer, Long>>() {}
     );
 
     Assert.assertEquals("Round trip", partitions, partitions2);
@@ -69,4 +73,26 @@ public class SeekableStreamPartitionsTest
         OBJECT_MAPPER.convertValue(asMap.get("partitionOffsetMap"), new TypeReference<Map<Integer, Long>>() {})
     );
   }
+
+  @Test
+  public void testConvertToStart()
+  {
+    final String stream = "topic";
+    final Map<Integer, Long> offsetMap = ImmutableMap.of(1, 2L, 3, 4L);
+
+    final SeekableStreamEndSequenceNumbers<Integer, Long> endSequenceNumbers = new SeekableStreamEndSequenceNumbers<>(
+        stream,
+        offsetMap
+    );
+
+    Assert.assertEquals(
+        new SeekableStreamStartSequenceNumbers<>(stream, offsetMap, ImmutableSet.of(1, 3)),
+        endSequenceNumbers.asStartPartitions(false)
+    );
+
+    Assert.assertEquals(
+        new SeekableStreamStartSequenceNumbers<>(stream, offsetMap, ImmutableSet.of()),
+        endSequenceNumbers.asStartPartitions(true)
+    );
+  }
 }
diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/DataSourceMetadata.java b/server/src/main/java/org/apache/druid/indexing/overlord/DataSourceMetadata.java
index 5148aed..23579a0 100644
--- a/server/src/main/java/org/apache/druid/indexing/overlord/DataSourceMetadata.java
+++ b/server/src/main/java/org/apache/druid/indexing/overlord/DataSourceMetadata.java
@@ -45,6 +45,15 @@ public interface DataSourceMetadata
   boolean isValidStart();
 
   /**
+   * As in {@link IndexerMetadataStorageCoordinator#announceHistoricalSegments}, this class can represent start and
+   * end of a sequence.
+   *
+   * This method converts itself into the one for start of a sequence. Most implementations can simply return
+   * {@code this}.
+   */
+  DataSourceMetadata asStartMetadata();
+
+  /**
    * Returns true if any information present in this instance matches analogous information from "other" and
    * so they are conflict-free. In other words, "one.plus(two)" and "two.plus(one)" should return equal
    * instances if "one" matches "two".
diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/ObjectMetadata.java b/server/src/main/java/org/apache/druid/indexing/overlord/ObjectMetadata.java
index bae6bee..7a8d010 100644
--- a/server/src/main/java/org/apache/druid/indexing/overlord/ObjectMetadata.java
+++ b/server/src/main/java/org/apache/druid/indexing/overlord/ObjectMetadata.java
@@ -49,6 +49,12 @@ public final class ObjectMetadata implements DataSourceMetadata
   }
 
   @Override
+  public DataSourceMetadata asStartMetadata()
+  {
+    return this;
+  }
+
+  @Override
   public boolean matches(DataSourceMetadata other)
   {
     return equals(other);
diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java
index 9df5673..421007d 100644
--- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java
+++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java
@@ -82,7 +82,6 @@ import java.util.stream.StreamSupport;
 public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStorageCoordinator
 {
   private static final Logger log = new Logger(IndexerSQLMetadataStorageCoordinator.class);
-  private static final int ALLOCATE_SEGMENT_QUIET_TRIES = 3;
 
   private final ObjectMapper jsonMapper;
   private final MetadataStorageTablesConfig dbTables;
@@ -878,9 +877,18 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
       oldCommitMetadataFromDb = jsonMapper.readValue(oldCommitMetadataBytesFromDb, DataSourceMetadata.class);
     }
 
-    final boolean startMetadataMatchesExisting = oldCommitMetadataFromDb == null
-                                                 ? startMetadata.isValidStart()
-                                                 : startMetadata.matches(oldCommitMetadataFromDb);
+    final boolean startMetadataMatchesExisting;
+
+    if (oldCommitMetadataFromDb == null) {
+      startMetadataMatchesExisting = startMetadata.isValidStart();
+    } else {
+      // Checking against the last committed metadata.
+      // Converting the last one into start metadata for checking since only the same type of metadata can be matched.
+      // Even though kafka/kinesis indexing services use different sequenceNumber types for representing
+      // start and end sequenceNumbers, the below conversion is fine because the new start sequenceNumbers are supposed
+      // to be same with end sequenceNumbers of the last commit.
+      startMetadataMatchesExisting = startMetadata.matches(oldCommitMetadataFromDb.asStartMetadata());
+    }
 
     if (!startMetadataMatchesExisting) {
       // Not in the desired start state.
@@ -889,6 +897,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
       return DataSourceMetadataUpdateResult.FAILURE;
     }
 
+    // Only endOffsets should be stored in metadata store
     final DataSourceMetadata newCommitMetadata = oldCommitMetadataFromDb == null
                                                  ? endMetadata
                                                  : oldCommitMetadataFromDb.plus(endMetadata);


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