diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java
similarity index 60%
rename from flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer.java
rename to flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java
index 69ed9bfaa18ec..543e0ff1a5e2c 100644
--- a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer.java
+++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java
@@ -23,14 +23,8 @@
import kafka.javaapi.TopicMetadata;
import kafka.javaapi.TopicMetadataRequest;
import kafka.javaapi.consumer.SimpleConsumer;
-import org.apache.commons.collections.map.LinkedMap;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier;
-import org.apache.flink.streaming.api.checkpoint.CheckpointedAsynchronously;
-import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.connectors.kafka.internals.Fetcher;
@@ -52,40 +46,26 @@
import java.io.IOException;
import java.net.URL;
import java.util.ArrayList;
+import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.Random;
-import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkNotNull;
/**
* The Flink Kafka Consumer is a streaming data source that pulls a parallel data stream from
- * Apache Kafka. The consumer can run in multiple parallel instances, each of which will pull
+ * Apache Kafka 0.8.x. The consumer can run in multiple parallel instances, each of which will pull
* data from one or more Kafka partitions.
*
* The Flink Kafka Consumer participates in checkpointing and guarantees that no data is lost
* during a failure, and that the computation processes elements "exactly once".
* (Note: These guarantees naturally assume that Kafka itself does not loose any data.)
*
- * To support a variety of Kafka brokers, protocol versions, and offset committing approaches,
- * the Flink Kafka Consumer can be parametrized with a fetcher and an offset handler.
- *
- * Fetcher
- *
- * The fetcher is responsible to pull data from Kafka. Because Kafka has undergone a change in
- * protocols and APIs, there are currently two fetchers available:
- *
- *
*
- * Offset handler
+ * Offset handling
*
- * Offsets whose records have been read and are checkpointed will be committed back to Kafka / ZooKeeper
+ *
Offsets whose records have been read and are checkpointed will be committed back to ZooKeeper
* by the offset handler. In addition, the offset handler finds the point where the source initially
* starts reading from the stream, when the streaming job is started.
- *
- * Currently, the source offers two different offset handlers exist:
- *
- * - {@link OffsetStore#KAFKA}: Use this offset handler when the Kafka brokers are managing the offsets,
- * and hence offsets need to be committed the Kafka brokers, rather than to ZooKeeper.
- * Note that this offset handler works only on new versions of Kafka (0.8.2.x +) and
- * with the {@link FetcherType#NEW_HIGH_LEVEL} fetcher.
- *
- * - {@link OffsetStore#FLINK_ZOOKEEPER}: Use this offset handler when the offsets are managed
- * by ZooKeeper, as in older versions of Kafka (0.8.1.x)
- *
- *
+ *
* Please note that Flink snapshots the offsets internally as part of its distributed checkpoints. The offsets
* committed to Kafka / ZooKeeper are only to bring the outside view of progress in sync with Flink's view
* of the progress. That way, monitoring and other jobs can get a view of how far the Flink Kafka consumer
* has consumed a topic.
+ *
+ * If checkpointing is disabled, the consumer will periodically commit the current offset
+ * to Zookeeper.
+ *
+ * When using a Kafka topic to send data between Flink jobs, we recommend using the
+ * {@see TypeInformationSerializationSchema} and {@see TypeInformationKeyValueSerializationSchema}.
*
* NOTE: The implementation currently accesses partition metadata when the consumer
* is constructed. That means that the client that submits the program needs to be able to
* reach the Kafka brokers or ZooKeeper.
*/
-public class FlinkKafkaConsumer extends RichParallelSourceFunction
- implements CheckpointNotifier, CheckpointedAsynchronously>, ResultTypeQueryable {
-
- /**
- * The offset store defines how acknowledged offsets are committed back to Kafka. Different
- * options include letting Flink periodically commit to ZooKeeper, or letting Kafka manage the
- * offsets (new Kafka versions only).
- */
- public enum OffsetStore {
-
- /**
- * Let Flink manage the offsets. Flink will periodically commit them to Zookeeper (usually after
- * successful checkpoints), in the same structure as Kafka 0.8.2.x
- *
- * Use this mode when using the source with Kafka 0.8.1.x brokers.
- */
- FLINK_ZOOKEEPER,
-
- /**
- * Use the mechanisms in Kafka to commit offsets. Depending on the Kafka configuration, different
- * mechanism will be used (broker coordinator, zookeeper)
- */
- KAFKA
- }
-
- /**
- * The fetcher type defines which code paths to use to pull data from teh Kafka broker.
- */
- public enum FetcherType {
-
- /**
- * The legacy fetcher uses Kafka's old low-level consumer API.
- *
- * Use this fetcher for Kafka 0.8.1 brokers.
- */
- LEGACY_LOW_LEVEL,
-
- /**
- * This fetcher uses a backport of the new consumer API to pull data from the Kafka broker.
- * It is the fetcher that will be maintained in the future, and it already
- * handles certain failure cases with less overhead than the legacy fetcher.
- *
- * This fetcher works only Kafka 0.8.2 and 0.8.3 (and future versions).
- */
- NEW_HIGH_LEVEL
- }
+public class FlinkKafkaConsumer08 extends FlinkKafkaConsumerBase {
// ------------------------------------------------------------------------
private static final long serialVersionUID = -6272159445203409112L;
- private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaConsumer.class);
+ private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaConsumer08.class);
/** Magic number to define an unset offset. Negative offsets are not used by Kafka (invalid),
* and we pick a number that is probably (hopefully) not used by Kafka as a magic number for anything else. */
public static final long OFFSET_NOT_SET = -915623761776L;
- /** The maximum number of pending non-committed checkpoints to track, to avoid memory leaks */
- public static final int MAX_NUM_PENDING_CHECKPOINTS = 100;
/** Configuration key for the number of retries for getting the partition info */
public static final String GET_PARTITIONS_RETRIES_KEY = "flink.get-partitions.retry";
@@ -189,14 +117,7 @@ public enum FetcherType {
public static final int DEFAULT_GET_PARTITIONS_RETRIES = 3;
-
// ------ Configuration of the Consumer -------
-
- /** The offset store where this consumer commits safe offsets */
- private final OffsetStore offsetStore;
-
- /** The type of fetcher to be used to pull data from Kafka */
- private final FetcherType fetcherType;
/** List of partitions (including topics and leaders) to consume */
private final List partitionInfos;
@@ -204,14 +125,8 @@ public enum FetcherType {
/** The properties to parametrize the Kafka consumer and ZooKeeper client */
private final Properties props;
- /** The schema to convert between Kafka#s byte messages, and Flink's objects */
- private final KeyedDeserializationSchema deserializer;
-
// ------ Runtime State -------
-
- /** Data for pending but uncommitted checkpoints */
- private final LinkedMap pendingCheckpoints = new LinkedMap();
/** The fetcher used to pull data from the Kafka brokers */
private transient Fetcher fetcher;
@@ -222,84 +137,80 @@ public enum FetcherType {
/** The partitions actually handled by this consumer at runtime */
private transient List subscribedPartitions;
- /** The offsets of the last returned elements */
- private transient HashMap lastOffsets;
-
/** The latest offsets that have been committed to Kafka or ZooKeeper. These are never
* newer then the last offsets (Flink's internal view is fresher) */
private transient HashMap committedOffsets;
-
- /** The offsets to restore to, if the consumer restores state from a checkpoint */
- private transient HashMap restoreToOffset;
-
- private volatile boolean running = true;
-
+
// ------------------------------------------------------------------------
+ /**
+ * Creates a new Kafka streaming source consumer for Kafka 0.8.x
+ *
+ * @param topic
+ * The name of the topic that should be consumed.
+ * @param valueDeserializer
+ * The de-/serializer used to convert between Kafka's byte messages and Flink's objects.
+ * @param props
+ * The properties used to configure the Kafka consumer client, and the ZooKeeper client.
+ */
+ public FlinkKafkaConsumer08(String topic, DeserializationSchema valueDeserializer, Properties props) {
+ this(Collections.singletonList(topic), valueDeserializer, props);
+ }
/**
- * Creates a new Flink Kafka Consumer, using the given type of fetcher and offset handler.
+ * Creates a new Kafka streaming source consumer for Kafka 0.8.x
*
- * To determine which kink of fetcher and offset handler to use, please refer to the docs
- * at the beginning of this class.
+ * This constructor allows passing a {@see KeyedDeserializationSchema} for reading key/value
+ * pairs, offsets, and topic names from Kafka.
*
* @param topic
- * The Kafka topic to read from.
+ * The name of the topic that should be consumed.
+ * @param deserializer
+ * The keyed de-/serializer used to convert between Kafka's byte messages and Flink's objects.
+ * @param props
+ * The properties used to configure the Kafka consumer client, and the ZooKeeper client.
+ */
+ public FlinkKafkaConsumer08(String topic, KeyedDeserializationSchema deserializer, Properties props) {
+ this(Collections.singletonList(topic), deserializer, props);
+ }
+
+ /**
+ * Creates a new Kafka streaming source consumer for Kafka 0.8.x
+ *
+ * This constructor allows passing multiple topics to the consumer.
+ *
+ * @param topics
+ * The Kafka topics to read from.
* @param deserializer
- * The deserializer to turn raw byte messages (without key) into Java/Scala objects.
+ * The de-/serializer used to convert between Kafka's byte messages and Flink's objects.
* @param props
* The properties that are used to configure both the fetcher and the offset handler.
- * @param offsetStore
- * The type of offset store to use (Kafka / ZooKeeper)
- * @param fetcherType
- * The type of fetcher to use (new high-level API, old low-level API).
*/
- public FlinkKafkaConsumer(List topic, DeserializationSchema deserializer, Properties props,
- OffsetStore offsetStore, FetcherType fetcherType) {
- this(topic, new KeyedDeserializationSchemaWrapper<>(deserializer),
- props, offsetStore, fetcherType);
+ public FlinkKafkaConsumer08(List topics, DeserializationSchema deserializer, Properties props) {
+ this(topics, new KeyedDeserializationSchemaWrapper<>(deserializer), props);
}
/**
- * Creates a new Flink Kafka Consumer, using the given type of fetcher and offset handler.
- *
- * To determine which kink of fetcher and offset handler to use, please refer to the docs
- * at the beginning of this class.
+ * Creates a new Kafka streaming source consumer for Kafka 0.8.x
+ *
+ * This constructor allows passing multiple topics and a key/value deserialization schema.
*
* @param topics
* The Kafka topics to read from.
* @param deserializer
- * The deserializer to turn raw byte messages into Java/Scala objects.
+ * The keyed de-/serializer used to convert between Kafka's byte messages and Flink's objects.
* @param props
* The properties that are used to configure both the fetcher and the offset handler.
- * @param offsetStore
- * The type of offset store to use (Kafka / ZooKeeper)
- * @param fetcherType
- * The type of fetcher to use (new high-level API, old low-level API).
*/
- public FlinkKafkaConsumer(List topics, KeyedDeserializationSchema deserializer, Properties props,
- OffsetStore offsetStore, FetcherType fetcherType) {
- this.offsetStore = checkNotNull(offsetStore);
- this.fetcherType = checkNotNull(fetcherType);
-
- if (fetcherType == FetcherType.NEW_HIGH_LEVEL) {
- throw new UnsupportedOperationException("The fetcher for Kafka 0.8.3 / 0.9.0 is not yet " +
- "supported in Flink");
- }
- if (offsetStore == OffsetStore.KAFKA && fetcherType == FetcherType.LEGACY_LOW_LEVEL) {
- throw new IllegalArgumentException(
- "The Kafka offset handler cannot be used together with the old low-level fetcher.");
- }
-
+ public FlinkKafkaConsumer08(List topics, KeyedDeserializationSchema deserializer, Properties props) {
+ super(deserializer, props);
+
checkNotNull(topics, "topics");
this.props = checkNotNull(props, "props");
- this.deserializer = checkNotNull(deserializer, "valueDeserializer");
// validate the zookeeper properties
- if (offsetStore == OffsetStore.FLINK_ZOOKEEPER) {
- validateZooKeeperConfig(props);
- }
-
+ validateZooKeeperConfig(props);
+
// Connect to a broker to get the partitions for all topics
this.partitionInfos = getPartitionsForTopic(topics, props);
@@ -309,21 +220,7 @@ public FlinkKafkaConsumer(List topics, KeyedDeserializationSchema des
}
if (LOG.isInfoEnabled()) {
- Map countPerTopic = new HashMap<>();
- for (KafkaTopicPartitionLeader partition : partitionInfos) {
- Integer count = countPerTopic.get(partition.getTopicPartition().getTopic());
- if (count == null) {
- count = 1;
- } else {
- count++;
- }
- countPerTopic.put(partition.getTopicPartition().getTopic(), count);
- }
- StringBuilder sb = new StringBuilder();
- for (Map.Entry e : countPerTopic.entrySet()) {
- sb.append(e.getKey()).append(" (").append(e.getValue()).append("), ");
- }
- LOG.info("Consumer is going to read the following topics (with number of partitions): ", sb.toString());
+ logPartitionInfo(KafkaTopicPartition.convertToPartitionInfo(partitionInfos));
}
}
@@ -349,31 +246,16 @@ public void open(Configuration parameters) throws Exception {
// we leave the fetcher as null, if we have no partitions
if (subscribedPartitions.isEmpty()) {
LOG.info("Kafka consumer {} has no partitions (empty source)", thisConsumerIndex);
+ this.fetcher = null; // fetcher remains null
return;
}
// create fetcher
- switch (fetcherType){
- case NEW_HIGH_LEVEL:
- throw new UnsupportedOperationException("Currently unsupported");
- case LEGACY_LOW_LEVEL:
- fetcher = new LegacyFetcher(this.subscribedPartitions, props, getRuntimeContext().getTaskName());
- break;
- default:
- throw new RuntimeException("Requested unknown fetcher " + fetcher);
- }
+ fetcher = new LegacyFetcher(this.subscribedPartitions, props, getRuntimeContext().getTaskName());
// offset handling
- switch (offsetStore){
- case FLINK_ZOOKEEPER:
- offsetHandler = new ZookeeperOffsetHandler(props);
- break;
- case KAFKA:
- throw new Exception("Kafka offset handler cannot work with legacy fetcher");
- default:
- throw new RuntimeException("Requested unknown offset store " + offsetStore);
- }
-
+ offsetHandler = new ZookeeperOffsetHandler(props);
+
committedOffsets = new HashMap<>();
// seek to last known pos, from restore request
@@ -390,12 +272,12 @@ public void open(Configuration parameters) throws Exception {
fetcher.seek(restorePartition.getKey(), restorePartition.getValue() + 1);
}
// initialize offsets with restored state
- this.lastOffsets = restoreToOffset;
+ this.offsetsState = restoreToOffset;
restoreToOffset = null;
}
else {
// start with empty offsets
- lastOffsets = new HashMap<>();
+ offsetsState = new HashMap<>();
// no restore request. Let the offset handler take care of the initial offset seeking
offsetHandler.seekFetcherToInitialOffsets(subscribedPartitions, fetcher);
@@ -422,7 +304,7 @@ public void run(SourceContext sourceContext) throws Exception {
}
try {
- fetcher.run(sourceContext, deserializer, lastOffsets);
+ fetcher.run(sourceContext, deserializer, offsetsState);
} finally {
if (offsetCommitter != null) {
offsetCommitter.close();
@@ -496,129 +378,60 @@ public void close() throws Exception {
super.close();
}
- @Override
- public TypeInformation getProducedType() {
- return deserializer.getProducedType();
- }
-
// ------------------------------------------------------------------------
// Checkpoint and restore
// ------------------------------------------------------------------------
- @Override
- public HashMap snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
- if (lastOffsets == null) {
- LOG.debug("snapshotState() requested on not yet opened source; returning null.");
- return null;
- }
- if (!running) {
- LOG.debug("snapshotState() called on closed source");
- return null;
- }
-
- if (LOG.isDebugEnabled()) {
- LOG.debug("Snapshotting state. Offsets: {}, checkpoint id: {}, timestamp: {}",
- KafkaTopicPartition.toString(lastOffsets), checkpointId, checkpointTimestamp);
+ /**
+ * Utility method to commit offsets.
+ *
+ * @param toCommit the offsets to commit
+ * @throws Exception
+ */
+ protected void commitOffsets(HashMap toCommit) throws Exception {
+ Map offsetsToCommit = new HashMap<>();
+ for (KafkaTopicPartitionLeader tp : this.subscribedPartitions) {
+ Long offset = toCommit.get(tp.getTopicPartition());
+ if(offset == null) {
+ // There was no data ever consumed from this topic, that's why there is no entry
+ // for this topicPartition in the map.
+ continue;
+ }
+ Long lastCommitted = this.committedOffsets.get(tp.getTopicPartition());
+ if (lastCommitted == null) {
+ lastCommitted = OFFSET_NOT_SET;
+ }
+ if (offset != OFFSET_NOT_SET) {
+ if (offset > lastCommitted) {
+ offsetsToCommit.put(tp.getTopicPartition(), offset);
+ this.committedOffsets.put(tp.getTopicPartition(), offset);
+ LOG.debug("Committing offset {} for partition {}", offset, tp.getTopicPartition());
+ } else {
+ LOG.debug("Ignoring offset {} for partition {} because it is already committed", offset, tp.getTopicPartition());
+ }
+ }
}
- // the use of clone() is okay here is okay, we just need a new map, the keys are not changed
- @SuppressWarnings("unchecked")
- HashMap currentOffsets = (HashMap) lastOffsets.clone();
-
- // the map cannot be asynchronously updated, because only one checkpoint call can happen
- // on this function at a time: either snapshotState() or notifyCheckpointComplete()
- pendingCheckpoints.put(checkpointId, currentOffsets);
-
- while (pendingCheckpoints.size() > MAX_NUM_PENDING_CHECKPOINTS) {
- pendingCheckpoints.remove(0);
+ if (LOG.isDebugEnabled() && offsetsToCommit.size() > 0) {
+ LOG.debug("Committing offsets {} to Zookeeper", KafkaTopicPartition.toString(offsetsToCommit));
}
- return currentOffsets;
+ this.offsetHandler.commit(offsetsToCommit);
}
- @Override
- public void restoreState(HashMap restoredOffsets) {
- restoreToOffset = restoredOffsets;
- }
-
- @Override
- public void notifyCheckpointComplete(long checkpointId) throws Exception {
- if (fetcher == null) {
- LOG.debug("notifyCheckpointComplete() called on uninitialized source");
- return;
- }
- if (!running) {
- LOG.debug("notifyCheckpointComplete() called on closed source");
- return;
- }
-
- // only one commit operation must be in progress
- if (LOG.isDebugEnabled()) {
- LOG.debug("Committing offsets externally for checkpoint {}", checkpointId);
- }
-
- try {
- HashMap checkpointOffsets;
-
- // the map may be asynchronously updates when snapshotting state, so we synchronize
- synchronized (pendingCheckpoints) {
- final int posInMap = pendingCheckpoints.indexOf(checkpointId);
- if (posInMap == -1) {
- LOG.warn("Received confirmation for unknown checkpoint id {}", checkpointId);
- return;
- }
-
- //noinspection unchecked
- checkpointOffsets = (HashMap) pendingCheckpoints.remove(posInMap);
-
- // remove older checkpoints in map
- for (int i = 0; i < posInMap; i++) {
- pendingCheckpoints.remove(0);
- }
- }
- if (checkpointOffsets == null || checkpointOffsets.size() == 0) {
- LOG.info("Checkpoint state was empty.");
- return;
- }
- commitOffsets(checkpointOffsets, this);
- }
- catch (Exception e) {
- if (running) {
- throw e;
- }
- // else ignore exception if we are no longer running
- }
- }
-
// ------------------------------------------------------------------------
// Miscellaneous utilities
// ------------------------------------------------------------------------
- protected static List assignPartitions(List partitions, int numConsumers, int consumerIndex) {
- checkArgument(numConsumers > 0);
- checkArgument(consumerIndex < numConsumers);
-
- List partitionsToSub = new ArrayList<>();
-
- for (int i = 0; i < partitions.size(); i++) {
- if (i % numConsumers == consumerIndex) {
- partitionsToSub.add(partitions.get(i));
- }
- }
- return partitionsToSub;
- }
-
/**
* Thread to periodically commit the current read offset into Zookeeper.
*/
private static class PeriodicOffsetCommitter extends Thread {
-
private final long commitInterval;
- private final FlinkKafkaConsumer consumer;
-
+ private final FlinkKafkaConsumer08 consumer;
private volatile boolean running = true;
- public PeriodicOffsetCommitter(long commitInterval, FlinkKafkaConsumer consumer) {
+ public PeriodicOffsetCommitter(long commitInterval, FlinkKafkaConsumer08 consumer) {
this.commitInterval = commitInterval;
this.consumer = consumer;
}
@@ -626,15 +439,16 @@ public PeriodicOffsetCommitter(long commitInterval, FlinkKafkaConsumer consum
@Override
public void run() {
try {
+
while (running) {
try {
Thread.sleep(commitInterval);
// ------------ commit current offsets ----------------
// create copy of current offsets
- //noinspection unchecked
- HashMap currentOffsets = (HashMap) consumer.lastOffsets.clone();
- commitOffsets(currentOffsets, this.consumer);
+ @SuppressWarnings("unchecked")
+ HashMap currentOffsets = (HashMap) consumer.offsetsState.clone();
+ consumer.commitOffsets(currentOffsets);
} catch (InterruptedException e) {
if (running) {
// throw unexpected interruption
@@ -655,45 +469,7 @@ public void close() {
}
- /**
- * Utility method to commit offsets.
- *
- * @param toCommit the offsets to commit
- * @param consumer consumer reference
- * @param message type
- * @throws Exception
- */
- private static void commitOffsets(HashMap toCommit, FlinkKafkaConsumer consumer) throws Exception {
- Map offsetsToCommit = new HashMap<>();
- for (KafkaTopicPartitionLeader tp : consumer.subscribedPartitions) {
- Long offset = toCommit.get(tp.getTopicPartition());
- if(offset == null) {
- // There was no data ever consumed from this topic, that's why there is no entry
- // for this topicPartition in the map.
- continue;
- }
- Long lastCommitted = consumer.committedOffsets.get(tp.getTopicPartition());
- if (lastCommitted == null) {
- lastCommitted = OFFSET_NOT_SET;
- }
- if (offset != OFFSET_NOT_SET) {
- if (offset > lastCommitted) {
- offsetsToCommit.put(tp.getTopicPartition(), offset);
- consumer.committedOffsets.put(tp.getTopicPartition(), offset);
- LOG.debug("Committing offset {} for partition {}", offset, tp.getTopicPartition());
- } else {
- LOG.debug("Ignoring offset {} for partition {} because it is already committed", offset, tp.getTopicPartition());
- }
- }
- }
- if (LOG.isDebugEnabled() && offsetsToCommit.size() > 0) {
- LOG.debug("Committing offsets {} to offset store: {}", KafkaTopicPartition.toString(offsetsToCommit), consumer.offsetStore);
- }
-
- consumer.offsetHandler.commit(offsetsToCommit);
- }
-
// ------------------------------------------------------------------------
// Kafka / ZooKeeper communication utilities
// ------------------------------------------------------------------------
@@ -763,7 +539,8 @@ public static List getPartitionsForTopic(final ListThe following additional configuration values are available:
- *
- * - socket.timeout.ms
- * - socket.receive.buffer.bytes
- * - fetch.message.max.bytes
- * - auto.offset.reset with the values "latest", "earliest" (unlike 0.8.2 behavior)
- * - fetch.wait.max.ms
- *
- *
- * @param The type of elements produced by this consumer.
+ * THIS CLASS IS DEPRECATED. Use FlinkKafkaConsumer08 instead.
*/
-public class FlinkKafkaConsumer081 extends FlinkKafkaConsumer {
+@Deprecated
+public class FlinkKafkaConsumer081 extends FlinkKafkaConsumer08 {
private static final long serialVersionUID = -5649906773771949146L;
/**
- * Creates a new Kafka 0.8.1.x streaming source consumer.
- *
- * @param topic
- * The name of the topic that should be consumed.
- * @param valueDeserializer
- * The de-/serializer used to convert between Kafka's byte messages and Flink's objects.
- * @param props
- * The properties used to configure the Kafka consumer client, and the ZooKeeper client.
+ * THIS CONSTRUCTOR IS DEPRECATED. Use FlinkKafkaConsumer08 instead.
*/
+ @Deprecated
public FlinkKafkaConsumer081(String topic, DeserializationSchema valueDeserializer, Properties props) {
- super(Collections.singletonList(topic), valueDeserializer, props, OffsetStore.FLINK_ZOOKEEPER, FetcherType.LEGACY_LOW_LEVEL);
+ super(topic, valueDeserializer, props);
}
}
+
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer082.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer082.java
new file mode 100644
index 0000000000000..0520336a585f3
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer082.java
@@ -0,0 +1,39 @@
+/*
+ * 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.flink.streaming.connectors.kafka;
+
+import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+
+import java.util.Properties;
+
+/**
+ * THIS CLASS IS DEPRECATED. Use FlinkKafkaConsumer08 instead.
+ */
+@Deprecated
+public class FlinkKafkaConsumer082 extends FlinkKafkaConsumer08 {
+
+ private static final long serialVersionUID = -5649906773771949146L;
+
+ /**
+ * THIS CONSTRUCTOR IS DEPRECATED. Use FlinkKafkaConsumer08 instead.
+ */
+ @Deprecated
+ public FlinkKafkaConsumer082(String topic, DeserializationSchema valueDeserializer, Properties props) {
+ super(topic, valueDeserializer, props);
+ }
+}
+
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java
new file mode 100644
index 0000000000000..1c2e0b7de8bf0
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka;
+
+import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
+import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
+import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
+import org.apache.flink.streaming.util.serialization.SerializationSchema;
+import java.util.Properties;
+
+
+/**
+ * THIS CLASS IS DEPRECATED. Use FlinkKafkaProducer08 instead.
+ */
+@Deprecated
+public class FlinkKafkaProducer extends FlinkKafkaProducer08 {
+
+ @Deprecated
+ public FlinkKafkaProducer(String brokerList, String topicId, SerializationSchema serializationSchema) {
+ super(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), null);
+ }
+
+ @Deprecated
+ public FlinkKafkaProducer(String topicId, SerializationSchema serializationSchema, Properties producerConfig) {
+ super(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, null);
+ }
+
+ @Deprecated
+ public FlinkKafkaProducer(String topicId, SerializationSchema serializationSchema, Properties producerConfig, KafkaPartitioner customPartitioner) {
+ super(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner);
+
+ }
+
+ @Deprecated
+ public FlinkKafkaProducer(String brokerList, String topicId, KeyedSerializationSchema serializationSchema) {
+ super(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), null);
+ }
+
+ @Deprecated
+ public FlinkKafkaProducer(String topicId, KeyedSerializationSchema serializationSchema, Properties producerConfig) {
+ super(topicId, serializationSchema, producerConfig, null);
+ }
+
+ @Deprecated
+ public FlinkKafkaProducer(String topicId, KeyedSerializationSchema serializationSchema, Properties producerConfig, KafkaPartitioner customPartitioner) {
+ super(topicId, serializationSchema, producerConfig, customPartitioner);
+ }
+
+}
\ No newline at end of file
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer08.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer08.java
new file mode 100644
index 0000000000000..4975f9ab4744c
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer08.java
@@ -0,0 +1,128 @@
+/*
+ * 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.flink.streaming.connectors.kafka;
+
+import org.apache.flink.streaming.connectors.kafka.partitioner.FixedPartitioner;
+import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
+import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
+import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
+import org.apache.flink.streaming.util.serialization.SerializationSchema;
+
+import java.util.Properties;
+
+
+/**
+ * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.8.
+ *
+ * Please note that this producer does not have any reliability guarantees.
+ *
+ * @param Type of the messages to write into Kafka.
+ */
+public class FlinkKafkaProducer08 extends FlinkKafkaProducerBase {
+
+ private static final long serialVersionUID = 1L;
+
+ // ------------------- Keyless serialization schema constructors ----------------------
+ /**
+ * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to
+ * the topic.
+ *
+ * @param brokerList
+ * Comma separated addresses of the brokers
+ * @param topicId
+ * ID of the Kafka topic.
+ * @param serializationSchema
+ * User defined (keyless) serialization schema.
+ */
+ public FlinkKafkaProducer08(String brokerList, String topicId, SerializationSchema serializationSchema) {
+ this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), new FixedPartitioner());
+ }
+
+ /**
+ * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to
+ * the topic.
+ *
+ * @param topicId
+ * ID of the Kafka topic.
+ * @param serializationSchema
+ * User defined (keyless) serialization schema.
+ * @param producerConfig
+ * Properties with the producer configuration.
+ */
+ public FlinkKafkaProducer08(String topicId, SerializationSchema serializationSchema, Properties producerConfig) {
+ this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FixedPartitioner());
+ }
+
+ /**
+ * The main constructor for creating a FlinkKafkaProducer.
+ *
+ * @param topicId The topic to write data to
+ * @param serializationSchema A (keyless) serializable serialization schema for turning user objects into a kafka-consumable byte[]
+ * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
+ * @param customPartitioner A serializable partitioner for assining messages to Kafka partitions.
+ */
+ public FlinkKafkaProducer08(String topicId, SerializationSchema serializationSchema, Properties producerConfig, KafkaPartitioner customPartitioner) {
+ this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner);
+
+ }
+
+ // ------------------- Key/Value serialization schema constructors ----------------------
+
+ /**
+ * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to
+ * the topic.
+ *
+ * @param brokerList
+ * Comma separated addresses of the brokers
+ * @param topicId
+ * ID of the Kafka topic.
+ * @param serializationSchema
+ * User defined serialization schema supporting key/value messages
+ */
+ public FlinkKafkaProducer08(String brokerList, String topicId, KeyedSerializationSchema serializationSchema) {
+ this(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), new FixedPartitioner());
+ }
+
+ /**
+ * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to
+ * the topic.
+ *
+ * @param topicId
+ * ID of the Kafka topic.
+ * @param serializationSchema
+ * User defined serialization schema supporting key/value messages
+ * @param producerConfig
+ * Properties with the producer configuration.
+ */
+ public FlinkKafkaProducer08(String topicId, KeyedSerializationSchema serializationSchema, Properties producerConfig) {
+ this(topicId, serializationSchema, producerConfig, new FixedPartitioner());
+ }
+
+ /**
+ * The main constructor for creating a FlinkKafkaProducer.
+ *
+ * @param topicId The topic to write data to
+ * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
+ * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
+ * @param customPartitioner A serializable partitioner for assining messages to Kafka partitions.
+ */
+ public FlinkKafkaProducer08(String topicId, KeyedSerializationSchema serializationSchema, Properties producerConfig, KafkaPartitioner customPartitioner) {
+ super(topicId, serializationSchema, producerConfig, customPartitioner);
+ }
+
+}
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Fetcher.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Fetcher.java
similarity index 100%
rename from flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Fetcher.java
rename to flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Fetcher.java
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java
similarity index 98%
rename from flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java
rename to flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java
index b51ad6166924f..9fec52d3e9671 100644
--- a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java
+++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java
@@ -29,7 +29,7 @@
import kafka.message.MessageAndOffset;
import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08;
import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
import org.apache.flink.util.StringUtils;
@@ -90,7 +90,7 @@ public LegacyFetcher(List partitions, Properties prop
//this.topic = checkNotNull(topic, "The topic cannot be null");
this.partitionsToRead = new HashMap<>();
for (KafkaTopicPartitionLeader p: partitions) {
- partitionsToRead.put(p, FlinkKafkaConsumer.OFFSET_NOT_SET);
+ partitionsToRead.put(p, FlinkKafkaConsumer08.OFFSET_NOT_SET);
}
this.taskName = taskName;
this.error = new AtomicReference<>();
@@ -370,7 +370,7 @@ public void run() {
List partitionsToGetOffsetsFor = new ArrayList<>();
for (FetchPartition fp : partitions) {
- if (fp.nextOffsetToRead == FlinkKafkaConsumer.OFFSET_NOT_SET) {
+ if (fp.nextOffsetToRead == FlinkKafkaConsumer08.OFFSET_NOT_SET) {
// retrieve the offset from the consumer
partitionsToGetOffsetsFor.add(fp);
}
@@ -383,7 +383,7 @@ public void run() {
// Now, the actual work starts :-)
int offsetOutOfRangeCount = 0;
- while (running) {
+ fetchLoop: while (running) {
FetchRequestBuilder frb = new FetchRequestBuilder();
frb.clientId(clientId);
frb.maxWait(maxWait);
@@ -471,7 +471,11 @@ public void run() {
keyPayload.get(keyBytes);
}
- final T value = deserializer.deserialize(keyBytes, valueBytes, fp.topic, offset);
+ final T value = deserializer.deserialize(keyBytes, valueBytes, fp.topic, fp.partition, offset);
+ if(deserializer.isEndOfStream(value)) {
+ running = false;
+ break fetchLoop; // leave running loop
+ }
synchronized (sourceContext.getCheckpointLock()) {
sourceContext.collect(value);
offsetsState.put(topicPartition, offset);
@@ -586,7 +590,7 @@ private static class PartitionInfoFetcher extends Thread {
@Override
public void run() {
try {
- result = FlinkKafkaConsumer.getPartitionsForTopic(topics, properties);
+ result = FlinkKafkaConsumer08.getPartitionsForTopic(topics, properties);
}
catch (Throwable t) {
this.error = t;
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/OffsetHandler.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/OffsetHandler.java
similarity index 100%
rename from flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/OffsetHandler.java
rename to flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/OffsetHandler.java
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PartitionerWrapper.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PartitionerWrapper.java
similarity index 100%
rename from flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PartitionerWrapper.java
rename to flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PartitionerWrapper.java
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java
similarity index 96%
rename from flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java
rename to flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java
index f72117d444941..1eca4dd540c73 100644
--- a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java
+++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java
@@ -24,7 +24,7 @@
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.CuratorFrameworkFactory;
import org.apache.curator.retry.ExponentialBackoffRetry;
-import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.slf4j.Logger;
@@ -35,11 +35,14 @@
import java.util.Map;
import java.util.Properties;
+/**
+ * Handler for committing Kafka offsets to Zookeeper and to retrieve them again.
+ */
public class ZookeeperOffsetHandler implements OffsetHandler {
private static final Logger LOG = LoggerFactory.getLogger(ZookeeperOffsetHandler.class);
- private static final long OFFSET_NOT_SET = FlinkKafkaConsumer.OFFSET_NOT_SET;
+ private static final long OFFSET_NOT_SET = FlinkKafkaConsumer08.OFFSET_NOT_SET;
private final String groupId;
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java
new file mode 100644
index 0000000000000..26e31f59987ed
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java
@@ -0,0 +1,266 @@
+/*
+ * 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.flink.streaming.connectors.kafka;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.flink.api.java.functions.FlatMapIterator;
+import org.apache.flink.api.java.operators.DataSource;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler;
+import org.junit.Test;
+
+import java.util.Iterator;
+import java.util.Properties;
+
+import static org.junit.Assert.assertTrue;
+
+
+public class Kafka08ITCase extends KafkaConsumerTestBase {
+
+ // ------------------------------------------------------------------------
+ // Suite of Tests
+ // ------------------------------------------------------------------------
+
+ @Test(timeout = 60000)
+ public void testCheckpointing() throws Exception {
+ runCheckpointingTest();
+ }
+
+ @Test(timeout = 60000)
+ public void testFailOnNoBroker() throws Exception {
+ runFailOnNoBrokerTest();
+ }
+
+
+ @Test(timeout = 60000)
+ public void testConcurrentProducerConsumerTopology() throws Exception {
+ runSimpleConcurrentProducerConsumerTopology();
+ }
+
+ @Test(timeout = 60000)
+ public void testKeyValueSupport() throws Exception {
+ runKeyValueTest();
+ }
+
+ // --- canceling / failures ---
+
+ @Test(timeout = 60000)
+ public void testCancelingEmptyTopic() throws Exception {
+ runCancelingOnEmptyInputTest();
+ }
+
+ @Test(timeout = 60000)
+ public void testCancelingFullTopic() throws Exception {
+ runCancelingOnFullInputTest();
+ }
+
+ @Test(timeout = 60000)
+ public void testFailOnDeploy() throws Exception {
+ runFailOnDeployTest();
+ }
+
+ @Test(timeout = 60000)
+ public void testInvalidOffset() throws Exception {
+ final String topic = "invalidOffsetTopic";
+ final int parallelism = 1;
+
+ // create topic
+ createTestTopic(topic, parallelism, 1);
+
+ final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+
+ // write 20 messages into topic:
+ writeSequence(env, topic, 20, parallelism);
+
+ // set invalid offset:
+ CuratorFramework curatorClient = ((KafkaTestEnvironmentImpl)kafkaServer).createCuratorClient();
+ ZookeeperOffsetHandler.setOffsetInZooKeeper(curatorClient, standardCC.groupId(), topic, 0, 1234);
+ curatorClient.close();
+
+ // read from topic
+ final int valuesCount = 20;
+ final int startFrom = 0;
+ readSequence(env, standardCC.props().props(), parallelism, topic, valuesCount, startFrom);
+
+ deleteTestTopic(topic);
+ }
+
+ // --- source to partition mappings and exactly once ---
+
+ @Test(timeout = 60000)
+ public void testOneToOneSources() throws Exception {
+ runOneToOneExactlyOnceTest();
+ }
+
+ @Test(timeout = 60000)
+ public void testOneSourceMultiplePartitions() throws Exception {
+ runOneSourceMultiplePartitionsExactlyOnceTest();
+ }
+
+ @Test(timeout = 60000)
+ public void testMultipleSourcesOnePartition() throws Exception {
+ runMultipleSourcesOnePartitionExactlyOnceTest();
+ }
+
+ // --- broker failure ---
+
+ @Test(timeout = 60000)
+ public void testBrokerFailure() throws Exception {
+ runBrokerFailureTest();
+ }
+
+ // --- special executions ---
+
+ @Test(timeout = 60000)
+ public void testBigRecordJob() throws Exception {
+ runBigRecordTestTopology();
+ }
+
+ @Test(timeout = 60000)
+ public void testMultipleTopics() throws Exception {
+ runConsumeMultipleTopics();
+ }
+
+ @Test(timeout = 60000)
+ public void testAllDeletes() throws Exception {
+ runAllDeletesTest();
+ }
+
+ @Test(timeout=60000)
+ public void testMetricsAndEndOfStream() throws Exception {
+ runMetricsAndEndOfStreamTest();
+ }
+
+
+ /**
+ * Tests that offsets are properly committed to ZooKeeper and initial offsets are read from ZooKeeper.
+ *
+ * This test is only applicable if the Flink Kafka Consumer uses the ZooKeeperOffsetHandler.
+ */
+ @Test(timeout = 60000)
+ public void testOffsetInZookeeper() throws Exception {
+ final String topicName = "testOffsetInZK";
+ final int parallelism = 3;
+
+ createTestTopic(topicName, parallelism, 1);
+
+ StreamExecutionEnvironment env1 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+ env1.getConfig().disableSysoutLogging();
+ env1.enableCheckpointing(50);
+ env1.setNumberOfExecutionRetries(0);
+ env1.setParallelism(parallelism);
+
+ StreamExecutionEnvironment env2 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+ env2.getConfig().disableSysoutLogging();
+ env2.enableCheckpointing(50);
+ env2.setNumberOfExecutionRetries(0);
+ env2.setParallelism(parallelism);
+
+ StreamExecutionEnvironment env3 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+ env3.getConfig().disableSysoutLogging();
+ env3.enableCheckpointing(50);
+ env3.setNumberOfExecutionRetries(0);
+ env3.setParallelism(parallelism);
+
+ // write a sequence from 0 to 99 to each of the 3 partitions.
+ writeSequence(env1, topicName, 100, parallelism);
+
+ readSequence(env2, standardProps, parallelism, topicName, 100, 0);
+
+ CuratorFramework curatorClient = ((KafkaTestEnvironmentImpl)kafkaServer).createCuratorClient();
+
+ long o1 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorClient, standardCC.groupId(), topicName, 0);
+ long o2 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorClient, standardCC.groupId(), topicName, 1);
+ long o3 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorClient, standardCC.groupId(), topicName, 2);
+
+ LOG.info("Got final offsets from zookeeper o1={}, o2={}, o3={}", o1, o2, o3);
+
+ assertTrue(o1 == FlinkKafkaConsumer08.OFFSET_NOT_SET || (o1 >= 0 && o1 <= 100));
+ assertTrue(o2 == FlinkKafkaConsumer08.OFFSET_NOT_SET || (o2 >= 0 && o2 <= 100));
+ assertTrue(o3 == FlinkKafkaConsumer08.OFFSET_NOT_SET || (o3 >= 0 && o3 <= 100));
+
+ LOG.info("Manipulating offsets");
+
+ // set the offset to 50 for the three partitions
+ ZookeeperOffsetHandler.setOffsetInZooKeeper(curatorClient, standardCC.groupId(), topicName, 0, 49);
+ ZookeeperOffsetHandler.setOffsetInZooKeeper(curatorClient, standardCC.groupId(), topicName, 1, 49);
+ ZookeeperOffsetHandler.setOffsetInZooKeeper(curatorClient, standardCC.groupId(), topicName, 2, 49);
+
+ curatorClient.close();
+
+ // create new env
+ readSequence(env3, standardProps, parallelism, topicName, 50, 50);
+
+ deleteTestTopic(topicName);
+ }
+
+ @Test(timeout = 60000)
+ public void testOffsetAutocommitTest() throws Exception {
+ final String topicName = "testOffsetAutocommit";
+ final int parallelism = 3;
+
+ createTestTopic(topicName, parallelism, 1);
+
+ StreamExecutionEnvironment env1 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+ env1.getConfig().disableSysoutLogging();
+ env1.setNumberOfExecutionRetries(0);
+ env1.setParallelism(parallelism);
+
+ StreamExecutionEnvironment env2 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+ // NOTE: We are not enabling the checkpointing!
+ env2.getConfig().disableSysoutLogging();
+ env2.setNumberOfExecutionRetries(0);
+ env2.setParallelism(parallelism);
+
+
+ // write a sequence from 0 to 99 to each of the 3 partitions.
+ writeSequence(env1, topicName, 100, parallelism);
+
+
+ // the readSequence operation sleeps for 20 ms between each record.
+ // setting a delay of 25*20 = 500 for the commit interval makes
+ // sure that we commit roughly 3-4 times while reading, however
+ // at least once.
+ Properties readProps = new Properties();
+ readProps.putAll(standardProps);
+ readProps.setProperty("auto.commit.interval.ms", "500");
+
+ // read so that the offset can be committed to ZK
+ readSequence(env2, readProps, parallelism, topicName, 100, 0);
+
+ // get the offset
+ CuratorFramework curatorFramework = ((KafkaTestEnvironmentImpl)kafkaServer).createCuratorClient();
+
+ long o1 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorFramework, standardCC.groupId(), topicName, 0);
+ long o2 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorFramework, standardCC.groupId(), topicName, 1);
+ long o3 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorFramework, standardCC.groupId(), topicName, 2);
+
+ LOG.info("Got final offsets from zookeeper o1={}, o2={}, o3={}", o1, o2, o3);
+
+ // ensure that the offset has been committed
+ assertTrue("Offset of o1=" + o1 + " was not in range", o1 > 0 && o1 <= 100);
+ assertTrue("Offset of o2=" + o2 + " was not in range", o2 > 0 && o2 <= 100);
+ assertTrue("Offset of o3=" + o3 + " was not in range", o3 > 0 && o3 <= 100);
+
+ deleteTestTopic(topicName);
+ }
+}
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ProducerITCase.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ProducerITCase.java
new file mode 100644
index 0000000000000..fc137198c3272
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ProducerITCase.java
@@ -0,0 +1,33 @@
+/*
+ * 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.flink.streaming.connectors.kafka;
+
+
+import org.junit.Test;
+
+
+@SuppressWarnings("serial")
+public class Kafka08ProducerITCase extends KafkaProducerTestBase {
+
+ @Test
+ public void testCustomPartitioning() {
+ runCustomPartitioningTest();
+ }
+
+}
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTest.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTest.java
similarity index 83%
rename from flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTest.java
rename to flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTest.java
index efae9228acfa4..113ad714883a9 100644
--- a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTest.java
+++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTest.java
@@ -20,7 +20,6 @@
import org.apache.commons.collections.map.LinkedMap;
-import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
import org.apache.kafka.clients.consumer.ConsumerConfig;
@@ -44,7 +43,7 @@ public void testValidateZooKeeperConfig() {
// empty
Properties emptyProperties = new Properties();
try {
- FlinkKafkaConsumer.validateZooKeeperConfig(emptyProperties);
+ FlinkKafkaConsumer08.validateZooKeeperConfig(emptyProperties);
fail("should fail with an exception");
}
catch (IllegalArgumentException e) {
@@ -55,7 +54,7 @@ public void testValidateZooKeeperConfig() {
Properties noConnect = new Properties();
noConnect.put(ConsumerConfig.GROUP_ID_CONFIG, "flink-test-group");
try {
- FlinkKafkaConsumer.validateZooKeeperConfig(noConnect);
+ FlinkKafkaConsumer08.validateZooKeeperConfig(noConnect);
fail("should fail with an exception");
}
catch (IllegalArgumentException e) {
@@ -66,7 +65,7 @@ public void testValidateZooKeeperConfig() {
Properties noGroup = new Properties();
noGroup.put("zookeeper.connect", "localhost:47574");
try {
- FlinkKafkaConsumer.validateZooKeeperConfig(noGroup);
+ FlinkKafkaConsumer08.validateZooKeeperConfig(noGroup);
fail("should fail with an exception");
}
catch (IllegalArgumentException e) {
@@ -82,15 +81,15 @@ public void testValidateZooKeeperConfig() {
@Test
public void testSnapshot() {
try {
- Field offsetsField = FlinkKafkaConsumer.class.getDeclaredField("lastOffsets");
- Field runningField = FlinkKafkaConsumer.class.getDeclaredField("running");
- Field mapField = FlinkKafkaConsumer.class.getDeclaredField("pendingCheckpoints");
+ Field offsetsField = FlinkKafkaConsumerBase.class.getDeclaredField("offsetsState");
+ Field runningField = FlinkKafkaConsumerBase.class.getDeclaredField("running");
+ Field mapField = FlinkKafkaConsumerBase.class.getDeclaredField("pendingCheckpoints");
offsetsField.setAccessible(true);
runningField.setAccessible(true);
mapField.setAccessible(true);
- FlinkKafkaConsumer> consumer = mock(FlinkKafkaConsumer.class);
+ FlinkKafkaConsumer08> consumer = mock(FlinkKafkaConsumer08.class);
when(consumer.snapshotState(anyLong(), anyLong())).thenCallRealMethod();
@@ -125,7 +124,7 @@ public void testSnapshot() {
assertEquals(checkpointCopy, checkpoint);
assertTrue(map.size() > 0);
- assertTrue(map.size() <= FlinkKafkaConsumer.MAX_NUM_PENDING_CHECKPOINTS);
+ assertTrue(map.size() <= FlinkKafkaConsumer08.MAX_NUM_PENDING_CHECKPOINTS);
}
}
catch (Exception e) {
@@ -143,9 +142,7 @@ public void testCreateSourceWithoutCluster() {
props.setProperty("bootstrap.servers", "localhost:11111, localhost:22222");
props.setProperty("group.id", "non-existent-group");
- new FlinkKafkaConsumer<>(Collections.singletonList("no op topic"), new SimpleStringSchema(), props,
- FlinkKafkaConsumer.OffsetStore.FLINK_ZOOKEEPER,
- FlinkKafkaConsumer.FetcherType.LEGACY_LOW_LEVEL);
+ new FlinkKafkaConsumer08<>(Collections.singletonList("no op topic"), new SimpleStringSchema(), props);
}
catch (Exception e) {
e.printStackTrace();
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaLocalSystemTime.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaLocalSystemTime.java
similarity index 100%
rename from flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaLocalSystemTime.java
rename to flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaLocalSystemTime.java
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java
similarity index 94%
rename from flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java
rename to flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java
index 531b2194b299e..8602ffec2f375 100644
--- a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java
+++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java
@@ -47,7 +47,7 @@
import static org.junit.Assert.*;
@RunWith(PowerMockRunner.class)
-@PrepareForTest(FlinkKafkaProducer.class)
+@PrepareForTest(FlinkKafkaProducerBase.class)
public class KafkaProducerTest extends TestLogger {
@Test
@@ -76,8 +76,8 @@ public Future answer(InvocationOnMock invocation) throws Throwab
whenNew(KafkaProducer.class).withAnyArguments().thenReturn(kafkaProducerMock);
// (1) producer that propagates errors
-
- FlinkKafkaProducer producerPropagating = new FlinkKafkaProducer(
+
+ FlinkKafkaProducer08 producerPropagating = new FlinkKafkaProducer08<>(
"mock_topic", new SimpleStringSchema(), new Properties(), null);
producerPropagating.setRuntimeContext(new MockRuntimeContext(17, 3));
@@ -95,8 +95,8 @@ public Future answer(InvocationOnMock invocation) throws Throwab
}
// (2) producer that only logs errors
-
- FlinkKafkaProducer producerLogging = new FlinkKafkaProducer(
+
+ FlinkKafkaProducer08 producerLogging = new FlinkKafkaProducer08<>(
"mock_topic", new SimpleStringSchema(), new Properties(), null);
producerLogging.setLogFailuresOnly(true);
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java
similarity index 56%
rename from flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java
rename to flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java
index c9ac75b059ce5..348b75de5ba29 100644
--- a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java
+++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java
@@ -15,127 +15,145 @@
* limitations under the License.
*/
+
package org.apache.flink.streaming.connectors.kafka;
import kafka.admin.AdminUtils;
+import kafka.api.PartitionMetadata;
import kafka.common.KafkaException;
import kafka.consumer.ConsumerConfig;
import kafka.network.SocketServer;
import kafka.server.KafkaConfig;
import kafka.server.KafkaServer;
-
import org.I0Itec.zkclient.ZkClient;
-
import org.apache.commons.io.FileUtils;
import org.apache.curator.RetryPolicy;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.CuratorFrameworkFactory;
import org.apache.curator.retry.ExponentialBackoffRetry;
import org.apache.curator.test.TestingServer;
-import org.apache.flink.client.program.ProgramInvocationException;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.client.JobExecutionException;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionLeader;
import org.apache.flink.streaming.connectors.kafka.internals.ZooKeeperStringSerializer;
-import org.apache.flink.streaming.connectors.kafka.testutils.SuccessException;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
+import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
+import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
+import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
import org.apache.flink.util.NetUtils;
-import org.apache.flink.util.TestLogger;
-
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-
-import scala.concurrent.duration.FiniteDuration;
+import scala.collection.Seq;
import java.io.File;
-import java.io.IOException;
import java.net.BindException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Properties;
import java.util.UUID;
-import java.util.concurrent.TimeUnit;
import static org.apache.flink.util.NetUtils.hostAndPortToUrlString;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
/**
- * The base for the Kafka tests. It brings up:
- *
- * - A ZooKeeper mini cluster
- * - Three Kafka Brokers (mini clusters)
- * - A Flink mini cluster
- *
- *
- * Code in this test is based on the following GitHub repository:
- *
- * https://github.com/sakserv/hadoop-mini-clusters (ASL licensed),
- * as per commit bc6b2b2d5f6424d5f377aa6c0871e82a956462ef
+ * An implementation of the KafkaServerProvider for Kafka 0.8
*/
-@SuppressWarnings("serial")
-public abstract class KafkaTestBase extends TestLogger {
+public class KafkaTestEnvironmentImpl extends KafkaTestEnvironment {
+
+ protected static final Logger LOG = LoggerFactory.getLogger(KafkaTestEnvironmentImpl.class);
+ private File tmpZkDir;
+ private File tmpKafkaParent;
+ private List tmpKafkaDirs;
+ private List brokers;
+ private TestingServer zookeeper;
+ private String zookeeperConnectionString;
+ private String brokerConnectionString = "";
+ private Properties standardProps;
+ private ConsumerConfig standardCC;
+
+
+ public String getBrokerConnectionString() {
+ return brokerConnectionString;
+ }
- protected static final Logger LOG = LoggerFactory.getLogger(KafkaTestBase.class);
-
- protected static final int NUMBER_OF_KAFKA_SERVERS = 3;
- protected static String zookeeperConnectionString;
+ @Override
+ public ConsumerConfig getStandardConsumerConfig() {
+ return standardCC;
+ }
- protected static File tmpZkDir;
+ @Override
+ public Properties getStandardProperties() {
+ return standardProps;
+ }
- protected static File tmpKafkaParent;
+ @Override
+ public String getVersion() {
+ return "0.8";
+ }
- protected static TestingServer zookeeper;
- protected static List brokers;
- protected static String brokerConnectionStrings = "";
+ @Override
+ public List getBrokers() {
+ return brokers;
+ }
- protected static ConsumerConfig standardCC;
- protected static Properties standardProps;
-
- protected static ForkableFlinkMiniCluster flink;
+ @Override
+ public FlinkKafkaConsumerBase getConsumer(List topics, KeyedDeserializationSchema readSchema, Properties props) {
+ return new FlinkKafkaConsumer08<>(topics, readSchema, props);
+ }
- protected static int flinkPort;
+ @Override
+ public FlinkKafkaProducerBase getProducer(String topic, KeyedSerializationSchema serSchema, Properties props, KafkaPartitioner partitioner) {
+ return new FlinkKafkaProducer08(topic, serSchema, props, partitioner);
+ }
- protected static FiniteDuration timeout = new FiniteDuration(10, TimeUnit.SECONDS);
+ @Override
+ public void restartBroker(int leaderId) throws Exception {
+ brokers.set(leaderId, getKafkaServer(leaderId, tmpKafkaDirs.get(leaderId), KAFKA_HOST, zookeeperConnectionString));
+ }
- protected static List tmpKafkaDirs;
+ @Override
+ public int getLeaderToShutDown(String topic) throws Exception {
+ ZkClient zkClient = createZkClient();
+ PartitionMetadata firstPart = null;
+ do {
+ if (firstPart != null) {
+ LOG.info("Unable to find leader. error code {}", firstPart.errorCode());
+ // not the first try. Sleep a bit
+ Thread.sleep(150);
+ }
- protected static String kafkaHost = "localhost";
+ Seq partitionMetadata = AdminUtils.fetchTopicMetadataFromZk(topic, zkClient).partitionsMetadata();
+ firstPart = partitionMetadata.head();
+ }
+ while (firstPart.errorCode() != 0);
+ zkClient.close();
- // ------------------------------------------------------------------------
- // Setup and teardown of the mini clusters
- // ------------------------------------------------------------------------
-
- @BeforeClass
- public static void prepare() throws IOException {
- LOG.info("-------------------------------------------------------------------------");
- LOG.info(" Starting KafkaITCase ");
- LOG.info("-------------------------------------------------------------------------");
-
- LOG.info("Starting KafkaITCase.prepare()");
-
+ return firstPart.leader().get().id();
+ }
+
+ @Override
+ public int getBrokerId(KafkaServer server) {
+ return server.socketServer().brokerId();
+ }
+
+
+ @Override
+ public void prepare(int numKafkaServers) {
File tempDir = new File(System.getProperty("java.io.tmpdir"));
-
+
tmpZkDir = new File(tempDir, "kafkaITcase-zk-dir-" + (UUID.randomUUID().toString()));
assertTrue("cannot create zookeeper temp dir", tmpZkDir.mkdirs());
tmpKafkaParent = new File(tempDir, "kafkaITcase-kafka-dir*" + (UUID.randomUUID().toString()));
assertTrue("cannot create kafka temp dir", tmpKafkaParent.mkdirs());
- tmpKafkaDirs = new ArrayList<>(NUMBER_OF_KAFKA_SERVERS);
- for (int i = 0; i < NUMBER_OF_KAFKA_SERVERS; i++) {
+ tmpKafkaDirs = new ArrayList<>(numKafkaServers);
+ for (int i = 0; i < numKafkaServers; i++) {
File tmpDir = new File(tmpKafkaParent, "server-" + i);
assertTrue("cannot create kafka temp dir", tmpDir.mkdir());
tmpKafkaDirs.add(tmpDir);
}
-
+
zookeeper = null;
brokers = null;
@@ -145,14 +163,14 @@ public static void prepare() throws IOException {
zookeeperConnectionString = zookeeper.getConnectString();
LOG.info("Starting KafkaServer");
- brokers = new ArrayList<>(NUMBER_OF_KAFKA_SERVERS);
-
- for (int i = 0; i < NUMBER_OF_KAFKA_SERVERS; i++) {
- brokers.add(getKafkaServer(i, tmpKafkaDirs.get(i), kafkaHost, zookeeperConnectionString));
+ brokers = new ArrayList<>(numKafkaServers);
+
+ for (int i = 0; i < numKafkaServers; i++) {
+ brokers.add(getKafkaServer(i, tmpKafkaDirs.get(i), KafkaTestEnvironment.KAFKA_HOST, zookeeperConnectionString));
SocketServer socketServer = brokers.get(i).socketServer();
-
+
String host = socketServer.host() == null ? "localhost" : socketServer.host();
- brokerConnectionStrings += hostAndPortToUrlString(host, socketServer.port()) + ",";
+ brokerConnectionString += hostAndPortToUrlString(host, socketServer.port()) + ",";
}
LOG.info("ZK and KafkaServer started.");
@@ -163,54 +181,30 @@ public static void prepare() throws IOException {
}
standardProps = new Properties();
-
standardProps.setProperty("zookeeper.connect", zookeeperConnectionString);
- standardProps.setProperty("bootstrap.servers", brokerConnectionStrings);
+ standardProps.setProperty("bootstrap.servers", brokerConnectionString);
standardProps.setProperty("group.id", "flink-tests");
standardProps.setProperty("auto.commit.enable", "false");
standardProps.setProperty("zookeeper.session.timeout.ms", "12000"); // 6 seconds is default. Seems to be too small for travis.
standardProps.setProperty("zookeeper.connection.timeout.ms", "20000");
standardProps.setProperty("auto.offset.reset", "earliest"); // read from the beginning.
standardProps.setProperty("fetch.message.max.bytes", "256"); // make a lot of fetches (MESSAGES MUST BE SMALLER!)
-
+
Properties consumerConfigProps = new Properties();
consumerConfigProps.putAll(standardProps);
consumerConfigProps.setProperty("auto.offset.reset", "smallest");
standardCC = new ConsumerConfig(consumerConfigProps);
-
- // start also a re-usable Flink mini cluster
-
- Configuration flinkConfig = new Configuration();
- flinkConfig.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1);
- flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 8);
- flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 16);
- flinkConfig.setString(ConfigConstants.EXECUTION_RETRY_DELAY_KEY, "0 s");
-
- flink = new ForkableFlinkMiniCluster(flinkConfig, false);
- flink.start();
-
- flinkPort = flink.getLeaderRPCPort();
}
- @AfterClass
- public static void shutDownServices() {
-
- LOG.info("-------------------------------------------------------------------------");
- LOG.info(" Shut down KafkaITCase ");
- LOG.info("-------------------------------------------------------------------------");
-
- flinkPort = -1;
- if (flink != null) {
- flink.shutdown();
- }
-
+ @Override
+ public void shutdown() {
for (KafkaServer broker : brokers) {
if (broker != null) {
broker.shutdown();
}
}
brokers.clear();
-
+
if (zookeeper != null) {
try {
zookeeper.stop();
@@ -220,9 +214,9 @@ public static void shutDownServices() {
}
zookeeper = null;
}
-
+
// clean up the temp spaces
-
+
if (tmpKafkaParent != null && tmpKafkaParent.exists()) {
try {
FileUtils.deleteDirectory(tmpKafkaParent);
@@ -239,10 +233,59 @@ public static void shutDownServices() {
// ignore
}
}
+ }
- LOG.info("-------------------------------------------------------------------------");
- LOG.info(" KafkaITCase finished");
- LOG.info("-------------------------------------------------------------------------");
+ @Override
+ public void createTestTopic(String topic, int numberOfPartitions, int replicationFactor) {
+ // create topic with one client
+ Properties topicConfig = new Properties();
+ LOG.info("Creating topic {}", topic);
+
+ ZkClient creator = createZkClient();
+
+ AdminUtils.createTopic(creator, topic, numberOfPartitions, replicationFactor, topicConfig);
+ creator.close();
+
+ // validate that the topic has been created
+ final long deadline = System.currentTimeMillis() + 30000;
+ do {
+ try {
+ Thread.sleep(100);
+ }
+ catch (InterruptedException e) {
+ // restore interrupted state
+ }
+ List partitions = FlinkKafkaConsumer08.getPartitionsForTopic(Collections.singletonList(topic), standardProps);
+ if (partitions != null && partitions.size() > 0) {
+ return;
+ }
+ }
+ while (System.currentTimeMillis() < deadline);
+ fail ("Test topic could not be created");
+ }
+
+ @Override
+ public void deleteTestTopic(String topic) {
+ LOG.info("Deleting topic {}", topic);
+
+ ZkClient zk = createZkClient();
+ AdminUtils.deleteTopic(zk, topic);
+ zk.close();
+ }
+
+ private ZkClient createZkClient() {
+ return new ZkClient(standardCC.zkConnect(), standardCC.zkSessionTimeoutMs(),
+ standardCC.zkConnectionTimeoutMs(), new ZooKeeperStringSerializer());
+ }
+
+ /**
+ * Only for the 0.8 server we need access to the zk client.
+ */
+ public CuratorFramework createCuratorClient() {
+ RetryPolicy retryPolicy = new ExponentialBackoffRetry(100, 10);
+ CuratorFramework curatorClient = CuratorFrameworkFactory.newClient(standardProps.getProperty("zookeeper.connect"), retryPolicy);
+ curatorClient.start();
+ return curatorClient;
}
/**
@@ -251,6 +294,7 @@ public static void shutDownServices() {
protected static KafkaServer getKafkaServer(int brokerId, File tmpFolder,
String kafkaHost,
String zookeeperConnectionString) throws Exception {
+ LOG.info("Starting broker with id {}", brokerId);
Properties kafkaProperties = new Properties();
// properties have to be Strings
@@ -260,13 +304,13 @@ protected static KafkaServer getKafkaServer(int brokerId, File tmpFolder,
kafkaProperties.put("zookeeper.connect", zookeeperConnectionString);
kafkaProperties.put("message.max.bytes", String.valueOf(50 * 1024 * 1024));
kafkaProperties.put("replica.fetch.max.bytes", String.valueOf(50 * 1024 * 1024));
-
+
// for CI stability, increase zookeeper session timeout
kafkaProperties.put("zookeeper.session.timeout.ms", "20000");
final int numTries = 5;
-
- for (int i = 1; i <= numTries; i++) {
+
+ for (int i = 1; i <= numTries; i++) {
int kafkaPort = NetUtils.getAvailablePort();
kafkaProperties.put("port", Integer.toString(kafkaPort));
KafkaConfig kafkaConfig = new KafkaConfig(kafkaProperties);
@@ -286,102 +330,8 @@ protected static KafkaServer getKafkaServer(int brokerId, File tmpFolder,
}
}
}
-
- throw new Exception("Could not start Kafka after " + numTries + " retries due to port conflicts.");
- }
- // ------------------------------------------------------------------------
- // Execution utilities
- // ------------------------------------------------------------------------
-
- protected CuratorFramework createZookeeperClient() {
- RetryPolicy retryPolicy = new ExponentialBackoffRetry(100, 10);
- CuratorFramework curatorClient = CuratorFrameworkFactory.newClient(standardProps.getProperty("zookeeper.connect"), retryPolicy);
- curatorClient.start();
- return curatorClient;
- }
-
- protected static void tryExecute(StreamExecutionEnvironment see, String name) throws Exception {
- try {
- see.execute(name);
- }
- catch (ProgramInvocationException | JobExecutionException root) {
- Throwable cause = root.getCause();
-
- // search for nested SuccessExceptions
- int depth = 0;
- while (!(cause instanceof SuccessException)) {
- if (cause == null || depth++ == 20) {
- root.printStackTrace();
- fail("Test failed: " + root.getMessage());
- }
- else {
- cause = cause.getCause();
- }
- }
- }
- }
-
- protected static void tryExecutePropagateExceptions(StreamExecutionEnvironment see, String name) throws Exception {
- try {
- see.execute(name);
- }
- catch (ProgramInvocationException | JobExecutionException root) {
- Throwable cause = root.getCause();
-
- // search for nested SuccessExceptions
- int depth = 0;
- while (!(cause instanceof SuccessException)) {
- if (cause == null || depth++ == 20) {
- throw root;
- }
- else {
- cause = cause.getCause();
- }
- }
- }
- }
-
-
-
- protected static void createTestTopic(String topic, int numberOfPartitions, int replicationFactor) {
-
- // create topic with one client
- Properties topicConfig = new Properties();
- LOG.info("Creating topic {}", topic);
-
- ZkClient creator = new ZkClient(standardCC.zkConnect(), standardCC.zkSessionTimeoutMs(),
- standardCC.zkConnectionTimeoutMs(), new ZooKeeperStringSerializer());
-
- AdminUtils.createTopic(creator, topic, numberOfPartitions, replicationFactor, topicConfig);
- creator.close();
-
- // validate that the topic has been created
- final long deadline = System.currentTimeMillis() + 30000;
- do {
- try {
- Thread.sleep(100);
- }
- catch (InterruptedException e) {
- // restore interrupted state
- }
- List partitions = FlinkKafkaConsumer.getPartitionsForTopic(Collections.singletonList(topic), standardProps);
- if (partitions != null && partitions.size() > 0) {
- return;
- }
- }
- while (System.currentTimeMillis() < deadline);
- fail ("Test topic could not be created");
+ throw new Exception("Could not start Kafka after " + numTries + " retries due to port conflicts.");
}
-
- protected static void deleteTestTopic(String topic) {
- LOG.info("Deleting topic {}", topic);
-
- ZkClient zk = new ZkClient(standardCC.zkConnect(), standardCC.zkSessionTimeoutMs(),
- standardCC.zkConnectionTimeoutMs(), new ZooKeeperStringSerializer());
- AdminUtils.deleteTopic(zk, topic);
-
- zk.close();
- }
}
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandlerTest.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandlerTest.java
similarity index 77%
rename from flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandlerTest.java
rename to flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandlerTest.java
index 8d16da0a46c8a..c99e133cd80a1 100644
--- a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandlerTest.java
+++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandlerTest.java
@@ -18,17 +18,12 @@
package org.apache.flink.streaming.connectors.kafka.internals;
-import kafka.admin.AdminUtils;
-
-import org.I0Itec.zkclient.ZkClient;
import org.apache.curator.framework.CuratorFramework;
+import org.apache.flink.streaming.connectors.kafka.KafkaTestEnvironmentImpl;
import org.apache.flink.streaming.connectors.kafka.KafkaTestBase;
-import org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler;
import org.junit.Test;
-import java.util.Properties;
-
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
@@ -42,15 +37,9 @@ public void runOffsetManipulationinZooKeeperTest() {
final long offset = (long) (Math.random() * Long.MAX_VALUE);
- CuratorFramework curatorFramework = createZookeeperClient();
+ CuratorFramework curatorFramework = ((KafkaTestEnvironmentImpl)kafkaServer ).createCuratorClient();
+ kafkaServer.createTestTopic(topicName, 3, 2);
- {
- ZkClient zkClient = new ZkClient(standardCC.zkConnect(), standardCC.zkSessionTimeoutMs(),
- standardCC.zkConnectionTimeoutMs(), new ZooKeeperStringSerializer());
- AdminUtils.createTopic(zkClient, topicName, 3, 2, new Properties());
- zkClient.close();
- }
-
ZookeeperOffsetHandler.setOffsetInZooKeeper(curatorFramework, groupId, topicName, 0, offset);
long fetchedOffset = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorFramework, groupId, topicName, 0);
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/resources/log4j-test.properties b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/resources/log4j-test.properties
similarity index 100%
rename from flink-streaming-connectors/flink-connector-kafka/src/test/resources/log4j-test.properties
rename to flink-streaming-connectors/flink-connector-kafka-0.8/src/test/resources/log4j-test.properties
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/resources/logback-test.xml b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/resources/logback-test.xml
similarity index 100%
rename from flink-streaming-connectors/flink-connector-kafka/src/test/resources/logback-test.xml
rename to flink-streaming-connectors/flink-connector-kafka-0.8/src/test/resources/logback-test.xml
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/pom.xml b/flink-streaming-connectors/flink-connector-kafka-0.9/pom.xml
new file mode 100644
index 0000000000000..b3c974954bea3
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka-0.9/pom.xml
@@ -0,0 +1,131 @@
+
+
+
+
+ 4.0.0
+
+
+ org.apache.flink
+ flink-streaming-connectors-parent
+ 1.0-SNAPSHOT
+ ..
+
+
+ flink-connector-kafka-0.9
+ flink-connector-kafka-0.9
+
+ jar
+
+
+
+ 0.9.0.0
+
+
+
+
+
+ org.apache.flink
+ flink-connector-kafka-base
+ ${project.version}
+
+
+ org.apache.kafka
+ kafka_${scala.binary.version}
+
+
+
+
+
+ org.apache.flink
+ flink-connector-kafka-base
+ ${project.version}
+
+
+
+ org.apache.kafka
+ kafka_${scala.binary.version}
+
+
+ test-jar
+ test
+
+
+
+
+ org.apache.kafka
+ kafka-clients
+ ${kafka.version}
+
+
+
+
+ org.apache.kafka
+ kafka_${scala.binary.version}
+ ${kafka.version}
+ test
+
+
+
+ org.apache.flink
+ flink-tests
+ ${project.version}
+ test-jar
+ test
+
+
+
+ org.apache.flink
+ flink-streaming-java
+ ${project.version}
+
+
+
+ com.google.guava
+ guava
+ ${guava.version}
+
+
+
+
+ org.apache.flink
+ flink-test-utils
+ ${project.version}
+ test
+
+
+
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-failsafe-plugin
+
+
+ 1
+
+
+
+
+
+
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java
new file mode 100644
index 0000000000000..9faa24956e9f7
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java
@@ -0,0 +1,501 @@
+/*
+ * 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.flink.streaming.connectors.kafka;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
+import org.apache.flink.streaming.connectors.kafka.internals.metrics.DefaultKafkaMetricAccumulator;
+import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+
+import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
+import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.WakeupException;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * The Flink Kafka Consumer is a streaming data source that pulls a parallel data stream from
+ * Apache Kafka 0.9.x. The consumer can run in multiple parallel instances, each of which will pull
+ * data from one or more Kafka partitions.
+ *
+ * The Flink Kafka Consumer participates in checkpointing and guarantees that no data is lost
+ * during a failure, and that the computation processes elements "exactly once".
+ * (Note: These guarantees naturally assume that Kafka itself does not loose any data.)
+ *
+ * Please note that Flink snapshots the offsets internally as part of its distributed checkpoints. The offsets
+ * committed to Kafka / ZooKeeper are only to bring the outside view of progress in sync with Flink's view
+ * of the progress. That way, monitoring and other jobs can get a view of how far the Flink Kafka consumer
+ * has consumed a topic.
+ *
+ * Please refer to Kafka's documentation for the available configuration properties:
+ * http://kafka.apache.org/documentation.html#newconsumerconfigs
+ *
+ * NOTE: The implementation currently accesses partition metadata when the consumer
+ * is constructed. That means that the client that submits the program needs to be able to
+ * reach the Kafka brokers or ZooKeeper.
+ */
+public class FlinkKafkaConsumer09 extends FlinkKafkaConsumerBase {
+
+ // ------------------------------------------------------------------------
+
+ private static final long serialVersionUID = 2324564345203409112L;
+
+ private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaConsumer09.class);
+
+ /** Configuration key to change the polling timeout **/
+ public static final String KEY_POLL_TIMEOUT = "flink.poll-timeout";
+
+ /** Boolean configuration key to disable metrics tracking **/
+ public static final String KEY_DISABLE_METRICS = "flink.disable-metrics";
+
+ /**
+ * From Kafka's Javadoc: The time, in milliseconds, spent waiting in poll if data is not
+ * available. If 0, returns immediately with any records that are available now.
+ */
+ public static final long DEFAULT_POLL_TIMEOUT = 100L;
+
+ /** User-supplied properties for Kafka **/
+ private final Properties properties;
+ /** Ordered list of all partitions available in all subscribed partitions **/
+ private final List partitionInfos;
+
+ // ------ Runtime State -------
+
+ /** The partitions actually handled by this consumer at runtime */
+ private transient List subscribedPartitions;
+ /** For performance reasons, we are keeping two representations of the subscribed partitions **/
+ private transient List subscribedPartitionsAsFlink;
+ /** The Kafka Consumer instance**/
+ private transient KafkaConsumer consumer;
+ /** The thread running Kafka's consumer **/
+ private transient ConsumerThread consumerThread;
+ /** Exception set from the ConsumerThread */
+ private transient Throwable consumerThreadException;
+ /** If the consumer doesn't have a Kafka partition assigned at runtime, it'll block on this waitThread **/
+ private transient Thread waitThread;
+
+
+ // ------------------------------------------------------------------------
+
+ /**
+ * Creates a new Kafka streaming source consumer for Kafka 0.9.x
+ *
+ * @param topic
+ * The name of the topic that should be consumed.
+ * @param valueDeserializer
+ * The de-/serializer used to convert between Kafka's byte messages and Flink's objects.
+ * @param props
+ * The properties used to configure the Kafka consumer client, and the ZooKeeper client.
+ */
+ public FlinkKafkaConsumer09(String topic, DeserializationSchema valueDeserializer, Properties props) {
+ this(Collections.singletonList(topic), valueDeserializer, props);
+ }
+
+ /**
+ * Creates a new Kafka streaming source consumer for Kafka 0.9.x
+ *
+ * This constructor allows passing a {@see KeyedDeserializationSchema} for reading key/value
+ * pairs, offsets, and topic names from Kafka.
+ *
+ * @param topic
+ * The name of the topic that should be consumed.
+ * @param deserializer
+ * The keyed de-/serializer used to convert between Kafka's byte messages and Flink's objects.
+ * @param props
+ * The properties used to configure the Kafka consumer client, and the ZooKeeper client.
+ */
+ public FlinkKafkaConsumer09(String topic, KeyedDeserializationSchema deserializer, Properties props) {
+ this(Collections.singletonList(topic), deserializer, props);
+ }
+
+ /**
+ * Creates a new Kafka streaming source consumer for Kafka 0.9.x
+ *
+ * This constructor allows passing multiple topics to the consumer.
+ *
+ * @param topics
+ * The Kafka topics to read from.
+ * @param deserializer
+ * The de-/serializer used to convert between Kafka's byte messages and Flink's objects.
+ * @param props
+ * The properties that are used to configure both the fetcher and the offset handler.
+ */
+ public FlinkKafkaConsumer09(List topics, DeserializationSchema deserializer, Properties props) {
+ this(topics, new KeyedDeserializationSchemaWrapper<>(deserializer), props);
+ }
+
+ /**
+ * Creates a new Kafka streaming source consumer for Kafka 0.9.x
+ *
+ * This constructor allows passing multiple topics and a key/value deserialization schema.
+ *
+ * @param topics
+ * The Kafka topics to read from.
+ * @param deserializer
+ * The keyed de-/serializer used to convert between Kafka's byte messages and Flink's objects.
+ * @param props
+ * The properties that are used to configure both the fetcher and the offset handler.
+ */
+ public FlinkKafkaConsumer09(List topics, KeyedDeserializationSchema deserializer, Properties props) {
+ super(deserializer, props);
+ checkNotNull(topics, "topics");
+ this.properties = checkNotNull(props, "props");
+ setDeserializer(this.properties);
+ KafkaConsumer consumer = null;
+ try {
+ consumer = new KafkaConsumer<>(this.properties);
+ this.partitionInfos = new ArrayList<>();
+ for (final String topic: topics) {
+ // get partitions for each topic
+ List partitionsForTopic = null;
+ for(int tri = 0; tri < 10; tri++) {
+ LOG.info("Trying to get partitions for topic {}", topic);
+ try {
+ partitionsForTopic = consumer.partitionsFor(topic);
+ if(partitionsForTopic != null && partitionsForTopic.size() > 0) {
+ break; // it worked
+ }
+ } catch (NullPointerException npe) {
+ // workaround for KAFKA-2880: Fetcher.getTopicMetadata NullPointerException when broker cannot be reached
+ // we ignore the NPE.
+ }
+ // create a new consumer
+ consumer.close();
+ try {
+ Thread.sleep(1000);
+ } catch (InterruptedException e) {
+ }
+ consumer = new KafkaConsumer<>(properties);
+ }
+ // for non existing topics, the list might be null.
+ if(partitionsForTopic != null) {
+ partitionInfos.addAll(convertToFlinkKafkaTopicPartition(partitionsForTopic));
+ }
+ }
+ } finally {
+ if(consumer != null) {
+ consumer.close();
+ }
+ }
+ if(partitionInfos.isEmpty()) {
+ throw new RuntimeException("Unable to retrieve any partitions for the requested topics " + topics);
+ }
+
+ // we now have a list of partitions which is the same for all parallel consumer instances.
+ LOG.info("Got {} partitions from these topics: {}", partitionInfos.size(), topics);
+
+ if (LOG.isInfoEnabled()) {
+ logPartitionInfo(partitionInfos);
+ }
+ }
+
+
+ /**
+ * Converts a list of Kafka PartitionInfo's to Flink's KafkaTopicPartition (which are serializable)
+ * @param partitions A list of Kafka PartitionInfos.
+ * @return A list of KafkaTopicPartitions
+ */
+ public static List convertToFlinkKafkaTopicPartition(List partitions) {
+ checkNotNull(partitions, "The given list of partitions was null");
+ List ret = new ArrayList<>(partitions.size());
+ for(PartitionInfo pi: partitions) {
+ ret.add(new KafkaTopicPartition(pi.topic(), pi.partition()));
+ }
+ return ret;
+ }
+
+ public static List convertToKafkaTopicPartition(List partitions) {
+ List ret = new ArrayList<>(partitions.size());
+ for(KafkaTopicPartition ktp: partitions) {
+ ret.add(new TopicPartition(ktp.getTopic(), ktp.getPartition()));
+ }
+ return ret;
+ }
+
+ // ------------------------------------------------------------------------
+ // Source life cycle
+ // ------------------------------------------------------------------------
+
+ @Override
+ public void open(Configuration parameters) throws Exception {
+ super.open(parameters);
+
+ final int numConsumers = getRuntimeContext().getNumberOfParallelSubtasks();
+ final int thisConsumerIndex = getRuntimeContext().getIndexOfThisSubtask();
+
+ // pick which partitions we work on
+ this.subscribedPartitionsAsFlink = assignPartitions(this.partitionInfos, numConsumers, thisConsumerIndex);
+ if(this.subscribedPartitionsAsFlink.isEmpty()) {
+ LOG.info("This consumer doesn't have any partitions assigned");
+ this.offsetsState = null;
+ return;
+ } else {
+ StreamingRuntimeContext streamingRuntimeContext = (StreamingRuntimeContext) getRuntimeContext();
+ // if checkpointing is enabled, we are not automatically committing to Kafka.
+ properties.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, Boolean.toString(!streamingRuntimeContext.isCheckpointingEnabled()));
+ this.consumer = new KafkaConsumer<>(properties);
+ }
+ subscribedPartitions = convertToKafkaTopicPartition(subscribedPartitionsAsFlink);
+
+ this.consumer.assign(this.subscribedPartitions);
+
+ // register Kafka metrics to Flink accumulators
+ if(!Boolean.getBoolean(properties.getProperty(KEY_DISABLE_METRICS, "false"))) {
+ Map metrics = this.consumer.metrics();
+ if(metrics == null) {
+ // MapR's Kafka implementation returns null here.
+ LOG.info("Consumer implementation does not support metrics");
+ } else {
+ for (Map.Entry metric : metrics.entrySet()) {
+ String name = "consumer-" + metric.getKey().name();
+ DefaultKafkaMetricAccumulator kafkaAccumulator = DefaultKafkaMetricAccumulator.createFor(metric.getValue());
+ // best effort: we only add the accumulator if available.
+ if (kafkaAccumulator != null) {
+ getRuntimeContext().addAccumulator(name, kafkaAccumulator);
+ }
+ }
+ }
+ }
+
+ // check if we need to explicitly seek to a specific offset (restore case)
+ if(restoreToOffset != null) {
+ // we are in a recovery scenario
+ for(Map.Entry offset: restoreToOffset.entrySet()) {
+ // seek all offsets to the right position
+ this.consumer.seek(new TopicPartition(offset.getKey().getTopic(), offset.getKey().getPartition()), offset.getValue() + 1);
+ }
+ this.offsetsState = restoreToOffset;
+ } else {
+ this.offsetsState = new HashMap<>();
+ }
+ }
+
+
+
+ @Override
+ public void run(SourceContext sourceContext) throws Exception {
+ if(consumer != null) {
+ consumerThread = new ConsumerThread<>(this, sourceContext);
+ consumerThread.start();
+ // wait for the consumer to stop
+ while(consumerThread.isAlive()) {
+ if(consumerThreadException != null) {
+ throw new RuntimeException("ConsumerThread threw an exception", consumerThreadException);
+ }
+ try {
+ consumerThread.join(50);
+ } catch (InterruptedException ie) {
+ consumerThread.shutdown();
+ }
+ }
+ // check again for an exception
+ if(consumerThreadException != null) {
+ throw new RuntimeException("ConsumerThread threw an exception", consumerThreadException);
+ }
+ } else {
+ // this source never completes, so emit a Long.MAX_VALUE watermark
+ // to not block watermark forwarding
+ if (getRuntimeContext().getExecutionConfig().areTimestampsEnabled()) {
+ sourceContext.emitWatermark(new Watermark(Long.MAX_VALUE));
+ }
+
+ final Object waitLock = new Object();
+ this.waitThread = Thread.currentThread();
+ while (running) {
+ // wait until we are canceled
+ try {
+ //noinspection SynchronizationOnLocalVariableOrMethodParameter
+ synchronized (waitLock) {
+ waitLock.wait();
+ }
+ }
+ catch (InterruptedException e) {
+ // do nothing, check our "running" status
+ }
+ }
+ }
+ // close the context after the work was done. this can actually only
+ // happen when the fetcher decides to stop fetching
+ sourceContext.close();
+ }
+
+ @Override
+ public void cancel() {
+ // set ourselves as not running
+ running = false;
+ if(this.consumerThread != null) {
+ this.consumerThread.shutdown();
+ } else {
+ // the consumer thread is not running, so we have to interrupt our own thread
+ if(waitThread != null) {
+ waitThread.interrupt();
+ }
+ }
+ }
+
+ @Override
+ public void close() throws Exception {
+ cancel();
+ super.close();
+ }
+
+ // ------------------------------------------------------------------------
+ // Checkpoint and restore
+ // ------------------------------------------------------------------------
+
+
+ @Override
+ protected void commitOffsets(HashMap checkpointOffsets) {
+ Map kafkaCheckpointOffsets = convertToCommitMap(checkpointOffsets);
+ synchronized (this.consumer) {
+ this.consumer.commitSync(kafkaCheckpointOffsets);
+ }
+ }
+
+ public static Map convertToCommitMap(HashMap checkpointOffsets) {
+ Map ret = new HashMap<>(checkpointOffsets.size());
+ for(Map.Entry partitionOffset: checkpointOffsets.entrySet()) {
+ ret.put(new TopicPartition(partitionOffset.getKey().getTopic(), partitionOffset.getKey().getPartition()),
+ new OffsetAndMetadata(partitionOffset.getValue(), ""));
+ }
+ return ret;
+ }
+
+ // ------------------------------------------------------------------------
+ // Miscellaneous utilities
+ // ------------------------------------------------------------------------
+
+
+ protected static void setDeserializer(Properties props) {
+ if (!props.contains(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG)) {
+ props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getCanonicalName());
+ } else {
+ LOG.warn("Overwriting the '{}' is not recommended", ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG);
+ }
+
+ if (!props.contains(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG)) {
+ props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getCanonicalName());
+ } else {
+ LOG.warn("Overwriting the '{}' is not recommended", ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG);
+ }
+ }
+
+ /**
+ * We use a separate thread for executing the KafkaConsumer.poll(timeout) call because Kafka is not
+ * handling interrupts properly. On an interrupt (which happens automatically by Flink if the task
+ * doesn't react to cancel() calls), the poll() method might never return.
+ * On cancel, we'll wakeup the .poll() call and wait for it to return
+ */
+ private static class ConsumerThread extends Thread {
+ private final FlinkKafkaConsumer09 flinkKafkaConsumer;
+ private final SourceContext sourceContext;
+ private boolean running = true;
+
+ public ConsumerThread(FlinkKafkaConsumer09 flinkKafkaConsumer, SourceContext sourceContext) {
+ this.flinkKafkaConsumer = flinkKafkaConsumer;
+ this.sourceContext = sourceContext;
+ }
+
+ @Override
+ public void run() {
+ try {
+ long pollTimeout = Long.parseLong(flinkKafkaConsumer.properties.getProperty(KEY_POLL_TIMEOUT, Long.toString(DEFAULT_POLL_TIMEOUT)));
+ pollLoop: while (running) {
+ ConsumerRecords records;
+ //noinspection SynchronizeOnNonFinalField
+ synchronized (flinkKafkaConsumer.consumer) {
+ try {
+ records = flinkKafkaConsumer.consumer.poll(pollTimeout);
+ } catch (WakeupException we) {
+ if (running) {
+ throw we;
+ }
+ // leave loop
+ continue;
+ }
+ }
+ // get the records for each topic partition
+ for (int i = 0; i < flinkKafkaConsumer.subscribedPartitions.size(); i++) {
+ TopicPartition partition = flinkKafkaConsumer.subscribedPartitions.get(i);
+ KafkaTopicPartition flinkPartition = flinkKafkaConsumer.subscribedPartitionsAsFlink.get(i);
+ List> partitionRecords = records.records(partition);
+ //noinspection ForLoopReplaceableByForEach
+ for (int j = 0; j < partitionRecords.size(); j++) {
+ ConsumerRecord record = partitionRecords.get(j);
+ T value = flinkKafkaConsumer.deserializer.deserialize(record.key(), record.value(), record.topic(), record.partition(),record.offset());
+ if(flinkKafkaConsumer.deserializer.isEndOfStream(value)) {
+ // end of stream signaled
+ running = false;
+ break pollLoop;
+ }
+ synchronized (sourceContext.getCheckpointLock()) {
+ sourceContext.collect(value);
+ flinkKafkaConsumer.offsetsState.put(flinkPartition, record.offset());
+ }
+ }
+ }
+ }
+ } catch(Throwable t) {
+ if(running) {
+ this.flinkKafkaConsumer.stopWithError(t);
+ } else {
+ LOG.debug("Stopped ConsumerThread threw exception", t);
+ }
+ } finally {
+ try {
+ flinkKafkaConsumer.consumer.close();
+ } catch(Throwable t) {
+ LOG.warn("Error while closing consumer", t);
+ }
+ }
+ }
+
+ /**
+ * Try to shutdown the thread
+ */
+ public void shutdown() {
+ this.running = false;
+ this.flinkKafkaConsumer.consumer.wakeup();
+ }
+ }
+
+ private void stopWithError(Throwable t) {
+ this.consumerThreadException = t;
+ }
+}
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer09.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer09.java
new file mode 100644
index 0000000000000..6f7f68703a6a5
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer09.java
@@ -0,0 +1,130 @@
+/*
+ * 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.flink.streaming.connectors.kafka;
+
+import org.apache.flink.streaming.connectors.kafka.partitioner.FixedPartitioner;
+import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
+import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
+import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
+import org.apache.flink.streaming.util.serialization.SerializationSchema;
+
+import java.util.Properties;
+
+
+/**
+ * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.8.
+ *
+ * Please note that this producer does not have any reliability guarantees.
+ *
+ * @param Type of the messages to write into Kafka.
+ */
+public class FlinkKafkaProducer09 extends FlinkKafkaProducerBase {
+
+ private static final long serialVersionUID = 1L;
+
+ // ------------------- Keyless serialization schema constructors ----------------------
+
+ /**
+ * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
+ * the topic.
+ *
+ * @param brokerList
+ * Comma separated addresses of the brokers
+ * @param topicId
+ * ID of the Kafka topic.
+ * @param serializationSchema
+ * User defined (keyless) serialization schema.
+ */
+ public FlinkKafkaProducer09(String brokerList, String topicId, SerializationSchema serializationSchema) {
+ this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), new FixedPartitioner());
+ }
+
+ /**
+ * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
+ * the topic.
+ *
+ * @param topicId
+ * ID of the Kafka topic.
+ * @param serializationSchema
+ * User defined (keyless) serialization schema.
+ * @param producerConfig
+ * Properties with the producer configuration.
+ */
+ public FlinkKafkaProducer09(String topicId, SerializationSchema serializationSchema, Properties producerConfig) {
+ this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FixedPartitioner());
+ }
+
+ /**
+ * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
+ * the topic.
+ *
+ * @param topicId The topic to write data to
+ * @param serializationSchema A (keyless) serializable serialization schema for turning user objects into a kafka-consumable byte[]
+ * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
+ * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions (when passing null, we'll use Kafka's partitioner)
+ */
+ public FlinkKafkaProducer09(String topicId, SerializationSchema serializationSchema, Properties producerConfig, KafkaPartitioner customPartitioner) {
+ this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner);
+
+ }
+
+ // ------------------- Key/Value serialization schema constructors ----------------------
+
+ /**
+ * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
+ * the topic.
+ *
+ * @param brokerList
+ * Comma separated addresses of the brokers
+ * @param topicId
+ * ID of the Kafka topic.
+ * @param serializationSchema
+ * User defined serialization schema supporting key/value messages
+ */
+ public FlinkKafkaProducer09(String brokerList, String topicId, KeyedSerializationSchema serializationSchema) {
+ this(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), new FixedPartitioner());
+ }
+
+ /**
+ * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
+ * the topic.
+ *
+ * @param topicId
+ * ID of the Kafka topic.
+ * @param serializationSchema
+ * User defined serialization schema supporting key/value messages
+ * @param producerConfig
+ * Properties with the producer configuration.
+ */
+ public FlinkKafkaProducer09(String topicId, KeyedSerializationSchema serializationSchema, Properties producerConfig) {
+ this(topicId, serializationSchema, producerConfig, new FixedPartitioner());
+ }
+
+ /**
+ * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
+ * the topic.
+ *
+ * @param topicId The topic to write data to
+ * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
+ * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
+ * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
+ */
+ public FlinkKafkaProducer09(String topicId, KeyedSerializationSchema serializationSchema, Properties producerConfig, KafkaPartitioner customPartitioner) {
+ super(topicId, serializationSchema, producerConfig, customPartitioner);
+ }
+}
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/examples/ReadFromKafka.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/examples/ReadFromKafka.java
new file mode 100644
index 0000000000000..643da66b11294
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/examples/ReadFromKafka.java
@@ -0,0 +1,56 @@
+/*
+ * 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.flink.streaming.connectors.kafka.examples;
+
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer09;
+import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
+
+
+/**
+ * Read Strings from Kafka and print them to standard out.
+ * Note: On a cluster, DataStream.print() will print to the TaskManager's .out file!
+ *
+ * Please pass the following arguments to run the example:
+ * --topic test --bootstrap.servers localhost:9092 --group.id myconsumer
+ *
+ */
+public class ReadFromKafka {
+
+ public static void main(String[] args) throws Exception {
+ StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+ env.getConfig().disableSysoutLogging();
+ env.setNumberOfExecutionRetries(4);
+ env.enableCheckpointing(5000);
+ env.setParallelism(2);
+
+ ParameterTool parameterTool = ParameterTool.fromArgs(args);
+
+ DataStream messageStream = env
+ .addSource(new FlinkKafkaConsumer09<>(
+ parameterTool.getRequired("topic"),
+ new SimpleStringSchema(),
+ parameterTool.getProperties()));
+
+ messageStream.print();
+
+ env.execute("Read from Kafka example");
+ }
+}
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/examples/WriteIntoKafka.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/examples/WriteIntoKafka.java
new file mode 100644
index 0000000000000..fbe53fae16f40
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/examples/WriteIntoKafka.java
@@ -0,0 +1,70 @@
+/*
+ * 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.flink.streaming.connectors.kafka.examples;
+
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer09;
+import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
+
+
+/**
+ * Generate a String every 500 ms and write it into a Kafka topic
+ *
+ * Please pass the following arguments to run the example:
+ * --topic test --bootstrap.servers localhost:9092
+ *
+ */
+public class WriteIntoKafka {
+
+ public static void main(String[] args) throws Exception {
+ StreamExecutionEnvironment env =
+ StreamExecutionEnvironment.getExecutionEnvironment();
+ env.getConfig().disableSysoutLogging();
+ env.setNumberOfExecutionRetries(4);
+ env.setParallelism(2);
+
+ ParameterTool parameterTool = ParameterTool.fromArgs(args);
+
+ // very simple data generator
+ DataStream messageStream = env.addSource(new SourceFunction() {
+ public boolean running = true;
+
+ @Override
+ public void run(SourceContext ctx) throws Exception {
+ long i = 0;
+ while(this.running) {
+ ctx.collect("Element - " + i++);
+ Thread.sleep(500);
+ }
+ }
+
+ @Override
+ public void cancel() {
+ running = false;
+ }
+ });
+
+ // write data into Kafka
+ messageStream.addSink(new FlinkKafkaProducer09<>(parameterTool.getRequired("topic"), new SimpleStringSchema(), parameterTool.getProperties()));
+
+ env.execute("Write into Kafka example");
+ }
+}
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/resources/log4j.properties b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/resources/log4j.properties
new file mode 100644
index 0000000000000..6bdfb48cd550f
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/resources/log4j.properties
@@ -0,0 +1,29 @@
+################################################################################
+# 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.
+################################################################################
+
+log4j.rootLogger=INFO, testlogger
+
+log4j.appender.testlogger=org.apache.log4j.ConsoleAppender
+log4j.appender.testlogger.target = System.err
+log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout
+log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
+
+# suppress the irrelevant (wrong) warnings from the netty channel handler
+log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger
+
+
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ITCase.java
similarity index 75%
rename from flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java
rename to flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ITCase.java
index 07e650ad1d57f..55abaaa9da12a 100644
--- a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java
+++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ITCase.java
@@ -17,47 +17,27 @@
package org.apache.flink.streaming.connectors.kafka;
-import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
-
import org.junit.Test;
-import java.util.List;
-import java.util.Properties;
+public class Kafka09ITCase extends KafkaConsumerTestBase {
-public class KafkaITCase extends KafkaConsumerTestBase {
-
- @Override
- protected FlinkKafkaConsumer getConsumer(List topics, KeyedDeserializationSchema deserializationSchema, Properties props) {
- return new FlinkKafkaConsumer082<>(topics, deserializationSchema, props);
- }
-
// ------------------------------------------------------------------------
// Suite of Tests
// ------------------------------------------------------------------------
-
- @Test
+
+ @Test(timeout = 60000)
public void testCheckpointing() throws Exception {
runCheckpointingTest();
}
- @Test()
+ @Test(timeout = 60000)
public void testFailOnNoBroker() throws Exception {
runFailOnNoBrokerTest();
}
- @Test
- public void testOffsetInZookeeper() throws Exception {
- runOffsetInZookeeperValidationTest();
- }
-
- @Test
- public void testOffsetAutocommitTest() throws Exception {
- runOffsetAutocommitTest();
- }
-
- @Test
+ @Test(timeout = 60000)
public void testConcurrentProducerConsumerTopology() throws Exception {
runSimpleConcurrentProducerConsumerTopology();
}
@@ -68,66 +48,67 @@ public void testKeyValueSupport() throws Exception {
}
// --- canceling / failures ---
-
- @Test
+
+ @Test(timeout = 60000)
public void testCancelingEmptyTopic() throws Exception {
runCancelingOnEmptyInputTest();
}
- @Test
+ @Test(timeout = 60000)
public void testCancelingFullTopic() throws Exception {
runCancelingOnFullInputTest();
}
- @Test
+ @Test(timeout = 60000)
public void testFailOnDeploy() throws Exception {
runFailOnDeployTest();
}
- @Test
- public void testInvalidOffset() throws Exception {
- runInvalidOffsetTest();
- }
// --- source to partition mappings and exactly once ---
-
- @Test
+
+ @Test(timeout = 60000)
public void testOneToOneSources() throws Exception {
runOneToOneExactlyOnceTest();
}
- @Test
+ @Test(timeout = 60000)
public void testOneSourceMultiplePartitions() throws Exception {
runOneSourceMultiplePartitionsExactlyOnceTest();
}
- @Test
+ @Test(timeout = 60000)
public void testMultipleSourcesOnePartition() throws Exception {
runMultipleSourcesOnePartitionExactlyOnceTest();
}
// --- broker failure ---
- @Test
+ @Test(timeout = 60000)
public void testBrokerFailure() throws Exception {
runBrokerFailureTest();
}
// --- special executions ---
-
- @Test
+
+ @Test(timeout = 60000)
public void testBigRecordJob() throws Exception {
runBigRecordTestTopology();
}
- @Test
+ @Test(timeout = 60000)
public void testMultipleTopics() throws Exception {
runConsumeMultipleTopics();
}
- @Test
+ @Test(timeout = 60000)
public void testAllDeletes() throws Exception {
runAllDeletesTest();
}
+ @Test(timeout = 60000)
+ public void testMetricsAndEndOfStream() throws Exception {
+ runMetricsAndEndOfStreamTest();
+ }
+
}
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ProducerITCase.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ProducerITCase.java
new file mode 100644
index 0000000000000..1288347e28ac5
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ProducerITCase.java
@@ -0,0 +1,33 @@
+/*
+ * 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.flink.streaming.connectors.kafka;
+
+
+import org.junit.Test;
+
+
+@SuppressWarnings("serial")
+public class Kafka09ProducerITCase extends KafkaProducerTestBase {
+
+ @Test
+ public void testCustomPartitioning() {
+ runCustomPartitioningTest();
+ }
+
+}
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java
new file mode 100644
index 0000000000000..a2c4f736dfe88
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java
@@ -0,0 +1,115 @@
+/*
+ * 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.flink.streaming.connectors.kafka;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.connectors.kafka.testutils.MockRuntimeContext;
+import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
+import org.apache.flink.util.TestLogger;
+import org.apache.kafka.clients.producer.Callback;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.PartitionInfo;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+import java.util.Arrays;
+import java.util.Properties;
+import java.util.concurrent.Future;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.anyString;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import static org.powermock.api.mockito.PowerMockito.whenNew;
+
+@RunWith(PowerMockRunner.class)
+@PrepareForTest(FlinkKafkaProducerBase.class)
+public class KafkaProducerTest extends TestLogger {
+
+ @Test
+ @SuppressWarnings("unchecked")
+ public void testPropagateExceptions() {
+ try {
+ // mock kafka producer
+ KafkaProducer, ?> kafkaProducerMock = mock(KafkaProducer.class);
+
+ // partition setup
+ when(kafkaProducerMock.partitionsFor(anyString())).thenReturn(
+ Arrays.asList(new PartitionInfo("mock_topic", 42, null, null, null)));
+
+ // failure when trying to send an element
+ when(kafkaProducerMock.send(any(ProducerRecord.class), any(Callback.class)))
+ .thenAnswer(new Answer>() {
+ @Override
+ public Future answer(InvocationOnMock invocation) throws Throwable {
+ Callback callback = (Callback) invocation.getArguments()[1];
+ callback.onCompletion(null, new Exception("Test error"));
+ return null;
+ }
+ });
+
+ // make sure the FlinkKafkaProducer instantiates our mock producer
+ whenNew(KafkaProducer.class).withAnyArguments().thenReturn(kafkaProducerMock);
+
+ // (1) producer that propagates errors
+
+ FlinkKafkaProducer09 producerPropagating = new FlinkKafkaProducer09<>(
+ "mock_topic", new SimpleStringSchema(), new Properties(), null);
+
+ producerPropagating.setRuntimeContext(new MockRuntimeContext(17, 3));
+ producerPropagating.open(new Configuration());
+
+ try {
+ producerPropagating.invoke("value");
+ producerPropagating.invoke("value");
+ fail("This should fail with an exception");
+ }
+ catch (Exception e) {
+ assertNotNull(e.getCause());
+ assertNotNull(e.getCause().getMessage());
+ assertTrue(e.getCause().getMessage().contains("Test error"));
+ }
+
+ // (2) producer that only logs errors
+
+ FlinkKafkaProducer09 producerLogging = new FlinkKafkaProducer09<>(
+ "mock_topic", new SimpleStringSchema(), new Properties(), null);
+ producerLogging.setLogFailuresOnly(true);
+
+ producerLogging.setRuntimeContext(new MockRuntimeContext(17, 3));
+ producerLogging.open(new Configuration());
+
+ producerLogging.invoke("value");
+ producerLogging.invoke("value");
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+}
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java
new file mode 100644
index 0000000000000..0855ba66ac10d
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java
@@ -0,0 +1,340 @@
+/*
+ * 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.flink.streaming.connectors.kafka;
+
+import kafka.admin.AdminUtils;
+import kafka.common.KafkaException;
+import kafka.consumer.ConsumerConfig;
+import kafka.api.PartitionMetadata;
+import kafka.network.SocketServer;
+import kafka.server.KafkaConfig;
+import kafka.server.KafkaServer;
+import kafka.utils.SystemTime$;
+import kafka.utils.ZkUtils;
+import org.I0Itec.zkclient.ZkClient;
+import org.apache.commons.io.FileUtils;
+import org.apache.curator.test.TestingServer;
+import org.apache.flink.streaming.connectors.kafka.internals.ZooKeeperStringSerializer;
+import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
+import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
+import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
+import org.apache.flink.util.NetUtils;
+import org.apache.kafka.common.protocol.SecurityProtocol;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.collection.Seq;
+
+import java.io.File;
+import java.net.BindException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+import java.util.UUID;
+
+import static org.apache.flink.util.NetUtils.hostAndPortToUrlString;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * An implementation of the KafkaServerProvider for Kafka 0.9
+ */
+public class KafkaTestEnvironmentImpl extends KafkaTestEnvironment {
+
+ protected static final Logger LOG = LoggerFactory.getLogger(KafkaTestEnvironmentImpl.class);
+ private File tmpZkDir;
+ private File tmpKafkaParent;
+ private List tmpKafkaDirs;
+ private List brokers;
+ private TestingServer zookeeper;
+ private String zookeeperConnectionString;
+ private String brokerConnectionString = "";
+ private Properties standardProps;
+ private ConsumerConfig standardCC;
+
+
+ public String getBrokerConnectionString() {
+ return brokerConnectionString;
+ }
+
+ @Override
+ public ConsumerConfig getStandardConsumerConfig() {
+ return standardCC;
+ }
+
+ @Override
+ public Properties getStandardProperties() {
+ return standardProps;
+ }
+
+ @Override
+ public String getVersion() {
+ return "0.9";
+ }
+
+ @Override
+ public List getBrokers() {
+ return brokers;
+ }
+
+ @Override
+ public FlinkKafkaConsumerBase getConsumer(List topics, KeyedDeserializationSchema readSchema, Properties props) {
+ return new FlinkKafkaConsumer09<>(topics, readSchema, props);
+ }
+
+ @Override
+ public FlinkKafkaProducerBase getProducer(String topic, KeyedSerializationSchema serSchema, Properties props, KafkaPartitioner partitioner) {
+ return new FlinkKafkaProducer09<>(topic, serSchema, props, partitioner);
+ }
+
+ @Override
+ public void restartBroker(int leaderId) throws Exception {
+ brokers.set(leaderId, getKafkaServer(leaderId, tmpKafkaDirs.get(leaderId), KAFKA_HOST, zookeeperConnectionString));
+ }
+
+ @Override
+ public int getLeaderToShutDown(String topic) throws Exception {
+ ZkUtils zkUtils = getZkUtils();
+ try {
+ PartitionMetadata firstPart = null;
+ do {
+ if (firstPart != null) {
+ LOG.info("Unable to find leader. error code {}", firstPart.errorCode());
+ // not the first try. Sleep a bit
+ Thread.sleep(150);
+ }
+
+ Seq partitionMetadata = AdminUtils.fetchTopicMetadataFromZk(topic, zkUtils).partitionsMetadata();
+ firstPart = partitionMetadata.head();
+ }
+ while (firstPart.errorCode() != 0);
+
+ return firstPart.leader().get().id();
+ } finally {
+ zkUtils.close();
+ }
+ }
+
+ @Override
+ public int getBrokerId(KafkaServer server) {
+ return server.config().brokerId();
+ }
+
+ @Override
+ public void prepare(int numKafkaServers) {
+ File tempDir = new File(System.getProperty("java.io.tmpdir"));
+
+ tmpZkDir = new File(tempDir, "kafkaITcase-zk-dir-" + (UUID.randomUUID().toString()));
+ assertTrue("cannot create zookeeper temp dir", tmpZkDir.mkdirs());
+
+ tmpKafkaParent = new File(tempDir, "kafkaITcase-kafka-dir*" + (UUID.randomUUID().toString()));
+ assertTrue("cannot create kafka temp dir", tmpKafkaParent.mkdirs());
+
+ tmpKafkaDirs = new ArrayList<>(numKafkaServers);
+ for (int i = 0; i < numKafkaServers; i++) {
+ File tmpDir = new File(tmpKafkaParent, "server-" + i);
+ assertTrue("cannot create kafka temp dir", tmpDir.mkdir());
+ tmpKafkaDirs.add(tmpDir);
+ }
+
+ zookeeper = null;
+ brokers = null;
+
+ try {
+ LOG.info("Starting Zookeeper");
+ zookeeper = new TestingServer(-1, tmpZkDir);
+ zookeeperConnectionString = zookeeper.getConnectString();
+
+ LOG.info("Starting KafkaServer");
+ brokers = new ArrayList<>(numKafkaServers);
+
+ for (int i = 0; i < numKafkaServers; i++) {
+ brokers.add(getKafkaServer(i, tmpKafkaDirs.get(i), KafkaTestEnvironment.KAFKA_HOST, zookeeperConnectionString));
+
+ SocketServer socketServer = brokers.get(i).socketServer();
+ brokerConnectionString += hostAndPortToUrlString(KafkaTestEnvironment.KAFKA_HOST, brokers.get(i).socketServer().boundPort(SecurityProtocol.PLAINTEXT)) + ",";
+ }
+
+ LOG.info("ZK and KafkaServer started.");
+ }
+ catch (Throwable t) {
+ t.printStackTrace();
+ fail("Test setup failed: " + t.getMessage());
+ }
+
+ standardProps = new Properties();
+ standardProps.setProperty("zookeeper.connect", zookeeperConnectionString);
+ standardProps.setProperty("bootstrap.servers", brokerConnectionString);
+ standardProps.setProperty("group.id", "flink-tests");
+ standardProps.setProperty("auto.commit.enable", "false");
+ standardProps.setProperty("zookeeper.session.timeout.ms", "12000"); // 6 seconds is default. Seems to be too small for travis.
+ standardProps.setProperty("zookeeper.connection.timeout.ms", "20000");
+ standardProps.setProperty("auto.offset.reset", "earliest"); // read from the beginning.
+ standardProps.setProperty("fetch.message.max.bytes", "256"); // make a lot of fetches (MESSAGES MUST BE SMALLER!)
+
+ Properties consumerConfigProps = new Properties();
+ consumerConfigProps.putAll(standardProps);
+ consumerConfigProps.setProperty("auto.offset.reset", "smallest");
+ standardCC = new ConsumerConfig(consumerConfigProps);
+ }
+
+ @Override
+ public void shutdown() {
+ for (KafkaServer broker : brokers) {
+ if (broker != null) {
+ broker.shutdown();
+ }
+ }
+ brokers.clear();
+
+ if (zookeeper != null) {
+ try {
+ zookeeper.stop();
+ }
+ catch (Exception e) {
+ LOG.warn("ZK.stop() failed", e);
+ }
+ zookeeper = null;
+ }
+
+ // clean up the temp spaces
+
+ if (tmpKafkaParent != null && tmpKafkaParent.exists()) {
+ try {
+ FileUtils.deleteDirectory(tmpKafkaParent);
+ }
+ catch (Exception e) {
+ // ignore
+ }
+ }
+ if (tmpZkDir != null && tmpZkDir.exists()) {
+ try {
+ FileUtils.deleteDirectory(tmpZkDir);
+ }
+ catch (Exception e) {
+ // ignore
+ }
+ }
+ }
+
+ public ZkUtils getZkUtils() {
+ ZkClient creator = new ZkClient(standardCC.zkConnect(), standardCC.zkSessionTimeoutMs(),
+ standardCC.zkConnectionTimeoutMs(), new ZooKeeperStringSerializer());
+ return ZkUtils.apply(creator, false);
+ }
+
+ @Override
+ public void createTestTopic(String topic, int numberOfPartitions, int replicationFactor) {
+ // create topic with one client
+ Properties topicConfig = new Properties();
+ LOG.info("Creating topic {}", topic);
+
+ ZkUtils zkUtils = getZkUtils();
+ try {
+ AdminUtils.createTopic(zkUtils, topic, numberOfPartitions, replicationFactor, topicConfig);
+ } finally {
+ zkUtils.close();
+ }
+
+ // validate that the topic has been created
+ final long deadline = System.currentTimeMillis() + 30000;
+ do {
+ try {
+ Thread.sleep(100);
+ } catch (InterruptedException e) {
+ // restore interrupted state
+ }
+ // we could use AdminUtils.topicExists(zkUtils, topic) here, but it's results are
+ // not always correct.
+
+ // create a new ZK utils connection
+ ZkUtils checkZKConn = getZkUtils();
+ if(AdminUtils.topicExists(checkZKConn, topic)) {
+ checkZKConn.close();
+ return;
+ }
+ checkZKConn.close();
+ }
+ while (System.currentTimeMillis() < deadline);
+ fail("Test topic could not be created");
+ }
+
+ @Override
+ public void deleteTestTopic(String topic) {
+ ZkUtils zkUtils = getZkUtils();
+ try {
+ LOG.info("Deleting topic {}", topic);
+
+ ZkClient zk = new ZkClient(standardCC.zkConnect(), standardCC.zkSessionTimeoutMs(),
+ standardCC.zkConnectionTimeoutMs(), new ZooKeeperStringSerializer());
+
+ AdminUtils.deleteTopic(zkUtils, topic);
+
+ zk.close();
+ } finally {
+ zkUtils.close();
+ }
+ }
+
+ /**
+ * Copied from com.github.sakserv.minicluster.KafkaLocalBrokerIntegrationTest (ASL licensed)
+ */
+ protected static KafkaServer getKafkaServer(int brokerId, File tmpFolder,
+ String kafkaHost,
+ String zookeeperConnectionString) throws Exception {
+ Properties kafkaProperties = new Properties();
+
+ // properties have to be Strings
+ kafkaProperties.put("advertised.host.name", kafkaHost);
+ kafkaProperties.put("broker.id", Integer.toString(brokerId));
+ kafkaProperties.put("log.dir", tmpFolder.toString());
+ kafkaProperties.put("zookeeper.connect", zookeeperConnectionString);
+ kafkaProperties.put("message.max.bytes", String.valueOf(50 * 1024 * 1024));
+ kafkaProperties.put("replica.fetch.max.bytes", String.valueOf(50 * 1024 * 1024));
+
+ // for CI stability, increase zookeeper session timeout
+ kafkaProperties.put("zookeeper.session.timeout.ms", "20000");
+
+ final int numTries = 5;
+
+ for (int i = 1; i <= numTries; i++) {
+ int kafkaPort = NetUtils.getAvailablePort();
+ kafkaProperties.put("port", Integer.toString(kafkaPort));
+ KafkaConfig kafkaConfig = new KafkaConfig(kafkaProperties);
+
+ try {
+ scala.Option stringNone = scala.Option.apply(null);
+ KafkaServer server = new KafkaServer(kafkaConfig, SystemTime$.MODULE$, stringNone);
+ server.startup();
+ return server;
+ }
+ catch (KafkaException e) {
+ if (e.getCause() instanceof BindException) {
+ // port conflict, retry...
+ LOG.info("Port conflict when starting Kafka Broker. Retrying...");
+ }
+ else {
+ throw e;
+ }
+ }
+ }
+
+ throw new Exception("Could not start Kafka after " + numTries + " retries due to port conflicts.");
+ }
+
+}
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/resources/log4j-test.properties b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/resources/log4j-test.properties
new file mode 100644
index 0000000000000..6bdfb48cd550f
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/resources/log4j-test.properties
@@ -0,0 +1,29 @@
+################################################################################
+# 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.
+################################################################################
+
+log4j.rootLogger=INFO, testlogger
+
+log4j.appender.testlogger=org.apache.log4j.ConsoleAppender
+log4j.appender.testlogger.target = System.err
+log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout
+log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
+
+# suppress the irrelevant (wrong) warnings from the netty channel handler
+log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger
+
+
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/resources/logback-test.xml b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/resources/logback-test.xml
new file mode 100644
index 0000000000000..45b3b92f0cf65
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/resources/logback-test.xml
@@ -0,0 +1,30 @@
+
+
+
+
+
+ %d{HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n
+
+
+
+
+
+
+
+
\ No newline at end of file
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/pom.xml b/flink-streaming-connectors/flink-connector-kafka-base/pom.xml
new file mode 100644
index 0000000000000..354d3538eb30b
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka-base/pom.xml
@@ -0,0 +1,169 @@
+
+
+
+
+ 4.0.0
+
+
+ org.apache.flink
+ flink-streaming-connectors-parent
+ 1.0-SNAPSHOT
+ ..
+
+
+ flink-connector-kafka-base
+ flink-connector-kafka-base
+
+ jar
+
+
+
+ 0.8.2.0
+
+
+
+
+
+ org.apache.flink
+ flink-streaming-java
+ ${project.version}
+
+
+
+ org.apache.kafka
+ kafka_${scala.binary.version}
+ ${kafka.version}
+
+
+ com.sun.jmx
+ jmxri
+
+
+ com.sun.jdmk
+ jmxtools
+
+
+ log4j
+ log4j
+
+
+ org.slf4j
+ slf4j-simple
+
+
+ net.sf.jopt-simple
+ jopt-simple
+
+
+ org.scala-lang
+ scala-reflect
+
+
+ org.scala-lang
+ scala-compiler
+
+
+ com.yammer.metrics
+ metrics-annotation
+
+
+ org.xerial.snappy
+ snappy-java
+
+
+
+
+
+
+ com.101tec
+ zkclient
+ 0.7
+ jar
+
+
+
+ com.google.guava
+ guava
+ ${guava.version}
+
+
+
+ org.apache.curator
+ curator-test
+ ${curator.version}
+ test
+
+
+
+ org.apache.flink
+ flink-tests
+ ${project.version}
+ test-jar
+ test
+
+
+
+ org.apache.flink
+ flink-test-utils
+ ${project.version}
+ test
+
+
+
+
+
+
+
+ com.101tec
+ zkclient
+ 0.7
+
+
+
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-jar-plugin
+
+
+
+ test-jar
+
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-failsafe-plugin
+
+
+ 1
+
+
+
+
+
+
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java
new file mode 100644
index 0000000000000..3c3658686a8aa
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java
@@ -0,0 +1,225 @@
+/*
+ * 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.flink.streaming.connectors.kafka;
+
+import org.apache.commons.collections.map.LinkedMap;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedAsynchronously;
+import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
+import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
+import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+
+public abstract class FlinkKafkaConsumerBase extends RichParallelSourceFunction
+ implements CheckpointNotifier, CheckpointedAsynchronously>, ResultTypeQueryable {
+
+ // ------------------------------------------------------------------------
+
+ private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaConsumerBase.class);
+
+ private static final long serialVersionUID = -6272159445203409112L;
+
+ /** The maximum number of pending non-committed checkpoints to track, to avoid memory leaks */
+ public static final int MAX_NUM_PENDING_CHECKPOINTS = 100;
+
+
+ /** The schema to convert between Kafka#s byte messages, and Flink's objects */
+ protected final KeyedDeserializationSchema deserializer;
+
+ // ------ Runtime State -------
+
+ /** Data for pending but uncommitted checkpoints */
+ protected final LinkedMap pendingCheckpoints = new LinkedMap();
+
+ /** The offsets of the last returned elements */
+ protected transient HashMap offsetsState;
+
+ /** The offsets to restore to, if the consumer restores state from a checkpoint */
+ protected transient HashMap restoreToOffset;
+
+ /** Flag indicating whether the consumer is still running **/
+ protected volatile boolean running = true;
+
+ // ------------------------------------------------------------------------
+
+
+ /**
+ * Creates a new Flink Kafka Consumer, using the given type of fetcher and offset handler.
+ *
+ * To determine which kink of fetcher and offset handler to use, please refer to the docs
+ * at the beginnign of this class.
+ *
+ * @param deserializer
+ * The deserializer to turn raw byte messages into Java/Scala objects.
+ * @param props
+ * The properties that are used to configure both the fetcher and the offset handler.
+ */
+ public FlinkKafkaConsumerBase(KeyedDeserializationSchema deserializer, Properties props) {
+ this.deserializer = checkNotNull(deserializer, "valueDeserializer");
+ }
+
+ // ------------------------------------------------------------------------
+ // Checkpoint and restore
+ // ------------------------------------------------------------------------
+
+ @Override
+ public HashMap snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
+ if (offsetsState == null) {
+ LOG.debug("snapshotState() requested on not yet opened source; returning null.");
+ return null;
+ }
+ if (!running) {
+ LOG.debug("snapshotState() called on closed source");
+ return null;
+ }
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Snapshotting state. Offsets: {}, checkpoint id: {}, timestamp: {}",
+ KafkaTopicPartition.toString(offsetsState), checkpointId, checkpointTimestamp);
+ }
+
+ // the use of clone() is okay here is okay, we just need a new map, the keys are not changed
+ //noinspection unchecked
+ HashMap currentOffsets = (HashMap) offsetsState.clone();
+
+ // the map cannot be asynchronously updated, because only one checkpoint call can happen
+ // on this function at a time: either snapshotState() or notifyCheckpointComplete()
+ pendingCheckpoints.put(checkpointId, currentOffsets);
+
+ while (pendingCheckpoints.size() > MAX_NUM_PENDING_CHECKPOINTS) {
+ pendingCheckpoints.remove(0);
+ }
+
+ return currentOffsets;
+ }
+
+ @Override
+ public void restoreState(HashMap restoredOffsets) {
+ LOG.info("Setting restore state in Kafka");
+ restoreToOffset = restoredOffsets;
+ }
+
+ @Override
+ public void notifyCheckpointComplete(long checkpointId) throws Exception {
+ if (offsetsState == null) {
+ LOG.debug("notifyCheckpointComplete() called on uninitialized source");
+ return;
+ }
+ if (!running) {
+ LOG.debug("notifyCheckpointComplete() called on closed source");
+ return;
+ }
+
+ // only one commit operation must be in progress
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Committing offsets externally for checkpoint {}", checkpointId);
+ }
+
+ try {
+ HashMap checkpointOffsets;
+
+ // the map may be asynchronously updates when snapshotting state, so we synchronize
+ synchronized (pendingCheckpoints) {
+ final int posInMap = pendingCheckpoints.indexOf(checkpointId);
+ if (posInMap == -1) {
+ LOG.warn("Received confirmation for unknown checkpoint id {}", checkpointId);
+ return;
+ }
+
+ //noinspection unchecked
+ checkpointOffsets = (HashMap) pendingCheckpoints.remove(posInMap);
+
+
+ // remove older checkpoints in map
+ for (int i = 0; i < posInMap; i++) {
+ pendingCheckpoints.remove(0);
+ }
+ }
+ if (checkpointOffsets == null || checkpointOffsets.size() == 0) {
+ LOG.debug("Checkpoint state was empty.");
+ return;
+ }
+ commitOffsets(checkpointOffsets);
+ }
+ catch (Exception e) {
+ if (running) {
+ throw e;
+ }
+ // else ignore exception if we are no longer running
+ }
+ }
+
+ protected abstract void commitOffsets(HashMap checkpointOffsets) throws Exception;
+
+
+ @Override
+ public TypeInformation getProducedType() {
+ return deserializer.getProducedType();
+ }
+
+ protected static List assignPartitions(List partitions, int numConsumers, int consumerIndex) {
+ checkArgument(numConsumers > 0);
+ checkArgument(consumerIndex < numConsumers);
+
+ List partitionsToSub = new ArrayList<>();
+
+ for (int i = 0; i < partitions.size(); i++) {
+ if (i % numConsumers == consumerIndex) {
+ partitionsToSub.add(partitions.get(i));
+ }
+ }
+ return partitionsToSub;
+ }
+
+ /**
+ * Method to log partition information.
+ * @param partitionInfos List of subscribed partitions
+ */
+ public static void logPartitionInfo(List partitionInfos) {
+ Map countPerTopic = new HashMap<>();
+ for (KafkaTopicPartition partition : partitionInfos) {
+ Integer count = countPerTopic.get(partition.getTopic());
+ if (count == null) {
+ count = 1;
+ } else {
+ count++;
+ }
+ countPerTopic.put(partition.getTopic(), count);
+ }
+ StringBuilder sb = new StringBuilder();
+ for (Map.Entry e : countPerTopic.entrySet()) {
+ sb.append(e.getKey()).append(" (").append(e.getValue()).append("), ");
+ }
+ LOG.info("Consumer is going to read the following topics (with number of partitions): ", sb.toString());
+ }
+
+
+}
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBase.java
similarity index 63%
rename from flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java
rename to flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBase.java
index 7e01b54f42c50..ebc02c997bdcd 100644
--- a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java
+++ b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBase.java
@@ -23,11 +23,9 @@
import org.apache.flink.api.java.ClosureCleaner;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
-import org.apache.flink.streaming.connectors.kafka.partitioner.FixedPartitioner;
+import org.apache.flink.streaming.connectors.kafka.internals.metrics.DefaultKafkaMetricAccumulator;
import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
-import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
import org.apache.flink.util.NetUtils;
import org.apache.kafka.clients.producer.Callback;
@@ -35,6 +33,8 @@
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.serialization.ByteArraySerializer;
@@ -42,6 +42,7 @@
import org.slf4j.LoggerFactory;
import java.util.List;
+import java.util.Map;
import java.util.Properties;
@@ -52,130 +53,60 @@
*
* @param Type of the messages to write into Kafka.
*/
-public class FlinkKafkaProducer extends RichSinkFunction {
+public abstract class FlinkKafkaProducerBase extends RichSinkFunction {
- private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaProducer.class);
+ private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaProducerBase.class);
private static final long serialVersionUID = 1L;
+ /**
+ * Configuration key for disabling the metrics reporting
+ */
+ public static final String KEY_DISABLE_METRICS = "flink.disable-metrics";
+
/**
* Array with the partition ids of the given topicId
* The size of this array is the number of partitions
*/
- private final int[] partitions;
+ protected final int[] partitions;
/**
* User defined properties for the Producer
*/
- private final Properties producerConfig;
+ protected final Properties producerConfig;
/**
* The name of the topic this producer is writing data to
*/
- private final String topicId;
+ protected final String topicId;
/**
* (Serializable) SerializationSchema for turning objects used with Flink into
* byte[] for Kafka.
*/
- private final KeyedSerializationSchema schema;
+ protected final KeyedSerializationSchema schema;
/**
* User-provided partitioner for assigning an object to a Kafka partition.
*/
- private final KafkaPartitioner partitioner;
+ protected final KafkaPartitioner partitioner;
/**
* Flag indicating whether to accept failures (and log them), or to fail on failures
*/
- private boolean logFailuresOnly;
+ protected boolean logFailuresOnly;
// -------------------------------- Runtime fields ------------------------------------------
/** KafkaProducer instance */
- private transient KafkaProducer producer;
+ protected transient KafkaProducer producer;
/** The callback than handles error propagation or logging callbacks */
- private transient Callback callback;
+ protected transient Callback callback;
/** Errors encountered in the async producer are stored here */
- private transient volatile Exception asyncException;
-
- // ------------------- Keyless serialization schema constructors ----------------------
- /**
- * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to
- * the topic.
- *
- * @param brokerList
- * Comma separated addresses of the brokers
- * @param topicId
- * ID of the Kafka topic.
- * @param serializationSchema
- * User defined (keyless) serialization schema.
- */
- public FlinkKafkaProducer(String brokerList, String topicId, SerializationSchema serializationSchema) {
- this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), null);
- }
-
- /**
- * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to
- * the topic.
- *
- * @param topicId
- * ID of the Kafka topic.
- * @param serializationSchema
- * User defined (keyless) serialization schema.
- * @param producerConfig
- * Properties with the producer configuration.
- */
- public FlinkKafkaProducer(String topicId, SerializationSchema serializationSchema, Properties producerConfig) {
- this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, null);
- }
-
- /**
- * The main constructor for creating a FlinkKafkaProducer.
- *
- * @param topicId The topic to write data to
- * @param serializationSchema A (keyless) serializable serialization schema for turning user objects into a kafka-consumable byte[]
- * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
- * @param customPartitioner A serializable partitioner for assining messages to Kafka partitions.
- */
- public FlinkKafkaProducer(String topicId, SerializationSchema serializationSchema, Properties producerConfig, KafkaPartitioner customPartitioner) {
- this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner);
-
- }
-
- // ------------------- Key/Value serialization schema constructors ----------------------
+ protected transient volatile Exception asyncException;
- /**
- * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to
- * the topic.
- *
- * @param brokerList
- * Comma separated addresses of the brokers
- * @param topicId
- * ID of the Kafka topic.
- * @param serializationSchema
- * User defined serialization schema supporting key/value messages
- */
- public FlinkKafkaProducer(String brokerList, String topicId, KeyedSerializationSchema serializationSchema) {
- this(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), null);
- }
-
- /**
- * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to
- * the topic.
- *
- * @param topicId
- * ID of the Kafka topic.
- * @param serializationSchema
- * User defined serialization schema supporting key/value messages
- * @param producerConfig
- * Properties with the producer configuration.
- */
- public FlinkKafkaProducer(String topicId, KeyedSerializationSchema serializationSchema, Properties producerConfig) {
- this(topicId, serializationSchema, producerConfig, null);
- }
/**
* The main constructor for creating a FlinkKafkaProducer.
@@ -183,9 +114,9 @@ public FlinkKafkaProducer(String topicId, KeyedSerializationSchema serializa
* @param topicId The topic to write data to
* @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
* @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
- * @param customPartitioner A serializable partitioner for assining messages to Kafka partitions.
+ * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner
*/
- public FlinkKafkaProducer(String topicId, KeyedSerializationSchema serializationSchema, Properties producerConfig, KafkaPartitioner customPartitioner) {
+ public FlinkKafkaProducerBase(String topicId, KeyedSerializationSchema serializationSchema, Properties producerConfig, KafkaPartitioner customPartitioner) {
Preconditions.checkNotNull(topicId, "TopicID not set");
Preconditions.checkNotNull(serializationSchema, "serializationSchema not set");
Preconditions.checkNotNull(producerConfig, "producerConfig not set");
@@ -223,11 +154,7 @@ public FlinkKafkaProducer(String topicId, KeyedSerializationSchema serializa
getPartitionsProd.close();
}
- if (customPartitioner == null) {
- this.partitioner = new FixedPartitioner();
- } else {
- this.partitioner = customPartitioner;
- }
+ this.partitioner = customPartitioner;
}
// ---------------------------------- Properties --------------------------
@@ -254,11 +181,32 @@ public void open(Configuration configuration) {
producer = new org.apache.kafka.clients.producer.KafkaProducer<>(this.producerConfig);
RuntimeContext ctx = getRuntimeContext();
- partitioner.open(ctx.getIndexOfThisSubtask(), ctx.getNumberOfParallelSubtasks(), partitions);
+ if(partitioner != null) {
+ partitioner.open(ctx.getIndexOfThisSubtask(), ctx.getNumberOfParallelSubtasks(), partitions);
+ }
LOG.info("Starting FlinkKafkaProducer ({}/{}) to produce into topic {}",
ctx.getIndexOfThisSubtask(), ctx.getNumberOfParallelSubtasks(), topicId);
-
+
+ // register Kafka metrics to Flink accumulators
+ if(!Boolean.getBoolean(producerConfig.getProperty(KEY_DISABLE_METRICS, "false"))) {
+ Map metrics = this.producer.metrics();
+
+ if(metrics == null) {
+ // MapR's Kafka implementation returns null here.
+ LOG.info("Producer implementation does not support metrics");
+ } else {
+ for(Map.Entry metric: metrics.entrySet()) {
+ String name = "producer-" + metric.getKey().name();
+ DefaultKafkaMetricAccumulator kafkaAccumulator = DefaultKafkaMetricAccumulator.createFor(metric.getValue());
+ // best effort: we only add the accumulator if available.
+ if(kafkaAccumulator != null) {
+ getRuntimeContext().addAccumulator(name, kafkaAccumulator);
+ }
+ }
+ }
+ }
+
if (logFailuresOnly) {
callback = new Callback() {
@@ -295,10 +243,13 @@ public void invoke(IN next) throws Exception {
byte[] serializedKey = schema.serializeKey(next);
byte[] serializedValue = schema.serializeValue(next);
- ProducerRecord record = new ProducerRecord<>(topicId,
- partitioner.partition(next, partitions.length),
- serializedKey, serializedValue);
-
+ ProducerRecord record;
+ if(partitioner == null) {
+ record = new ProducerRecord<>(topicId, serializedKey, serializedValue);
+ } else {
+ record = new ProducerRecord<>(topicId, partitioner.partition(next, serializedKey, serializedValue, partitions.length), serializedKey, serializedValue);
+ }
+
producer.send(record, callback);
}
@@ -316,7 +267,7 @@ public void close() throws Exception {
// ----------------------------------- Utilities --------------------------
- private void checkErroneous() throws Exception {
+ protected void checkErroneous() throws Exception {
Exception e = asyncException;
if (e != null) {
// prevent double throwing
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java
similarity index 92%
rename from flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java
rename to flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java
index f269aa32d34f7..1168b2734322d 100644
--- a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java
+++ b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java
@@ -121,4 +121,12 @@ public boolean isContained(Map map) {
}
return false;
}
+
+ public static List convertToPartitionInfo(List partitionInfos) {
+ List ret = new ArrayList<>(partitionInfos.size());
+ for(KafkaTopicPartitionLeader ktpl: partitionInfos) {
+ ret.add(ktpl.getTopicPartition());
+ }
+ return ret;
+ }
}
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionLeader.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionLeader.java
similarity index 100%
rename from flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionLeader.java
rename to flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionLeader.java
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZooKeeperStringSerializer.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZooKeeperStringSerializer.java
similarity index 100%
rename from flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZooKeeperStringSerializer.java
rename to flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZooKeeperStringSerializer.java
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/AvgKafkaMetricAccumulator.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/AvgKafkaMetricAccumulator.java
new file mode 100644
index 0000000000000..a0387116cd263
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/AvgKafkaMetricAccumulator.java
@@ -0,0 +1,141 @@
+/*
+ * 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.flink.streaming.connectors.kafka.internals.metrics;
+
+import org.apache.flink.api.common.accumulators.Accumulator;
+import org.apache.kafka.common.metrics.KafkaMetric;
+import org.apache.kafka.common.metrics.Measurable;
+import org.apache.kafka.common.metrics.stats.Avg;
+import org.apache.kafka.common.metrics.stats.SampledStat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+import java.lang.reflect.Field;
+import java.util.List;
+
+public class AvgKafkaMetricAccumulator extends DefaultKafkaMetricAccumulator {
+ private static final Logger LOG = LoggerFactory.getLogger(AvgKafkaMetricAccumulator.class);
+
+ /** The last sum/count before the serialization **/
+ private AvgSumCount lastSumCount;
+
+ public AvgKafkaMetricAccumulator(KafkaMetric kafkaMetric) {
+ super(kafkaMetric);
+ }
+
+ @Override
+ public void merge(Accumulator other) {
+ if(!(other instanceof AvgKafkaMetricAccumulator)) {
+ throw new RuntimeException("Trying to merge incompatible accumulators: "+this+" with "+other);
+ }
+ AvgKafkaMetricAccumulator otherMetric = (AvgKafkaMetricAccumulator) other;
+
+ AvgSumCount thisAvg;
+ if(this.lastSumCount == null) {
+ Measurable thisMeasurable = DefaultKafkaMetricAccumulator.getMeasurableFromKafkaMetric(this.kafkaMetric);
+ if (!(thisMeasurable instanceof Avg)) {
+ throw new RuntimeException("Must be of type Avg");
+ }
+ thisAvg = getAvgSumCount((Avg) thisMeasurable);
+ } else {
+ thisAvg = this.lastSumCount;
+ }
+
+ AvgSumCount otherAvg;
+ if(otherMetric.lastSumCount == null) {
+ Measurable otherMeasurable = DefaultKafkaMetricAccumulator.getMeasurableFromKafkaMetric(otherMetric.kafkaMetric);
+ if(!(otherMeasurable instanceof Avg) ) {
+ throw new RuntimeException("Must be of type Avg");
+ }
+ otherAvg = getAvgSumCount((Avg) otherMeasurable);
+ } else {
+ otherAvg = otherMetric.lastSumCount;
+ }
+
+ thisAvg.count += otherAvg.count;
+ thisAvg.sum += otherAvg.sum;
+ this.mergedValue = thisAvg.sum / thisAvg.count;
+ }
+
+ @Override
+ public Accumulator clone() {
+ AvgKafkaMetricAccumulator clone = new AvgKafkaMetricAccumulator(kafkaMetric);
+ clone.lastSumCount = this.lastSumCount;
+ clone.isMerged = this.isMerged;
+ clone.mergedValue = this.mergedValue;
+ return clone;
+ }
+
+ // ------------ Utilities
+
+ private static class AvgSumCount implements Serializable {
+ double sum;
+ long count;
+
+ @Override
+ public String toString() {
+ return "AvgSumCount{" +
+ "sum=" + sum +
+ ", count=" + count +
+ ", avg="+(sum/count)+"}";
+ }
+ }
+
+ /**
+ * Extracts sum and count from Avg using reflection
+ *
+ * @param avg Avg SampledStat from Kafka
+ * @return A KV pair with the average's sum and count
+ */
+ private static AvgSumCount getAvgSumCount(Avg avg) {
+ try {
+ Field samplesField = SampledStat.class.getDeclaredField("samples");
+ Field sampleValue = Class.forName("org.apache.kafka.common.metrics.stats.SampledStat$Sample").getDeclaredField("value");
+ Field sampleEventCount = Class.forName("org.apache.kafka.common.metrics.stats.SampledStat$Sample").getDeclaredField("eventCount");
+ samplesField.setAccessible(true);
+ sampleValue.setAccessible(true);
+ sampleEventCount.setAccessible(true);
+ List samples = (List) samplesField.get(avg);
+ AvgSumCount res = new AvgSumCount();
+ for(int i = 0; i < samples.size(); i++) {
+ res.sum += (double)sampleValue.get(samples.get(i));
+ res.count += (long)sampleEventCount.get(samples.get(i));
+ }
+ return res;
+ } catch(Throwable t) {
+ throw new RuntimeException("Unable to extract sum and count from Avg using reflection. " +
+ "You can turn off the metrics from Flink's Kafka connector if this issue persists.", t);
+ }
+ }
+
+ private void writeObject(ObjectOutputStream out) throws IOException {
+ Measurable thisMeasurable = DefaultKafkaMetricAccumulator.getMeasurableFromKafkaMetric(this.kafkaMetric);
+ if(!(thisMeasurable instanceof Avg) ) {
+ throw new RuntimeException("Must be of type Avg");
+ }
+ this.lastSumCount = getAvgSumCount((Avg) thisMeasurable);
+ out.defaultWriteObject();
+ }
+
+ private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
+ in.defaultReadObject();
+ }
+}
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/DefaultKafkaMetricAccumulator.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/DefaultKafkaMetricAccumulator.java
new file mode 100644
index 0000000000000..06b79300c7bd8
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/DefaultKafkaMetricAccumulator.java
@@ -0,0 +1,159 @@
+/*
+ * 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.flink.streaming.connectors.kafka.internals.metrics;
+
+import org.apache.flink.api.common.accumulators.Accumulator;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.metrics.KafkaMetric;
+import org.apache.kafka.common.metrics.Measurable;
+import org.apache.kafka.common.metrics.stats.Avg;
+import org.apache.kafka.common.metrics.stats.Max;
+import org.apache.kafka.common.metrics.stats.Min;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+import java.lang.reflect.Field;
+
+public class DefaultKafkaMetricAccumulator implements Accumulator, Serializable {
+
+ private static final Logger LOG = LoggerFactory.getLogger(DefaultKafkaMetricAccumulator.class);
+
+ protected boolean isMerged = false;
+ protected double mergedValue;
+ protected transient KafkaMetric kafkaMetric;
+
+
+ public static DefaultKafkaMetricAccumulator createFor(Metric metric) {
+ if(!(metric instanceof KafkaMetric)) {
+ return null;
+ }
+ KafkaMetric kafkaMetric = (KafkaMetric) metric;
+ Measurable measurable = getMeasurableFromKafkaMetric(kafkaMetric);
+ if(measurable == null) {
+ return null;
+ }
+ if (measurable instanceof Max) {
+ return new MaxKafkaMetricAccumulator(kafkaMetric);
+ } else if (measurable instanceof Min) {
+ return new MinKafkaMetricAccumulator(kafkaMetric);
+ } else if (measurable instanceof Avg) {
+ return new AvgKafkaMetricAccumulator(kafkaMetric);
+ } else {
+ // fallback accumulator. works for Rate, Total, Count.
+ return new DefaultKafkaMetricAccumulator(kafkaMetric);
+ }
+ }
+
+ /**
+ * This utility method is using reflection to get the Measurable from the KafkaMetric.
+ * Since Kafka 0.9, Kafka is exposing the Measurable properly, but Kafka 0.8.2 does not yet expose it.
+ *
+ * @param kafkaMetric the metric to extract the field form
+ * @return Measurable type (or null in case of an error)
+ */
+ protected static Measurable getMeasurableFromKafkaMetric(KafkaMetric kafkaMetric) {
+ try {
+ Field measurableField = kafkaMetric.getClass().getDeclaredField("measurable");
+ measurableField.setAccessible(true);
+ return (Measurable) measurableField.get(kafkaMetric);
+ } catch (Throwable e) {
+ LOG.warn("Unable to initialize Kafka metric: " + kafkaMetric, e);
+ return null;
+ }
+ }
+
+
+ DefaultKafkaMetricAccumulator(KafkaMetric kafkaMetric) {
+ this.kafkaMetric = kafkaMetric;
+ }
+
+ @Override
+ public void add(Void value) {
+ // noop
+ }
+
+ @Override
+ public Double getLocalValue() {
+ if(isMerged && kafkaMetric == null) {
+ return mergedValue;
+ }
+ return kafkaMetric.value();
+ }
+
+ @Override
+ public void resetLocal() {
+ // noop
+ }
+
+ @Override
+ public void merge(Accumulator other) {
+ if(!(other instanceof DefaultKafkaMetricAccumulator)) {
+ throw new RuntimeException("Trying to merge incompatible accumulators");
+ }
+ DefaultKafkaMetricAccumulator otherMetric = (DefaultKafkaMetricAccumulator) other;
+ if(this.isMerged) {
+ if(otherMetric.isMerged) {
+ this.mergedValue += otherMetric.mergedValue;
+ } else {
+ this.mergedValue += otherMetric.getLocalValue();
+ }
+ } else {
+ this.isMerged = true;
+ if(otherMetric.isMerged) {
+ this.mergedValue = this.getLocalValue() + otherMetric.mergedValue;
+ } else {
+ this.mergedValue = this.getLocalValue() + otherMetric.getLocalValue();
+ }
+
+ }
+ }
+
+ @Override
+ public Accumulator clone() {
+ DefaultKafkaMetricAccumulator clone = new DefaultKafkaMetricAccumulator(this.kafkaMetric);
+ clone.isMerged = this.isMerged;
+ clone.mergedValue = this.mergedValue;
+ return clone;
+ }
+
+ @Override
+ public String toString() {
+ if(isMerged) {
+ return Double.toString(mergedValue);
+ }
+ if(kafkaMetric == null) {
+ return "null";
+ }
+ return Double.toString(kafkaMetric.value());
+ }
+
+ // -------- custom serialization methods
+ private void writeObject(ObjectOutputStream out) throws IOException {
+ this.isMerged = true;
+ this.mergedValue = kafkaMetric.value();
+ out.defaultWriteObject();
+ }
+
+ private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
+ in.defaultReadObject();
+ }
+}
\ No newline at end of file
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/MaxKafkaMetricAccumulator.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/MaxKafkaMetricAccumulator.java
new file mode 100644
index 0000000000000..c1770ff4eda60
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/MaxKafkaMetricAccumulator.java
@@ -0,0 +1,57 @@
+/*
+ * 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.flink.streaming.connectors.kafka.internals.metrics;
+
+import org.apache.flink.api.common.accumulators.Accumulator;
+import org.apache.kafka.common.metrics.KafkaMetric;
+
+
+public class MaxKafkaMetricAccumulator extends DefaultKafkaMetricAccumulator {
+ public MaxKafkaMetricAccumulator(KafkaMetric kafkaMetric) {
+ super(kafkaMetric);
+ }
+
+ @Override
+ public void merge(Accumulator