Skip to content

Commit

Permalink
[FLINK-3102] Allow reading from multiple topics with one FlinkKafkaCo…
Browse files Browse the repository at this point in the history
…nsumer instance

This closes apache#1437
  • Loading branch information
rmetzger committed Dec 8, 2015
1 parent 4dbb10f commit fc8be1c
Show file tree
Hide file tree
Showing 24 changed files with 822 additions and 517 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@
import java.util.Collection;
import java.util.List;

import org.apache.commons.lang.StringUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.flink.api.common.JobExecutionResult;
import org.apache.flink.api.common.accumulators.Accumulator;
import org.apache.flink.api.java.DataSet;
Expand Down

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

import org.apache.flink.streaming.util.serialization.DeserializationSchema;

import java.util.Collections;
import java.util.Properties;

/**
Expand Down Expand Up @@ -52,6 +53,6 @@ public class FlinkKafkaConsumer081<T> extends FlinkKafkaConsumer<T> {
* The properties used to configure the Kafka consumer client, and the ZooKeeper client.
*/
public FlinkKafkaConsumer081(String topic, DeserializationSchema<T> valueDeserializer, Properties props) {
super(topic, valueDeserializer, props, OffsetStore.FLINK_ZOOKEEPER, FetcherType.LEGACY_LOW_LEVEL);
super(Collections.singletonList(topic), valueDeserializer, props, OffsetStore.FLINK_ZOOKEEPER, FetcherType.LEGACY_LOW_LEVEL);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,8 @@
import org.apache.flink.streaming.util.serialization.DeserializationSchema;
import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;

import java.util.Collections;
import java.util.List;
import java.util.Properties;

/**
Expand Down Expand Up @@ -47,9 +49,12 @@ public class FlinkKafkaConsumer082<T> extends FlinkKafkaConsumer<T> {
* The properties used to configure the Kafka consumer client, and the ZooKeeper client.
*/
public FlinkKafkaConsumer082(String topic, DeserializationSchema<T> valueDeserializer, Properties props) {
super(topic, valueDeserializer, props, OffsetStore.FLINK_ZOOKEEPER, FetcherType.LEGACY_LOW_LEVEL);
super(Collections.singletonList(topic), valueDeserializer, props, OffsetStore.FLINK_ZOOKEEPER, FetcherType.LEGACY_LOW_LEVEL);
}


//----- key-value deserializer constructor

/**
* Creates a new Kafka 0.8.2.x streaming source consumer.
*
Expand All @@ -64,6 +69,17 @@ public FlinkKafkaConsumer082(String topic, DeserializationSchema<T> valueDeseria
* The properties used to configure the Kafka consumer client, and the ZooKeeper client.
*/
public FlinkKafkaConsumer082(String topic, KeyedDeserializationSchema<T> deserializer, Properties props) {
super(topic, deserializer, props, OffsetStore.FLINK_ZOOKEEPER, FetcherType.LEGACY_LOW_LEVEL);
super(Collections.singletonList(topic), deserializer, props, OffsetStore.FLINK_ZOOKEEPER, FetcherType.LEGACY_LOW_LEVEL);
}

//----- topic list constructors


public FlinkKafkaConsumer082(List<String> topics, DeserializationSchema<T> valueDeserializer, Properties props) {
super(topics, valueDeserializer, props, OffsetStore.FLINK_ZOOKEEPER, FetcherType.LEGACY_LOW_LEVEL);
}

public FlinkKafkaConsumer082(List<String> topics, KeyedDeserializationSchema<T> deserializer, Properties props) {
super(topics, deserializer, props, OffsetStore.FLINK_ZOOKEEPER, FetcherType.LEGACY_LOW_LEVEL);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -198,13 +198,13 @@ public FlinkKafkaProducer(String topicId, KeyedSerializationSchema<IN> serializa

// set the producer configuration properties.

if(!producerConfig.contains(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG)) {
if (!producerConfig.contains(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG)) {
this.producerConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getCanonicalName());
} else {
LOG.warn("Overwriting the '{}' is not recommended", ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG);
}

if(!producerConfig.contains(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG)) {
if (!producerConfig.contains(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG)) {
this.producerConfig.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getCanonicalName());
} else {
LOG.warn("Overwriting the '{}' is not recommended", ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG);
Expand Down

This file was deleted.

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -19,27 +19,19 @@

import org.apache.flink.streaming.api.functions.source.SourceFunction;
import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
import org.apache.kafka.common.TopicPartition;

import java.io.IOException;
import java.util.List;
import java.util.HashMap;

/**
* A fetcher pulls data from Kafka, from a fix set of partitions.
* The fetcher supports "seeking" inside the partitions, i.e., moving to a different offset.
*/
public interface Fetcher {

/**
* Set which partitions the fetcher should pull from.
*
* @param partitions The list of partitions for a topic that the fetcher will pull from.
*/
void setPartitionsToRead(List<TopicPartition> partitions);

/**
* Closes the fetcher. This will stop any operation in the
* {@link #run(SourceFunction.SourceContext, DeserializationSchema, long[])} method and eventually
* {@link #run(SourceFunction.SourceContext, KeyedDeserializationSchema, HashMap)} method and eventually
* close underlying connections and release all resources.
*/
void close() throws IOException;
Expand All @@ -61,15 +53,14 @@ public interface Fetcher {
* }
* }
* }</pre>
*
*
* @param <T> The type of elements produced by the fetcher and emitted to the source context.
* @param sourceContext The source context to emit elements to.
* @param valueDeserializer The deserializer to decode the raw values with.
* @param lastOffsets The array into which to store the offsets for which elements are emitted (operator state)
*
* @param <T> The type of elements produced by the fetcher and emitted to the source context.
* @param lastOffsets The map into which to store the offsets for which elements are emitted (operator state)
*/
<T> void run(SourceFunction.SourceContext<T> sourceContext, KeyedDeserializationSchema<T> valueDeserializer,
long[] lastOffsets) throws Exception;
HashMap<KafkaTopicPartition, Long> lastOffsets) throws Exception;

/**
* Set the next offset to read from for the given partition.
Expand All @@ -79,7 +70,7 @@ <T> void run(SourceFunction.SourceContext<T> sourceContext, KeyedDeserialization
* @param topicPartition The partition for which to seek the offset.
* @param offsetToRead To offset to seek to.
*/
void seek(TopicPartition topicPartition, long offsetToRead);
void seek(KafkaTopicPartition topicPartition, long offsetToRead);

/**
* Exit run loop with given error and release all resources.
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,124 @@
/*
* 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:https://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.flink.streaming.connectors.kafka.internals;

import java.io.Serializable;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Map;

import static com.google.common.base.Preconditions.checkNotNull;

/**
* A serializable representation of a kafka topic and a partition.
* Used as an operator state for the Kafka consumer
*/
public class KafkaTopicPartition implements Serializable {

private static final long serialVersionUID = 722083576322742325L;

private final String topic;
private final int partition;
private final int cachedHash;

public KafkaTopicPartition(String topic, int partition) {
this.topic = checkNotNull(topic);
this.partition = partition;
this.cachedHash = 31 * topic.hashCode() + partition;
}

public String getTopic() {
return topic;
}

public int getPartition() {
return partition;
}

@Override
public String toString() {
return "KafkaTopicPartition{" +
"topic='" + topic + '\'' +
", partition=" + partition +
'}';
}

@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (!(o instanceof KafkaTopicPartition)) {
return false;
}

KafkaTopicPartition that = (KafkaTopicPartition) o;

if (partition != that.partition) {
return false;
}
return topic.equals(that.topic);
}

@Override
public int hashCode() {
return cachedHash;
}


// ------------------- Utilities -------------------------------------

/**
* Returns a unique list of topics from the topic partition map
*
* @param topicPartitionMap A map of KafkaTopicPartition's
* @return A unique list of topics from the input map
*/
public static List<String> getTopics(Map<KafkaTopicPartition, ?> topicPartitionMap) {
HashSet<String> uniqueTopics = new HashSet<>();
for (KafkaTopicPartition ktp: topicPartitionMap.keySet()) {
uniqueTopics.add(ktp.getTopic());
}
return new ArrayList<>(uniqueTopics);
}

public static String toString(Map<KafkaTopicPartition, Long> map) {
StringBuilder sb = new StringBuilder();
for (Map.Entry<KafkaTopicPartition, Long> p: map.entrySet()) {
KafkaTopicPartition ktp = p.getKey();
sb.append(ktp.getTopic()).append(":").append(ktp.getPartition()).append("=").append(p.getValue()).append(", ");
}
return sb.toString();
}

/**
* Checks whether this partition is contained in the map with KafkaTopicPartitionLeaders
*
* @param map The map of KafkaTopicPartitionLeaders
* @return true if the element is contained.
*/
public boolean isContained(Map<KafkaTopicPartitionLeader, ?> map) {
for(Map.Entry<KafkaTopicPartitionLeader, ?> entry : map.entrySet()) {
if(entry.getKey().getTopicPartition().equals(this)) {
return true;
}
}
return false;
}
}
Loading

0 comments on commit fc8be1c

Please sign in to comment.