Skip to content
Merged
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

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -18,36 +18,29 @@

package org.apache.storm.kafka.spout;

import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;
import org.apache.storm.task.TopologyContext;

public class ManualPartitionNamedSubscription extends NamedSubscription {
public class ManualPartitionSubscription extends Subscription {
private static final long serialVersionUID = 5633018073527583826L;
private final ManualPartitioner partitioner;
private final TopicFilter partitionFilter;
private Set<TopicPartition> currentAssignment = null;
private KafkaConsumer<?, ?> consumer = null;
private ConsumerRebalanceListener listener = null;
private TopologyContext context = null;

public ManualPartitionNamedSubscription(ManualPartitioner parter, Collection<String> topics) {
super(topics);
public ManualPartitionSubscription(ManualPartitioner parter, TopicFilter partitionFilter) {
this.partitionFilter = partitionFilter;
this.partitioner = parter;
}

public ManualPartitionNamedSubscription(ManualPartitioner parter, String ... topics) {
this(parter, Arrays.asList(topics));
}

@Override
public <K, V> void subscribe(KafkaConsumer<K, V> consumer, ConsumerRebalanceListener listener, TopologyContext context) {
this.consumer = consumer;
Expand All @@ -58,21 +51,21 @@ public <K, V> void subscribe(KafkaConsumer<K, V> consumer, ConsumerRebalanceList

@Override
public void refreshAssignment() {
List<TopicPartition> allPartitions = new ArrayList<>();
for (String topic : topics) {
for (PartitionInfo partitionInfo: consumer.partitionsFor(topic)) {
allPartitions.add(new TopicPartition(partitionInfo.topic(), partitionInfo.partition()));
}
}
List<TopicPartition> allPartitions = partitionFilter.getFilteredTopicPartitions(consumer);
Collections.sort(allPartitions, TopicPartitionComparator.INSTANCE);
Set<TopicPartition> newAssignment = new HashSet<>(partitioner.partition(allPartitions, context));
if (!newAssignment.equals(currentAssignment)) {
consumer.assign(newAssignment);
if (currentAssignment != null) {
listener.onPartitionsRevoked(currentAssignment);
listener.onPartitionsAssigned(newAssignment);
}
currentAssignment = newAssignment;
consumer.assign(currentAssignment);
listener.onPartitionsAssigned(newAssignment);
}
}

@Override
public String getTopicsString() {
return partitionFilter.getTopicsString();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,67 @@
/*
* Copyright 2017 The Apache Software Foundation.
*
* Licensed 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.storm.kafka.spout;

import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;

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

private final Set<String> topics;

/**
* Create filter based on a set of topic names.
* @param topics The topic names the filter will pass.
*/
public NamedTopicFilter(Set<String> topics) {
this.topics = Collections.unmodifiableSet(topics);
}

/**
* Convenience constructor.
* @param topics The topic names the filter will pass.
*/
public NamedTopicFilter(String... topics) {
this(new HashSet<>(Arrays.asList(topics)));
}

@Override
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()));
}
}
return allPartitions;
}

@Override
public String getTopicsString() {
return String.join(",", topics);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,69 @@
/*
* Copyright 2017 The Apache Software Foundation.
*
* Licensed 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.storm.kafka.spout;

import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.regex.Pattern;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;

/**
* Filter that returns all partitions for topics matching the given {@link Pattern}.
*/
public class PatternTopicFilter implements TopicFilter {

private final Pattern pattern;
private final Set<String> topics = new HashSet<>();

/**
* Creates filter based on a Pattern. Only topic names matching the Pattern are passed by the filter.
*
* @param pattern The Pattern to use.
*/
public PatternTopicFilter(Pattern pattern) {
this.pattern = pattern;
}

@Override
public List<TopicPartition> getFilteredTopicPartitions(KafkaConsumer<?, ?> consumer) {
topics.clear();
List<TopicPartition> allPartitions = new ArrayList<>();
for (Map.Entry<String, List<PartitionInfo>> entry : consumer.listTopics().entrySet()) {
if (pattern.matcher(entry.getKey()).matches()) {
for (PartitionInfo partitionInfo : entry.getValue()) {
allPartitions.add(new TopicPartition(partitionInfo.topic(), partitionInfo.partition()));
topics.add(partitionInfo.topic());
}
}
}
return allPartitions;
}

@Override
public String getTopicsString() {
return String.join(",", topics);
}

public String getTopicsPattern() {
return pattern.pattern();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@ public abstract class Subscription implements Serializable {
public abstract <K, V> void subscribe(KafkaConsumer<K,V> consumer, ConsumerRebalanceListener listener, TopologyContext context);

/**
* @return a string representing the subscribed topics.
* @return A human-readable string representing the subscribed topics.
*/
public abstract String getTopicsString();

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,38 @@
/*
* Copyright 2017 The Apache Software Foundation.
*
* Licensed 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.storm.kafka.spout;

import java.io.Serializable;
import java.util.List;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.common.TopicPartition;

public interface TopicFilter extends Serializable {

/**
* Get the Kafka TopicPartitions passed by this filter.
* @param consumer The Kafka consumer to use to read the list of existing partitions
* @return The Kafka partitions passed by this filter.
*/
List<TopicPartition> getFilteredTopicPartitions(KafkaConsumer<?, ?> consumer);

/**
* @return A human-readable string representing the topics that pass the filter.
*/
String getTopicsString();

}
Loading