mirror of https://github.com/apache/druid.git
Fix exclusivity for start offset in kinesis indexing service & check exclusivity properly in IndexerSQLMetadataStorageCoordinator (#7291)
* Fix exclusivity for start offset in kinesis indexing service * some adjustment * Fix SeekableStreamDataSourceMetadata * Add missing javadocs * Add missing comments and unit test * fix SeekableStreamStartSequenceNumbers.plus and add comments * remove extra exclusivePartitions in KafkaIOConfig and fix downgrade issue * Add javadocs * fix compilation * fix test * remove unused variable
This commit is contained in:
parent
e170203876
commit
0c5dcf5586
|
@ -73,6 +73,12 @@ public class DerivativeDataSourceMetadata implements DataSourceMetadata
|
|||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataSourceMetadata asStartMetadata()
|
||||
{
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean matches(DataSourceMetadata other)
|
||||
{
|
||||
|
|
|
@ -26,8 +26,9 @@ import org.apache.druid.data.input.impl.InputRowParser;
|
|||
import org.apache.druid.indexing.common.TaskToolbox;
|
||||
import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamSequenceNumbers;
|
||||
import org.apache.druid.indexing.seekablestream.SequenceMetadata;
|
||||
import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord;
|
||||
import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber;
|
||||
|
@ -112,14 +113,14 @@ public class IncrementalPublishingKafkaIndexTaskRunner extends SeekableStreamInd
|
|||
}
|
||||
|
||||
@Override
|
||||
protected SeekableStreamPartitions<Integer, Long> deserializePartitionsFromMetadata(
|
||||
protected SeekableStreamEndSequenceNumbers<Integer, Long> deserializePartitionsFromMetadata(
|
||||
ObjectMapper mapper,
|
||||
Object object
|
||||
)
|
||||
{
|
||||
return mapper.convertValue(object, mapper.getTypeFactory().constructParametrizedType(
|
||||
SeekableStreamPartitions.class,
|
||||
SeekableStreamPartitions.class,
|
||||
SeekableStreamEndSequenceNumbers.class,
|
||||
SeekableStreamEndSequenceNumbers.class,
|
||||
Integer.class,
|
||||
Long.class
|
||||
));
|
||||
|
@ -185,7 +186,7 @@ public class IncrementalPublishingKafkaIndexTaskRunner extends SeekableStreamInd
|
|||
|
||||
@Override
|
||||
protected SeekableStreamDataSourceMetadata<Integer, Long> createDataSourceMetadata(
|
||||
SeekableStreamPartitions<Integer, Long> partitions
|
||||
SeekableStreamSequenceNumbers<Integer, Long> partitions
|
||||
)
|
||||
{
|
||||
return new KafkaDataSourceMetadata(partitions);
|
||||
|
@ -201,8 +202,7 @@ public class IncrementalPublishingKafkaIndexTaskRunner extends SeekableStreamInd
|
|||
protected void possiblyResetDataSourceMetadata(
|
||||
TaskToolbox toolbox,
|
||||
RecordSupplier<Integer, Long> recordSupplier,
|
||||
Set<StreamPartition<Integer>> assignment,
|
||||
Map<Integer, Long> currOffsets
|
||||
Set<StreamPartition<Integer>> assignment
|
||||
)
|
||||
{
|
||||
// do nothing
|
||||
|
|
|
@ -21,28 +21,40 @@ package org.apache.druid.indexing.kafka;
|
|||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.apache.druid.indexing.overlord.DataSourceMetadata;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
|
||||
|
||||
import java.util.Map;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamSequenceNumbers;
|
||||
|
||||
public class KafkaDataSourceMetadata extends SeekableStreamDataSourceMetadata<Integer, Long>
|
||||
{
|
||||
|
||||
@JsonCreator
|
||||
public KafkaDataSourceMetadata(
|
||||
@JsonProperty("partitions") SeekableStreamPartitions<Integer, Long> kafkaPartitions
|
||||
@JsonProperty("partitions") SeekableStreamSequenceNumbers<Integer, Long> kafkaPartitions
|
||||
)
|
||||
{
|
||||
super(kafkaPartitions);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataSourceMetadata asStartMetadata()
|
||||
{
|
||||
final SeekableStreamSequenceNumbers<Integer, Long> sequenceNumbers = getSeekableStreamSequenceNumbers();
|
||||
if (sequenceNumbers instanceof SeekableStreamEndSequenceNumbers) {
|
||||
return createConcreteDataSourceMetaData(
|
||||
((SeekableStreamEndSequenceNumbers<Integer, Long>) sequenceNumbers).asStartPartitions(true)
|
||||
);
|
||||
} else {
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected SeekableStreamDataSourceMetadata<Integer, Long> createConcreteDataSourceMetaData(
|
||||
String streamId,
|
||||
Map<Integer, Long> newMap
|
||||
SeekableStreamSequenceNumbers<Integer, Long> seekableStreamSequenceNumbers
|
||||
)
|
||||
{
|
||||
return new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(streamId, newMap));
|
||||
return new KafkaDataSourceMetadata(seekableStreamSequenceNumbers);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,8 +23,9 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -39,8 +40,14 @@ public class KafkaIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig<Inte
|
|||
public KafkaIndexTaskIOConfig(
|
||||
@JsonProperty("taskGroupId") @Nullable Integer taskGroupId, // can be null for backward compabitility
|
||||
@JsonProperty("baseSequenceName") String baseSequenceName,
|
||||
@JsonProperty("startPartitions") SeekableStreamPartitions<Integer, Long> startPartitions,
|
||||
@JsonProperty("endPartitions") SeekableStreamPartitions<Integer, Long> endPartitions,
|
||||
// startPartitions and endPartitions exist to be able to read old ioConfigs in metadata store
|
||||
@JsonProperty("startPartitions") @Nullable SeekableStreamEndSequenceNumbers<Integer, Long> startPartitions,
|
||||
@JsonProperty("endPartitions") @Nullable SeekableStreamEndSequenceNumbers<Integer, Long> endPartitions,
|
||||
// startSequenceNumbers and endSequenceNumbers must be set for new versions
|
||||
@JsonProperty("startSequenceNumbers")
|
||||
@Nullable SeekableStreamStartSequenceNumbers<Integer, Long> startSequenceNumbers,
|
||||
@JsonProperty("endSequenceNumbers")
|
||||
@Nullable SeekableStreamEndSequenceNumbers<Integer, Long> endSequenceNumbers,
|
||||
@JsonProperty("consumerProperties") Map<String, Object> consumerProperties,
|
||||
@JsonProperty("pollTimeout") Long pollTimeout,
|
||||
@JsonProperty("useTransaction") Boolean useTransaction,
|
||||
|
@ -51,28 +58,83 @@ public class KafkaIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig<Inte
|
|||
super(
|
||||
taskGroupId,
|
||||
baseSequenceName,
|
||||
startPartitions,
|
||||
endPartitions,
|
||||
startSequenceNumbers == null
|
||||
? Preconditions.checkNotNull(startPartitions, "startPartitions").asStartPartitions(true)
|
||||
: startSequenceNumbers,
|
||||
endSequenceNumbers == null ? endPartitions : endSequenceNumbers,
|
||||
useTransaction,
|
||||
minimumMessageTime,
|
||||
maximumMessageTime,
|
||||
null
|
||||
maximumMessageTime
|
||||
);
|
||||
|
||||
this.consumerProperties = Preconditions.checkNotNull(consumerProperties, "consumerProperties");
|
||||
this.pollTimeout = pollTimeout != null ? pollTimeout : KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS;
|
||||
|
||||
for (int partition : endPartitions.getPartitionSequenceNumberMap().keySet()) {
|
||||
final SeekableStreamEndSequenceNumbers<Integer, Long> myEndSequenceNumbers = getEndSequenceNumbers();
|
||||
for (int partition : myEndSequenceNumbers.getPartitionSequenceNumberMap().keySet()) {
|
||||
Preconditions.checkArgument(
|
||||
endPartitions.getPartitionSequenceNumberMap()
|
||||
myEndSequenceNumbers.getPartitionSequenceNumberMap()
|
||||
.get(partition)
|
||||
.compareTo(startPartitions.getPartitionSequenceNumberMap().get(partition)) >= 0,
|
||||
.compareTo(getStartSequenceNumbers().getPartitionSequenceNumberMap().get(partition)) >= 0,
|
||||
"end offset must be >= start offset for partition[%s]",
|
||||
partition
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
public KafkaIndexTaskIOConfig(
|
||||
int taskGroupId,
|
||||
String baseSequenceName,
|
||||
SeekableStreamStartSequenceNumbers<Integer, Long> startSequenceNumbers,
|
||||
SeekableStreamEndSequenceNumbers<Integer, Long> endSequenceNumbers,
|
||||
Map<String, Object> consumerProperties,
|
||||
Long pollTimeout,
|
||||
Boolean useTransaction,
|
||||
DateTime minimumMessageTime,
|
||||
DateTime maximumMessageTime
|
||||
)
|
||||
{
|
||||
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<Integer, Long> getStartPartitions()
|
||||
{
|
||||
// Converting to start sequence numbers. This is allowed for Kafka because the start offset is always inclusive.
|
||||
final SeekableStreamStartSequenceNumbers<Integer, Long> startSequenceNumbers = getStartSequenceNumbers();
|
||||
return new SeekableStreamEndSequenceNumbers<>(
|
||||
startSequenceNumbers.getStream(),
|
||||
startSequenceNumbers.getPartitionSequenceNumberMap()
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* This method is for compatibilty so that newer version of KafkaIndexTaskIOConfig can be read by
|
||||
* old version of Druid.
|
||||
*/
|
||||
@JsonProperty
|
||||
public SeekableStreamEndSequenceNumbers<Integer, Long> getEndPartitions()
|
||||
{
|
||||
return getEndSequenceNumbers();
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Map<String, Object> getConsumerProperties()
|
||||
{
|
||||
|
@ -91,8 +153,8 @@ public class KafkaIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig<Inte
|
|||
return "KafkaIndexTaskIOConfig{" +
|
||||
"taskGroupId=" + getTaskGroupId() +
|
||||
", baseSequenceName='" + getBaseSequenceName() + '\'' +
|
||||
", startPartitions=" + getStartPartitions() +
|
||||
", endPartitions=" + getEndPartitions() +
|
||||
", startSequenceNumbers=" + getStartSequenceNumbers() +
|
||||
", endSequenceNumbers=" + getEndSequenceNumbers() +
|
||||
", consumerProperties=" + consumerProperties +
|
||||
", pollTimeout=" + pollTimeout +
|
||||
", useTransaction=" + isUseTransaction() +
|
||||
|
|
|
@ -23,6 +23,8 @@ import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber;
|
|||
|
||||
import javax.validation.constraints.NotNull;
|
||||
|
||||
// OrderedSequenceNumber.equals() should be used instead.
|
||||
@SuppressWarnings("ComparableImplementedButEqualsNotOverridden")
|
||||
public class KafkaSequenceNumber extends OrderedSequenceNumber<Long>
|
||||
{
|
||||
private KafkaSequenceNumber(Long sequenceNumber)
|
||||
|
|
|
@ -48,9 +48,11 @@ import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory;
|
|||
import org.apache.druid.indexing.common.task.IndexTaskUtils;
|
||||
import org.apache.druid.indexing.common.task.RealtimeIndexTask;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamSequenceNumbers;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers;
|
||||
import org.apache.druid.indexing.seekablestream.SequenceMetadata;
|
||||
import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord;
|
||||
import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber;
|
||||
|
@ -210,7 +212,7 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner<In
|
|||
this.savedParseExceptions = savedParseExceptions;
|
||||
this.rowIngestionMeters = rowIngestionMetersFactory.createRowIngestionMeters();
|
||||
|
||||
this.endOffsets.putAll(ioConfig.getEndPartitions().getPartitionSequenceNumberMap());
|
||||
this.endOffsets.putAll(ioConfig.getEndSequenceNumbers().getPartitionSequenceNumberMap());
|
||||
this.ingestionState = IngestionState.NOT_STARTED;
|
||||
}
|
||||
|
||||
|
@ -294,19 +296,19 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner<In
|
|||
|
||||
appenderator = appenderator0;
|
||||
|
||||
final String topic = ioConfig.getStartPartitions().getStream();
|
||||
final String topic = ioConfig.getStartSequenceNumbers().getStream();
|
||||
|
||||
// Start up, set up initial offsets.
|
||||
final Object restoredMetadata = driver.startJob();
|
||||
if (restoredMetadata == null) {
|
||||
nextOffsets.putAll(ioConfig.getStartPartitions().getPartitionSequenceNumberMap());
|
||||
nextOffsets.putAll(ioConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap());
|
||||
} else {
|
||||
final Map<String, Object> restoredMetadataMap = (Map) restoredMetadata;
|
||||
final SeekableStreamPartitions<Integer, Long> restoredNextPartitions = toolbox.getObjectMapper().convertValue(
|
||||
final SeekableStreamEndSequenceNumbers<Integer, Long> restoredNextPartitions = toolbox.getObjectMapper().convertValue(
|
||||
restoredMetadataMap.get(METADATA_NEXT_PARTITIONS),
|
||||
toolbox.getObjectMapper().getTypeFactory().constructParametrizedType(
|
||||
SeekableStreamPartitions.class,
|
||||
SeekableStreamPartitions.class,
|
||||
SeekableStreamStartSequenceNumbers.class,
|
||||
SeekableStreamStartSequenceNumbers.class,
|
||||
Integer.class,
|
||||
Long.class
|
||||
)
|
||||
|
@ -314,19 +316,19 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner<In
|
|||
nextOffsets.putAll(restoredNextPartitions.getPartitionSequenceNumberMap());
|
||||
|
||||
// Sanity checks.
|
||||
if (!restoredNextPartitions.getStream().equals(ioConfig.getStartPartitions().getStream())) {
|
||||
if (!restoredNextPartitions.getStream().equals(ioConfig.getStartSequenceNumbers().getStream())) {
|
||||
throw new ISE(
|
||||
"WTF?! Restored topic[%s] but expected topic[%s]",
|
||||
restoredNextPartitions.getStream(),
|
||||
ioConfig.getStartPartitions().getStream()
|
||||
ioConfig.getStartSequenceNumbers().getStream()
|
||||
);
|
||||
}
|
||||
|
||||
if (!nextOffsets.keySet().equals(ioConfig.getStartPartitions().getPartitionSequenceNumberMap().keySet())) {
|
||||
if (!nextOffsets.keySet().equals(ioConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().keySet())) {
|
||||
throw new ISE(
|
||||
"WTF?! Restored partitions[%s] but expected partitions[%s]",
|
||||
nextOffsets.keySet(),
|
||||
ioConfig.getStartPartitions().getPartitionSequenceNumberMap().keySet()
|
||||
ioConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().keySet()
|
||||
);
|
||||
}
|
||||
}
|
||||
|
@ -351,8 +353,9 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner<In
|
|||
public Object getMetadata()
|
||||
{
|
||||
return ImmutableMap.of(
|
||||
METADATA_NEXT_PARTITIONS, new SeekableStreamPartitions<>(
|
||||
ioConfig.getStartPartitions().getStream(),
|
||||
METADATA_NEXT_PARTITIONS,
|
||||
new SeekableStreamEndSequenceNumbers<>(
|
||||
ioConfig.getStartSequenceNumbers().getStream(),
|
||||
snapshot
|
||||
)
|
||||
);
|
||||
|
@ -501,13 +504,13 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner<In
|
|||
}
|
||||
|
||||
final TransactionalSegmentPublisher publisher = (segments, commitMetadata) -> {
|
||||
final SeekableStreamPartitions<Integer, Long> finalPartitions = toolbox.getObjectMapper().convertValue(
|
||||
final SeekableStreamEndSequenceNumbers<Integer, Long> finalPartitions = toolbox.getObjectMapper().convertValue(
|
||||
((Map) Preconditions.checkNotNull(commitMetadata, "commitMetadata")).get(METADATA_NEXT_PARTITIONS),
|
||||
toolbox.getObjectMapper()
|
||||
.getTypeFactory()
|
||||
.constructParametrizedType(
|
||||
SeekableStreamPartitions.class,
|
||||
SeekableStreamPartitions.class,
|
||||
SeekableStreamEndSequenceNumbers.class,
|
||||
SeekableStreamEndSequenceNumbers.class,
|
||||
Integer.class,
|
||||
Long.class
|
||||
)
|
||||
|
@ -523,7 +526,7 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner<In
|
|||
if (ioConfig.isUseTransaction()) {
|
||||
action = new SegmentTransactionalInsertAction(
|
||||
segments,
|
||||
new KafkaDataSourceMetadata(ioConfig.getStartPartitions()),
|
||||
new KafkaDataSourceMetadata(ioConfig.getStartSequenceNumbers()),
|
||||
new KafkaDataSourceMetadata(finalPartitions)
|
||||
);
|
||||
} else {
|
||||
|
@ -698,8 +701,7 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner<In
|
|||
protected void possiblyResetDataSourceMetadata(
|
||||
TaskToolbox toolbox,
|
||||
RecordSupplier<Integer, Long> recordSupplier,
|
||||
Set<StreamPartition<Integer>> assignment,
|
||||
Map<Integer, Long> currOffsets
|
||||
Set<StreamPartition<Integer>> assignment
|
||||
)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
|
@ -712,7 +714,7 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner<In
|
|||
}
|
||||
|
||||
@Override
|
||||
protected SeekableStreamPartitions<Integer, Long> deserializePartitionsFromMetadata(
|
||||
protected SeekableStreamEndSequenceNumbers<Integer, Long> deserializePartitionsFromMetadata(
|
||||
ObjectMapper mapper,
|
||||
Object object
|
||||
)
|
||||
|
@ -774,11 +776,13 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner<In
|
|||
boolean result = taskToolbox.getTaskActionClient()
|
||||
.submit(new ResetDataSourceMetadataAction(
|
||||
task.getDataSource(),
|
||||
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(
|
||||
ioConfig.getStartPartitions()
|
||||
.getStream(),
|
||||
partitionOffsetMap
|
||||
))
|
||||
new KafkaDataSourceMetadata(
|
||||
new SeekableStreamStartSequenceNumbers<>(
|
||||
ioConfig.getStartSequenceNumbers().getStream(),
|
||||
partitionOffsetMap,
|
||||
Collections.emptySet()
|
||||
)
|
||||
)
|
||||
));
|
||||
|
||||
if (result) {
|
||||
|
@ -1200,7 +1204,7 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner<In
|
|||
|
||||
@Override
|
||||
protected SeekableStreamDataSourceMetadata<Integer, Long> createDataSourceMetadata(
|
||||
SeekableStreamPartitions<Integer, Long> partitions
|
||||
SeekableStreamSequenceNumbers<Integer, Long> partitions
|
||||
)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
|
|
|
@ -39,10 +39,11 @@ import org.apache.druid.indexing.overlord.DataSourceMetadata;
|
|||
import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
|
||||
import org.apache.druid.indexing.overlord.TaskMaster;
|
||||
import org.apache.druid.indexing.overlord.TaskStorage;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers;
|
||||
import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber;
|
||||
import org.apache.druid.indexing.seekablestream.common.RecordSupplier;
|
||||
import org.apache.druid.indexing.seekablestream.common.StreamPartition;
|
||||
|
@ -59,6 +60,7 @@ import org.apache.druid.server.metrics.DruidMonitorSchedulerConfig;
|
|||
import org.joda.time.DateTime;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
|
@ -206,8 +208,8 @@ public class KafkaSupervisor extends SeekableStreamSupervisor<Integer, Long>
|
|||
return new KafkaIndexTaskIOConfig(
|
||||
groupId,
|
||||
baseSequenceName,
|
||||
new SeekableStreamPartitions<>(kafkaIoConfig.getTopic(), startPartitions),
|
||||
new SeekableStreamPartitions<>(kafkaIoConfig.getTopic(), endPartitions),
|
||||
new SeekableStreamStartSequenceNumbers<>(kafkaIoConfig.getTopic(), startPartitions, Collections.emptySet()),
|
||||
new SeekableStreamEndSequenceNumbers<>(kafkaIoConfig.getTopic(), endPartitions),
|
||||
kafkaIoConfig.getConsumerProperties(),
|
||||
kafkaIoConfig.getPollTimeout(),
|
||||
true,
|
||||
|
@ -281,9 +283,9 @@ public class KafkaSupervisor extends SeekableStreamSupervisor<Integer, Long>
|
|||
}
|
||||
|
||||
@Override
|
||||
protected KafkaDataSourceMetadata createDataSourceMetaData(String topic, Map<Integer, Long> map)
|
||||
protected KafkaDataSourceMetadata createDataSourceMetaDataForReset(String topic, Map<Integer, Long> map)
|
||||
{
|
||||
return new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, map));
|
||||
return new KafkaDataSourceMetadata(new SeekableStreamStartSequenceNumbers<>(topic, map, Collections.emptySet()));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -355,6 +357,12 @@ public class KafkaSupervisor extends SeekableStreamSupervisor<Integer, Long>
|
|||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean useExclusiveStartSequenceNumberForStartSequence()
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void updateLatestSequenceFromStream(
|
||||
RecordSupplier<Integer, Long> recordSupplier,
|
||||
|
|
|
@ -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<Integer, Long> offsets)
|
||||
private static KafkaDataSourceMetadata startMetadata(Map<Integer, Long> offsets)
|
||||
{
|
||||
return new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, offsets));
|
||||
return new KafkaDataSourceMetadata(new SeekableStreamStartSequenceNumbers<>("foo", offsets, ImmutableSet.of()));
|
||||
}
|
||||
|
||||
private static KafkaDataSourceMetadata endMetadata(Map<Integer, Long> offsets)
|
||||
{
|
||||
return new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>("foo", offsets));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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<Integer, Long> startPartitions;
|
||||
private final SeekableStreamEndSequenceNumbers<Integer, Long> endPartitions;
|
||||
private final Map<String, Object> consumerProperties;
|
||||
private final long pollTimeout;
|
||||
private final boolean useTransaction;
|
||||
private final Optional<DateTime> minimumMessageTime;
|
||||
private final Optional<DateTime> maximumMessageTime;
|
||||
|
||||
@JsonCreator
|
||||
private OldKafkaIndexTaskIoConfig(
|
||||
@JsonProperty("taskGroupId") int taskGroupId,
|
||||
@JsonProperty("baseSequenceName") String baseSequenceName,
|
||||
@JsonProperty("startPartitions") @Nullable SeekableStreamEndSequenceNumbers<Integer, Long> startPartitions,
|
||||
@JsonProperty("endPartitions") @Nullable SeekableStreamEndSequenceNumbers<Integer, Long> endPartitions,
|
||||
@JsonProperty("consumerProperties") Map<String, Object> consumerProperties,
|
||||
@JsonProperty("pollTimeout") Long pollTimeout,
|
||||
@JsonProperty("useTransaction") Boolean useTransaction,
|
||||
@JsonProperty("minimumMessageTime") DateTime minimumMessageTime,
|
||||
@JsonProperty("maximumMessageTime") DateTime maximumMessageTime
|
||||
)
|
||||
{
|
||||
this.taskGroupId = taskGroupId;
|
||||
this.baseSequenceName = baseSequenceName;
|
||||
this.startPartitions = startPartitions;
|
||||
this.endPartitions = endPartitions;
|
||||
this.consumerProperties = consumerProperties;
|
||||
this.pollTimeout = pollTimeout;
|
||||
this.useTransaction = useTransaction;
|
||||
this.minimumMessageTime = Optional.fromNullable(minimumMessageTime);
|
||||
this.maximumMessageTime = Optional.fromNullable(maximumMessageTime);
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public int getTaskGroupId()
|
||||
{
|
||||
return taskGroupId;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getBaseSequenceName()
|
||||
{
|
||||
return baseSequenceName;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public SeekableStreamEndSequenceNumbers<Integer, Long> getStartPartitions()
|
||||
{
|
||||
return startPartitions;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public SeekableStreamEndSequenceNumbers<Integer, Long> getEndPartitions()
|
||||
{
|
||||
return endPartitions;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Map<String, Object> getConsumerProperties()
|
||||
{
|
||||
return consumerProperties;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public long getPollTimeout()
|
||||
{
|
||||
return pollTimeout;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public boolean isUseTransaction()
|
||||
{
|
||||
return useTransaction;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Optional<DateTime> getMinimumMessageTime()
|
||||
{
|
||||
return minimumMessageTime;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Optional<DateTime> getMaximumMessageTime()
|
||||
{
|
||||
return maximumMessageTime;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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<String, Object> consumerProps = kafkaServer.consumerProperties();
|
||||
consumerProps.put("max.poll.records", "1");
|
||||
|
||||
final SeekableStreamPartitions<Integer, Long> startPartitions = new SeekableStreamPartitions<>(
|
||||
final SeekableStreamStartSequenceNumbers<Integer, Long> startPartitions = new SeekableStreamStartSequenceNumbers<>(
|
||||
topic,
|
||||
ImmutableMap.of(
|
||||
0,
|
||||
0L,
|
||||
1,
|
||||
0L
|
||||
)
|
||||
ImmutableMap.of(0, 0L, 1, 0L),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
// Checkpointing will happen at either checkpoint1 or checkpoint2 depending on ordering
|
||||
// of events fetched across two partitions from Kafka
|
||||
final SeekableStreamPartitions<Integer, Long> checkpoint1 = new SeekableStreamPartitions<>(
|
||||
final SeekableStreamEndSequenceNumbers<Integer, Long> checkpoint1 = new SeekableStreamEndSequenceNumbers<>(
|
||||
topic,
|
||||
ImmutableMap.of(
|
||||
0,
|
||||
5L,
|
||||
1,
|
||||
0L
|
||||
)
|
||||
ImmutableMap.of(0, 5L, 1, 0L)
|
||||
);
|
||||
final SeekableStreamPartitions<Integer, Long> checkpoint2 = new SeekableStreamPartitions<>(
|
||||
final SeekableStreamEndSequenceNumbers<Integer, Long> checkpoint2 = new SeekableStreamEndSequenceNumbers<>(
|
||||
topic,
|
||||
ImmutableMap.of(
|
||||
0,
|
||||
4L,
|
||||
1,
|
||||
2L
|
||||
)
|
||||
ImmutableMap.of(0, 4L, 1, 2L)
|
||||
);
|
||||
final SeekableStreamPartitions<Integer, Long> endPartitions = new SeekableStreamPartitions<>(
|
||||
final SeekableStreamEndSequenceNumbers<Integer, Long> endPartitions = new SeekableStreamEndSequenceNumbers<>(
|
||||
topic,
|
||||
ImmutableMap.of(
|
||||
0,
|
||||
10L,
|
||||
1,
|
||||
2L
|
||||
)
|
||||
ImmutableMap.of(0, 10L, 1, 2L)
|
||||
);
|
||||
final KafkaIndexTask task = createTask(
|
||||
null,
|
||||
|
@ -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<String, Object> consumerProps = kafkaServer.consumerProperties();
|
||||
consumerProps.put("max.poll.records", "1");
|
||||
|
||||
final SeekableStreamPartitions<Integer, Long> startPartitions = new SeekableStreamPartitions<>(
|
||||
final SeekableStreamStartSequenceNumbers<Integer, Long> startPartitions = new SeekableStreamStartSequenceNumbers<>(
|
||||
topic,
|
||||
ImmutableMap.of(
|
||||
0,
|
||||
0L,
|
||||
1,
|
||||
0L
|
||||
)
|
||||
ImmutableMap.of(0, 0L, 1, 0L),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
final SeekableStreamPartitions<Integer, Long> checkpoint1 = new SeekableStreamPartitions<>(
|
||||
final SeekableStreamEndSequenceNumbers<Integer, Long> checkpoint1 = new SeekableStreamEndSequenceNumbers<>(
|
||||
topic,
|
||||
ImmutableMap.of(
|
||||
0,
|
||||
3L,
|
||||
1,
|
||||
0L
|
||||
)
|
||||
ImmutableMap.of(0, 3L, 1, 0L)
|
||||
);
|
||||
final SeekableStreamPartitions<Integer, Long> checkpoint2 = new SeekableStreamPartitions<>(
|
||||
final SeekableStreamEndSequenceNumbers<Integer, Long> checkpoint2 = new SeekableStreamEndSequenceNumbers<>(
|
||||
topic,
|
||||
ImmutableMap.of(
|
||||
0,
|
||||
10L,
|
||||
1,
|
||||
0L
|
||||
)
|
||||
ImmutableMap.of(0, 10L, 1, 0L)
|
||||
);
|
||||
|
||||
final SeekableStreamPartitions<Integer, Long> endPartitions = new SeekableStreamPartitions<>(
|
||||
final SeekableStreamEndSequenceNumbers<Integer, Long> endPartitions = new SeekableStreamEndSequenceNumbers<>(
|
||||
topic,
|
||||
ImmutableMap.of(
|
||||
0,
|
||||
10L,
|
||||
1,
|
||||
2L
|
||||
)
|
||||
ImmutableMap.of(0, 10L, 1, 2L)
|
||||
);
|
||||
final KafkaIndexTask task = createTask(
|
||||
null,
|
||||
|
@ -696,7 +663,7 @@ public class KafkaIndexTaskTest
|
|||
}
|
||||
final Map<Integer, Long> currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets());
|
||||
|
||||
Assert.assertTrue(checkpoint1.getPartitionSequenceNumberMap().equals(currentOffsets));
|
||||
Assert.assertEquals(checkpoint1.getPartitionSequenceNumberMap(), currentOffsets);
|
||||
task.getRunner().setEndOffsets(currentOffsets, false);
|
||||
|
||||
while (task.getRunner().getStatus() != Status.PAUSED) {
|
||||
|
@ -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<String, Object> consumerProps = kafkaServer.consumerProperties();
|
||||
consumerProps.put("max.poll.records", "1");
|
||||
|
||||
final SeekableStreamPartitions<Integer, Long> startPartitions = new SeekableStreamPartitions<>(
|
||||
final SeekableStreamStartSequenceNumbers<Integer, Long> startPartitions = new SeekableStreamStartSequenceNumbers<>(
|
||||
topic,
|
||||
ImmutableMap.of(
|
||||
0,
|
||||
0L,
|
||||
1,
|
||||
0L
|
||||
)
|
||||
ImmutableMap.of(0, 0L, 1, 0L),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
// Checkpointing will happen at checkpoint
|
||||
final SeekableStreamPartitions<Integer, Long> checkpoint = new SeekableStreamPartitions<>(
|
||||
final SeekableStreamEndSequenceNumbers<Integer, Long> checkpoint = new SeekableStreamEndSequenceNumbers<>(
|
||||
topic,
|
||||
ImmutableMap.of(
|
||||
0,
|
||||
1L,
|
||||
1,
|
||||
0L
|
||||
)
|
||||
ImmutableMap.of(0, 1L, 1, 0L)
|
||||
);
|
||||
final SeekableStreamPartitions<Integer, Long> endPartitions = new SeekableStreamPartitions<>(
|
||||
final SeekableStreamEndSequenceNumbers<Integer, Long> endPartitions = new SeekableStreamEndSequenceNumbers<>(
|
||||
topic,
|
||||
ImmutableMap.of(
|
||||
0,
|
||||
2L,
|
||||
1,
|
||||
0L
|
||||
)
|
||||
ImmutableMap.of(0, 2L, 1, 0L)
|
||||
);
|
||||
final KafkaIndexTask task = createTask(
|
||||
null,
|
||||
|
@ -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<String, Object> consumerProps = kafkaServer.consumerProperties();
|
||||
consumerProps.put("max.poll.records", "1");
|
||||
|
||||
final SeekableStreamPartitions<Integer, Long> startPartitions =
|
||||
new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L));
|
||||
final SeekableStreamPartitions<Integer, Long> checkpoint1 =
|
||||
new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L));
|
||||
final SeekableStreamPartitions<Integer, Long> checkpoint2 =
|
||||
new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 9L));
|
||||
final SeekableStreamPartitions<Integer, Long> endPartitions =
|
||||
new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, Long.MAX_VALUE));
|
||||
final SeekableStreamStartSequenceNumbers<Integer, Long> startPartitions =
|
||||
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of());
|
||||
final SeekableStreamEndSequenceNumbers<Integer, Long> checkpoint1 =
|
||||
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L));
|
||||
final SeekableStreamEndSequenceNumbers<Integer, Long> checkpoint2 =
|
||||
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 9L));
|
||||
final SeekableStreamEndSequenceNumbers<Integer, Long> endPartitions =
|
||||
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, Long.MAX_VALUE));
|
||||
|
||||
final KafkaIndexTask normalReplica = createTask(
|
||||
null,
|
||||
|
@ -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<Integer, Long> checkpoint = new SeekableStreamPartitions<>(
|
||||
final SeekableStreamStartSequenceNumbers<Integer, Long> checkpoint = new SeekableStreamStartSequenceNumbers<>(
|
||||
topic,
|
||||
ImmutableMap.of(0, 5L)
|
||||
ImmutableMap.of(0, 5L),
|
||||
ImmutableSet.of(0)
|
||||
);
|
||||
|
||||
final ListenableFuture<TaskStatus> 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<byte[], byte[]> kafkaProducer = kafkaServer.newProducer()) {
|
||||
kafkaProducer.initTransactions();
|
||||
kafkaProducer.beginTransaction();
|
||||
|
@ -2389,24 +2356,15 @@ public class KafkaIndexTaskTest
|
|||
Map<String, Object> consumerProps = kafkaServer.consumerProperties();
|
||||
consumerProps.put("max.poll.records", "1");
|
||||
|
||||
final SeekableStreamPartitions<Integer, Long> startPartitions = new SeekableStreamPartitions<>(
|
||||
final SeekableStreamStartSequenceNumbers<Integer, Long> startPartitions = new SeekableStreamStartSequenceNumbers<>(
|
||||
topic,
|
||||
ImmutableMap.of(
|
||||
0,
|
||||
0L,
|
||||
1,
|
||||
1L
|
||||
)
|
||||
ImmutableMap.of(0, 0L, 1, 1L),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
|
||||
final SeekableStreamPartitions<Integer, Long> endPartitions = new SeekableStreamPartitions<>(
|
||||
final SeekableStreamEndSequenceNumbers<Integer, Long> endPartitions = new SeekableStreamEndSequenceNumbers<>(
|
||||
topic,
|
||||
ImmutableMap.of(
|
||||
0,
|
||||
10L,
|
||||
1,
|
||||
2L
|
||||
)
|
||||
ImmutableMap.of(0, 10L, 1, 2L)
|
||||
);
|
||||
|
||||
final KafkaIndexTask task = createTask(
|
||||
|
|
|
@ -187,5 +187,4 @@ public class KafkaSupervisorIOConfigTest
|
|||
exception.expectMessage(CoreMatchers.containsString("bootstrap.servers"));
|
||||
mapper.readValue(jsonStr, KafkaSupervisorIOConfig.class);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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<String> captureDataSource = EasyMock.newCapture();
|
||||
Capture<DataSourceMetadata> captureDataSourceMetadata = EasyMock.newCapture();
|
||||
|
||||
KafkaDataSourceMetadata kafkaDataSourceMetadata = new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(
|
||||
topic,
|
||||
ImmutableMap.of(0, 1000L, 1, 1000L, 2, 1000L)
|
||||
));
|
||||
KafkaDataSourceMetadata kafkaDataSourceMetadata = new KafkaDataSourceMetadata(
|
||||
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 1000L, 1, 1000L, 2, 1000L), ImmutableSet.of())
|
||||
);
|
||||
|
||||
KafkaDataSourceMetadata resetMetadata = new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(
|
||||
topic,
|
||||
ImmutableMap.of(1, 1000L, 2, 1000L)
|
||||
));
|
||||
KafkaDataSourceMetadata resetMetadata = new KafkaDataSourceMetadata(
|
||||
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(1, 1000L, 2, 1000L), ImmutableSet.of())
|
||||
);
|
||||
|
||||
KafkaDataSourceMetadata expectedMetadata = new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(
|
||||
topic,
|
||||
ImmutableMap.of(0, 1000L)
|
||||
));
|
||||
KafkaDataSourceMetadata expectedMetadata = new KafkaDataSourceMetadata(
|
||||
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 1000L), ImmutableSet.of()));
|
||||
|
||||
EasyMock.reset(indexerMetadataStorageCoordinator);
|
||||
expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE))
|
||||
|
@ -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<Integer, Long> startPartitions,
|
||||
SeekableStreamPartitions<Integer, Long> endPartitions,
|
||||
SeekableStreamStartSequenceNumbers<Integer, Long> startPartitions,
|
||||
SeekableStreamEndSequenceNumbers<Integer, Long> endPartitions,
|
||||
DateTime minimumMessageTime,
|
||||
DateTime maximumMessageTime
|
||||
)
|
||||
|
|
|
@ -21,24 +21,39 @@ package org.apache.druid.indexing.kinesis;
|
|||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.apache.druid.indexing.overlord.DataSourceMetadata;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
|
||||
|
||||
import java.util.Map;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamSequenceNumbers;
|
||||
|
||||
public class KinesisDataSourceMetadata extends SeekableStreamDataSourceMetadata<String, String>
|
||||
{
|
||||
@JsonCreator
|
||||
public KinesisDataSourceMetadata(
|
||||
@JsonProperty("partitions") SeekableStreamPartitions<String, String> kinesisPartitions
|
||||
@JsonProperty("partitions") SeekableStreamSequenceNumbers<String, String> kinesisPartitions
|
||||
)
|
||||
{
|
||||
super(kinesisPartitions);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected KinesisDataSourceMetadata createConcreteDataSourceMetaData(String streamName, Map<String, String> newMap)
|
||||
public DataSourceMetadata asStartMetadata()
|
||||
{
|
||||
return new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(streamName, newMap));
|
||||
final SeekableStreamSequenceNumbers<String, String> sequenceNumbers = getSeekableStreamSequenceNumbers();
|
||||
if (sequenceNumbers instanceof SeekableStreamEndSequenceNumbers) {
|
||||
return createConcreteDataSourceMetaData(
|
||||
((SeekableStreamEndSequenceNumbers<String, String>) sequenceNumbers).asStartPartitions(false)
|
||||
);
|
||||
} else {
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected KinesisDataSourceMetadata createConcreteDataSourceMetaData(
|
||||
SeekableStreamSequenceNumbers<String, String> seekableStreamSequenceNumbers
|
||||
)
|
||||
{
|
||||
return new KinesisDataSourceMetadata(seekableStreamSequenceNumbers);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -90,7 +90,7 @@ public class KinesisIndexTask extends SeekableStreamIndexTask<String, String>
|
|||
KinesisIndexTaskTuningConfig tuningConfig = ((KinesisIndexTaskTuningConfig) super.tuningConfig);
|
||||
int fetchThreads = tuningConfig.getFetchThreads() != null
|
||||
? tuningConfig.getFetchThreads()
|
||||
: Math.max(1, ioConfig.getStartPartitions().getPartitionSequenceNumberMap().size());
|
||||
: Math.max(1, ioConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().size());
|
||||
|
||||
return new KinesisRecordSupplier(
|
||||
KinesisRecordSupplier.getAmazonKinesisClient(
|
||||
|
|
|
@ -22,12 +22,12 @@ package org.apache.druid.indexing.kinesis;
|
|||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Set;
|
||||
|
||||
public class KinesisIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig<String, String>
|
||||
{
|
||||
|
@ -46,15 +46,14 @@ public class KinesisIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig<St
|
|||
public KinesisIndexTaskIOConfig(
|
||||
@JsonProperty("taskGroupId") @Nullable Integer taskGroupId,
|
||||
@JsonProperty("baseSequenceName") String baseSequenceName,
|
||||
@JsonProperty("startPartitions") SeekableStreamPartitions<String, String> startPartitions,
|
||||
@JsonProperty("endPartitions") SeekableStreamPartitions<String, String> endPartitions,
|
||||
@JsonProperty("startSequenceNumbers") SeekableStreamStartSequenceNumbers<String, String> startSequenceNumbers,
|
||||
@JsonProperty("endSequenceNumbers") SeekableStreamEndSequenceNumbers<String, String> endSequenceNumbers,
|
||||
@JsonProperty("useTransaction") Boolean useTransaction,
|
||||
@JsonProperty("minimumMessageTime") DateTime minimumMessageTime,
|
||||
@JsonProperty("maximumMessageTime") DateTime maximumMessageTime,
|
||||
@JsonProperty("endpoint") String endpoint,
|
||||
@JsonProperty("recordsPerFetch") Integer recordsPerFetch,
|
||||
@JsonProperty("fetchDelayMillis") Integer fetchDelayMillis,
|
||||
@JsonProperty("exclusiveStartSequenceNumberPartitions") Set<String> exclusiveStartSequenceNumberPartitions,
|
||||
@JsonProperty("awsAssumedRoleArn") String awsAssumedRoleArn,
|
||||
@JsonProperty("awsExternalId") String awsExternalId,
|
||||
@JsonProperty("deaggregate") boolean deaggregate
|
||||
|
@ -63,17 +62,19 @@ public class KinesisIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig<St
|
|||
super(
|
||||
taskGroupId,
|
||||
baseSequenceName,
|
||||
startPartitions,
|
||||
endPartitions,
|
||||
startSequenceNumbers,
|
||||
endSequenceNumbers,
|
||||
useTransaction,
|
||||
minimumMessageTime,
|
||||
maximumMessageTime,
|
||||
exclusiveStartSequenceNumberPartitions
|
||||
maximumMessageTime
|
||||
);
|
||||
Preconditions.checkArgument(
|
||||
endSequenceNumbers.getPartitionSequenceNumberMap()
|
||||
.values()
|
||||
.stream()
|
||||
.noneMatch(x -> x.equals(KinesisSequenceNumber.END_OF_SHARD_MARKER)),
|
||||
"End sequenceNumbers must not have the end of shard marker (EOS)"
|
||||
);
|
||||
Preconditions.checkArgument(endPartitions.getPartitionSequenceNumberMap()
|
||||
.values()
|
||||
.stream()
|
||||
.noneMatch(x -> x.equals(KinesisSequenceNumber.END_OF_SHARD_MARKER)));
|
||||
|
||||
this.endpoint = Preconditions.checkNotNull(endpoint, "endpoint");
|
||||
this.recordsPerFetch = recordsPerFetch != null ? recordsPerFetch : DEFAULT_RECORDS_PER_FETCH;
|
||||
|
@ -124,15 +125,14 @@ public class KinesisIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig<St
|
|||
{
|
||||
return "KinesisIndexTaskIOConfig{" +
|
||||
"baseSequenceName='" + getBaseSequenceName() + '\'' +
|
||||
", startPartitions=" + getStartPartitions() +
|
||||
", endPartitions=" + getEndPartitions() +
|
||||
", startPartitions=" + getStartSequenceNumbers() +
|
||||
", endPartitions=" + getEndSequenceNumbers() +
|
||||
", useTransaction=" + isUseTransaction() +
|
||||
", minimumMessageTime=" + getMinimumMessageTime() +
|
||||
", maximumMessageTime=" + getMaximumMessageTime() +
|
||||
", endpoint='" + endpoint + '\'' +
|
||||
", recordsPerFetch=" + recordsPerFetch +
|
||||
", fetchDelayMillis=" + fetchDelayMillis +
|
||||
", exclusiveStartSequenceNumberPartitions=" + getExclusiveStartSequenceNumberPartitions() +
|
||||
", awsAssumedRoleArn='" + awsAssumedRoleArn + '\'' +
|
||||
", awsExternalId='" + awsExternalId + '\'' +
|
||||
", deaggregate=" + deaggregate +
|
||||
|
|
|
@ -26,8 +26,9 @@ import org.apache.druid.data.input.impl.InputRowParser;
|
|||
import org.apache.druid.indexing.common.TaskToolbox;
|
||||
import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamSequenceNumbers;
|
||||
import org.apache.druid.indexing.seekablestream.SequenceMetadata;
|
||||
import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord;
|
||||
import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber;
|
||||
|
@ -47,6 +48,7 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.TreeMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner<String, String>
|
||||
|
@ -93,14 +95,14 @@ public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner<String
|
|||
}
|
||||
|
||||
@Override
|
||||
protected SeekableStreamPartitions<String, String> deserializePartitionsFromMetadata(
|
||||
protected SeekableStreamEndSequenceNumbers<String, String> deserializePartitionsFromMetadata(
|
||||
ObjectMapper mapper,
|
||||
Object object
|
||||
)
|
||||
{
|
||||
return mapper.convertValue(object, mapper.getTypeFactory().constructParametrizedType(
|
||||
SeekableStreamPartitions.class,
|
||||
SeekableStreamPartitions.class,
|
||||
SeekableStreamEndSequenceNumbers.class,
|
||||
SeekableStreamEndSequenceNumbers.class,
|
||||
String.class,
|
||||
String.class
|
||||
));
|
||||
|
@ -108,7 +110,7 @@ public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner<String
|
|||
|
||||
@Override
|
||||
protected SeekableStreamDataSourceMetadata<String, String> createDataSourceMetadata(
|
||||
SeekableStreamPartitions<String, String> partitions
|
||||
SeekableStreamSequenceNumbers<String, String> partitions
|
||||
)
|
||||
{
|
||||
return new KinesisDataSourceMetadata(partitions);
|
||||
|
@ -124,11 +126,11 @@ public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner<String
|
|||
protected void possiblyResetDataSourceMetadata(
|
||||
TaskToolbox toolbox,
|
||||
RecordSupplier<String, String> recordSupplier,
|
||||
Set<StreamPartition<String>> assignment,
|
||||
Map<String, String> currOffsets
|
||||
Set<StreamPartition<String>> assignment
|
||||
)
|
||||
{
|
||||
if (!task.getTuningConfig().isSkipSequenceNumberAvailabilityCheck()) {
|
||||
final ConcurrentMap<String, String> currOffsets = getCurrentOffsets();
|
||||
for (final StreamPartition<String> streamPartition : assignment) {
|
||||
String sequence = currOffsets.get(streamPartition.getPartitionId());
|
||||
String earliestSequenceNumber = recordSupplier.getEarliestSequenceNumber(streamPartition);
|
||||
|
|
|
@ -20,15 +20,14 @@
|
|||
package org.apache.druid.indexing.kinesis;
|
||||
|
||||
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
|
||||
import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber;
|
||||
|
||||
import javax.validation.constraints.NotNull;
|
||||
import java.math.BigInteger;
|
||||
|
||||
// OrderedSequenceNumber.equals() should be used instead.
|
||||
@SuppressWarnings("ComparableImplementedButEqualsNotOverridden")
|
||||
public class KinesisSequenceNumber extends OrderedSequenceNumber<String>
|
||||
{
|
||||
|
||||
/**
|
||||
* In Kinesis, when a shard is closed due to shard splitting, a null ShardIterator is returned.
|
||||
* The EOS marker is placed at the end of the Kinesis Record Supplier buffer, such that when
|
||||
|
@ -36,17 +35,24 @@ public class KinesisSequenceNumber extends OrderedSequenceNumber<String>
|
|||
* reading and start publishing
|
||||
*/
|
||||
public static final String END_OF_SHARD_MARKER = "EOS";
|
||||
|
||||
// this special marker is used by the KinesisSupervisor to set the endOffsets
|
||||
// of newly created indexing tasks. This is necessary because streaming tasks do not
|
||||
// have endPartitionOffsets. This marker signals to the task that it should continue
|
||||
// to ingest data until taskDuration has elapsed or the task was stopped or paused or killed
|
||||
public static final String NO_END_SEQUENCE_NUMBER = "NO_END_SEQUENCE_NUMBER";
|
||||
|
||||
// this flag is used to indicate either END_OF_SHARD_MARKER
|
||||
// or NO_END_SEQUENCE_NUMBER so that they can be properly compared
|
||||
// with other sequence numbers
|
||||
private final boolean isMaxSequenceNumber;
|
||||
private final BigInteger intSequence;
|
||||
|
||||
private KinesisSequenceNumber(@NotNull String sequenceNumber, boolean isExclusive)
|
||||
private KinesisSequenceNumber(String sequenceNumber, boolean isExclusive)
|
||||
{
|
||||
super(sequenceNumber, isExclusive);
|
||||
if (END_OF_SHARD_MARKER.equals(sequenceNumber)
|
||||
|| SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER.equals(sequenceNumber)) {
|
||||
|| NO_END_SEQUENCE_NUMBER.equals(sequenceNumber)) {
|
||||
isMaxSequenceNumber = true;
|
||||
this.intSequence = null;
|
||||
} else {
|
||||
|
@ -66,7 +72,7 @@ public class KinesisSequenceNumber extends OrderedSequenceNumber<String>
|
|||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(@NotNull OrderedSequenceNumber<String> o)
|
||||
public int compareTo(OrderedSequenceNumber<String> o)
|
||||
{
|
||||
KinesisSequenceNumber num = (KinesisSequenceNumber) o;
|
||||
if (isMaxSequenceNumber && num.isMaxSequenceNumber) {
|
||||
|
@ -79,5 +85,4 @@ public class KinesisSequenceNumber extends OrderedSequenceNumber<String>
|
|||
return this.intSequence.compareTo(new BigInteger(o.get()));
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -40,10 +40,11 @@ import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
|
|||
import org.apache.druid.indexing.overlord.TaskMaster;
|
||||
import org.apache.druid.indexing.overlord.TaskStorage;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers;
|
||||
import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber;
|
||||
import org.apache.druid.indexing.seekablestream.common.RecordSupplier;
|
||||
import org.apache.druid.indexing.seekablestream.common.StreamPartition;
|
||||
|
@ -55,6 +56,7 @@ import org.apache.druid.java.util.common.StringUtils;
|
|||
import org.joda.time.DateTime;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
@ -120,15 +122,18 @@ public class KinesisSupervisor extends SeekableStreamSupervisor<String, String>
|
|||
return new KinesisIndexTaskIOConfig(
|
||||
groupId,
|
||||
baseSequenceName,
|
||||
new SeekableStreamPartitions<>(ioConfig.getStream(), startPartitions),
|
||||
new SeekableStreamPartitions<>(ioConfig.getStream(), endPartitions),
|
||||
new SeekableStreamStartSequenceNumbers<>(
|
||||
ioConfig.getStream(),
|
||||
startPartitions,
|
||||
exclusiveStartSequenceNumberPartitions
|
||||
),
|
||||
new SeekableStreamEndSequenceNumbers<>(ioConfig.getStream(), endPartitions),
|
||||
true,
|
||||
minimumMessageTime,
|
||||
maximumMessageTime,
|
||||
ioConfig.getEndpoint(),
|
||||
ioConfig.getRecordsPerFetch(),
|
||||
ioConfig.getFetchDelayMillis(),
|
||||
exclusiveStartSequenceNumberPartitions,
|
||||
ioConfig.getAwsAssumedRoleArn(),
|
||||
ioConfig.getAwsExternalId(),
|
||||
ioConfig.isDeaggregate()
|
||||
|
@ -204,10 +209,8 @@ public class KinesisSupervisor extends SeekableStreamSupervisor<String, String>
|
|||
taskTuningConfig.getFetchSequenceNumberTimeout(),
|
||||
taskTuningConfig.getMaxRecordsPerPoll()
|
||||
);
|
||||
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
protected void scheduleReporting(ScheduledExecutorService reportingExec)
|
||||
{
|
||||
|
@ -261,13 +264,13 @@ public class KinesisSupervisor extends SeekableStreamSupervisor<String, String>
|
|||
}
|
||||
|
||||
@Override
|
||||
protected SeekableStreamDataSourceMetadata<String, String> createDataSourceMetaData(
|
||||
protected SeekableStreamDataSourceMetadata<String, String> createDataSourceMetaDataForReset(
|
||||
String stream,
|
||||
Map<String, String> map
|
||||
)
|
||||
{
|
||||
return new KinesisDataSourceMetadata(
|
||||
new SeekableStreamPartitions<>(stream, map)
|
||||
new SeekableStreamStartSequenceNumbers<>(stream, map, Collections.emptySet())
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -300,7 +303,7 @@ public class KinesisSupervisor extends SeekableStreamSupervisor<String, String>
|
|||
@Override
|
||||
protected String getEndOfPartitionMarker()
|
||||
{
|
||||
return SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER;
|
||||
return KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -308,4 +311,10 @@ public class KinesisSupervisor extends SeekableStreamSupervisor<String, String>
|
|||
{
|
||||
return KinesisSequenceNumber.END_OF_SHARD_MARKER.equals(seqNum);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean useExclusiveStartSequenceNumberForStartSequence()
|
||||
{
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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<String, String> sequences)
|
||||
private static KinesisDataSourceMetadata simpleStartMetadata(Map<String, String> sequences)
|
||||
{
|
||||
return new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, sequences));
|
||||
return startMetadata(sequences, sequences.keySet());
|
||||
}
|
||||
|
||||
private static KinesisDataSourceMetadata startMetadata(Map<String, String> sequences, Set<String> exclusivePartitions)
|
||||
{
|
||||
return new KinesisDataSourceMetadata(
|
||||
new SeekableStreamStartSequenceNumbers<>("foo", sequences, exclusivePartitions)
|
||||
);
|
||||
}
|
||||
|
||||
private static KinesisDataSourceMetadata endMetadata(Map<String, String> sequences)
|
||||
{
|
||||
return new KinesisDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>("foo", sequences));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
|
@ -23,26 +23,24 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||
import org.apache.druid.indexing.overlord.DataSourceMetadata;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
public abstract class SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType>
|
||||
implements DataSourceMetadata
|
||||
{
|
||||
private final SeekableStreamPartitions<PartitionIdType, SequenceOffsetType> seekableStreamPartitions;
|
||||
private final SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> seekableStreamSequenceNumbers;
|
||||
|
||||
public SeekableStreamDataSourceMetadata(
|
||||
SeekableStreamPartitions<PartitionIdType, SequenceOffsetType> seekableStreamPartitions
|
||||
SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> seekableStreamSequenceNumbers
|
||||
)
|
||||
{
|
||||
this.seekableStreamPartitions = seekableStreamPartitions;
|
||||
this.seekableStreamSequenceNumbers = seekableStreamSequenceNumbers;
|
||||
}
|
||||
|
||||
@JsonProperty("partitions")
|
||||
public SeekableStreamPartitions<PartitionIdType, SequenceOffsetType> getSeekableStreamPartitions()
|
||||
public SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> getSeekableStreamSequenceNumbers()
|
||||
{
|
||||
return seekableStreamPartitions;
|
||||
return seekableStreamSequenceNumbers;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -64,7 +62,7 @@ public abstract class SeekableStreamDataSourceMetadata<PartitionIdType, Sequence
|
|||
@Override
|
||||
public DataSourceMetadata plus(DataSourceMetadata other)
|
||||
{
|
||||
if (!(this.getClass().isInstance(other))) {
|
||||
if (this.getClass() != other.getClass()) {
|
||||
throw new IAE(
|
||||
"Expected instance of %s, got %s",
|
||||
this.getClass().getCanonicalName(),
|
||||
|
@ -72,36 +70,17 @@ public abstract class SeekableStreamDataSourceMetadata<PartitionIdType, Sequence
|
|||
);
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
final SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType> that = (SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType>) other;
|
||||
//noinspection unchecked
|
||||
final SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType> that =
|
||||
(SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType>) other;
|
||||
|
||||
if (that.getSeekableStreamPartitions().getStream().equals(seekableStreamPartitions.getStream())) {
|
||||
// Same stream, merge sequences.
|
||||
final Map<PartitionIdType, SequenceOffsetType> newMap = new HashMap<>();
|
||||
|
||||
for (Map.Entry<PartitionIdType, SequenceOffsetType> entry : seekableStreamPartitions.getPartitionSequenceNumberMap()
|
||||
.entrySet()) {
|
||||
newMap.put(entry.getKey(), entry.getValue());
|
||||
}
|
||||
|
||||
for (Map.Entry<PartitionIdType, SequenceOffsetType> entry : that.getSeekableStreamPartitions()
|
||||
.getPartitionSequenceNumberMap()
|
||||
.entrySet()) {
|
||||
newMap.put(entry.getKey(), entry.getValue());
|
||||
}
|
||||
|
||||
return createConcreteDataSourceMetaData(seekableStreamPartitions.getStream(), newMap);
|
||||
} else {
|
||||
// Different stream, prefer "other".
|
||||
return other;
|
||||
}
|
||||
return createConcreteDataSourceMetaData(seekableStreamSequenceNumbers.plus(that.seekableStreamSequenceNumbers));
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public DataSourceMetadata minus(DataSourceMetadata other)
|
||||
{
|
||||
if (!(this.getClass().isInstance(other))) {
|
||||
if (this.getClass() != other.getClass()) {
|
||||
throw new IAE(
|
||||
"Expected instance of %s, got %s",
|
||||
this.getClass().getCanonicalName(),
|
||||
|
@ -109,25 +88,11 @@ public abstract class SeekableStreamDataSourceMetadata<PartitionIdType, Sequence
|
|||
);
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
final SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType> that = (SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType>) other;
|
||||
//noinspection unchecked
|
||||
final SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType> that =
|
||||
(SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType>) other;
|
||||
|
||||
if (that.getSeekableStreamPartitions().getStream().equals(seekableStreamPartitions.getStream())) {
|
||||
// Same stream, remove partitions present in "that" from "this"
|
||||
final Map<PartitionIdType, SequenceOffsetType> newMap = new HashMap<>();
|
||||
|
||||
for (Map.Entry<PartitionIdType, SequenceOffsetType> entry : seekableStreamPartitions.getPartitionSequenceNumberMap()
|
||||
.entrySet()) {
|
||||
if (!that.getSeekableStreamPartitions().getPartitionSequenceNumberMap().containsKey(entry.getKey())) {
|
||||
newMap.put(entry.getKey(), entry.getValue());
|
||||
}
|
||||
}
|
||||
|
||||
return createConcreteDataSourceMetaData(seekableStreamPartitions.getStream(), newMap);
|
||||
} else {
|
||||
// Different stream, prefer "this".
|
||||
return this;
|
||||
}
|
||||
return createConcreteDataSourceMetaData(seekableStreamSequenceNumbers.minus(that.seekableStreamSequenceNumbers));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -140,25 +105,24 @@ public abstract class SeekableStreamDataSourceMetadata<PartitionIdType, Sequence
|
|||
return false;
|
||||
}
|
||||
SeekableStreamDataSourceMetadata that = (SeekableStreamDataSourceMetadata) o;
|
||||
return Objects.equals(getSeekableStreamPartitions(), that.getSeekableStreamPartitions());
|
||||
return Objects.equals(getSeekableStreamSequenceNumbers(), that.getSeekableStreamSequenceNumbers());
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(getSeekableStreamPartitions());
|
||||
return seekableStreamSequenceNumbers.hashCode();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "SeekableStreamDataSourceMetadata{" +
|
||||
"SeekableStreamPartitions=" + getSeekableStreamPartitions() +
|
||||
return getClass().getSimpleName() + "{" +
|
||||
"SeekableStreamStartSequenceNumbers=" + getSeekableStreamSequenceNumbers() +
|
||||
'}';
|
||||
}
|
||||
|
||||
protected abstract SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType> createConcreteDataSourceMetaData(
|
||||
String streamId,
|
||||
Map<PartitionIdType, SequenceOffsetType> newMap
|
||||
SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> seekableStreamSequenceNumbers
|
||||
);
|
||||
}
|
||||
|
|
|
@ -0,0 +1,221 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.indexing.seekablestream;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* Represents the end sequenceNumber per partition of a sequence. Note that end sequenceNumbers are always
|
||||
* exclusive/inclusive in Kafka/Kinesis indexing service, respectively.
|
||||
*
|
||||
* To be backward compatible with both Kafka and Kinesis datasource metadata when
|
||||
* serializing and deserializing json, redundant constructor fields stream, topic,
|
||||
* partitionSequenceNumberMap and partitionOffsetMap are created. Only one of topic, stream
|
||||
* should have a non-null value and only one of partitionOffsetMap and partitionSequenceNumberMap
|
||||
* should have a non-null value.
|
||||
*
|
||||
* Redundant getters are used for proper Jackson serialization/deserialization when processing terminologies
|
||||
* used by Kafka and Kinesis (i.e. topic vs. stream)
|
||||
*/
|
||||
public class SeekableStreamEndSequenceNumbers<PartitionIdType, SequenceOffsetType> implements
|
||||
SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType>
|
||||
{
|
||||
// stream/topic
|
||||
private final String stream;
|
||||
// partitionId -> sequence number
|
||||
private final Map<PartitionIdType, SequenceOffsetType> partitionSequenceNumberMap;
|
||||
|
||||
@JsonCreator
|
||||
public SeekableStreamEndSequenceNumbers(
|
||||
@JsonProperty("stream") final String stream,
|
||||
// kept for backward compatibility
|
||||
@JsonProperty("topic") final String topic,
|
||||
@JsonProperty("partitionSequenceNumberMap")
|
||||
final Map<PartitionIdType, SequenceOffsetType> partitionSequenceNumberMap,
|
||||
// kept for backward compatibility
|
||||
@JsonProperty("partitionOffsetMap") final Map<PartitionIdType, SequenceOffsetType> partitionOffsetMap
|
||||
)
|
||||
{
|
||||
this.stream = stream == null ? topic : stream;
|
||||
this.partitionSequenceNumberMap = partitionOffsetMap == null ? partitionSequenceNumberMap : partitionOffsetMap;
|
||||
|
||||
Preconditions.checkNotNull(this.stream, "stream");
|
||||
Preconditions.checkNotNull(this.partitionSequenceNumberMap, "partitionIdToSequenceNumberMap");
|
||||
}
|
||||
|
||||
public SeekableStreamEndSequenceNumbers(
|
||||
final String stream,
|
||||
final Map<PartitionIdType, SequenceOffsetType> partitionSequenceNumberMap
|
||||
)
|
||||
{
|
||||
this(stream, null, partitionSequenceNumberMap, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts this end sequence numbers into start sequence numbers. This conversion is required when checking two
|
||||
* sequence numbers are "matched" in {@code IndexerSQLMetadataStorageCoordinator#updateDataSourceMetadataWithHandle}
|
||||
* because only sequences numbers of the same type can be compared.
|
||||
*
|
||||
* @param isExclusiveEndOffset flag that end offsets are exclusive. Should be true for Kafka and false for Kinesis.
|
||||
*/
|
||||
public SeekableStreamStartSequenceNumbers<PartitionIdType, SequenceOffsetType> asStartPartitions(
|
||||
boolean isExclusiveEndOffset
|
||||
)
|
||||
{
|
||||
return new SeekableStreamStartSequenceNumbers<>(
|
||||
stream,
|
||||
partitionSequenceNumberMap,
|
||||
// All start offsets are supposed to be opposite
|
||||
isExclusiveEndOffset ? Collections.emptySet() : partitionSequenceNumberMap.keySet()
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public String getStream()
|
||||
{
|
||||
return stream;
|
||||
}
|
||||
|
||||
/**
|
||||
* Identical to {@link #getStream()}. Here for backwards compatibility, so a serialized
|
||||
* SeekableStreamStartSequenceNumbers can be read by older Druid versions as a KafkaPartitions object.
|
||||
*/
|
||||
@JsonProperty
|
||||
public String getTopic()
|
||||
{
|
||||
return stream;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public Map<PartitionIdType, SequenceOffsetType> getPartitionSequenceNumberMap()
|
||||
{
|
||||
return partitionSequenceNumberMap;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> plus(
|
||||
SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> other
|
||||
)
|
||||
{
|
||||
if (this.getClass() != other.getClass()) {
|
||||
throw new IAE(
|
||||
"Expected instance of %s, got %s",
|
||||
this.getClass().getCanonicalName(),
|
||||
other.getClass().getCanonicalName()
|
||||
);
|
||||
}
|
||||
|
||||
final SeekableStreamEndSequenceNumbers<PartitionIdType, SequenceOffsetType> otherEnd =
|
||||
(SeekableStreamEndSequenceNumbers<PartitionIdType, SequenceOffsetType>) other;
|
||||
|
||||
if (stream.equals(otherEnd.stream)) {
|
||||
// Same stream, merge sequences.
|
||||
final Map<PartitionIdType, SequenceOffsetType> newMap = new HashMap<>(partitionSequenceNumberMap);
|
||||
newMap.putAll(otherEnd.partitionSequenceNumberMap);
|
||||
return new SeekableStreamEndSequenceNumbers<>(stream, newMap);
|
||||
} else {
|
||||
// Different stream, prefer "other".
|
||||
return other;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> minus(
|
||||
SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> other
|
||||
)
|
||||
{
|
||||
if (this.getClass() != other.getClass()) {
|
||||
throw new IAE(
|
||||
"Expected instance of %s, got %s",
|
||||
this.getClass().getCanonicalName(),
|
||||
other.getClass().getCanonicalName()
|
||||
);
|
||||
}
|
||||
|
||||
final SeekableStreamEndSequenceNumbers<PartitionIdType, SequenceOffsetType> otherEnd =
|
||||
(SeekableStreamEndSequenceNumbers<PartitionIdType, SequenceOffsetType>) other;
|
||||
|
||||
if (stream.equals(otherEnd.stream)) {
|
||||
// Same stream, remove partitions present in "that" from "this"
|
||||
final Map<PartitionIdType, SequenceOffsetType> newMap = new HashMap<>();
|
||||
|
||||
for (Entry<PartitionIdType, SequenceOffsetType> entry : partitionSequenceNumberMap.entrySet()) {
|
||||
if (!otherEnd.partitionSequenceNumberMap.containsKey(entry.getKey())) {
|
||||
newMap.put(entry.getKey(), entry.getValue());
|
||||
}
|
||||
}
|
||||
|
||||
return new SeekableStreamEndSequenceNumbers<>(stream, newMap);
|
||||
} else {
|
||||
// Different stream, prefer "this".
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Identical to {@link #getPartitionSequenceNumberMap()} ()}. Here for backwards compatibility, so a serialized
|
||||
* SeekableStreamStartSequenceNumbers can be read by older Druid versions as a KafkaPartitions object.
|
||||
*/
|
||||
@JsonProperty
|
||||
public Map<PartitionIdType, SequenceOffsetType> getPartitionOffsetMap()
|
||||
{
|
||||
return partitionSequenceNumberMap;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
SeekableStreamEndSequenceNumbers<?, ?> that = (SeekableStreamEndSequenceNumbers<?, ?>) o;
|
||||
return Objects.equals(stream, that.stream) &&
|
||||
Objects.equals(partitionSequenceNumberMap, that.partitionSequenceNumberMap);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(stream, partitionSequenceNumberMap);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "SeekableStreamEndSequenceNumbers{" +
|
||||
"stream='" + stream + '\'' +
|
||||
", partitionSequenceNumberMap=" + partitionSequenceNumberMap +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -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<PartitionIdType, SequenceOffsetType> 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<PartitionIdType, SequenceOffsetType> startPartitions;
|
||||
private final SeekableStreamPartitions<PartitionIdType, SequenceOffsetType> endPartitions;
|
||||
private final SeekableStreamStartSequenceNumbers<PartitionIdType, SequenceOffsetType> startSequenceNumbers;
|
||||
private final SeekableStreamEndSequenceNumbers<PartitionIdType, SequenceOffsetType> endSequenceNumbers;
|
||||
private final boolean useTransaction;
|
||||
private final Optional<DateTime> minimumMessageTime;
|
||||
private final Optional<DateTime> maximumMessageTime;
|
||||
private final Set<PartitionIdType> exclusiveStartSequenceNumberPartitions;
|
||||
|
||||
public SeekableStreamIndexTaskIOConfig(
|
||||
final @Nullable Integer taskGroupId, // can be null for backward compabitility
|
||||
final String baseSequenceName,
|
||||
final SeekableStreamPartitions<PartitionIdType, SequenceOffsetType> startPartitions,
|
||||
final SeekableStreamPartitions<PartitionIdType, SequenceOffsetType> endPartitions,
|
||||
final SeekableStreamStartSequenceNumbers<PartitionIdType, SequenceOffsetType> startSequenceNumbers,
|
||||
final SeekableStreamEndSequenceNumbers<PartitionIdType, SequenceOffsetType> endSequenceNumbers,
|
||||
final Boolean useTransaction,
|
||||
final DateTime minimumMessageTime,
|
||||
final DateTime maximumMessageTime,
|
||||
final Set<PartitionIdType> 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<PartitionIdType> getExclusiveStartSequenceNumberPartitions()
|
||||
{
|
||||
return exclusiveStartSequenceNumberPartitions;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@JsonProperty
|
||||
public Integer getTaskGroupId()
|
||||
|
@ -102,15 +85,15 @@ public abstract class SeekableStreamIndexTaskIOConfig<PartitionIdType, SequenceO
|
|||
}
|
||||
|
||||
@JsonProperty
|
||||
public SeekableStreamPartitions<PartitionIdType, SequenceOffsetType> getStartPartitions()
|
||||
public SeekableStreamStartSequenceNumbers<PartitionIdType, SequenceOffsetType> getStartSequenceNumbers()
|
||||
{
|
||||
return startPartitions;
|
||||
return startSequenceNumbers;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public SeekableStreamPartitions<PartitionIdType, SequenceOffsetType> getEndPartitions()
|
||||
public SeekableStreamEndSequenceNumbers<PartitionIdType, SequenceOffsetType> getEndSequenceNumbers()
|
||||
{
|
||||
return endPartitions;
|
||||
return endSequenceNumbers;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
|
|
|
@ -230,9 +230,9 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
|||
this.authorizerMapper = authorizerMapper;
|
||||
this.chatHandlerProvider = chatHandlerProvider;
|
||||
this.savedParseExceptions = savedParseExceptions;
|
||||
this.stream = ioConfig.getStartPartitions().getStream();
|
||||
this.stream = ioConfig.getStartSequenceNumbers().getStream();
|
||||
this.rowIngestionMeters = rowIngestionMetersFactory.createRowIngestionMeters();
|
||||
this.endOffsets = new ConcurrentHashMap<>(ioConfig.getEndPartitions().getPartitionSequenceNumberMap());
|
||||
this.endOffsets = new ConcurrentHashMap<>(ioConfig.getEndSequenceNumbers().getPartitionSequenceNumberMap());
|
||||
this.sequences = new CopyOnWriteArrayList<>();
|
||||
this.ingestionState = IngestionState.NOT_STARTED;
|
||||
|
||||
|
@ -263,7 +263,6 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
|||
status = Status.STARTING;
|
||||
this.toolbox = toolbox;
|
||||
|
||||
|
||||
if (!restoreSequences()) {
|
||||
final TreeMap<Integer, Map<PartitionIdType, SequenceOffsetType>> checkpoints = getCheckPointsFromContext(
|
||||
toolbox,
|
||||
|
@ -299,7 +298,7 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
|||
addSequence(new SequenceMetadata<>(
|
||||
0,
|
||||
StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), 0),
|
||||
ioConfig.getStartPartitions().getPartitionSequenceNumberMap(),
|
||||
ioConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap(),
|
||||
endOffsets,
|
||||
false,
|
||||
null
|
||||
|
@ -350,8 +349,6 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
|||
appenderator = task.newAppenderator(fireDepartmentMetrics, toolbox);
|
||||
driver = task.newDriver(appenderator, toolbox, fireDepartmentMetrics);
|
||||
|
||||
final String stream = ioConfig.getStartPartitions().getStream();
|
||||
|
||||
// Start up, set up initial sequences.
|
||||
final Object restoredMetadata = driver.startJob();
|
||||
if (restoredMetadata == null) {
|
||||
|
@ -360,7 +357,7 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
|||
Preconditions.checkState(sequences.get(0).startOffsets.entrySet().stream().allMatch(
|
||||
partitionOffsetEntry ->
|
||||
createSequenceNumber(partitionOffsetEntry.getValue()).compareTo(
|
||||
createSequenceNumber(ioConfig.getStartPartitions()
|
||||
createSequenceNumber(ioConfig.getStartSequenceNumbers()
|
||||
.getPartitionSequenceNumberMap()
|
||||
.get(partitionOffsetEntry.getKey())
|
||||
)) >= 0
|
||||
|
@ -369,27 +366,28 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
|||
} else {
|
||||
@SuppressWarnings("unchecked")
|
||||
final Map<String, Object> restoredMetadataMap = (Map) restoredMetadata;
|
||||
final SeekableStreamPartitions<PartitionIdType, SequenceOffsetType> restoredNextPartitions = deserializePartitionsFromMetadata(
|
||||
toolbox.getObjectMapper(),
|
||||
restoredMetadataMap.get(METADATA_NEXT_PARTITIONS)
|
||||
);
|
||||
final SeekableStreamEndSequenceNumbers<PartitionIdType, SequenceOffsetType> restoredNextPartitions =
|
||||
deserializePartitionsFromMetadata(
|
||||
toolbox.getObjectMapper(),
|
||||
restoredMetadataMap.get(METADATA_NEXT_PARTITIONS)
|
||||
);
|
||||
|
||||
currOffsets.putAll(restoredNextPartitions.getPartitionSequenceNumberMap());
|
||||
|
||||
// Sanity checks.
|
||||
if (!restoredNextPartitions.getStream().equals(ioConfig.getStartPartitions().getStream())) {
|
||||
if (!restoredNextPartitions.getStream().equals(ioConfig.getStartSequenceNumbers().getStream())) {
|
||||
throw new ISE(
|
||||
"WTF?! Restored stream[%s] but expected stream[%s]",
|
||||
restoredNextPartitions.getStream(),
|
||||
ioConfig.getStartPartitions().getStream()
|
||||
ioConfig.getStartSequenceNumbers().getStream()
|
||||
);
|
||||
}
|
||||
|
||||
if (!currOffsets.keySet().equals(ioConfig.getStartPartitions().getPartitionSequenceNumberMap().keySet())) {
|
||||
if (!currOffsets.keySet().equals(ioConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().keySet())) {
|
||||
throw new ISE(
|
||||
"WTF?! Restored partitions[%s] but expected partitions[%s]",
|
||||
currOffsets.keySet(),
|
||||
ioConfig.getStartPartitions().getPartitionSequenceNumberMap().keySet()
|
||||
ioConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().keySet()
|
||||
);
|
||||
}
|
||||
// sequences size can be 0 only when all sequences got published and task stopped before it could finish
|
||||
|
@ -418,10 +416,10 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
|||
if (!isEndOffsetExclusive()) {
|
||||
for (Map.Entry<PartitionIdType, SequenceOffsetType> entry : currOffsets.entrySet()) {
|
||||
final boolean isAtStart = entry.getValue().equals(
|
||||
ioConfig.getStartPartitions().getPartitionSequenceNumberMap().get(entry.getKey())
|
||||
ioConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(entry.getKey())
|
||||
);
|
||||
|
||||
if (!isAtStart || ioConfig.getExclusiveStartSequenceNumberPartitions().contains(entry.getKey())) {
|
||||
if (!isAtStart || ioConfig.getStartSequenceNumbers().getExclusivePartitions().contains(entry.getKey())) {
|
||||
lastReadOffsets.put(entry.getKey(), entry.getValue());
|
||||
}
|
||||
}
|
||||
|
@ -438,12 +436,7 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
|||
@Override
|
||||
public Object getMetadata()
|
||||
{
|
||||
return ImmutableMap.of(
|
||||
METADATA_NEXT_PARTITIONS, new SeekableStreamPartitions<>(
|
||||
ioConfig.getStartPartitions().getStream(),
|
||||
snapshot
|
||||
)
|
||||
);
|
||||
return ImmutableMap.of(METADATA_NEXT_PARTITIONS, new SeekableStreamEndSequenceNumbers<>(stream, snapshot));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -458,7 +451,7 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
|||
maybePersistAndPublishSequences(committerSupplier);
|
||||
|
||||
Set<StreamPartition<PartitionIdType>> assignment = assignPartitions(recordSupplier);
|
||||
possiblyResetDataSourceMetadata(toolbox, recordSupplier, assignment, currOffsets);
|
||||
possiblyResetDataSourceMetadata(toolbox, recordSupplier, assignment);
|
||||
seekToStartingSequence(recordSupplier, assignment);
|
||||
|
||||
ingestionState = IngestionState.BUILD_SEGMENTS;
|
||||
|
@ -476,7 +469,7 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
|||
// partitions upon resuming. Don't call "seekToStartingSequence" after "assignPartitions", because there's
|
||||
// no need to re-seek here. All we're going to be doing is dropping partitions.
|
||||
assignment = assignPartitions(recordSupplier);
|
||||
possiblyResetDataSourceMetadata(toolbox, recordSupplier, assignment, currOffsets);
|
||||
possiblyResetDataSourceMetadata(toolbox, recordSupplier, assignment);
|
||||
|
||||
if (assignment.isEmpty()) {
|
||||
log.info("All partitions have been fully read");
|
||||
|
@ -512,7 +505,7 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
|||
// note: getRecords() also updates assignment
|
||||
stillReading = !assignment.isEmpty();
|
||||
|
||||
SequenceMetadata sequenceToCheckpoint = null;
|
||||
SequenceMetadata<PartitionIdType, SequenceOffsetType> sequenceToCheckpoint = null;
|
||||
for (OrderedPartitionableRecord<PartitionIdType, SequenceOffsetType> record : records) {
|
||||
final boolean shouldProcess = verifyRecordInRange(record.getPartitionId(), record.getSequenceNumber());
|
||||
|
||||
|
@ -652,11 +645,19 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
|||
task.getDataSource(),
|
||||
ioConfig.getTaskGroupId(),
|
||||
task.getIOConfig().getBaseSequenceName(),
|
||||
createDataSourceMetadata(new SeekableStreamPartitions<>(
|
||||
stream,
|
||||
sequenceToCheckpoint.getStartOffsets()
|
||||
)),
|
||||
createDataSourceMetadata(new SeekableStreamPartitions<>(stream, currOffsets))
|
||||
createDataSourceMetadata(
|
||||
new SeekableStreamStartSequenceNumbers<>(
|
||||
stream,
|
||||
sequenceToCheckpoint.getStartOffsets(),
|
||||
ioConfig.getStartSequenceNumbers().getExclusivePartitions()
|
||||
)
|
||||
),
|
||||
createDataSourceMetadata(
|
||||
new SeekableStreamEndSequenceNumbers<>(
|
||||
stream,
|
||||
currOffsets
|
||||
)
|
||||
)
|
||||
);
|
||||
if (!toolbox.getTaskActionClient().submit(checkpointAction)) {
|
||||
throw new ISE("Checkpoint request with sequences [%s] failed, dying", currOffsets);
|
||||
|
@ -1232,9 +1233,11 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
|||
new ResetDataSourceMetadataAction(
|
||||
task.getDataSource(),
|
||||
createDataSourceMetadata(
|
||||
new SeekableStreamPartitions<>(
|
||||
ioConfig.getStartPartitions().getStream(),
|
||||
partitionOffsetMap
|
||||
new SeekableStreamStartSequenceNumbers<>(
|
||||
ioConfig.getStartSequenceNumbers().getStream(),
|
||||
partitionOffsetMap,
|
||||
// Clear all exclusive start offsets for automatic reset
|
||||
Collections.emptySet()
|
||||
)
|
||||
)
|
||||
)
|
||||
|
@ -1743,14 +1746,14 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
|||
protected abstract SequenceOffsetType getNextStartOffset(SequenceOffsetType sequenceNumber);
|
||||
|
||||
/**
|
||||
* deserializes stored metadata into SeekableStreamPartitions
|
||||
* deserializes stored metadata into SeekableStreamStartSequenceNumbers
|
||||
*
|
||||
* @param mapper json objectMapper
|
||||
* @param object metadata
|
||||
*
|
||||
* @return SeekableStreamPartitions
|
||||
* @return SeekableStreamEndSequenceNumbers
|
||||
*/
|
||||
protected abstract SeekableStreamPartitions<PartitionIdType, SequenceOffsetType> deserializePartitionsFromMetadata(
|
||||
protected abstract SeekableStreamEndSequenceNumbers<PartitionIdType, SequenceOffsetType> deserializePartitionsFromMetadata(
|
||||
ObjectMapper mapper,
|
||||
Object object
|
||||
);
|
||||
|
@ -1780,7 +1783,7 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
|||
* @return datasource metadata
|
||||
*/
|
||||
protected abstract SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType> createDataSourceMetadata(
|
||||
SeekableStreamPartitions<PartitionIdType, SequenceOffsetType> partitions
|
||||
SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> partitions
|
||||
);
|
||||
|
||||
/**
|
||||
|
@ -1795,17 +1798,11 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
|||
/**
|
||||
* check if the sequence offsets stored in currOffsets are still valid sequence offsets compared to
|
||||
* earliest sequence offsets fetched from stream
|
||||
*
|
||||
* @param toolbox
|
||||
* @param recordSupplier
|
||||
* @param assignment
|
||||
* @param currOffsets
|
||||
*/
|
||||
protected abstract void possiblyResetDataSourceMetadata(
|
||||
TaskToolbox toolbox,
|
||||
RecordSupplier<PartitionIdType, SequenceOffsetType> recordSupplier,
|
||||
Set<StreamPartition<PartitionIdType>> assignment,
|
||||
Map<PartitionIdType, SequenceOffsetType> currOffsets
|
||||
Set<StreamPartition<PartitionIdType>> assignment
|
||||
);
|
||||
|
||||
/**
|
||||
|
|
|
@ -1,151 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.indexing.seekablestream;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
|
||||
import javax.validation.constraints.NotNull;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* class that encapsulates a partitionIdToSequenceNumberMap of partitionId -> sequenceNumber.
|
||||
* To be backward compatible with both Kafka and Kinesis datasource metadata when
|
||||
* serializing and deserializing json, redundant constructor fields stream, topic,
|
||||
* partitionSequenceNumberMap and partitionOffsetMap are created. Only one of topic, stream
|
||||
* should have a non-null value and only one of partitionOffsetMap and partitionSequenceNumberMap
|
||||
* should have a non-null value.
|
||||
*
|
||||
* Redundant getters are used for proper Jackson serialization/deserialization when processing terminologies
|
||||
* used by Kafka and Kinesis (i.e. topic vs. stream)
|
||||
*
|
||||
* @param <PartitionIdType> partition id type
|
||||
* @param <SequenceOffsetType> sequence number type
|
||||
*/
|
||||
public class SeekableStreamPartitions<PartitionIdType, SequenceOffsetType>
|
||||
{
|
||||
// this special marker is used by the KinesisSupervisor to set the endOffsets
|
||||
// of newly created indexing tasks. This is necessary because streaming tasks do not
|
||||
// have endPartitionOffsets. This marker signals to the task that it should continue
|
||||
// to ingest data until taskDuration has elapsed or the task was stopped or paused or killed
|
||||
public static final String NO_END_SEQUENCE_NUMBER = "NO_END_SEQUENCE_NUMBER";
|
||||
|
||||
// stream/topic
|
||||
private final String stream;
|
||||
// partitionId -> sequence number
|
||||
private final Map<PartitionIdType, SequenceOffsetType> partitionIdToSequenceNumberMap;
|
||||
|
||||
@JsonCreator
|
||||
public SeekableStreamPartitions(
|
||||
@JsonProperty("stream") final String stream,
|
||||
// kept for backward compatibility
|
||||
@JsonProperty("topic") final String topic,
|
||||
@JsonProperty("partitionSequenceNumberMap")
|
||||
final Map<PartitionIdType, SequenceOffsetType> partitionSequenceNumberMap,
|
||||
// kept for backward compatibility
|
||||
@JsonProperty("partitionOffsetMap") final Map<PartitionIdType, SequenceOffsetType> partitionOffsetMap
|
||||
)
|
||||
{
|
||||
this.stream = stream == null ? topic : stream;
|
||||
this.partitionIdToSequenceNumberMap = ImmutableMap.copyOf(partitionOffsetMap == null
|
||||
? partitionSequenceNumberMap
|
||||
: partitionOffsetMap);
|
||||
Preconditions.checkArgument(this.stream != null);
|
||||
Preconditions.checkArgument(partitionIdToSequenceNumberMap != null);
|
||||
}
|
||||
|
||||
// constructor for backward compatibility
|
||||
public SeekableStreamPartitions(
|
||||
@NotNull final String stream,
|
||||
final Map<PartitionIdType, SequenceOffsetType> partitionOffsetMap
|
||||
)
|
||||
{
|
||||
this(
|
||||
Preconditions.checkNotNull(stream, "stream"),
|
||||
null,
|
||||
Preconditions.checkNotNull(partitionOffsetMap, "partitionOffsetMap"),
|
||||
null
|
||||
);
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getStream()
|
||||
{
|
||||
return stream;
|
||||
}
|
||||
|
||||
/**
|
||||
* Identical to {@link #getStream()}. Here for backwards compatibility, so a serialized SeekableStreamPartitions can
|
||||
* be read by older Druid versions as a KafkaPartitions object.
|
||||
*/
|
||||
@JsonProperty
|
||||
public String getTopic()
|
||||
{
|
||||
return stream;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Map<PartitionIdType, SequenceOffsetType> getPartitionSequenceNumberMap()
|
||||
{
|
||||
return partitionIdToSequenceNumberMap;
|
||||
}
|
||||
|
||||
/**
|
||||
* Identical to {@link #getPartitionSequenceNumberMap()} ()}. Here for backwards compatibility, so a serialized
|
||||
* SeekableStreamPartitions can be read by older Druid versions as a KafkaPartitions object.
|
||||
*/
|
||||
@JsonProperty
|
||||
public Map<PartitionIdType, SequenceOffsetType> getPartitionOffsetMap()
|
||||
{
|
||||
return partitionIdToSequenceNumberMap;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
SeekableStreamPartitions that = (SeekableStreamPartitions) o;
|
||||
return Objects.equals(stream, that.stream) &&
|
||||
Objects.equals(partitionIdToSequenceNumberMap, that.partitionIdToSequenceNumberMap);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(stream, partitionIdToSequenceNumberMap);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return getClass().getSimpleName() + "{" +
|
||||
"stream='" + stream + '\'' +
|
||||
", partitionSequenceNumberMap=" + partitionIdToSequenceNumberMap +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -0,0 +1,64 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.indexing.seekablestream;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||
import com.fasterxml.jackson.annotation.JsonSubTypes.Type;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeInfo.Id;
|
||||
import org.apache.druid.indexing.overlord.DataSourceMetadata;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
@JsonTypeInfo(use = Id.NAME, property = "type", defaultImpl = SeekableStreamEndSequenceNumbers.class)
|
||||
@JsonSubTypes({
|
||||
@Type(name = "start", value = SeekableStreamStartSequenceNumbers.class),
|
||||
@Type(name = "end", value = SeekableStreamEndSequenceNumbers.class)
|
||||
})
|
||||
public interface SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType>
|
||||
{
|
||||
/**
|
||||
* Returns the stream/topic name.
|
||||
*/
|
||||
String getStream();
|
||||
|
||||
/**
|
||||
* Returns a map of partitionId -> sequenceNumber.
|
||||
*/
|
||||
Map<PartitionIdType, SequenceOffsetType> getPartitionSequenceNumberMap();
|
||||
|
||||
/**
|
||||
* Merges this and the given other and returns the merged result.
|
||||
*
|
||||
* @see DataSourceMetadata#plus
|
||||
*/
|
||||
SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> plus(
|
||||
SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> other
|
||||
);
|
||||
|
||||
/**
|
||||
* Subtracts the given other from this and returns the result.
|
||||
*
|
||||
* @see DataSourceMetadata#minus
|
||||
*/
|
||||
SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> minus(
|
||||
SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> other
|
||||
);
|
||||
}
|
|
@ -0,0 +1,208 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.indexing.seekablestream;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Represents the start sequenceNumber per partition of a sequence. This class keeps an additional set of
|
||||
* {@link #exclusivePartitions} for Kinesis indexing service in where each start offset can be either inclusive
|
||||
* or exclusive.
|
||||
*/
|
||||
public class SeekableStreamStartSequenceNumbers<PartitionIdType, SequenceOffsetType> implements
|
||||
SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType>
|
||||
{
|
||||
// stream/topic
|
||||
private final String stream;
|
||||
// partitionId -> sequence number
|
||||
private final Map<PartitionIdType, SequenceOffsetType> partitionSequenceNumberMap;
|
||||
private final Set<PartitionIdType> exclusivePartitions;
|
||||
|
||||
@JsonCreator
|
||||
public SeekableStreamStartSequenceNumbers(
|
||||
@JsonProperty("stream") final String stream,
|
||||
@JsonProperty("partitionSequenceNumberMap")
|
||||
final Map<PartitionIdType, SequenceOffsetType> partitionSequenceNumberMap,
|
||||
@JsonProperty("exclusivePartitions") @Nullable final Set<PartitionIdType> exclusivePartitions
|
||||
)
|
||||
{
|
||||
this.stream = Preconditions.checkNotNull(stream, "stream");
|
||||
this.partitionSequenceNumberMap = Preconditions.checkNotNull(
|
||||
partitionSequenceNumberMap,
|
||||
"partitionIdToSequenceNumberMap"
|
||||
);
|
||||
// exclusiveOffset can be null if this class is deserialized from metadata store. Note that only end offsets are
|
||||
// stored in metadata store.
|
||||
// The default is true because there was only Kafka indexing service before in which the end offset is always
|
||||
// exclusive.
|
||||
this.exclusivePartitions = exclusivePartitions == null ? Collections.emptySet() : exclusivePartitions;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public String getStream()
|
||||
{
|
||||
return stream;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public Map<PartitionIdType, SequenceOffsetType> getPartitionSequenceNumberMap()
|
||||
{
|
||||
return partitionSequenceNumberMap;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> plus(
|
||||
SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> other
|
||||
)
|
||||
{
|
||||
if (this.getClass() != other.getClass()) {
|
||||
throw new IAE(
|
||||
"Expected instance of %s, got %s",
|
||||
this.getClass().getCanonicalName(),
|
||||
other.getClass().getCanonicalName()
|
||||
);
|
||||
}
|
||||
|
||||
final SeekableStreamStartSequenceNumbers<PartitionIdType, SequenceOffsetType> otherStart =
|
||||
(SeekableStreamStartSequenceNumbers<PartitionIdType, SequenceOffsetType>) other;
|
||||
|
||||
if (stream.equals(otherStart.stream)) {
|
||||
// Same stream, merge sequences.
|
||||
final Map<PartitionIdType, SequenceOffsetType> newMap = new HashMap<>(partitionSequenceNumberMap);
|
||||
newMap.putAll(otherStart.partitionSequenceNumberMap);
|
||||
|
||||
// A partition is exclusive if it's
|
||||
// 1) exclusive in "this" and it's not in "other"'s partitionSequenceNumberMap or
|
||||
// 2) exclusive in "other"
|
||||
final Set<PartitionIdType> newExclusivePartitions = new HashSet<>();
|
||||
partitionSequenceNumberMap.forEach(
|
||||
(partitionId, sequenceOffset) -> {
|
||||
if (exclusivePartitions.contains(partitionId)
|
||||
&& !otherStart.partitionSequenceNumberMap.containsKey(partitionId)) {
|
||||
newExclusivePartitions.add(partitionId);
|
||||
}
|
||||
}
|
||||
);
|
||||
newExclusivePartitions.addAll(otherStart.exclusivePartitions);
|
||||
|
||||
return new SeekableStreamStartSequenceNumbers<>(
|
||||
stream,
|
||||
newMap,
|
||||
newExclusivePartitions
|
||||
);
|
||||
} else {
|
||||
// Different stream, prefer "other".
|
||||
return other;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> minus(
|
||||
SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> other
|
||||
)
|
||||
{
|
||||
if (this.getClass() != other.getClass()) {
|
||||
throw new IAE(
|
||||
"Expected instance of %s, got %s",
|
||||
this.getClass().getCanonicalName(),
|
||||
other.getClass().getCanonicalName()
|
||||
);
|
||||
}
|
||||
|
||||
final SeekableStreamStartSequenceNumbers<PartitionIdType, SequenceOffsetType> otherStart =
|
||||
(SeekableStreamStartSequenceNumbers<PartitionIdType, SequenceOffsetType>) other;
|
||||
|
||||
if (stream.equals(otherStart.stream)) {
|
||||
// Same stream, remove partitions present in "that" from "this"
|
||||
final Map<PartitionIdType, SequenceOffsetType> newMap = new HashMap<>();
|
||||
final Set<PartitionIdType> newExclusivePartitions = new HashSet<>();
|
||||
|
||||
for (Entry<PartitionIdType, SequenceOffsetType> entry : partitionSequenceNumberMap.entrySet()) {
|
||||
if (!otherStart.partitionSequenceNumberMap.containsKey(entry.getKey())) {
|
||||
newMap.put(entry.getKey(), entry.getValue());
|
||||
// A partition is exclusive if it's exclusive in "this" and not in "other"'s partitionSequenceNumberMap
|
||||
if (exclusivePartitions.contains(entry.getKey())) {
|
||||
newExclusivePartitions.add(entry.getKey());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return new SeekableStreamStartSequenceNumbers<>(
|
||||
stream,
|
||||
newMap,
|
||||
newExclusivePartitions
|
||||
);
|
||||
} else {
|
||||
// Different stream, prefer "this".
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Set<PartitionIdType> getExclusivePartitions()
|
||||
{
|
||||
return exclusivePartitions;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
SeekableStreamStartSequenceNumbers<?, ?> that = (SeekableStreamStartSequenceNumbers<?, ?>) o;
|
||||
return Objects.equals(stream, that.stream) &&
|
||||
Objects.equals(partitionSequenceNumberMap, that.partitionSequenceNumberMap) &&
|
||||
Objects.equals(exclusivePartitions, that.exclusivePartitions);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(stream, partitionSequenceNumberMap, exclusivePartitions);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "SeekableStreamStartSequenceNumbers{" +
|
||||
"stream='" + stream + '\'' +
|
||||
", partitionSequenceNumberMap=" + partitionSequenceNumberMap +
|
||||
", exclusivePartitions=" + exclusivePartitions +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -47,7 +47,6 @@ public class SequenceMetadata<PartitionIdType, SequenceOffsetType>
|
|||
private final Set<PartitionIdType> exclusiveStartPartitions;
|
||||
private final Set<PartitionIdType> assignments;
|
||||
private final boolean sentinel;
|
||||
private boolean checkpointed;
|
||||
/**
|
||||
* Lock for accessing {@link #endOffsets} and {@link #checkpointed}. This lock is required because
|
||||
* {@link #setEndOffsets)} can be called by both the main thread and the HTTP thread.
|
||||
|
@ -57,6 +56,8 @@ public class SequenceMetadata<PartitionIdType, SequenceOffsetType>
|
|||
final Map<PartitionIdType, SequenceOffsetType> startOffsets;
|
||||
final Map<PartitionIdType, SequenceOffsetType> endOffsets;
|
||||
|
||||
private boolean checkpointed;
|
||||
|
||||
@JsonCreator
|
||||
public SequenceMetadata(
|
||||
@JsonProperty("sequenceId") int sequenceId,
|
||||
|
@ -274,9 +275,9 @@ public class SequenceMetadata<PartitionIdType, SequenceOffsetType>
|
|||
// subset of segments
|
||||
return ImmutableMap.of(
|
||||
SeekableStreamIndexTaskRunner.METADATA_NEXT_PARTITIONS,
|
||||
new SeekableStreamPartitions<>(stream, lastPersistedOffsets),
|
||||
new SeekableStreamStartSequenceNumbers<>(stream, lastPersistedOffsets, exclusiveStartPartitions),
|
||||
SeekableStreamIndexTaskRunner.METADATA_PUBLISH_PARTITIONS,
|
||||
new SeekableStreamPartitions<>(stream, endOffsets)
|
||||
new SeekableStreamEndSequenceNumbers<>(stream, endOffsets)
|
||||
);
|
||||
}
|
||||
finally {
|
||||
|
@ -301,7 +302,7 @@ public class SequenceMetadata<PartitionIdType, SequenceOffsetType>
|
|||
{
|
||||
return (segments, commitMetadata) -> {
|
||||
final Map commitMetaMap = (Map) Preconditions.checkNotNull(commitMetadata, "commitMetadata");
|
||||
final SeekableStreamPartitions<PartitionIdType, SequenceOffsetType> finalPartitions =
|
||||
final SeekableStreamEndSequenceNumbers<PartitionIdType, SequenceOffsetType> finalPartitions =
|
||||
runner.deserializePartitionsFromMetadata(
|
||||
toolbox.getObjectMapper(),
|
||||
commitMetaMap.get(SeekableStreamIndexTaskRunner.METADATA_PUBLISH_PARTITIONS)
|
||||
|
@ -322,7 +323,11 @@ public class SequenceMetadata<PartitionIdType, SequenceOffsetType>
|
|||
action = new SegmentTransactionalInsertAction(
|
||||
segments,
|
||||
runner.createDataSourceMetadata(
|
||||
new SeekableStreamPartitions<>(finalPartitions.getStream(), getStartOffsets())
|
||||
new SeekableStreamStartSequenceNumbers<>(
|
||||
finalPartitions.getStream(),
|
||||
getStartOffsets(),
|
||||
exclusiveStartPartitions
|
||||
)
|
||||
),
|
||||
runner.createDataSourceMetadata(finalPartitions)
|
||||
);
|
||||
|
|
|
@ -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<PartitionIdType, SequenceOffsetTy
|
|||
Map<PartitionIdType, SequenceOffsetType> checkpoint = checkpoints.get(sequenceId);
|
||||
// We have already verified the stream of the current checkpoint is same with that in ioConfig.
|
||||
// See checkpoint().
|
||||
if (checkpoint.equals(previousCheckpoint.getSeekableStreamPartitions()
|
||||
if (checkpoint.equals(previousCheckpoint.getSeekableStreamSequenceNumbers()
|
||||
.getPartitionSequenceNumberMap()
|
||||
)) {
|
||||
break;
|
||||
|
@ -1097,7 +1097,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
@SuppressWarnings("unchecked")
|
||||
final SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType> resetMetadata = (SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType>) dataSourceMetadata;
|
||||
|
||||
if (resetMetadata.getSeekableStreamPartitions().getStream().equals(ioConfig.getStream())) {
|
||||
if (resetMetadata.getSeekableStreamSequenceNumbers().getStream().equals(ioConfig.getStream())) {
|
||||
// metadata can be null
|
||||
final DataSourceMetadata metadata = indexerMetadataStorageCoordinator.getDataSourceMetadata(dataSource);
|
||||
if (metadata != null && !checkSourceMetadataMatch(metadata)) {
|
||||
|
@ -1113,12 +1113,12 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
// defend against consecutive reset requests from replicas
|
||||
// as well as the case where the metadata store do not have an entry for the reset partitions
|
||||
boolean doReset = false;
|
||||
for (Entry<PartitionIdType, SequenceOffsetType> resetPartitionOffset : resetMetadata.getSeekableStreamPartitions()
|
||||
for (Entry<PartitionIdType, SequenceOffsetType> resetPartitionOffset : resetMetadata.getSeekableStreamSequenceNumbers()
|
||||
.getPartitionSequenceNumberMap()
|
||||
.entrySet()) {
|
||||
final SequenceOffsetType partitionOffsetInMetadataStore = currentMetadata == null
|
||||
? null
|
||||
: currentMetadata.getSeekableStreamPartitions()
|
||||
: currentMetadata.getSeekableStreamSequenceNumbers()
|
||||
.getPartitionSequenceNumberMap()
|
||||
.get(resetPartitionOffset.getKey());
|
||||
final TaskGroup partitionTaskGroup = activelyReadingTaskGroups.get(
|
||||
|
@ -1152,7 +1152,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
}
|
||||
}
|
||||
if (metadataUpdateSuccess) {
|
||||
resetMetadata.getSeekableStreamPartitions().getPartitionSequenceNumberMap().keySet().forEach(partition -> {
|
||||
resetMetadata.getSeekableStreamSequenceNumbers().getPartitionSequenceNumberMap().keySet().forEach(partition -> {
|
||||
final int groupId = getTaskGroupIdForPartition(partition);
|
||||
killTaskGroupForPartitions(ImmutableSet.of(partition), "DataSourceMetadata is updated while reset");
|
||||
activelyReadingTaskGroups.remove(groupId);
|
||||
|
@ -1164,7 +1164,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
} else {
|
||||
log.warn(
|
||||
"Reset metadata stream [%s] and supervisor's stream name [%s] do not match",
|
||||
resetMetadata.getSeekableStreamPartitions().getStream(),
|
||||
resetMetadata.getSeekableStreamSequenceNumbers().getStream(),
|
||||
ioConfig.getStream()
|
||||
);
|
||||
}
|
||||
|
@ -1240,7 +1240,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
// seamless schema migration.
|
||||
|
||||
Iterator<PartitionIdType> it = seekableStreamIndexTask.getIOConfig()
|
||||
.getStartPartitions()
|
||||
.getStartSequenceNumbers()
|
||||
.getPartitionSequenceNumberMap()
|
||||
.keySet()
|
||||
.iterator();
|
||||
|
@ -1264,7 +1264,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
log.debug("Task [%s], status [%s]", taskId, status);
|
||||
if (status == SeekableStreamIndexTaskRunner.Status.PUBLISHING) {
|
||||
seekableStreamIndexTask.getIOConfig()
|
||||
.getStartPartitions()
|
||||
.getStartSequenceNumbers()
|
||||
.getPartitionSequenceNumberMap()
|
||||
.keySet()
|
||||
.forEach(
|
||||
|
@ -1272,7 +1272,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
getTaskGroupIdForPartition(partition),
|
||||
taskId,
|
||||
seekableStreamIndexTask.getIOConfig()
|
||||
.getStartPartitions()
|
||||
.getStartSequenceNumbers()
|
||||
.getPartitionSequenceNumberMap()
|
||||
));
|
||||
|
||||
|
@ -1300,7 +1300,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
}
|
||||
} else {
|
||||
for (PartitionIdType partition : seekableStreamIndexTask.getIOConfig()
|
||||
.getStartPartitions()
|
||||
.getStartSequenceNumbers()
|
||||
.getPartitionSequenceNumberMap()
|
||||
.keySet()) {
|
||||
if (!taskGroupId.equals(getTaskGroupIdForPartition(partition))) {
|
||||
|
@ -1340,12 +1340,14 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
taskGroupId,
|
||||
ImmutableMap.copyOf(
|
||||
seekableStreamIndexTask.getIOConfig()
|
||||
.getStartPartitions()
|
||||
.getStartSequenceNumbers()
|
||||
.getPartitionSequenceNumberMap()
|
||||
),
|
||||
seekableStreamIndexTask.getIOConfig().getMinimumMessageTime(),
|
||||
seekableStreamIndexTask.getIOConfig().getMaximumMessageTime(),
|
||||
seekableStreamIndexTask.getIOConfig().getExclusiveStartSequenceNumberPartitions()
|
||||
seekableStreamIndexTask.getIOConfig()
|
||||
.getStartSequenceNumbers()
|
||||
.getExclusivePartitions()
|
||||
);
|
||||
}
|
||||
);
|
||||
|
@ -1468,13 +1470,13 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
final SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType> latestDataSourceMetadata = (SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType>) rawDataSourceMetadata;
|
||||
|
||||
final boolean hasValidOffsetsFromDb = latestDataSourceMetadata != null &&
|
||||
latestDataSourceMetadata.getSeekableStreamPartitions() != null &&
|
||||
latestDataSourceMetadata.getSeekableStreamSequenceNumbers() != null &&
|
||||
ioConfig.getStream().equals(
|
||||
latestDataSourceMetadata.getSeekableStreamPartitions().getStream()
|
||||
latestDataSourceMetadata.getSeekableStreamSequenceNumbers().getStream()
|
||||
);
|
||||
final Map<PartitionIdType, SequenceOffsetType> latestOffsetsFromDb;
|
||||
if (hasValidOffsetsFromDb) {
|
||||
latestOffsetsFromDb = latestDataSourceMetadata.getSeekableStreamPartitions().getPartitionSequenceNumberMap();
|
||||
latestOffsetsFromDb = latestDataSourceMetadata.getSeekableStreamSequenceNumbers().getPartitionSequenceNumberMap();
|
||||
} else {
|
||||
latestOffsetsFromDb = null;
|
||||
}
|
||||
|
@ -1647,7 +1649,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
} else {
|
||||
return generateSequenceName(
|
||||
task.getIOConfig()
|
||||
.getStartPartitions()
|
||||
.getStartSequenceNumbers()
|
||||
.getPartitionSequenceNumberMap(),
|
||||
task.getIOConfig().getMinimumMessageTime(),
|
||||
task.getIOConfig().getMaximumMessageTime()
|
||||
|
@ -2246,17 +2248,18 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
) : Optional.absent());
|
||||
|
||||
|
||||
Map<PartitionIdType, OrderedSequenceNumber<SequenceOffsetType>> startingOffsets = generateStartingSequencesForPartitionGroup(
|
||||
groupId);
|
||||
final Map<PartitionIdType, OrderedSequenceNumber<SequenceOffsetType>> startingOffsets =
|
||||
generateStartingSequencesForPartitionGroup(groupId);
|
||||
|
||||
ImmutableMap<PartitionIdType, SequenceOffsetType> simpleStartingOffsets = startingOffsets
|
||||
.entrySet()
|
||||
.stream()
|
||||
.filter(x -> x.getValue().get() != null)
|
||||
.collect(Collectors.collectingAndThen(
|
||||
Collectors.toMap(Entry::getKey, x -> x.getValue().get()),
|
||||
ImmutableMap::copyOf
|
||||
));
|
||||
.collect(
|
||||
Collectors.collectingAndThen(
|
||||
Collectors.toMap(Entry::getKey, x -> x.getValue().get()), ImmutableMap::copyOf
|
||||
)
|
||||
);
|
||||
|
||||
Set<PartitionIdType> exclusiveStartSequenceNumberPartitions = !useExclusiveStartingSequence
|
||||
? Collections.emptySet()
|
||||
|
@ -2344,7 +2347,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
if (!getNotSetMarker().equals(sequence)) {
|
||||
// if we are given a startingOffset (set by a previous task group which is pending completion) then use it
|
||||
if (!isEndOfShard(sequence)) {
|
||||
builder.put(partition, makeSequenceNumber(sequence, false));
|
||||
builder.put(partition, makeSequenceNumber(sequence, useExclusiveStartSequenceNumberForStartSequence()));
|
||||
}
|
||||
} else {
|
||||
// if we don't have a startingOffset (first run or we had some previous failures and reset the sequences) then
|
||||
|
@ -2373,7 +2376,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
if (!checkSequenceAvailability(partition, sequence)) {
|
||||
if (taskTuningConfig.isResetOffsetAutomatically()) {
|
||||
resetInternal(
|
||||
createDataSourceMetaData(ioConfig.getStream(), ImmutableMap.of(partition, sequence))
|
||||
createDataSourceMetaDataForReset(ioConfig.getStream(), ImmutableMap.of(partition, sequence))
|
||||
);
|
||||
throw new ISE(
|
||||
"Previous sequenceNumber [%s] is no longer available for partition [%s] - automatically resetting sequence",
|
||||
|
@ -2390,7 +2393,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
}
|
||||
}
|
||||
}
|
||||
return makeSequenceNumber(sequence, true);
|
||||
return makeSequenceNumber(sequence, useExclusiveStartSequenceNumberForStartSequence());
|
||||
} else {
|
||||
boolean useEarliestSequenceNumber = ioConfig.isUseEarliestSequenceNumber();
|
||||
if (subsequentlyDiscoveredPartitions.contains(partition)) {
|
||||
|
@ -2416,8 +2419,8 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
if (dataSourceMetadata instanceof SeekableStreamDataSourceMetadata
|
||||
&& checkSourceMetadataMatch(dataSourceMetadata)) {
|
||||
@SuppressWarnings("unchecked")
|
||||
SeekableStreamPartitions<PartitionIdType, SequenceOffsetType> partitions = ((SeekableStreamDataSourceMetadata) dataSourceMetadata)
|
||||
.getSeekableStreamPartitions();
|
||||
SeekableStreamSequenceNumbers<PartitionIdType, SequenceOffsetType> partitions = ((SeekableStreamDataSourceMetadata) dataSourceMetadata)
|
||||
.getSeekableStreamSequenceNumbers();
|
||||
if (partitions != null) {
|
||||
if (!ioConfig.getStream().equals(partitions.getStream())) {
|
||||
log.warn(
|
||||
|
@ -2639,11 +2642,11 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
Preconditions.checkArgument(
|
||||
spec.getIoConfig()
|
||||
.getStream()
|
||||
.equals(((SeekableStreamDataSourceMetadata) currentCheckPoint).getSeekableStreamPartitions()
|
||||
.equals(((SeekableStreamDataSourceMetadata) currentCheckPoint).getSeekableStreamSequenceNumbers()
|
||||
.getStream()),
|
||||
"Supervisor stream [%s] and stream in checkpoint [%s] does not match",
|
||||
spec.getIoConfig().getStream(),
|
||||
((SeekableStreamDataSourceMetadata) currentCheckPoint).getSeekableStreamPartitions().getStream()
|
||||
((SeekableStreamDataSourceMetadata) currentCheckPoint).getSeekableStreamSequenceNumbers().getStream()
|
||||
);
|
||||
|
||||
log.info("Checkpointing [%s] for taskGroup [%s]", currentCheckPoint, taskGroupId);
|
||||
|
@ -2722,14 +2725,14 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
protected abstract boolean doesTaskTypeMatchSupervisor(Task task);
|
||||
|
||||
/**
|
||||
* creates a specific instance of kafka/kinesis datasource metadata
|
||||
* creates a specific instance of kafka/kinesis datasource metadata. Only used for reset.
|
||||
*
|
||||
* @param stream stream name
|
||||
* @param map partitionId -> sequence
|
||||
*
|
||||
* @return specific instance of datasource metadata
|
||||
*/
|
||||
protected abstract SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType> createDataSourceMetaData(
|
||||
protected abstract SeekableStreamDataSourceMetadata<PartitionIdType, SequenceOffsetType> createDataSourceMetaDataForReset(
|
||||
String stream,
|
||||
Map<PartitionIdType, SequenceOffsetType> map
|
||||
);
|
||||
|
@ -2818,4 +2821,11 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
* checks if seqNum marks the end of a Kinesis shard. Used by Kinesis only.
|
||||
*/
|
||||
protected abstract boolean isEndOfShard(SequenceOffsetType seqNum);
|
||||
|
||||
/**
|
||||
* Returns true if the start sequence number should be exclusive for the non-first sequences for the whole partition.
|
||||
* For example, in Kinesis, the start offsets are inclusive for the first sequence, but exclusive for following
|
||||
* sequences. In Kafka, start offsets are always inclusive.
|
||||
*/
|
||||
protected abstract boolean useExclusiveStartSequenceNumberForStartSequence();
|
||||
}
|
||||
|
|
|
@ -22,6 +22,7 @@ package org.apache.druid.indexing.seekablestream;
|
|||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import org.apache.druid.java.util.common.jackson.JacksonUtils;
|
||||
import org.apache.druid.segment.TestHelper;
|
||||
import org.junit.Assert;
|
||||
|
@ -29,7 +30,7 @@ import org.junit.Test;
|
|||
|
||||
import java.util.Map;
|
||||
|
||||
public class SeekableStreamPartitionsTest
|
||||
public class SeekableStreamEndSequenceNumbersTest
|
||||
{
|
||||
private static final ObjectMapper OBJECT_MAPPER = TestHelper.makeJsonMapper();
|
||||
|
||||
|
@ -39,13 +40,16 @@ public class SeekableStreamPartitionsTest
|
|||
final String stream = "theStream";
|
||||
final Map<Integer, Long> offsetMap = ImmutableMap.of(1, 2L, 3, 4L);
|
||||
|
||||
final SeekableStreamPartitions<Integer, Long> partitions = new SeekableStreamPartitions<>(stream, offsetMap);
|
||||
final SeekableStreamEndSequenceNumbers<Integer, Long> partitions = new SeekableStreamEndSequenceNumbers<>(
|
||||
stream,
|
||||
offsetMap
|
||||
);
|
||||
final String serializedString = OBJECT_MAPPER.writeValueAsString(partitions);
|
||||
|
||||
// Check round-trip.
|
||||
final SeekableStreamPartitions<Integer, Long> partitions2 = OBJECT_MAPPER.readValue(
|
||||
final SeekableStreamEndSequenceNumbers<Integer, Long> partitions2 = OBJECT_MAPPER.readValue(
|
||||
serializedString,
|
||||
new TypeReference<SeekableStreamPartitions<Integer, Long>>() {}
|
||||
new TypeReference<SeekableStreamEndSequenceNumbers<Integer, Long>>() {}
|
||||
);
|
||||
|
||||
Assert.assertEquals("Round trip", partitions, partitions2);
|
||||
|
@ -69,4 +73,26 @@ public class SeekableStreamPartitionsTest
|
|||
OBJECT_MAPPER.convertValue(asMap.get("partitionOffsetMap"), new TypeReference<Map<Integer, Long>>() {})
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConvertToStart()
|
||||
{
|
||||
final String stream = "topic";
|
||||
final Map<Integer, Long> offsetMap = ImmutableMap.of(1, 2L, 3, 4L);
|
||||
|
||||
final SeekableStreamEndSequenceNumbers<Integer, Long> endSequenceNumbers = new SeekableStreamEndSequenceNumbers<>(
|
||||
stream,
|
||||
offsetMap
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
new SeekableStreamStartSequenceNumbers<>(stream, offsetMap, ImmutableSet.of(1, 3)),
|
||||
endSequenceNumbers.asStartPartitions(false)
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
new SeekableStreamStartSequenceNumbers<>(stream, offsetMap, ImmutableSet.of()),
|
||||
endSequenceNumbers.asStartPartitions(true)
|
||||
);
|
||||
}
|
||||
}
|
|
@ -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
|
||||
|
|
|
@ -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)
|
||||
{
|
||||
|
|
|
@ -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);
|
||||
|
|
Loading…
Reference in New Issue