Add support to read from multiple kafka topics in same supervisor (#14424)

This PR adds support to read from multiple Kafka topics in the same supervisor. A multi-topic ingestion can be useful in scenarios where a cluster admin has no control over input streams. Different teams in an org may create different input topics that they can write the data to. However, the cluster admin wants all this data to be queryable in one data source.
This commit is contained in:
Abhishek Agarwal 2023-08-14 22:24:49 +05:30 committed by GitHub
parent e16096735b
commit 30b5dd4ca7
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
22 changed files with 1249 additions and 879 deletions

View File

@ -53,6 +53,7 @@ This topic contains configuration reference information for the Apache Kafka sup
|`earlyMessageRejectionPeriod`|ISO8601 Period|Configure tasks to reject messages with timestamps later than this period after the task reached its taskDuration; for example if this is set to `PT1H`, the taskDuration is set to `PT1H` and the supervisor creates a task at *2016-01-01T12:00Z*, messages with timestamps later than *2016-01-01T14:00Z* will be dropped. **Note:** Tasks sometimes run past their task duration, for example, in cases of supervisor failover. Setting earlyMessageRejectionPeriod too low may cause messages to be dropped unexpectedly whenever a task runs past its originally configured task duration.|no (default == none)|
|`autoScalerConfig`|Object|Defines auto scaling behavior for Kafka ingest tasks. See [Tasks Autoscaler Properties](#task-autoscaler-properties).|no (default == null)|
|`idleConfig`|Object|Defines how and when Kafka Supervisor can become idle. See [Idle Supervisor Configuration](#idle-supervisor-configuration) for more details.|no (default == null)|
|`multiTopic`|Boolean|Set this to true if you want to ingest data from multiple Kafka topics using a single supervisor. See [Ingesting from multiple topics](#ingesting-from-multiple-topics) for more details.|no (default == false)|
## Task Autoscaler Properties
@ -136,6 +137,12 @@ The following example demonstrates supervisor spec with `lagBased` autoScaler an
}
}
```
## Ingesting from multiple topics
To ingest from multiple topics, you have to set `multiTopic` in the supervisor IO config to `true`. Multiple topics
can be passed as a regex pattern as the value for `topic` in the IO config. For example, to ingest data from clicks and
impressions, you will set `topic` to `clicks|impressions` in the IO config. If new topics are added to the cluster that
match the regex, druid will automatically start ingesting from those new topics. If you enable multi-topic
ingestion for a datasource, downgrading will cause the ingestion to fail for that datasource.
## More on consumerProperties

View File

@ -0,0 +1,214 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.data.input.kafka;
import com.fasterxml.jackson.core.JsonGenerator;
import com.fasterxml.jackson.core.JsonParser;
import com.fasterxml.jackson.databind.DeserializationContext;
import com.fasterxml.jackson.databind.JsonDeserializer;
import com.fasterxml.jackson.databind.JsonSerializer;
import com.fasterxml.jackson.databind.KeyDeserializer;
import com.fasterxml.jackson.databind.SerializerProvider;
import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
import com.fasterxml.jackson.databind.annotation.JsonSerialize;
import org.apache.druid.error.DruidException;
import org.apache.kafka.common.TopicPartition;
import javax.annotation.Nullable;
import java.io.IOException;
import java.util.Objects;
import java.util.Optional;
/**
* This class represents the partition id for kafka ingestion. This partition id includes topic name along with an
* integer partition. The topic name is required in multi-topic mode because the same partition id can be used for
* different topics.
* This class is used as a key in {@link org.apache.druid.indexing.kafka.KafkaDataSourceMetadata} to store the offsets
* for each partition.
*/
@JsonSerialize(using = KafkaTopicPartition.KafkaTopicPartitionSerializer.class, keyUsing =
KafkaTopicPartition.KafkaTopicPartitionKeySerializer.class)
@JsonDeserialize(using = KafkaTopicPartition.KafkaTopicPartitionDeserializer.class, keyUsing =
KafkaTopicPartition.KafkaTopicPartitionKeyDeserializer.class)
public class KafkaTopicPartition
{
private final int partition;
@Nullable
private final String topic;
/**
* This flag is used to maintain backward compatibilty with older versions of kafka indexing. If this flag
* is set to false,
* - KafkaTopicPartition will be serialized as an integer and can be read back by older version.
* - topic field is ensured to be null.
* This flag must be explicitly passed while constructing KafkaTopicPartition object. That way, we can ensure that
* a particular supervisor is always running in multi topic mode or single topic mode.
*/
private final boolean multiTopicPartition;
public KafkaTopicPartition(boolean multiTopicPartition, @Nullable String topic, int partition)
{
this.partition = partition;
this.multiTopicPartition = multiTopicPartition;
if (multiTopicPartition) {
if (topic == null) {
throw DruidException.defensive("the topic cannot be null in multi-topic mode of kafka ingestion");
}
this.topic = topic;
} else {
this.topic = null;
}
}
public int partition()
{
return partition;
}
public Optional<String> topic()
{
return Optional.ofNullable(topic);
}
public boolean isMultiTopicPartition()
{
return multiTopicPartition;
}
/**
* A utility method to convert KafkaTopicPartition to {@link TopicPartition} object. For single topic ingestion,
* the fallback topic is used to populate the topic name in {@link TopicPartition} object.
*/
public TopicPartition asTopicPartition(String fallbackTopic)
{
return new TopicPartition(topic != null ? topic : fallbackTopic, partition);
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
KafkaTopicPartition that = (KafkaTopicPartition) o;
return partition == that.partition && multiTopicPartition == that.multiTopicPartition && Objects.equals(
topic,
that.topic
);
}
@Override
public int hashCode()
{
return Objects.hash(partition, multiTopicPartition, topic);
}
@Override
public String toString()
{
return "KafkaTopicPartition{" +
"partition=" + partition +
", topic='" + topic + '\'' +
", multiTopicPartition=" + multiTopicPartition +
'}';
}
public static class KafkaTopicPartitionDeserializer extends JsonDeserializer<KafkaTopicPartition>
{
@Override
public KafkaTopicPartition deserialize(JsonParser p, DeserializationContext ctxt)
throws IOException
{
return fromString(p.getValueAsString());
}
@Override
public Class<KafkaTopicPartition> handledType()
{
return KafkaTopicPartition.class;
}
}
public static class KafkaTopicPartitionSerializer extends JsonSerializer<KafkaTopicPartition>
{
@Override
public void serialize(KafkaTopicPartition value, JsonGenerator gen, SerializerProvider serializers)
throws IOException
{
if (null != value.topic && value.multiTopicPartition) {
gen.writeString(value.topic + ":" + value.partition);
} else {
gen.writeString(String.valueOf(value.partition));
}
}
@Override
public Class<KafkaTopicPartition> handledType()
{
return KafkaTopicPartition.class;
}
}
public static class KafkaTopicPartitionKeySerializer extends JsonSerializer<KafkaTopicPartition>
{
@Override
public void serialize(KafkaTopicPartition value, JsonGenerator gen, SerializerProvider serializers)
throws IOException
{
if (null != value.topic && value.multiTopicPartition) {
gen.writeFieldName(value.topic + ":" + value.partition);
} else {
gen.writeFieldName(String.valueOf(value.partition));
}
}
@Override
public Class<KafkaTopicPartition> handledType()
{
return KafkaTopicPartition.class;
}
}
public static class KafkaTopicPartitionKeyDeserializer extends KeyDeserializer
{
@Override
public KafkaTopicPartition deserializeKey(String key, DeserializationContext ctxt)
{
return fromString(key);
}
}
public static KafkaTopicPartition fromString(String str)
{
int index = str.lastIndexOf(':');
if (index < 0) {
return new KafkaTopicPartition(false, null, Integer.parseInt(str));
} else {
return new KafkaTopicPartition(
true,
str.substring(0, index),
Integer.parseInt(str.substring(index + 1))
);
}
}
}

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.druid.data.input.impl.InputRowParser;
import org.apache.druid.data.input.kafka.KafkaRecordEntity;
import org.apache.druid.data.input.kafka.KafkaTopicPartition;
import org.apache.druid.indexing.common.LockGranularity;
import org.apache.druid.indexing.common.TaskToolbox;
import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata;
@ -57,7 +58,7 @@ import java.util.concurrent.TimeUnit;
/**
* Kafka indexing task runner supporting incremental segments publishing
*/
public class IncrementalPublishingKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner<Integer, Long, KafkaRecordEntity>
public class IncrementalPublishingKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner<KafkaTopicPartition, Long, KafkaRecordEntity>
{
private static final EmittingLogger log = new EmittingLogger(IncrementalPublishingKafkaIndexTaskRunner.class);
private final KafkaIndexTask task;
@ -86,8 +87,8 @@ public class IncrementalPublishingKafkaIndexTaskRunner extends SeekableStreamInd
@Nonnull
@Override
protected List<OrderedPartitionableRecord<Integer, Long, KafkaRecordEntity>> getRecords(
RecordSupplier<Integer, Long, KafkaRecordEntity> recordSupplier,
protected List<OrderedPartitionableRecord<KafkaTopicPartition, Long, KafkaRecordEntity>> getRecords(
RecordSupplier<KafkaTopicPartition, Long, KafkaRecordEntity> recordSupplier,
TaskToolbox toolbox
) throws Exception
{
@ -107,7 +108,7 @@ public class IncrementalPublishingKafkaIndexTaskRunner extends SeekableStreamInd
}
@Override
protected SeekableStreamEndSequenceNumbers<Integer, Long> deserializePartitionsFromMetadata(
protected SeekableStreamEndSequenceNumbers<KafkaTopicPartition, Long> deserializePartitionsFromMetadata(
ObjectMapper mapper,
Object object
)
@ -115,17 +116,19 @@ public class IncrementalPublishingKafkaIndexTaskRunner extends SeekableStreamInd
return mapper.convertValue(object, mapper.getTypeFactory().constructParametrizedType(
SeekableStreamEndSequenceNumbers.class,
SeekableStreamEndSequenceNumbers.class,
Integer.class,
KafkaTopicPartition.class,
Long.class
));
}
private void possiblyResetOffsetsOrWait(
Map<TopicPartition, Long> outOfRangePartitions,
RecordSupplier<Integer, Long, KafkaRecordEntity> recordSupplier,
RecordSupplier<KafkaTopicPartition, Long, KafkaRecordEntity> recordSupplier,
TaskToolbox taskToolbox
) throws InterruptedException, IOException
{
final String stream = task.getIOConfig().getStartSequenceNumbers().getStream();
final boolean isMultiTopic = task.getIOConfig().isMultiTopic();
final Map<TopicPartition, Long> resetPartitions = new HashMap<>();
boolean doReset = false;
if (task.getTuningConfig().isResetOffsetAutomatically()) {
@ -133,9 +136,9 @@ public class IncrementalPublishingKafkaIndexTaskRunner extends SeekableStreamInd
final TopicPartition topicPartition = outOfRangePartition.getKey();
final long nextOffset = outOfRangePartition.getValue();
// seek to the beginning to get the least available offset
StreamPartition<Integer> streamPartition = StreamPartition.of(
topicPartition.topic(),
topicPartition.partition()
StreamPartition<KafkaTopicPartition> streamPartition = StreamPartition.of(
stream,
new KafkaTopicPartition(isMultiTopic, topicPartition.topic(), topicPartition.partition())
);
final Long leastAvailableOffset = recordSupplier.getEarliestSequenceNumber(streamPartition);
if (leastAvailableOffset == null) {
@ -157,9 +160,9 @@ public class IncrementalPublishingKafkaIndexTaskRunner extends SeekableStreamInd
}
if (doReset) {
sendResetRequestAndWait(CollectionUtils.mapKeys(resetPartitions, streamPartition -> StreamPartition.of(
streamPartition.topic(),
streamPartition.partition()
sendResetRequestAndWait(CollectionUtils.mapKeys(resetPartitions, topicPartition -> StreamPartition.of(
stream,
new KafkaTopicPartition(isMultiTopic, topicPartition.topic(), topicPartition.partition())
)), taskToolbox);
} else {
log.warn("Retrying in %dms", task.getPollRetryMs());
@ -177,8 +180,8 @@ public class IncrementalPublishingKafkaIndexTaskRunner extends SeekableStreamInd
}
@Override
protected SeekableStreamDataSourceMetadata<Integer, Long> createDataSourceMetadata(
SeekableStreamSequenceNumbers<Integer, Long> partitions
protected SeekableStreamDataSourceMetadata<KafkaTopicPartition, Long> createDataSourceMetadata(
SeekableStreamSequenceNumbers<KafkaTopicPartition, Long> partitions
)
{
return new KafkaDataSourceMetadata(partitions);
@ -193,8 +196,8 @@ public class IncrementalPublishingKafkaIndexTaskRunner extends SeekableStreamInd
@Override
protected void possiblyResetDataSourceMetadata(
TaskToolbox toolbox,
RecordSupplier<Integer, Long, KafkaRecordEntity> recordSupplier,
Set<StreamPartition<Integer>> assignment
RecordSupplier<KafkaTopicPartition, Long, KafkaRecordEntity> recordSupplier,
Set<StreamPartition<KafkaTopicPartition>> assignment
)
{
// do nothing
@ -213,16 +216,16 @@ public class IncrementalPublishingKafkaIndexTaskRunner extends SeekableStreamInd
}
@Override
public TypeReference<List<SequenceMetadata<Integer, Long>>> getSequenceMetadataTypeReference()
public TypeReference<List<SequenceMetadata<KafkaTopicPartition, Long>>> getSequenceMetadataTypeReference()
{
return new TypeReference<List<SequenceMetadata<Integer, Long>>>()
return new TypeReference<List<SequenceMetadata<KafkaTopicPartition, Long>>>()
{
};
}
@Nullable
@Override
protected TreeMap<Integer, Map<Integer, Long>> getCheckPointsFromContext(
protected TreeMap<Integer, Map<KafkaTopicPartition, Long>> getCheckPointsFromContext(
TaskToolbox toolbox,
String checkpointsString
) throws IOException
@ -231,7 +234,7 @@ public class IncrementalPublishingKafkaIndexTaskRunner extends SeekableStreamInd
log.debug("Got checkpoints from task context[%s].", checkpointsString);
return toolbox.getJsonMapper().readValue(
checkpointsString,
new TypeReference<TreeMap<Integer, Map<Integer, Long>>>()
new TypeReference<TreeMap<Integer, Map<KafkaTopicPartition, Long>>>()
{
}
);

View File

@ -21,17 +21,18 @@ package org.apache.druid.indexing.kafka;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.druid.data.input.kafka.KafkaTopicPartition;
import org.apache.druid.indexing.overlord.DataSourceMetadata;
import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata;
import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers;
import org.apache.druid.indexing.seekablestream.SeekableStreamSequenceNumbers;
public class KafkaDataSourceMetadata extends SeekableStreamDataSourceMetadata<Integer, Long>
public class KafkaDataSourceMetadata extends SeekableStreamDataSourceMetadata<KafkaTopicPartition, Long>
{
@JsonCreator
public KafkaDataSourceMetadata(
@JsonProperty("partitions") SeekableStreamSequenceNumbers<Integer, Long> kafkaPartitions
@JsonProperty("partitions") SeekableStreamSequenceNumbers<KafkaTopicPartition, Long> kafkaPartitions
)
{
super(kafkaPartitions);
@ -40,10 +41,10 @@ public class KafkaDataSourceMetadata extends SeekableStreamDataSourceMetadata<In
@Override
public DataSourceMetadata asStartMetadata()
{
final SeekableStreamSequenceNumbers<Integer, Long> sequenceNumbers = getSeekableStreamSequenceNumbers();
final SeekableStreamSequenceNumbers<KafkaTopicPartition, Long> sequenceNumbers = getSeekableStreamSequenceNumbers();
if (sequenceNumbers instanceof SeekableStreamEndSequenceNumbers) {
return createConcreteDataSourceMetaData(
((SeekableStreamEndSequenceNumbers<Integer, Long>) sequenceNumbers).asStartPartitions(true)
((SeekableStreamEndSequenceNumbers<KafkaTopicPartition, Long>) sequenceNumbers).asStartPartitions(true)
);
} else {
return this;
@ -51,8 +52,8 @@ public class KafkaDataSourceMetadata extends SeekableStreamDataSourceMetadata<In
}
@Override
protected SeekableStreamDataSourceMetadata<Integer, Long> createConcreteDataSourceMetaData(
SeekableStreamSequenceNumbers<Integer, Long> seekableStreamSequenceNumbers
protected SeekableStreamDataSourceMetadata<KafkaTopicPartition, Long> createConcreteDataSourceMetaData(
SeekableStreamSequenceNumbers<KafkaTopicPartition, Long> seekableStreamSequenceNumbers
)
{
return new KafkaDataSourceMetadata(seekableStreamSequenceNumbers);

View File

@ -27,6 +27,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import org.apache.druid.data.input.kafka.KafkaRecordEntity;
import org.apache.druid.data.input.kafka.KafkaTopicPartition;
import org.apache.druid.indexing.common.TaskToolbox;
import org.apache.druid.indexing.common.task.TaskResource;
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask;
@ -43,7 +44,7 @@ import java.util.HashMap;
import java.util.Map;
import java.util.Set;
public class KafkaIndexTask extends SeekableStreamIndexTask<Integer, Long, KafkaRecordEntity>
public class KafkaIndexTask extends SeekableStreamIndexTask<KafkaTopicPartition, Long, KafkaRecordEntity>
{
private static final String TYPE = "index_kafka";
@ -86,7 +87,7 @@ public class KafkaIndexTask extends SeekableStreamIndexTask<Integer, Long, Kafka
}
@Override
protected SeekableStreamIndexTaskRunner<Integer, Long, KafkaRecordEntity> createTaskRunner()
protected SeekableStreamIndexTaskRunner<KafkaTopicPartition, Long, KafkaRecordEntity> createTaskRunner()
{
//noinspection unchecked
return new IncrementalPublishingKafkaIndexTaskRunner(
@ -109,7 +110,8 @@ public class KafkaIndexTask extends SeekableStreamIndexTask<Integer, Long, Kafka
props.put("auto.offset.reset", "none");
final KafkaRecordSupplier recordSupplier =
new KafkaRecordSupplier(props, configMapper, kafkaIndexTaskIOConfig.getConfigOverrides());
new KafkaRecordSupplier(props, configMapper, kafkaIndexTaskIOConfig.getConfigOverrides(),
kafkaIndexTaskIOConfig.isMultiTopic());
if (toolbox.getMonitorScheduler() != null) {
toolbox.getMonitorScheduler().addMonitor(recordSupplier.monitor());

View File

@ -21,6 +21,7 @@ package org.apache.druid.indexing.kafka;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.inject.Inject;
import org.apache.druid.data.input.kafka.KafkaTopicPartition;
import org.apache.druid.guice.LazySingleton;
import org.apache.druid.guice.annotations.EscalatedGlobal;
import org.apache.druid.guice.annotations.Json;
@ -28,7 +29,7 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFac
import org.apache.druid.rpc.ServiceClientFactory;
@LazySingleton
public class KafkaIndexTaskClientFactory extends SeekableStreamIndexTaskClientFactory<Integer, Long>
public class KafkaIndexTaskClientFactory extends SeekableStreamIndexTaskClientFactory<KafkaTopicPartition, Long>
{
@Inject
public KafkaIndexTaskClientFactory(
@ -40,9 +41,9 @@ public class KafkaIndexTaskClientFactory extends SeekableStreamIndexTaskClientFa
}
@Override
public Class<Integer> getPartitionType()
public Class<KafkaTopicPartition> getPartitionType()
{
return Integer.class;
return KafkaTopicPartition.class;
}
@Override

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import org.apache.druid.data.input.InputFormat;
import org.apache.druid.data.input.kafka.KafkaTopicPartition;
import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig;
import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers;
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig;
@ -33,33 +34,36 @@ import org.joda.time.DateTime;
import javax.annotation.Nullable;
import java.util.Map;
public class KafkaIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig<Integer, Long>
public class KafkaIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig<KafkaTopicPartition, Long>
{
private final Map<String, Object> consumerProperties;
private final long pollTimeout;
private final KafkaConfigOverrides configOverrides;
private final boolean multiTopic;
@JsonCreator
public KafkaIndexTaskIOConfig(
@JsonProperty("taskGroupId") @Nullable Integer taskGroupId, // can be null for backward compabitility
@JsonProperty("baseSequenceName") String baseSequenceName,
// startPartitions and endPartitions exist to be able to read old ioConfigs in metadata store
@JsonProperty("startPartitions") @Nullable
@Deprecated SeekableStreamEndSequenceNumbers<Integer, Long> startPartitions,
@Deprecated SeekableStreamEndSequenceNumbers<KafkaTopicPartition, Long> startPartitions,
@JsonProperty("endPartitions") @Nullable
@Deprecated SeekableStreamEndSequenceNumbers<Integer, Long> endPartitions,
@Deprecated SeekableStreamEndSequenceNumbers<KafkaTopicPartition, Long> endPartitions,
// startSequenceNumbers and endSequenceNumbers must be set for new versions
@JsonProperty("startSequenceNumbers")
@Nullable SeekableStreamStartSequenceNumbers<Integer, Long> startSequenceNumbers,
@Nullable SeekableStreamStartSequenceNumbers<KafkaTopicPartition, Long> startSequenceNumbers,
@JsonProperty("endSequenceNumbers")
@Nullable SeekableStreamEndSequenceNumbers<Integer, Long> endSequenceNumbers,
@Nullable SeekableStreamEndSequenceNumbers<KafkaTopicPartition, Long> endSequenceNumbers,
@JsonProperty("consumerProperties") Map<String, Object> consumerProperties,
@JsonProperty("pollTimeout") Long pollTimeout,
@JsonProperty("useTransaction") Boolean useTransaction,
@JsonProperty("minimumMessageTime") DateTime minimumMessageTime,
@JsonProperty("maximumMessageTime") DateTime maximumMessageTime,
@JsonProperty("inputFormat") @Nullable InputFormat inputFormat,
@JsonProperty("configOverrides") @Nullable KafkaConfigOverrides configOverrides
@JsonProperty("configOverrides") @Nullable KafkaConfigOverrides configOverrides,
@JsonProperty("multiTopic") @Nullable Boolean multiTopic
)
{
super(
@ -78,9 +82,10 @@ public class KafkaIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig<Inte
this.consumerProperties = Preconditions.checkNotNull(consumerProperties, "consumerProperties");
this.pollTimeout = pollTimeout != null ? pollTimeout : KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS;
this.configOverrides = configOverrides;
this.multiTopic = multiTopic != null ? multiTopic : KafkaSupervisorIOConfig.DEFAULT_IS_MULTI_TOPIC;
final SeekableStreamEndSequenceNumbers<Integer, Long> myEndSequenceNumbers = getEndSequenceNumbers();
for (int partition : myEndSequenceNumbers.getPartitionSequenceNumberMap().keySet()) {
final SeekableStreamEndSequenceNumbers<KafkaTopicPartition, Long> myEndSequenceNumbers = getEndSequenceNumbers();
for (KafkaTopicPartition partition : myEndSequenceNumbers.getPartitionSequenceNumberMap().keySet()) {
Preconditions.checkArgument(
myEndSequenceNumbers.getPartitionSequenceNumberMap()
.get(partition)
@ -94,8 +99,8 @@ public class KafkaIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig<Inte
public KafkaIndexTaskIOConfig(
int taskGroupId,
String baseSequenceName,
SeekableStreamStartSequenceNumbers<Integer, Long> startSequenceNumbers,
SeekableStreamEndSequenceNumbers<Integer, Long> endSequenceNumbers,
SeekableStreamStartSequenceNumbers<KafkaTopicPartition, Long> startSequenceNumbers,
SeekableStreamEndSequenceNumbers<KafkaTopicPartition, Long> endSequenceNumbers,
Map<String, Object> consumerProperties,
Long pollTimeout,
Boolean useTransaction,
@ -118,7 +123,8 @@ public class KafkaIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig<Inte
minimumMessageTime,
maximumMessageTime,
inputFormat,
configOverrides
configOverrides,
KafkaSupervisorIOConfig.DEFAULT_IS_MULTI_TOPIC
);
}
@ -129,10 +135,10 @@ public class KafkaIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig<Inte
*/
@JsonProperty
@Deprecated
public SeekableStreamEndSequenceNumbers<Integer, Long> getStartPartitions()
public SeekableStreamEndSequenceNumbers<KafkaTopicPartition, 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();
final SeekableStreamStartSequenceNumbers<KafkaTopicPartition, Long> startSequenceNumbers = getStartSequenceNumbers();
return new SeekableStreamEndSequenceNumbers<>(
startSequenceNumbers.getStream(),
startSequenceNumbers.getPartitionSequenceNumberMap()
@ -145,7 +151,7 @@ public class KafkaIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig<Inte
*/
@JsonProperty
@Deprecated
public SeekableStreamEndSequenceNumbers<Integer, Long> getEndPartitions()
public SeekableStreamEndSequenceNumbers<KafkaTopicPartition, Long> getEndPartitions()
{
return getEndSequenceNumbers();
}
@ -168,6 +174,12 @@ public class KafkaIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig<Inte
return configOverrides;
}
@JsonProperty
public boolean isMultiTopic()
{
return multiTopic;
}
@Override
public String toString()
{

View File

@ -24,6 +24,7 @@ import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.google.common.collect.ImmutableList;
import com.google.inject.Binder;
import org.apache.druid.data.input.kafka.KafkaTopicPartition;
import org.apache.druid.data.input.kafkainput.KafkaInputFormat;
import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpec;
import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorTuningConfig;
@ -52,6 +53,7 @@ public class KafkaIndexTaskModule implements DruidModule
new NamedType(KafkaSamplerSpec.class, SCHEME),
new NamedType(KafkaInputFormat.class, SCHEME)
)
.addKeySerializer(KafkaTopicPartition.class, new KafkaTopicPartition.KafkaTopicPartitionKeySerializer())
);
}

View File

@ -24,6 +24,9 @@ import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.ImmutableList;
import org.apache.druid.common.utils.IdUtils;
import org.apache.druid.data.input.kafka.KafkaRecordEntity;
import org.apache.druid.data.input.kafka.KafkaTopicPartition;
import org.apache.druid.error.DruidException;
import org.apache.druid.error.InvalidInput;
import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig;
import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord;
import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber;
@ -31,7 +34,6 @@ import org.apache.druid.indexing.seekablestream.common.RecordSupplier;
import org.apache.druid.indexing.seekablestream.common.StreamException;
import org.apache.druid.indexing.seekablestream.common.StreamPartition;
import org.apache.druid.indexing.seekablestream.extension.KafkaConfigOverrides;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.metrics.Monitor;
import org.apache.druid.metadata.DynamicConfigProvider;
@ -39,7 +41,6 @@ import org.apache.druid.metadata.PasswordProvider;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.serialization.ByteArrayDeserializer;
import org.apache.kafka.common.serialization.Deserializer;
@ -56,87 +57,114 @@ import java.util.Map;
import java.util.Properties;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
public class KafkaRecordSupplier implements RecordSupplier<Integer, Long, KafkaRecordEntity>
public class KafkaRecordSupplier implements RecordSupplier<KafkaTopicPartition, Long, KafkaRecordEntity>
{
private final KafkaConsumer<byte[], byte[]> consumer;
private final KafkaConsumerMonitor monitor;
private boolean closed;
private final boolean multiTopic;
/**
* Store the stream information when partitions get assigned. This is required because the consumer does not
* know about the parent stream which could be a list of topics.
*/
private String stream;
public KafkaRecordSupplier(
Map<String, Object> consumerProperties,
ObjectMapper sortingMapper,
KafkaConfigOverrides configOverrides
KafkaConfigOverrides configOverrides,
boolean multiTopic
)
{
this(getKafkaConsumer(sortingMapper, consumerProperties, configOverrides));
this(getKafkaConsumer(sortingMapper, consumerProperties, configOverrides), multiTopic);
}
@VisibleForTesting
public KafkaRecordSupplier(
KafkaConsumer<byte[], byte[]> consumer
KafkaConsumer<byte[], byte[]> consumer,
boolean multiTopic
)
{
this.consumer = consumer;
this.multiTopic = multiTopic;
this.monitor = new KafkaConsumerMonitor(consumer);
}
@Override
public void assign(Set<StreamPartition<Integer>> streamPartitions)
public void assign(Set<StreamPartition<KafkaTopicPartition>> streamPartitions)
{
if (streamPartitions.isEmpty()) {
wrapExceptions(() -> consumer.assign(Collections.emptyList()));
return;
}
Set<String> streams = streamPartitions.stream().map(StreamPartition::getStream).collect(Collectors.toSet());
if (streams.size() > 1) {
throw InvalidInput.exception("[%s] streams found. Only one stream is supported.", streams);
}
this.stream = streams.iterator().next();
wrapExceptions(() -> consumer.assign(streamPartitions
.stream()
.map(x -> new TopicPartition(x.getStream(), x.getPartitionId()))
.map(x -> x.getPartitionId().asTopicPartition(x.getStream()))
.collect(Collectors.toSet())));
}
@Override
public void seek(StreamPartition<Integer> partition, Long sequenceNumber)
public void seek(StreamPartition<KafkaTopicPartition> partition, Long sequenceNumber)
{
wrapExceptions(() -> consumer.seek(
new TopicPartition(partition.getStream(), partition.getPartitionId()),
partition.getPartitionId().asTopicPartition(partition.getStream()),
sequenceNumber
));
}
@Override
public void seekToEarliest(Set<StreamPartition<Integer>> partitions)
public void seekToEarliest(Set<StreamPartition<KafkaTopicPartition>> partitions)
{
wrapExceptions(() -> consumer.seekToBeginning(partitions
.stream()
.map(e -> new TopicPartition(e.getStream(), e.getPartitionId()))
.map(e -> e.getPartitionId().asTopicPartition(e.getStream()))
.collect(Collectors.toList())));
}
@Override
public void seekToLatest(Set<StreamPartition<Integer>> partitions)
public void seekToLatest(Set<StreamPartition<KafkaTopicPartition>> partitions)
{
wrapExceptions(() -> consumer.seekToEnd(partitions
.stream()
.map(e -> new TopicPartition(e.getStream(), e.getPartitionId()))
.map(e -> e.getPartitionId().asTopicPartition(e.getStream()))
.collect(Collectors.toList())));
}
@Override
public Set<StreamPartition<Integer>> getAssignment()
public Set<StreamPartition<KafkaTopicPartition>> getAssignment()
{
return wrapExceptions(() -> consumer.assignment()
.stream()
.map(e -> new StreamPartition<>(e.topic(), e.partition()))
.map(e -> new StreamPartition<>(
stream,
new KafkaTopicPartition(multiTopic, e.topic(),
e.partition()
)
))
.collect(Collectors.toSet()));
}
@Nonnull
@Override
public List<OrderedPartitionableRecord<Integer, Long, KafkaRecordEntity>> poll(long timeout)
public List<OrderedPartitionableRecord<KafkaTopicPartition, Long, KafkaRecordEntity>> poll(long timeout)
{
List<OrderedPartitionableRecord<Integer, Long, KafkaRecordEntity>> polledRecords = new ArrayList<>();
List<OrderedPartitionableRecord<KafkaTopicPartition, Long, KafkaRecordEntity>> polledRecords = new ArrayList<>();
for (ConsumerRecord<byte[], byte[]> record : consumer.poll(Duration.ofMillis(timeout))) {
polledRecords.add(new OrderedPartitionableRecord<>(
record.topic(),
record.partition(),
new KafkaTopicPartition(multiTopic, record.topic(), record.partition()),
record.offset(),
record.value() == null ? null : ImmutableList.of(new KafkaRecordEntity(record))
));
@ -145,7 +173,7 @@ public class KafkaRecordSupplier implements RecordSupplier<Integer, Long, KafkaR
}
@Override
public Long getLatestSequenceNumber(StreamPartition<Integer> partition)
public Long getLatestSequenceNumber(StreamPartition<KafkaTopicPartition> partition)
{
Long currPos = getPosition(partition);
seekToLatest(Collections.singleton(partition));
@ -155,7 +183,7 @@ public class KafkaRecordSupplier implements RecordSupplier<Integer, Long, KafkaR
}
@Override
public Long getEarliestSequenceNumber(StreamPartition<Integer> partition)
public Long getEarliestSequenceNumber(StreamPartition<KafkaTopicPartition> partition)
{
Long currPos = getPosition(partition);
seekToEarliest(Collections.singleton(partition));
@ -165,7 +193,7 @@ public class KafkaRecordSupplier implements RecordSupplier<Integer, Long, KafkaR
}
@Override
public boolean isOffsetAvailable(StreamPartition<Integer> partition, OrderedSequenceNumber<Long> offset)
public boolean isOffsetAvailable(StreamPartition<KafkaTopicPartition> partition, OrderedSequenceNumber<Long> offset)
{
final Long earliestOffset = getEarliestSequenceNumber(partition);
return earliestOffset != null
@ -173,23 +201,42 @@ public class KafkaRecordSupplier implements RecordSupplier<Integer, Long, KafkaR
}
@Override
public Long getPosition(StreamPartition<Integer> partition)
public Long getPosition(StreamPartition<KafkaTopicPartition> partition)
{
return wrapExceptions(() -> consumer.position(new TopicPartition(
partition.getStream(),
partition.getPartitionId()
)));
return wrapExceptions(() -> consumer.position(partition.getPartitionId().asTopicPartition(partition.getStream())));
}
@Override
public Set<Integer> getPartitionIds(String stream)
public Set<KafkaTopicPartition> getPartitionIds(String stream)
{
return wrapExceptions(() -> {
List<PartitionInfo> partitions = consumer.partitionsFor(stream);
if (partitions == null) {
throw new ISE("Topic [%s] is not found in KafkaConsumer's list of topics", stream);
List<PartitionInfo> allPartitions;
if (multiTopic) {
Pattern pattern = Pattern.compile(stream);
allPartitions = consumer.listTopics()
.entrySet()
.stream()
.filter(e -> pattern.matcher(e.getKey()).matches())
.flatMap(e -> e.getValue().stream())
.collect(Collectors.toList());
if (allPartitions.isEmpty()) {
throw DruidException.forPersona(DruidException.Persona.OPERATOR)
.ofCategory(DruidException.Category.INVALID_INPUT)
.build("No partitions found for topics that match given pattern [%s]."
+ "Check that the pattern regex is correct and matching topics exists", stream);
}
} else {
allPartitions = consumer.partitionsFor(stream);
if (allPartitions == null) {
throw DruidException.forPersona(DruidException.Persona.OPERATOR)
.ofCategory(DruidException.Category.INVALID_INPUT)
.build("Topic [%s] is not found."
+ "Check that the topic exists in Kafka cluster", stream);
}
}
return partitions.stream().map(PartitionInfo::partition).collect(Collectors.toSet());
return allPartitions.stream()
.map(p -> new KafkaTopicPartition(multiTopic, p.topic(), p.partition()))
.collect(Collectors.toSet());
});
}

View File

@ -71,8 +71,11 @@ public class KafkaSamplerSpec extends SeekableStreamSamplerSpec
props.put("enable.auto.commit", "false");
props.put("auto.offset.reset", "none");
props.put("request.timeout.ms", Integer.toString(samplerConfig.getTimeoutMs()));
KafkaSupervisorIOConfig kafkaSupervisorIOConfig = (KafkaSupervisorIOConfig) ioConfig;
return new KafkaRecordSupplier(props, objectMapper, ((KafkaSupervisorIOConfig) ioConfig).getConfigOverrides());
return new KafkaRecordSupplier(props, objectMapper, kafkaSupervisorIOConfig.getConfigOverrides(),
kafkaSupervisorIOConfig.isMultiTopic()
);
}
finally {
Thread.currentThread().setContextClassLoader(currCtxCl);

View File

@ -25,6 +25,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.annotations.VisibleForTesting;
import org.apache.druid.common.utils.IdUtils;
import org.apache.druid.data.input.kafka.KafkaRecordEntity;
import org.apache.druid.data.input.kafka.KafkaTopicPartition;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.common.task.TaskResource;
import org.apache.druid.indexing.kafka.KafkaDataSourceMetadata;
@ -59,7 +60,6 @@ import org.apache.druid.server.metrics.DruidMonitorSchedulerConfig;
import org.joda.time.DateTime;
import javax.annotation.Nullable;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
@ -79,10 +79,10 @@ import java.util.stream.Collectors;
* tasks to satisfy the desired number of replicas. As tasks complete, new tasks are queued to process the next range of
* Kafka offsets.
*/
public class KafkaSupervisor extends SeekableStreamSupervisor<Integer, Long, KafkaRecordEntity>
public class KafkaSupervisor extends SeekableStreamSupervisor<KafkaTopicPartition, Long, KafkaRecordEntity>
{
public static final TypeReference<TreeMap<Integer, Map<Integer, Long>>> CHECKPOINTS_TYPE_REF =
new TypeReference<TreeMap<Integer, Map<Integer, Long>>>()
public static final TypeReference<TreeMap<Integer, Map<KafkaTopicPartition, Long>>> CHECKPOINTS_TYPE_REF =
new TypeReference<TreeMap<Integer, Map<KafkaTopicPartition, Long>>>()
{
};
@ -92,7 +92,7 @@ public class KafkaSupervisor extends SeekableStreamSupervisor<Integer, Long, Kaf
private final ServiceEmitter emitter;
private final DruidMonitorSchedulerConfig monitorSchedulerConfig;
private volatile Map<Integer, Long> latestSequenceFromStream;
private volatile Map<KafkaTopicPartition, Long> latestSequenceFromStream;
private final KafkaSupervisorSpec spec;
@ -126,19 +126,25 @@ public class KafkaSupervisor extends SeekableStreamSupervisor<Integer, Long, Kaf
@Override
protected RecordSupplier<Integer, Long, KafkaRecordEntity> setupRecordSupplier()
protected RecordSupplier<KafkaTopicPartition, Long, KafkaRecordEntity> setupRecordSupplier()
{
return new KafkaRecordSupplier(
spec.getIoConfig().getConsumerProperties(),
sortingMapper,
spec.getIoConfig().getConfigOverrides()
spec.getIoConfig().getConfigOverrides(),
spec.getIoConfig().isMultiTopic()
);
}
@Override
protected int getTaskGroupIdForPartition(Integer partitionId)
protected int getTaskGroupIdForPartition(KafkaTopicPartition partitionId)
{
return partitionId % spec.getIoConfig().getTaskCount();
Integer taskCount = spec.getIoConfig().getTaskCount();
if (partitionId.isMultiTopicPartition()) {
return Math.abs(31 * partitionId.topic().hashCode() + partitionId.partition()) % taskCount;
} else {
return partitionId.partition() % taskCount;
}
}
@Override
@ -154,13 +160,13 @@ public class KafkaSupervisor extends SeekableStreamSupervisor<Integer, Long, Kaf
}
@Override
protected SeekableStreamSupervisorReportPayload<Integer, Long> createReportPayload(
protected SeekableStreamSupervisorReportPayload<KafkaTopicPartition, Long> createReportPayload(
int numPartitions,
boolean includeOffsets
)
{
KafkaSupervisorIOConfig ioConfig = spec.getIoConfig();
Map<Integer, Long> partitionLag = getRecordLagPerPartitionInLatestSequences(getHighestCurrentOffsets());
Map<KafkaTopicPartition, Long> partitionLag = getRecordLagPerPartitionInLatestSequences(getHighestCurrentOffsets());
return new KafkaSupervisorReportPayload(
spec.getDataSchema().getDataSource(),
ioConfig.getTopic(),
@ -183,12 +189,12 @@ public class KafkaSupervisor extends SeekableStreamSupervisor<Integer, Long, Kaf
@Override
protected SeekableStreamIndexTaskIOConfig createTaskIoConfig(
int groupId,
Map<Integer, Long> startPartitions,
Map<Integer, Long> endPartitions,
Map<KafkaTopicPartition, Long> startPartitions,
Map<KafkaTopicPartition, Long> endPartitions,
String baseSequenceName,
DateTime minimumMessageTime,
DateTime maximumMessageTime,
Set<Integer> exclusiveStartSequenceNumberPartitions,
Set<KafkaTopicPartition> exclusiveStartSequenceNumberPartitions,
SeekableStreamSupervisorIOConfig ioConfig
)
{
@ -196,6 +202,8 @@ public class KafkaSupervisor extends SeekableStreamSupervisor<Integer, Long, Kaf
return new KafkaIndexTaskIOConfig(
groupId,
baseSequenceName,
null,
null,
new SeekableStreamStartSequenceNumbers<>(kafkaIoConfig.getTopic(), startPartitions, Collections.emptySet()),
new SeekableStreamEndSequenceNumbers<>(kafkaIoConfig.getTopic(), endPartitions),
kafkaIoConfig.getConsumerProperties(),
@ -204,16 +212,17 @@ public class KafkaSupervisor extends SeekableStreamSupervisor<Integer, Long, Kaf
minimumMessageTime,
maximumMessageTime,
ioConfig.getInputFormat(),
kafkaIoConfig.getConfigOverrides()
kafkaIoConfig.getConfigOverrides(),
kafkaIoConfig.isMultiTopic()
);
}
@Override
protected List<SeekableStreamIndexTask<Integer, Long, KafkaRecordEntity>> createIndexTasks(
protected List<SeekableStreamIndexTask<KafkaTopicPartition, Long, KafkaRecordEntity>> createIndexTasks(
int replicas,
String baseSequenceName,
ObjectMapper sortingMapper,
TreeMap<Integer, Map<Integer, Long>> sequenceOffsets,
TreeMap<Integer, Map<KafkaTopicPartition, Long>> sequenceOffsets,
SeekableStreamIndexTaskIOConfig taskIoConfig,
SeekableStreamIndexTaskTuningConfig taskTuningConfig,
RowIngestionMetersFactory rowIngestionMetersFactory
@ -227,7 +236,7 @@ public class KafkaSupervisor extends SeekableStreamSupervisor<Integer, Long, Kaf
// Kafka index task will pick up LegacyKafkaIndexTaskRunner without the below configuration.
context.put("IS_INCREMENTAL_HANDOFF_SUPPORTED", true);
List<SeekableStreamIndexTask<Integer, Long, KafkaRecordEntity>> taskList = new ArrayList<>();
List<SeekableStreamIndexTask<KafkaTopicPartition, Long, KafkaRecordEntity>> taskList = new ArrayList<>();
for (int i = 0; i < replicas; i++) {
String taskId = IdUtils.getRandomIdWithPrefix(baseSequenceName);
taskList.add(new KafkaIndexTask(
@ -244,9 +253,9 @@ public class KafkaSupervisor extends SeekableStreamSupervisor<Integer, Long, Kaf
}
@Override
protected Map<Integer, Long> getPartitionRecordLag()
protected Map<KafkaTopicPartition, Long> getPartitionRecordLag()
{
Map<Integer, Long> highestCurrentOffsets = getHighestCurrentOffsets();
Map<KafkaTopicPartition, Long> highestCurrentOffsets = getHighestCurrentOffsets();
if (latestSequenceFromStream == null) {
return null;
@ -265,7 +274,7 @@ public class KafkaSupervisor extends SeekableStreamSupervisor<Integer, Long, Kaf
@Nullable
@Override
protected Map<Integer, Long> getPartitionTimeLag()
protected Map<KafkaTopicPartition, Long> getPartitionTimeLag()
{
// time lag not currently support with kafka
return null;
@ -274,7 +283,7 @@ public class KafkaSupervisor extends SeekableStreamSupervisor<Integer, Long, Kaf
// suppress use of CollectionUtils.mapValues() since the valueMapper function is dependent on map key here
@SuppressWarnings("SSBasedInspection")
// Used while calculating cummulative lag for entire stream
private Map<Integer, Long> getRecordLagPerPartitionInLatestSequences(Map<Integer, Long> currentOffsets)
private Map<KafkaTopicPartition, Long> getRecordLagPerPartitionInLatestSequences(Map<KafkaTopicPartition, Long> currentOffsets)
{
if (latestSequenceFromStream == null) {
return Collections.emptyMap();
@ -297,7 +306,7 @@ public class KafkaSupervisor extends SeekableStreamSupervisor<Integer, Long, Kaf
// suppress use of CollectionUtils.mapValues() since the valueMapper function is dependent on map key here
@SuppressWarnings("SSBasedInspection")
// Used while generating Supervisor lag reports per task
protected Map<Integer, Long> getRecordLagPerPartition(Map<Integer, Long> currentOffsets)
protected Map<KafkaTopicPartition, Long> getRecordLagPerPartition(Map<KafkaTopicPartition, Long> currentOffsets)
{
if (latestSequenceFromStream == null || currentOffsets == null) {
return Collections.emptyMap();
@ -318,13 +327,13 @@ public class KafkaSupervisor extends SeekableStreamSupervisor<Integer, Long, Kaf
}
@Override
protected Map<Integer, Long> getTimeLagPerPartition(Map<Integer, Long> currentOffsets)
protected Map<KafkaTopicPartition, Long> getTimeLagPerPartition(Map<KafkaTopicPartition, Long> currentOffsets)
{
return null;
}
@Override
protected KafkaDataSourceMetadata createDataSourceMetaDataForReset(String topic, Map<Integer, Long> map)
protected KafkaDataSourceMetadata createDataSourceMetaDataForReset(String topic, Map<KafkaTopicPartition, Long> map)
{
return new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, map));
}
@ -368,7 +377,7 @@ public class KafkaSupervisor extends SeekableStreamSupervisor<Integer, Long, Kaf
@Override
public LagStats computeLagStats()
{
Map<Integer, Long> partitionRecordLag = getPartitionRecordLag();
Map<KafkaTopicPartition, Long> partitionRecordLag = getPartitionRecordLag();
if (partitionRecordLag == null) {
return new LagStats(0, 0, 0);
}
@ -381,7 +390,7 @@ public class KafkaSupervisor extends SeekableStreamSupervisor<Integer, Long, Kaf
{
getRecordSupplierLock().lock();
try {
Set<Integer> partitionIds;
Set<KafkaTopicPartition> partitionIds;
try {
partitionIds = recordSupplier.getPartitionIds(getIoConfig().getStream());
}
@ -390,7 +399,7 @@ public class KafkaSupervisor extends SeekableStreamSupervisor<Integer, Long, Kaf
throw new StreamException(e);
}
Set<StreamPartition<Integer>> partitions = partitionIds
Set<StreamPartition<KafkaTopicPartition>> partitions = partitionIds
.stream()
.map(e -> new StreamPartition<>(getIoConfig().getStream(), e))
.collect(Collectors.toSet());
@ -412,7 +421,7 @@ public class KafkaSupervisor extends SeekableStreamSupervisor<Integer, Long, Kaf
}
@Override
protected Map<Integer, Long> getLatestSequencesFromStream()
protected Map<KafkaTopicPartition, Long> getLatestSequencesFromStream()
{
return latestSequenceFromStream != null ? latestSequenceFromStream : new HashMap<>();
}

View File

@ -43,9 +43,12 @@ public class KafkaSupervisorIOConfig extends SeekableStreamSupervisorIOConfig
public static final String KEY_PASSWORD_KEY = "ssl.key.password";
public static final long DEFAULT_POLL_TIMEOUT_MILLIS = 100;
public static final boolean DEFAULT_IS_MULTI_TOPIC = false;
private final Map<String, Object> consumerProperties;
private final long pollTimeout;
private final KafkaConfigOverrides configOverrides;
private final boolean multiTopic;
@JsonCreator
public KafkaSupervisorIOConfig(
@ -66,6 +69,7 @@ public class KafkaSupervisorIOConfig extends SeekableStreamSupervisorIOConfig
@JsonProperty("lateMessageRejectionStartDateTime") DateTime lateMessageRejectionStartDateTime,
@JsonProperty("configOverrides") KafkaConfigOverrides configOverrides,
@JsonProperty("idleConfig") IdleConfig idleConfig,
@JsonProperty("multiTopic") Boolean multiTopic,
@JsonProperty("stopTaskCount") Integer stopTaskCount
)
{
@ -94,6 +98,7 @@ public class KafkaSupervisorIOConfig extends SeekableStreamSupervisorIOConfig
);
this.pollTimeout = pollTimeout != null ? pollTimeout : DEFAULT_POLL_TIMEOUT_MILLIS;
this.configOverrides = configOverrides;
this.multiTopic = multiTopic != null ? multiTopic : DEFAULT_IS_MULTI_TOPIC;
}
@JsonProperty
@ -126,6 +131,12 @@ public class KafkaSupervisorIOConfig extends SeekableStreamSupervisorIOConfig
return configOverrides;
}
@JsonProperty
public boolean isMultiTopic()
{
return multiTopic;
}
@Override
public String toString()
{

View File

@ -19,6 +19,7 @@
package org.apache.druid.indexing.kafka.supervisor;
import org.apache.druid.data.input.kafka.KafkaTopicPartition;
import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager;
import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload;
import org.joda.time.DateTime;
@ -27,7 +28,7 @@ import javax.annotation.Nullable;
import java.util.List;
import java.util.Map;
public class KafkaSupervisorReportPayload extends SeekableStreamSupervisorReportPayload<Integer, Long>
public class KafkaSupervisorReportPayload extends SeekableStreamSupervisorReportPayload<KafkaTopicPartition, Long>
{
public KafkaSupervisorReportPayload(
String dataSource,
@ -35,8 +36,8 @@ public class KafkaSupervisorReportPayload extends SeekableStreamSupervisorReport
int partitions,
int replicas,
long durationSeconds,
@Nullable Map<Integer, Long> latestOffsets,
@Nullable Map<Integer, Long> minimumLag,
@Nullable Map<KafkaTopicPartition, Long> latestOffsets,
@Nullable Map<KafkaTopicPartition, Long> minimumLag,
@Nullable Long aggregateLag,
@Nullable DateTime offsetsLastUpdated,
boolean suspended,

View File

@ -0,0 +1,119 @@
/*
* 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.data.input.kafka;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.module.SimpleModule;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
public class KafkaTopicPartitionTest
{
private ObjectMapper objectMapper;
@Before
public void setUp()
{
objectMapper = new ObjectMapper();
objectMapper.registerSubtypes(KafkaTopicPartition.class);
SimpleModule simpleModule = new SimpleModule();
simpleModule.addKeySerializer(KafkaTopicPartition.class,
new KafkaTopicPartition.KafkaTopicPartitionKeySerializer());
objectMapper.registerModule(simpleModule);
}
@Test
public void testEquals()
{
KafkaTopicPartition partition1 = new KafkaTopicPartition(false, "topic", 0);
KafkaTopicPartition partition2 = new KafkaTopicPartition(false, "topic", 0);
KafkaTopicPartition partition3 = new KafkaTopicPartition(false, "topic", 1);
KafkaTopicPartition partition4 = new KafkaTopicPartition(false, "topic2", 0);
KafkaTopicPartition partition5 = new KafkaTopicPartition(false, null, 0);
KafkaTopicPartition partition6 = new KafkaTopicPartition(false, null, 0);
KafkaTopicPartition partition7 = new KafkaTopicPartition(true, "topic", 0);
KafkaTopicPartition partition8 = new KafkaTopicPartition(true, "topic2", 0);
Assert.assertEquals(partition1, partition2);
Assert.assertNotEquals(partition1, partition3);
Assert.assertEquals(partition1, partition4);
Assert.assertEquals(partition5, partition6);
Assert.assertEquals(partition1, partition5);
Assert.assertNotEquals(partition1, partition7);
Assert.assertNotEquals(partition7, partition8);
}
@Test
public void testHashCode()
{
KafkaTopicPartition partition1 = new KafkaTopicPartition(false, "topic", 0);
KafkaTopicPartition partition2 = new KafkaTopicPartition(false, "topic", 0);
KafkaTopicPartition partition3 = new KafkaTopicPartition(false, "topic", 1);
KafkaTopicPartition partition4 = new KafkaTopicPartition(false, "topic2", 0);
KafkaTopicPartition partition5 = new KafkaTopicPartition(false, null, 0);
KafkaTopicPartition partition6 = new KafkaTopicPartition(false, null, 0);
KafkaTopicPartition partition7 = new KafkaTopicPartition(true, "topic", 0);
KafkaTopicPartition partition8 = new KafkaTopicPartition(true, "topic2", 0);
Assert.assertEquals(partition1.hashCode(), partition2.hashCode());
Assert.assertNotEquals(partition1.hashCode(), partition3.hashCode());
Assert.assertEquals(partition1.hashCode(), partition4.hashCode());
Assert.assertEquals(partition5.hashCode(), partition6.hashCode());
Assert.assertEquals(partition1.hashCode(), partition5.hashCode());
Assert.assertNotEquals(partition1.hashCode(), partition7.hashCode());
Assert.assertNotEquals(partition7.hashCode(), partition8.hashCode());
}
@Test
public void testMultiTopicDeserialization() throws JsonProcessingException
{
KafkaTopicPartition partition = objectMapper.readerFor(KafkaTopicPartition.class).readValue("\"topic:0\"");
Assert.assertEquals(0, partition.partition());
Assert.assertEquals("topic", partition.topic().orElse(null));
Assert.assertTrue(partition.isMultiTopicPartition());
}
@Test
public void testSingleTopicDeserialization() throws JsonProcessingException
{
KafkaTopicPartition partition = objectMapper.readerFor(KafkaTopicPartition.class).readValue("0");
Assert.assertEquals(0, partition.partition());
Assert.assertNull(partition.topic().orElse(null));
Assert.assertFalse(partition.isMultiTopicPartition());
}
@Test
public void testMultiTopicSerialization() throws JsonProcessingException
{
KafkaTopicPartition partition = new KafkaTopicPartition(true, "topic", 0);
KafkaTopicPartition reincarnated = objectMapper.readerFor(KafkaTopicPartition.class).readValue(objectMapper.writeValueAsString(partition));
Assert.assertEquals(partition, reincarnated);
}
@Test
public void testSingleTopicSerialization() throws JsonProcessingException
{
KafkaTopicPartition partition = new KafkaTopicPartition(false, null, 0);
KafkaTopicPartition reincarnated = objectMapper.readerFor(KafkaTopicPartition.class).readValue(objectMapper.writeValueAsString(partition));
Assert.assertEquals(partition, reincarnated);
}
}

View File

@ -20,30 +20,46 @@
package org.apache.druid.indexing.kafka;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import org.apache.druid.data.input.kafka.KafkaTopicPartition;
import org.apache.druid.indexing.common.actions.CheckPointDataSourceMetadataAction;
import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import java.io.IOException;
public class KafkaCheckpointDataSourceMetadataSerdeTest
{
private static final ObjectMapper MAPPER = new DefaultObjectMapper();
private ObjectMapper objectMapper;
@Before
public void setUp()
{
objectMapper = new DefaultObjectMapper();
objectMapper.registerSubtypes(KafkaDataSourceMetadata.class);
objectMapper.registerSubtypes(KafkaTopicPartition.class);
SimpleModule simpleModule = new SimpleModule();
simpleModule.addKeySerializer(KafkaTopicPartition.class,
new KafkaTopicPartition.KafkaTopicPartitionKeySerializer());
objectMapper.registerModule(simpleModule);
}
@Test
public void testCheckPointDataSourceMetadataActionSerde() throws IOException
{
MAPPER.registerSubtypes(KafkaDataSourceMetadata.class);
final KafkaDataSourceMetadata kafkaDataSourceMetadata =
new KafkaDataSourceMetadata(
new SeekableStreamStartSequenceNumbers<>(
"topic",
ImmutableMap.of(0, 10L, 1, 20L, 2, 30L),
ImmutableMap.of(
new KafkaTopicPartition(false, "topic", 0), 10L,
new KafkaTopicPartition(false, "topic", 1), 20L,
new KafkaTopicPartition(false, "topic", 2), 30L),
ImmutableSet.of()
)
);
@ -54,8 +70,37 @@ public class KafkaCheckpointDataSourceMetadataSerdeTest
kafkaDataSourceMetadata
);
final String serialized = MAPPER.writeValueAsString(checkpointAction);
final CheckPointDataSourceMetadataAction deserialized = MAPPER.readValue(
final String serialized = objectMapper.writeValueAsString(checkpointAction);
final CheckPointDataSourceMetadataAction deserialized = objectMapper.readValue(
serialized,
CheckPointDataSourceMetadataAction.class
);
Assert.assertEquals(checkpointAction, deserialized);
}
@Test
public void testMultiTopicCheckPointDataSourceMetadataActionSerde() throws IOException
{
final KafkaDataSourceMetadata kafkaDataSourceMetadata =
new KafkaDataSourceMetadata(
new SeekableStreamStartSequenceNumbers<>(
"topic1,topic2",
ImmutableMap.of(
new KafkaTopicPartition(true, "topic1", 0), 10L,
new KafkaTopicPartition(true, "topic1", 1), 20L,
new KafkaTopicPartition(true, "topic2", 0), 30L),
ImmutableSet.of()
)
);
final CheckPointDataSourceMetadataAction checkpointAction = new CheckPointDataSourceMetadataAction(
"id_1",
1,
null,
kafkaDataSourceMetadata
);
final String serialized = objectMapper.writeValueAsString(checkpointAction);
final CheckPointDataSourceMetadataAction deserialized = objectMapper.readValue(
serialized,
CheckPointDataSourceMetadataAction.class
);
@ -65,7 +110,6 @@ public class KafkaCheckpointDataSourceMetadataSerdeTest
@Test
public void testCheckPointDataSourceMetadataActionOldJsonSerde() throws IOException
{
MAPPER.registerSubtypes(KafkaDataSourceMetadata.class);
final String jsonStr = "{\n"
+ "\t\"type\": \"checkPointDataSourceMetadata\",\n"
+ "\t\"supervisorId\": \"id_1\",\n"
@ -130,7 +174,7 @@ public class KafkaCheckpointDataSourceMetadataSerdeTest
+ "\t\"sequenceName\": \"dummy\"\n"
+ "}";
final CheckPointDataSourceMetadataAction actual = MAPPER.readValue(
final CheckPointDataSourceMetadataAction actual = objectMapper.readValue(
jsonStr,
CheckPointDataSourceMetadataAction.class
);
@ -139,7 +183,10 @@ public class KafkaCheckpointDataSourceMetadataSerdeTest
new KafkaDataSourceMetadata(
new SeekableStreamStartSequenceNumbers<>(
"topic",
ImmutableMap.of(0, 10L, 1, 20L, 2, 30L),
ImmutableMap.of(
new KafkaTopicPartition(false, null, 0), 10L,
new KafkaTopicPartition(false, null, 1), 20L,
new KafkaTopicPartition(false, null, 2), 30L),
ImmutableSet.of()
)
);

View File

@ -21,8 +21,10 @@ package org.apache.druid.indexing.kafka;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import org.apache.druid.data.input.kafka.KafkaTopicPartition;
import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers;
import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers;
import org.apache.druid.utils.CollectionUtils;
import org.junit.Assert;
import org.junit.Test;
@ -163,11 +165,27 @@ public class KafkaDataSourceMetadataTest
private static KafkaDataSourceMetadata startMetadata(Map<Integer, Long> offsets)
{
return new KafkaDataSourceMetadata(new SeekableStreamStartSequenceNumbers<>("foo", offsets, ImmutableSet.of()));
Map<KafkaTopicPartition, Long> newOffsets = CollectionUtils.mapKeys(
offsets,
k -> new KafkaTopicPartition(
false,
"foo",
k
)
);
return new KafkaDataSourceMetadata(new SeekableStreamStartSequenceNumbers<>("foo", newOffsets, ImmutableSet.of()));
}
private static KafkaDataSourceMetadata endMetadata(Map<Integer, Long> offsets)
{
return new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>("foo", offsets));
Map<KafkaTopicPartition, Long> newOffsets = CollectionUtils.mapKeys(
offsets,
k -> new KafkaTopicPartition(
false,
"foo",
k
)
);
return new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>("foo", newOffsets));
}
}

View File

@ -19,30 +19,21 @@
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.data.input.kafka.KafkaTopicPartition;
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
{
@ -80,9 +71,11 @@ public class KafkaIOConfigTest
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(ImmutableMap.of(new KafkaTopicPartition(false, null, 0), 1L, new KafkaTopicPartition(false, null, 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(new KafkaTopicPartition(false, null, 0), 15L, new KafkaTopicPartition(false, null, 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());
@ -113,9 +106,11 @@ public class KafkaIOConfigTest
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(ImmutableMap.of(new KafkaTopicPartition(false, null, 0), 1L, new KafkaTopicPartition(false, null, 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(new KafkaTopicPartition(false, null, 0), 15L, new KafkaTopicPartition(false, null, 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());
@ -149,9 +144,11 @@ public class KafkaIOConfigTest
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(ImmutableMap.of(new KafkaTopicPartition(false, null, 0), 1L, new KafkaTopicPartition(false, null, 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(new KafkaTopicPartition(false, null, 0), 15L, new KafkaTopicPartition(false, null, 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());
@ -301,161 +298,4 @@ 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(),
null,
null
);
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;
}
}
}

View File

@ -49,6 +49,7 @@ import org.apache.druid.data.input.impl.LongDimensionSchema;
import org.apache.druid.data.input.impl.StringDimensionSchema;
import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.data.input.kafka.KafkaRecordEntity;
import org.apache.druid.data.input.kafka.KafkaTopicPartition;
import org.apache.druid.data.input.kafkainput.KafkaInputFormat;
import org.apache.druid.data.input.kafkainput.KafkaStringHeaderFormat;
import org.apache.druid.indexer.TaskState;
@ -348,8 +349,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -377,7 +378,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
publishedDescriptors()
);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L))),
newDataSchemaMetadata()
);
}
@ -404,8 +405,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -452,8 +453,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -487,8 +488,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -515,7 +516,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
publishedDescriptors()
);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L))),
newDataSchemaMetadata()
);
}
@ -528,8 +529,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -564,7 +565,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
publishedDescriptors()
);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L))),
newDataSchemaMetadata()
);
}
@ -579,8 +580,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 12L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 12L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -632,24 +633,24 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
Map<String, Object> consumerProps = kafkaServer.consumerProperties();
consumerProps.put("max.poll.records", "1");
final SeekableStreamStartSequenceNumbers<Integer, Long> startPartitions = new SeekableStreamStartSequenceNumbers<>(
final SeekableStreamStartSequenceNumbers<KafkaTopicPartition, Long> startPartitions = new SeekableStreamStartSequenceNumbers<>(
topic,
ImmutableMap.of(0, 0L, 1, 0L),
ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 0L, new KafkaTopicPartition(false, topic, 1), 0L),
ImmutableSet.of()
);
// Checkpointing will happen at either checkpoint1 or checkpoint2 depending on ordering
// of events fetched across two partitions from Kafka
final SeekableStreamEndSequenceNumbers<Integer, Long> checkpoint1 = new SeekableStreamEndSequenceNumbers<>(
final SeekableStreamEndSequenceNumbers<KafkaTopicPartition, Long> checkpoint1 = new SeekableStreamEndSequenceNumbers<>(
topic,
ImmutableMap.of(0, 5L, 1, 0L)
ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L, new KafkaTopicPartition(false, topic, 1), 0L)
);
final SeekableStreamEndSequenceNumbers<Integer, Long> checkpoint2 = new SeekableStreamEndSequenceNumbers<>(
final SeekableStreamEndSequenceNumbers<KafkaTopicPartition, Long> checkpoint2 = new SeekableStreamEndSequenceNumbers<>(
topic,
ImmutableMap.of(0, 4L, 1, 2L)
ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 4L, new KafkaTopicPartition(false, topic, 1), 2L)
);
final SeekableStreamEndSequenceNumbers<Integer, Long> endPartitions = new SeekableStreamEndSequenceNumbers<>(
final SeekableStreamEndSequenceNumbers<KafkaTopicPartition, Long> endPartitions = new SeekableStreamEndSequenceNumbers<>(
topic,
ImmutableMap.of(0, 10L, 1, 2L)
ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 10L, new KafkaTopicPartition(false, topic, 1), 2L)
);
final KafkaIndexTask task = createTask(
null,
@ -671,7 +672,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
while (task.getRunner().getStatus() != Status.PAUSED) {
Thread.sleep(10);
}
final Map<Integer, Long> currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets());
final Map<KafkaTopicPartition, Long> currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets());
Assert.assertTrue(checkpoint1.getPartitionSequenceNumberMap().equals(currentOffsets)
|| checkpoint2.getPartitionSequenceNumberMap()
.equals(currentOffsets));
@ -707,7 +708,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
);
Assert.assertEquals(
new KafkaDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L, 1, 2L))
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 10L, new KafkaTopicPartition(false, topic, 1), 2L))
),
newDataSchemaMetadata()
);
@ -736,23 +737,23 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
Map<String, Object> consumerProps = kafkaServer.consumerProperties();
consumerProps.put("max.poll.records", "1");
final SeekableStreamStartSequenceNumbers<Integer, Long> startPartitions = new SeekableStreamStartSequenceNumbers<>(
final SeekableStreamStartSequenceNumbers<KafkaTopicPartition, Long> startPartitions = new SeekableStreamStartSequenceNumbers<>(
topic,
ImmutableMap.of(0, 0L, 1, 0L),
ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 0L, new KafkaTopicPartition(false, topic, 1), 0L),
ImmutableSet.of()
);
final SeekableStreamEndSequenceNumbers<Integer, Long> checkpoint1 = new SeekableStreamEndSequenceNumbers<>(
final SeekableStreamEndSequenceNumbers<KafkaTopicPartition, Long> checkpoint1 = new SeekableStreamEndSequenceNumbers<>(
topic,
ImmutableMap.of(0, 3L, 1, 0L)
ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 3L, new KafkaTopicPartition(false, topic, 1), 0L)
);
final SeekableStreamEndSequenceNumbers<Integer, Long> checkpoint2 = new SeekableStreamEndSequenceNumbers<>(
final SeekableStreamEndSequenceNumbers<KafkaTopicPartition, Long> checkpoint2 = new SeekableStreamEndSequenceNumbers<>(
topic,
ImmutableMap.of(0, 10L, 1, 0L)
ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 10L, new KafkaTopicPartition(false, topic, 1), 0L)
);
final SeekableStreamEndSequenceNumbers<Integer, Long> endPartitions = new SeekableStreamEndSequenceNumbers<>(
final SeekableStreamEndSequenceNumbers<KafkaTopicPartition, Long> endPartitions = new SeekableStreamEndSequenceNumbers<>(
topic,
ImmutableMap.of(0, 10L, 1, 2L)
ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 10L, new KafkaTopicPartition(false, topic, 1), 2L)
);
final KafkaIndexTask task = createTask(
null,
@ -774,7 +775,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
while (task.getRunner().getStatus() != Status.PAUSED) {
Thread.sleep(10);
}
final Map<Integer, Long> currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets());
final Map<KafkaTopicPartition, Long> currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets());
Assert.assertEquals(checkpoint1.getPartitionSequenceNumberMap(), currentOffsets);
task.getRunner().setEndOffsets(currentOffsets, false);
@ -792,7 +793,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
}
kafkaProducer.commitTransaction();
}
final Map<Integer, Long> nextOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets());
final Map<KafkaTopicPartition, Long> nextOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets());
Assert.assertEquals(checkpoint2.getPartitionSequenceNumberMap(), nextOffsets);
@ -840,14 +841,14 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
);
Assert.assertEquals(
new KafkaDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L, 1, 2L))
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 10L, new KafkaTopicPartition(false, topic, 1), 2L))
),
newDataSchemaMetadata()
);
Assert.assertEquals(
new KafkaDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L, 1, 2L))
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 10L, new KafkaTopicPartition(false, topic, 1), 2L))
),
newDataSchemaMetadata()
);
@ -866,19 +867,19 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
Map<String, Object> consumerProps = kafkaServer.consumerProperties();
consumerProps.put("max.poll.records", "1");
final SeekableStreamStartSequenceNumbers<Integer, Long> startPartitions = new SeekableStreamStartSequenceNumbers<>(
final SeekableStreamStartSequenceNumbers<KafkaTopicPartition, Long> startPartitions = new SeekableStreamStartSequenceNumbers<>(
topic,
ImmutableMap.of(0, 0L, 1, 0L),
ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 0L, new KafkaTopicPartition(false, topic, 1), 0L),
ImmutableSet.of()
);
// Checkpointing will happen at checkpoint
final SeekableStreamEndSequenceNumbers<Integer, Long> checkpoint = new SeekableStreamEndSequenceNumbers<>(
final SeekableStreamEndSequenceNumbers<KafkaTopicPartition, Long> checkpoint = new SeekableStreamEndSequenceNumbers<>(
topic,
ImmutableMap.of(0, 1L, 1, 0L)
ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 1L, new KafkaTopicPartition(false, topic, 1), 0L)
);
final SeekableStreamEndSequenceNumbers<Integer, Long> endPartitions = new SeekableStreamEndSequenceNumbers<>(
final SeekableStreamEndSequenceNumbers<KafkaTopicPartition, Long> endPartitions = new SeekableStreamEndSequenceNumbers<>(
topic,
ImmutableMap.of(0, 2L, 1, 0L)
ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 2L, new KafkaTopicPartition(false, topic, 1), 0L)
);
final KafkaIndexTask task = createTask(
null,
@ -902,7 +903,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
while (task.getRunner().getStatus() != Status.PAUSED) {
Thread.sleep(10);
}
final Map<Integer, Long> currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets());
final Map<KafkaTopicPartition, Long> currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets());
Assert.assertEquals(checkpoint.getPartitionSequenceNumberMap(), currentOffsets);
task.getRunner().setEndOffsets(currentOffsets, false);
Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode());
@ -930,7 +931,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
);
Assert.assertEquals(
new KafkaDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 2L, 1, 0L))
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 2L, new KafkaTopicPartition(false, topic, 1), 0L))
),
newDataSchemaMetadata()
);
@ -949,14 +950,14 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
Map<String, Object> consumerProps = kafkaServer.consumerProperties();
consumerProps.put("max.poll.records", "1");
final SeekableStreamStartSequenceNumbers<Integer, Long> startPartitions = new SeekableStreamStartSequenceNumbers<>(
final SeekableStreamStartSequenceNumbers<KafkaTopicPartition, Long> startPartitions = new SeekableStreamStartSequenceNumbers<>(
topic,
ImmutableMap.of(0, 0L, 1, 0L),
ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 0L, new KafkaTopicPartition(false, topic, 1), 0L),
ImmutableSet.of()
);
final SeekableStreamEndSequenceNumbers<Integer, Long> endPartitions = new SeekableStreamEndSequenceNumbers<>(
final SeekableStreamEndSequenceNumbers<KafkaTopicPartition, Long> endPartitions = new SeekableStreamEndSequenceNumbers<>(
topic,
ImmutableMap.of(0, 2L, 1, 0L)
ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 2L, new KafkaTopicPartition(false, topic, 1), 0L)
);
final KafkaIndexTask task = createTask(
null,
@ -981,7 +982,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
Thread.sleep(10);
}
long currentNextCheckpointTime = task.getRunner().getNextCheckpointTime();
final Map<Integer, Long> nextEndOffsets = task.getRunner().getLastSequenceMetadata().getStartOffsets();
final Map<KafkaTopicPartition, Long> nextEndOffsets = task.getRunner().getLastSequenceMetadata().getStartOffsets();
task.getRunner().setEndOffsets(nextEndOffsets, false);
long newNextCheckpointTime = task.getRunner().getNextCheckpointTime();
Assert.assertTrue(
@ -1012,14 +1013,14 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
Map<String, Object> consumerProps = kafkaServer.consumerProperties();
consumerProps.put("max.poll.records", "1");
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 SeekableStreamStartSequenceNumbers<KafkaTopicPartition, Long> startPartitions =
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 0L), ImmutableSet.of());
final SeekableStreamEndSequenceNumbers<KafkaTopicPartition, Long> checkpoint1 =
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L));
final SeekableStreamEndSequenceNumbers<KafkaTopicPartition, Long> checkpoint2 =
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 9L));
final SeekableStreamEndSequenceNumbers<KafkaTopicPartition, Long> endPartitions =
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), Long.MAX_VALUE));
final KafkaIndexTask normalReplica = createTask(
null,
@ -1071,7 +1072,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
while (staleReplica.getRunner().getStatus() != Status.PAUSED) {
Thread.sleep(10);
}
Map<Integer, Long> currentOffsets = ImmutableMap.copyOf(normalReplica.getRunner().getCurrentOffsets());
Map<KafkaTopicPartition, Long> currentOffsets = ImmutableMap.copyOf(normalReplica.getRunner().getCurrentOffsets());
Assert.assertEquals(checkpoint1.getPartitionSequenceNumberMap(), currentOffsets);
normalReplica.getRunner().setEndOffsets(currentOffsets, false);
@ -1107,8 +1108,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 0L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -1142,7 +1143,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
publishedDescriptors()
);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L))),
newDataSchemaMetadata()
);
}
@ -1155,8 +1156,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 0L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -1191,7 +1192,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
publishedDescriptors()
);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L))),
newDataSchemaMetadata()
);
}
@ -1212,8 +1213,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 0L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -1242,7 +1243,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
assertEqualsExceptVersion(ImmutableList.of(sdd("2009/P1D", 0)), publishedDescriptors);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L))),
newDataSchemaMetadata()
);
@ -1284,8 +1285,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 0L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -1357,8 +1358,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 0L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -1408,8 +1409,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 2L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 2L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -1443,8 +1444,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -1470,7 +1471,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
publishedDescriptors()
);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L))),
newDataSchemaMetadata()
);
}
@ -1489,8 +1490,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -1517,7 +1518,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
);
Assert.assertEquals(
new KafkaDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L))
),
newDataSchemaMetadata()
);
@ -1540,8 +1541,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 7L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 7L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -1578,8 +1579,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 13L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 13L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -1609,7 +1610,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
publishedDescriptors()
);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 13L))),
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 13L))),
newDataSchemaMetadata()
);
@ -1666,8 +1667,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 10L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -1728,8 +1729,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -1744,8 +1745,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -1779,7 +1780,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
publishedDescriptors()
);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L))),
newDataSchemaMetadata()
);
}
@ -1792,8 +1793,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -1808,8 +1809,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
1,
"sequence1",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 3L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 3L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 10L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -1845,7 +1846,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
publishedDescriptors
);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L))),
newDataSchemaMetadata()
);
}
@ -1858,8 +1859,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
false,
@ -1874,8 +1875,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
1,
"sequence1",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 3L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 3L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 10L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
false,
@ -1922,8 +1923,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L, 1, 0L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L, 1, 2L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 2L, new KafkaTopicPartition(false, topic, 1), 0L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L, new KafkaTopicPartition(false, topic, 1), 2L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -1954,7 +1955,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
assertEqualsExceptVersion(ImmutableList.of(desc1, desc2, desc4), publishedDescriptors());
Assert.assertEquals(
new KafkaDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L, 1, 2L))
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L, new KafkaTopicPartition(false, topic, 1), 2L))
),
newDataSchemaMetadata()
);
@ -1968,8 +1969,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -1984,8 +1985,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
1,
"sequence1",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(1, 0L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(1, 1L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 1), 0L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 1), 1L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -2020,7 +2021,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
);
Assert.assertEquals(
new KafkaDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L, 1, 1L))
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L, new KafkaTopicPartition(false, topic, 1), 1L))
),
newDataSchemaMetadata()
);
@ -2034,8 +2035,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 6L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 6L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -2068,8 +2069,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 6L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 6L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -2100,7 +2101,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
publishedDescriptors()
);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 6L))),
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 6L))),
newDataSchemaMetadata()
);
}
@ -2118,8 +2119,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 0L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 10L)),
consumerProps,
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -2130,10 +2131,10 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
)
);
final SeekableStreamStartSequenceNumbers<Integer, Long> checkpoint = new SeekableStreamStartSequenceNumbers<>(
final SeekableStreamStartSequenceNumbers<KafkaTopicPartition, Long> checkpoint = new SeekableStreamStartSequenceNumbers<>(
topic,
ImmutableMap.of(0, 5L),
ImmutableSet.of(0)
ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L),
ImmutableSet.of(new KafkaTopicPartition(false, topic, 0))
);
final ListenableFuture<TaskStatus> future1 = runTask(task1);
@ -2144,10 +2145,10 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
while (task1.getRunner().getStatus() != Status.PAUSED) {
Thread.sleep(10);
}
final Map<Integer, Long> currentOffsets = ImmutableMap.copyOf(task1.getRunner().getCurrentOffsets());
final Map<KafkaTopicPartition, Long> currentOffsets = ImmutableMap.copyOf(task1.getRunner().getCurrentOffsets());
Assert.assertEquals(checkpoint.getPartitionSequenceNumberMap(), currentOffsets);
// Set endOffsets to persist sequences
task1.getRunner().setEndOffsets(ImmutableMap.of(0, 5L), false);
task1.getRunner().setEndOffsets(ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L), false);
// Stop without publishing segment
task1.stopGracefully(toolboxFactory.build(task1).getConfig());
@ -2161,8 +2162,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 0L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 10L)),
consumerProps,
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -2199,7 +2200,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
publishedDescriptors()
);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L))),
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 10L))),
newDataSchemaMetadata()
);
}
@ -2212,8 +2213,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 6L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 6L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -2235,9 +2236,9 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
Assert.assertEquals(2, countEvents(task));
Assert.assertEquals(Status.READING, task.getRunner().getStatus());
Map<Integer, Long> currentOffsets = OBJECT_MAPPER.readValue(
Map<KafkaTopicPartition, Long> currentOffsets = OBJECT_MAPPER.readValue(
task.getRunner().pause().getEntity().toString(),
new TypeReference<Map<Integer, Long>>()
new TypeReference<Map<KafkaTopicPartition, Long>>()
{
}
);
@ -2270,7 +2271,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
publishedDescriptors()
);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 6L))),
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 6L))),
newDataSchemaMetadata()
);
}
@ -2283,8 +2284,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -2320,8 +2321,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 200L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 500L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 200L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 500L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -2341,7 +2342,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
for (int i = 0; i < 5; i++) {
Assert.assertEquals(Status.READING, task.getRunner().getStatus());
// Offset should not be reset
Assert.assertEquals(200L, (long) task.getRunner().getCurrentOffsets().get(0));
Assert.assertEquals(200L, (long) task.getRunner().getCurrentOffsets().get(new KafkaTopicPartition(false, topic, 0)));
}
}
@ -2351,10 +2352,10 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
// Insert data
insertData();
final TreeMap<Integer, Map<Integer, Long>> sequences = new TreeMap<>();
final TreeMap<Integer, Map<KafkaTopicPartition, Long>> sequences = new TreeMap<>();
// Here the sequence number is 1 meaning that one incremental handoff was done by the failed task
// and this task should start reading from offset 2 for partition 0
sequences.put(1, ImmutableMap.of(0, 2L));
sequences.put(1, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 2L));
final Map<String, Object> context = new HashMap<>();
context.put(
SeekableStreamSupervisor.CHECKPOINTS_CTX_KEY,
@ -2367,8 +2368,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
0,
"sequence0",
// task should ignore these and use sequence info sent in the context
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 0L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -2395,7 +2396,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
publishedDescriptors()
);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L))),
newDataSchemaMetadata()
);
}
@ -2411,8 +2412,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 200L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 500L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 200L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 500L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -2431,12 +2432,12 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
// first setEndOffsets request
task.getRunner().pause();
task.getRunner().setEndOffsets(ImmutableMap.of(0, 500L), true);
task.getRunner().setEndOffsets(ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 500L), true);
Assert.assertEquals(Status.READING, task.getRunner().getStatus());
// duplicate setEndOffsets request
task.getRunner().pause();
task.getRunner().setEndOffsets(ImmutableMap.of(0, 500L), true);
task.getRunner().setEndOffsets(ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 500L), true);
Assert.assertEquals(Status.READING, task.getRunner().getStatus());
}
@ -2448,10 +2449,10 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 0L), ImmutableSet.of()),
// End offset is one after 12 real messages + 2 txn control messages (last seen message: offset 13).
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 14L)),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 14L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -2467,7 +2468,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
// Insert 2 records initially
insertData(records.subList(0, 2));
awaitConsumedOffsets(task, ImmutableMap.of(0, 1L)); // Consume two real messages
awaitConsumedOffsets(task, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 1L)); // Consume two real messages
Assert.assertEquals(2, countEvents(task));
Assert.assertEquals(Status.READING, task.getRunner().getStatus());
@ -2482,7 +2483,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
// Insert up through first 8 items
insertData(records.subList(5, 8));
awaitConsumedOffsets(task, ImmutableMap.of(0, 9L)); // Consume 8 real messages + 2 txn controls
awaitConsumedOffsets(task, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 9L)); // Consume 8 real messages + 2 txn controls
Assert.assertEquals(2, countEvents(task));
final QuerySegmentSpec rollbackedInterval = OBJECT_MAPPER.readValue("\"2010/2012\"", QuerySegmentSpec.class);
@ -2516,7 +2517,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
publishedDescriptors()
);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 14L))),
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 14L))),
newDataSchemaMetadata()
);
}
@ -2531,8 +2532,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 13L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 0L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 13L)),
configs,
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -2575,15 +2576,15 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
Map<String, Object> consumerProps = kafkaServer.consumerProperties();
consumerProps.put("max.poll.records", "1");
final SeekableStreamStartSequenceNumbers<Integer, Long> startPartitions = new SeekableStreamStartSequenceNumbers<>(
final SeekableStreamStartSequenceNumbers<KafkaTopicPartition, Long> startPartitions = new SeekableStreamStartSequenceNumbers<>(
topic,
ImmutableMap.of(0, 0L, 1, 1L),
ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 0L, new KafkaTopicPartition(false, topic, 1), 1L),
ImmutableSet.of()
);
final SeekableStreamEndSequenceNumbers<Integer, Long> endPartitions = new SeekableStreamEndSequenceNumbers<>(
final SeekableStreamEndSequenceNumbers<KafkaTopicPartition, Long> endPartitions = new SeekableStreamEndSequenceNumbers<>(
topic,
ImmutableMap.of(0, 10L, 1, 2L)
ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 10L, new KafkaTopicPartition(false, topic, 1), 2L)
);
final KafkaIndexTask task = createTask(
@ -2614,8 +2615,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 2L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -2718,15 +2719,15 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
/**
* Wait for a task to consume certain offsets (inclusive).
*/
private void awaitConsumedOffsets(final KafkaIndexTask task, final Map<Integer, Long> targetOffsets)
private void awaitConsumedOffsets(final KafkaIndexTask task, final Map<KafkaTopicPartition, Long> targetOffsets)
throws InterruptedException
{
while (true) {
final ConcurrentMap<Integer, Long> currentOffsets = task.getRunner().getCurrentOffsets();
final ConcurrentMap<KafkaTopicPartition, Long> currentOffsets = task.getRunner().getCurrentOffsets();
// For Kafka, currentOffsets are the last read offsets plus one.
boolean allDone = true;
for (final Map.Entry<Integer, Long> entry : targetOffsets.entrySet()) {
for (final Map.Entry<KafkaTopicPartition, Long> entry : targetOffsets.entrySet()) {
final Long currentOffset = currentOffsets.get(entry.getKey());
if (currentOffset == null || currentOffset <= entry.getValue()) {
allDone = false;
@ -2852,7 +2853,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
maxSavedParseExceptions
);
if (!context.containsKey(SeekableStreamSupervisor.CHECKPOINTS_CTX_KEY)) {
final TreeMap<Integer, Map<Integer, Long>> checkpoints = new TreeMap<>();
final TreeMap<Integer, Map<KafkaTopicPartition, Long>> checkpoints = new TreeMap<>();
checkpoints.put(0, ioConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap());
final String checkpointsJson = OBJECT_MAPPER
.writerFor(KafkaSupervisor.CHECKPOINTS_TYPE_REF)
@ -2964,8 +2965,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 4L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 0L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 4L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -2991,7 +2992,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
publishedDescriptors()
);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 4L))),
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 4L))),
newDataSchemaMetadata()
);
}
@ -3025,8 +3026,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 4L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 0L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 4L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -3053,7 +3054,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
publishedDescriptors()
);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 4L))),
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 4L))),
newDataSchemaMetadata()
);
@ -3093,8 +3094,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 2L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 0L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 2L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
@ -3121,7 +3122,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
publishedDescriptors()
);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 2L))),
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 2L))),
newDataSchemaMetadata()
);
@ -3167,8 +3168,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
new KafkaIndexTaskIOConfig(
0,
"sequence0",
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 6L)),
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 0L), ImmutableSet.of()),
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 6L)),
kafkaServer.consumerProperties(),
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,

View File

@ -25,6 +25,7 @@ import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import org.apache.curator.test.TestingCluster;
import org.apache.druid.data.input.kafka.KafkaRecordEntity;
import org.apache.druid.data.input.kafka.KafkaTopicPartition;
import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig;
import org.apache.druid.indexing.kafka.test.TestBroker;
import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord;
@ -77,21 +78,21 @@ public class KafkaRecordSupplierTest
private static List<ProducerRecord<byte[], byte[]>> generateRecords(String topic)
{
return ImmutableList.of(
new ProducerRecord<>(topic, 0, null, jb("2008", "a", "y", "10", "20.0", "1.0")),
new ProducerRecord<>(topic, 0, null, jb("2009", "b", "y", "10", "20.0", "1.0")),
new ProducerRecord<>(topic, 0, null, jb("2010", "c", "y", "10", "20.0", "1.0")),
new ProducerRecord<>(topic, 0, null, jb("2011", "d", "y", "10", "20.0", "1.0")),
new ProducerRecord<>(topic, 0, null, jb("2011", "e", "y", "10", "20.0", "1.0")),
new ProducerRecord<>(topic, 0, null, jb("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")),
new ProducerRecord<>(topic, 0, null, StringUtils.toUtf8("unparseable")),
new ProducerRecord<>(topic, 0, null, StringUtils.toUtf8("unparseable2")),
new ProducerRecord<>(topic, 0, null, null),
new ProducerRecord<>(topic, 0, null, jb("2013", "f", "y", "10", "20.0", "1.0")),
new ProducerRecord<>(topic, 0, null, jb("2049", "f", "y", "notanumber", "20.0", "1.0")),
new ProducerRecord<>(topic, 1, null, jb("2049", "f", "y", "10", "notanumber", "1.0")),
new ProducerRecord<>(topic, 1, null, jb("2049", "f", "y", "10", "20.0", "notanumber")),
new ProducerRecord<>(topic, 1, null, jb("2012", "g", "y", "10", "20.0", "1.0")),
new ProducerRecord<>(topic, 1, null, jb("2011", "h", "y", "10", "20.0", "1.0"))
new ProducerRecord<>(TOPIC, 0, null, jb("2008", "a", "y", "10", "20.0", "1.0")),
new ProducerRecord<>(TOPIC, 0, null, jb("2009", "b", "y", "10", "20.0", "1.0")),
new ProducerRecord<>(TOPIC, 0, null, jb("2010", "c", "y", "10", "20.0", "1.0")),
new ProducerRecord<>(TOPIC, 0, null, jb("2011", "d", "y", "10", "20.0", "1.0")),
new ProducerRecord<>(TOPIC, 0, null, jb("2011", "e", "y", "10", "20.0", "1.0")),
new ProducerRecord<>(TOPIC, 0, null, jb("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")),
new ProducerRecord<>(TOPIC, 0, null, StringUtils.toUtf8("unparseable")),
new ProducerRecord<>(TOPIC, 0, null, StringUtils.toUtf8("unparseable2")),
new ProducerRecord<>(TOPIC, 0, null, null),
new ProducerRecord<>(TOPIC, 0, null, jb("2013", "f", "y", "10", "20.0", "1.0")),
new ProducerRecord<>(TOPIC, 0, null, jb("2049", "f", "y", "notanumber", "20.0", "1.0")),
new ProducerRecord<>(TOPIC, 1, null, jb("2049", "f", "y", "10", "notanumber", "1.0")),
new ProducerRecord<>(TOPIC, 1, null, jb("2049", "f", "y", "10", "20.0", "notanumber")),
new ProducerRecord<>(TOPIC, 1, null, jb("2012", "g", "y", "10", "20.0", "1.0")),
new ProducerRecord<>(TOPIC, 1, null, jb("2011", "h", "y", "10", "20.0", "1.0"))
);
}
@ -119,20 +120,21 @@ public class KafkaRecordSupplierTest
return "topic-" + TOPIC_POS_FIX++;
}
private List<OrderedPartitionableRecord<Integer, Long, KafkaRecordEntity>> createOrderedPartitionableRecords()
private List<OrderedPartitionableRecord<KafkaTopicPartition, Long, KafkaRecordEntity>> createOrderedPartitionableRecords()
{
Map<Integer, Long> partitionToOffset = new HashMap<>();
Map<KafkaTopicPartition, Long> partitionToOffset = new HashMap<>();
return records.stream().map(r -> {
long offset = 0;
if (partitionToOffset.containsKey(r.partition())) {
offset = partitionToOffset.get(r.partition());
partitionToOffset.put(r.partition(), offset + 1);
KafkaTopicPartition tp = new KafkaTopicPartition(false, r.topic(), r.partition());
if (partitionToOffset.containsKey(tp)) {
offset = partitionToOffset.get(tp);
partitionToOffset.put(tp, offset + 1);
} else {
partitionToOffset.put(r.partition(), 1L);
partitionToOffset.put(tp, 1L);
}
return new OrderedPartitionableRecord<>(
TOPIC,
r.partition(),
tp,
offset,
r.value() == null ? null : Collections.singletonList(new KafkaRecordEntity(
new ConsumerRecord<>(r.topic(), r.partition(), offset, r.key(), r.value())
@ -156,7 +158,7 @@ public class KafkaRecordSupplierTest
}
@Override
public byte[] deserialize(String topic, byte[] data)
public byte[] deserialize(String TOPIC, byte[] data)
{
return data;
}
@ -180,7 +182,7 @@ public class KafkaRecordSupplierTest
}
@Override
public byte[] deserialize(String topic, byte[] data)
public byte[] deserialize(String TOPIC, byte[] data)
{
return data;
}
@ -226,20 +228,21 @@ public class KafkaRecordSupplierTest
// Insert data
insertData();
Set<StreamPartition<Integer>> partitions = ImmutableSet.of(
StreamPartition.of(TOPIC, 0),
StreamPartition.of(TOPIC, 1)
Set<StreamPartition<KafkaTopicPartition>> partitions = ImmutableSet.of(
StreamPartition.of(TOPIC, new KafkaTopicPartition(false, TOPIC, 0)),
StreamPartition.of(TOPIC, new KafkaTopicPartition(false, TOPIC, 1))
);
KafkaRecordSupplier recordSupplier = new KafkaRecordSupplier(
KAFKA_SERVER.consumerProperties(), OBJECT_MAPPER, null);
KAFKA_SERVER.consumerProperties(), OBJECT_MAPPER, null, false);
Assert.assertTrue(recordSupplier.getAssignment().isEmpty());
recordSupplier.assign(partitions);
Assert.assertEquals(partitions, recordSupplier.getAssignment());
Assert.assertEquals(ImmutableSet.of(0, 1), recordSupplier.getPartitionIds(TOPIC));
Assert.assertEquals(ImmutableSet.of(new KafkaTopicPartition(false, TOPIC, 0), new KafkaTopicPartition(false, TOPIC, 1)),
recordSupplier.getPartitionIds(TOPIC));
recordSupplier.close();
}
@ -251,9 +254,9 @@ public class KafkaRecordSupplierTest
// Insert data
insertData();
Set<StreamPartition<Integer>> partitions = ImmutableSet.of(
StreamPartition.of(TOPIC, 0),
StreamPartition.of(TOPIC, 1)
Set<StreamPartition<KafkaTopicPartition>> partitions = ImmutableSet.of(
StreamPartition.of(TOPIC, new KafkaTopicPartition(false, TOPIC, 0)),
StreamPartition.of(TOPIC, new KafkaTopicPartition(false, TOPIC, 1))
);
Map<String, Object> properties = KAFKA_SERVER.consumerProperties();
@ -263,7 +266,8 @@ public class KafkaRecordSupplierTest
KafkaRecordSupplier recordSupplier = new KafkaRecordSupplier(
properties,
OBJECT_MAPPER,
null
null,
false
);
Assert.assertTrue(recordSupplier.getAssignment().isEmpty());
@ -271,7 +275,8 @@ public class KafkaRecordSupplierTest
recordSupplier.assign(partitions);
Assert.assertEquals(partitions, recordSupplier.getAssignment());
Assert.assertEquals(ImmutableSet.of(0, 1), recordSupplier.getPartitionIds(TOPIC));
Assert.assertEquals(ImmutableSet.of(new KafkaTopicPartition(false, TOPIC, 0), new KafkaTopicPartition(false, TOPIC, 1)),
recordSupplier.getPartitionIds(TOPIC));
recordSupplier.close();
}
@ -289,7 +294,8 @@ public class KafkaRecordSupplierTest
KafkaRecordSupplier recordSupplier = new KafkaRecordSupplier(
properties,
OBJECT_MAPPER,
null
null,
false
);
Assert.assertTrue(recordSupplier.getAssignment().isEmpty()); //just test recordSupplier is initiated
@ -307,7 +313,8 @@ public class KafkaRecordSupplierTest
KafkaRecordSupplier recordSupplier = new KafkaRecordSupplier(
properties,
OBJECT_MAPPER,
null
null,
false
);
Assert.assertTrue(recordSupplier.getAssignment().isEmpty()); //just test recordSupplier is initiated
@ -321,9 +328,9 @@ public class KafkaRecordSupplierTest
// Insert data
insertData();
Set<StreamPartition<Integer>> partitions = ImmutableSet.of(
StreamPartition.of(TOPIC, 0),
StreamPartition.of(TOPIC, 1)
Set<StreamPartition<KafkaTopicPartition>> partitions = ImmutableSet.of(
StreamPartition.of(TOPIC, new KafkaTopicPartition(false, TOPIC, 0)),
StreamPartition.of(TOPIC, new KafkaTopicPartition(false, TOPIC, 1))
);
Map<String, Object> properties = KAFKA_SERVER.consumerProperties();
@ -333,16 +340,17 @@ public class KafkaRecordSupplierTest
KafkaRecordSupplier recordSupplier = new KafkaRecordSupplier(
properties,
OBJECT_MAPPER,
null
null,
false
);
recordSupplier.assign(partitions);
recordSupplier.seekToEarliest(partitions);
List<OrderedPartitionableRecord<Integer, Long, KafkaRecordEntity>> initialRecords = new ArrayList<>(createOrderedPartitionableRecords());
List<OrderedPartitionableRecord<KafkaTopicPartition, Long, KafkaRecordEntity>> initialRecords = new ArrayList<>(createOrderedPartitionableRecords());
List<OrderedPartitionableRecord<Integer, Long, KafkaRecordEntity>> polledRecords = recordSupplier.poll(
POLL_TIMEOUT_MILLIS);
List<OrderedPartitionableRecord<KafkaTopicPartition, Long, KafkaRecordEntity>> polledRecords =
recordSupplier.poll(POLL_TIMEOUT_MILLIS);
for (int i = 0; polledRecords.size() != initialRecords.size() && i < POLL_RETRY; i++) {
polledRecords.addAll(recordSupplier.poll(POLL_TIMEOUT_MILLIS));
Thread.sleep(200);
@ -362,15 +370,16 @@ public class KafkaRecordSupplierTest
// Insert data
insertData();
Set<StreamPartition<Integer>> partitions = ImmutableSet.of(
StreamPartition.of(TOPIC, 0),
StreamPartition.of(TOPIC, 1)
Set<StreamPartition<KafkaTopicPartition>> partitions = ImmutableSet.of(
StreamPartition.of(TOPIC, new KafkaTopicPartition(false, TOPIC, 0)),
StreamPartition.of(TOPIC, new KafkaTopicPartition(false, TOPIC, 1))
);
KafkaRecordSupplier recordSupplier = new KafkaRecordSupplier(
KAFKA_SERVER.consumerProperties(),
OBJECT_MAPPER,
null
null,
false
);
final Monitor monitor = recordSupplier.monitor();
@ -378,10 +387,9 @@ public class KafkaRecordSupplierTest
recordSupplier.assign(partitions);
recordSupplier.seekToEarliest(partitions);
List<OrderedPartitionableRecord<Integer, Long, KafkaRecordEntity>> initialRecords = new ArrayList<>(createOrderedPartitionableRecords());
List<OrderedPartitionableRecord<KafkaTopicPartition, Long, KafkaRecordEntity>> initialRecords = new ArrayList<>(createOrderedPartitionableRecords());
List<OrderedPartitionableRecord<Integer, Long, KafkaRecordEntity>> polledRecords =
recordSupplier.poll(POLL_TIMEOUT_MILLIS);
List<OrderedPartitionableRecord<KafkaTopicPartition, Long, KafkaRecordEntity>> polledRecords = recordSupplier.poll(POLL_TIMEOUT_MILLIS);
for (int i = 0; polledRecords.size() != initialRecords.size() && i < POLL_RETRY; i++) {
polledRecords.addAll(recordSupplier.poll(POLL_TIMEOUT_MILLIS));
Thread.sleep(200);
@ -415,20 +423,18 @@ public class KafkaRecordSupplierTest
kafkaProducer.commitTransaction();
}
Set<StreamPartition<Integer>> partitions = ImmutableSet.of(
StreamPartition.of(TOPIC, 0),
StreamPartition.of(TOPIC, 1)
Set<StreamPartition<KafkaTopicPartition>> partitions = ImmutableSet.of(
StreamPartition.of(TOPIC, new KafkaTopicPartition(false, TOPIC, 0)),
StreamPartition.of(TOPIC, new KafkaTopicPartition(false, TOPIC, 1))
);
KafkaRecordSupplier recordSupplier = new KafkaRecordSupplier(
KAFKA_SERVER.consumerProperties(), OBJECT_MAPPER, null);
KAFKA_SERVER.consumerProperties(), OBJECT_MAPPER, null, false);
recordSupplier.assign(partitions);
recordSupplier.seekToEarliest(partitions);
List<OrderedPartitionableRecord<Integer, Long, KafkaRecordEntity>> polledRecords = recordSupplier.poll(
POLL_TIMEOUT_MILLIS);
List<OrderedPartitionableRecord<KafkaTopicPartition, Long, KafkaRecordEntity>> polledRecords = recordSupplier.poll(POLL_TIMEOUT_MILLIS);
for (int i = 0; polledRecords.size() != 13 && i < POLL_RETRY; i++) {
polledRecords.addAll(recordSupplier.poll(POLL_TIMEOUT_MILLIS));
Thread.sleep(200);
@ -450,26 +456,26 @@ public class KafkaRecordSupplierTest
Thread.sleep(200);
}
List<OrderedPartitionableRecord<Integer, Long, KafkaRecordEntity>> initialRecords = createOrderedPartitionableRecords();
List<OrderedPartitionableRecord<KafkaTopicPartition, Long, KafkaRecordEntity>> initialRecords = createOrderedPartitionableRecords();
Assert.assertEquals(records.size(), polledRecords.size());
Assert.assertEquals(partitions, recordSupplier.getAssignment());
final int initialRecordsPartition0Size = initialRecords.stream()
.filter(r -> r.getPartitionId().equals(0))
.filter(r -> r.getPartitionId().partition() == 0)
.collect(Collectors.toSet())
.size();
final int initialRecordsPartition1Size = initialRecords.stream()
.filter(r -> r.getPartitionId().equals(1))
.filter(r -> r.getPartitionId().partition() == 1)
.collect(Collectors.toSet())
.size();
final int polledRecordsPartition0Size = polledRecords.stream()
.filter(r -> r.getPartitionId().equals(0))
.filter(r -> r.getPartitionId().partition() == 0)
.collect(Collectors.toSet())
.size();
final int polledRecordsPartition1Size = polledRecords.stream()
.filter(r -> r.getPartitionId().equals(1))
.filter(r -> r.getPartitionId().partition() == 1)
.collect(Collectors.toSet())
.size();
@ -485,16 +491,16 @@ public class KafkaRecordSupplierTest
// Insert data
insertData();
StreamPartition<Integer> partition0 = StreamPartition.of(TOPIC, 0);
StreamPartition<Integer> partition1 = StreamPartition.of(TOPIC, 1);
StreamPartition<KafkaTopicPartition> partition0 = StreamPartition.of(TOPIC, new KafkaTopicPartition(false, TOPIC, 0));
StreamPartition<KafkaTopicPartition> partition1 = StreamPartition.of(TOPIC, new KafkaTopicPartition(false, TOPIC, 1));
Set<StreamPartition<Integer>> partitions = ImmutableSet.of(
StreamPartition.of(TOPIC, 0),
StreamPartition.of(TOPIC, 1)
Set<StreamPartition<KafkaTopicPartition>> partitions = ImmutableSet.of(
StreamPartition.of(TOPIC, new KafkaTopicPartition(false, TOPIC, 0)),
StreamPartition.of(TOPIC, new KafkaTopicPartition(false, TOPIC, 1))
);
KafkaRecordSupplier recordSupplier = new KafkaRecordSupplier(
KAFKA_SERVER.consumerProperties(), OBJECT_MAPPER, null);
KAFKA_SERVER.consumerProperties(), OBJECT_MAPPER, null, false);
recordSupplier.assign(partitions);
recordSupplier.seekToEarliest(partitions);
@ -505,10 +511,9 @@ public class KafkaRecordSupplierTest
recordSupplier.seek(partition0, 2L);
recordSupplier.seek(partition1, 2L);
List<OrderedPartitionableRecord<Integer, Long, KafkaRecordEntity>> initialRecords = createOrderedPartitionableRecords();
List<OrderedPartitionableRecord<KafkaTopicPartition, Long, KafkaRecordEntity>> initialRecords = createOrderedPartitionableRecords();
List<OrderedPartitionableRecord<Integer, Long, KafkaRecordEntity>> polledRecords = recordSupplier.poll(
POLL_TIMEOUT_MILLIS);
List<OrderedPartitionableRecord<KafkaTopicPartition, Long, KafkaRecordEntity>> polledRecords = recordSupplier.poll(POLL_TIMEOUT_MILLIS);
for (int i = 0; polledRecords.size() != 11 && i < POLL_RETRY; i++) {
polledRecords.addAll(recordSupplier.poll(POLL_TIMEOUT_MILLIS));
Thread.sleep(200);
@ -529,16 +534,16 @@ public class KafkaRecordSupplierTest
// Insert data
insertData();
StreamPartition<Integer> partition0 = StreamPartition.of(TOPIC, 0);
StreamPartition<Integer> partition1 = StreamPartition.of(TOPIC, 1);
StreamPartition<KafkaTopicPartition> partition0 = StreamPartition.of(TOPIC, new KafkaTopicPartition(false, TOPIC, 0));
StreamPartition<KafkaTopicPartition> partition1 = StreamPartition.of(TOPIC, new KafkaTopicPartition(false, TOPIC, 1));
Set<StreamPartition<Integer>> partitions = ImmutableSet.of(
StreamPartition.of(TOPIC, 0),
StreamPartition.of(TOPIC, 1)
Set<StreamPartition<KafkaTopicPartition>> partitions = ImmutableSet.of(
StreamPartition.of(TOPIC, new KafkaTopicPartition(false, TOPIC, 0)),
StreamPartition.of(TOPIC, new KafkaTopicPartition(false, TOPIC, 1))
);
KafkaRecordSupplier recordSupplier = new KafkaRecordSupplier(
KAFKA_SERVER.consumerProperties(), OBJECT_MAPPER, null);
KAFKA_SERVER.consumerProperties(), OBJECT_MAPPER, null, false);
recordSupplier.assign(partitions);
recordSupplier.seekToEarliest(partitions);
@ -547,8 +552,7 @@ public class KafkaRecordSupplierTest
Assert.assertEquals(0L, (long) recordSupplier.getEarliestSequenceNumber(partition1));
recordSupplier.seekToLatest(partitions);
List<OrderedPartitionableRecord<Integer, Long, KafkaRecordEntity>> polledRecords = recordSupplier.poll(
POLL_TIMEOUT_MILLIS);
List<OrderedPartitionableRecord<KafkaTopicPartition, Long, KafkaRecordEntity>> polledRecords = recordSupplier.poll(POLL_TIMEOUT_MILLIS);
Assert.assertEquals(Collections.emptyList(), polledRecords);
recordSupplier.close();
@ -564,15 +568,15 @@ public class KafkaRecordSupplierTest
}
}
StreamPartition<Integer> partition0 = StreamPartition.of(TOPIC, 0);
StreamPartition<Integer> partition1 = StreamPartition.of(TOPIC, 1);
StreamPartition<KafkaTopicPartition> partition0 = StreamPartition.of(TOPIC, new KafkaTopicPartition(false, TOPIC, 0));
StreamPartition<KafkaTopicPartition> partition1 = StreamPartition.of(TOPIC, new KafkaTopicPartition(false, TOPIC, 1));
Set<StreamPartition<Integer>> partitions = ImmutableSet.of(
StreamPartition.of(TOPIC, 0)
Set<StreamPartition<KafkaTopicPartition>> partitions = ImmutableSet.of(
StreamPartition.of(TOPIC, new KafkaTopicPartition(false, TOPIC, 0))
);
KafkaRecordSupplier recordSupplier = new KafkaRecordSupplier(
KAFKA_SERVER.consumerProperties(), OBJECT_MAPPER, null);
KAFKA_SERVER.consumerProperties(), OBJECT_MAPPER, null, false);
recordSupplier.assign(partitions);
@ -589,16 +593,16 @@ public class KafkaRecordSupplierTest
// Insert data
insertData();
StreamPartition<Integer> partition0 = StreamPartition.of(TOPIC, 0);
StreamPartition<Integer> partition1 = StreamPartition.of(TOPIC, 1);
StreamPartition<KafkaTopicPartition> partition0 = StreamPartition.of(TOPIC, new KafkaTopicPartition(false, TOPIC, 0));
StreamPartition<KafkaTopicPartition> partition1 = StreamPartition.of(TOPIC, new KafkaTopicPartition(false, TOPIC, 1));
Set<StreamPartition<Integer>> partitions = ImmutableSet.of(
StreamPartition.of(TOPIC, 0),
StreamPartition.of(TOPIC, 1)
Set<StreamPartition<KafkaTopicPartition>> partitions = ImmutableSet.of(
StreamPartition.of(TOPIC, new KafkaTopicPartition(false, TOPIC, 0)),
StreamPartition.of(TOPIC, new KafkaTopicPartition(false, TOPIC, 1))
);
KafkaRecordSupplier recordSupplier = new KafkaRecordSupplier(
KAFKA_SERVER.consumerProperties(), OBJECT_MAPPER, null);
KAFKA_SERVER.consumerProperties(), OBJECT_MAPPER, null, false);
recordSupplier.assign(partitions);
recordSupplier.seekToEarliest(partitions);
@ -633,9 +637,9 @@ public class KafkaRecordSupplierTest
public void getLatestSequenceNumberWhenPartitionIsEmptyAndUseEarliestOffsetShouldReturnsValidNonNull()
{
KafkaRecordSupplier recordSupplier = new KafkaRecordSupplier(
KAFKA_SERVER.consumerProperties(), OBJECT_MAPPER, null);
StreamPartition<Integer> streamPartition = StreamPartition.of(TOPIC, 0);
Set<StreamPartition<Integer>> partitions = ImmutableSet.of(streamPartition);
KAFKA_SERVER.consumerProperties(), OBJECT_MAPPER, null, false);
StreamPartition<KafkaTopicPartition> streamPartition = StreamPartition.of(TOPIC, new KafkaTopicPartition(false, TOPIC, 0));
Set<StreamPartition<KafkaTopicPartition>> partitions = ImmutableSet.of(streamPartition);
recordSupplier.assign(partitions);
recordSupplier.seekToEarliest(partitions);
Assert.assertEquals(new Long(0), recordSupplier.getLatestSequenceNumber(streamPartition));
@ -645,9 +649,9 @@ public class KafkaRecordSupplierTest
public void getEarliestSequenceNumberWhenPartitionIsEmptyAndUseEarliestOffsetShouldReturnsValidNonNull()
{
KafkaRecordSupplier recordSupplier = new KafkaRecordSupplier(
KAFKA_SERVER.consumerProperties(), OBJECT_MAPPER, null);
StreamPartition<Integer> streamPartition = StreamPartition.of(TOPIC, 0);
Set<StreamPartition<Integer>> partitions = ImmutableSet.of(streamPartition);
KAFKA_SERVER.consumerProperties(), OBJECT_MAPPER, null, false);
StreamPartition<KafkaTopicPartition> streamPartition = StreamPartition.of(TOPIC, new KafkaTopicPartition(false, TOPIC, 0));
Set<StreamPartition<KafkaTopicPartition>> partitions = ImmutableSet.of(streamPartition);
recordSupplier.assign(partitions);
recordSupplier.seekToEarliest(partitions);
Assert.assertEquals(new Long(0), recordSupplier.getEarliestSequenceNumber(streamPartition));
@ -657,9 +661,9 @@ public class KafkaRecordSupplierTest
public void getLatestSequenceNumberWhenPartitionIsEmptyAndUseLatestOffsetShouldReturnsValidNonNull()
{
KafkaRecordSupplier recordSupplier = new KafkaRecordSupplier(
KAFKA_SERVER.consumerProperties(), OBJECT_MAPPER, null);
StreamPartition<Integer> streamPartition = StreamPartition.of(TOPIC, 0);
Set<StreamPartition<Integer>> partitions = ImmutableSet.of(streamPartition);
KAFKA_SERVER.consumerProperties(), OBJECT_MAPPER, null, false);
StreamPartition<KafkaTopicPartition> streamPartition = StreamPartition.of(TOPIC, new KafkaTopicPartition(false, TOPIC, 0));
Set<StreamPartition<KafkaTopicPartition>> partitions = ImmutableSet.of(streamPartition);
recordSupplier.assign(partitions);
recordSupplier.seekToLatest(partitions);
Assert.assertEquals(new Long(0), recordSupplier.getLatestSequenceNumber(streamPartition));
@ -669,9 +673,9 @@ public class KafkaRecordSupplierTest
public void getEarliestSequenceNumberWhenPartitionIsEmptyAndUseLatestOffsetShouldReturnsValidNonNull()
{
KafkaRecordSupplier recordSupplier = new KafkaRecordSupplier(
KAFKA_SERVER.consumerProperties(), OBJECT_MAPPER, null);
StreamPartition<Integer> streamPartition = StreamPartition.of(TOPIC, 0);
Set<StreamPartition<Integer>> partitions = ImmutableSet.of(streamPartition);
KAFKA_SERVER.consumerProperties(), OBJECT_MAPPER, null, false);
StreamPartition<KafkaTopicPartition> streamPartition = StreamPartition.of(TOPIC, new KafkaTopicPartition(false, TOPIC, 0));
Set<StreamPartition<KafkaTopicPartition>> partitions = ImmutableSet.of(streamPartition);
recordSupplier.assign(partitions);
recordSupplier.seekToLatest(partitions);
Assert.assertEquals(new Long(0), recordSupplier.getEarliestSequenceNumber(streamPartition));

View File

@ -179,6 +179,7 @@ public class KafkaSamplerSpecTest extends InitializedNullHandlingTest
null,
null,
null,
false,
null
),
null,
@ -239,6 +240,7 @@ public class KafkaSamplerSpecTest extends InitializedNullHandlingTest
null,
null,
null,
false,
null
),
null,
@ -341,6 +343,7 @@ public class KafkaSamplerSpecTest extends InitializedNullHandlingTest
null,
null,
null,
false,
null
),
null,
@ -524,6 +527,7 @@ public class KafkaSamplerSpecTest extends InitializedNullHandlingTest
null,
null,
null,
false,
null
),
null,
@ -579,6 +583,7 @@ public class KafkaSamplerSpecTest extends InitializedNullHandlingTest
null,
null,
null,
false,
null
),
null,

View File

@ -308,6 +308,7 @@ public class KafkaSupervisorIOConfigTest
null,
null,
null,
false,
null
);
String ioConfig = mapper.writeValueAsString(kafkaSupervisorIOConfig);
@ -350,6 +351,7 @@ public class KafkaSupervisorIOConfigTest
null,
null,
mapper.convertValue(idleConfig, IdleConfig.class),
false,
null
);
String ioConfig = mapper.writeValueAsString(kafkaSupervisorIOConfig);