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 89d3c8c36ce..0c38b2fd3bd 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 @@ -73,6 +73,12 @@ public class DerivativeDataSourceMetadata implements DataSourceMetadata return false; } + @Override + public DataSourceMetadata asStartMetadata() + { + return this; + } + @Override public boolean matches(DataSourceMetadata 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 9e38a97e136..bf0580cd7ad 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 deserializePartitionsFromMetadata( + protected SeekableStreamEndSequenceNumbers 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 createDataSourceMetadata( - SeekableStreamPartitions partitions + SeekableStreamSequenceNumbers partitions ) { return new KafkaDataSourceMetadata(partitions); @@ -201,8 +202,7 @@ public class IncrementalPublishingKafkaIndexTaskRunner extends SeekableStreamInd protected void possiblyResetDataSourceMetadata( TaskToolbox toolbox, RecordSupplier recordSupplier, - Set> assignment, - Map currOffsets + Set> 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 55b03c000f3..81ea6de396b 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 { @JsonCreator public KafkaDataSourceMetadata( - @JsonProperty("partitions") SeekableStreamPartitions kafkaPartitions + @JsonProperty("partitions") SeekableStreamSequenceNumbers kafkaPartitions ) { super(kafkaPartitions); } + @Override + public DataSourceMetadata asStartMetadata() + { + final SeekableStreamSequenceNumbers sequenceNumbers = getSeekableStreamSequenceNumbers(); + if (sequenceNumbers instanceof SeekableStreamEndSequenceNumbers) { + return createConcreteDataSourceMetaData( + ((SeekableStreamEndSequenceNumbers) sequenceNumbers).asStartPartitions(true) + ); + } else { + return this; + } + } + @Override protected SeekableStreamDataSourceMetadata createConcreteDataSourceMetaData( - String streamId, - Map newMap + SeekableStreamSequenceNumbers 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 5f3681623fa..2200e47d03c 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 startPartitions, - @JsonProperty("endPartitions") SeekableStreamPartitions endPartitions, + // startPartitions and endPartitions exist to be able to read old ioConfigs in metadata store + @JsonProperty("startPartitions") @Nullable SeekableStreamEndSequenceNumbers startPartitions, + @JsonProperty("endPartitions") @Nullable SeekableStreamEndSequenceNumbers endPartitions, + // startSequenceNumbers and endSequenceNumbers must be set for new versions + @JsonProperty("startSequenceNumbers") + @Nullable SeekableStreamStartSequenceNumbers startSequenceNumbers, + @JsonProperty("endSequenceNumbers") + @Nullable SeekableStreamEndSequenceNumbers endSequenceNumbers, @JsonProperty("consumerProperties") Map consumerProperties, @JsonProperty("pollTimeout") Long pollTimeout, @JsonProperty("useTransaction") Boolean useTransaction, @@ -51,28 +58,83 @@ public class KafkaIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig 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 startSequenceNumbers, + SeekableStreamEndSequenceNumbers endSequenceNumbers, + Map consumerProperties, + Long pollTimeout, + Boolean useTransaction, + DateTime minimumMessageTime, + DateTime maximumMessageTime + ) + { + this( + taskGroupId, + baseSequenceName, + null, + null, + startSequenceNumbers, + endSequenceNumbers, + consumerProperties, + pollTimeout, + useTransaction, + minimumMessageTime, + maximumMessageTime + ); + } + + /** + * 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 getStartPartitions() + { + // Converting to start sequence numbers. This is allowed for Kafka because the start offset is always inclusive. + final SeekableStreamStartSequenceNumbers 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 getEndPartitions() + { + return getEndSequenceNumbers(); + } + @JsonProperty public Map getConsumerProperties() { @@ -91,8 +153,8 @@ public class KafkaIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig { 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 911a64a8a04..5b186c44ca0 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 restoredMetadataMap = (Map) restoredMetadata; - final SeekableStreamPartitions restoredNextPartitions = toolbox.getObjectMapper().convertValue( + final SeekableStreamEndSequenceNumbers 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( - ioConfig.getStartPartitions().getStream(), + METADATA_NEXT_PARTITIONS, + new SeekableStreamEndSequenceNumbers<>( + ioConfig.getStartSequenceNumbers().getStream(), snapshot ) ); @@ -501,13 +504,13 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner { - final SeekableStreamPartitions finalPartitions = toolbox.getObjectMapper().convertValue( + final SeekableStreamEndSequenceNumbers 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 ) @@ -523,7 +526,7 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner recordSupplier, - Set> assignment, - Map currOffsets + Set> assignment ) { throw new UnsupportedOperationException(); @@ -712,7 +714,7 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner deserializePartitionsFromMetadata( + protected SeekableStreamEndSequenceNumbers deserializePartitionsFromMetadata( ObjectMapper mapper, Object object ) @@ -774,11 +776,13 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner( - ioConfig.getStartPartitions() - .getStream(), - partitionOffsetMap - )) + new KafkaDataSourceMetadata( + new SeekableStreamStartSequenceNumbers<>( + ioConfig.getStartSequenceNumbers().getStream(), + partitionOffsetMap, + Collections.emptySet() + ) + ) )); if (result) { @@ -1200,7 +1204,7 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner createDataSourceMetadata( - SeekableStreamPartitions partitions + SeekableStreamSequenceNumbers 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 bb388cd55e2..98494d7388a 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 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, @@ -281,9 +283,9 @@ public class KafkaSupervisor extends SeekableStreamSupervisor } @Override - protected KafkaDataSourceMetadata createDataSourceMetaData(String topic, Map map) + protected KafkaDataSourceMetadata createDataSourceMetaDataForReset(String topic, Map map) { - return new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, map)); + return new KafkaDataSourceMetadata(new SeekableStreamStartSequenceNumbers<>(topic, map, Collections.emptySet())); } @Override @@ -355,6 +357,12 @@ public class KafkaSupervisor extends SeekableStreamSupervisor return false; } + @Override + protected boolean useExclusiveStartSequenceNumberForStartSequence() + { + return false; + } + @Override protected void updateLatestSequenceFromStream( RecordSupplier recordSupplier, 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 5b609132af2..19336184ac0 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(START0.matches(START0)); + Assert.assertTrue(START0.matches(START1)); + Assert.assertTrue(START0.matches(START2)); + Assert.assertTrue(START0.matches(START3)); - Assert.assertTrue(KM1.matches(KM0)); - Assert.assertTrue(KM1.matches(KM1)); - Assert.assertFalse(KM1.matches(KM2)); - Assert.assertTrue(KM1.matches(KM3)); + Assert.assertTrue(START1.matches(START0)); + Assert.assertTrue(START1.matches(START1)); + Assert.assertFalse(START1.matches(START2)); + Assert.assertTrue(START1.matches(START3)); - Assert.assertTrue(KM2.matches(KM0)); - Assert.assertFalse(KM2.matches(KM1)); - Assert.assertTrue(KM2.matches(KM2)); - Assert.assertTrue(KM2.matches(KM3)); + Assert.assertTrue(START2.matches(START0)); + Assert.assertFalse(START2.matches(START1)); + Assert.assertTrue(START2.matches(START2)); + Assert.assertTrue(START2.matches(START3)); - Assert.assertTrue(KM3.matches(KM0)); - Assert.assertTrue(KM3.matches(KM1)); - Assert.assertTrue(KM3.matches(KM2)); - Assert.assertTrue(KM3.matches(KM3)); + 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 km(String topic, Map offsets) + private static KafkaDataSourceMetadata startMetadata(Map offsets) { - return new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, offsets)); + return new KafkaDataSourceMetadata(new SeekableStreamStartSequenceNumbers<>("foo", offsets, ImmutableSet.of())); + } + + private static KafkaDataSourceMetadata endMetadata(Map 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 556cba1a39b..ac93c1e5b6a 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,15 +79,48 @@ 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.EMPTY_SET, config.getExclusiveStartSequenceNumberPartitions()); + 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.assertEquals(Collections.emptySet(), config.getStartSequenceNumbers().getExclusivePartitions()); } @Test @@ -105,15 +148,15 @@ 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.assertEquals(Collections.EMPTY_SET, config.getExclusiveStartSequenceNumberPartitions()); + Assert.assertEquals(Collections.emptySet(), config.getStartSequenceNumbers().getExclusivePartitions()); } @Test @@ -172,7 +215,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); } @@ -258,4 +301,159 @@ 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() + ); + 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 startPartitions; + private final SeekableStreamEndSequenceNumbers endPartitions; + private final Map consumerProperties; + private final long pollTimeout; + private final boolean useTransaction; + private final Optional minimumMessageTime; + private final Optional maximumMessageTime; + + @JsonCreator + private OldKafkaIndexTaskIoConfig( + @JsonProperty("taskGroupId") int taskGroupId, + @JsonProperty("baseSequenceName") String baseSequenceName, + @JsonProperty("startPartitions") @Nullable SeekableStreamEndSequenceNumbers startPartitions, + @JsonProperty("endPartitions") @Nullable SeekableStreamEndSequenceNumbers endPartitions, + @JsonProperty("consumerProperties") Map 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 getStartPartitions() + { + return startPartitions; + } + + @JsonProperty + public SeekableStreamEndSequenceNumbers getEndPartitions() + { + return endPartitions; + } + + @JsonProperty + public Map getConsumerProperties() + { + return consumerProperties; + } + + @JsonProperty + public long getPollTimeout() + { + return pollTimeout; + } + + @JsonProperty + public boolean isUseTransaction() + { + return useTransaction; + } + + @JsonProperty + public Optional getMinimumMessageTime() + { + return minimumMessageTime; + } + + @JsonProperty + public Optional 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 0a427a0c855..ebef3fc4596 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 @@ -75,8 +75,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; @@ -408,8 +409,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, @@ -433,7 +434,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()) ); @@ -450,8 +451,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()) ); @@ -507,43 +508,24 @@ public class KafkaIndexTaskTest Map consumerProps = kafkaServer.consumerProperties(); consumerProps.put("max.poll.records", "1"); - final SeekableStreamPartitions startPartitions = new SeekableStreamPartitions<>( + final SeekableStreamStartSequenceNumbers 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 checkpoint1 = new SeekableStreamPartitions<>( + final SeekableStreamEndSequenceNumbers checkpoint1 = new SeekableStreamEndSequenceNumbers<>( topic, - ImmutableMap.of( - 0, - 5L, - 1, - 0L - ) + ImmutableMap.of(0, 5L, 1, 0L) ); - final SeekableStreamPartitions checkpoint2 = new SeekableStreamPartitions<>( + final SeekableStreamEndSequenceNumbers checkpoint2 = new SeekableStreamEndSequenceNumbers<>( topic, - ImmutableMap.of( - 0, - 4L, - 1, - 2L - ) + ImmutableMap.of(0, 4L, 1, 2L) ); - final SeekableStreamPartitions endPartitions = new SeekableStreamPartitions<>( + final SeekableStreamEndSequenceNumbers endPartitions = new SeekableStreamEndSequenceNumbers<>( topic, - ImmutableMap.of( - 0, - 10L, - 1, - 2L - ) + ImmutableMap.of(0, 10L, 1, 2L) ); final KafkaIndexTask task = createTask( null, @@ -577,7 +559,9 @@ public class KafkaIndexTaskTest DATA_SCHEMA.getDataSource(), 0, new KafkaDataSourceMetadata(startPartitions), - new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, currentOffsets)) + new KafkaDataSourceMetadata( + new SeekableStreamEndSequenceNumbers<>(topic, currentOffsets) + ) ) ) ); @@ -597,7 +581,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()) ); @@ -639,42 +625,23 @@ public class KafkaIndexTaskTest Map consumerProps = kafkaServer.consumerProperties(); consumerProps.put("max.poll.records", "1"); - final SeekableStreamPartitions startPartitions = new SeekableStreamPartitions<>( + final SeekableStreamStartSequenceNumbers startPartitions = new SeekableStreamStartSequenceNumbers<>( topic, - ImmutableMap.of( - 0, - 0L, - 1, - 0L - ) + ImmutableMap.of(0, 0L, 1, 0L), + ImmutableSet.of() ); - final SeekableStreamPartitions checkpoint1 = new SeekableStreamPartitions<>( + final SeekableStreamEndSequenceNumbers checkpoint1 = new SeekableStreamEndSequenceNumbers<>( topic, - ImmutableMap.of( - 0, - 3L, - 1, - 0L - ) + ImmutableMap.of(0, 3L, 1, 0L) ); - final SeekableStreamPartitions checkpoint2 = new SeekableStreamPartitions<>( + final SeekableStreamEndSequenceNumbers checkpoint2 = new SeekableStreamEndSequenceNumbers<>( topic, - ImmutableMap.of( - 0, - 10L, - 1, - 0L - ) + ImmutableMap.of(0, 10L, 1, 0L) ); - final SeekableStreamPartitions endPartitions = new SeekableStreamPartitions<>( + final SeekableStreamEndSequenceNumbers endPartitions = new SeekableStreamEndSequenceNumbers<>( topic, - ImmutableMap.of( - 0, - 10L, - 1, - 2L - ) + ImmutableMap.of(0, 10L, 1, 2L) ); final KafkaIndexTask task = createTask( null, @@ -696,7 +663,7 @@ public class KafkaIndexTaskTest } final Map 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) { @@ -727,7 +694,9 @@ public class KafkaIndexTaskTest DATA_SCHEMA.getDataSource(), 0, new KafkaDataSourceMetadata(startPartitions), - new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, currentOffsets)) + new KafkaDataSourceMetadata( + new SeekableStreamEndSequenceNumbers<>(topic, currentOffsets) + ) ) ) ); @@ -736,8 +705,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)) ) ) ); @@ -757,12 +728,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 SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 2L))), + new KafkaDataSourceMetadata( + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L, 1, 2L)) + ), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -794,33 +769,19 @@ public class KafkaIndexTaskTest Map consumerProps = kafkaServer.consumerProperties(); consumerProps.put("max.poll.records", "1"); - final SeekableStreamPartitions startPartitions = new SeekableStreamPartitions<>( + final SeekableStreamStartSequenceNumbers 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 checkpoint = new SeekableStreamPartitions<>( + final SeekableStreamEndSequenceNumbers checkpoint = new SeekableStreamEndSequenceNumbers<>( topic, - ImmutableMap.of( - 0, - 1L, - 1, - 0L - ) + ImmutableMap.of(0, 1L, 1, 0L) ); - final SeekableStreamPartitions endPartitions = new SeekableStreamPartitions<>( + final SeekableStreamEndSequenceNumbers endPartitions = new SeekableStreamEndSequenceNumbers<>( topic, - ImmutableMap.of( - 0, - 2L, - 1, - 0L - ) + ImmutableMap.of(0, 2L, 1, 0L) ); final KafkaIndexTask task = createTask( null, @@ -854,10 +815,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()) + ) ) ) ); @@ -872,7 +832,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()) ); @@ -905,14 +867,14 @@ public class KafkaIndexTaskTest Map consumerProps = kafkaServer.consumerProperties(); consumerProps.put("max.poll.records", "1"); - final SeekableStreamPartitions startPartitions = - new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L)); - final SeekableStreamPartitions checkpoint1 = - new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)); - final SeekableStreamPartitions checkpoint2 = - new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 9L)); - final SeekableStreamPartitions endPartitions = - new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, Long.MAX_VALUE)); + final SeekableStreamStartSequenceNumbers startPartitions = + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()); + final SeekableStreamEndSequenceNumbers checkpoint1 = + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)); + final SeekableStreamEndSequenceNumbers checkpoint2 = + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 9L)); + final SeekableStreamEndSequenceNumbers endPartitions = + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, Long.MAX_VALUE)); final KafkaIndexTask normalReplica = createTask( null, @@ -1000,8 +962,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, @@ -1033,7 +995,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()) ); @@ -1050,8 +1012,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, @@ -1084,7 +1046,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()) ); @@ -1110,8 +1072,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, @@ -1142,7 +1104,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()) ); @@ -1162,8 +1124,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, @@ -1199,8 +1161,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, @@ -1224,7 +1186,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()) ); @@ -1247,8 +1209,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, @@ -1272,7 +1234,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()) ); @@ -1298,8 +1262,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, @@ -1338,8 +1302,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, @@ -1369,7 +1333,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()) ); @@ -1416,8 +1380,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, @@ -1476,8 +1440,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, @@ -1490,8 +1454,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, @@ -1523,7 +1487,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()) ); @@ -1540,8 +1504,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, @@ -1554,8 +1518,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, @@ -1588,7 +1552,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()) ); @@ -1605,8 +1569,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, @@ -1619,8 +1583,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, @@ -1675,8 +1639,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, @@ -1708,7 +1672,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()) ); @@ -1735,8 +1701,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, @@ -1749,8 +1715,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, @@ -1783,7 +1749,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()) ); @@ -1801,8 +1769,8 @@ public class KafkaIndexTaskTest new KafkaIndexTaskIOConfig( 0, "sequence0", - new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), - new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 6L)), + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 6L)), kafkaServer.consumerProperties(), KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, @@ -1841,8 +1809,8 @@ public class KafkaIndexTaskTest new KafkaIndexTaskIOConfig( 0, "sequence0", - new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), - new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 6L)), + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 6L)), kafkaServer.consumerProperties(), KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, @@ -1880,7 +1848,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, 6L))), + new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 6L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -1906,8 +1874,8 @@ public class KafkaIndexTaskTest new KafkaIndexTaskIOConfig( 0, "sequence0", - 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)), consumerProps, KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, @@ -1916,9 +1884,10 @@ public class KafkaIndexTaskTest ) ); - final SeekableStreamPartitions checkpoint = new SeekableStreamPartitions<>( + final SeekableStreamStartSequenceNumbers checkpoint = new SeekableStreamStartSequenceNumbers<>( topic, - ImmutableMap.of(0, 5L) + ImmutableMap.of(0, 5L), + ImmutableSet.of(0) ); final ListenableFuture future1 = runTask(task1); @@ -1953,8 +1922,8 @@ public class KafkaIndexTaskTest new KafkaIndexTaskIOConfig( 0, "sequence0", - 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)), consumerProps, KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, @@ -1997,7 +1966,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, 10L))), + new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); } @@ -2010,8 +1979,8 @@ public class KafkaIndexTaskTest new KafkaIndexTaskIOConfig( 0, "sequence0", - new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), - new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 6L)), + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 6L)), kafkaServer.consumerProperties(), KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, @@ -2082,7 +2051,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, 6L))), + new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 6L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -2099,8 +2068,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, @@ -2134,8 +2103,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, @@ -2184,8 +2153,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, @@ -2210,7 +2179,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()) ); @@ -2230,8 +2199,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, @@ -2265,8 +2234,8 @@ public class KafkaIndexTaskTest new KafkaIndexTaskIOConfig( 0, "sequence0", - new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L)), - new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 13L)), + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 13L)), kafkaServer.consumerProperties(), KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, @@ -2353,7 +2322,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()) ); @@ -2375,8 +2344,6 @@ public class KafkaIndexTaskTest maxTotalRows = null; // Insert data - int numToAdd = records.size(); - try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { kafkaProducer.initTransactions(); kafkaProducer.beginTransaction(); @@ -2389,24 +2356,15 @@ public class KafkaIndexTaskTest Map consumerProps = kafkaServer.consumerProperties(); consumerProps.put("max.poll.records", "1"); - final SeekableStreamPartitions startPartitions = new SeekableStreamPartitions<>( + final SeekableStreamStartSequenceNumbers startPartitions = new SeekableStreamStartSequenceNumbers<>( topic, - ImmutableMap.of( - 0, - 0L, - 1, - 1L - ) + ImmutableMap.of(0, 0L, 1, 1L), + ImmutableSet.of() ); - final SeekableStreamPartitions endPartitions = new SeekableStreamPartitions<>( + final SeekableStreamEndSequenceNumbers 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 ee3dfe8e01c..63b27866fa8 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 @@ -187,5 +187,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 bde9052359b..b30b75ee788 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 @@ -60,8 +60,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; @@ -290,15 +291,24 @@ public class KafkaSupervisorTest extends EasyMockSupport Assert.assertFalse("minimumMessageTime", taskConfig.getMinimumMessageTime().isPresent()); Assert.assertFalse("maximumMessageTime", taskConfig.getMaximumMessageTime().isPresent()); - 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 @@ -352,26 +362,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( - Long.MAX_VALUE, - (long) task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(0) + 0L, + task1.getIOConfig().getStartSequenceNumbers().getPartitionSequenceNumberMap().get(0).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(0).longValue() + ); + Assert.assertEquals( + 0L, + task1.getIOConfig().getStartSequenceNumbers().getPartitionSequenceNumberMap().get(2).longValue() + ); + Assert.assertEquals( + Long.MAX_VALUE, + 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() ); } @@ -398,18 +417,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 @@ -516,9 +553,18 @@ public class KafkaSupervisorTest extends EasyMockSupport verifyAll(); KafkaIndexTask task = captured.getValue(); - Assert.assertEquals(1101L, (long) task.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(0)); - Assert.assertEquals(1101L, (long) task.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(1)); - Assert.assertEquals(1101L, (long) task.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(2)); + Assert.assertEquals( + 1101L, + task.getIOConfig().getStartSequenceNumbers().getPartitionSequenceNumberMap().get(0).longValue() + ); + Assert.assertEquals( + 1101L, + task.getIOConfig().getStartSequenceNumbers().getPartitionSequenceNumberMap().get(1).longValue() + ); + Assert.assertEquals( + 1101L, + task.getIOConfig().getStartSequenceNumbers().getPartitionSequenceNumberMap().get(2).longValue() + ); } @Test @@ -537,7 +583,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(0, 1, 2)) ) ).anyTimes(); expect(taskQueue.add(capture(captured))).andReturn(true); @@ -550,9 +596,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) @@ -565,7 +620,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(0, 1, 2)) ) ).anyTimes(); replayAll(); @@ -585,8 +640,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 ); @@ -596,8 +651,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 ); @@ -607,8 +662,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 ); @@ -618,8 +673,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 ); @@ -689,8 +744,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 ); @@ -698,8 +753,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 ); @@ -707,8 +762,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) ), @@ -719,8 +774,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 ); @@ -728,8 +783,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 ); @@ -738,7 +793,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(); @@ -887,8 +942,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 ); @@ -1164,10 +1219,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)); } } @@ -1183,8 +1238,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) ), @@ -1257,23 +1312,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() ); } @@ -1289,8 +1353,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 ); @@ -1354,23 +1418,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() ); } @@ -1388,8 +1461,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) ), @@ -1401,8 +1474,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) ), @@ -1623,8 +1696,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)); } } @@ -1716,8 +1789,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)); } } @@ -1757,8 +1830,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) ), @@ -1770,8 +1843,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) ), @@ -1783,8 +1856,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) ), @@ -1894,20 +1967,16 @@ public class KafkaSupervisorTest extends EasyMockSupport Capture captureDataSource = EasyMock.newCapture(); Capture 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)) @@ -1947,10 +2016,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 @@ -1975,8 +2047,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) ), @@ -1988,8 +2060,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) ), @@ -2001,8 +2073,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) ), @@ -2072,8 +2144,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) ), @@ -2085,8 +2157,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) ), @@ -2098,8 +2170,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) ), @@ -2168,8 +2240,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 ); @@ -2178,8 +2253,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 ); @@ -2188,8 +2266,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 ); @@ -2246,8 +2327,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) { @@ -2271,8 +2352,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 ); @@ -2281,8 +2365,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 ); @@ -2291,8 +2378,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 ); @@ -2317,8 +2407,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) { @@ -2331,8 +2421,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() @@ -2350,8 +2441,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 ); @@ -2360,8 +2451,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 ); @@ -2370,8 +2461,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 ); @@ -2422,8 +2513,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) { @@ -2476,8 +2569,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) ), @@ -2489,8 +2582,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) ), @@ -2502,8 +2595,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) ), @@ -2668,15 +2761,33 @@ public class KafkaSupervisorTest extends EasyMockSupport Assert.assertFalse("minimumMessageTime", taskConfig.getMinimumMessageTime().isPresent()); Assert.assertFalse("maximumMessageTime", taskConfig.getMaximumMessageTime().isPresent()); - 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 @@ -2723,7 +2834,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 @@ -2883,8 +2993,8 @@ public class KafkaSupervisorTest extends EasyMockSupport String id, String dataSource, int taskGroupId, - SeekableStreamPartitions startPartitions, - SeekableStreamPartitions endPartitions, + SeekableStreamStartSequenceNumbers startPartitions, + SeekableStreamEndSequenceNumbers 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 4592c1b6774..d3c1630cf90 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 { @JsonCreator public KinesisDataSourceMetadata( - @JsonProperty("partitions") SeekableStreamPartitions kinesisPartitions + @JsonProperty("partitions") SeekableStreamSequenceNumbers kinesisPartitions ) { super(kinesisPartitions); } @Override - protected KinesisDataSourceMetadata createConcreteDataSourceMetaData(String streamName, Map newMap) + public DataSourceMetadata asStartMetadata() { - return new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(streamName, newMap)); + final SeekableStreamSequenceNumbers sequenceNumbers = getSeekableStreamSequenceNumbers(); + if (sequenceNumbers instanceof SeekableStreamEndSequenceNumbers) { + return createConcreteDataSourceMetaData( + ((SeekableStreamEndSequenceNumbers) sequenceNumbers).asStartPartitions(false) + ); + } else { + return this; + } + } + + @Override + protected KinesisDataSourceMetadata createConcreteDataSourceMetaData( + SeekableStreamSequenceNumbers seekableStreamSequenceNumbers + ) + { + 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 3467481762a..ff1847a6de1 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 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 8dd32f24634..f312dd6c581 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 { @@ -46,15 +46,14 @@ public class KinesisIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig startPartitions, - @JsonProperty("endPartitions") SeekableStreamPartitions endPartitions, + @JsonProperty("startSequenceNumbers") SeekableStreamStartSequenceNumbers startSequenceNumbers, + @JsonProperty("endSequenceNumbers") SeekableStreamEndSequenceNumbers 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 exclusiveStartSequenceNumberPartitions, @JsonProperty("awsAssumedRoleArn") String awsAssumedRoleArn, @JsonProperty("awsExternalId") String awsExternalId, @JsonProperty("deaggregate") boolean deaggregate @@ -63,17 +62,19 @@ public class KinesisIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig 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; @@ -124,15 +125,14 @@ public class KinesisIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig { - /** * 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 * 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 } @Override - public int compareTo(@NotNull OrderedSequenceNumber o) + public int compareTo(OrderedSequenceNumber o) { KinesisSequenceNumber num = (KinesisSequenceNumber) o; if (isMaxSequenceNumber && num.isMaxSequenceNumber) { @@ -79,5 +85,4 @@ public class KinesisSequenceNumber extends OrderedSequenceNumber 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 f53bdf2ad12..332e69379af 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 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 taskTuningConfig.getFetchSequenceNumberTimeout(), taskTuningConfig.getMaxRecordsPerPoll() ); - } - @Override protected void scheduleReporting(ScheduledExecutorService reportingExec) { @@ -261,13 +264,13 @@ public class KinesisSupervisor extends SeekableStreamSupervisor } @Override - protected SeekableStreamDataSourceMetadata createDataSourceMetaData( + protected SeekableStreamDataSourceMetadata createDataSourceMetaDataForReset( String stream, Map map ) { return new KinesisDataSourceMetadata( - new SeekableStreamPartitions<>(stream, map) + new SeekableStreamStartSequenceNumbers<>(stream, map, Collections.emptySet()) ); } @@ -300,7 +303,7 @@ public class KinesisSupervisor extends SeekableStreamSupervisor @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 { 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 2c5bce1744b..fbb3d6405ff 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(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(KM1.matches(KM0)); - Assert.assertTrue(KM1.matches(KM1)); - Assert.assertFalse(KM1.matches(KM2)); - Assert.assertTrue(KM1.matches(KM3)); + 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(KM2.matches(KM0)); - Assert.assertFalse(KM2.matches(KM1)); - Assert.assertTrue(KM2.matches(KM2)); - Assert.assertTrue(KM2.matches(KM3)); + 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(KM3.matches(KM0)); - Assert.assertTrue(KM3.matches(KM1)); - Assert.assertTrue(KM3.matches(KM2)); - Assert.assertTrue(KM3.matches(KM3)); + 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 km(String stream, Map sequences) + private static KinesisDataSourceMetadata simpleStartMetadata(Map sequences) { - return new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, sequences)); + return startMetadata(sequences, sequences.keySet()); + } + + private static KinesisDataSourceMetadata startMetadata(Map sequences, Set exclusivePartitions) + { + return new KinesisDataSourceMetadata( + new SeekableStreamStartSequenceNumbers<>("foo", sequences, exclusivePartitions) + ); + } + + private static KinesisDataSourceMetadata endMetadata(Map 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 ab782299ee7..e0c77901639 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()); @@ -98,15 +98,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" @@ -123,15 +122,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()); @@ -139,7 +138,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()); @@ -153,8 +152,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); @@ -164,34 +163,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); } @@ -202,8 +201,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); @@ -213,14 +212,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); @@ -235,8 +234,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 439584fd76d..435bb2abf8e 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; @@ -396,14 +397,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, @@ -412,7 +407,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport null, null, null, - null, false ) @@ -435,13 +429,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()) ); @@ -477,14 +467,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, @@ -493,7 +477,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport null, null, null, - null, false ) @@ -516,13 +499,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()) ); @@ -558,34 +536,20 @@ public class KinesisIndexTaskTest extends EasyMockSupport replayAll(); - final SeekableStreamPartitions startPartitions = new SeekableStreamPartitions<>( + final SeekableStreamStartSequenceNumbers startPartitions = new SeekableStreamStartSequenceNumbers<>( stream, - ImmutableMap.of( - shardId1, - "0", - shardId0, - "0" - ) + ImmutableMap.of(shardId1, "0", shardId0, "0"), + ImmutableSet.of() ); - final SeekableStreamPartitions checkpoint1 = new SeekableStreamPartitions<>( + final SeekableStreamEndSequenceNumbers checkpoint1 = new SeekableStreamEndSequenceNumbers<>( stream, - ImmutableMap.of( - shardId1, - "4", - shardId0, - "0" - ) + ImmutableMap.of(shardId1, "4", shardId0, "0") ); - final SeekableStreamPartitions endPartitions = new SeekableStreamPartitions<>( + final SeekableStreamEndSequenceNumbers endPartitions = new SeekableStreamEndSequenceNumbers<>( stream, - ImmutableMap.of( - shardId1, - "9", - shardId0, - "1" - ) + ImmutableMap.of(shardId1, "9", shardId0, "1") ); final KinesisIndexTask task = createTask( null, @@ -602,7 +566,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport null, null, null, - null, false ) ); @@ -611,7 +574,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport Thread.sleep(10); } final Map 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()); @@ -625,7 +588,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)) ) ) ); @@ -645,12 +608,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()) ); @@ -694,35 +657,24 @@ public class KinesisIndexTaskTest extends EasyMockSupport replayAll(); // Insert data - final SeekableStreamPartitions startPartitions = new SeekableStreamPartitions<>( + final SeekableStreamStartSequenceNumbers 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 checkpoint1 = new SeekableStreamPartitions<>( + final SeekableStreamEndSequenceNumbers checkpoint1 = new SeekableStreamEndSequenceNumbers<>( stream, - ImmutableMap.of( - shardId1, - "2" - ) + ImmutableMap.of(shardId1, "2") ); - final SeekableStreamPartitions checkpoint2 = new SeekableStreamPartitions<>( + final SeekableStreamEndSequenceNumbers checkpoint2 = new SeekableStreamEndSequenceNumbers<>( stream, - ImmutableMap.of( - shardId1, - "9" - ) + ImmutableMap.of(shardId1, "9") ); - final SeekableStreamPartitions endPartitions = new SeekableStreamPartitions<>( + final SeekableStreamEndSequenceNumbers endPartitions = new SeekableStreamEndSequenceNumbers<>( stream, - ImmutableMap.of( - shardId1, - "10" - ) + ImmutableMap.of(shardId1, "10") ); final KinesisIndexTask task = createTask( @@ -740,7 +692,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport null, null, null, - null, false ) ); @@ -775,7 +726,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)) ) ) ); @@ -784,8 +735,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)) ) ) ); @@ -804,10 +757,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()) ); @@ -844,14 +794,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, @@ -860,7 +804,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport null, null, null, - null, false ) ); @@ -887,14 +830,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()) ); @@ -927,14 +863,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"), @@ -943,7 +873,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport null, null, null, - null, false ) ); @@ -972,13 +901,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()) ); @@ -1020,14 +943,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, @@ -1036,7 +953,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport null, null, null, - null, false ) ); @@ -1063,13 +979,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()) ); @@ -1104,14 +1014,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, @@ -1120,7 +1024,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport null, null, null, - null, false ) ); @@ -1167,14 +1070,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, @@ -1183,7 +1080,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport null, null, null, - null, false ) ); @@ -1205,10 +1101,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()) ); @@ -1244,14 +1139,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, @@ -1260,7 +1149,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport null, null, null, - null, false ) ); @@ -1282,10 +1170,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()) ); @@ -1321,14 +1208,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, @@ -1337,7 +1218,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport null, null, null, - null, false ) ); @@ -1387,14 +1267,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, @@ -1403,7 +1277,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport null, null, null, - null, false ) ); @@ -1432,13 +1305,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()) ); @@ -1497,14 +1365,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, @@ -1513,7 +1375,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport null, null, null, - null, false ) ); @@ -1586,14 +1447,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, @@ -1602,7 +1457,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport null, null, null, - null, false ) ); @@ -1611,14 +1465,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, @@ -1627,7 +1475,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport null, null, null, - null, false ) ); @@ -1655,13 +1502,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()) ); @@ -1697,14 +1539,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, @@ -1713,7 +1549,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport null, null, null, - null, false ) ); @@ -1722,14 +1557,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, @@ -1738,7 +1567,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport null, null, null, - null, false ) ); @@ -1765,14 +1593,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()) ); @@ -1808,14 +1629,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, @@ -1824,7 +1639,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport null, null, null, - null, false ) ); @@ -1833,14 +1647,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, @@ -1849,7 +1657,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport null, null, null, - null, false ) ); @@ -1916,18 +1723,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, @@ -1936,7 +1737,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport null, null, null, - null, false ) ); @@ -1963,12 +1763,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()) ); @@ -2010,14 +1807,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, @@ -2026,7 +1817,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport null, null, null, - null, false ) ); @@ -2035,14 +1825,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, @@ -2051,7 +1835,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport null, null, null, - null, false ) ); @@ -2081,12 +1864,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()) ); @@ -2121,14 +1900,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, @@ -2137,7 +1910,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport null, null, null, - null, false ) ); @@ -2177,21 +1949,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 @@ -2225,10 +1990,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()) ); @@ -2271,14 +2033,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, @@ -2287,14 +2043,14 @@ public class KinesisIndexTaskTest extends EasyMockSupport null, null, null, - null, false ) ); - final SeekableStreamPartitions checkpoint1 = new SeekableStreamPartitions<>( + final SeekableStreamStartSequenceNumbers checkpoint1 = new SeekableStreamStartSequenceNumbers<>( stream, - ImmutableMap.of(shardId1, "4") + ImmutableMap.of(shardId1, "4"), + ImmutableSet.of() ); final ListenableFuture future1 = runTask(task1); @@ -2339,21 +2095,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 @@ -2385,10 +2134,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()) ); } @@ -2413,14 +2160,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, @@ -2429,7 +2170,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport null, null, null, - null, false ) ); @@ -2491,12 +2231,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()) ); @@ -2506,7 +2243,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); } - @Test(timeout = 60_000L) public void testRunContextSequenceAheadOfStartingOffsets() throws Exception { @@ -2546,14 +2282,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, @@ -2562,7 +2292,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport null, null, null, - null, false ), context @@ -2582,10 +2311,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()) ); @@ -2630,22 +2356,23 @@ public class KinesisIndexTaskTest extends EasyMockSupport replayAll(); - final SeekableStreamPartitions startPartitions = new SeekableStreamPartitions<>( + final SeekableStreamStartSequenceNumbers startPartitions = new SeekableStreamStartSequenceNumbers<>( stream, - ImmutableMap.of(shardId1, "0") + ImmutableMap.of(shardId1, "0"), + ImmutableSet.of() ); - final SeekableStreamPartitions checkpoint1 = new SeekableStreamPartitions<>( + final SeekableStreamEndSequenceNumbers checkpoint1 = new SeekableStreamEndSequenceNumbers<>( stream, ImmutableMap.of(shardId1, "4") ); - final SeekableStreamPartitions checkpoint2 = new SeekableStreamPartitions<>( + final SeekableStreamEndSequenceNumbers checkpoint2 = new SeekableStreamEndSequenceNumbers<>( stream, ImmutableMap.of(shardId1, "9") ); - final SeekableStreamPartitions endPartitions = new SeekableStreamPartitions<>( + final SeekableStreamEndSequenceNumbers endPartitions = new SeekableStreamEndSequenceNumbers<>( stream, ImmutableMap.of(shardId1, "100") // simulating unlimited ); @@ -2662,7 +2389,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport null, null, null, - null, false ); final KinesisIndexTask normalReplica = createTask( @@ -2740,10 +2466,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 5e28b70b1f1..73c1d4692fa 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 @@ -46,6 +46,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; @@ -55,7 +56,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; @@ -246,24 +248,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) ); } @@ -301,27 +303,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) ); } @@ -358,43 +360,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) ); } @@ -526,12 +528,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(); @@ -547,11 +544,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) ); } @@ -577,12 +574,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(); @@ -613,14 +605,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 ); @@ -630,18 +616,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 ); @@ -651,18 +630,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 ); @@ -672,18 +644,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 ); @@ -769,14 +735,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 ); @@ -784,14 +744,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 ); @@ -799,18 +753,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 ); @@ -818,14 +765,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 ); @@ -833,14 +774,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 ); @@ -1029,18 +964,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 ); @@ -1050,7 +983,7 @@ public class KinesisSupervisorTest extends EasyMockSupport Capture 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(); @@ -1278,17 +1211,15 @@ public class KinesisSupervisorTest extends EasyMockSupport supervisorRecordSupplier.seek(anyObject(), anyString()); expectLastCall().anyTimes(); - Capture captured = Capture.newInstance(CaptureType.ALL); + final Capture 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(); @@ -1296,14 +1227,14 @@ public class KinesisSupervisorTest extends EasyMockSupport supervisor.runInternal(); verifyAll(); - List tasks = captured.getValues(); + final List 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 secondTasks = Capture.newInstance(CaptureType.ALL); EasyMock.expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); for (Task task : tasks) { EasyMock.expect(taskStorage.getStatus(task.getId())) @@ -1346,7 +1277,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> checkpoints1 = new TreeMap<>(); checkpoints1.put(0, ImmutableMap.of( @@ -1370,7 +1301,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()); @@ -1379,14 +1310,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() ); } } @@ -1415,18 +1351,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 ); @@ -1521,24 +1455,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) ); } @@ -1565,18 +1499,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 ); @@ -1660,24 +1592,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) ); } @@ -1706,21 +1638,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 ); @@ -1729,18 +1660,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 ); @@ -2029,10 +1961,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) ); } } @@ -2150,7 +2082,7 @@ public class KinesisSupervisorTest extends EasyMockSupport KinesisIndexTaskIOConfig taskConfig = ((KinesisIndexTask) task).getIOConfig(); Assert.assertEquals( "0", - taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) + taskConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(shardId1) ); } } @@ -2205,18 +2137,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 ); @@ -2225,18 +2155,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 ); @@ -2245,18 +2182,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 ); @@ -2391,35 +2330,44 @@ public class KinesisSupervisorTest extends EasyMockSupport Capture captureDataSource = EasyMock.newCapture(); Capture 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) @@ -2447,7 +2395,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(); @@ -2456,13 +2404,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 @@ -2498,21 +2446,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 ); @@ -2521,21 +2464,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 ); @@ -2544,21 +2486,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 ); @@ -2645,18 +2586,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 ); @@ -2665,18 +2608,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 ); @@ -2685,18 +2630,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 ); @@ -2784,18 +2731,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 ); @@ -2804,18 +2753,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 ); @@ -2824,18 +2775,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 ); @@ -2884,12 +2837,7 @@ public class KinesisSupervisorTest extends EasyMockSupport expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); final TreeMap> 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); @@ -2911,8 +2859,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) { @@ -2951,18 +2901,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 ); @@ -2971,18 +2923,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 ); @@ -2991,18 +2945,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 ); @@ -3028,8 +2984,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) { @@ -3061,14 +3017,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 ); @@ -3077,14 +3030,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 ); @@ -3093,14 +3043,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 ); @@ -3153,8 +3100,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) { @@ -3221,18 +3168,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 ); @@ -3241,18 +3186,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 ); @@ -3261,18 +3208,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 ); @@ -3589,8 +3538,8 @@ public class KinesisSupervisorTest extends EasyMockSupport String id, String dataSource, int taskGroupId, - SeekableStreamPartitions startPartitions, - SeekableStreamPartitions endPartitions, + SeekableStreamStartSequenceNumbers startPartitions, + SeekableStreamEndSequenceNumbers endPartitions, DateTime minimumMessageTime, DateTime maximumMessageTime ) @@ -3613,7 +3562,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 b9e8d9ab82f..d0c4f98b1ed 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 implements DataSourceMetadata { - private final SeekableStreamPartitions seekableStreamPartitions; + private final SeekableStreamSequenceNumbers seekableStreamSequenceNumbers; public SeekableStreamDataSourceMetadata( - SeekableStreamPartitions seekableStreamPartitions + SeekableStreamSequenceNumbers seekableStreamSequenceNumbers ) { - this.seekableStreamPartitions = seekableStreamPartitions; + this.seekableStreamSequenceNumbers = seekableStreamSequenceNumbers; } @JsonProperty("partitions") - public SeekableStreamPartitions getSeekableStreamPartitions() + public SeekableStreamSequenceNumbers getSeekableStreamSequenceNumbers() { - return seekableStreamPartitions; + return seekableStreamSequenceNumbers; } @Override @@ -64,7 +62,7 @@ public abstract class SeekableStreamDataSourceMetadata that = (SeekableStreamDataSourceMetadata) other; + //noinspection unchecked + final SeekableStreamDataSourceMetadata that = + (SeekableStreamDataSourceMetadata) other; - if (that.getSeekableStreamPartitions().getStream().equals(seekableStreamPartitions.getStream())) { - // Same stream, merge sequences. - final Map newMap = new HashMap<>(); - - for (Map.Entry entry : seekableStreamPartitions.getPartitionSequenceNumberMap() - .entrySet()) { - newMap.put(entry.getKey(), entry.getValue()); - } - - for (Map.Entry 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 that = (SeekableStreamDataSourceMetadata) other; + //noinspection unchecked + final SeekableStreamDataSourceMetadata that = + (SeekableStreamDataSourceMetadata) other; - if (that.getSeekableStreamPartitions().getStream().equals(seekableStreamPartitions.getStream())) { - // Same stream, remove partitions present in "that" from "this" - final Map newMap = new HashMap<>(); - - for (Map.Entry 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 implements + SeekableStreamSequenceNumbers +{ + // stream/topic + private final String stream; + // partitionId -> sequence number + private final Map partitionSequenceNumberMap; + + @JsonCreator + public SeekableStreamEndSequenceNumbers( + @JsonProperty("stream") final String stream, + // kept for backward compatibility + @JsonProperty("topic") final String topic, + @JsonProperty("partitionSequenceNumberMap") + final Map partitionSequenceNumberMap, + // kept for backward compatibility + @JsonProperty("partitionOffsetMap") final Map 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 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 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 getPartitionSequenceNumberMap() + { + return partitionSequenceNumberMap; + } + + @Override + public SeekableStreamSequenceNumbers plus( + SeekableStreamSequenceNumbers other + ) + { + if (this.getClass() != other.getClass()) { + throw new IAE( + "Expected instance of %s, got %s", + this.getClass().getCanonicalName(), + other.getClass().getCanonicalName() + ); + } + + final SeekableStreamEndSequenceNumbers otherEnd = + (SeekableStreamEndSequenceNumbers) other; + + if (stream.equals(otherEnd.stream)) { + // Same stream, merge sequences. + final Map newMap = new HashMap<>(partitionSequenceNumberMap); + newMap.putAll(otherEnd.partitionSequenceNumberMap); + return new SeekableStreamEndSequenceNumbers<>(stream, newMap); + } else { + // Different stream, prefer "other". + return other; + } + } + + @Override + public SeekableStreamSequenceNumbers minus( + SeekableStreamSequenceNumbers other + ) + { + if (this.getClass() != other.getClass()) { + throw new IAE( + "Expected instance of %s, got %s", + this.getClass().getCanonicalName(), + other.getClass().getCanonicalName() + ); + } + + final SeekableStreamEndSequenceNumbers otherEnd = + (SeekableStreamEndSequenceNumbers) other; + + if (stream.equals(otherEnd.stream)) { + // Same stream, remove partitions present in "that" from "this" + final Map newMap = new HashMap<>(); + + for (Entry 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 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 7e16c35268e..1ecb7c26859 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,68 +26,51 @@ 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 implements IOConfig { private static final boolean DEFAULT_USE_TRANSACTION = true; - private static final boolean DEFAULT_SKIP_OFFSET_GAPS = false; @Nullable private final Integer taskGroupId; private final String baseSequenceName; - private final SeekableStreamPartitions startPartitions; - private final SeekableStreamPartitions endPartitions; + private final SeekableStreamStartSequenceNumbers startSequenceNumbers; + private final SeekableStreamEndSequenceNumbers endSequenceNumbers; private final boolean useTransaction; private final Optional minimumMessageTime; private final Optional maximumMessageTime; - private final Set exclusiveStartSequenceNumberPartitions; public SeekableStreamIndexTaskIOConfig( final @Nullable Integer taskGroupId, // can be null for backward compabitility final String baseSequenceName, - final SeekableStreamPartitions startPartitions, - final SeekableStreamPartitions endPartitions, + final SeekableStreamStartSequenceNumbers startSequenceNumbers, + final SeekableStreamEndSequenceNumbers endSequenceNumbers, final Boolean useTransaction, final DateTime minimumMessageTime, - final DateTime maximumMessageTime, - final Set exclusiveStartSequenceNumberPartitions + final DateTime maximumMessageTime ) { 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.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 getExclusiveStartSequenceNumberPartitions() - { - return exclusiveStartSequenceNumberPartitions; - } - @Nullable @JsonProperty public Integer getTaskGroupId() @@ -102,15 +85,15 @@ public abstract class SeekableStreamIndexTaskIOConfig getStartPartitions() + public SeekableStreamStartSequenceNumbers getStartSequenceNumbers() { - return startPartitions; + return startSequenceNumbers; } @JsonProperty - public SeekableStreamPartitions getEndPartitions() + public SeekableStreamEndSequenceNumbers 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 ee9a7e2453e..3907d545117 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(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> checkpoints = getCheckPointsFromContext( toolbox, @@ -299,7 +298,7 @@ public abstract class SeekableStreamIndexTaskRunner( 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 createSequenceNumber(partitionOffsetEntry.getValue()).compareTo( - createSequenceNumber(ioConfig.getStartPartitions() + createSequenceNumber(ioConfig.getStartSequenceNumbers() .getPartitionSequenceNumberMap() .get(partitionOffsetEntry.getKey()) )) >= 0 @@ -369,27 +366,28 @@ public abstract class SeekableStreamIndexTaskRunner restoredMetadataMap = (Map) restoredMetadata; - final SeekableStreamPartitions restoredNextPartitions = deserializePartitionsFromMetadata( - toolbox.getObjectMapper(), - restoredMetadataMap.get(METADATA_NEXT_PARTITIONS) - ); + final SeekableStreamEndSequenceNumbers 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 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( - ioConfig.getStartPartitions().getStream(), - snapshot - ) - ); + return ImmutableMap.of(METADATA_NEXT_PARTITIONS, new SeekableStreamEndSequenceNumbers<>(stream, snapshot)); } @Override @@ -458,7 +451,7 @@ public abstract class SeekableStreamIndexTaskRunner> 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 sequenceToCheckpoint = null; for (OrderedPartitionableRecord record : records) { final boolean shouldProcess = verifyRecordInRange(record.getPartitionId(), record.getSequenceNumber()); @@ -652,11 +645,19 @@ public abstract class SeekableStreamIndexTaskRunner( - 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( - 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 deserializePartitionsFromMetadata( + protected abstract SeekableStreamEndSequenceNumbers deserializePartitionsFromMetadata( ObjectMapper mapper, Object object ); @@ -1780,7 +1783,7 @@ public abstract class SeekableStreamIndexTaskRunner createDataSourceMetadata( - SeekableStreamPartitions partitions + SeekableStreamSequenceNumbers partitions ); /** @@ -1795,17 +1798,11 @@ public abstract class SeekableStreamIndexTaskRunner recordSupplier, - Set> assignment, - Map currOffsets + Set> 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 dc3ff87aff1..00000000000 --- 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 partition id type - * @param sequence number type - */ -public class SeekableStreamPartitions -{ - // 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 partitionIdToSequenceNumberMap; - - @JsonCreator - public SeekableStreamPartitions( - @JsonProperty("stream") final String stream, - // kept for backward compatibility - @JsonProperty("topic") final String topic, - @JsonProperty("partitionSequenceNumberMap") - final Map partitionSequenceNumberMap, - // kept for backward compatibility - @JsonProperty("partitionOffsetMap") final Map 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 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 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 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 00000000000..a790974e25f --- /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 +{ + /** + * Returns the stream/topic name. + */ + String getStream(); + + /** + * Returns a map of partitionId -> sequenceNumber. + */ + Map getPartitionSequenceNumberMap(); + + /** + * Merges this and the given other and returns the merged result. + * + * @see DataSourceMetadata#plus + */ + SeekableStreamSequenceNumbers plus( + SeekableStreamSequenceNumbers other + ); + + /** + * Subtracts the given other from this and returns the result. + * + * @see DataSourceMetadata#minus + */ + SeekableStreamSequenceNumbers minus( + SeekableStreamSequenceNumbers 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 00000000000..f737292275b --- /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 implements + SeekableStreamSequenceNumbers +{ + // stream/topic + private final String stream; + // partitionId -> sequence number + private final Map partitionSequenceNumberMap; + private final Set exclusivePartitions; + + @JsonCreator + public SeekableStreamStartSequenceNumbers( + @JsonProperty("stream") final String stream, + @JsonProperty("partitionSequenceNumberMap") + final Map partitionSequenceNumberMap, + @JsonProperty("exclusivePartitions") @Nullable final Set 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 getPartitionSequenceNumberMap() + { + return partitionSequenceNumberMap; + } + + @Override + public SeekableStreamSequenceNumbers plus( + SeekableStreamSequenceNumbers other + ) + { + if (this.getClass() != other.getClass()) { + throw new IAE( + "Expected instance of %s, got %s", + this.getClass().getCanonicalName(), + other.getClass().getCanonicalName() + ); + } + + final SeekableStreamStartSequenceNumbers otherStart = + (SeekableStreamStartSequenceNumbers) other; + + if (stream.equals(otherStart.stream)) { + // Same stream, merge sequences. + final Map 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 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 minus( + SeekableStreamSequenceNumbers other + ) + { + if (this.getClass() != other.getClass()) { + throw new IAE( + "Expected instance of %s, got %s", + this.getClass().getCanonicalName(), + other.getClass().getCanonicalName() + ); + } + + final SeekableStreamStartSequenceNumbers otherStart = + (SeekableStreamStartSequenceNumbers) other; + + if (stream.equals(otherStart.stream)) { + // Same stream, remove partitions present in "that" from "this" + final Map newMap = new HashMap<>(); + final Set newExclusivePartitions = new HashSet<>(); + + for (Entry 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 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 61bb35a7778..9cbafd0c392 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 private final Set exclusiveStartPartitions; private final Set 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 final Map startOffsets; final Map endOffsets; + private boolean checkpointed; + @JsonCreator public SequenceMetadata( @JsonProperty("sequenceId") int sequenceId, @@ -274,9 +275,9 @@ public class SequenceMetadata // 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 { return (segments, commitMetadata) -> { final Map commitMetaMap = (Map) Preconditions.checkNotNull(commitMetadata, "commitMetadata"); - final SeekableStreamPartitions finalPartitions = + final SeekableStreamEndSequenceNumbers finalPartitions = runner.deserializePartitionsFromMetadata( toolbox.getObjectMapper(), commitMetaMap.get(SeekableStreamIndexTaskRunner.METADATA_PUBLISH_PARTITIONS) @@ -322,7 +323,11 @@ public class SequenceMetadata 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 fc94ca26171..ab7f3bcbd7c 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 @@ -60,7 +60,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; @@ -375,7 +375,7 @@ public abstract class SeekableStreamSupervisor 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; @@ -1097,7 +1097,7 @@ public abstract class SeekableStreamSupervisor resetMetadata = (SeekableStreamDataSourceMetadata) 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)) { @@ -1113,12 +1113,12 @@ public abstract class SeekableStreamSupervisor resetPartitionOffset : resetMetadata.getSeekableStreamPartitions() + for (Entry resetPartitionOffset : resetMetadata.getSeekableStreamSequenceNumbers() .getPartitionSequenceNumberMap() .entrySet()) { final SequenceOffsetType partitionOffsetInMetadataStore = currentMetadata == null ? null - : currentMetadata.getSeekableStreamPartitions() + : currentMetadata.getSeekableStreamSequenceNumbers() .getPartitionSequenceNumberMap() .get(resetPartitionOffset.getKey()); final TaskGroup partitionTaskGroup = activelyReadingTaskGroups.get( @@ -1152,7 +1152,7 @@ public abstract class SeekableStreamSupervisor { + resetMetadata.getSeekableStreamSequenceNumbers().getPartitionSequenceNumberMap().keySet().forEach(partition -> { final int groupId = getTaskGroupIdForPartition(partition); killTaskGroupForPartitions(ImmutableSet.of(partition), "DataSourceMetadata is updated while reset"); activelyReadingTaskGroups.remove(groupId); @@ -1164,7 +1164,7 @@ public abstract class SeekableStreamSupervisor it = seekableStreamIndexTask.getIOConfig() - .getStartPartitions() + .getStartSequenceNumbers() .getPartitionSequenceNumberMap() .keySet() .iterator(); @@ -1264,7 +1264,7 @@ public abstract class SeekableStreamSupervisor latestDataSourceMetadata = (SeekableStreamDataSourceMetadata) rawDataSourceMetadata; final boolean hasValidOffsetsFromDb = latestDataSourceMetadata != null && - latestDataSourceMetadata.getSeekableStreamPartitions() != null && + latestDataSourceMetadata.getSeekableStreamSequenceNumbers() != null && ioConfig.getStream().equals( - latestDataSourceMetadata.getSeekableStreamPartitions().getStream() + latestDataSourceMetadata.getSeekableStreamSequenceNumbers().getStream() ); final Map latestOffsetsFromDb; if (hasValidOffsetsFromDb) { - latestOffsetsFromDb = latestDataSourceMetadata.getSeekableStreamPartitions().getPartitionSequenceNumberMap(); + latestOffsetsFromDb = latestDataSourceMetadata.getSeekableStreamSequenceNumbers().getPartitionSequenceNumberMap(); } else { latestOffsetsFromDb = null; } @@ -1647,7 +1649,7 @@ public abstract class SeekableStreamSupervisor> startingOffsets = generateStartingSequencesForPartitionGroup( - groupId); + final Map> startingOffsets = + generateStartingSequencesForPartitionGroup(groupId); ImmutableMap 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 exclusiveStartSequenceNumberPartitions = !useExclusiveStartingSequence ? Collections.emptySet() @@ -2344,7 +2347,7 @@ public abstract class SeekableStreamSupervisor partitions = ((SeekableStreamDataSourceMetadata) dataSourceMetadata) - .getSeekableStreamPartitions(); + SeekableStreamSequenceNumbers partitions = ((SeekableStreamDataSourceMetadata) dataSourceMetadata) + .getSeekableStreamSequenceNumbers(); if (partitions != null) { if (!ioConfig.getStream().equals(partitions.getStream())) { log.warn( @@ -2639,11 +2642,11 @@ public abstract class SeekableStreamSupervisor sequence * * @return specific instance of datasource metadata */ - protected abstract SeekableStreamDataSourceMetadata createDataSourceMetaData( + protected abstract SeekableStreamDataSourceMetadata createDataSourceMetaDataForReset( String stream, Map map ); @@ -2818,4 +2821,11 @@ public abstract class SeekableStreamSupervisor offsetMap = ImmutableMap.of(1, 2L, 3, 4L); - final SeekableStreamPartitions partitions = new SeekableStreamPartitions<>(stream, offsetMap); + final SeekableStreamEndSequenceNumbers partitions = new SeekableStreamEndSequenceNumbers<>( + stream, + offsetMap + ); final String serializedString = OBJECT_MAPPER.writeValueAsString(partitions); // Check round-trip. - final SeekableStreamPartitions partitions2 = OBJECT_MAPPER.readValue( + final SeekableStreamEndSequenceNumbers partitions2 = OBJECT_MAPPER.readValue( serializedString, - new TypeReference>() {} + new TypeReference>() {} ); Assert.assertEquals("Round trip", partitions, partitions2); @@ -69,4 +73,26 @@ public class SeekableStreamPartitionsTest OBJECT_MAPPER.convertValue(asMap.get("partitionOffsetMap"), new TypeReference>() {}) ); } + + @Test + public void testConvertToStart() + { + final String stream = "topic"; + final Map offsetMap = ImmutableMap.of(1, 2L, 3, 4L); + + final SeekableStreamEndSequenceNumbers 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 5148aede455..23579a0c8a4 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 @@ -44,6 +44,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 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 bae6bee5e64..7a8d0104248 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 @@ -48,6 +48,12 @@ public final class ObjectMetadata implements DataSourceMetadata return theObject == null; } + @Override + public DataSourceMetadata asStartMetadata() + { + return this; + } + @Override public boolean matches(DataSourceMetadata 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 3cdbd0d022e..16f320e536f 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -81,7 +81,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; @@ -877,9 +876,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. @@ -888,6 +896,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);