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