Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -26,12 +26,15 @@
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
* Filter that returns all partitions for the specified topics.
*/
public class NamedTopicFilter implements TopicFilter {

private static final Logger LOG = LoggerFactory.getLogger(NamedTopicFilter.class);
private final Set<String> topics;

/**
Expand All @@ -54,8 +57,13 @@ public NamedTopicFilter(String... topics) {
public List<TopicPartition> getFilteredTopicPartitions(KafkaConsumer<?, ?> consumer) {
List<TopicPartition> allPartitions = new ArrayList<>();
for (String topic : topics) {
for (PartitionInfo partitionInfo: consumer.partitionsFor(topic)) {
allPartitions.add(new TopicPartition(partitionInfo.topic(), partitionInfo.partition()));
List<PartitionInfo> partitionInfoList = consumer.partitionsFor(topic);
if(partitionInfoList != null) {
for (PartitionInfo partitionInfo : partitionInfoList) {
allPartitions.add(new TopicPartition(partitionInfo.topic(), partitionInfo.partition()));
}
} else {
LOG.warn("Topic {} not found, skipping addition of the topic", topic);
}
}
return allPartitions;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

import org.apache.storm.kafka.spout.NamedTopicFilter;

import static org.hamcrest.Matchers.contains;
import static org.hamcrest.Matchers.containsInAnyOrder;
import static org.junit.Assert.assertThat;
import static org.mockito.Mockito.mock;
Expand Down Expand Up @@ -63,6 +64,20 @@ public void testFilter() {

}

@Test
public void testFilterOnAbsentTopic() {
String presentTopic = "present";
String absentTopic = "absent";

NamedTopicFilter filter = new NamedTopicFilter(presentTopic, absentTopic);
when(consumerMock.partitionsFor(presentTopic)).thenReturn(Collections.singletonList(createPartitionInfo(presentTopic, 2)));
when(consumerMock.partitionsFor(absentTopic)).thenReturn(null);

List<TopicPartition> presentPartitions = filter.getFilteredTopicPartitions(consumerMock);
assertThat("Expected filter to pass only topics which are present", presentPartitions,
contains(new TopicPartition(presentTopic, 2)));
}

private PartitionInfo createPartitionInfo(String topic, int partition) {
return new PartitionInfo(topic, partition, null, null, null);
}
Expand Down