diff --git a/flink-connectors/flink-connector-kafka-0.10/pom.xml b/flink-connectors/flink-connector-kafka-0.10/pom.xml index 231b22e6cb875..143cb7f1baf78 100644 --- a/flink-connectors/flink-connector-kafka-0.10/pom.xml +++ b/flink-connectors/flink-connector-kafka-0.10/pom.xml @@ -130,7 +130,7 @@ under the License. ${kafka.version} test - + org.apache.flink flink-tests_${scala.binary.version} @@ -209,5 +209,5 @@ under the License. - + diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java index 23fc84e463198..1bbd1dcc6fca3 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java @@ -29,21 +29,21 @@ import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper; import org.apache.flink.util.PropertiesUtil; import org.apache.flink.util.SerializedValue; + import org.apache.kafka.clients.consumer.ConsumerConfig; import java.util.Collections; -import java.util.Map; import java.util.List; +import java.util.Map; import java.util.Properties; - /** * The Flink Kafka Consumer is a streaming data source that pulls a parallel data stream from * Apache Kafka 0.10.x. The consumer can run in multiple parallel instances, each of which will pull - * data from one or more Kafka partitions. - * + * 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". + * 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 @@ -62,11 +62,10 @@ public class FlinkKafkaConsumer010 extends FlinkKafkaConsumer09 { private static final long serialVersionUID = 2324564345203409112L; - // ------------------------------------------------------------------------ /** - * Creates a new Kafka streaming source consumer for Kafka 0.10.x + * Creates a new Kafka streaming source consumer for Kafka 0.10.x. * * @param topic * The name of the topic that should be consumed. @@ -82,7 +81,7 @@ public FlinkKafkaConsumer010(String topic, DeserializationSchema valueDeseria /** * Creates a new Kafka streaming source consumer for Kafka 0.10.x * - * This constructor allows passing a {@see KeyedDeserializationSchema} for reading key/value + *

This constructor allows passing a {@see KeyedDeserializationSchema} for reading key/value * pairs, offsets, and topic names from Kafka. * * @param topic @@ -99,7 +98,7 @@ public FlinkKafkaConsumer010(String topic, KeyedDeserializationSchema deseria /** * Creates a new Kafka streaming source consumer for Kafka 0.10.x * - * This constructor allows passing multiple topics to the consumer. + *

This constructor allows passing multiple topics to the consumer. * * @param topics * The Kafka topics to read from. @@ -115,7 +114,7 @@ public FlinkKafkaConsumer010(List topics, DeserializationSchema deser /** * Creates a new Kafka streaming source consumer for Kafka 0.10.x * - * This constructor allows passing multiple topics and a key/value deserialization schema. + *

This constructor allows passing multiple topics and a key/value deserialization schema. * * @param topics * The Kafka topics to read from. diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java index 711fe072343da..805bc4e13e084 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java @@ -17,8 +17,6 @@ package org.apache.flink.streaming.connectors.kafka; -import java.util.Properties; - import org.apache.flink.api.common.functions.IterationRuntimeContext; import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.functions.RuntimeContext; @@ -37,34 +35,35 @@ 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.kafka.clients.producer.ProducerRecord; +import java.util.Properties; + import static org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducerBase.getPartitionsByTopic; import static org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducerBase.getPropertiesFromBrokerList; - /** * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.10.x * - * Implementation note: This producer is a hybrid between a regular regular sink function (a) + *

Implementation note: This producer is a hybrid between a regular regular sink function (a) * and a custom operator (b). * - * For (a), the class implements the SinkFunction and RichFunction interfaces. + *

For (a), the class implements the SinkFunction and RichFunction interfaces. * For (b), it extends the StreamTask class. * - * Details about approach (a): - * + *

Details about approach (a): * Pre Kafka 0.10 producers only follow approach (a), allowing users to use the producer using the * DataStream.addSink() method. * Since the APIs exposed in that variant do not allow accessing the the timestamp attached to the record * the Kafka 0.10 producer has a second invocation option, approach (b). * - * Details about approach (b): + *

Details about approach (b): * Kafka 0.10 supports writing the timestamp attached to a record to Kafka. When adding the * FlinkKafkaProducer010 using the FlinkKafkaProducer010.writeToKafkaWithTimestamps() method, the Kafka producer * can access the internal record timestamp of the record and write it to Kafka. * - * All methods and constructors in this class are marked with the approach they are needed for. + *

All methods and constructors in this class are marked with the approach they are needed for. */ public class FlinkKafkaProducer010 extends StreamSink implements SinkFunction, RichFunction { @@ -79,7 +78,7 @@ public class FlinkKafkaProducer010 extends StreamSink implements SinkFunct * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to * the topic. * - * This constructor allows writing timestamps to Kafka, it follow approach (b) (see above) + *

This constructor allows writing timestamps to Kafka, it follow approach (b) (see above) * * @param inStream The stream to write to Kafka * @param topicId ID of the Kafka topic. @@ -93,12 +92,11 @@ public static FlinkKafkaProducer010Configuration writeToKafkaWithTimestam return writeToKafkaWithTimestamps(inStream, topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner()); } - /** * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to * the topic. * - * This constructor allows writing timestamps to Kafka, it follow approach (b) (see above) + *

This constructor allows writing timestamps to Kafka, it follow approach (b) (see above) * * @param inStream The stream to write to Kafka * @param topicId ID of the Kafka topic. @@ -116,7 +114,7 @@ public static FlinkKafkaProducer010Configuration writeToKafkaWithTimestam * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to * the topic. * - * This constructor allows writing timestamps to Kafka, it follow approach (b) (see above) + *

This constructor allows writing timestamps to Kafka, it follow approach (b) (see above) * * @param inStream The stream to write to Kafka * @param topicId The name of the target topic @@ -212,11 +210,11 @@ public FlinkKafkaProducer010(String brokerList, String topicId, KeyedSerializati public FlinkKafkaProducer010(String topicId, KeyedSerializationSchema serializationSchema, Properties producerConfig) { this(topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner()); } - + /** - * Create Kafka producer + * Create Kafka producer. * - * This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above) + *

This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above) */ public FlinkKafkaProducer010(String topicId, KeyedSerializationSchema serializationSchema, Properties producerConfig, FlinkKafkaPartitioner customPartitioner) { // We create a Kafka 09 producer instance here and only "override" (by intercepting) the @@ -230,7 +228,7 @@ public FlinkKafkaProducer010(String topicId, KeyedSerializationSchema seriali * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to * the topic. * - * This constructor allows writing timestamps to Kafka, it follow approach (b) (see above) + *

This constructor allows writing timestamps to Kafka, it follow approach (b) (see above) * * @param inStream The stream to write to Kafka * @param topicId The name of the target topic @@ -275,9 +273,9 @@ public FlinkKafkaProducer010(String topicId, SerializationSchema serializatio } /** - * Create Kafka producer + * Create Kafka producer. * - * This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above) + *

This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above) * * @deprecated This is a deprecated constructor that does not correctly handle partitioning when * producing to multiple topics. Use @@ -306,13 +304,13 @@ private void invokeInternal(T next, long elementTimestamp) throws Exception { } Long timestamp = null; - if(this.writeTimestampToKafka) { + if (this.writeTimestampToKafka) { timestamp = elementTimestamp; } ProducerRecord record; int[] partitions = internalProducer.topicPartitionsMap.get(targetTopic); - if(null == partitions) { + if (null == partitions) { partitions = getPartitionsByTopic(targetTopic, internalProducer.producer); internalProducer.topicPartitionsMap.put(targetTopic, partitions); } @@ -329,10 +327,8 @@ record = new ProducerRecord<>(targetTopic, internalProducer.flinkKafkaPartitione internalProducer.producer.send(record, internalProducer.callback); } - // ----------------- Helper methods implementing methods from SinkFunction and RichFunction (Approach (a)) ---- - // ---- Configuration setters /** @@ -341,7 +337,7 @@ record = new ProducerRecord<>(targetTopic, internalProducer.flinkKafkaPartitione * exceptions will be eventually thrown and cause the streaming program to * fail (and enter recovery). * - * Method is only accessible for approach (a) (see above) + *

Method is only accessible for approach (a) (see above) * * @param logFailuresOnly The flag to indicate logging-only on exceptions. */ @@ -355,7 +351,7 @@ public void setLogFailuresOnly(boolean logFailuresOnly) { * to be acknowledged by the Kafka producer on a checkpoint. * This way, the producer can guarantee that messages in the Kafka buffers are part of the checkpoint. * - * Method is only accessible for approach (a) (see above) + *

Method is only accessible for approach (a) (see above) * * @param flush Flag indicating the flushing mode (true = flush on checkpoint) */ @@ -365,8 +361,7 @@ public void setFlushOnCheckpoint(boolean flush) { } /** - * This method is used for approach (a) (see above) - * + * This method is used for approach (a) (see above). */ @Override public void open(Configuration parameters) throws Exception { @@ -375,7 +370,7 @@ public void open(Configuration parameters) throws Exception { } /** - * This method is used for approach (a) (see above) + * This method is used for approach (a) (see above). */ @Override public IterationRuntimeContext getIterationRuntimeContext() { @@ -384,7 +379,7 @@ public IterationRuntimeContext getIterationRuntimeContext() { } /** - * This method is used for approach (a) (see above) + * This method is used for approach (a) (see above). */ @Override public void setRuntimeContext(RuntimeContext t) { @@ -395,7 +390,7 @@ public void setRuntimeContext(RuntimeContext t) { /** * Invoke method for using the Sink as DataStream.addSink() sink. * - * This method is used for approach (a) (see above) + *

This method is used for approach (a) (see above) * * @param value The input record. */ @@ -404,14 +399,12 @@ public void invoke(T value) throws Exception { invokeInternal(value, Long.MAX_VALUE); } - // ----------------- Helper methods and classes implementing methods from StreamSink (Approach (b)) ---- - /** * Process method for using the sink with timestamp support. * - * This method is used for approach (b) (see above) + *

This method is used for approach (b) (see above) */ @Override public void processElement(StreamRecord element) throws Exception { @@ -467,5 +460,4 @@ public void setWriteTimestampToKafka(boolean writeTimestampToKafka) { } } - } diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSource.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSource.java index 1b2abcc4bd224..99214282c71e3 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSource.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSource.java @@ -18,13 +18,15 @@ package org.apache.flink.streaming.connectors.kafka; -import java.util.Properties; -import org.apache.avro.specific.SpecificRecord; -import org.apache.avro.specific.SpecificRecordBase; import org.apache.flink.streaming.util.serialization.DeserializationSchema; import org.apache.flink.table.sources.StreamTableSource; import org.apache.flink.types.Row; +import org.apache.avro.specific.SpecificRecord; +import org.apache.avro.specific.SpecificRecordBase; + +import java.util.Properties; + /** * Kafka {@link StreamTableSource} for Kafka 0.10. */ diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java index 78ef28e4a5288..f400f6ba6a06a 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java @@ -19,9 +19,9 @@ package org.apache.flink.streaming.connectors.kafka; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.types.Row; -import org.apache.flink.table.sources.StreamTableSource; import org.apache.flink.streaming.util.serialization.DeserializationSchema; +import org.apache.flink.table.sources.StreamTableSource; +import org.apache.flink.types.Row; import java.util.Properties; diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java index 03e9125184492..a6de13abf5562 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java @@ -19,9 +19,9 @@ package org.apache.flink.streaming.connectors.kafka; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.types.Row; -import org.apache.flink.table.sources.StreamTableSource; import org.apache.flink.streaming.util.serialization.DeserializationSchema; +import org.apache.flink.table.sources.StreamTableSource; +import org.apache.flink.types.Row; import java.util.Properties; diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010Fetcher.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010Fetcher.java index 586d841342367..eb4dfee2ca419 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010Fetcher.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010Fetcher.java @@ -36,10 +36,10 @@ /** * A fetcher that fetches data from Kafka brokers via the Kafka 0.10 consumer API. - * + * *

This fetcher re-uses basically all functionality of the 0.9 fetcher. It only additionally * takes the KafkaRecord-attached timestamp and attaches it to the Flink records. - * + * * @param The type of elements produced by the fetcher. */ public class Kafka010Fetcher extends Kafka09Fetcher { @@ -57,8 +57,7 @@ public Kafka010Fetcher( KeyedDeserializationSchema deserializer, Properties kafkaProperties, long pollTimeout, - boolean useMetrics) throws Exception - { + boolean useMetrics) throws Exception { super( sourceContext, assignedPartitionsWithInitialOffsets, @@ -88,7 +87,7 @@ protected void emitRecord( /** * This method needs to be overridden because Kafka broke binary compatibility between 0.9 and 0.10, - * changing binary signatures + * changing binary signatures. */ @Override protected KafkaConsumerCallBridge010 createCallBridge() { diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge010.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge010.java index 0fda9a6fad3b2..b62114036492c 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge010.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge010.java @@ -26,11 +26,11 @@ /** * The ConsumerCallBridge simply calls the {@link KafkaConsumer#assign(java.util.Collection)} method. - * - * This indirection is necessary, because Kafka broke binary compatibility between 0.9 and 0.10, + * + *

This indirection is necessary, because Kafka broke binary compatibility between 0.9 and 0.10, * changing {@code assign(List)} to {@code assign(Collection)}. - * - * Because of that, we need two versions whose compiled code goes against different method signatures. + * + *

Because of that, we need two versions whose compiled code goes against different method signatures. */ public class KafkaConsumerCallBridge010 extends KafkaConsumerCallBridge { diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/resources/log4j.properties b/flink-connectors/flink-connector-kafka-0.10/src/main/resources/log4j.properties index 6bdfb48cd550f..6eef1747ddfe4 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/resources/log4j.properties +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/resources/log4j.properties @@ -26,4 +26,3 @@ 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-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSourceTest.java b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSourceTest.java index ed93725fffbb6..025fefc9ceb33 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSourceTest.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSourceTest.java @@ -18,12 +18,16 @@ package org.apache.flink.streaming.connectors.kafka; -import java.util.Properties; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.streaming.util.serialization.AvroRowDeserializationSchema; import org.apache.flink.streaming.util.serialization.DeserializationSchema; import org.apache.flink.types.Row; +import java.util.Properties; + +/** + * Tests for the {@link Kafka010AvroTableSource}. + */ public class Kafka010AvroTableSourceTest extends KafkaTableSourceTestBase { @Override diff --git a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010FetcherTest.java b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010FetcherTest.java index 2d0551dcac939..aedd4ba2db0dc 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010FetcherTest.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010FetcherTest.java @@ -39,10 +39,8 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.OffsetCommitCallback; import org.apache.kafka.common.TopicPartition; - import org.junit.Test; import org.junit.runner.RunWith; - import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -63,7 +61,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; - import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyLong; import static org.powermock.api.mockito.PowerMockito.doAnswer; diff --git a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java index add623edae3ea..22193b7a807e3 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java @@ -38,13 +38,17 @@ import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper; import org.apache.flink.streaming.util.serialization.TypeInformationSerializationSchema; + import org.junit.Test; import javax.annotation.Nullable; + import java.io.ByteArrayInputStream; import java.io.IOException; - +/** + * IT cases for Kafka 0.10 . + */ public class Kafka010ITCase extends KafkaConsumerTestBase { // ------------------------------------------------------------------------ @@ -83,7 +87,6 @@ public void testFailOnDeploy() throws Exception { runFailOnDeployTest(); } - // --- source to partition mappings and exactly once --- @Test(timeout = 60000) @@ -170,7 +173,7 @@ public void testAutoOffsetRetrievalAndCommitToKafka() throws Exception { } /** - * Kafka 0.10 specific test, ensuring Timestamps are properly written to and read from Kafka + * Kafka 0.10 specific test, ensuring Timestamps are properly written to and read from Kafka. */ @Test(timeout = 60000) public void testTimestamps() throws Exception { @@ -193,9 +196,9 @@ public void testTimestamps() throws Exception { @Override public void run(SourceContext ctx) throws Exception { long i = 0; - while(running) { - ctx.collectWithTimestamp(i, i*2); - if(i++ == 1000L) { + while (running) { + ctx.collectWithTimestamp(i, i * 2); + if (i++ == 1000L) { running = false; } } @@ -213,7 +216,7 @@ public void cancel() { @Override public int partition(Long next, byte[] key, byte[] value, String targetTopic, int[] partitions) { - return (int)(next % 3); + return (int) (next % 3); } }); prod.setParallelism(3); @@ -235,7 +238,7 @@ public int partition(Long next, byte[] key, byte[] value, String targetTopic, in @Nullable @Override public Watermark checkAndGetNextWatermark(Long lastElement, long extractedTimestamp) { - if(lastElement % 10 == 0) { + if (lastElement % 10 == 0) { return new Watermark(lastElement); } return null; @@ -278,7 +281,7 @@ public void invoke(Long value) throws Exception { @Override public void processElement(StreamRecord element) throws Exception { elCount++; - if(element.getValue() * 2 != element.getTimestamp()) { + if (element.getValue() * 2 != element.getTimestamp()) { throw new RuntimeException("Invalid timestamp: " + element); } } @@ -287,13 +290,13 @@ public void processElement(StreamRecord element) throws Exception { public void processWatermark(Watermark mark) throws Exception { wmCount++; - if(lastWM <= mark.getTimestamp()) { + if (lastWM <= mark.getTimestamp()) { lastWM = mark.getTimestamp(); } else { throw new RuntimeException("Received watermark higher than the last one"); } - if( mark.getTimestamp() % 10 != 0 && mark.getTimestamp() != Long.MAX_VALUE ) { + if (mark.getTimestamp() % 10 != 0 && mark.getTimestamp() != Long.MAX_VALUE) { throw new RuntimeException("Invalid watermark: " + mark.getTimestamp()); } } @@ -301,11 +304,11 @@ public void processWatermark(Watermark mark) throws Exception { @Override public void close() throws Exception { super.close(); - if(elCount != 1000L) { + if (elCount != 1000L) { throw new RuntimeException("Wrong final element count " + elCount); } - if(wmCount <= 2) { + if (wmCount <= 2) { throw new RuntimeException("Almost no watermarks have been sent " + wmCount); } } @@ -322,6 +325,7 @@ public LimitedLongDeserializer() { this.ti = TypeInfoParser.parse("Long"); this.ser = ti.createSerializer(new ExecutionConfig()); } + @Override public TypeInformation getProducedType() { return ti; diff --git a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSourceTest.java b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSourceTest.java index 55e8b9c0da188..092f5ea122b34 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSourceTest.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSourceTest.java @@ -18,12 +18,16 @@ package org.apache.flink.streaming.connectors.kafka; -import java.util.Properties; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.streaming.util.serialization.DeserializationSchema; import org.apache.flink.streaming.util.serialization.JsonRowDeserializationSchema; import org.apache.flink.types.Row; +import java.util.Properties; + +/** + * Tests for the {@link Kafka010JsonTableSource}. + */ public class Kafka010JsonTableSourceTest extends KafkaTableSourceTestBase { @Override diff --git a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ProducerITCase.java b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ProducerITCase.java index 42b96828403e2..64a5a3f22a41b 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ProducerITCase.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ProducerITCase.java @@ -18,10 +18,11 @@ package org.apache.flink.streaming.connectors.kafka; - import org.junit.Test; - +/** + * IT cases for the {@link FlinkKafkaProducer010}. + */ @SuppressWarnings("serial") public class Kafka010ProducerITCase extends KafkaProducerTestBase { diff --git a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java index c88c8585d82c7..cb30fbf6691ef 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java @@ -15,9 +15,17 @@ * limitations under the License. */ - package org.apache.flink.streaming.connectors.kafka; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.streaming.api.operators.StreamSink; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.streaming.connectors.kafka.testutils.ZooKeeperStringSerializer; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema; +import org.apache.flink.util.NetUtils; + import kafka.admin.AdminUtils; import kafka.common.KafkaException; import kafka.server.KafkaConfig; @@ -27,14 +35,6 @@ import org.I0Itec.zkclient.ZkClient; import org.apache.commons.io.FileUtils; import org.apache.curator.test.TestingServer; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSink; -import org.apache.flink.streaming.api.operators.StreamSink; -import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; -import org.apache.flink.streaming.connectors.kafka.testutils.ZooKeeperStringSerializer; -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.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; @@ -57,7 +57,7 @@ import static org.junit.Assert.fail; /** - * An implementation of the KafkaServerProvider for Kafka 0.10 + * An implementation of the KafkaServerProvider for Kafka 0.10 . */ public class KafkaTestEnvironmentImpl extends KafkaTestEnvironment { @@ -87,7 +87,7 @@ public Properties getStandardProperties() { @Override public Properties getSecureProperties() { Properties prop = new Properties(); - if(secureMode) { + if (secureMode) { prop.put("security.inter.broker.protocol", "SASL_PLAINTEXT"); prop.put("security.protocol", "SASL_PLAINTEXT"); prop.put("sasl.kerberos.service.name", "kafka"); @@ -95,7 +95,7 @@ public Properties getSecureProperties() { //add special timeout for Travis prop.setProperty("zookeeper.session.timeout.ms", String.valueOf(zkTimeout)); prop.setProperty("zookeeper.connection.timeout.ms", String.valueOf(zkTimeout)); - prop.setProperty("metadata.fetch.timeout.ms","120000"); + prop.setProperty("metadata.fetch.timeout.ms", "120000"); } return prop; } @@ -122,7 +122,6 @@ public StreamSink getProducerSink(String topic, KeyedSerializationSchema< return new StreamSink<>(prod); } - @Override public DataStreamSink produceIntoKafka(DataStream stream, String topic, KeyedSerializationSchema serSchema, Properties props, FlinkKafkaPartitioner partitioner) { FlinkKafkaProducer010 prod = new FlinkKafkaProducer010<>(topic, serSchema, props, partitioner); @@ -176,7 +175,7 @@ public boolean isSecureRunSupported() { @Override public void prepare(int numKafkaServers, Properties additionalServerProperties, boolean secureMode) { //increase the timeout since in Travis ZK connection takes long time for secure connection. - if(secureMode) { + if (secureMode) { //run only one kafka server to avoid multiple ZK connections from many instances - Travis timeout numKafkaServers = 1; zkTimeout = zkTimeout * 15; @@ -203,7 +202,7 @@ public void prepare(int numKafkaServers, Properties additionalServerProperties, brokers = null; try { - zookeeper = new TestingServer(- 1, tmpZkDir); + zookeeper = new TestingServer(-1, tmpZkDir); zookeeperConnectionString = zookeeper.getConnectString(); LOG.info("Starting Zookeeper with zookeeperConnectionString: {}", zookeeperConnectionString); @@ -213,7 +212,7 @@ public void prepare(int numKafkaServers, Properties additionalServerProperties, for (int i = 0; i < numKafkaServers; i++) { brokers.add(getKafkaServer(i, tmpKafkaDirs.get(i))); - if(secureMode) { + if (secureMode) { brokerConnectionString += hostAndPortToUrlString(KafkaTestEnvironment.KAFKA_HOST, brokers.get(i).socketServer().boundPort(SecurityProtocol.SASL_PLAINTEXT)) + ","; } else { brokerConnectionString += hostAndPortToUrlString(KafkaTestEnvironment.KAFKA_HOST, brokers.get(i).socketServer().boundPort(SecurityProtocol.PLAINTEXT)) + ","; @@ -299,7 +298,7 @@ public void createTestTopic(String topic, int numberOfPartitions, int replicatio final long deadline = System.nanoTime() + 30_000_000_000L; do { try { - if(secureMode) { + if (secureMode) { //increase wait time since in Travis ZK timeout occurs frequently int wait = zkTimeout / 100; LOG.info("waiting for {} msecs before the topic {} can be checked", wait, topic); @@ -315,7 +314,7 @@ public void createTestTopic(String topic, int numberOfPartitions, int replicatio // create a new ZK utils connection ZkUtils checkZKConn = getZkUtils(); - if(AdminUtils.topicExists(checkZKConn, topic)) { + if (AdminUtils.topicExists(checkZKConn, topic)) { checkZKConn.close(); return; } @@ -343,7 +342,7 @@ public void deleteTestTopic(String topic) { } /** - * Copied from com.github.sakserv.minicluster.KafkaLocalBrokerIntegrationTest (ASL licensed) + * Copied from com.github.sakserv.minicluster.KafkaLocalBrokerIntegrationTest (ASL licensed). */ protected KafkaServer getKafkaServer(int brokerId, File tmpFolder) throws Exception { Properties kafkaProperties = new Properties(); @@ -359,7 +358,7 @@ protected KafkaServer getKafkaServer(int brokerId, File tmpFolder) throws Except // for CI stability, increase zookeeper session timeout kafkaProperties.put("zookeeper.session.timeout.ms", zkTimeout); kafkaProperties.put("zookeeper.connection.timeout.ms", zkTimeout); - if(additionalServerProperties != null) { + if (additionalServerProperties != null) { kafkaProperties.putAll(additionalServerProperties); } @@ -370,7 +369,7 @@ protected KafkaServer getKafkaServer(int brokerId, File tmpFolder) throws Except kafkaProperties.put("port", Integer.toString(kafkaPort)); //to support secure kafka cluster - if(secureMode) { + if (secureMode) { LOG.info("Adding Kafka secure configurations"); kafkaProperties.put("listeners", "SASL_PLAINTEXT://" + KAFKA_HOST + ":" + kafkaPort); kafkaProperties.put("advertised.listeners", "SASL_PLAINTEXT://" + KAFKA_HOST + ":" + kafkaPort); diff --git a/flink-connectors/flink-connector-kafka-0.8/pom.xml b/flink-connectors/flink-connector-kafka-0.8/pom.xml index 5e2ed2d4338d2..b6b03360a52b7 100644 --- a/flink-connectors/flink-connector-kafka-0.8/pom.xml +++ b/flink-connectors/flink-connector-kafka-0.8/pom.xml @@ -175,7 +175,6 @@ under the License. - @@ -215,5 +214,5 @@ under the License. - + diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java index 858a7903b6748..6c7b94d8efcfc 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java @@ -17,13 +17,6 @@ package org.apache.flink.streaming.connectors.kafka; -import kafka.cluster.Broker; -import kafka.common.ErrorMapping; -import kafka.javaapi.PartitionMetadata; -import kafka.javaapi.TopicMetadata; -import kafka.javaapi.TopicMetadataRequest; -import kafka.javaapi.consumer.SimpleConsumer; - import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; @@ -39,6 +32,12 @@ import org.apache.flink.util.PropertiesUtil; import org.apache.flink.util.SerializedValue; +import kafka.cluster.Broker; +import kafka.common.ErrorMapping; +import kafka.javaapi.PartitionMetadata; +import kafka.javaapi.TopicMetadata; +import kafka.javaapi.TopicMetadataRequest; +import kafka.javaapi.consumer.SimpleConsumer; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.common.Node; @@ -46,25 +45,25 @@ import java.net.URL; import java.net.UnknownHostException; import java.nio.channels.ClosedChannelException; -import java.util.Collections; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Properties; import java.util.Random; -import static org.apache.flink.util.PropertiesUtil.getInt; import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.PropertiesUtil.getInt; /** * The Flink Kafka Consumer is a streaming data source that pulls a parallel data stream from * 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. - * + * 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". + * during a failure, and that the computation processes elements "exactly once". * (Note: These guarantees naturally assume that Kafka itself does not loose any data.)

- * + * *

Flink's Kafka Consumer is designed to be compatible with Kafka's High-Level Consumer API (0.8.x). * Most of Kafka's configuration variables can be used with this consumer as well: *

- * - * - * + * *

Offset handling

- * + * *

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.

@@ -93,7 +90,7 @@ * *

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.

@@ -102,7 +99,7 @@ public class FlinkKafkaConsumer08 extends FlinkKafkaConsumerBase { private static final long serialVersionUID = -6272159445203409112L; - /** Configuration key for the number of retries for getting the partition info */ + /** Configuration key for the number of retries for getting the partition info. */ public static final String GET_PARTITIONS_RETRIES_KEY = "flink.get-partitions.retry"; /** Default number of retries for getting the partition info. One retry means going through the full list of brokers */ @@ -110,13 +107,13 @@ public class FlinkKafkaConsumer08 extends FlinkKafkaConsumerBase { // ------------------------------------------------------------------------ - /** The properties to parametrize the Kafka consumer and ZooKeeper client */ + /** The properties to parametrize the Kafka consumer and ZooKeeper client. */ private final Properties kafkaProperties; // ------------------------------------------------------------------------ /** - * Creates a new Kafka streaming source consumer for Kafka 0.8.x + * Creates a new Kafka streaming source consumer for Kafka 0.8.x. * * @param topic * The name of the topic that should be consumed. @@ -132,7 +129,7 @@ public FlinkKafkaConsumer08(String topic, DeserializationSchema valueDeserial /** * Creates a new Kafka streaming source consumer for Kafka 0.8.x * - * This constructor allows passing a {@see KeyedDeserializationSchema} for reading key/value + *

This constructor allows passing a {@see KeyedDeserializationSchema} for reading key/value * pairs, offsets, and topic names from Kafka. * * @param topic @@ -149,7 +146,7 @@ public FlinkKafkaConsumer08(String topic, KeyedDeserializationSchema deserial /** * Creates a new Kafka streaming source consumer for Kafka 0.8.x * - * This constructor allows passing multiple topics to the consumer. + *

This constructor allows passing multiple topics to the consumer. * * @param topics * The Kafka topics to read from. @@ -165,8 +162,8 @@ public FlinkKafkaConsumer08(List topics, DeserializationSchema deseri /** * Creates a new Kafka streaming source consumer for Kafka 0.8.x * - * This constructor allows passing multiple topics and a key/value deserialization schema. - * + *

This constructor allows passing multiple topics and a key/value deserialization schema. + * * @param topics * The Kafka topics to read from. * @param deserializer @@ -245,14 +242,14 @@ protected boolean getIsAutoCommitEnabled() { /** * Send request to Kafka to get partitions for topic. - * + * * @param topics The name of the topics. - * @param properties The properties for the Kafka Consumer that is used to query the partitions for the topic. + * @param properties The properties for the Kafka Consumer that is used to query the partitions for the topic. */ public static List getPartitionsForTopic(List topics, Properties properties) { String seedBrokersConfString = properties.getProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG); final int numRetries = getInt(properties, GET_PARTITIONS_RETRIES_KEY, DEFAULT_GET_PARTITIONS_RETRIES); - + checkNotNull(seedBrokersConfString, "Configuration property %s not set", ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG); String[] seedBrokers = seedBrokersConfString.split(","); List partitions = new ArrayList<>(); @@ -328,7 +325,7 @@ public static List getPartitionsForTopic(List } /** - * Turn a broker instance into a node instance + * Turn a broker instance into a node instance. * @param broker broker instance * @return Node representing the given broker */ @@ -337,7 +334,7 @@ private static Node brokerToNode(Broker broker) { } /** - * Validate the ZK configuration, checking for required parameters + * Validate the ZK configuration, checking for required parameters. * @param props Properties to check */ protected static void validateZooKeeperConfig(Properties props) { @@ -348,7 +345,7 @@ protected static void validateZooKeeperConfig(Properties props) { throw new IllegalArgumentException("Required property '" + ConsumerConfig.GROUP_ID_CONFIG + "' has not been set in the properties"); } - + try { //noinspection ResultOfMethodCallIgnored Integer.parseInt(props.getProperty("zookeeper.session.timeout.ms", "0")); @@ -356,7 +353,7 @@ protected static void validateZooKeeperConfig(Properties props) { catch (NumberFormatException e) { throw new IllegalArgumentException("Property 'zookeeper.session.timeout.ms' is not a valid integer"); } - + try { //noinspection ResultOfMethodCallIgnored Integer.parseInt(props.getProperty("zookeeper.connection.timeout.ms", "0")); @@ -369,7 +366,7 @@ protected static void validateZooKeeperConfig(Properties props) { /** * Validate that at least one seed broker is valid in case of a * ClosedChannelException. - * + * * @param seedBrokers * array containing the seed brokers e.g. ["host1:port1", * "host2:port2"] diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer081.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer081.java index 4e4050f1194e9..4102bf8107fdf 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer081.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer081.java @@ -14,6 +14,7 @@ * 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; diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer082.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer082.java index aeefcc8df877a..7ba510308fc37 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer082.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer082.java @@ -14,6 +14,7 @@ * 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; diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java index 98dac3eebab28..434286e5e89d0 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java @@ -22,8 +22,8 @@ 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; +import java.util.Properties; /** * THIS CLASS IS DEPRECATED. Use FlinkKafkaProducer08 instead. @@ -38,7 +38,7 @@ public class FlinkKafkaProducer extends FlinkKafkaProducer08 { */ @Deprecated public FlinkKafkaProducer(String brokerList, String topicId, SerializationSchema serializationSchema) { - super(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), (FlinkKafkaPartitioner)null); + super(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), (FlinkKafkaPartitioner) null); } /** @@ -46,7 +46,7 @@ public FlinkKafkaProducer(String brokerList, String topicId, SerializationSchema */ @Deprecated public FlinkKafkaProducer(String topicId, SerializationSchema serializationSchema, Properties producerConfig) { - super(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, (FlinkKafkaPartitioner)null); + super(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, (FlinkKafkaPartitioner) null); } /** @@ -63,7 +63,7 @@ public FlinkKafkaProducer(String topicId, SerializationSchema serializationS */ @Deprecated public FlinkKafkaProducer(String brokerList, String topicId, KeyedSerializationSchema serializationSchema) { - super(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), (FlinkKafkaPartitioner)null); + super(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), (FlinkKafkaPartitioner) null); } /** @@ -71,7 +71,7 @@ public FlinkKafkaProducer(String brokerList, String topicId, KeyedSerializationS */ @Deprecated public FlinkKafkaProducer(String topicId, KeyedSerializationSchema serializationSchema, Properties producerConfig) { - super(topicId, serializationSchema, producerConfig, (FlinkKafkaPartitioner)null); + super(topicId, serializationSchema, producerConfig, (FlinkKafkaPartitioner) null); } /** diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer08.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer08.java index 08dcb2fd27a92..a14768b351796 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer08.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer08.java @@ -27,11 +27,10 @@ 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. + *

Please note that this producer does not have any reliability guarantees. * * @param Type of the messages to write into Kafka. */ diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSource.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSource.java index 1a68c05cc9c01..a1bea78ff3adf 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSource.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSource.java @@ -18,13 +18,15 @@ package org.apache.flink.streaming.connectors.kafka; -import java.util.Properties; -import org.apache.avro.specific.SpecificRecord; -import org.apache.avro.specific.SpecificRecordBase; import org.apache.flink.streaming.util.serialization.DeserializationSchema; import org.apache.flink.table.sources.StreamTableSource; import org.apache.flink.types.Row; +import org.apache.avro.specific.SpecificRecord; +import org.apache.avro.specific.SpecificRecordBase; + +import java.util.Properties; + /** * Kafka {@link StreamTableSource} for Kafka 0.8. */ diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSink.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSink.java index 80bd18052c92e..79406d86dbf00 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSink.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSink.java @@ -15,13 +15,14 @@ * 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.FlinkKafkaDelegatePartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; -import org.apache.flink.types.Row; import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; import org.apache.flink.streaming.util.serialization.SerializationSchema; +import org.apache.flink.types.Row; import java.util.Properties; @@ -29,9 +30,9 @@ * Kafka 0.8 {@link KafkaTableSink} that serializes data in JSON format. */ public class Kafka08JsonTableSink extends KafkaJsonTableSink { - + /** - * Creates {@link KafkaTableSink} for Kafka 0.8 + * Creates {@link KafkaTableSink} for Kafka 0.8. * * @param topic topic in Kafka to which table is written * @param properties properties to connect to Kafka @@ -42,7 +43,7 @@ public Kafka08JsonTableSink(String topic, Properties properties, FlinkKafkaParti } /** - * Creates {@link KafkaTableSink} for Kafka 0.8 + * Creates {@link KafkaTableSink} for Kafka 0.8. * * @param topic topic in Kafka to which table is written * @param properties properties to connect to Kafka diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.java index 1555a3bc72828..05a2c71c558a5 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.java @@ -19,9 +19,9 @@ package org.apache.flink.streaming.connectors.kafka; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.types.Row; -import org.apache.flink.table.sources.StreamTableSource; import org.apache.flink.streaming.util.serialization.DeserializationSchema; +import org.apache.flink.table.sources.StreamTableSource; +import org.apache.flink.types.Row; import java.util.Properties; diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java index e1e481cddf04d..95363064f3640 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java @@ -19,9 +19,9 @@ package org.apache.flink.streaming.connectors.kafka; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.types.Row; -import org.apache.flink.table.sources.StreamTableSource; import org.apache.flink.streaming.util.serialization.DeserializationSchema; +import org.apache.flink.table.sources.StreamTableSource; +import org.apache.flink.types.Row; import java.util.Properties; diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueue.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueue.java index e31dcacc4fe6c..da61dd0cd11d8 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueue.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueue.java @@ -38,27 +38,27 @@ * are available and mark themselves as shut down. *

  • The queue allows to poll batches of elements in one polling call.
  • * - * - * The queue has no capacity restriction and is safe for multiple producers and consumers. - * + * + *

    The queue has no capacity restriction and is safe for multiple producers and consumers. + * *

    Note: Null elements are prohibited. - * + * * @param The type of elements in the queue. */ public class ClosableBlockingQueue { - /** The lock used to make queue accesses and open checks atomic */ + /** The lock used to make queue accesses and open checks atomic. */ private final ReentrantLock lock; - - /** The condition on which blocking get-calls wait if the queue is empty */ + + /** The condition on which blocking get-calls wait if the queue is empty. */ private final Condition nonEmpty; - - /** The deque of elements */ + + /** The deque of elements. */ private final ArrayDeque elements; - - /** Flag marking the status of the queue */ + + /** Flag marking the status of the queue. */ private volatile boolean open; - + // ------------------------------------------------------------------------ /** @@ -72,22 +72,21 @@ public ClosableBlockingQueue() { * Creates a new empty queue, reserving space for at least the specified number * of elements. The queu can still grow, of more elements are added than the * reserved space. - * + * * @param initialSize The number of elements to reserve space for. */ public ClosableBlockingQueue(int initialSize) { this.lock = new ReentrantLock(true); this.nonEmpty = this.lock.newCondition(); - + this.elements = new ArrayDeque<>(initialSize); this.open = true; - - + } /** * Creates a new queue that contains the given elements. - * + * * @param initialElements The elements to initially add to the queue. */ public ClosableBlockingQueue(Collection initialElements) { @@ -127,7 +126,7 @@ public boolean isEmpty() { public boolean isOpen() { return open; } - + /** * Tries to close the queue. Closing the queue only succeeds when no elements are * in the queue when this method is called. Checking whether the queue is empty, and @@ -155,25 +154,25 @@ public boolean close() { lock.unlock(); } } - + // ------------------------------------------------------------------------ // Adding / Removing elements // ------------------------------------------------------------------------ - + /** * Tries to add an element to the queue, if the queue is still open. Checking whether the queue * is open and adding the element is one atomic operation. - * + * *

    Unlike the {@link #add(Object)} method, this method never throws an exception, * but only indicates via the return code if the element was added or the * queue was closed. - * + * * @param element The element to add. * @return True, if the element was added, false if the queue was closes. */ public boolean addIfOpen(E element) { requireNonNull(element); - + lock.lock(); try { if (open) { @@ -191,7 +190,7 @@ public boolean addIfOpen(E element) { /** * Adds the element to the queue, or fails with an exception, if the queue is closed. * Checking whether the queue is open and adding the element is one atomic operation. - * + * * @param element The element to add. * @throws IllegalStateException Thrown, if the queue is closed. */ @@ -215,13 +214,13 @@ public void add(E element) throws IllegalStateException { /** * Returns the queue's next element without removing it, if the queue is non-empty. - * Otherwise, returns null. + * Otherwise, returns null. * *

    The method throws an {@code IllegalStateException} if the queue is closed. * Checking whether the queue is open and getting the next element is one atomic operation. - * + * *

    This method never blocks. - * + * * @return The queue's next element, or null, if the queue is empty. * @throws IllegalStateException Thrown, if the queue is closed. */ @@ -244,7 +243,7 @@ public E peek() { /** * Returns the queue's next element and removes it, the queue is non-empty. - * Otherwise, this method returns null. + * Otherwise, this method returns null. * *

    The method throws an {@code IllegalStateException} if the queue is closed. * Checking whether the queue is open and removing the next element is one atomic operation. @@ -273,7 +272,7 @@ public E poll() { /** * Returns all of the queue's current elements in a list, if the queue is non-empty. - * Otherwise, this method returns null. + * Otherwise, this method returns null. * *

    The method throws an {@code IllegalStateException} if the queue is closed. * Checking whether the queue is open and removing the elements is one atomic operation. @@ -305,12 +304,12 @@ public List pollBatch() { /** * Returns the next element in the queue. If the queue is empty, this method * waits until at least one element is added. - * + * *

    The method throws an {@code IllegalStateException} if the queue is closed. * Checking whether the queue is open and removing the next element is one atomic operation. - * + * * @return The next element in the queue, never null. - * + * * @throws IllegalStateException Thrown, if the queue is closed. * @throws InterruptedException Throw, if the thread is interrupted while waiting for an * element to be added. @@ -321,7 +320,7 @@ public E getElementBlocking() throws InterruptedException { while (open && elements.isEmpty()) { nonEmpty.await(); } - + if (open) { return elements.removeFirst(); } else { @@ -336,13 +335,13 @@ public E getElementBlocking() throws InterruptedException { * Returns the next element in the queue. If the queue is empty, this method * waits at most a certain time until an element becomes available. If no element * is available after that time, the method returns null. - * + * *

    The method throws an {@code IllegalStateException} if the queue is closed. * Checking whether the queue is open and removing the next element is one atomic operation. - * + * * @param timeoutMillis The number of milliseconds to block, at most. * @return The next element in the queue, or null, if the timeout expires before an element is available. - * + * * @throws IllegalStateException Thrown, if the queue is closed. * @throws InterruptedException Throw, if the thread is interrupted while waiting for an * element to be added. @@ -354,16 +353,16 @@ public E getElementBlocking(long timeoutMillis) throws InterruptedException { } else if (timeoutMillis < 0L) { throw new IllegalArgumentException("invalid timeout"); } - + final long deadline = System.nanoTime() + timeoutMillis * 1_000_000L; - + lock.lock(); try { - while (open && elements.isEmpty() && timeoutMillis > 0) { + while (open && elements.isEmpty() && timeoutMillis > 0) { nonEmpty.await(timeoutMillis, TimeUnit.MILLISECONDS); timeoutMillis = (deadline - System.nanoTime()) / 1_000_000L; } - + if (!open) { throw new IllegalStateException("queue is closed"); } @@ -383,12 +382,12 @@ else if (elements.isEmpty()) { * at least one element is added. * *

    This method always returns a list with at least one element. - * + * *

    The method throws an {@code IllegalStateException} if the queue is closed. * Checking whether the queue is open and removing the next element is one atomic operation. - * + * * @return A list with all elements in the queue, always at least one element. - * + * * @throws IllegalStateException Thrown, if the queue is closed. * @throws InterruptedException Throw, if the thread is interrupted while waiting for an * element to be added. @@ -415,13 +414,13 @@ public List getBatchBlocking() throws InterruptedException { * Gets all the elements found in the list, or blocks until at least one element * was added. This method is similar as {@link #getBatchBlocking()}, but takes * a number of milliseconds that the method will maximally wait before returning. - * + * *

    This method never returns null, but an empty list, if the queue is empty when * the method is called and the request times out before an element was added. - * + * *

    The method throws an {@code IllegalStateException} if the queue is closed. * Checking whether the queue is open and removing the next element is one atomic operation. - * + * * @param timeoutMillis The number of milliseconds to wait, at most. * @return A list with all elements in the queue, possible an empty list. * @@ -461,11 +460,11 @@ else if (elements.isEmpty()) { lock.unlock(); } } - + // ------------------------------------------------------------------------ // Standard Utilities // ------------------------------------------------------------------------ - + @Override public int hashCode() { int hashCode = 17; @@ -482,7 +481,7 @@ public boolean equals(Object obj) { } else if (obj != null && obj.getClass() == ClosableBlockingQueue.class) { @SuppressWarnings("unchecked") ClosableBlockingQueue that = (ClosableBlockingQueue) obj; - + if (this.elements.size() == that.elements.size()) { Iterator thisElements = this.elements.iterator(); for (E thatNext : that.elements) { diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Kafka08Fetcher.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Kafka08Fetcher.java index de201e5d4a494..659bbd7d6b4cb 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Kafka08Fetcher.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Kafka08Fetcher.java @@ -18,22 +18,21 @@ package org.apache.flink.streaming.connectors.kafka.internals; -import kafka.api.OffsetRequest; -import kafka.common.TopicAndPartition; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.metrics.MetricGroup; -import org.apache.flink.streaming.connectors.kafka.config.StartupMode; -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.common.Node; - import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.connectors.kafka.config.StartupMode; import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.SerializedValue; +import kafka.api.OffsetRequest; +import kafka.common.TopicAndPartition; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.common.Node; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -56,39 +55,38 @@ * @param The type of elements produced by the fetcher. */ public class Kafka08Fetcher extends AbstractFetcher { - - static final KafkaTopicPartitionState MARKER = + + static final KafkaTopicPartitionState MARKER = new KafkaTopicPartitionState<>(new KafkaTopicPartition("n/a", -1), new TopicAndPartition("n/a", -1)); private static final Logger LOG = LoggerFactory.getLogger(Kafka08Fetcher.class); // ------------------------------------------------------------------------ - /** The schema to convert between Kafka's byte messages, and Flink's objects */ + /** The schema to convert between Kafka's byte messages, and Flink's objects. */ private final KeyedDeserializationSchema deserializer; - /** The properties that configure the Kafka connection */ + /** The properties that configure the Kafka connection. */ private final Properties kafkaConfig; - /** The subtask's runtime context */ + /** The subtask's runtime context. */ private final RuntimeContext runtimeContext; - /** The queue of partitions that are currently not assigned to a broker connection */ + /** The queue of partitions that are currently not assigned to a broker connection. */ private final ClosableBlockingQueue> unassignedPartitionsQueue; - /** The behavior to use in case that an offset is not valid (any more) for a partition */ + /** The behavior to use in case that an offset is not valid (any more) for a partition. */ private final long invalidOffsetBehavior; - /** The interval in which to automatically commit (-1 if deactivated) */ + /** The interval in which to automatically commit (-1 if deactivated). */ private final long autoCommitInterval; - /** The handler that reads/writes offsets from/to ZooKeeper */ + /** The handler that reads/writes offsets from/to ZooKeeper. */ private volatile ZookeeperOffsetHandler zookeeperOffsetHandler; - /** Flag to track the main work loop as alive */ + /** Flag to track the main work loop as alive. */ private volatile boolean running = true; - public Kafka08Fetcher( SourceContext sourceContext, Map assignedPartitionsWithInitialOffsets, @@ -98,8 +96,7 @@ public Kafka08Fetcher( KeyedDeserializationSchema deserializer, Properties kafkaProperties, long autoCommitInterval, - boolean useMetrics) throws Exception - { + boolean useMetrics) throws Exception { super( sourceContext, assignedPartitionsWithInitialOffsets, @@ -175,7 +172,7 @@ public void runFetchLoop() throws Exception { if (autoCommitInterval > 0) { LOG.info("Starting periodic offset committer, with commit interval of {}ms", autoCommitInterval); - periodicCommitter = new PeriodicOffsetCommitter(zookeeperOffsetHandler, + periodicCommitter = new PeriodicOffsetCommitter(zookeeperOffsetHandler, subscribedPartitionStates(), errorHandler, autoCommitInterval); periodicCommitter.setName("Periodic Kafka partition offset committer"); periodicCommitter.setDaemon(true); @@ -196,19 +193,18 @@ public void runFetchLoop() throws Exception { // wait for max 5 seconds trying to get partitions to assign // if threads shut down, this poll returns earlier, because the threads inject the // special marker into the queue - List> partitionsToAssign = + List> partitionsToAssign = unassignedPartitionsQueue.getBatchBlocking(5000); partitionsToAssign.remove(MARKER); if (!partitionsToAssign.isEmpty()) { LOG.info("Assigning {} partitions to broker threads", partitionsToAssign.size()); - Map>> partitionsWithLeaders = + Map>> partitionsWithLeaders = findLeaderForPartitions(partitionsToAssign, kafkaConfig); // assign the partitions to the leaders (maybe start the threads) - for (Map.Entry>> partitionsWithLeader : - partitionsWithLeaders.entrySet()) - { + for (Map.Entry>> partitionsWithLeader : + partitionsWithLeaders.entrySet()) { final Node leader = partitionsWithLeader.getKey(); final List> partitions = partitionsWithLeader.getValue(); SimpleConsumerThread brokerThread = brokerToThread.get(leader); @@ -224,9 +220,9 @@ public void runFetchLoop() throws Exception { } else { // put elements into queue of thread - ClosableBlockingQueue> newPartitionsQueue = + ClosableBlockingQueue> newPartitionsQueue = brokerThread.getNewPartitionsQueue(); - + for (KafkaTopicPartitionState fp : partitions) { if (!newPartitionsQueue.addIfOpen(fp)) { // we were unable to add the partition to the broker's queue @@ -389,8 +385,7 @@ public void commitInternalOffsetsToKafka(Map offsets) private SimpleConsumerThread createAndStartSimpleConsumerThread( List> seedPartitions, Node leader, - ExceptionProxy errorHandler) throws IOException, ClassNotFoundException - { + ExceptionProxy errorHandler) throws IOException, ClassNotFoundException { // each thread needs its own copy of the deserializer, because the deserializer is // not necessarily thread safe final KeyedDeserializationSchema clonedDeserializer = @@ -398,7 +393,7 @@ private SimpleConsumerThread createAndStartSimpleConsumerThread( // seed thread with list of fetch partitions (otherwise it would shut down immediately again SimpleConsumerThread brokerThread = new SimpleConsumerThread<>( - this, errorHandler, kafkaConfig, leader, seedPartitions, unassignedPartitionsQueue, + this, errorHandler, kafkaConfig, leader, seedPartitions, unassignedPartitionsQueue, clonedDeserializer, invalidOffsetBehavior); brokerThread.setName(String.format("SimpleConsumer - %s - broker-%s (%s:%d)", @@ -411,7 +406,7 @@ private SimpleConsumerThread createAndStartSimpleConsumerThread( } /** - * Returns a list of unique topics from for the given partitions + * Returns a list of unique topics from for the given partitions. * * @param partitions A the partitions * @return A list of unique topics @@ -425,26 +420,25 @@ private static List getTopics(ListFrom a high level, the method does the following: * - Get a list of FetchPartitions (usually only a few partitions) * - Get the list of topics from the FetchPartitions list and request the partitions for the topics. (Kafka doesn't support getting leaders for a set of partitions) - * - Build a Map> where only the requested partitions are contained. + * - Build a Map<Leader, List<FetchPartition>> where only the requested partitions are contained. * * @param partitionsToAssign fetch partitions list * @return leader to partitions map */ private static Map>> findLeaderForPartitions( List> partitionsToAssign, - Properties kafkaProperties) throws Exception - { + Properties kafkaProperties) throws Exception { if (partitionsToAssign.isEmpty()) { throw new IllegalArgumentException("Leader request for empty partitions list"); } LOG.info("Refreshing leader information for partitions {}", partitionsToAssign); - + // this request is based on the topic names PartitionInfoFetcher infoFetcher = new PartitionInfoFetcher(getTopics(partitionsToAssign), kafkaProperties); infoFetcher.start(); @@ -465,7 +459,7 @@ private static Map>> find // final mapping from leader -> list(fetchPartition) Map>> leaderToPartitions = new HashMap<>(); - for(KafkaTopicPartitionLeader partitionLeader: topicPartitionWithLeaderList) { + for (KafkaTopicPartitionLeader partitionLeader: topicPartitionWithLeaderList) { if (unassignedPartitions.size() == 0) { // we are done: all partitions are assigned break; diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KillerWatchDog.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KillerWatchDog.java index 574d9f7650d58..b5998f43edfdb 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KillerWatchDog.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KillerWatchDog.java @@ -21,7 +21,7 @@ /** * A watch dog thread that forcibly kills another thread, if that thread does not * finish in time. - * + * *

    This uses the discouraged {@link Thread#stop()} method. While this is not * advisable, this watch dog is only for extreme cases of thread that simply * to not terminate otherwise. diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PartitionInfoFetcher.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PartitionInfoFetcher.java index d8d927d9ec16b..ecf13783466a1 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PartitionInfoFetcher.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PartitionInfoFetcher.java @@ -31,7 +31,6 @@ class PartitionInfoFetcher extends Thread { private volatile List result; private volatile Throwable error; - PartitionInfoFetcher(List topics, Properties properties) { this.topics = topics; this.properties = properties; @@ -63,4 +62,4 @@ public List getPartitions() throws Exception { } throw new Exception("Partition fetching failed"); } -} \ No newline at end of file +} diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PeriodicOffsetCommitter.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PeriodicOffsetCommitter.java index 27d90f2e98561..13a926f4c49a3 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PeriodicOffsetCommitter.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PeriodicOffsetCommitter.java @@ -28,30 +28,29 @@ */ public class PeriodicOffsetCommitter extends Thread { - /** The ZooKeeper handler */ + /** The ZooKeeper handler. */ private final ZookeeperOffsetHandler offsetHandler; - + private final KafkaTopicPartitionState[] partitionStates; - - /** The proxy to forward exceptions to the main thread */ + + /** The proxy to forward exceptions to the main thread. */ private final ExceptionProxy errorHandler; - - /** Interval in which to commit, in milliseconds */ + + /** Interval in which to commit, in milliseconds. */ private final long commitInterval; - - /** Flag to mark the periodic committer as running */ + + /** Flag to mark the periodic committer as running. */ private volatile boolean running = true; PeriodicOffsetCommitter(ZookeeperOffsetHandler offsetHandler, KafkaTopicPartitionState[] partitionStates, ExceptionProxy errorHandler, - long commitInterval) - { + long commitInterval) { this.offsetHandler = checkNotNull(offsetHandler); this.partitionStates = checkNotNull(partitionStates); this.errorHandler = checkNotNull(errorHandler); this.commitInterval = commitInterval; - + checkArgument(commitInterval > 0); } @@ -66,7 +65,7 @@ public void run() { for (KafkaTopicPartitionState partitionState : partitionStates) { offsetsToCommit.put(partitionState.getKafkaTopicPartition(), partitionState.getOffset()); } - + offsetHandler.prepareAndCommitOffsets(offsetsToCommit); } } diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SimpleConsumerThread.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SimpleConsumerThread.java index c78c085fb3384..abc61fa53ad66 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SimpleConsumerThread.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SimpleConsumerThread.java @@ -18,6 +18,9 @@ package org.apache.flink.streaming.connectors.kafka.internals; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +import org.apache.flink.util.ExceptionUtils; + import kafka.api.FetchRequestBuilder; import kafka.api.OffsetRequest; import kafka.api.PartitionOffsetRequestInfo; @@ -28,12 +31,7 @@ import kafka.javaapi.consumer.SimpleConsumer; import kafka.javaapi.message.ByteBufferMessageSet; import kafka.message.MessageAndOffset; - -import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; -import org.apache.flink.util.ExceptionUtils; - import org.apache.kafka.common.Node; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -53,8 +51,8 @@ /** * This class implements a thread with a connection to a single Kafka broker. The thread * pulls records for a set of topic partitions for which the connected broker is currently - * the leader. The thread deserializes these records and emits them. - * + * the leader. The thread deserializes these records and emits them. + * * @param The type of elements that this consumer thread creates from Kafka's byte messages * and emits into the Flink DataStream. */ @@ -63,28 +61,27 @@ class SimpleConsumerThread extends Thread { private static final Logger LOG = LoggerFactory.getLogger(SimpleConsumerThread.class); private static final KafkaTopicPartitionState MARKER = Kafka08Fetcher.MARKER; - + // ------------------------------------------------------------------------ private final Kafka08Fetcher owner; - + private final KeyedDeserializationSchema deserializer; private final List> partitions; private final Node broker; - /** Queue containing new fetch partitions for the consumer thread */ + /** Queue containing new fetch partitions for the consumer thread. */ private final ClosableBlockingQueue> newPartitionsQueue; - + private final ClosableBlockingQueue> unassignedPartitions; - + private final ExceptionProxy errorHandler; - + private final long invalidOffsetBehavior; - + private volatile boolean running = true; - // ----------------- Simple Consumer ---------------------- private volatile SimpleConsumer consumer; @@ -96,7 +93,6 @@ class SimpleConsumerThread extends Thread { private final int bufferSize; private final int reconnectLimit; - // exceptions are thrown locally public SimpleConsumerThread( Kafka08Fetcher owner, @@ -106,8 +102,7 @@ public SimpleConsumerThread( List> seedPartitions, ClosableBlockingQueue> unassignedPartitions, KeyedDeserializationSchema deserializer, - long invalidOffsetBehavior) - { + long invalidOffsetBehavior) { this.owner = owner; this.errorHandler = errorHandler; this.broker = broker; @@ -118,7 +113,7 @@ public SimpleConsumerThread( this.unassignedPartitions = requireNonNull(unassignedPartitions); this.newPartitionsQueue = new ClosableBlockingQueue<>(); this.invalidOffsetBehavior = invalidOffsetBehavior; - + // these are the actual configuration values of Kafka + their original default values. this.soTimeout = getInt(config, "socket.timeout.ms", 30000); this.minBytes = getInt(config, "fetch.min.bytes", 1); @@ -131,11 +126,11 @@ public SimpleConsumerThread( public ClosableBlockingQueue> getNewPartitionsQueue() { return newPartitionsQueue; } - + // ------------------------------------------------------------------------ // main work loop // ------------------------------------------------------------------------ - + @Override public void run() { LOG.info("Starting to fetch from {}", this.partitions); @@ -146,7 +141,7 @@ public void run() { try { // create the Kafka consumer that we actually use for fetching consumer = new SimpleConsumer(broker.host(), broker.port(), soTimeout, bufferSize, clientId); - + // replace earliest of latest starting offsets with actual offset values fetched from Kafka requestAndSetEarliestOrLatestOffsetsFromKafka(consumer, partitions); @@ -169,16 +164,16 @@ public void run() { // if the new partitions are to start from earliest or latest offsets, // we need to replace them with actual values from Kafka requestAndSetEarliestOrLatestOffsetsFromKafka(consumer, newPartitions); - + // add the new partitions (and check they are not already in there) for (KafkaTopicPartitionState newPartition: newPartitions) { if (partitions.contains(newPartition)) { - throw new IllegalStateException("Adding partition " + newPartition + + throw new IllegalStateException("Adding partition " + newPartition + " to subscribed partitions even though it is already subscribed"); } partitions.add(newPartition); } - + LOG.info("Adding {} new partitions to consumer thread {}", newPartitions.size(), getName()); LOG.debug("Partitions list: {}", newPartitions); } @@ -187,8 +182,8 @@ public void run() { if (newPartitionsQueue.close()) { // close succeeded. Closing thread running = false; - - LOG.info("Consumer thread {} does not have any partitions assigned anymore. Stopping thread.", + + LOG.info("Consumer thread {} does not have any partitions assigned anymore. Stopping thread.", getName()); // add the wake-up marker into the queue to make the main thread @@ -199,7 +194,7 @@ public void run() { } else { // close failed: fetcher main thread concurrently added new partitions into the queue. // go to top of loop again and get the new partitions - continue; + continue; } } @@ -217,7 +212,7 @@ public void run() { partition.getOffset() + 1, // request the next record fetchSize); } - + kafka.api.FetchRequest fetchRequest = frb.build(); LOG.debug("Issuing fetch request {}", fetchRequest); @@ -230,7 +225,7 @@ public void run() { if (cce instanceof ClosedChannelException) { LOG.warn("Fetch failed because of ClosedChannelException."); LOG.debug("Full exception", cce); - + // we don't know if the broker is overloaded or unavailable. // retry a few times, then return ALL partitions for new leader lookup if (++reconnects >= reconnectLimit) { @@ -261,15 +256,15 @@ public void run() { if (fetchResponse == null) { throw new IOException("Fetch from Kafka failed (request returned null)"); } - + if (fetchResponse.hasError()) { String exception = ""; List> partitionsToGetOffsetsFor = new ArrayList<>(); - + // iterate over partitions to get individual error codes Iterator> partitionsIterator = partitions.iterator(); boolean partitionsRemoved = false; - + while (partitionsIterator.hasNext()) { final KafkaTopicPartitionState fp = partitionsIterator.next(); short code = fetchResponse.errorCode(fp.getTopic(), fp.getPartition()); @@ -282,8 +277,7 @@ public void run() { else if (code == ErrorMapping.NotLeaderForPartitionCode() || code == ErrorMapping.LeaderNotAvailableCode() || code == ErrorMapping.BrokerNotAvailableCode() || - code == ErrorMapping.UnknownCode()) - { + code == ErrorMapping.UnknownCode()) { // the broker we are connected to is not the leader for the partition. LOG.warn("{} is not the leader of {}. Reassigning leader for partition", broker, fp); LOG.debug("Error code = {}", code); @@ -294,7 +288,7 @@ else if (code == ErrorMapping.NotLeaderForPartitionCode() || partitionsRemoved = true; } else if (code != ErrorMapping.NoError()) { - exception += "\nException for " + fp.getTopic() +":"+ fp.getPartition() + ": " + + exception += "\nException for " + fp.getTopic() + ":" + fp.getPartition() + ": " + ExceptionUtils.stringifyException(ErrorMapping.exceptionFor(code)); } } @@ -307,7 +301,7 @@ else if (code != ErrorMapping.NoError()) { // get valid offsets for these partitions and try again. LOG.warn("The following partitions had an invalid offset: {}", partitionsToGetOffsetsFor); requestAndSetSpecificTimeOffsetsFromKafka(consumer, partitionsToGetOffsetsFor, invalidOffsetBehavior); - + LOG.warn("The new partition offsets are {}", partitionsToGetOffsetsFor); continue; // jump back to create a new fetch request. The offset has not been touched. } @@ -316,7 +310,7 @@ else if (partitionsRemoved) { } else { // partitions failed on an error - throw new IOException("Error while fetching from broker '" + broker +"': " + exception); + throw new IOException("Error while fetching from broker '" + broker + "': " + exception); } } else { // successful fetch, reset offsetOutOfRangeCount. @@ -328,11 +322,11 @@ else if (partitionsRemoved) { int messagesInFetch = 0; int deletedMessages = 0; Iterator> partitionsIterator = partitions.iterator(); - + partitionsLoop: while (partitionsIterator.hasNext()) { final KafkaTopicPartitionState currentPartition = partitionsIterator.next(); - + final ByteBufferMessageSet messageSet = fetchResponse.messageSet( currentPartition.getTopic(), currentPartition.getPartition()); @@ -341,7 +335,7 @@ else if (partitionsRemoved) { messagesInFetch++; final ByteBuffer payload = msg.message().payload(); final long offset = msg.offset(); - + if (offset <= currentPartition.getOffset()) { // we have seen this message already LOG.info("Skipping message with offset " + msg.offset() @@ -373,15 +367,15 @@ else if (partitionsRemoved) { keyPayload.get(keyBytes); } - final T value = deserializer.deserialize(keyBytes, valueBytes, + final T value = deserializer.deserialize(keyBytes, valueBytes, currentPartition.getTopic(), currentPartition.getPartition(), offset); - + if (deserializer.isEndOfStream(value)) { // remove partition from subscribed partitions. partitionsIterator.remove(); continue partitionsLoop; } - + owner.emitRecord(value, currentPartition, offset); } else { @@ -427,7 +421,7 @@ public void cancel() { this.interrupt(); } - + // ------------------------------------------------------------------------ // Kafka Request Utils // ------------------------------------------------------------------------ @@ -442,8 +436,7 @@ public void cancel() { private static void requestAndSetSpecificTimeOffsetsFromKafka( SimpleConsumer consumer, List> partitions, - long whichTime) throws IOException - { + long whichTime) throws IOException { Map requestInfo = new HashMap<>(); for (KafkaTopicPartitionState part : partitions) { requestInfo.put(part.getKafkaPartitionHandle(), new PartitionOffsetRequestInfo(whichTime, 1)); @@ -461,8 +454,7 @@ private static void requestAndSetSpecificTimeOffsetsFromKafka( */ private static void requestAndSetEarliestOrLatestOffsetsFromKafka( SimpleConsumer consumer, - List> partitions) throws Exception - { + List> partitions) throws Exception { Map requestInfo = new HashMap<>(); for (KafkaTopicPartitionState part : partitions) { if (part.getOffset() == OffsetRequest.EarliestTime() || part.getOffset() == OffsetRequest.LatestTime()) { @@ -486,8 +478,7 @@ private static void requestAndSetEarliestOrLatestOffsetsFromKafka( private static void requestAndSetOffsetsFromKafka( SimpleConsumer consumer, List> partitionStates, - Map partitionToRequestInfo) throws IOException - { + Map partitionToRequestInfo) throws IOException { int retries = 0; OffsetResponse response; while (true) { @@ -529,8 +520,7 @@ private static void requestAndSetOffsetsFromKafka( } private static void checkAllPartitionsHaveDefinedStartingOffsets( - List> partitions) - { + List> partitions) { for (KafkaTopicPartitionState part : partitions) { if (!part.isOffsetDefined()) { throw new IllegalArgumentException("SimpleConsumerThread received a partition with undefined starting offset"); diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java index c02c2cbc03df9..b6822e2cacfae 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java @@ -18,15 +18,14 @@ package org.apache.flink.streaming.connectors.kafka.internals; -import kafka.utils.ZKGroupTopicDirs; +import org.apache.flink.configuration.ConfigConstants; +import kafka.utils.ZKGroupTopicDirs; 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.flink.configuration.ConfigConstants; import org.apache.kafka.clients.consumer.ConsumerConfig; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -45,14 +44,13 @@ public class ZookeeperOffsetHandler { private final CuratorFramework curatorClient; - public ZookeeperOffsetHandler(Properties props) { this.groupId = props.getProperty(ConsumerConfig.GROUP_ID_CONFIG); if (this.groupId == null) { throw new IllegalArgumentException("Required property '" + ConsumerConfig.GROUP_ID_CONFIG + "' has not been set"); } - + String zkConnect = props.getProperty("zookeeper.connect"); if (zkConnect == null) { throw new IllegalArgumentException("Required property 'zookeeper.connect' has not been set"); @@ -61,16 +59,16 @@ public ZookeeperOffsetHandler(Properties props) { // we use Curator's default timeouts int sessionTimeoutMs = Integer.valueOf(props.getProperty("zookeeper.session.timeout.ms", "60000")); int connectionTimeoutMs = Integer.valueOf(props.getProperty("zookeeper.connection.timeout.ms", "15000")); - + // undocumented config options allowing users to configure the retry policy. (they are "flink." prefixed as they are no official kafka configs) int backoffBaseSleepTime = Integer.valueOf(props.getProperty("flink.zookeeper.base-sleep-time.ms", "100")); int backoffMaxRetries = Integer.valueOf(props.getProperty("flink.zookeeper.max-retries", "10")); - + RetryPolicy retryPolicy = new ExponentialBackoffRetry(backoffBaseSleepTime, backoffMaxRetries); curatorClient = CuratorFrameworkFactory.newClient(zkConnect, sessionTimeoutMs, connectionTimeoutMs, retryPolicy); curatorClient.start(); } - + // ------------------------------------------------------------------------ // Offset access and manipulation // ------------------------------------------------------------------------ @@ -79,7 +77,7 @@ public ZookeeperOffsetHandler(Properties props) { * Commits offsets for Kafka partitions to ZooKeeper. The given offsets to this method should be the offsets of * the last processed records; this method will take care of incrementing the offsets by 1 before committing them so * that the committed offsets to Zookeeper represent the next record to process. - * + * * @param internalOffsets The internal offsets (representing last processed records) for the partitions to commit. * @throws Exception The method forwards exceptions. */ @@ -105,7 +103,7 @@ public Long getCommittedOffset(KafkaTopicPartition partition) throws Exception { /** * Closes the offset handler. - * + * * @throws IOException Thrown, if the handler cannot be closed properly. */ public void close() throws IOException { @@ -115,7 +113,7 @@ public void close() throws IOException { // ------------------------------------------------------------------------ // Communication with Zookeeper // ------------------------------------------------------------------------ - + public static void setOffsetInZooKeeper(CuratorFramework curatorClient, String groupId, String topic, int partition, long offset) throws Exception { ZKGroupTopicDirs topicDirs = new ZKGroupTopicDirs(groupId, topic); String path = topicDirs.consumerOffsetDir() + "/" + partition; @@ -128,9 +126,9 @@ public static Long getOffsetFromZooKeeper(CuratorFramework curatorClient, String ZKGroupTopicDirs topicDirs = new ZKGroupTopicDirs(groupId, topic); String path = topicDirs.consumerOffsetDir() + "/" + partition; curatorClient.newNamespaceAwareEnsurePath(path).ensure(curatorClient.getZookeeperClient()); - + byte[] data = curatorClient.getData().forPath(path); - + if (data == null) { return null; } else { diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSourceTest.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSourceTest.java index 2dedecbe31de9..a704c2f416708 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSourceTest.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSourceTest.java @@ -18,12 +18,16 @@ package org.apache.flink.streaming.connectors.kafka; -import java.util.Properties; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.streaming.util.serialization.AvroRowDeserializationSchema; import org.apache.flink.streaming.util.serialization.DeserializationSchema; import org.apache.flink.types.Row; +import java.util.Properties; + +/** + * Tests for the {@link Kafka08AvroTableSource}. + */ public class Kafka08AvroTableSourceTest extends KafkaTableSourceTestBase { @Override diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java index 8cc735d0a420f..20dc6b73a8252 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java @@ -17,12 +17,12 @@ package org.apache.flink.streaming.connectors.kafka; -import org.apache.curator.framework.CuratorFramework; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.connectors.kafka.config.StartupMode; import org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler; +import org.apache.curator.framework.CuratorFramework; import org.junit.Test; import java.util.Properties; @@ -31,6 +31,9 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +/** + * IT cases for Kafka 0.8 . + */ public class Kafka08ITCase extends KafkaConsumerTestBase { // ------------------------------------------------------------------------ @@ -42,7 +45,6 @@ public void testFailOnNoBroker() throws Exception { runFailOnNoBrokerTest(); } - @Test(timeout = 60000) public void testConcurrentProducerConsumerTopology() throws Exception { runSimpleConcurrentProducerConsumerTopology(); @@ -79,7 +81,7 @@ public void testInvalidOffset() throws Exception { final String topic = writeSequence("invalidOffsetTopic", 20, parallelism, 1); // set invalid offset: - CuratorFramework curatorClient = ((KafkaTestEnvironmentImpl)kafkaServer).createCuratorClient(); + CuratorFramework curatorClient = ((KafkaTestEnvironmentImpl) kafkaServer).createCuratorClient(); ZookeeperOffsetHandler.setOffsetInZooKeeper(curatorClient, standardProps.getProperty("group.id"), topic, 0, 1234); curatorClient.close(); @@ -166,7 +168,7 @@ public void runOffsetManipulationInZooKeeperTest() { final Long offset = (long) (Math.random() * Long.MAX_VALUE); - CuratorFramework curatorFramework = ((KafkaTestEnvironmentImpl)kafkaServer ).createCuratorClient(); + CuratorFramework curatorFramework = ((KafkaTestEnvironmentImpl) kafkaServer).createCuratorClient(); kafkaServer.createTestTopic(topicName, 3, 2); ZookeeperOffsetHandler.setOffsetInZooKeeper(curatorFramework, groupId, topicName, 0, offset); @@ -211,7 +213,7 @@ public void testOffsetAutocommitTest() throws Exception { readSequence(env, StartupMode.GROUP_OFFSETS, null, readProps, parallelism, topicName, 100, 0); // get the offset - CuratorFramework curatorFramework = ((KafkaTestEnvironmentImpl)kafkaServer).createCuratorClient(); + CuratorFramework curatorFramework = ((KafkaTestEnvironmentImpl) kafkaServer).createCuratorClient(); Long o1 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorFramework, standardProps.getProperty("group.id"), topicName, 0); Long o2 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorFramework, standardProps.getProperty("group.id"), topicName, 1); @@ -223,7 +225,7 @@ public void testOffsetAutocommitTest() throws Exception { boolean atLeastOneOffsetSet = (o1 != null && o1 > 0 && o1 <= 100) || (o2 != null && o2 > 0 && o2 <= 100) || (o3 != null && o3 > 0 && o3 <= 100); - assertTrue("Expecting at least one offset to be set o1="+o1+" o2="+o2+" o3="+o3, atLeastOneOffsetSet); + assertTrue("Expecting at least one offset to be set o1=" + o1 + " o2=" + o2 + " o3=" + o3, atLeastOneOffsetSet); deleteTestTopic(topicName); } @@ -245,7 +247,7 @@ public void testAllDeletes() throws Exception { runAllDeletesTest(); } - @Test(timeout=60000) + @Test(timeout = 60000) public void testEndOfStream() throws Exception { runEndOfStreamTest(); } diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSinkTest.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSinkTest.java index 213647644247c..ac92c8a1876db 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSinkTest.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSinkTest.java @@ -15,15 +15,19 @@ * 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.FlinkKafkaPartitioner; -import org.apache.flink.types.Row; import org.apache.flink.streaming.util.serialization.JsonRowSerializationSchema; import org.apache.flink.streaming.util.serialization.SerializationSchema; +import org.apache.flink.types.Row; import java.util.Properties; +/** + * Tests for the {@link Kafka08JsonTableSink}. + */ public class Kafka08JsonTableSinkTest extends KafkaTableSinkTestBase { @Override diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceTest.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceTest.java index 27faff4ca78c7..adcd3a2d5a0db 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceTest.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceTest.java @@ -18,12 +18,16 @@ package org.apache.flink.streaming.connectors.kafka; -import java.util.Properties; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.types.Row; import org.apache.flink.streaming.util.serialization.DeserializationSchema; import org.apache.flink.streaming.util.serialization.JsonRowDeserializationSchema; +import org.apache.flink.types.Row; +import java.util.Properties; + +/** + * Tests for the {@link Kafka08JsonTableSource}. + */ public class Kafka08JsonTableSourceTest extends KafkaTableSourceTestBase { @Override diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ProducerITCase.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ProducerITCase.java index 5c951db27e253..80747654964ce 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ProducerITCase.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ProducerITCase.java @@ -18,9 +18,11 @@ package org.apache.flink.streaming.connectors.kafka; - import org.junit.Test; +/** + * IT cases for the {@link FlinkKafkaProducer08}. + */ @SuppressWarnings("serial") public class Kafka08ProducerITCase extends KafkaProducerTestBase { diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumer08Test.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumer08Test.java index 83cdd90a1287e..a43609a633cdc 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumer08Test.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumer08Test.java @@ -18,34 +18,11 @@ package org.apache.flink.streaming.connectors.kafka; - -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; -import static org.mockito.Mockito.mock; -import static org.powermock.api.mockito.PowerMockito.when; - -import java.net.InetAddress; -import java.net.URL; -import java.net.UnknownHostException; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Properties; - -import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.configuration.Configuration; -import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; -import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; -import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; -import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; -import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; -import org.apache.flink.streaming.util.serialization.DeserializationSchema; -import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; import org.apache.flink.streaming.util.serialization.SimpleStringSchema; import org.apache.flink.util.NetUtils; -import org.apache.flink.util.SerializedValue; + import org.apache.kafka.clients.consumer.ConsumerConfig; import org.junit.Test; import org.junit.runner.RunWith; @@ -56,6 +33,20 @@ import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; +import java.net.InetAddress; +import java.net.URL; +import java.net.UnknownHostException; +import java.util.Collections; +import java.util.Properties; + +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.Mockito.mock; +import static org.powermock.api.mockito.PowerMockito.when; + +/** + * Tests for the {@link FlinkKafkaConsumer08}. + */ @RunWith(PowerMockRunner.class) @PrepareForTest(FlinkKafkaConsumer08.class) @PowerMockIgnore("javax.management.*") @@ -101,7 +92,7 @@ public void testValidateZooKeeperConfig() { fail(e.getMessage()); } } - + @Test public void testCreateSourceWithoutCluster() { try { @@ -181,7 +172,7 @@ public void testAtLeastOneBootstrapServerHostIsValid() { + " config are invalid")); } } - + private Properties createKafkaProps(String zookeeperConnect, String bootstrapServers, String groupId) { Properties props = new Properties(); props.setProperty("zookeeper.connect", zookeeperConnect); diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaLocalSystemTime.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaLocalSystemTime.java deleted file mode 100644 index 72d277206a72d..0000000000000 --- a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaLocalSystemTime.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * 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.utils.Time; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class KafkaLocalSystemTime implements Time { - - private static final Logger LOG = LoggerFactory.getLogger(KafkaLocalSystemTime.class); - - @Override - public long milliseconds() { - return System.currentTimeMillis(); - } - - @Override - public long nanoseconds() { - return System.nanoTime(); - } - - @Override - public void sleep(long ms) { - try { - Thread.sleep(ms); - } catch (InterruptedException e) { - LOG.warn("Interruption", e); - } - } - -} - diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java index c7da5afc0b9f7..fc8678fa37b11 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java @@ -20,9 +20,9 @@ import org.apache.flink.streaming.api.operators.StreamSink; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.streaming.connectors.kafka.testutils.FakeStandardProducerConfig; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; -import org.apache.flink.streaming.connectors.kafka.testutils.FakeStandardProducerConfig; import org.apache.flink.streaming.util.serialization.SimpleStringSchema; import org.apache.flink.util.TestLogger; @@ -33,7 +33,6 @@ 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; @@ -51,17 +50,20 @@ import static org.mockito.Mockito.when; import static org.powermock.api.mockito.PowerMockito.whenNew; +/** + * Tests for the {@link KafkaProducer}. + */ @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( // returning a unmodifiable list to mimic KafkaProducer#partitionsFor() behaviour @@ -77,14 +79,14 @@ public Future answer(InvocationOnMock invocation) throws Throwab return null; } }); - + // make sure the FlinkKafkaProducer instantiates our mock producer whenNew(KafkaProducer.class).withAnyArguments().thenReturn(kafkaProducerMock); - + // (1) producer that propagates errors FlinkKafkaProducer08 producerPropagating = new FlinkKafkaProducer08<>( - "mock_topic", new SimpleStringSchema(), FakeStandardProducerConfig.get(), (FlinkKafkaPartitioner)null); + "mock_topic", new SimpleStringSchema(), FakeStandardProducerConfig.get(), (FlinkKafkaPartitioner) null); OneInputStreamOperatorTestHarness testHarness = new OneInputStreamOperatorTestHarness<>(new StreamSink<>(producerPropagating)); @@ -107,7 +109,7 @@ public Future answer(InvocationOnMock invocation) throws Throwab // (2) producer that only logs errors FlinkKafkaProducer08 producerLogging = new FlinkKafkaProducer08<>( - "mock_topic", new SimpleStringSchema(), FakeStandardProducerConfig.get(), (FlinkKafkaPartitioner)null); + "mock_topic", new SimpleStringSchema(), FakeStandardProducerConfig.get(), (FlinkKafkaPartitioner) null); producerLogging.setLogFailuresOnly(true); testHarness = new OneInputStreamOperatorTestHarness<>(new StreamSink(producerLogging)); diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetention08ITCase.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetention08ITCase.java index c28799cfd7f4a..091fae352b65d 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetention08ITCase.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetention08ITCase.java @@ -19,15 +19,18 @@ import org.junit.Test; +/** + * {@link KafkaShortRetentionTestBase} for Kafka 0.8 . + */ @SuppressWarnings("serial") public class KafkaShortRetention08ITCase extends KafkaShortRetentionTestBase { - @Test(timeout=60000) + @Test(timeout = 60000) public void testAutoOffsetReset() throws Exception { runAutoOffsetResetTest(); } - @Test(timeout=60000) + @Test(timeout = 60000) public void testAutoOffsetResetNone() throws Exception { runFailOnAutoOffsetResetNoneEager(); } diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java index 2419b533a1ac8..6b15007871d7f 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java @@ -15,15 +15,26 @@ * limitations under the License. */ - package org.apache.flink.streaming.connectors.kafka; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.streaming.api.operators.StreamSink; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionLeader; +import org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.streaming.connectors.kafka.testutils.ZooKeeperStringSerializer; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema; +import org.apache.flink.util.NetUtils; + import kafka.admin.AdminUtils; import kafka.api.PartitionMetadata; import kafka.common.KafkaException; import kafka.network.SocketServer; import kafka.server.KafkaConfig; import kafka.server.KafkaServer; +import kafka.utils.SystemTime$; import org.I0Itec.zkclient.ZkClient; import org.apache.commons.io.FileUtils; import org.apache.curator.RetryPolicy; @@ -31,19 +42,8 @@ import org.apache.curator.framework.CuratorFrameworkFactory; import org.apache.curator.retry.ExponentialBackoffRetry; import org.apache.curator.test.TestingServer; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSink; -import org.apache.flink.streaming.api.operators.StreamSink; -import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionLeader; -import org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler; -import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; -import org.apache.flink.streaming.connectors.kafka.testutils.ZooKeeperStringSerializer; -import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; -import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema; -import org.apache.flink.util.NetUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import scala.collection.Seq; import java.io.File; import java.io.IOException; @@ -55,12 +55,14 @@ import java.util.Properties; import java.util.UUID; +import scala.collection.Seq; + 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.8 + * An implementation of the KafkaServerProvider for Kafka 0.8 . */ public class KafkaTestEnvironmentImpl extends KafkaTestEnvironment { @@ -166,7 +168,6 @@ public boolean isSecureRunSupported() { return false; } - @Override public void prepare(int numKafkaServers, Properties additionalServerProperties, boolean secureMode) { this.additionalServerProperties = additionalServerProperties; @@ -325,7 +326,7 @@ public CuratorFramework createCuratorClient() { } /** - * Copied from com.github.sakserv.minicluster.KafkaLocalBrokerIntegrationTest (ASL licensed) + * Copied from com.github.sakserv.minicluster.KafkaLocalBrokerIntegrationTest (ASL licensed). */ protected KafkaServer getKafkaServer(int brokerId, File tmpFolder) throws Exception { LOG.info("Starting broker with id {}", brokerId); @@ -342,7 +343,7 @@ protected KafkaServer getKafkaServer(int brokerId, File tmpFolder) throws Except // for CI stability, increase zookeeper session timeout kafkaProperties.put("zookeeper.session.timeout.ms", "30000"); kafkaProperties.put("zookeeper.connection.timeout.ms", "30000"); - if(additionalServerProperties != null) { + if (additionalServerProperties != null) { kafkaProperties.putAll(additionalServerProperties); } @@ -354,7 +355,7 @@ protected KafkaServer getKafkaServer(int brokerId, File tmpFolder) throws Except KafkaConfig kafkaConfig = new KafkaConfig(kafkaProperties); try { - KafkaServer server = new KafkaServer(kafkaConfig, new KafkaLocalSystemTime()); + KafkaServer server = new KafkaServer(kafkaConfig, SystemTime$.MODULE$); server.startup(); return server; } diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueueTest.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueueTest.java index 2df67d98318bd..eb07118c58228 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueueTest.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueueTest.java @@ -28,7 +28,6 @@ import static java.util.Arrays.asList; import static java.util.Collections.emptyList; import static java.util.Collections.singletonList; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; @@ -36,12 +35,15 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +/** + * Tests for the {@link ClosableBlockingQueue}. + */ public class ClosableBlockingQueueTest { // ------------------------------------------------------------------------ // single-threaded unit tests // ------------------------------------------------------------------------ - + @Test public void testCreateQueueHashCodeEquals() { try { @@ -54,14 +56,14 @@ public void testCreateQueueHashCodeEquals() { assertTrue(queue2.isEmpty()); assertEquals(0, queue1.size()); assertEquals(0, queue2.size()); - + assertTrue(queue1.hashCode() == queue2.hashCode()); //noinspection EqualsWithItself assertTrue(queue1.equals(queue1)); //noinspection EqualsWithItself assertTrue(queue2.equals(queue2)); assertTrue(queue1.equals(queue2)); - + assertNotNull(queue1.toString()); assertNotNull(queue2.toString()); @@ -86,7 +88,7 @@ public void testCreateQueueHashCodeEquals() { //noinspection EqualsWithItself assertTrue(queue4.equals(queue4)); assertTrue(queue3.equals(queue4)); - + assertNotNull(queue3.toString()); assertNotNull(queue4.toString()); } @@ -95,7 +97,7 @@ public void testCreateQueueHashCodeEquals() { fail(e.getMessage()); } } - + @Test public void testCloseEmptyQueue() { try { @@ -103,10 +105,10 @@ public void testCloseEmptyQueue() { assertTrue(queue.isOpen()); assertTrue(queue.close()); assertFalse(queue.isOpen()); - + assertFalse(queue.addIfOpen("element")); assertTrue(queue.isEmpty()); - + try { queue.add("some element"); fail("should cause an exception"); @@ -125,15 +127,15 @@ public void testCloseNonEmptyQueue() { try { ClosableBlockingQueue queue = new ClosableBlockingQueue<>(asList(1, 2, 3)); assertTrue(queue.isOpen()); - + assertFalse(queue.close()); assertFalse(queue.close()); - + queue.poll(); assertFalse(queue.close()); assertFalse(queue.close()); - + queue.pollBatch(); assertTrue(queue.close()); @@ -154,36 +156,36 @@ public void testCloseNonEmptyQueue() { fail(e.getMessage()); } } - + @Test public void testPeekAndPoll() { try { ClosableBlockingQueue queue = new ClosableBlockingQueue<>(); - + assertNull(queue.peek()); assertNull(queue.peek()); assertNull(queue.poll()); assertNull(queue.poll()); - + assertEquals(0, queue.size()); - + queue.add("a"); queue.add("b"); queue.add("c"); assertEquals(3, queue.size()); - + assertEquals("a", queue.peek()); assertEquals("a", queue.peek()); assertEquals("a", queue.peek()); assertEquals(3, queue.size()); - + assertEquals("a", queue.poll()); assertEquals("b", queue.poll()); assertEquals(1, queue.size()); - + assertEquals("c", queue.peek()); assertEquals("c", queue.peek()); @@ -193,9 +195,9 @@ public void testPeekAndPoll() { assertNull(queue.poll()); assertNull(queue.peek()); assertNull(queue.peek()); - + assertTrue(queue.close()); - + try { queue.peek(); fail("should cause an exception"); @@ -222,13 +224,13 @@ public void testPollBatch() { ClosableBlockingQueue queue = new ClosableBlockingQueue<>(); assertNull(queue.pollBatch()); - + queue.add("a"); queue.add("b"); - + assertEquals(asList("a", "b"), queue.pollBatch()); assertNull(queue.pollBatch()); - + queue.add("c"); assertEquals(singletonList("c"), queue.pollBatch()); @@ -363,16 +365,16 @@ public void testGetBatchBlocking() { fail(e.getMessage()); } } - + // ------------------------------------------------------------------------ // multi-threaded tests // ------------------------------------------------------------------------ - + @Test public void notifyOnClose() { try { final long oneYear = 365L * 24 * 60 * 60 * 1000; - + // test "getBatchBlocking()" final ClosableBlockingQueue queue1 = new ClosableBlockingQueue<>(); QueueCall call1 = new QueueCall() { @@ -418,7 +420,7 @@ public void call() throws Exception { fail(e.getMessage()); } } - + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") @Test public void testMultiThreadedAddGet() { @@ -426,9 +428,9 @@ public void testMultiThreadedAddGet() { final ClosableBlockingQueue queue = new ClosableBlockingQueue<>(); final AtomicReference pushErrorRef = new AtomicReference<>(); final AtomicReference pollErrorRef = new AtomicReference<>(); - + final int numElements = 2000; - + Thread pusher = new Thread("pusher") { @Override @@ -437,14 +439,14 @@ public void run() { final Random rnd = new Random(); for (int i = 0; i < numElements; i++) { queue.add(i); - + // sleep a bit, sometimes int sleepTime = rnd.nextInt(3); if (sleepTime > 1) { Thread.sleep(sleepTime); } } - + while (true) { if (queue.close()) { break; @@ -466,11 +468,11 @@ public void run() { public void run() { try { int count = 0; - + try { final Random rnd = new Random(); int nextExpected = 0; - + while (true) { int getMethod = count % 7; switch (getMethod) { @@ -534,7 +536,7 @@ public void run() { count++; } } - + // sleep a bit, sometimes int sleepTime = rnd.nextInt(3); if (sleepTime > 1) { @@ -551,10 +553,10 @@ public void run() { } }; poller.start(); - + pusher.join(); poller.join(); - + if (pushErrorRef.get() != null) { Throwable t = pushErrorRef.get(); t.printStackTrace(); @@ -571,16 +573,16 @@ public void run() { fail(e.getMessage()); } } - + // ------------------------------------------------------------------------ // Utils // ------------------------------------------------------------------------ - + private static void testCallExitsOnClose( final QueueCall call, ClosableBlockingQueue queue) throws Exception { - + final AtomicReference errorRef = new AtomicReference<>(); - + Runnable runnable = new Runnable() { @Override public void run() { @@ -602,7 +604,7 @@ public void run() { Throwable cause = errorRef.get(); assertTrue(cause instanceof IllegalStateException); } - + private interface QueueCall { void call() throws Exception; } diff --git a/flink-connectors/flink-connector-kafka-0.9/pom.xml b/flink-connectors/flink-connector-kafka-0.9/pom.xml index 0140353778eec..248f7e12c9878 100644 --- a/flink-connectors/flink-connector-kafka-0.9/pom.xml +++ b/flink-connectors/flink-connector-kafka-0.9/pom.xml @@ -118,7 +118,7 @@ under the License. ${project.version} test - + org.apache.flink flink-tests_${scala.binary.version} @@ -208,5 +208,5 @@ under the License. - + diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java index e63834820fb61..a1d896793d184 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java @@ -34,7 +34,6 @@ import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.serialization.ByteArrayDeserializer; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -49,10 +48,10 @@ /** * 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. - * + * 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". + * 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 @@ -73,17 +72,16 @@ public class FlinkKafkaConsumer09 extends FlinkKafkaConsumerBase { private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaConsumer09.class); - /** Configuration key to change the polling timeout **/ + /** Configuration key to change the polling timeout. **/ public static final String KEY_POLL_TIMEOUT = "flink.poll-timeout"; - /** 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 **/ + /** User-supplied properties for Kafka. **/ protected final Properties properties; /** From Kafka's Javadoc: The time, in milliseconds, spent waiting in poll if data is not @@ -93,7 +91,7 @@ public class FlinkKafkaConsumer09 extends FlinkKafkaConsumerBase { // ------------------------------------------------------------------------ /** - * Creates a new Kafka streaming source consumer for Kafka 0.9.x + * Creates a new Kafka streaming source consumer for Kafka 0.9.x . * * @param topic * The name of the topic that should be consumed. @@ -109,7 +107,7 @@ public FlinkKafkaConsumer09(String topic, DeserializationSchema valueDeserial /** * Creates a new Kafka streaming source consumer for Kafka 0.9.x * - * This constructor allows passing a {@see KeyedDeserializationSchema} for reading key/value + *

    This constructor allows passing a {@see KeyedDeserializationSchema} for reading key/value * pairs, offsets, and topic names from Kafka. * * @param topic @@ -126,7 +124,7 @@ public FlinkKafkaConsumer09(String topic, KeyedDeserializationSchema deserial /** * Creates a new Kafka streaming source consumer for Kafka 0.9.x * - * This constructor allows passing multiple topics to the consumer. + *

    This constructor allows passing multiple topics to the consumer. * * @param topics * The Kafka topics to read from. @@ -142,7 +140,7 @@ public FlinkKafkaConsumer09(List topics, DeserializationSchema deseri /** * Creates a new Kafka streaming source consumer for Kafka 0.9.x * - * This constructor allows passing multiple topics and a key/value deserialization schema. + *

    This constructor allows passing multiple topics and a key/value deserialization schema. * * @param topics * The Kafka topics to read from. @@ -216,7 +214,7 @@ protected List getKafkaPartitions(List topics) { if (partitionsForTopic != null) { partitions.addAll(convertToFlinkKafkaTopicPartition(partitionsForTopic)); } - else{ + else { LOG.info("Unable to retrieve any partitions for the requested topic: {}", topic); } } @@ -243,12 +241,12 @@ protected boolean getIsAutoCommitEnabled() { } // ------------------------------------------------------------------------ - // Utilities + // Utilities // ------------------------------------------------------------------------ /** - * Converts a list of Kafka PartitionInfo's to Flink's KafkaTopicPartition (which are serializable) - * + * 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 */ @@ -264,7 +262,7 @@ private static List convertToFlinkKafkaTopicPartition(List< /** * Makes sure that the ByteArrayDeserializer is registered in the Kafka properties. - * + * * @param props The Kafka properties to register the serializer in. */ private static void setDeserializer(Properties props) { diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer09.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer09.java index cbed3612e1e27..6b9768ed1c69a 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer09.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer09.java @@ -27,11 +27,10 @@ import java.util.Properties; - /** * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.9. * - * Please note that this producer does not have any reliability guarantees. + *

    Please note that this producer does not have any reliability guarantees. * * @param Type of the messages to write into Kafka. */ diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSource.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSource.java index 9e1172be12c82..d69187e5f60ba 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSource.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSource.java @@ -18,13 +18,15 @@ package org.apache.flink.streaming.connectors.kafka; -import java.util.Properties; -import org.apache.avro.specific.SpecificRecord; -import org.apache.avro.specific.SpecificRecordBase; import org.apache.flink.streaming.util.serialization.DeserializationSchema; import org.apache.flink.table.sources.StreamTableSource; import org.apache.flink.types.Row; +import org.apache.avro.specific.SpecificRecord; +import org.apache.avro.specific.SpecificRecordBase; + +import java.util.Properties; + /** * Kafka {@link StreamTableSource} for Kafka 0.9. */ diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSink.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSink.java index a81422eeb2ddc..b2227cd67d3d1 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSink.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSink.java @@ -15,13 +15,14 @@ * 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.FlinkKafkaDelegatePartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; -import org.apache.flink.types.Row; import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; import org.apache.flink.streaming.util.serialization.SerializationSchema; +import org.apache.flink.types.Row; import java.util.Properties; @@ -29,9 +30,9 @@ * Kafka 0.9 {@link KafkaTableSink} that serializes data in JSON format. */ public class Kafka09JsonTableSink extends KafkaJsonTableSink { - + /** - * Creates {@link KafkaTableSink} for Kafka 0.9 + * Creates {@link KafkaTableSink} for Kafka 0.9 . * * @param topic topic in Kafka to which table is written * @param properties properties to connect to Kafka @@ -42,7 +43,7 @@ public Kafka09JsonTableSink(String topic, Properties properties, FlinkKafkaParti } /** - * Creates {@link KafkaTableSink} for Kafka 0.9 + * Creates {@link KafkaTableSink} for Kafka 0.9 . * * @param topic topic in Kafka to which table is written * @param properties properties to connect to Kafka diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.java index 26fffa57428de..80811b24577ec 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.java @@ -19,9 +19,9 @@ package org.apache.flink.streaming.connectors.kafka; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.types.Row; -import org.apache.flink.table.sources.StreamTableSource; import org.apache.flink.streaming.util.serialization.DeserializationSchema; +import org.apache.flink.table.sources.StreamTableSource; +import org.apache.flink.types.Row; import java.util.Properties; diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java index c581332afd6fb..bc50a4cb1292c 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java @@ -19,9 +19,9 @@ package org.apache.flink.streaming.connectors.kafka; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.types.Row; -import org.apache.flink.table.sources.StreamTableSource; import org.apache.flink.streaming.util.serialization.DeserializationSchema; +import org.apache.flink.table.sources.StreamTableSource; +import org.apache.flink.types.Row; import java.util.Properties; diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Handover.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Handover.java index e6e3c51d2cca7..0897f53359275 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Handover.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Handover.java @@ -19,10 +19,12 @@ package org.apache.flink.streaming.connectors.kafka.internal; import org.apache.flink.util.ExceptionUtils; + import org.apache.kafka.clients.consumer.ConsumerRecords; import javax.annotation.Nonnull; import javax.annotation.concurrent.ThreadSafe; + import java.io.Closeable; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -32,13 +34,13 @@ * producer thread to a consumer thread. It effectively behaves like a * "size one blocking queue", with some extras around exception reporting, closing, and * waking up thread without {@link Thread#interrupt() interrupting} threads. - * + * *

    This class is used in the Flink Kafka Consumer to hand over data and exceptions between * the thread that runs the KafkaConsumer class and the main thread. - * + * *

    The Handover has the notion of "waking up" the producer thread with a {@link WakeupException} * rather than a thread interrupt. - * + * *

    The Handover can also be "closed", signalling from one thread to the other that it * the thread has terminated. */ @@ -54,12 +56,12 @@ public final class Handover implements Closeable { /** * Polls the next element from the Handover, possibly blocking until the next element is * available. This method behaves similar to polling from a blocking queue. - * + * *

    If an exception was handed in by the producer ({@link #reportError(Throwable)}), then * that exception is thrown rather than an element being returned. - * + * * @return The next element (buffer of records, never null). - * + * * @throws ClosedException Thrown if the Handover was {@link #close() closed}. * @throws Exception Rethrows exceptions from the {@link #reportError(Throwable)} method. */ @@ -81,7 +83,7 @@ public ConsumerRecords pollNext() throws Exception { // this statement cannot be reached since the above method always throws an exception // this is only here to silence the compiler and any warnings - return ConsumerRecords.empty(); + return ConsumerRecords.empty(); } } } @@ -90,11 +92,11 @@ public ConsumerRecords pollNext() throws Exception { * Hands over an element from the producer. If the Handover already has an element that was * not yet picked up by the consumer thread, this call blocks until the consumer picks up that * previous element. - * + * *

    This behavior is similar to a "size one" blocking queue. - * + * * @param element The next element to hand over. - * + * * @throws InterruptedException * Thrown, if the thread is interrupted while blocking for the Handover to be empty. * @throws WakeupException @@ -135,15 +137,15 @@ else if (error == null) { * Reports an exception. The consumer will throw the given exception immediately, if * it is currently blocked in the {@link #pollNext()} method, or the next time it * calls that method. - * + * *

    After this method has been called, no call to either {@link #produce(ConsumerRecords)} * or {@link #pollNext()} will ever return regularly any more, but will always return * exceptionally. - * + * *

    If another exception was already reported, this method does nothing. - * + * *

    For the producer, the Handover will appear as if it was {@link #close() closed}. - * + * * @param t The exception to report. */ public void reportError(Throwable t) { @@ -163,7 +165,7 @@ public void reportError(Throwable t) { * Closes the handover. Both the {@link #produce(ConsumerRecords)} method and the * {@link #pollNext()} will throw a {@link ClosedException} on any currently blocking and * future invocations. - * + * *

    If an exception was previously reported via the {@link #reportError(Throwable)} method, * that exception will not be overridden. The consumer thread will throw that exception upon * calling {@link #pollNext()}, rather than the {@code ClosedException}. diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java index 1c87542f8ae85..82294d711fc15 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java @@ -33,7 +33,6 @@ import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -44,7 +43,7 @@ /** * A fetcher that fetches data from Kafka brokers via the Kafka 0.9 consumer API. - * + * * @param The type of elements produced by the fetcher. */ public class Kafka09Fetcher extends AbstractFetcher { @@ -53,16 +52,16 @@ public class Kafka09Fetcher extends AbstractFetcher { // ------------------------------------------------------------------------ - /** The schema to convert between Kafka's byte messages, and Flink's objects */ + /** The schema to convert between Kafka's byte messages, and Flink's objects. */ private final KeyedDeserializationSchema deserializer; - /** The handover of data and exceptions between the consumer thread and the task thread */ + /** The handover of data and exceptions between the consumer thread and the task thread. */ private final Handover handover; - /** The thread that runs the actual KafkaConsumer and hand the record batches to this fetcher */ + /** The thread that runs the actual KafkaConsumer and hand the record batches to this fetcher. */ private final KafkaConsumerThread consumerThread; - /** Flag to mark the main work loop as alive */ + /** Flag to mark the main work loop as alive. */ private volatile boolean running = true; // ------------------------------------------------------------------------ @@ -80,8 +79,7 @@ public Kafka09Fetcher( KeyedDeserializationSchema deserializer, Properties kafkaProperties, long pollTimeout, - boolean useMetrics) throws Exception - { + boolean useMetrics) throws Exception { super( sourceContext, assignedPartitionsWithInitialOffsets, @@ -97,7 +95,7 @@ public Kafka09Fetcher( final MetricGroup kafkaMetricGroup = metricGroup.addGroup("KafkaConsumer"); addOffsetStateGauge(kafkaMetricGroup); - + this.consumerThread = new KafkaConsumerThread( LOG, handover, diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge.java index 37ba34caea08b..c0b94412917ee 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge.java @@ -25,11 +25,11 @@ /** * The ConsumerCallBridge simply calls methods on the {@link KafkaConsumer}. - * - * This indirection is necessary, because Kafka broke binary compatibility between 0.9 and 0.10, + * + *

    This indirection is necessary, because Kafka broke binary compatibility between 0.9 and 0.10, * for example changing {@code assign(List)} to {@code assign(Collection)}. - * - * Because of that, we need to have two versions whose compiled code goes against different method signatures. + * + *

    Because of that, we need to have two versions whose compiled code goes against different method signatures. * Even though the source of subclasses may look identical, the byte code will be different, because they * are compiled against different dependencies. */ diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerThread.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerThread.java index cbe15511cdb06..0c5482a9d6708 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerThread.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerThread.java @@ -22,6 +22,7 @@ import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionState; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionStateSentinel; import org.apache.flink.streaming.connectors.kafka.internals.metrics.KafkaMetricWrapper; + import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.consumer.OffsetAndMetadata; @@ -30,7 +31,6 @@ import org.apache.kafka.common.MetricName; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.WakeupException; - import org.slf4j.Logger; import java.util.ArrayList; @@ -45,54 +45,53 @@ * The thread the runs the {@link KafkaConsumer}, connecting to the brokers and polling records. * The thread pushes the data into a {@link Handover} to be picked up by the fetcher that will * deserialize and emit the records. - * + * *

    IMPORTANT: This thread must not be interrupted when attempting to shut it down. * The Kafka consumer code was found to not always handle interrupts well, and to even * deadlock in certain situations. - * + * *

    Implementation Note: This code is written to be reusable in later versions of the KafkaConsumer. * Because Kafka is not maintaining binary compatibility, we use a "call bridge" as an indirection * to the KafkaConsumer calls that change signature. */ public class KafkaConsumerThread extends Thread { - /** Logger for this consumer */ + /** Logger for this consumer. */ private final Logger log; - /** The handover of data and exceptions between the consumer thread and the task thread */ + /** The handover of data and exceptions between the consumer thread and the task thread. */ private final Handover handover; - /** The next offsets that the main thread should commit */ + /** The next offsets that the main thread should commit. */ private final AtomicReference> nextOffsetsToCommit; - /** The configuration for the Kafka consumer */ + /** The configuration for the Kafka consumer. */ private final Properties kafkaProperties; - /** The partitions that this consumer reads from */ + /** The partitions that this consumer reads from. */ private final KafkaTopicPartitionState[] subscribedPartitionStates; /** We get this from the outside to publish metrics. **/ private final MetricGroup kafkaMetricGroup; - /** The indirections on KafkaConsumer methods, for cases where KafkaConsumer compatibility is broken */ + /** The indirections on KafkaConsumer methods, for cases where KafkaConsumer compatibility is broken. */ private final KafkaConsumerCallBridge consumerCallBridge; - /** The maximum number of milliseconds to wait for a fetch batch */ + /** The maximum number of milliseconds to wait for a fetch batch. */ private final long pollTimeout; - /** Flag whether to add Kafka's metrics to the Flink metrics */ + /** Flag whether to add Kafka's metrics to the Flink metrics. */ private final boolean useMetrics; - /** Reference to the Kafka consumer, once it is created */ + /** Reference to the Kafka consumer, once it is created. */ private volatile KafkaConsumer consumer; - /** Flag to mark the main work loop as alive */ + /** Flag to mark the main work loop as alive. */ private volatile boolean running; - /** Flag tracking whether the latest commit request has completed */ + /** Flag tracking whether the latest commit request has completed. */ private volatile boolean commitInProgress; - public KafkaConsumerThread( Logger log, Handover handover, @@ -271,7 +270,7 @@ public void shutdown() { // this wakes up the consumer if it is blocked handing over records handover.wakeupProducer(); - // this wakes up the consumer if it is blocked in a kafka poll + // this wakes up the consumer if it is blocked in a kafka poll if (consumer != null) { consumer.wakeup(); } @@ -280,11 +279,11 @@ public void shutdown() { /** * Tells this thread to commit a set of offsets. This method does not block, the committing * operation will happen asynchronously. - * + * *

    Only one commit operation may be pending at any time. If the committing takes longer than * the frequency with which this method is called, then some commits may be skipped due to being * superseded by newer ones. - * + * * @param offsetsToCommit The offsets to commit */ public void setOffsetsToCommit(Map offsetsToCommit) { diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/resources/log4j.properties b/flink-connectors/flink-connector-kafka-0.9/src/main/resources/log4j.properties index 6bdfb48cd550f..6eef1747ddfe4 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/resources/log4j.properties +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/resources/log4j.properties @@ -26,4 +26,3 @@ 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-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSourceTest.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSourceTest.java index eff82648c0686..5e3c42c989cfb 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSourceTest.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSourceTest.java @@ -18,12 +18,16 @@ package org.apache.flink.streaming.connectors.kafka; -import java.util.Properties; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.streaming.util.serialization.AvroRowDeserializationSchema; import org.apache.flink.streaming.util.serialization.DeserializationSchema; import org.apache.flink.types.Row; +import java.util.Properties; + +/** + * Tests for the {@link Kafka09AvroTableSource}. + */ public class Kafka09AvroTableSourceTest extends KafkaTableSourceTestBase { @Override diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09FetcherTest.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09FetcherTest.java index 6e13db2935f9e..f55c264634af6 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09FetcherTest.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09FetcherTest.java @@ -39,10 +39,8 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.OffsetCommitCallback; import org.apache.kafka.common.TopicPartition; - import org.junit.Test; import org.junit.runner.RunWith; - import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -63,7 +61,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; - import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyLong; import static org.powermock.api.mockito.PowerMockito.doAnswer; @@ -91,10 +88,10 @@ public void testCommitDoesNotBlock() throws Exception { // ----- the mock consumer with blocking poll calls ---- final MultiShotLatch blockerLatch = new MultiShotLatch(); - + KafkaConsumer mockConsumer = mock(KafkaConsumer.class); when(mockConsumer.poll(anyLong())).thenAnswer(new Answer>() { - + @Override public ConsumerRecords answer(InvocationOnMock invocation) throws InterruptedException { sync.trigger(); @@ -157,7 +154,7 @@ public void run() { sync.await(); // ----- trigger the offset commit ----- - + final AtomicReference commitError = new AtomicReference<>(); final Thread committer = new Thread("committer runner") { @Override @@ -192,11 +189,11 @@ public void run() { @Test public void ensureOffsetsGetCommitted() throws Exception { - + // test data final KafkaTopicPartition testPartition1 = new KafkaTopicPartition("test", 42); final KafkaTopicPartition testPartition2 = new KafkaTopicPartition("another", 99); - + final Map testCommitData1 = new HashMap<>(); testCommitData1.put(testPartition1, 11L); testCommitData1.put(testPartition2, 18L); @@ -207,7 +204,6 @@ public void ensureOffsetsGetCommitted() throws Exception { final BlockingQueue> commitStore = new LinkedBlockingQueue<>(); - // ----- the mock consumer with poll(), wakeup(), and commit(A)sync calls ---- final MultiShotLatch blockerLatch = new MultiShotLatch(); @@ -234,7 +230,7 @@ public Void answer(InvocationOnMock invocation) { @Override public Void answer(InvocationOnMock invocation) { @SuppressWarnings("unchecked") - Map offsets = + Map offsets = (Map) invocation.getArguments()[0]; OffsetCommitCallback callback = (OffsetCommitCallback) invocation.getArguments()[1]; @@ -242,7 +238,7 @@ public Void answer(InvocationOnMock invocation) { commitStore.add(offsets); callback.onComplete(offsets, null); - return null; + return null; } }).when(mockConsumer).commitAsync( Mockito.>any(), any(OffsetCommitCallback.class)); @@ -322,7 +318,7 @@ else if (partition.topic().equals("another")) { assertEquals(27L, entry.getValue().offset()); } } - + // ----- test done, wait till the fetcher is done for a clean shutdown ----- fetcher.cancel(); fetcherRunner.join(); @@ -387,7 +383,6 @@ public void testCancellationWhenEmitBlocks() throws Exception { 0L, false); - // ----- run the fetcher ----- final AtomicReference error = new AtomicReference<>(); diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ITCase.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ITCase.java index ca9965c0a8aed..de4d010143d6c 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ITCase.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ITCase.java @@ -19,6 +19,9 @@ import org.junit.Test; +/** + * IT cases for Kafka 0.9 . + */ public class Kafka09ITCase extends KafkaConsumerTestBase { // ------------------------------------------------------------------------ @@ -35,7 +38,6 @@ public void testConcurrentProducerConsumerTopology() throws Exception { runSimpleConcurrentProducerConsumerTopology(); } - @Test(timeout = 60000) public void testKeyValueSupport() throws Exception { runKeyValueTest(); @@ -58,7 +60,6 @@ public void testFailOnDeploy() throws Exception { runFailOnDeployTest(); } - // --- source to partition mappings and exactly once --- @Test(timeout = 60000) diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSinkTest.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSinkTest.java index 3afb5e4d68be5..c8fb4cd4d0cad 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSinkTest.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSinkTest.java @@ -15,15 +15,19 @@ * 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.FlinkKafkaPartitioner; -import org.apache.flink.types.Row; import org.apache.flink.streaming.util.serialization.JsonRowSerializationSchema; import org.apache.flink.streaming.util.serialization.SerializationSchema; +import org.apache.flink.types.Row; import java.util.Properties; +/** + * Tests for the {@link Kafka09JsonTableSink}. + */ public class Kafka09JsonTableSinkTest extends KafkaTableSinkTestBase { @Override diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceTest.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceTest.java index 35cd9ce56d16b..ec70386949f66 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceTest.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceTest.java @@ -18,12 +18,16 @@ package org.apache.flink.streaming.connectors.kafka; -import java.util.Properties; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.types.Row; import org.apache.flink.streaming.util.serialization.DeserializationSchema; import org.apache.flink.streaming.util.serialization.JsonRowDeserializationSchema; +import org.apache.flink.types.Row; +import java.util.Properties; + +/** + * Tests for the {@link Kafka09JsonTableSource}. + */ public class Kafka09JsonTableSourceTest extends KafkaTableSourceTestBase { @Override diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ProducerITCase.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ProducerITCase.java index ae4f5b247f0db..fe8a1a5d8ffe8 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ProducerITCase.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ProducerITCase.java @@ -18,9 +18,11 @@ package org.apache.flink.streaming.connectors.kafka; - import org.junit.Test; +/** + * IT cases for the {@link FlinkKafkaProducer09}. + */ @SuppressWarnings("serial") public class Kafka09ProducerITCase extends KafkaProducerTestBase { diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09SecuredRunITCase.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09SecuredRunITCase.java index 16a13c061b695..d41cd91a65375 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09SecuredRunITCase.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09SecuredRunITCase.java @@ -18,16 +18,15 @@ package org.apache.flink.streaming.connectors.kafka; import org.apache.flink.test.util.SecureTestEnvironment; + import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; - -/* - * Kafka Secure Connection (kerberos) IT test case +/** + * Kafka Secure Connection (kerberos) IT test case. */ public class Kafka09SecuredRunITCase extends KafkaConsumerTestBase { @@ -51,7 +50,6 @@ public static void shutDownServices() { SecureTestEnvironment.cleanup(); } - //timeout interval is large since in Travis, ZK connection timeout occurs frequently //The timeout for the test case is 2 times timeout of ZK connection @Test(timeout = 600000) diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java index e9a4947b6b18b..6b6c43f72144b 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java @@ -20,9 +20,9 @@ import org.apache.flink.streaming.api.operators.StreamSink; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.streaming.connectors.kafka.testutils.FakeStandardProducerConfig; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; -import org.apache.flink.streaming.connectors.kafka.testutils.FakeStandardProducerConfig; import org.apache.flink.streaming.util.serialization.SimpleStringSchema; import org.apache.flink.util.TestLogger; @@ -31,13 +31,10 @@ 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; @@ -53,17 +50,20 @@ import static org.mockito.Mockito.when; import static org.powermock.api.mockito.PowerMockito.whenNew; +/** + * Tests for the {@link KafkaProducer}. + */ @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( // returning a unmodifiable list to mimic KafkaProducer#partitionsFor() behaviour @@ -79,13 +79,13 @@ public Future answer(InvocationOnMock invocation) throws Throwab 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(), FakeStandardProducerConfig.get(), (FlinkKafkaPartitioner)null); + "mock_topic", new SimpleStringSchema(), FakeStandardProducerConfig.get(), (FlinkKafkaPartitioner) null); OneInputStreamOperatorTestHarness testHarness = new OneInputStreamOperatorTestHarness<>(new StreamSink(producerPropagating)); @@ -106,7 +106,7 @@ public Future answer(InvocationOnMock invocation) throws Throwab // (2) producer that only logs errors FlinkKafkaProducer09 producerLogging = new FlinkKafkaProducer09<>( - "mock_topic", new SimpleStringSchema(), FakeStandardProducerConfig.get(), (FlinkKafkaPartitioner)null); + "mock_topic", new SimpleStringSchema(), FakeStandardProducerConfig.get(), (FlinkKafkaPartitioner) null); producerLogging.setLogFailuresOnly(true); testHarness = new OneInputStreamOperatorTestHarness<>(new StreamSink(producerLogging)); diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java index 84fdbf869a50c..fc38e24df1321 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java @@ -15,9 +15,17 @@ * limitations under the License. */ - package org.apache.flink.streaming.connectors.kafka; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.streaming.api.operators.StreamSink; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.streaming.connectors.kafka.testutils.ZooKeeperStringSerializer; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema; +import org.apache.flink.util.NetUtils; + import kafka.admin.AdminUtils; import kafka.api.PartitionMetadata; import kafka.common.KafkaException; @@ -29,21 +37,12 @@ import org.I0Itec.zkclient.ZkClient; import org.apache.commons.io.FileUtils; import org.apache.curator.test.TestingServer; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSink; -import org.apache.flink.streaming.api.operators.StreamSink; -import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; -import org.apache.flink.streaming.connectors.kafka.testutils.ZooKeeperStringSerializer; -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.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; 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; @@ -54,12 +53,14 @@ import java.util.Properties; import java.util.UUID; +import scala.collection.Seq; + 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 + * An implementation of the KafkaServerProvider for Kafka 0.9 . */ public class KafkaTestEnvironmentImpl extends KafkaTestEnvironment { @@ -166,7 +167,7 @@ public boolean isSecureRunSupported() { public void prepare(int numKafkaServers, Properties additionalServerProperties, boolean secureMode) { //increase the timeout since in Travis ZK connection takes long time for secure connection. - if(secureMode) { + if (secureMode) { //run only one kafka server to avoid multiple ZK connections from many instances - Travis timeout numKafkaServers = 1; zkTimeout = String.valueOf(Integer.parseInt(zkTimeout) * 15); @@ -205,7 +206,7 @@ public void prepare(int numKafkaServers, Properties additionalServerProperties, brokers.add(getKafkaServer(i, tmpKafkaDirs.get(i))); SocketServer socketServer = brokers.get(i).socketServer(); - if(secureMode) { + if (secureMode) { brokerConnectionString += hostAndPortToUrlString(KafkaTestEnvironment.KAFKA_HOST, brokers.get(i).socketServer().boundPort(SecurityProtocol.SASL_PLAINTEXT)) + ","; } else { brokerConnectionString += hostAndPortToUrlString(KafkaTestEnvironment.KAFKA_HOST, brokers.get(i).socketServer().boundPort(SecurityProtocol.PLAINTEXT)) + ","; @@ -298,7 +299,7 @@ public void createTestTopic(String topic, int numberOfPartitions, int replicatio final long deadline = System.nanoTime() + Integer.parseInt(zkTimeout) * 1_000_000L; do { try { - if(secureMode) { + if (secureMode) { //increase wait time since in Travis ZK timeout occurs frequently int wait = Integer.parseInt(zkTimeout) / 100; LOG.info("waiting for {} msecs before the topic {} can be checked", wait, topic); @@ -317,7 +318,7 @@ public void createTestTopic(String topic, int numberOfPartitions, int replicatio // create a new ZK utils connection ZkUtils checkZKConn = getZkUtils(); - if(AdminUtils.topicExists(checkZKConn, topic)) { + if (AdminUtils.topicExists(checkZKConn, topic)) { LOG.info("topic {} has been created successfully", topic); checkZKConn.close(); return; @@ -347,7 +348,7 @@ public void deleteTestTopic(String topic) { } /** - * Copied from com.github.sakserv.minicluster.KafkaLocalBrokerIntegrationTest (ASL licensed) + * Copied from com.github.sakserv.minicluster.KafkaLocalBrokerIntegrationTest (ASL licensed). */ protected KafkaServer getKafkaServer(int brokerId, File tmpFolder) throws Exception { Properties kafkaProperties = new Properties(); @@ -363,7 +364,7 @@ protected KafkaServer getKafkaServer(int brokerId, File tmpFolder) throws Except // for CI stability, increase zookeeper session timeout kafkaProperties.put("zookeeper.session.timeout.ms", zkTimeout); kafkaProperties.put("zookeeper.connection.timeout.ms", zkTimeout); - if(additionalServerProperties != null) { + if (additionalServerProperties != null) { kafkaProperties.putAll(additionalServerProperties); } @@ -374,7 +375,7 @@ protected KafkaServer getKafkaServer(int brokerId, File tmpFolder) throws Except kafkaProperties.put("port", Integer.toString(kafkaPort)); //to support secure kafka cluster - if(secureMode) { + if (secureMode) { LOG.info("Adding Kafka secure configurations"); kafkaProperties.put("listeners", "SASL_PLAINTEXT://" + KAFKA_HOST + ":" + kafkaPort); kafkaProperties.put("advertised.listeners", "SASL_PLAINTEXT://" + KAFKA_HOST + ":" + kafkaPort); @@ -405,7 +406,7 @@ protected KafkaServer getKafkaServer(int brokerId, File tmpFolder) throws Except public Properties getSecureProperties() { Properties prop = new Properties(); - if(secureMode) { + if (secureMode) { prop.put("security.inter.broker.protocol", "SASL_PLAINTEXT"); prop.put("security.protocol", "SASL_PLAINTEXT"); prop.put("sasl.kerberos.service.name", "kafka"); @@ -413,7 +414,7 @@ public Properties getSecureProperties() { //add special timeout for Travis prop.setProperty("zookeeper.session.timeout.ms", zkTimeout); prop.setProperty("zookeeper.connection.timeout.ms", zkTimeout); - prop.setProperty("metadata.fetch.timeout.ms","120000"); + prop.setProperty("metadata.fetch.timeout.ms", "120000"); } return prop; } diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/HandoverTest.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/HandoverTest.java index e95b51be31edf..5bd4affcfe097 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/HandoverTest.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/HandoverTest.java @@ -20,8 +20,8 @@ import org.apache.flink.streaming.connectors.kafka.internal.Handover.WakeupException; import org.apache.flink.util.ExceptionUtils; -import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.ConsumerRecords; import org.junit.Test; import java.io.IOException; @@ -34,7 +34,7 @@ import static org.mockito.Mockito.mock; /** - * Tests for the {@link Handover} between Kafka Consumer Thread and the fetcher's main thread. + * Tests for the {@link Handover} between Kafka Consumer Thread and the fetcher's main thread. */ public class HandoverTest { @@ -219,7 +219,7 @@ public void testWakeupDoesNotWakeWhenEmpty() throws Exception { // empty the handover assertNotNull(handover.pollNext()); - + // producing into an empty handover should work try { handover.produce(createTestRecords()); @@ -292,7 +292,7 @@ private static ConsumerRecords createTestRecords() { // ------------------------------------------------------------------------ - private static abstract class CheckedThread extends Thread { + private abstract static class CheckedThread extends Thread { private volatile Throwable error; @@ -317,7 +317,7 @@ public void sync() throws Exception { public void waitUntilThreadHoldsLock(long timeoutMillis) throws InterruptedException, TimeoutException { final long deadline = System.nanoTime() + timeoutMillis * 1_000_000; - + while (!isBlockedOrWaiting() && (System.nanoTime() < deadline)) { Thread.sleep(1); } diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/resources/log4j-test.properties b/flink-connectors/flink-connector-kafka-0.9/src/test/resources/log4j-test.properties index 4ac1773190f18..bc93a2dead19e 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/test/resources/log4j-test.properties +++ b/flink-connectors/flink-connector-kafka-0.9/src/test/resources/log4j-test.properties @@ -29,4 +29,4 @@ log4j.logger.org.apache.zookeeper=OFF, testlogger log4j.logger.state.change.logger=OFF, testlogger log4j.logger.kafka=OFF, testlogger -log4j.logger.org.apache.directory=OFF, testlogger \ No newline at end of file +log4j.logger.org.apache.directory=OFF, testlogger diff --git a/flink-connectors/flink-connector-kafka-base/pom.xml b/flink-connectors/flink-connector-kafka-base/pom.xml index 2cc94b0b766b6..fc0045e2ece14 100644 --- a/flink-connectors/flink-connector-kafka-base/pom.xml +++ b/flink-connectors/flink-connector-kafka-base/pom.xml @@ -106,7 +106,7 @@ under the License. - + com.101tec @@ -187,7 +187,6 @@ under the License. - @@ -233,5 +232,5 @@ under the License. - + diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java index 87bedce6b6574..18748d023422a 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java @@ -17,7 +17,6 @@ package org.apache.flink.streaming.connectors.kafka; -import org.apache.commons.collections.map.LinkedMap; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.OperatorStateStore; @@ -46,6 +45,8 @@ import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; import org.apache.flink.util.Preconditions; import org.apache.flink.util.SerializedValue; + +import org.apache.commons.collections.map.LinkedMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -60,13 +61,13 @@ /** * Base class of all Flink Kafka Consumer data sources. * This implements the common behavior across all Kafka versions. - * + * *

    The Kafka version specific behavior is defined mainly in the specific subclasses of the * {@link AbstractFetcher}. - * + * * @param The type of records produced by this data source */ -public abstract class FlinkKafkaConsumerBase extends RichParallelSourceFunction implements +public abstract class FlinkKafkaConsumerBase extends RichParallelSourceFunction implements CheckpointListener, ResultTypeQueryable, CheckpointedFunction, @@ -75,11 +76,11 @@ public abstract class FlinkKafkaConsumerBase extends RichParallelSourceFuncti private static final long serialVersionUID = -6272159445203409112L; protected static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaConsumerBase.class); - - /** The maximum number of pending non-committed checkpoints to track, to avoid memory leaks */ + + /** The maximum number of pending non-committed checkpoints to track, to avoid memory leaks. */ public static final int MAX_NUM_PENDING_CHECKPOINTS = 100; - /** Boolean configuration key to disable metrics tracking **/ + /** Boolean configuration key to disable metrics tracking. **/ public static final String KEY_DISABLE_METRICS = "flink.disable-metrics"; // ------------------------------------------------------------------------ @@ -87,20 +88,20 @@ public abstract class FlinkKafkaConsumerBase extends RichParallelSourceFuncti // ------------------------------------------------------------------------ private final List topics; - - /** The schema to convert between Kafka's byte messages, and Flink's objects */ + + /** The schema to convert between Kafka's byte messages, and Flink's objects. */ protected final KeyedDeserializationSchema deserializer; - /** The set of topic partitions that the source will read, with their initial offsets to start reading from */ + /** The set of topic partitions that the source will read, with their initial offsets to start reading from. */ private Map subscribedPartitionsToStartOffsets; - + /** Optional timestamp extractor / watermark generator that will be run per Kafka partition, * to exploit per-partition timestamp characteristics. * The assigner is kept in serialized form, to deserialize it into multiple copies */ private SerializedValue> periodicWatermarkAssigner; - + /** Optional timestamp extractor / watermark generator that will be run per Kafka partition, - * to exploit per-partition timestamp characteristics. + * to exploit per-partition timestamp characteristics. * The assigner is kept in serialized form, to deserialize it into multiple copies */ private SerializedValue> punctuatedWatermarkAssigner; @@ -119,26 +120,26 @@ public abstract class FlinkKafkaConsumerBase extends RichParallelSourceFuncti */ private OffsetCommitMode offsetCommitMode; - /** The startup mode for the consumer (default is {@link StartupMode#GROUP_OFFSETS}) */ + /** The startup mode for the consumer (default is {@link StartupMode#GROUP_OFFSETS}). */ private StartupMode startupMode = StartupMode.GROUP_OFFSETS; - /** Specific startup offsets; only relevant when startup mode is {@link StartupMode#SPECIFIC_OFFSETS} */ + /** Specific startup offsets; only relevant when startup mode is {@link StartupMode#SPECIFIC_OFFSETS}. */ protected Map specificStartupOffsets; // ------------------------------------------------------------------------ - // runtime state (used individually by each parallel subtask) + // runtime state (used individually by each parallel subtask) // ------------------------------------------------------------------------ - - /** Data for pending but uncommitted offsets */ + + /** Data for pending but uncommitted offsets. */ private final LinkedMap pendingOffsetsToCommit = new LinkedMap(); - /** The fetcher implements the connections to the Kafka brokers */ + /** The fetcher implements the connections to the Kafka brokers. */ private transient volatile AbstractFetcher kafkaFetcher; - - /** The offsets to restore to, if the consumer restores state from a checkpoint */ + + /** The offsets to restore to, if the consumer restores state from a checkpoint. */ private transient volatile HashMap restoredState; - - /** Flag indicating whether the consumer is still running **/ + + /** Flag indicating whether the consumer is still running. **/ private volatile boolean running = true; // ------------------------------------------------------------------------ @@ -158,30 +159,30 @@ public FlinkKafkaConsumerBase(List topics, KeyedDeserializationSchema // ------------------------------------------------------------------------ // Configuration // ------------------------------------------------------------------------ - + /** * Specifies an {@link AssignerWithPunctuatedWatermarks} to emit watermarks in a punctuated manner. * The watermark extractor will run per Kafka partition, watermarks will be merged across partitions * in the same way as in the Flink runtime, when streams are merged. - * + * *

    When a subtask of a FlinkKafkaConsumer source reads multiple Kafka partitions, * the streams from the partitions are unioned in a "first come first serve" fashion. Per-partition * characteristics are usually lost that way. For example, if the timestamps are strictly ascending * per Kafka partition, they will not be strictly ascending in the resulting Flink DataStream, if the * parallel source subtask reads more that one partition. - * + * *

    Running timestamp extractors / watermark generators directly inside the Kafka source, per Kafka * partition, allows users to let them exploit the per-partition characteristics. - * + * *

    Note: One can use either an {@link AssignerWithPunctuatedWatermarks} or an * {@link AssignerWithPeriodicWatermarks}, not both at the same time. - * + * * @param assigner The timestamp assigner / watermark generator to use. * @return The consumer object, to allow function chaining. */ public FlinkKafkaConsumerBase assignTimestampsAndWatermarks(AssignerWithPunctuatedWatermarks assigner) { checkNotNull(assigner); - + if (this.periodicWatermarkAssigner != null) { throw new IllegalStateException("A periodic watermark emitter has already been set."); } @@ -216,7 +217,7 @@ public FlinkKafkaConsumerBase assignTimestampsAndWatermarks(AssignerWithPunct */ public FlinkKafkaConsumerBase assignTimestampsAndWatermarks(AssignerWithPeriodicWatermarks assigner) { checkNotNull(assigner); - + if (this.punctuatedWatermarkAssigner != null) { throw new IllegalStateException("A punctuated watermark emitter has already been set."); } @@ -232,7 +233,7 @@ public FlinkKafkaConsumerBase assignTimestampsAndWatermarks(AssignerWithPerio /** * Specifies whether or not the consumer should commit offsets back to Kafka on checkpoints. * - * This setting will only have effect if checkpointing is enabled for the job. + *

    This setting will only have effect if checkpointing is enabled for the job. * If checkpointing isn't enabled, only the "auto.commit.enable" (for 0.8) / "enable.auto.commit" (for 0.9+) * property settings will be * @@ -247,7 +248,7 @@ public FlinkKafkaConsumerBase setCommitOffsetsOnCheckpoints(boolean commitOnC * Specifies the consumer to start reading from the earliest offset for all partitions. * This lets the consumer ignore any committed group offsets in Zookeeper / Kafka brokers. * - * This method does not effect where partitions are read from when the consumer is restored + *

    This method does not effect where partitions are read from when the consumer is restored * from a checkpoint or savepoint. When the consumer is restored from a checkpoint or * savepoint, only the offsets in the restored state will be used. * @@ -263,7 +264,7 @@ public FlinkKafkaConsumerBase setStartFromEarliest() { * Specifies the consumer to start reading from the latest offset for all partitions. * This lets the consumer ignore any committed group offsets in Zookeeper / Kafka brokers. * - * This method does not effect where partitions are read from when the consumer is restored + *

    This method does not effect where partitions are read from when the consumer is restored * from a checkpoint or savepoint. When the consumer is restored from a checkpoint or * savepoint, only the offsets in the restored state will be used. * @@ -281,7 +282,7 @@ public FlinkKafkaConsumerBase setStartFromLatest() { * properties. If no offset can be found for a partition, the behaviour in "auto.offset.reset" * set in the configuration properties will be used for the partition. * - * This method does not effect where partitions are read from when the consumer is restored + *

    This method does not effect where partitions are read from when the consumer is restored * from a checkpoint or savepoint. When the consumer is restored from a checkpoint or * savepoint, only the offsets in the restored state will be used. * @@ -298,16 +299,16 @@ public FlinkKafkaConsumerBase setStartFromGroupOffsets() { * The specified offset should be the offset of the next record that will be read from partitions. * This lets the consumer ignore any committed group offsets in Zookeeper / Kafka brokers. * - * If the provided map of offsets contains entries whose {@link KafkaTopicPartition} is not subscribed by the + *

    If the provided map of offsets contains entries whose {@link KafkaTopicPartition} is not subscribed by the * consumer, the entry will be ignored. If the consumer subscribes to a partition that does not exist in the provided * map of offsets, the consumer will fallback to the default group offset behaviour (see * {@link FlinkKafkaConsumerBase#setStartFromGroupOffsets()}) for that particular partition. * - * If the specified offset for a partition is invalid, or the behaviour for that partition is defaulted to group + *

    If the specified offset for a partition is invalid, or the behaviour for that partition is defaulted to group * offsets but still no group offset could be found for it, then the "auto.offset.reset" behaviour set in the * configuration properties will be used for the partition * - * This method does not effect where partitions are read from when the consumer is restored + *

    This method does not effect where partitions are read from when the consumer is restored * from a checkpoint or savepoint. When the consumer is restored from a checkpoint or * savepoint, only the offsets in the restored state will be used. * @@ -444,7 +445,7 @@ public void run(SourceContext sourceContext) throws Exception { if (!running) { return; } - + // (3) run the fetcher' main work method fetcher.runFetchLoop(); } @@ -476,7 +477,7 @@ public void run(SourceContext sourceContext) throws Exception { public void cancel() { // set ourselves as not running running = false; - + // abort the fetcher, if there is one if (kafkaFetcher != null) { kafkaFetcher.cancel(); @@ -494,7 +495,7 @@ public void close() throws Exception { super.close(); } } - + // ------------------------------------------------------------------------ // Checkpoint and restore // ------------------------------------------------------------------------ @@ -635,19 +636,19 @@ public void notifyCheckpointComplete(long checkpointId) throws Exception { // ------------------------------------------------------------------------ // Kafka Consumer specific methods // ------------------------------------------------------------------------ - + /** * Creates the fetcher that connect to the Kafka brokers, pulls data, deserialized the * data, and emits it into the data streams. - * + * * @param sourceContext The source context to emit data to. * @param subscribedPartitionsToStartOffsets The set of partitions that this subtask should handle, with their start offsets. * @param watermarksPeriodic Optional, a serialized timestamp extractor / periodic watermark generator. * @param watermarksPunctuated Optional, a serialized timestamp extractor / punctuated watermark generator. * @param runtimeContext The task's runtime context. - * + * * @return The instantiated fetcher - * + * * @throws Exception The method should forward exceptions */ protected abstract AbstractFetcher createFetcher( @@ -661,11 +662,11 @@ public void notifyCheckpointComplete(long checkpointId) throws Exception { protected abstract List getKafkaPartitions(List topics); protected abstract boolean getIsAutoCommitEnabled(); - + // ------------------------------------------------------------------------ - // ResultTypeQueryable methods + // ResultTypeQueryable methods // ------------------------------------------------------------------------ - + @Override public TypeInformation getProducedType() { return deserializer.getProducedType(); @@ -726,7 +727,7 @@ protected static void initializeSubscribedPartitionsToStartOffsets( /** * Logs the partition information in INFO level. - * + * * @param logger The logger to log to. * @param partitionInfos List of subscribed partitions */ @@ -743,11 +744,11 @@ protected static void logPartitionInfo(Logger logger, List } StringBuilder sb = new StringBuilder( "Consumer is going to read the following topics (with number of partitions): "); - + for (Map.Entry e : countPerTopic.entrySet()) { sb.append(e.getKey()).append(" (").append(e.getValue()).append("), "); } - + logger.info(sb.toString()); } diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBase.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBase.java index 46d7d47c41d08..76a2f845e8530 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBase.java @@ -17,14 +17,6 @@ package org.apache.flink.streaming.connectors.kafka; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Properties; - import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.java.ClosureCleaner; @@ -41,6 +33,7 @@ import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema; import org.apache.flink.util.NetUtils; + import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerConfig; @@ -53,13 +46,20 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static java.util.Objects.requireNonNull; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import static java.util.Objects.requireNonNull; /** * Flink Sink to produce data into a Kafka topic. * - * Please note that this producer provides at-least-once reliability guarantees when + *

    Please note that this producer provides at-least-once reliability guarantees when * checkpoints are enabled and setFlushOnCheckpoint(true) is set. * Otherwise, the producer doesn't provide any reliability guarantees. * @@ -98,7 +98,7 @@ public abstract class FlinkKafkaProducerBase extends RichSinkFunction im protected final FlinkKafkaPartitioner flinkKafkaPartitioner; /** - * Partitions of each topic + * Partitions of each topic. */ protected final Map topicPartitionsMap; @@ -114,16 +114,16 @@ public abstract class FlinkKafkaProducerBase extends RichSinkFunction im // -------------------------------- Runtime fields ------------------------------------------ - /** KafkaProducer instance */ + /** KafkaProducer instance. */ protected transient KafkaProducer producer; - /** The callback than handles error propagation or logging callbacks */ + /** The callback than handles error propagation or logging callbacks. */ protected transient Callback callback; - /** Errors encountered in the async producer are stored here */ + /** Errors encountered in the async producer are stored here. */ protected transient volatile Exception asyncException; - /** Lock for accessing the pending records */ + /** Lock for accessing the pending records. */ protected final SerializableObject pendingRecordsLock = new SerializableObject(); /** Number of unacknowledged records. */ @@ -196,9 +196,10 @@ public void setFlushOnCheckpoint(boolean flush) { } /** - * Used for testing only + * Used for testing only. */ - protected KafkaProducer getKafkaProducer(Properties props) { + @VisibleForTesting + protected KafkaProducer getKafkaProducer(Properties props) { return new KafkaProducer<>(props); } @@ -213,8 +214,8 @@ public void open(Configuration configuration) { RuntimeContext ctx = getRuntimeContext(); - if(null != flinkKafkaPartitioner) { - if(flinkKafkaPartitioner instanceof FlinkKafkaDelegatePartitioner) { + if (null != flinkKafkaPartitioner) { + if (flinkKafkaPartitioner instanceof FlinkKafkaDelegatePartitioner) { ((FlinkKafkaDelegatePartitioner) flinkKafkaPartitioner).setPartitions( getPartitionsByTopic(this.defaultTopicId, this.producer)); } @@ -239,7 +240,7 @@ public void open(Configuration configuration) { } } - if (flushOnCheckpoint && !((StreamingRuntimeContext)this.getRuntimeContext()).isCheckpointingEnabled()) { + if (flushOnCheckpoint && !((StreamingRuntimeContext) this.getRuntimeContext()).isCheckpointingEnabled()) { LOG.warn("Flushing on checkpoint is enabled, but checkpointing is not enabled. Disabling flushing."); flushOnCheckpoint = false; } @@ -287,7 +288,7 @@ public void invoke(IN next) throws Exception { } int[] partitions = this.topicPartitionsMap.get(targetTopic); - if(null == partitions) { + if (null == partitions) { partitions = getPartitionsByTopic(targetTopic, producer); this.topicPartitionsMap.put(targetTopic, partitions); } @@ -310,7 +311,6 @@ record = new ProducerRecord<>( producer.send(record, callback); } - @Override public void close() throws Exception { if (producer != null) { diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSource.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSource.java index b88fb83a4fa84..5c9a629561c80 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSource.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSource.java @@ -18,13 +18,6 @@ package org.apache.flink.streaming.connectors.kafka; -import java.util.List; -import java.util.Properties; -import org.apache.avro.Schema; -import org.apache.avro.specific.SpecificData; -import org.apache.avro.specific.SpecificRecord; -import org.apache.avro.specific.SpecificRecordBase; -import org.apache.avro.util.Utf8; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.AvroTypeInfo; @@ -35,6 +28,15 @@ import org.apache.flink.table.sources.StreamTableSource; import org.apache.flink.types.Row; +import org.apache.avro.Schema; +import org.apache.avro.specific.SpecificData; +import org.apache.avro.specific.SpecificRecord; +import org.apache.avro.specific.SpecificRecordBase; +import org.apache.avro.util.Utf8; + +import java.util.List; +import java.util.Properties; + /** * A version-agnostic Kafka Avro {@link StreamTableSource}. * diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSink.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSink.java index 41bb329d9d0ca..51fd9523397a3 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSink.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSink.java @@ -15,6 +15,7 @@ * 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.FlinkKafkaPartitioner; @@ -25,12 +26,12 @@ import java.util.Properties; /** - * Base class for {@link KafkaTableSink} that serializes data in JSON format + * Base class for {@link KafkaTableSink} that serializes data in JSON format. */ public abstract class KafkaJsonTableSink extends KafkaTableSink { - + /** - * Creates KafkaJsonTableSink + * Creates KafkaJsonTableSink. * * @param topic topic in Kafka to which table is written * @param properties properties to connect to Kafka diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java index 460f9484e9c89..1c8e0a09ed2b7 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java @@ -19,12 +19,12 @@ package org.apache.flink.streaming.connectors.kafka; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.table.sources.StreamTableSource; import org.apache.flink.streaming.util.serialization.DeserializationSchema; import org.apache.flink.streaming.util.serialization.JsonRowDeserializationSchema; +import org.apache.flink.table.sources.StreamTableSource; +import org.apache.flink.types.Row; import java.util.Properties; -import org.apache.flink.types.Row; /** * A version-agnostic Kafka JSON {@link StreamTableSource}. diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java index 1c38816c562dd..a94936c354574 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java @@ -15,15 +15,16 @@ * 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.api.common.typeinfo.TypeInformation; -import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; -import org.apache.flink.types.Row; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.table.sinks.AppendStreamTableSink; import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.streaming.util.serialization.SerializationSchema; +import org.apache.flink.table.sinks.AppendStreamTableSink; +import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; import java.util.Properties; @@ -44,8 +45,8 @@ public abstract class KafkaTableSink implements AppendStreamTableSink { protected TypeInformation[] fieldTypes; /** - * Creates KafkaTableSink - * + * Creates KafkaTableSink. + * * @param topic Kafka topic to write to. * @param properties Properties for the Kafka consumer. * @param partitioner Partitioner to select Kafka partition for each item diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java index 029aa453253b0..8969f904efda6 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java @@ -18,7 +18,6 @@ package org.apache.flink.streaming.connectors.kafka; -import java.util.Properties; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -27,6 +26,8 @@ import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; +import java.util.Properties; + /** * A version-agnostic Kafka {@link StreamTableSource}. * diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/config/OffsetCommitMode.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/config/OffsetCommitMode.java index 8bb75b4dd2f0b..0642e7ebc804e 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/config/OffsetCommitMode.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/config/OffsetCommitMode.java @@ -21,7 +21,7 @@ * The offset commit mode represents the behaviour of how offsets are externally committed * back to Kafka brokers / Zookeeper. * - * The exact value of this is determined at runtime in the consumer subtasks. + *

    The exact value of this is determined at runtime in the consumer subtasks. */ public enum OffsetCommitMode { diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/config/StartupMode.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/config/StartupMode.java index 8fc2fe0d3eff0..81c4138c3cc0b 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/config/StartupMode.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/config/StartupMode.java @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.flink.streaming.connectors.kafka.config; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionStateSentinel; @@ -23,13 +24,13 @@ */ public enum StartupMode { - /** Start from committed offsets in ZK / Kafka brokers of a specific consumer group (default) */ + /** Start from committed offsets in ZK / Kafka brokers of a specific consumer group (default). */ GROUP_OFFSETS(KafkaTopicPartitionStateSentinel.GROUP_OFFSET), - /** Start from the earliest offset possible */ + /** Start from the earliest offset possible. */ EARLIEST(KafkaTopicPartitionStateSentinel.EARLIEST_OFFSET), - /** Start from the latest offset */ + /** Start from the latest offset. */ LATEST(KafkaTopicPartitionStateSentinel.LATEST_OFFSET), /** @@ -39,7 +40,7 @@ public enum StartupMode { */ SPECIFIC_OFFSETS(Long.MIN_VALUE); - /** The sentinel offset value corresponding to this startup mode */ + /** The sentinel offset value corresponding to this startup mode. */ private long stateSentinel; StartupMode(long stateSentinel) { diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java index 0b311a9bafab9..cfd7c3b8b0913 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java @@ -38,43 +38,43 @@ /** * Base class for all fetchers, which implement the connections to Kafka brokers and * pull records from Kafka partitions. - * + * *

    This fetcher base class implements the logic around emitting records and tracking offsets, - * as well as around the optional timestamp assignment and watermark generation. - * + * as well as around the optional timestamp assignment and watermark generation. + * * @param The type of elements deserialized from Kafka's byte records, and emitted into * the Flink data streams. * @param The type of topic/partition identifier used by Kafka in the specific version. */ public abstract class AbstractFetcher { - + protected static final int NO_TIMESTAMPS_WATERMARKS = 0; protected static final int PERIODIC_WATERMARKS = 1; protected static final int PUNCTUATED_WATERMARKS = 2; - + // ------------------------------------------------------------------------ - - /** The source context to emit records and watermarks to */ + + /** The source context to emit records and watermarks to. */ protected final SourceContext sourceContext; /** The lock that guarantees that record emission and state updates are atomic, - * from the view of taking a checkpoint */ + * from the view of taking a checkpoint. */ protected final Object checkpointLock; - /** All partitions (and their state) that this fetcher is subscribed to */ + /** All partitions (and their state) that this fetcher is subscribed to. */ private final KafkaTopicPartitionState[] subscribedPartitionStates; - /** The mode describing whether the fetcher also generates timestamps and watermarks */ + /** The mode describing whether the fetcher also generates timestamps and watermarks. */ protected final int timestampWatermarkMode; - /** Flag whether to register metrics for the fetcher */ + /** Flag whether to register metrics for the fetcher. */ protected final boolean useMetrics; - /** Only relevant for punctuated watermarks: The current cross partition watermark */ + /** Only relevant for punctuated watermarks: The current cross partition watermark. */ private volatile long maxWatermarkSoFar = Long.MIN_VALUE; // ------------------------------------------------------------------------ - + protected AbstractFetcher( SourceContext sourceContext, Map assignedPartitionsWithInitialOffsets, @@ -83,14 +83,13 @@ protected AbstractFetcher( ProcessingTimeService processingTimeProvider, long autoWatermarkInterval, ClassLoader userCodeClassLoader, - boolean useMetrics) throws Exception - { + boolean useMetrics) throws Exception { this.sourceContext = checkNotNull(sourceContext); this.checkpointLock = sourceContext.getCheckpointLock(); this.useMetrics = useMetrics; - + // figure out what we watermark mode we will be using - + if (watermarksPeriodic == null) { if (watermarksPunctuated == null) { // simple case, no watermarks involved @@ -106,7 +105,7 @@ protected AbstractFetcher( } } - // create our partition state according to the timestamp/watermark mode + // create our partition state according to the timestamp/watermark mode this.subscribedPartitionStates = initializeSubscribedPartitionStates( assignedPartitionsWithInitialOffsets, timestampWatermarkMode, @@ -119,13 +118,13 @@ protected AbstractFetcher( throw new IllegalArgumentException("The fetcher was assigned partitions with undefined initial offsets."); } } - + // if we have periodic watermarks, kick off the interval scheduler if (timestampWatermarkMode == PERIODIC_WATERMARKS) { - KafkaTopicPartitionStateWithPeriodicWatermarks[] parts = + KafkaTopicPartitionStateWithPeriodicWatermarks[] parts = (KafkaTopicPartitionStateWithPeriodicWatermarks[]) subscribedPartitionStates; - - PeriodicWatermarkEmitter periodicEmitter = + + PeriodicWatermarkEmitter periodicEmitter = new PeriodicWatermarkEmitter(parts, sourceContext, processingTimeProvider, autoWatermarkInterval); periodicEmitter.start(); } @@ -149,17 +148,17 @@ protected final KafkaTopicPartitionState[] subscribedPartitionStates() { // ------------------------------------------------------------------------ public abstract void runFetchLoop() throws Exception; - + public abstract void cancel(); // ------------------------------------------------------------------------ // Kafka version specifics // ------------------------------------------------------------------------ - + /** * Creates the Kafka version specific representation of the given * topic partition. - * + * * @param partition The Flink representation of the Kafka topic partition. * @return The specific Kafka representation of the Kafka topic partition. */ @@ -170,7 +169,7 @@ protected final KafkaTopicPartitionState[] subscribedPartitionStates() { * older Kafka versions). This method is only ever called when the offset commit mode of * the consumer is {@link OffsetCommitMode#ON_CHECKPOINTS}. * - * The given offsets are the internal checkpointed offsets, representing + *

    The given offsets are the internal checkpointed offsets, representing * the last processed record of each partition. Version-specific implementations of this method * need to hold the contract that the given offsets must be incremented by 1 before * committing them, so that committed offsets to Kafka represent "the next record to process". @@ -179,16 +178,16 @@ protected final KafkaTopicPartitionState[] subscribedPartitionStates() { * @throws Exception This method forwards exceptions. */ public abstract void commitInternalOffsetsToKafka(Map offsets) throws Exception; - + // ------------------------------------------------------------------------ // snapshot and restore the state // ------------------------------------------------------------------------ /** * Takes a snapshot of the partition offsets. - * + * *

    Important: This method mus be called under the checkpoint lock. - * + * * @return A map from partition to current offset. */ public HashMap snapshotCurrentState() { @@ -208,10 +207,10 @@ public HashMap snapshotCurrentState() { /** * Emits a record without attaching an existing timestamp to it. - * + * *

    Implementation Note: This method is kept brief to be JIT inlining friendly. * That makes the fast path efficient, the extended paths are called as separate methods. - * + * * @param record The record to emit * @param partitionState The state of the Kafka partition from which the record was fetched * @param offset The offset of the record @@ -282,8 +281,7 @@ protected void emitRecordWithTimestamp( * also a periodic watermark generator. */ protected void emitRecordWithTimestampAndPeriodicWatermark( - T record, KafkaTopicPartitionState partitionState, long offset, long kafkaEventTimestamp) - { + T record, KafkaTopicPartitionState partitionState, long offset, long kafkaEventTimestamp) { @SuppressWarnings("unchecked") final KafkaTopicPartitionStateWithPeriodicWatermarks withWatermarksState = (KafkaTopicPartitionStateWithPeriodicWatermarks) partitionState; @@ -298,7 +296,7 @@ protected void emitRecordWithTimestampAndPeriodicWatermark( } // emit the record with timestamp, using the usual checkpoint lock to guarantee - // atomicity of record emission and offset state update + // atomicity of record emission and offset state update synchronized (checkpointLock) { sourceContext.collectWithTimestamp(record, timestamp); partitionState.setOffset(offset); @@ -310,8 +308,7 @@ protected void emitRecordWithTimestampAndPeriodicWatermark( * also a punctuated watermark generator. */ protected void emitRecordWithTimestampAndPunctuatedWatermark( - T record, KafkaTopicPartitionState partitionState, long offset, long kafkaEventTimestamp) - { + T record, KafkaTopicPartitionState partitionState, long offset, long kafkaEventTimestamp) { @SuppressWarnings("unchecked") final KafkaTopicPartitionStateWithPunctuatedWatermarks withWatermarksState = (KafkaTopicPartitionStateWithPunctuatedWatermarks) partitionState; @@ -322,7 +319,7 @@ protected void emitRecordWithTimestampAndPunctuatedWatermark( final Watermark newWatermark = withWatermarksState.checkAndGetNewWatermark(record, timestamp); // emit the record with timestamp, using the usual checkpoint lock to guarantee - // atomicity of record emission and offset state update + // atomicity of record emission and offset state update synchronized (checkpointLock) { sourceContext.collectWithTimestamp(record, timestamp); partitionState.setOffset(offset); @@ -346,7 +343,7 @@ private void updateMinPunctuatedWatermark(Watermark nextWatermark) { @SuppressWarnings("unchecked") final KafkaTopicPartitionStateWithPunctuatedWatermarks withWatermarksState = (KafkaTopicPartitionStateWithPunctuatedWatermarks) state; - + newMin = Math.min(newMin, withWatermarksState.getCurrentPartitionWatermark()); } @@ -375,11 +372,9 @@ private KafkaTopicPartitionState[] initializeSubscribedPartitionStates( int timestampWatermarkMode, SerializedValue> watermarksPeriodic, SerializedValue> watermarksPunctuated, - ClassLoader userCodeClassLoader) - throws IOException, ClassNotFoundException - { + ClassLoader userCodeClassLoader) throws IOException, ClassNotFoundException { switch (timestampWatermarkMode) { - + case NO_TIMESTAMPS_WATERMARKS: { @SuppressWarnings("unchecked") KafkaTopicPartitionState[] partitions = @@ -410,7 +405,7 @@ private KafkaTopicPartitionState[] initializeSubscribedPartitionStates( AssignerWithPeriodicWatermarks assignerInstance = watermarksPeriodic.deserializeValue(userCodeClassLoader); - + partitions[pos] = new KafkaTopicPartitionStateWithPeriodicWatermarks<>( partition.getKey(), kafkaHandle, assignerInstance); partitions[pos].setOffset(partition.getValue()); @@ -452,7 +447,7 @@ private KafkaTopicPartitionState[] initializeSubscribedPartitionStates( // ------------------------- Metrics ---------------------------------- /** - * Add current and committed offsets to metric group + * Add current and committed offsets to metric group. * * @param metricGroup The metric group to use */ @@ -467,7 +462,7 @@ protected void addOffsetStateGauge(MetricGroup metricGroup) { } /** - * Gauge types + * Gauge types. */ private enum OffsetGaugeType { CURRENT_OFFSET, @@ -500,7 +495,7 @@ public Long getValue() { } } // ------------------------------------------------------------------------ - + /** * The periodic watermark emitter. In its given interval, it checks all partitions for * the current event time watermark, and possibly emits the next watermark. @@ -508,23 +503,22 @@ public Long getValue() { private static class PeriodicWatermarkEmitter implements ProcessingTimeCallback { private final KafkaTopicPartitionStateWithPeriodicWatermarks[] allPartitions; - + private final SourceContext emitter; - + private final ProcessingTimeService timerService; private final long interval; - + private long lastWatermarkTimestamp; - + //------------------------------------------------- PeriodicWatermarkEmitter( KafkaTopicPartitionStateWithPeriodicWatermarks[] allPartitions, SourceContext emitter, ProcessingTimeService timerService, - long autoWatermarkInterval) - { + long autoWatermarkInterval) { this.allPartitions = checkNotNull(allPartitions); this.emitter = checkNotNull(emitter); this.timerService = checkNotNull(timerService); @@ -533,17 +527,17 @@ private static class PeriodicWatermarkEmitter implements ProcessingTimeCallback } //------------------------------------------------- - + public void start() { timerService.registerTimer(timerService.getCurrentProcessingTime() + interval, this); } - + @Override public void onProcessingTime(long timestamp) throws Exception { long minAcrossAll = Long.MAX_VALUE; for (KafkaTopicPartitionStateWithPeriodicWatermarks state : allPartitions) { - + // we access the current watermark for the periodic assigners under the state // lock, to prevent concurrent modification to any internal variables final long curr; @@ -551,16 +545,16 @@ public void onProcessingTime(long timestamp) throws Exception { synchronized (state) { curr = state.getCurrentWatermarkTimestamp(); } - + minAcrossAll = Math.min(minAcrossAll, curr); } - + // emit next watermark, if there is one if (minAcrossAll > lastWatermarkTimestamp) { lastWatermarkTimestamp = minAcrossAll; emitter.emitWatermark(new Watermark(minAcrossAll)); } - + // schedule the next watermark timerService.registerTimer(timerService.getCurrentProcessingTime() + interval, this); } diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ExceptionProxy.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ExceptionProxy.java index c736493effd2d..06cdf2c681e1e 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ExceptionProxy.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ExceptionProxy.java @@ -19,24 +19,25 @@ package org.apache.flink.streaming.connectors.kafka.internals; import javax.annotation.Nullable; + import java.util.concurrent.atomic.AtomicReference; /** * A proxy that communicates exceptions between threads. Typically used if an exception * from a spawned thread needs to be recognized by the "parent" (spawner) thread. - * + * *

    The spawned thread would set the exception via {@link #reportError(Throwable)}. * The parent would check (at certain points) for exceptions via {@link #checkAndThrowException()}. * Optionally, the parent can pass itself in the constructor to be interrupted as soon as * an exception occurs. - * + * *

      * {@code
    - * 
    + *
      * final ExceptionProxy errorProxy = new ExceptionProxy(Thread.currentThread());
    - * 
    + *
      * Thread subThread = new Thread() {
    - * 
    + *
      *     public void run() {
      *         try {
      *             doSomething();
    @@ -48,13 +49,13 @@
      *     }
      * };
      * subThread.start();
    - * 
    + *
      * doSomethingElse();
      * errorProxy.checkAndThrowException();
    - * 
    + *
      * doSomethingMore();
      * errorProxy.checkAndThrowException();
    - * 
    + *
      * try {
      *     subThread.join();
      * } catch (InterruptedException e) {
    @@ -66,33 +67,33 @@
      * 
    */ public class ExceptionProxy { - - /** The thread that should be interrupted when an exception occurs */ + + /** The thread that should be interrupted when an exception occurs. */ private final Thread toInterrupt; - - /** The exception to throw */ + + /** The exception to throw. */ private final AtomicReference exception; /** * Creates an exception proxy that interrupts the given thread upon * report of an exception. The thread to interrupt may be null. - * + * * @param toInterrupt The thread to interrupt upon an exception. May be null. */ public ExceptionProxy(@Nullable Thread toInterrupt) { this.toInterrupt = toInterrupt; this.exception = new AtomicReference<>(); } - + // ------------------------------------------------------------------------ - + /** * Sets the exception and interrupts the target thread, * if no other exception has occurred so far. - * + * *

    The exception is only set (and the interruption is only triggered), * if no other exception was set before. - * + * * @param t The exception that occurred */ public void reportError(Throwable t) { @@ -105,7 +106,7 @@ public void reportError(Throwable t) { /** * Checks whether an exception has been set via {@link #reportError(Throwable)}. * If yes, that exception if re-thrown by this method. - * + * * @throws Exception This method re-throws the exception, if set. */ public void checkAndThrowException() throws Exception { diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java index c68fe28a9b172..f3645e3dd3a03 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java @@ -27,16 +27,16 @@ /** * Flink's description of a partition in a Kafka topic. * Serializable, and common across all Kafka consumer subclasses (0.8, 0.9, ...) - * + * *

    Note: This class must not change in its structure, because it would change the * serialization format and make previous savepoints unreadable. */ public final class KafkaTopicPartition implements Serializable { /** THIS SERIAL VERSION UID MUST NOT CHANGE, BECAUSE IT WOULD BREAK - * READING OLD SERIALIZED INSTANCES FROM SAVEPOINTS */ + * READING OLD SERIALIZED INSTANCES FROM SAVEPOINTS. */ private static final long serialVersionUID = 722083576322742325L; - + // ------------------------------------------------------------------------ private final String topic; @@ -50,7 +50,7 @@ public KafkaTopicPartition(String topic, int partition) { } // ------------------------------------------------------------------------ - + public String getTopic() { return topic; } @@ -60,7 +60,7 @@ public int getPartition() { } // ------------------------------------------------------------------------ - + @Override public String toString() { return "KafkaTopicPartition{" + @@ -87,7 +87,7 @@ else if (o instanceof KafkaTopicPartition) { public int hashCode() { return cachedHash; } - + // ------------------------------------------------------------------------ // Utilities // ------------------------------------------------------------------------ @@ -109,10 +109,9 @@ public static String toString(List partitions) { return sb.toString(); } - public static List dropLeaderData(List partitionInfos) { List ret = new ArrayList<>(partitionInfos.size()); - for(KafkaTopicPartitionLeader ktpl: partitionInfos) { + for (KafkaTopicPartitionLeader ktpl: partitionInfos) { ret.add(ktpl.getTopicPartition()); } return ret; diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionState.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionState.java index adfbf7952d2c3..78ab612231c20 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionState.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionState.java @@ -20,31 +20,31 @@ /** * The state that the Flink Kafka Consumer holds for each Kafka partition. * Includes the Kafka descriptor for partitions. - * + * *

    This class describes the most basic state (only the offset), subclasses * define more elaborate state, containing current watermarks and timestamp * extractors. - * + * * @param The type of the Kafka partition descriptor, which varies across Kafka versions. */ public class KafkaTopicPartitionState { - + // ------------------------------------------------------------------------ - /** The Flink description of a Kafka partition */ + /** The Flink description of a Kafka partition. */ private final KafkaTopicPartition partition; - /** The Kafka description of a Kafka partition (varies across different Kafka versions) */ + /** The Kafka description of a Kafka partition (varies across different Kafka versions). */ private final KPH kafkaPartitionHandle; - - /** The offset within the Kafka partition that we already processed */ + + /** The offset within the Kafka partition that we already processed. */ private volatile long offset; - /** The offset of the Kafka partition that has been committed */ + /** The offset of the Kafka partition that has been committed. */ private volatile long committedOffset; // ------------------------------------------------------------------------ - + public KafkaTopicPartitionState(KafkaTopicPartition partition, KPH kafkaPartitionHandle) { this.partition = partition; this.kafkaPartitionHandle = kafkaPartitionHandle; @@ -103,7 +103,6 @@ public final long getCommittedOffset() { return committedOffset; } - // ------------------------------------------------------------------------ @Override diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateSentinel.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateSentinel.java index 153a32627072f..c218618c8bd74 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateSentinel.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateSentinel.java @@ -20,7 +20,7 @@ /** * Magic values used to represent special offset states before partitions are actually read. * - * The values are all negative. Negative offsets are not used by Kafka (invalid), so we + *

    The values are all negative. Negative offsets are not used by Kafka (invalid), so we * pick a number that is probably (hopefully) not used by Kafka as a magic number for anything else. */ public class KafkaTopicPartitionStateSentinel { @@ -31,7 +31,7 @@ public class KafkaTopicPartitionStateSentinel { /** * Magic number that defines the partition should start from the earliest offset. * - * This is used as a placeholder so that the actual earliest offset can be evaluated lazily + *

    This is used as a placeholder so that the actual earliest offset can be evaluated lazily * when the partition will actually start to be read by the consumer. */ public static final long EARLIEST_OFFSET = -915623761775L; @@ -39,7 +39,7 @@ public class KafkaTopicPartitionStateSentinel { /** * Magic number that defines the partition should start from the latest offset. * - * This is used as a placeholder so that the actual latest offset can be evaluated lazily + *

    This is used as a placeholder so that the actual latest offset can be evaluated lazily * when the partition will actually start to be read by the consumer. */ public static final long LATEST_OFFSET = -915623761774L; @@ -47,7 +47,7 @@ public class KafkaTopicPartitionStateSentinel { /** * Magic number that defines the partition should start from its committed group offset in Kafka. * - * This is used as a placeholder so that the actual committed group offset can be evaluated lazily + *

    This is used as a placeholder so that the actual committed group offset can be evaluated lazily * when the partition will actually start to be read by the consumer. */ public static final long GROUP_OFFSET = -915623761773L; diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithPeriodicWatermarks.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithPeriodicWatermarks.java index efdc73fc0f390..5116e9f021c57 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithPeriodicWatermarks.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithPeriodicWatermarks.java @@ -23,36 +23,35 @@ /** * A special version of the per-kafka-partition-state that additionally holds * a periodic watermark generator (and timestamp extractor) per partition. - * + * * @param The type of records handled by the watermark generator * @param The type of the Kafka partition descriptor, which varies across Kafka versions. */ public final class KafkaTopicPartitionStateWithPeriodicWatermarks extends KafkaTopicPartitionState { - - /** The timestamp assigner and watermark generator for the partition */ + + /** The timestamp assigner and watermark generator for the partition. */ private final AssignerWithPeriodicWatermarks timestampsAndWatermarks; - - /** The last watermark timestamp generated by this partition */ + + /** The last watermark timestamp generated by this partition. */ private long partitionWatermark; // ------------------------------------------------------------------------ - + public KafkaTopicPartitionStateWithPeriodicWatermarks( KafkaTopicPartition partition, KPH kafkaPartitionHandle, - AssignerWithPeriodicWatermarks timestampsAndWatermarks) - { + AssignerWithPeriodicWatermarks timestampsAndWatermarks) { super(partition, kafkaPartitionHandle); - + this.timestampsAndWatermarks = timestampsAndWatermarks; this.partitionWatermark = Long.MIN_VALUE; } // ------------------------------------------------------------------------ - + public long getTimestampForRecord(T record, long kafkaEventTimestamp) { return timestampsAndWatermarks.extractTimestamp(record, kafkaEventTimestamp); } - + public long getCurrentWatermarkTimestamp() { Watermark wm = timestampsAndWatermarks.getCurrentWatermark(); if (wm != null) { @@ -62,7 +61,7 @@ public long getCurrentWatermarkTimestamp() { } // ------------------------------------------------------------------------ - + @Override public String toString() { return "KafkaTopicPartitionStateWithPeriodicWatermarks: partition=" + getKafkaTopicPartition() diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithPunctuatedWatermarks.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithPunctuatedWatermarks.java index edf40ce671e26..f4a80a4df17a9 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithPunctuatedWatermarks.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithPunctuatedWatermarks.java @@ -25,35 +25,34 @@ /** * A special version of the per-kafka-partition-state that additionally holds * a periodic watermark generator (and timestamp extractor) per partition. - * + * *

    This class is not thread safe, but it gives volatile access to the current * partition watermark ({@link #getCurrentPartitionWatermark()}). - * + * * @param The type of records handled by the watermark generator * @param The type of the Kafka partition descriptor, which varies across Kafka versions */ public final class KafkaTopicPartitionStateWithPunctuatedWatermarks extends KafkaTopicPartitionState { - - /** The timestamp assigner and watermark generator for the partition */ + + /** The timestamp assigner and watermark generator for the partition. */ private final AssignerWithPunctuatedWatermarks timestampsAndWatermarks; - - /** The last watermark timestamp generated by this partition */ + + /** The last watermark timestamp generated by this partition. */ private volatile long partitionWatermark; // ------------------------------------------------------------------------ - + public KafkaTopicPartitionStateWithPunctuatedWatermarks( KafkaTopicPartition partition, KPH kafkaPartitionHandle, - AssignerWithPunctuatedWatermarks timestampsAndWatermarks) - { + AssignerWithPunctuatedWatermarks timestampsAndWatermarks) { super(partition, kafkaPartitionHandle); - + this.timestampsAndWatermarks = timestampsAndWatermarks; this.partitionWatermark = Long.MIN_VALUE; } // ------------------------------------------------------------------------ - + public long getTimestampForRecord(T record, long kafkaEventTimestamp) { return timestampsAndWatermarks.extractTimestamp(record, kafkaEventTimestamp); } @@ -69,13 +68,13 @@ public Watermark checkAndGetNewWatermark(T record, long timestamp) { return null; } } - + public long getCurrentPartitionWatermark() { return partitionWatermark; } // ------------------------------------------------------------------------ - + @Override public String toString() { return "KafkaTopicPartitionStateWithPunctuatedWatermarks: partition=" + getKafkaTopicPartition() diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkFixedPartitioner.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkFixedPartitioner.java index e47c6674697ce..6ed3717d8be59 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkFixedPartitioner.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkFixedPartitioner.java @@ -15,6 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.flink.streaming.connectors.kafka.partitioner; import org.apache.flink.util.Preconditions; @@ -22,9 +23,9 @@ /** * A partitioner ensuring that each internal Flink partition ends up in one Kafka partition. * - * Note, one Kafka partition can contain multiple Flink partitions. + *

    Note, one Kafka partition can contain multiple Flink partitions. * - * Cases: + *

    Cases: * # More Flink partitions than kafka partitions *

      * 		Flink Sinks:		Kafka Partitions
    @@ -35,7 +36,7 @@
      * 
    * Some (or all) kafka partitions contain the output of more than one flink partition * - *# Fewer Flink partitions than Kafka + *

    Fewer Flink partitions than Kafka *

      * 		Flink Sinks:		Kafka Partitions
      * 			1	---------------->	1
    @@ -45,9 +46,9 @@
      * 										5
      * 
    * - * Not all Kafka partitions contain data - * To avoid such an unbalanced partitioning, use a round-robin kafka partitioner (note that this will - * cause a lot of network connections between all the Flink instances and all the Kafka brokers). + *

    Not all Kafka partitions contain data + * To avoid such an unbalanced partitioning, use a round-robin kafka partitioner (note that this will + * cause a lot of network connections between all the Flink instances and all the Kafka brokers). */ public class FlinkFixedPartitioner extends FlinkKafkaPartitioner { @@ -60,13 +61,13 @@ public void open(int parallelInstanceId, int parallelInstances) { this.parallelInstanceId = parallelInstanceId; } - + @Override public int partition(T record, byte[] key, byte[] value, String targetTopic, int[] partitions) { Preconditions.checkArgument( partitions != null && partitions.length > 0, "Partitions of the target topic is empty."); - + return partitions[parallelInstanceId % partitions.length]; } } diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkKafkaDelegatePartitioner.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkKafkaDelegatePartitioner.java index b7b414383a7cd..168e76b8e6941 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkKafkaDelegatePartitioner.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkKafkaDelegatePartitioner.java @@ -15,6 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.flink.streaming.connectors.kafka.partitioner; /** diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/AvroRowDeserializationSchema.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/AvroRowDeserializationSchema.java index 37241f56827e0..0713738d5d8ca 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/AvroRowDeserializationSchema.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/AvroRowDeserializationSchema.java @@ -14,11 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.flink.streaming.util.serialization; -import java.io.ByteArrayInputStream; -import java.io.IOException; -import java.util.List; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; + import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.avro.io.DatumReader; @@ -28,13 +29,15 @@ import org.apache.avro.specific.SpecificDatumReader; import org.apache.avro.specific.SpecificRecord; import org.apache.avro.util.Utf8; -import org.apache.flink.types.Row; -import org.apache.flink.util.Preconditions; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.util.List; /** * Deserialization schema from Avro bytes over {@link SpecificRecord} to {@link Row}. * - * Deserializes the byte[] messages into (nested) Flink Rows. + *

    Deserializes the byte[] messages into (nested) Flink Rows. * * {@link Utf8} is converted to regular Java Strings. */ @@ -56,7 +59,7 @@ public class AvroRowDeserializationSchema extends AbstractDeserializationSchema< private final MutableByteArrayInputStream inputStream; /** - * Avro decoder that decodes binary data + * Avro decoder that decodes binary data. */ private final Decoder decoder; @@ -133,9 +136,7 @@ private static Object convertToRow(Schema schema, Object recordObj) { * InputStream instance, copying message to process, and creation of Decoder on every new message. */ private static final class MutableByteArrayInputStream extends ByteArrayInputStream { - /** - * Create MutableByteArrayInputStream - */ + public MutableByteArrayInputStream() { super(new byte[0]); } diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/AvroRowSerializationSchema.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/AvroRowSerializationSchema.java index 8388ab5ab1418..450c78feefdd4 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/AvroRowSerializationSchema.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/AvroRowSerializationSchema.java @@ -15,11 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.flink.streaming.util.serialization; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.util.List; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; + import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; @@ -30,8 +31,10 @@ import org.apache.avro.specific.SpecificDatumWriter; import org.apache.avro.specific.SpecificRecord; import org.apache.avro.util.Utf8; -import org.apache.flink.types.Row; -import org.apache.flink.util.Preconditions; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.List; /** * Serialization schema that serializes {@link Row} over {@link SpecificRecord} into a Avro bytes. diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JSONDeserializationSchema.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JSONDeserializationSchema.java index d17005835fb58..095e96434e774 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JSONDeserializationSchema.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JSONDeserializationSchema.java @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.flink.streaming.util.serialization; import com.fasterxml.jackson.databind.ObjectMapper; @@ -21,11 +22,10 @@ import java.io.IOException; - /** * DeserializationSchema that deserializes a JSON String into an ObjectNode. - *

    - * Fields can be accessed by calling objectNode.get(<name>).as(<type>) + * + *

    Fields can be accessed by calling objectNode.get(<name>).as(<type>) */ public class JSONDeserializationSchema extends AbstractDeserializationSchema { private ObjectMapper mapper; diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JSONKeyValueDeserializationSchema.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JSONKeyValueDeserializationSchema.java index 261a111e146f8..f75df0ce99e5d 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JSONKeyValueDeserializationSchema.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JSONKeyValueDeserializationSchema.java @@ -14,12 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.flink.streaming.util.serialization; +import org.apache.flink.api.common.typeinfo.TypeInformation; + import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.ObjectNode; -import org.apache.flink.api.common.typeinfo.TypeInformation; import java.io.IOException; @@ -27,12 +29,12 @@ /** * DeserializationSchema that deserializes a JSON String into an ObjectNode. - *

    - * Key fields can be accessed by calling objectNode.get("key").get(<name>).as(<type>) - *

    - * Value fields can be accessed by calling objectNode.get("value").get(<name>).as(<type>) - *

    - * Metadata fields can be accessed by calling objectNode.get("metadata").get(<name>).as(<type>) and include + * + *

    Key fields can be accessed by calling objectNode.get("key").get(<name>).as(<type>) + * + *

    Value fields can be accessed by calling objectNode.get("value").get(<name>).as(<type>) + * + *

    Metadata fields can be accessed by calling objectNode.get("metadata").get(<name>).as(<type>) and include * the "offset" (long), "topic" (String) and "partition" (int). */ public class JSONKeyValueDeserializationSchema implements KeyedDeserializationSchema { diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowDeserializationSchema.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowDeserializationSchema.java index be201fa3f9585..f335c309cad4d 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowDeserializationSchema.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowDeserializationSchema.java @@ -18,14 +18,16 @@ package org.apache.flink.streaming.util.serialization; -import com.fasterxml.jackson.databind.JsonNode; -import com.fasterxml.jackson.databind.ObjectMapper; -import java.io.IOException; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; + +import java.io.IOException; + /** * Deserialization schema from JSON to {@link Row}. * diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowSerializationSchema.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowSerializationSchema.java index 1998aa6b7f568..a3fa379017857 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowSerializationSchema.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowSerializationSchema.java @@ -14,14 +14,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.flink.streaming.util.serialization; -import com.fasterxml.jackson.databind.JsonNode; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.node.ObjectNode; import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ObjectNode; /** * Serialization schema that serializes an object into a JSON bytes. @@ -33,9 +34,9 @@ * {@link JsonRowDeserializationSchema}. */ public class JsonRowSerializationSchema implements SerializationSchema { - /** Fields names in the input Row object */ + /** Fields names in the input Row object. */ private final String[] fieldNames; - /** Object mapper that is used to create output JSON objects */ + /** Object mapper that is used to create output JSON objects. */ private static ObjectMapper mapper = new ObjectMapper(); /** diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchema.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchema.java index b5a33bcac9d7d..234a96d435f73 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchema.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchema.java @@ -26,7 +26,7 @@ * The deserialization schema describes how to turn the byte key / value messages delivered by certain * data sources (for example Apache Kafka) into data types (Java/Scala objects) that are * processed by Flink. - * + * * @param The type created by the keyed deserialization schema. */ public interface KeyedDeserializationSchema extends Serializable, ResultTypeQueryable { @@ -46,7 +46,7 @@ public interface KeyedDeserializationSchema extends Serializable, ResultTypeQ /** * Method to decide whether the element signals the end of the stream. If * true is returned the element won't be emitted. - * + * * @param nextElement The element to test for the end-of-stream signal. * * @return True, if the element signals end of stream, false otherwise. diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchemaWrapper.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchemaWrapper.java index 4b9dba2c4e031..e128abab3fb6a 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchemaWrapper.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchemaWrapper.java @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.flink.streaming.util.serialization; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -22,7 +23,7 @@ /** * A simple wrapper for using the DeserializationSchema with the KeyedDeserializationSchema - * interface + * interface. * @param The type created by the deserialization schema. */ public class KeyedDeserializationSchemaWrapper implements KeyedDeserializationSchema { @@ -34,6 +35,7 @@ public class KeyedDeserializationSchemaWrapper implements KeyedDeserializatio public KeyedDeserializationSchemaWrapper(DeserializationSchema deserializationSchema) { this.deserializationSchema = deserializationSchema; } + @Override public T deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset) throws IOException { return deserializationSchema.deserialize(message); diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchema.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchema.java index 701281edb87e0..12bcab9d4025c 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchema.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchema.java @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.flink.streaming.util.serialization; import java.io.Serializable; @@ -22,7 +23,7 @@ * The serialization schema describes how to turn a data object into a different serialized * representation. Most data sinks (for example Apache Kafka) require the data to be handed * to them in a specific format (for example as byte strings). - * + * * @param The type to be serialized. */ public interface KeyedSerializationSchema extends Serializable { @@ -36,17 +37,16 @@ public interface KeyedSerializationSchema extends Serializable { */ byte[] serializeKey(T element); - /** - * Serializes the value of the incoming element to a byte array - * + * Serializes the value of the incoming element to a byte array. + * * @param element The incoming element to be serialized * @return the value of the element as a byte array */ byte[] serializeValue(T element); /** - * Optional method to determine the target topic for the element + * Optional method to determine the target topic for the element. * * @param element Incoming element to determine the target topic from * @return null or the target topic diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchemaWrapper.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchemaWrapper.java index 1b3e4860fbfac..0a181d1e1a2db 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchemaWrapper.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchemaWrapper.java @@ -14,11 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.flink.streaming.util.serialization; /** * A simple wrapper for using the SerializationSchema with the KeyedDeserializationSchema - * interface + * interface. * @param The type to serialize */ public class KeyedSerializationSchemaWrapper implements KeyedSerializationSchema { diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationKeyValueSerializationSchema.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationKeyValueSerializationSchema.java index 51bc8d1b5d55d..3e0cdb52d54b6 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationKeyValueSerializationSchema.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationKeyValueSerializationSchema.java @@ -32,30 +32,29 @@ /** * A serialization and deserialization schema for Key Value Pairs that uses Flink's serialization stack to * transform typed from and to byte arrays. - * + * * @param The key type to be serialized. * @param The value type to be serialized. */ -public class TypeInformationKeyValueSerializationSchema implements KeyedDeserializationSchema>, KeyedSerializationSchema> { +public class TypeInformationKeyValueSerializationSchema implements KeyedDeserializationSchema>, KeyedSerializationSchema> { private static final long serialVersionUID = -5359448468131559102L; - /** The serializer for the key */ + /** The serializer for the key. */ private final TypeSerializer keySerializer; - /** The serializer for the value */ + /** The serializer for the value. */ private final TypeSerializer valueSerializer; - /** reusable input deserialization buffer */ + /** reusable input deserialization buffer. */ private final DataInputDeserializer inputDeserializer; - - /** reusable output serialization buffer for the key */ + + /** reusable output serialization buffer for the key. */ private transient DataOutputSerializer keyOutputSerializer; - /** reusable output serialization buffer for the value */ + /** reusable output serialization buffer for the value. */ private transient DataOutputSerializer valueOutputSerializer; - - + /** The type information, to be returned by {@link #getProducedType()}. It is * transient, because it is not serializable. Note that this means that the type information * is not available at runtime, but only prior to the first serialization / deserialization */ @@ -80,10 +79,10 @@ public TypeInformationKeyValueSerializationSchema(TypeInformation keyTypeInfo /** * Creates a new de-/serialization schema for the given types. This constructor accepts the types * as classes and internally constructs the type information from the classes. - * + * *

    If the types are parametrized and cannot be fully defined via classes, use the constructor * that accepts {@link TypeInformation} instead. - * + * * @param keyClass The class of the key de-/serialized by this schema. * @param valueClass The class of the value de-/serialized by this schema. * @param config The execution config, which is used to parametrize the type serializers. @@ -94,12 +93,11 @@ public TypeInformationKeyValueSerializationSchema(Class keyClass, Class va // ------------------------------------------------------------------------ - @Override public Tuple2 deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset) throws IOException { K key = null; V value = null; - + if (messageKey != null) { inputDeserializer.setBuffer(messageKey, 0, messageKey.length); key = keySerializer.deserialize(inputDeserializer); @@ -117,11 +115,10 @@ public Tuple2 deserialize(byte[] messageKey, byte[] message, String topic, * @return Returns false. */ @Override - public boolean isEndOfStream(Tuple2 nextElement) { + public boolean isEndOfStream(Tuple2 nextElement) { return false; } - @Override public byte[] serializeKey(Tuple2 element) { if (element.f0 == null) { @@ -182,9 +179,8 @@ public String getTargetTopic(Tuple2 element) { return null; // we are never overriding the topic } - @Override - public TypeInformation> getProducedType() { + public TypeInformation> getProducedType() { if (typeInfo != null) { return typeInfo; } diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/AvroRowDeSerializationSchemaTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/AvroRowDeSerializationSchemaTest.java index e13968e6e2447..d5be274f77953 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/AvroRowDeSerializationSchemaTest.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/AvroRowDeSerializationSchemaTest.java @@ -15,18 +15,22 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.flink.streaming.connectors.kafka; -import java.io.IOException; -import org.apache.avro.specific.SpecificRecord; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.streaming.connectors.kafka.testutils.AvroTestUtils; import org.apache.flink.streaming.util.serialization.AvroRowDeserializationSchema; import org.apache.flink.streaming.util.serialization.AvroRowSerializationSchema; import org.apache.flink.types.Row; -import static org.junit.Assert.assertEquals; + +import org.apache.avro.specific.SpecificRecord; import org.junit.Test; +import java.io.IOException; + +import static org.junit.Assert.assertEquals; + /** * Test for the Avro serialization and deserialization schema. */ diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/TestFlinkFixedPartitioner.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkFixedPartitionerTest.java similarity index 92% rename from flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/TestFlinkFixedPartitioner.java rename to flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkFixedPartitionerTest.java index fa84199a12f35..b62bdd54a65d1 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/TestFlinkFixedPartitioner.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkFixedPartitionerTest.java @@ -15,16 +15,21 @@ * 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.FlinkFixedPartitioner; + import org.junit.Assert; import org.junit.Test; -public class TestFlinkFixedPartitioner { - +/** + * Tests for the {@link FlinkFixedPartitioner}. + */ +public class FlinkFixedPartitionerTest { /** + * Test for when there are more sinks than partitions. *

     	 *   		Flink Sinks:		Kafka Partitions
     	 * 			1	---------------->	1
    @@ -54,7 +59,7 @@ public void testMoreFlinkThanBrokers() {
     	}
     
     	/**
    -	 *
    +	 * Tests for when there are more partitions than sinks.
     	 * 
     	 * 		Flink Sinks:		Kafka Partitions
     	 * 			1	---------------->	1
    @@ -88,7 +93,7 @@ public void testFewerPartitions() {
     	@Test
     	public void testMixedCase() {
     		FlinkFixedPartitioner part = new FlinkFixedPartitioner<>();
    -		int[] partitions = new int[]{0,1};
    +		int[] partitions = new int[]{0, 1};
     
     		part.open(0, 3);
     		Assert.assertEquals(0, part.partition("abc1", null, null, null, partitions));
    diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom11MigrationTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom11MigrationTest.java
    index c07ebd5b21e09..77417ab036e46 100644
    --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom11MigrationTest.java
    +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom11MigrationTest.java
    @@ -15,6 +15,7 @@
      * 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.api.TimeCharacteristic;
    @@ -29,6 +30,7 @@
     import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
     import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
     import org.apache.flink.util.SerializedValue;
    +
     import org.junit.Assert;
     import org.junit.Test;
     
    @@ -36,9 +38,9 @@
     import java.util.ArrayList;
     import java.util.Arrays;
     import java.util.Collections;
    -import java.util.Map;
     import java.util.HashMap;
     import java.util.List;
    +import java.util.Map;
     
     import static org.mockito.Mockito.mock;
     
    @@ -316,7 +318,6 @@ public void close() {
     		}
     	}
     
    -
     	// ------------------------------------------------------------------------
     
     	private interface FetcherFactory extends Serializable {
    diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom12MigrationTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom12MigrationTest.java
    index f11bf9f0b3a61..f13cbe0cb6d93 100644
    --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom12MigrationTest.java
    +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom12MigrationTest.java
    @@ -15,19 +15,9 @@
      * See the License for the specific language governing permissions and
      * limitations under the License.
      */
    -package org.apache.flink.streaming.connectors.kafka;
     
    -import static org.junit.Assert.assertTrue;
    -import static org.mockito.Mockito.doAnswer;
    -import static org.mockito.Mockito.mock;
    -import static org.mockito.Mockito.when;
    +package org.apache.flink.streaming.connectors.kafka;
     
    -import java.util.ArrayList;
    -import java.util.Arrays;
    -import java.util.Collections;
    -import java.util.HashMap;
    -import java.util.List;
    -import java.util.Map;
     import org.apache.flink.core.testutils.OneShotLatch;
     import org.apache.flink.streaming.api.TimeCharacteristic;
     import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
    @@ -45,12 +35,25 @@
     import org.apache.flink.streaming.util.OperatorSnapshotUtil;
     import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
     import org.apache.flink.util.SerializedValue;
    +
     import org.junit.Assert;
     import org.junit.Ignore;
     import org.junit.Test;
     import org.mockito.invocation.InvocationOnMock;
     import org.mockito.stubbing.Answer;
     
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +import static org.junit.Assert.assertTrue;
    +import static org.mockito.Mockito.doAnswer;
    +import static org.mockito.Mockito.mock;
    +import static org.mockito.Mockito.when;
    +
     /**
      * Tests for checking whether {@link FlinkKafkaConsumerBase} can restore from snapshots that were
      * done using the Flink 1.2 {@link FlinkKafkaConsumerBase}.
    @@ -60,7 +63,7 @@
      */
     public class FlinkKafkaConsumerBaseFrom12MigrationTest {
     
    -	final static HashMap PARTITION_STATE = new HashMap<>();
    +	private static final HashMap PARTITION_STATE = new HashMap<>();
     
     	static {
     		PARTITION_STATE.put(new KafkaTopicPartition("abc", 13), 16768L);
    @@ -101,7 +104,6 @@ public Void answer(InvocationOnMock invocation) throws Throwable {
     		StreamSource> consumerOperator =
     				new StreamSource<>(consumerFunction);
     
    -
     		final AbstractStreamOperatorTestHarness testHarness =
     				new AbstractStreamOperatorTestHarness<>(consumerOperator, 1, 1, 0);
     
    @@ -307,8 +309,7 @@ protected boolean getIsAutoCommitEnabled() {
     		}
     	}
     
    -
    -	private static abstract class DummySourceContext
    +	private abstract static class DummySourceContext
     			implements SourceFunction.SourceContext {
     
     		private final Object lock = new Object();
    diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java
    index ccf2ed2044a0c..d673e8e32943d 100644
    --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java
    +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java
    @@ -18,7 +18,6 @@
     
     package org.apache.flink.streaming.connectors.kafka;
     
    -import org.apache.commons.collections.map.LinkedMap;
     import org.apache.flink.api.common.state.ListState;
     import org.apache.flink.api.common.state.ListStateDescriptor;
     import org.apache.flink.api.common.state.OperatorStateStore;
    @@ -35,6 +34,8 @@
     import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
     import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
     import org.apache.flink.util.SerializedValue;
    +
    +import org.apache.commons.collections.map.LinkedMap;
     import org.junit.Assert;
     import org.junit.Test;
     import org.mockito.Matchers;
    @@ -61,6 +62,9 @@
     import static org.mockito.Mockito.verify;
     import static org.mockito.Mockito.when;
     
    +/**
    + * Tests for the {@link FlinkKafkaConsumerBase}.
    + */
     public class FlinkKafkaConsumerBaseTest {
     
     	/**
    @@ -77,12 +81,12 @@ public void testEitherWatermarkExtractor() {
     			new DummyFlinkKafkaConsumer<>().assignTimestampsAndWatermarks((AssignerWithPunctuatedWatermarks) null);
     			fail();
     		} catch (NullPointerException ignored) {}
    -		
    +
     		@SuppressWarnings("unchecked")
     		final AssignerWithPeriodicWatermarks periodicAssigner = mock(AssignerWithPeriodicWatermarks.class);
     		@SuppressWarnings("unchecked")
     		final AssignerWithPunctuatedWatermarks punctuatedAssigner = mock(AssignerWithPunctuatedWatermarks.class);
    -		
    +
     		DummyFlinkKafkaConsumer c1 = new DummyFlinkKafkaConsumer<>();
     		c1.assignTimestampsAndWatermarks(periodicAssigner);
     		try {
    @@ -189,7 +193,7 @@ public void checkRestoredNullCheckpointWhenFetcherNotReady() throws Exception {
     	}
     
     	/**
    -	 * Tests that on snapshots, states and offsets to commit to Kafka are correct
    +	 * Tests that on snapshots, states and offsets to commit to Kafka are correct.
     	 */
     	@SuppressWarnings("unchecked")
     	@Test
    @@ -301,7 +305,7 @@ public void testSnapshotStateWithCommitOnCheckpointsEnabled() throws Exception {
     		state3.put(new KafkaTopicPartition("def", 7), 987654377L);
     
     		// --------------------------------------------------------------------
    -		
    +
     		final AbstractFetcher fetcher = mock(AbstractFetcher.class);
     		when(fetcher.snapshotCurrentState()).thenReturn(state1, state2, state3);
     
    @@ -356,7 +360,7 @@ public void testSnapshotStateWithCommitOnCheckpointsEnabled() throws Exception {
     		assertEquals(state2, snapshot2);
     		assertEquals(2, pendingOffsetsToCommit.size());
     		assertEquals(state2, pendingOffsetsToCommit.get(140L));
    -		
    +
     		// ack checkpoint 1
     		consumer.notifyCheckpointComplete(138L);
     		assertEquals(1, pendingOffsetsToCommit.size());
    @@ -375,12 +379,11 @@ public void testSnapshotStateWithCommitOnCheckpointsEnabled() throws Exception {
     		assertEquals(state3, snapshot3);
     		assertEquals(2, pendingOffsetsToCommit.size());
     		assertEquals(state3, pendingOffsetsToCommit.get(141L));
    -		
    +
     		// ack checkpoint 3, subsumes number 2
     		consumer.notifyCheckpointComplete(141L);
     		assertEquals(0, pendingOffsetsToCommit.size());
     
    -
     		consumer.notifyCheckpointComplete(666); // invalid checkpoint
     		assertEquals(0, pendingOffsetsToCommit.size());
     
    @@ -504,7 +507,6 @@ public void testSnapshotStateWithCommitOnCheckpointsDisabled() throws Exception
     		consumer.notifyCheckpointComplete(141L);
     		verify(fetcher, never()).commitInternalOffsetsToKafka(anyMap()); // not offsets should be committed
     
    -
     		consumer.notifyCheckpointComplete(666); // invalid checkpoint
     		verify(fetcher, never()).commitInternalOffsetsToKafka(anyMap()); // not offsets should be committed
     
    @@ -535,8 +537,7 @@ public void testSnapshotStateWithCommitOnCheckpointsDisabled() throws Exception
     	// ------------------------------------------------------------------------
     
     	private static  FlinkKafkaConsumerBase getConsumer(
    -			AbstractFetcher fetcher, LinkedMap pendingOffsetsToCommit, boolean running) throws Exception
    -	{
    +			AbstractFetcher fetcher, LinkedMap pendingOffsetsToCommit, boolean running) throws Exception {
     		FlinkKafkaConsumerBase consumer = new DummyFlinkKafkaConsumer<>();
     		StreamingRuntimeContext mockRuntimeContext = mock(StreamingRuntimeContext.class);
     		Mockito.when(mockRuntimeContext.isCheckpointingEnabled()).thenReturn(true);
    diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBaseTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBaseTest.java
    index 6b2cc025e9408..08c5f010ac544 100644
    --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBaseTest.java
    +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBaseTest.java
    @@ -15,21 +15,23 @@
      * 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.api.common.functions.RuntimeContext;
    +import org.apache.flink.configuration.Configuration;
     import org.apache.flink.core.testutils.CheckedThread;
     import org.apache.flink.core.testutils.MultiShotLatch;
     import org.apache.flink.runtime.state.FunctionSnapshotContext;
     import org.apache.flink.streaming.api.operators.StreamSink;
     import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
    +import org.apache.flink.streaming.connectors.kafka.testutils.FakeStandardProducerConfig;
     import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
     import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
    -import org.apache.flink.configuration.Configuration;
    -import org.apache.flink.streaming.connectors.kafka.testutils.FakeStandardProducerConfig;
     import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
     import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
     import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
    +
     import org.apache.kafka.clients.producer.Callback;
     import org.apache.kafka.clients.producer.KafkaProducer;
     import org.apache.kafka.clients.producer.ProducerConfig;
    @@ -53,10 +55,13 @@
     import static org.mockito.Mockito.verify;
     import static org.mockito.Mockito.when;
     
    +/**
    + * Tests for the {@link FlinkKafkaProducerBase}.
    + */
     public class FlinkKafkaProducerBaseTest {
     
     	/**
    -	 * Tests that the constructor eagerly checks bootstrap servers are set in config
    +	 * Tests that the constructor eagerly checks bootstrap servers are set in config.
     	 */
     	@Test(expected = IllegalArgumentException.class)
     	public void testInstantiationFailsWhenBootstrapServersMissing() throws Exception {
    @@ -67,7 +72,7 @@ public void testInstantiationFailsWhenBootstrapServersMissing() throws Exception
     	}
     
     	/**
    -	 * Tests that constructor defaults to key value serializers in config to byte array deserializers if not set
    +	 * Tests that constructor defaults to key value serializers in config to byte array deserializers if not set.
     	 */
     	@Test
     	public void testKeyValueDeserializersSetIfMissing() throws Exception {
    @@ -83,7 +88,7 @@ public void testKeyValueDeserializersSetIfMissing() throws Exception {
     	}
     
     	/**
    -	 * Tests that partitions list is determinate and correctly provided to custom partitioner
    +	 * Tests that partitions list is determinate and correctly provided to custom partitioner.
     	 */
     	@SuppressWarnings("unchecked")
     	@Test
    @@ -93,7 +98,7 @@ public void testPartitionerInvokedWithDeterminatePartitionList() throws Exceptio
     		RuntimeContext mockRuntimeContext = mock(RuntimeContext.class);
     		when(mockRuntimeContext.getIndexOfThisSubtask()).thenReturn(0);
     		when(mockRuntimeContext.getNumberOfParallelSubtasks()).thenReturn(1);
    -		
    +
     		// out-of-order list of 4 partitions
     		List mockPartitionsList = new ArrayList<>(4);
     		mockPartitionsList.add(new PartitionInfo(DummyFlinkKafkaProducer.DUMMY_TOPIC, 3, null, null, null));
    @@ -118,7 +123,7 @@ public void testPartitionerInvokedWithDeterminatePartitionList() throws Exceptio
     	}
     
     	/**
    -	 * Test ensuring that if an invoke call happens right after an async exception is caught, it should be rethrown
    +	 * Test ensuring that if an invoke call happens right after an async exception is caught, it should be rethrown.
     	 */
     	@Test
     	public void testAsyncErrorRethrownOnInvoke() throws Throwable {
    @@ -149,7 +154,7 @@ public void testAsyncErrorRethrownOnInvoke() throws Throwable {
     	}
     
     	/**
    -	 * Test ensuring that if a snapshot call happens right after an async exception is caught, it should be rethrown
    +	 * Test ensuring that if a snapshot call happens right after an async exception is caught, it should be rethrown.
     	 */
     	@Test
     	public void testAsyncErrorRethrownOnCheckpoint() throws Throwable {
    @@ -183,11 +188,11 @@ public void testAsyncErrorRethrownOnCheckpoint() throws Throwable {
     	 * Test ensuring that if an async exception is caught for one of the flushed requests on checkpoint,
     	 * it should be rethrown; we set a timeout because the test will not finish if the logic is broken.
     	 *
    -	 * Note that this test does not test the snapshot method is blocked correctly when there are pending recorrds.
    +	 * 

    Note that this test does not test the snapshot method is blocked correctly when there are pending recorrds. * The test for that is covered in testAtLeastOnceProducer. */ @SuppressWarnings("unchecked") - @Test(timeout=5000) + @Test(timeout = 5000) public void testAsyncErrorRethrownOnCheckpointAfterFlush() throws Throwable { final DummyFlinkKafkaProducer producer = new DummyFlinkKafkaProducer<>( FakeStandardProducerConfig.get(), new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()), null); @@ -237,10 +242,10 @@ public void go() throws Exception { /** * Test ensuring that the producer is not dropping buffered records; - * we set a timeout because the test will not finish if the logic is broken + * we set a timeout because the test will not finish if the logic is broken. */ @SuppressWarnings("unchecked") - @Test(timeout=10000) + @Test(timeout = 10000) public void testAtLeastOnceProducer() throws Throwable { final DummyFlinkKafkaProducer producer = new DummyFlinkKafkaProducer<>( FakeStandardProducerConfig.get(), new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()), null); @@ -297,10 +302,10 @@ public void go() throws Exception { /** * This test is meant to assure that testAtLeastOnceProducer is valid by testing that if flushing is disabled, * the snapshot method does indeed finishes without waiting for pending records; - * we set a timeout because the test will not finish if the logic is broken + * we set a timeout because the test will not finish if the logic is broken. */ @SuppressWarnings("unchecked") - @Test(timeout=5000) + @Test(timeout = 5000) public void testDoesNotWaitForPendingRecordsIfFlushingDisabled() throws Throwable { final DummyFlinkKafkaProducer producer = new DummyFlinkKafkaProducer<>( FakeStandardProducerConfig.get(), new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()), null); @@ -328,8 +333,8 @@ public void testDoesNotWaitForPendingRecordsIfFlushingDisabled() throws Throwabl private static class DummyFlinkKafkaProducer extends FlinkKafkaProducerBase { private static final long serialVersionUID = 1L; - - private final static String DUMMY_TOPIC = "dummy-topic"; + + private static final String DUMMY_TOPIC = "dummy-topic"; private transient KafkaProducer mockProducer; private transient List pendingCallbacks; diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONDeserializationSchemaTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONDeserializationSchemaTest.java index 1882a7ed67ed9..51e483bbac6b2 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONDeserializationSchemaTest.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONDeserializationSchemaTest.java @@ -14,16 +14,21 @@ * 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.JSONDeserializationSchema; + import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.ObjectNode; -import org.apache.flink.streaming.util.serialization.JSONDeserializationSchema; import org.junit.Assert; import org.junit.Test; import java.io.IOException; +/** + * Tests for the {@link JSONDeserializationSchema}. + */ public class JSONDeserializationSchemaTest { @Test public void testDeserialize() throws IOException { diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONKeyValueDeserializationSchemaTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONKeyValueDeserializationSchemaTest.java index 86d3105dea684..565ef00a355c5 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONKeyValueDeserializationSchemaTest.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONKeyValueDeserializationSchemaTest.java @@ -14,16 +14,21 @@ * 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.JSONKeyValueDeserializationSchema; + import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.ObjectNode; -import org.apache.flink.streaming.util.serialization.JSONKeyValueDeserializationSchema; import org.junit.Assert; import org.junit.Test; import java.io.IOException; +/** + * Tests for the{@link JSONKeyValueDeserializationSchema}. + */ public class JSONKeyValueDeserializationSchemaTest { @Test public void testDeserializeWithoutMetadata() throws IOException { @@ -39,7 +44,6 @@ public void testDeserializeWithoutMetadata() throws IOException { JSONKeyValueDeserializationSchema schema = new JSONKeyValueDeserializationSchema(false); ObjectNode deserializedValue = schema.deserialize(serializedKey, serializedValue, "", 0, 0); - Assert.assertTrue(deserializedValue.get("metadata") == null); Assert.assertEquals(4, deserializedValue.get("key").get("index").asInt()); Assert.assertEquals("world", deserializedValue.get("value").get("word").asText()); diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JsonRowDeserializationSchemaTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JsonRowDeserializationSchemaTest.java index f03feeb26d9d1..186e3645a3579 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JsonRowDeserializationSchemaTest.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JsonRowDeserializationSchemaTest.java @@ -18,12 +18,13 @@ package org.apache.flink.streaming.connectors.kafka; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.node.ObjectNode; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.streaming.util.serialization.JsonRowDeserializationSchema; import org.apache.flink.table.api.Types; import org.apache.flink.types.Row; -import org.apache.flink.streaming.util.serialization.JsonRowDeserializationSchema; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ObjectNode; import org.junit.Test; import java.io.IOException; @@ -35,6 +36,9 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +/** + * Tests for the {@link JsonRowDeserializationSchema}. + */ public class JsonRowDeserializationSchemaTest { /** diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JsonRowSerializationSchemaTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JsonRowSerializationSchemaTest.java index 523eafef16c00..43bde35bb6a05 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JsonRowSerializationSchemaTest.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JsonRowSerializationSchemaTest.java @@ -14,19 +14,24 @@ * 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.api.common.typeinfo.TypeInformation; -import org.apache.flink.table.api.Types; -import org.apache.flink.types.Row; import org.apache.flink.streaming.util.serialization.JsonRowDeserializationSchema; import org.apache.flink.streaming.util.serialization.JsonRowSerializationSchema; +import org.apache.flink.table.api.Types; +import org.apache.flink.types.Row; + import org.junit.Test; import java.io.IOException; import static org.junit.Assert.assertEquals; +/** + * Tests for the {@link JsonRowSerializationSchema}. + */ public class JsonRowSerializationSchemaTest { @Test diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerPartitionAssignmentTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerPartitionAssignmentTest.java index c24640dfd2a38..0be1d5761f99b 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerPartitionAssignmentTest.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerPartitionAssignmentTest.java @@ -25,10 +25,10 @@ import java.util.ArrayList; import java.util.Arrays; -import java.util.HashSet; -import java.util.Map; import java.util.HashMap; +import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; import static org.junit.Assert.assertEquals; diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java index ac278fbda6232..03a23f519e13b 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java @@ -18,14 +18,6 @@ package org.apache.flink.streaming.connectors.kafka; -import kafka.consumer.Consumer; -import kafka.consumer.ConsumerConfig; -import kafka.consumer.ConsumerIterator; -import kafka.consumer.KafkaStream; -import kafka.javaapi.consumer.ConsumerConnector; -import kafka.message.MessageAndMetadata; -import kafka.server.KafkaServer; -import org.apache.commons.io.output.ByteArrayOutputStream; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.functions.FlatMapFunction; @@ -82,6 +74,15 @@ import org.apache.flink.testutils.junit.RetryOnException; import org.apache.flink.testutils.junit.RetryRule; import org.apache.flink.util.Collector; + +import kafka.consumer.Consumer; +import kafka.consumer.ConsumerConfig; +import kafka.consumer.ConsumerIterator; +import kafka.consumer.KafkaStream; +import kafka.javaapi.consumer.ConsumerConnector; +import kafka.message.MessageAndMetadata; +import kafka.server.KafkaServer; +import org.apache.commons.io.output.ByteArrayOutputStream; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.errors.TimeoutException; import org.junit.Assert; @@ -90,6 +91,7 @@ import javax.management.MBeanServer; import javax.management.ObjectName; + import java.io.ByteArrayInputStream; import java.io.IOException; import java.lang.management.ManagementFactory; @@ -114,14 +116,15 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; - +/** + * Abstract test base for all Kafka consumer tests. + */ @SuppressWarnings("serial") public abstract class KafkaConsumerTestBase extends KafkaTestBase { - + @Rule public RetryRule retryRule = new RetryRule(); - // ------------------------------------------------------------------------ // Common Test Preparation // ------------------------------------------------------------------------ @@ -134,8 +137,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase { public void ensureNoJobIsLingering() throws Exception { JobManagerCommunicationUtils.waitUntilNoJobIsRunning(flink.getLeaderGateway(timeout)); } - - + // ------------------------------------------------------------------------ // Suite of Tests // @@ -146,7 +148,7 @@ public void ensureNoJobIsLingering() throws Exception { /** * Test that ensures the KafkaConsumer is properly failing if the topic doesnt exist - * and a wrong broker was specified + * and a wrong broker was specified. * * @throws Exception */ @@ -173,8 +175,8 @@ public void runFailOnNoBrokerTest() throws Exception { DataStream stream = see.addSource(source); stream.print(); see.execute("No broker test"); - } catch(JobExecutionException jee) { - if(kafkaServer.getVersion().equals("0.9") || kafkaServer.getVersion().equals("0.10")) { + } catch (JobExecutionException jee) { + if (kafkaServer.getVersion().equals("0.9") || kafkaServer.getVersion().equals("0.10")) { assertTrue(jee.getCause() instanceof TimeoutException); TimeoutException te = (TimeoutException) jee.getCause(); @@ -191,7 +193,7 @@ public void runFailOnNoBrokerTest() throws Exception { } /** - * Ensures that the committed offsets to Kafka are the offsets of "the next record to process" + * Ensures that the committed offsets to Kafka are the offsets of "the next record to process". */ public void runCommitOffsetsToKafka() throws Exception { // 3 partitions with 50 records each (0-49, so the expected commit offset of each partition should be 50) @@ -354,13 +356,13 @@ public Object map(String value) throws Exception { * This test ensures that when the consumers retrieve some start offset from kafka (earliest, latest), that this offset * is committed to Kafka, even if some partitions are not read. * - * Test: + *

    Test: * - Create 3 partitions * - write 50 messages into each. * - Start three consumers with auto.offset.reset='latest' and wait until they committed into Kafka. * - Check if the offsets in Kafka are set to 50 for the three partitions * - * See FLINK-3440 as well + *

    See FLINK-3440 as well */ public void runAutoOffsetRetrievalAndCommitToKafka() throws Exception { // 3 partitions with 50 records each (0-49, so the expected commit offset of each partition should be 50) @@ -518,7 +520,7 @@ public void runStartFromLatestOffsets() throws Exception { env .addSource(latestReadingConsumer).setParallelism(parallelism) - .flatMap(new FlatMapFunction, Object>() { + .flatMap(new FlatMapFunction, Object>() { @Override public void flatMap(Tuple2 value, Collector out) throws Exception { if (value.f1 - recordsInEachPartition < 0) { @@ -605,12 +607,12 @@ public void cancel() { * This test ensures that the consumer correctly uses group offsets in Kafka, and defaults to "auto.offset.reset" * behaviour when necessary, when explicitly configured to start from group offsets. * - * The partitions and their committed group offsets are setup as: + *

    The partitions and their committed group offsets are setup as: * partition 0 --> committed offset 23 * partition 1 --> no commit offset * partition 2 --> committed offset 43 * - * When configured to start from group offsets, each partition should read: + *

    When configured to start from group offsets, each partition should read: * partition 0 --> start from offset 23, read to offset 49 (27 records) * partition 1 --> default to "auto.offset.reset" (set to earliest), so start from offset 0, read to offset 49 (50 records) * partition 2 --> start from offset 43, read to offset 49 (7 records) @@ -653,20 +655,20 @@ public void runStartFromGroupOffsets() throws Exception { * start from specific offsets. For partitions which a specific offset can not be found for, the starting position * for them should fallback to the group offsets behaviour. * - * 4 partitions will have 50 records with offsets 0 to 49. The supplied specific offsets map is: + *

    4 partitions will have 50 records with offsets 0 to 49. The supplied specific offsets map is: * partition 0 --> start from offset 19 * partition 1 --> not set * partition 2 --> start from offset 22 * partition 3 --> not set * partition 4 --> start from offset 26 (this should be ignored because the partition does not exist) * - * The partitions and their committed group offsets are setup as: + *

    The partitions and their committed group offsets are setup as: * partition 0 --> committed offset 23 * partition 1 --> committed offset 31 * partition 2 --> committed offset 43 * partition 3 --> no commit offset * - * When configured to start from these specific offsets, each partition should read: + *

    When configured to start from these specific offsets, each partition should read: * partition 0 --> start from offset 19, read to offset 49 (31 records) * partition 1 --> fallback to group offsets, so start from offset 31, read to offset 49 (19 records) * partition 2 --> start from offset 22, read to offset 49 (28 records) @@ -711,7 +713,7 @@ public void runStartFromSpecificOffsets() throws Exception { kafkaOffsetHandler.close(); deleteTestTopic(topicName); } - + /** * Ensure Kafka is working on both producer and consumer side. * This executes a job that contains two Flink pipelines. @@ -719,22 +721,22 @@ public void runStartFromSpecificOffsets() throws Exception { *

     	 * (generator source) --> (kafka sink)-[KAFKA-TOPIC]-(kafka source) --> (validating sink)
     	 * 
    - * - * We need to externally retry this test. We cannot let Flink's retry mechanism do it, because the Kafka producer + * + *

    We need to externally retry this test. We cannot let Flink's retry mechanism do it, because the Kafka producer * does not guarantee exactly-once output. Hence a recovery would introduce duplicates that * cause the test to fail. * - * This test also ensures that FLINK-3156 doesn't happen again: + *

    This test also ensures that FLINK-3156 doesn't happen again: * - * The following situation caused a NPE in the FlinkKafkaConsumer + *

    The following situation caused a NPE in the FlinkKafkaConsumer * - * topic-1 <-- elements are only produced into topic1. + *

    topic-1 <-- elements are only produced into topic1. * topic-2 * - * Therefore, this test is consuming as well from an empty topic. + *

    Therefore, this test is consuming as well from an empty topic. * */ - @RetryOnException(times=2, exception=kafka.common.NotLeaderForPartitionException.class) + @RetryOnException(times = 2, exception = kafka.common.NotLeaderForPartitionException.class) public void runSimpleConcurrentProducerConsumerTopology() throws Exception { final String topic = "concurrentProducerConsumerTopic_" + UUID.randomUUID().toString(); final String additionalEmptyTopic = "additionalEmptyTopic_" + UUID.randomUUID().toString(); @@ -763,7 +765,7 @@ public void runSimpleConcurrentProducerConsumerTopology() throws Exception { // ----------- add producer dataflow ---------- - DataStream> stream = env.addSource(new RichParallelSourceFunction>() { + DataStream> stream = env.addSource(new RichParallelSourceFunction>() { private boolean running = true; @@ -772,7 +774,6 @@ public void run(SourceContext> ctx) throws InterruptedExcep int cnt = getRuntimeContext().getIndexOfThisSubtask() * elementsPerPartition; int limit = cnt + elementsPerPartition; - while (running && cnt < limit) { ctx.collect(new Tuple2<>(1000L + cnt, "kafka-" + cnt)); cnt++; @@ -1002,13 +1003,11 @@ public void runMultipleSourcesOnePartitionExactlyOnceTest() throws Exception { FailingIdentityMapper.failedBefore = false; tryExecute(env, "multi-source-one-partitions exactly once test"); - deleteTestTopic(topic); } - - + /** - * Tests that the source can be properly canceled when reading full partitions. + * Tests that the source can be properly canceled when reading full partitions. */ public void runCancelingOnFullInputTest() throws Exception { final String topic = "cancelingOnFullTopic"; @@ -1056,7 +1055,7 @@ public void run() { Thread.sleep(2000); Throwable failueCause = jobError.get(); - if(failueCause != null) { + if (failueCause != null) { failueCause.printStackTrace(); Assert.fail("Test failed prematurely with: " + failueCause.getMessage()); } @@ -1089,7 +1088,7 @@ public void run() { } /** - * Tests that the source can be properly canceled when reading empty partitions. + * Tests that the source can be properly canceled when reading empty partitions. */ public void runCancelingOnEmptyInputTest() throws Exception { final String topic = "cancelingOnEmptyInputTopic"; @@ -1149,7 +1148,7 @@ public void run() { } /** - * Tests that the source can be properly canceled when reading full partitions. + * Tests that the source can be properly canceled when reading full partitions. */ public void runFailOnDeployTest() throws Exception { final String topic = "failOnDeployTopic"; @@ -1198,19 +1197,19 @@ public void runFailOnDeployTest() throws Exception { } /** - * Test producing and consuming into multiple topics + * Test producing and consuming into multiple topics. * @throws Exception */ public void runProduceConsumeMultipleTopics() throws Exception { - final int NUM_TOPICS = 5; - final int NUM_ELEMENTS = 20; + final int numTopics = 5; + final int numElements = 20; StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.getConfig().disableSysoutLogging(); - + // create topics with content final List topics = new ArrayList<>(); - for (int i = 0; i < NUM_TOPICS; i++) { + for (int i = 0; i < numTopics; i++) { final String topic = "topic-" + i; topics.add(topic); // create topic @@ -1227,8 +1226,8 @@ public void runProduceConsumeMultipleTopics() throws Exception { public void run(SourceContext> ctx) throws Exception { int partition = getRuntimeContext().getIndexOfThisSubtask(); - for (int topicId = 0; topicId < NUM_TOPICS; topicId++) { - for (int i = 0; i < NUM_ELEMENTS; i++) { + for (int topicId = 0; topicId < numTopics; topicId++) { + for (int i = 0; i < numElements; i++) { ctx.collect(new Tuple3<>(partition, i, "topic-" + topicId)); } } @@ -1255,7 +1254,7 @@ public void cancel() { stream = env.addSource(kafkaServer.getConsumer(topics, schema, props)); stream.flatMap(new FlatMapFunction, Integer>() { - Map countPerTopic = new HashMap<>(NUM_TOPICS); + Map countPerTopic = new HashMap<>(numTopics); @Override public void flatMap(Tuple3 value, Collector out) throws Exception { Integer count = countPerTopic.get(value.f2); @@ -1268,10 +1267,10 @@ public void flatMap(Tuple3 value, Collector o // check map: for (Map.Entry el: countPerTopic.entrySet()) { - if (el.getValue() < NUM_ELEMENTS) { + if (el.getValue() < numElements) { break; // not enough yet } - if (el.getValue() > NUM_ELEMENTS) { + if (el.getValue() > numElements) { throw new RuntimeException("There is a failure in the test. I've read " + el.getValue() + " from topic " + el.getKey()); } @@ -1283,17 +1282,17 @@ public void flatMap(Tuple3 value, Collector o tryExecute(env, "Count elements from the topics"); - // delete all topics again - for (int i = 0; i < NUM_TOPICS; i++) { + for (int i = 0; i < numTopics; i++) { final String topic = "topic-" + i; deleteTestTopic(topic); } } /** - * Test Flink's Kafka integration also with very big records (30MB) - * see http://stackoverflow.com/questions/21020347/kafka-sending-a-15mb-message + * Test Flink's Kafka integration also with very big records (30MB). + * + *

    see http://stackoverflow.com/questions/21020347/kafka-sending-a-15mb-message * */ public void runBigRecordTestTopology() throws Exception { @@ -1337,11 +1336,11 @@ public void invoke(Tuple2 value) throws Exception { if (elCnt == 11) { throw new SuccessException(); } else { - throw new RuntimeException("There have been "+elCnt+" elements"); + throw new RuntimeException("There have been " + elCnt + " elements"); } } if (elCnt > 10) { - throw new RuntimeException("More than 10 elements seen: "+elCnt); + throw new RuntimeException("More than 10 elements seen: " + elCnt); } } }); @@ -1395,7 +1394,6 @@ public void cancel() { deleteTestTopic(topic); } - public void runBrokerFailureTest() throws Exception { final String topic = "brokerFailureTestTopic"; @@ -1404,7 +1402,6 @@ public void runBrokerFailureTest() throws Exception { final int totalElements = parallelism * numElementsPerPartition; final int failAfterElements = numElementsPerPartition / 3; - createTestTopic(topic, parallelism, 2); DataGenerators.generateRandomizedIntegerSequence( @@ -1417,7 +1414,6 @@ public void runBrokerFailureTest() throws Exception { LOG.info("Leader to shutdown {}", leaderId); - // run the topology (the consumers must handle the failures) DeserializationSchema schema = @@ -1450,7 +1446,7 @@ public void runBrokerFailureTest() throws Exception { public void runKeyValueTest() throws Exception { final String topic = "keyvaluetest"; createTestTopic(topic, 1, 1); - final int ELEMENT_COUNT = 5000; + final int elementCount = 5000; // ----------- Write some data into Kafka ------------------- @@ -1463,7 +1459,7 @@ public void runKeyValueTest() throws Exception { @Override public void run(SourceContext> ctx) throws Exception { Random rnd = new Random(1337); - for (long i = 0; i < ELEMENT_COUNT; i++) { + for (long i = 0; i < elementCount; i++) { PojoValue pojo = new PojoValue(); pojo.when = new Date(rnd.nextLong()); pojo.lon = rnd.nextLong(); @@ -1473,6 +1469,7 @@ public void run(SourceContext> ctx) throws Exception { ctx.collect(new Tuple2<>(key, pojo)); } } + @Override public void cancel() { } @@ -1491,26 +1488,25 @@ public void cancel() { env.setRestartStrategy(RestartStrategies.noRestart()); env.getConfig().disableSysoutLogging(); - KeyedDeserializationSchema> readSchema = new TypeInformationKeyValueSerializationSchema<>(Long.class, PojoValue.class, env.getConfig()); Properties props = new Properties(); props.putAll(standardProps); props.putAll(secureProps); DataStream> fromKafka = env.addSource(kafkaServer.getConsumer(topic, readSchema, props)); - fromKafka.flatMap(new RichFlatMapFunction, Object>() { + fromKafka.flatMap(new RichFlatMapFunction, Object>() { long counter = 0; @Override public void flatMap(Tuple2 value, Collector out) throws Exception { // the elements should be in order. - Assert.assertTrue("Wrong value " + value.f1.lat, value.f1.lat == counter ); + Assert.assertTrue("Wrong value " + value.f1.lat, value.f1.lat == counter); if (value.f1.lat % 2 == 0) { assertNull("key was not null", value.f0); } else { Assert.assertTrue("Wrong value " + value.f0, value.f0 == counter); } counter++; - if (counter == ELEMENT_COUNT) { + if (counter == elementCount) { // we got the right number of elements throw new SuccessException(); } @@ -1522,22 +1518,21 @@ public void flatMap(Tuple2 value, Collector out) throws deleteTestTopic(topic); } - public static class PojoValue { + private static class PojoValue { public Date when; public long lon; public long lat; public PojoValue() {} } - /** - * Test delete behavior and metrics for producer + * Test delete behavior and metrics for producer. * @throws Exception */ public void runAllDeletesTest() throws Exception { final String topic = "alldeletestest"; createTestTopic(topic, 1, 1); - final int ELEMENT_COUNT = 300; + final int elementCount = 300; // ----------- Write some data into Kafka ------------------- @@ -1550,12 +1545,13 @@ public void runAllDeletesTest() throws Exception { @Override public void run(SourceContext> ctx) throws Exception { Random rnd = new Random(1337); - for (long i = 0; i < ELEMENT_COUNT; i++) { + for (long i = 0; i < elementCount; i++) { final byte[] key = new byte[200]; rnd.nextBytes(key); ctx.collect(new Tuple2<>(key, (PojoValue) null)); } } + @Override public void cancel() { } @@ -1589,7 +1585,7 @@ public void flatMap(Tuple2 value, Collector out) thro // ensure that deleted messages are passed as nulls assertNull(value.f1); counter++; - if (counter == ELEMENT_COUNT) { + if (counter == elementCount) { // we got the right number of elements throw new SuccessException(); } @@ -1608,8 +1604,8 @@ public void flatMap(Tuple2 value, Collector out) thro */ public void runEndOfStreamTest() throws Exception { - final int ELEMENT_COUNT = 300; - final String topic = writeSequence("testEndOfStream", ELEMENT_COUNT, 1, 1); + final int elementCount = 300; + final String topic = writeSequence("testEndOfStream", elementCount, 1, 1); // read using custom schema final StreamExecutionEnvironment env1 = StreamExecutionEnvironment.getExecutionEnvironment(); @@ -1621,21 +1617,21 @@ public void runEndOfStreamTest() throws Exception { props.putAll(standardProps); props.putAll(secureProps); - DataStream> fromKafka = env1.addSource(kafkaServer.getConsumer(topic, new FixedNumberDeserializationSchema(ELEMENT_COUNT), props)); - fromKafka.flatMap(new FlatMapFunction, Void>() { + DataStream> fromKafka = env1.addSource(kafkaServer.getConsumer(topic, new FixedNumberDeserializationSchema(elementCount), props)); + fromKafka.flatMap(new FlatMapFunction, Void>() { @Override public void flatMap(Tuple2 value, Collector out) throws Exception { // noop ;) } }); - JobExecutionResult result = tryExecute(env1, "Consume " + ELEMENT_COUNT + " elements from Kafka"); + JobExecutionResult result = tryExecute(env1, "Consume " + elementCount + " elements from Kafka"); deleteTestTopic(topic); } /** - * Test metrics reporting for consumer + * Test metrics reporting for consumer. * * @throws Exception */ @@ -1690,9 +1686,9 @@ public void cancel() { kafkaServer.produceIntoKafka(fromGen, topic, new KeyedSerializationSchemaWrapper<>(schema), standardProps, null); env1.execute("Metrics test job"); - } catch(Throwable t) { + } catch (Throwable t) { LOG.warn("Got exception during execution", t); - if(!(t instanceof JobCancellationException)) { // we'll cancel the job + if (!(t instanceof JobCancellationException)) { // we'll cancel the job error.f0 = t; } } @@ -1723,7 +1719,7 @@ public void cancel() { // check that offsets are correctly reported for (ObjectName object : offsetMetrics) { Object offset = mBeanServer.getAttribute(object, "Value"); - if((long) offset >= 0) { + if ((long) offset >= 0) { numPosOffsets++; } } @@ -1738,7 +1734,6 @@ public void cancel() { Set producerMetrics = mBeanServer.queryNames(new ObjectName("*KafkaProducer*:*"), null); Assert.assertTrue("No producer metrics found", producerMetrics.size() > 30); - LOG.info("Found all JMX metrics. Cancelling job."); } finally { // cancel @@ -1755,12 +1750,11 @@ public void cancel() { deleteTestTopic(topic); } + private static class FixedNumberDeserializationSchema implements DeserializationSchema> { - public static class FixedNumberDeserializationSchema implements DeserializationSchema> { - final int finalCount; int count = 0; - + TypeInformation> ti = TypeInfoParser.parse("Tuple2"); TypeSerializer> ser = ti.createSerializer(new ExecutionConfig()); @@ -1785,7 +1779,6 @@ public TypeInformation> getProducedType() { } } - // ------------------------------------------------------------------------ // Reading writing test data sets // ------------------------------------------------------------------------ @@ -1916,13 +1909,12 @@ protected String writeSequence( String baseTopicName, final int numElements, final int parallelism, - final int replicationFactor) throws Exception - { + final int replicationFactor) throws Exception { LOG.info("\n===================================\n" + "== Writing sequence of " + numElements + " into " + baseTopicName + " with p=" + parallelism + "\n" + "==================================="); - final TypeInformation> resultType = + final TypeInformation> resultType = TypeInformation.of(new TypeHint>() {}); final KeyedSerializationSchema> serSchema = @@ -1932,15 +1924,15 @@ protected String writeSequence( final KeyedDeserializationSchema> deserSchema = new KeyedDeserializationSchemaWrapper<>( new TypeInformationSerializationSchema<>(resultType, new ExecutionConfig())); - + final int maxNumAttempts = 10; for (int attempt = 1; attempt <= maxNumAttempts; attempt++) { - + final String topicName = baseTopicName + '-' + attempt; - + LOG.info("Writing attempt #1"); - + // -------- Write the Sequence -------- createTestTopic(topicName, parallelism, replicationFactor); @@ -1948,33 +1940,33 @@ protected String writeSequence( StreamExecutionEnvironment writeEnv = StreamExecutionEnvironment.getExecutionEnvironment(); writeEnv.getConfig().setRestartStrategy(RestartStrategies.noRestart()); writeEnv.getConfig().disableSysoutLogging(); - + DataStream> stream = writeEnv.addSource(new RichParallelSourceFunction>() { - + private boolean running = true; - + @Override public void run(SourceContext> ctx) throws Exception { int cnt = 0; int partition = getRuntimeContext().getIndexOfThisSubtask(); - + while (running && cnt < numElements) { ctx.collect(new Tuple2<>(partition, cnt)); cnt++; } } - + @Override public void cancel() { running = false; } }).setParallelism(parallelism); - + // the producer must not produce duplicates Properties producerProperties = FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings); producerProperties.setProperty("retries", "0"); producerProperties.putAll(secureProps); - + kafkaServer.produceIntoKafka(stream, topicName, serSchema, producerProperties, new Tuple2FlinkPartitioner(parallelism)) .setParallelism(parallelism); @@ -1987,21 +1979,21 @@ public void cancel() { JobManagerCommunicationUtils.waitUntilNoJobIsRunning(flink.getLeaderGateway(timeout)); continue; } - + LOG.info("Finished writing sequence"); // -------- Validate the Sequence -------- - + // we need to validate the sequence, because kafka's producers are not exactly once LOG.info("Validating sequence"); JobManagerCommunicationUtils.waitUntilNoJobIsRunning(flink.getLeaderGateway(timeout)); - + final StreamExecutionEnvironment readEnv = StreamExecutionEnvironment.getExecutionEnvironment(); readEnv.getConfig().setRestartStrategy(RestartStrategies.noRestart()); readEnv.getConfig().disableSysoutLogging(); readEnv.setParallelism(parallelism); - + Properties readProps = (Properties) standardProps.clone(); readProps.setProperty("group.id", "flink-tests-validator"); readProps.putAll(secureProps); @@ -2010,10 +2002,10 @@ public void cancel() { readEnv .addSource(consumer) .map(new RichMapFunction, Tuple2>() { - + private final int totalCount = parallelism * numElements; private int count = 0; - + @Override public Tuple2 map(Tuple2 value) throws Exception { if (++count == totalCount) { @@ -2024,9 +2016,9 @@ public Tuple2 map(Tuple2 value) throws Excep } }).setParallelism(1) .addSink(new DiscardingSink>()).setParallelism(1); - + final AtomicReference errorRef = new AtomicReference<>(); - + Thread runner = new Thread() { @Override public void run() { @@ -2038,15 +2030,15 @@ public void run() { } }; runner.start(); - + final long deadline = System.nanoTime() + 10_000_000_000L; long delay; while (runner.isAlive() && (delay = deadline - System.nanoTime()) > 0) { - runner.join(delay/1_000_000L); + runner.join(delay / 1_000_000L); } - + boolean success; - + if (runner.isAlive()) { // did not finish in time, maybe the producer dropped one or more records and // the validation did not reach the exit point @@ -2065,7 +2057,7 @@ public void run() { } JobManagerCommunicationUtils.waitUntilNoJobIsRunning(flink.getLeaderGateway(timeout)); - + if (success) { // everything is good! return topicName; @@ -2075,7 +2067,7 @@ public void run() { // fall through the loop } } - + throw new Exception("Could not write a valid sequence to Kafka after " + maxNumAttempts + " attempts"); } @@ -2090,28 +2082,28 @@ private static List> readTopicToList(String t ConsumerConnector consumerConnector = Consumer.createJavaConsumerConnector(config); // we request only one stream per consumer instance. Kafka will make sure that each consumer group // will see each message only once. - Map topicCountMap = Collections.singletonMap(topicName, 1); + Map topicCountMap = Collections.singletonMap(topicName, 1); Map>> streams = consumerConnector.createMessageStreams(topicCountMap); if (streams.size() != 1) { - throw new RuntimeException("Expected only one message stream but got "+streams.size()); + throw new RuntimeException("Expected only one message stream but got " + streams.size()); } List> kafkaStreams = streams.get(topicName); if (kafkaStreams == null) { - throw new RuntimeException("Requested stream not available. Available streams: "+streams.toString()); + throw new RuntimeException("Requested stream not available. Available streams: " + streams.toString()); } if (kafkaStreams.size() != 1) { - throw new RuntimeException("Requested 1 stream from Kafka, bot got "+kafkaStreams.size()+" streams"); + throw new RuntimeException("Requested 1 stream from Kafka, bot got " + kafkaStreams.size() + " streams"); } LOG.info("Opening Consumer instance for topic '{}' on group '{}'", topicName, config.groupId()); ConsumerIterator iteratorToRead = kafkaStreams.get(0).iterator(); List> result = new ArrayList<>(); int read = 0; - while(iteratorToRead.hasNext()) { + while (iteratorToRead.hasNext()) { read++; result.add(iteratorToRead.next()); if (read == stopAfter) { - LOG.info("Read "+read+" elements"); + LOG.info("Read " + read + " elements"); return result; } } @@ -2131,12 +2123,11 @@ private static void printTopic(String topicName, ConsumerConfig config, } } - private static void printTopic(String topicName, int elements,DeserializationSchema deserializer) - throws IOException - { + private static void printTopic(String topicName, int elements, DeserializationSchema deserializer) + throws IOException { // write the sequence to log for debugging purposes Properties newProps = new Properties(standardProps); - newProps.setProperty("group.id", "topic-printer"+ UUID.randomUUID().toString()); + newProps.setProperty("group.id", "topic-printer" + UUID.randomUUID().toString()); newProps.setProperty("auto.offset.reset", "smallest"); newProps.setProperty("zookeeper.connect", standardProps.getProperty("zookeeper.connect")); newProps.putAll(secureProps); @@ -2145,15 +2136,14 @@ private static void printTopic(String topicName, int elements,DeserializationSch printTopic(topicName, printerConfig, deserializer, elements); } - - public static class BrokerKillingMapper extends RichMapFunction + private static class BrokerKillingMapper extends RichMapFunction implements ListCheckpointed, CheckpointListener { private static final long serialVersionUID = 6334389850158707313L; public static volatile boolean killedLeaderBefore; public static volatile boolean hasBeenCheckpointedBeforeFailure; - + private final int shutdownBrokerId; private final int failCount; private int numElementsTotal; @@ -2174,10 +2164,10 @@ public void open(Configuration parameters) { @Override public T map(T value) throws Exception { numElementsTotal++; - + if (!killedLeaderBefore) { Thread.sleep(10); - + if (failer && numElementsTotal >= failCount) { // shut down a Kafka broker KafkaServer toShutDown = null; @@ -2188,14 +2178,14 @@ public T map(T value) throws Exception { break; } } - + if (toShutDown == null) { StringBuilder listOfBrokers = new StringBuilder(); for (KafkaServer server : kafkaServer.getBrokers()) { listOfBrokers.append(kafkaServer.getBrokerId(server)); listOfBrokers.append(" ; "); } - + throw new Exception("Cannot find broker to shut down: " + shutdownBrokerId + " ; available brokers: " + listOfBrokers.toString()); } @@ -2266,7 +2256,7 @@ public byte[] serializeValue(Tuple3 element) { try { ts.serialize(new Tuple2<>(element.f0, element.f1), out); } catch (IOException e) { - throw new RuntimeException("Error" ,e); + throw new RuntimeException("Error" , e); } return by.toByteArray(); } diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java index bcc8328f53a49..e292e1395317a 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java @@ -43,6 +43,9 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; +/** + * Abstract test base for all Kafka producer tests. + */ @SuppressWarnings("serial") public abstract class KafkaProducerTestBase extends KafkaTestBase { @@ -50,7 +53,7 @@ public abstract class KafkaProducerTestBase extends KafkaTestBase { * This tests verifies that custom partitioning works correctly, with a default topic * and dynamic topic. The number of partitions for each topic is deliberately different. * - * Test topology: + *

    Test topology: * *

     	 *             +------> (sink) --+--> [DEFAULT_TOPIC-1] --> (source) -> (map) -----+
    @@ -67,11 +70,11 @@ public abstract class KafkaProducerTestBase extends KafkaTestBase {
     	 *            \                  |                             |          |        |
     	 *             +------> (sink) --+--> [DYNAMIC_TOPIC-3] --> (source) -> (map) -----+
     	 * 
    - * - * Each topic has an independent mapper that validates the values come consistently from + * + *

    Each topic has an independent mapper that validates the values come consistently from * the correct Kafka partition of the topic is is responsible of. - * - * Each topic also has a final sink that validates that there are no duplicates and that all + * + *

    Each topic also has a final sink that validates that there are no duplicates and that all * partitions are present. */ public void runCustomPartitioningTest() { @@ -171,7 +174,7 @@ public void cancel() { // ------------------------------------------------------------------------ - public static class CustomPartitioner extends FlinkKafkaPartitioner> implements Serializable { + private static class CustomPartitioner extends FlinkKafkaPartitioner> implements Serializable { private final Map expectedTopicsToNumPartitions; diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java index f688660b2aea0..d5c9276244e8a 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java @@ -37,7 +37,6 @@ import org.junit.AfterClass; import org.junit.BeforeClass; - import org.junit.ClassRule; import org.junit.rules.TemporaryFolder; import org.slf4j.Logger; @@ -58,7 +57,7 @@ */ @SuppressWarnings("serial") public class KafkaShortRetentionTestBase implements Serializable { - + protected static final Logger LOG = LoggerFactory.getLogger(KafkaShortRetentionTestBase.class); protected static final int NUM_TMS = 1; @@ -66,7 +65,7 @@ public class KafkaShortRetentionTestBase implements Serializable { protected static final int TM_SLOTS = 8; protected static final int PARALLELISM = NUM_TMS * TM_SLOTS; - + private static KafkaTestEnvironment kafkaServer; private static Properties standardProps; private static LocalFlinkMiniCluster flink; @@ -90,7 +89,7 @@ public static void prepare() throws IOException, ClassNotFoundException { LOG.info("Starting KafkaTestBase.prepare() for Kafka " + kafkaServer.getVersion()); - if(kafkaServer.isSecureRunSupported()) { + if (kafkaServer.isSecureRunSupported()) { secureProps = kafkaServer.getSecureProperties(); } @@ -151,10 +150,8 @@ public void runAutoOffsetResetTest() throws Exception { env.setRestartStrategy(RestartStrategies.noRestart()); // fail immediately env.getConfig().disableSysoutLogging(); - // ----------- add producer dataflow ---------- - DataStream stream = env.addSource(new RichParallelSourceFunction() { private boolean running = true; @@ -164,7 +161,6 @@ public void run(SourceContext ctx) throws InterruptedException { int cnt = getRuntimeContext().getIndexOfThisSubtask() * elementsPerPartition; int limit = cnt + elementsPerPartition; - while (running && !stopProducer && cnt < limit) { ctx.collect("element-" + cnt); cnt++; @@ -196,14 +192,13 @@ public void cancel() { kafkaServer.deleteTestTopic(topic); } - private class NonContinousOffsetsDeserializationSchema implements KeyedDeserializationSchema { private int numJumps; long nextExpected = 0; @Override public String deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset) throws IOException { - if(offset != nextExpected) { + if (offset != nextExpected) { numJumps++; nextExpected = offset; LOG.info("Registered now jump at offset {}", offset); @@ -219,7 +214,7 @@ public String deserialize(byte[] messageKey, byte[] message, String topic, int p @Override public boolean isEndOfStream(String nextElement) { - if( numJumps >= 5) { + if (numJumps >= 5) { // we saw 5 jumps and no failures --> consumer can handle auto.offset.reset stopProducer = true; return true; @@ -233,15 +228,14 @@ public TypeInformation getProducedType() { } } - /** - * Ensure that the consumer is properly failing if "auto.offset.reset" is set to "none" + * Ensure that the consumer is properly failing if "auto.offset.reset" is set to "none". * @throws Exception */ public void runFailOnAutoOffsetResetNone() throws Exception { final String topic = "auto-offset-reset-none-test"; final int parallelism = 1; - + kafkaServer.createTestTopic(topic, parallelism, 1); final StreamExecutionEnvironment env = @@ -249,7 +243,7 @@ public void runFailOnAutoOffsetResetNone() throws Exception { env.setParallelism(parallelism); env.setRestartStrategy(RestartStrategies.noRestart()); // fail immediately env.getConfig().disableSysoutLogging(); - + // ----------- add consumer ---------- Properties customProps = new Properties(); @@ -263,10 +257,10 @@ public void runFailOnAutoOffsetResetNone() throws Exception { try { env.execute("Test auto offset reset none"); - } catch(Throwable e) { + } catch (Throwable e) { // check if correct exception has been thrown - if(!e.getCause().getCause().getMessage().contains("Unable to find previous offset") // kafka 0.8 - && !e.getCause().getCause().getMessage().contains("Undefined offset with no reset policy for partition") // kafka 0.9 + if (!e.getCause().getCause().getMessage().contains("Unable to find previous offset") // kafka 0.8 + && !e.getCause().getCause().getMessage().contains("Undefined offset with no reset policy for partition") // kafka 0.9 ) { throw e; } @@ -287,7 +281,7 @@ public void runFailOnAutoOffsetResetNoneEager() throws Exception { customProps.putAll(standardProps); customProps.putAll(secureProps); customProps.setProperty("auto.offset.reset", "none"); // test that "none" leads to an exception - + try { kafkaServer.getConsumer(topic, new SimpleStringSchema(), customProps); fail("should fail with an exception"); diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSinkTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSinkTestBase.java index d4fe9cc1e63a9..dcf316713b9ec 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSinkTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSinkTestBase.java @@ -15,9 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.streaming.connectors.kafka; -import java.util.Properties; +package org.apache.flink.streaming.connectors.kafka; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -27,8 +26,11 @@ import org.apache.flink.streaming.util.serialization.SerializationSchema; import org.apache.flink.table.api.Types; import org.apache.flink.types.Row; + import org.junit.Test; +import java.util.Properties; + import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotSame; @@ -38,6 +40,9 @@ import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; +/** + * Abstract test base for all Kafka table sink tests. + */ public abstract class KafkaTableSinkTestBase { private static final String TOPIC = "testTopic"; @@ -46,7 +51,7 @@ public abstract class KafkaTableSinkTestBase { private static final FlinkKafkaPartitioner PARTITIONER = new CustomPartitioner(); private static final Properties PROPERTIES = createSinkProperties(); @SuppressWarnings("unchecked") - private final FlinkKafkaProducerBase PRODUCER = new FlinkKafkaProducerBase( + private final FlinkKafkaProducerBase producer = new FlinkKafkaProducerBase( TOPIC, new KeyedSerializationSchemaWrapper(getSerializationSchema()), PROPERTIES, PARTITIONER) { @Override @@ -61,7 +66,7 @@ public void testKafkaTableSink() throws Exception { KafkaTableSink kafkaTableSink = spy(createTableSink()); kafkaTableSink.emitDataStream(dataStream); - verify(dataStream).addSink(eq(PRODUCER)); + verify(dataStream).addSink(eq(producer)); verify(kafkaTableSink).createKafkaProducer( eq(TOPIC), @@ -87,7 +92,7 @@ protected abstract KafkaTableSink createTableSink(String topic, Properties prope protected abstract SerializationSchema getSerializationSchema(); private KafkaTableSink createTableSink() { - return createTableSink(TOPIC, PROPERTIES, PARTITIONER, PRODUCER); + return createTableSink(TOPIC, PROPERTIES, PARTITIONER, producer); } private static Properties createSinkProperties() { diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceTestBase.java index 341df45c54248..8028bfc3bfb1c 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceTestBase.java @@ -18,23 +18,29 @@ package org.apache.flink.streaming.connectors.kafka; -import java.util.Properties; -import org.apache.avro.Schema; -import org.apache.avro.specific.SpecificRecordBase; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.connectors.kafka.testutils.AvroTestUtils; +import org.apache.flink.streaming.util.serialization.DeserializationSchema; import org.apache.flink.table.api.Types; import org.apache.flink.types.Row; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.util.serialization.DeserializationSchema; + +import org.apache.avro.Schema; +import org.apache.avro.specific.SpecificRecordBase; import org.junit.Test; + +import java.util.Properties; + import static org.mockito.Matchers.any; import static org.mockito.Matchers.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; +/** + * Abstract test base for all Kafka table sources. + */ public abstract class KafkaTableSourceTestBase { private static final String TOPIC = "testTopic"; @@ -47,10 +53,14 @@ public abstract class KafkaTableSourceTestBase { BasicTypeInfo.LONG_TYPE_INFO }; private static final Properties PROPERTIES = createSourceProperties(); - // Avro record that matches above schema + /** + * Avro record that matches above schema. + */ public static class AvroSpecificRecord extends SpecificRecordBase { + //CHECKSTYLE.OFF: StaticVariableNameCheck - Avro accesses this field by name via reflection. public static Schema SCHEMA$ = AvroTestUtils.createFlatAvroSchema(FIELD_NAMES, FIELD_TYPES); + //CHECKSTYLE.ON: StaticVariableNameCheck public Long mylong; public String mystring; diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java index 1837af695b809..c484a4beefbba 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java @@ -33,17 +33,15 @@ import org.junit.AfterClass; import org.junit.BeforeClass; - import org.junit.ClassRule; import org.junit.rules.TemporaryFolder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import scala.concurrent.duration.FiniteDuration; - import java.util.Properties; import java.util.concurrent.TimeUnit; +import scala.concurrent.duration.FiniteDuration; /** * The base for the Kafka tests. It brings up: @@ -52,7 +50,7 @@ *

  • 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), @@ -62,7 +60,7 @@ public abstract class KafkaTestBase extends TestLogger { protected static final Logger LOG = LoggerFactory.getLogger(KafkaTestBase.class); - + protected static final int NUMBER_OF_KAFKA_SERVERS = 3; protected static final int NUM_TMS = 1; @@ -74,7 +72,7 @@ public abstract class KafkaTestBase extends TestLogger { protected static String brokerConnectionStrings; protected static Properties standardProps; - + protected static LocalFlinkMiniCluster flink; protected static FiniteDuration timeout = new FiniteDuration(10, TimeUnit.SECONDS); @@ -89,7 +87,7 @@ public abstract class KafkaTestBase extends TestLogger { // ------------------------------------------------------------------------ // Setup and teardown of the mini clusters // ------------------------------------------------------------------------ - + @BeforeClass public static void prepare() throws ClassNotFoundException { @@ -162,7 +160,7 @@ protected static void shutdownClusters() { flink.shutdown(); } - if(secureProps != null) { + if (secureProps != null) { secureProps.clear(); } @@ -170,12 +168,9 @@ protected static void shutdownClusters() { } - - // ------------------------------------------------------------------------ // Execution utilities // ------------------------------------------------------------------------ - protected static void tryExecutePropagateExceptions(StreamExecutionEnvironment see, String name) throws Exception { try { @@ -200,7 +195,7 @@ protected static void tryExecutePropagateExceptions(StreamExecutionEnvironment s protected static void createTestTopic(String topic, int numberOfPartitions, int replicationFactor) { kafkaServer.createTestTopic(topic, numberOfPartitions, replicationFactor); } - + protected static void deleteTestTopic(String topic) { kafkaServer.deleteTestTopic(topic); } diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java index 311a1a49eaeda..4df346578491b 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java @@ -17,11 +17,6 @@ package org.apache.flink.streaming.connectors.kafka; -import java.util.Collections; -import java.util.List; -import java.util.Properties; - -import kafka.server.KafkaServer; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.operators.StreamSink; @@ -31,8 +26,14 @@ import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper; import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema; +import kafka.server.KafkaServer; + +import java.util.Collections; +import java.util.List; +import java.util.Properties; + /** - * Abstract class providing a Kafka test environment + * Abstract class providing a Kafka test environment. */ public abstract class KafkaTestEnvironment { @@ -89,9 +90,14 @@ public abstract DataStreamSink produceIntoKafka(DataStream stream, Str // -- offset handlers + /** + * Simple interface to commit and retrieve offsets. + */ public interface KafkaOffsetHandler { Long getCommittedOffset(String topicName, int partition); + void setCommittedOffset(String topicName, int partition, long offset); + void close(); } diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTest.java index c1a64c4bf20b3..c83a97ea00a59 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTest.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTest.java @@ -23,13 +23,14 @@ import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; +import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService; import org.apache.flink.util.SerializedValue; import org.junit.Test; import javax.annotation.Nullable; + import java.util.HashMap; import java.util.Map; @@ -38,6 +39,9 @@ import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.mock; +/** + * Tests for the {@link AbstractFetcher}. + */ @SuppressWarnings("serial") public class AbstractFetcherTest { @@ -191,7 +195,7 @@ public void testPunctuatedWatermarks() throws Exception { final KafkaTopicPartitionState part3 = fetcher.subscribedPartitionStates()[2]; // elements generate a watermark if the timestamp is a multiple of three - + // elements for partition 1 fetcher.emitRecord(1L, part1, 1L); fetcher.emitRecord(2L, part1, 2L); @@ -211,11 +215,11 @@ public void testPunctuatedWatermarks() throws Exception { fetcher.emitRecord(102L, part3, 2L); assertEquals(102L, sourceContext.getLatestElement().getValue().longValue()); assertEquals(102L, sourceContext.getLatestElement().getTimestamp()); - + // now, we should have a watermark assertTrue(sourceContext.hasWatermark()); assertEquals(3L, sourceContext.getLatestWatermark().getTimestamp()); - + // advance partition 3 fetcher.emitRecord(1003L, part3, 3L); fetcher.emitRecord(1004L, part3, 4L); @@ -239,7 +243,7 @@ public void testPunctuatedWatermarks() throws Exception { assertTrue(sourceContext.hasWatermark()); assertEquals(15L, sourceContext.getLatestWatermark().getTimestamp()); } - + @Test public void testPeriodicWatermarks() throws Exception { final String testTopic = "test topic name"; @@ -329,8 +333,7 @@ protected TestFetcher( SerializedValue> watermarksPeriodic, SerializedValue> watermarksPunctuated, ProcessingTimeService processingTimeProvider, - long autoWatermarkInterval) throws Exception - { + long autoWatermarkInterval) throws Exception { super( sourceContext, assignedPartitionsWithStartOffsets, @@ -391,7 +394,6 @@ public void emitWatermark(Watermark mark) { } } - @Override public void markAsTemporarilyIdle() { throw new UnsupportedOperationException(); @@ -412,7 +414,7 @@ public StreamRecord getLatestElement() { public boolean hasWatermark() { return currentWatermark != null; } - + public Watermark getLatestWatermark() throws InterruptedException { synchronized (watermarkLock) { while (currentWatermark == null) { @@ -430,7 +432,7 @@ public Watermark getLatestWatermark() throws InterruptedException { private static class PeriodicTestExtractor implements AssignerWithPeriodicWatermarks { private volatile long maxTimestamp = Long.MIN_VALUE; - + @Override public long extractTimestamp(Long element, long previousElementTimestamp) { maxTimestamp = Math.max(maxTimestamp, element); @@ -456,6 +458,6 @@ public long extractTimestamp(Long element, long previousElementTimestamp) { public Watermark checkAndGetNextWatermark(Long lastElement, long extractedTimestamp) { return extractedTimestamp % 3 == 0 ? new Watermark(extractedTimestamp) : null; } - + } } diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionTest.java index b215bd3a85cd0..4496a26e3c77b 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionTest.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionTest.java @@ -27,8 +27,11 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +/** + * Tests for the {@link KafkaTopicPartition}. + */ public class KafkaTopicPartitionTest { - + @Test public void validateUid() { Field uidField; @@ -40,14 +43,14 @@ public void validateUid() { fail("serialVersionUID is not defined"); return; } - + assertTrue(Modifier.isStatic(uidField.getModifiers())); assertTrue(Modifier.isFinal(uidField.getModifiers())); assertTrue(Modifier.isPrivate(uidField.getModifiers())); - + assertEquals(long.class, uidField.getType()); - - // the UID has to be constant to make sure old checkpoints/savepoints can be read + + // the UID has to be constant to make sure old checkpoints/savepoints can be read try { assertEquals(722083576322742325L, uidField.getLong(null)); } diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/AvroTestUtils.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/AvroTestUtils.java index 075b79b8e93a6..a41125a5a3dfd 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/AvroTestUtils.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/AvroTestUtils.java @@ -18,13 +18,6 @@ package org.apache.flink.streaming.connectors.kafka.testutils; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import org.apache.avro.Schema; -import org.apache.avro.SchemaBuilder; -import org.apache.avro.reflect.ReflectData; -import org.apache.avro.specific.SpecificRecord; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.io.avro.generated.Address; import org.apache.flink.api.io.avro.generated.Colors; @@ -32,12 +25,21 @@ import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.types.Row; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.apache.avro.reflect.ReflectData; +import org.apache.avro.specific.SpecificRecord; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; + /** * Utilities for creating Avro Schemas. */ public final class AvroTestUtils { - private static String NAMESPACE = "org.apache.flink.streaming.connectors.kafka"; + private static final String NAMESPACE = "org.apache.flink.streaming.connectors.kafka"; /** * Creates a flat Avro Schema for testing. diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java index c0fb83631c693..b204ea991050b 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java @@ -18,10 +18,6 @@ package org.apache.flink.streaming.connectors.kafka.testutils; -import java.util.Collection; -import java.util.Properties; -import java.util.Random; - import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.restartstrategy.RestartStrategies; @@ -43,14 +39,22 @@ import org.apache.flink.streaming.util.serialization.SimpleStringSchema; import org.apache.flink.streaming.util.serialization.TypeInformationSerializationSchema; +import java.util.Collection; +import java.util.Properties; +import java.util.Random; + +/** + * Test data generators. + */ @SuppressWarnings("serial") public class DataGenerators { - public static void generateRandomizedIntegerSequence(StreamExecutionEnvironment env, - KafkaTestEnvironment testServer, String topic, - final int numPartitions, - final int numElements, - final boolean randomizeOrder) throws Exception { + public static void generateRandomizedIntegerSequence( + StreamExecutionEnvironment env, + KafkaTestEnvironment testServer, String topic, + final int numPartitions, + final int numElements, + final boolean randomizeOrder) throws Exception { env.setParallelism(numPartitions); env.getConfig().disableSysoutLogging(); env.setRestartStrategy(RestartStrategies.noRestart()); @@ -65,8 +69,8 @@ public void run(SourceContext ctx) { // create a sequence int[] elements = new int[numElements]; for (int i = 0, val = getRuntimeContext().getIndexOfThisSubtask(); - i < numElements; - i++, val += getRuntimeContext().getNumberOfParallelSubtasks()) { + i < numElements; + i++, val += getRuntimeContext().getNumberOfParallelSubtasks()) { elements[i] = val; } @@ -99,7 +103,7 @@ public void cancel() { Properties props = new Properties(); props.putAll(FlinkKafkaProducerBase.getPropertiesFromBrokerList(testServer.getBrokerConnectionString())); Properties secureProps = testServer.getSecureProperties(); - if(secureProps != null) { + if (secureProps != null) { props.putAll(testServer.getSecureProperties()); } @@ -119,6 +123,10 @@ public int partition(Integer next, byte[] serializedKey, byte[] serializedValue, // ------------------------------------------------------------------------ + /** + * A generator that continuously writes strings into the configured topic. The generation is stopped if an exception + * occurs or {@link #shutdown()} is called. + */ public static class InfiniteStringsGenerator extends Thread { private final KafkaTestEnvironment server; @@ -129,7 +137,6 @@ public static class InfiniteStringsGenerator extends Thread { private volatile boolean running = true; - public InfiniteStringsGenerator(KafkaTestEnvironment server, String topic) { this.server = server; this.topic = topic; @@ -164,7 +171,7 @@ public void run() { int len = rnd.nextInt(100) + 1; for (int i = 0; i < len; i++) { - bld.append((char) (rnd.nextInt(20) + 'a') ); + bld.append((char) (rnd.nextInt(20) + 'a')); } String next = bld.toString(); @@ -211,7 +218,7 @@ public Collection> getTransitivePredecessors() { } } - public static class DummyStreamExecutionEnvironment extends StreamExecutionEnvironment { + private static class DummyStreamExecutionEnvironment extends StreamExecutionEnvironment { @Override public JobExecutionResult execute(String jobName) throws Exception { diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FailingIdentityMapper.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FailingIdentityMapper.java index ec64b006fe199..c25eefb247d1f 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FailingIdentityMapper.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FailingIdentityMapper.java @@ -22,30 +22,35 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.state.CheckpointListener; import org.apache.flink.streaming.api.checkpoint.ListCheckpointed; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.Collections; import java.util.List; - -public class FailingIdentityMapper extends RichMapFunction implements +/** + * A {@link RichMapFunction} that fails after the configured number of records have been processed. + * + * @param + */ +public class FailingIdentityMapper extends RichMapFunction implements ListCheckpointed, CheckpointListener, Runnable { - + private static final Logger LOG = LoggerFactory.getLogger(FailingIdentityMapper.class); - + private static final long serialVersionUID = 6334389850158707313L; - + public static volatile boolean failedBefore; public static volatile boolean hasBeenCheckpointedBeforeFailure; private final int failCount; private int numElementsTotal; private int numElementsThisTime; - + private boolean failer; private boolean hasBeenCheckpointed; - + private Thread printer; private volatile boolean printerRunning = true; @@ -64,10 +69,10 @@ public void open(Configuration parameters) { public T map(T value) throws Exception { numElementsTotal++; numElementsThisTime++; - + if (!failedBefore) { Thread.sleep(10); - + if (failer && numElementsTotal >= failCount) { hasBeenCheckpointedBeforeFailure = hasBeenCheckpointed; failedBefore = true; diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FakeStandardProducerConfig.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FakeStandardProducerConfig.java index 055326d9e5d70..0fbe55480a75b 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FakeStandardProducerConfig.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FakeStandardProducerConfig.java @@ -15,12 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.flink.streaming.connectors.kafka.testutils; import org.apache.kafka.common.serialization.ByteArraySerializer; import java.util.Properties; +/** + * Test configuration for a kafka producer. + */ public class FakeStandardProducerConfig { public static Properties get() { diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/JobManagerCommunicationUtils.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/JobManagerCommunicationUtils.java index 131325f4237f9..9bbe1d31b386f 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/JobManagerCommunicationUtils.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/JobManagerCommunicationUtils.java @@ -23,13 +23,16 @@ import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.messages.JobManagerMessages; +import java.util.List; +import java.util.concurrent.TimeUnit; + import scala.concurrent.Await; import scala.concurrent.Future; import scala.concurrent.duration.FiniteDuration; -import java.util.List; -import java.util.concurrent.TimeUnit; - +/** + * Utilities for communicating with a jobmanager through a {@link ActorGateway}. + */ public class JobManagerCommunicationUtils { private static final FiniteDuration askTimeout = new FiniteDuration(30, TimeUnit.SECONDS); @@ -43,7 +46,6 @@ public static void waitUntilNoJobIsRunning(ActorGateway jobManager) throws Excep Object result = Await.result(listResponse, askTimeout); List jobs = ((JobManagerMessages.RunningJobsStatus) result).getStatusMessages(); - if (jobs.isEmpty()) { return; } @@ -84,13 +86,13 @@ public static void cancelCurrentJob(ActorGateway jobManager) throws Exception { public static void cancelCurrentJob(ActorGateway jobManager, String name) throws Exception { JobStatusMessage status = null; - + for (int i = 0; i < 200; i++) { // find the jobID Future listResponse = jobManager.ask( JobManagerMessages.getRequestRunningJobsStatus(), askTimeout); - + List jobs; try { Object result = Await.result(listResponse, askTimeout); @@ -99,7 +101,7 @@ public static void cancelCurrentJob(ActorGateway jobManager, String name) throws catch (Exception e) { throw new Exception("Could not cancel job - failed to retrieve running jobs from the JobManager.", e); } - + if (jobs.isEmpty()) { // try again, fall through the loop Thread.sleep(50); @@ -107,33 +109,33 @@ public static void cancelCurrentJob(ActorGateway jobManager, String name) throws else if (jobs.size() == 1) { status = jobs.get(0); } - else if(name != null) { - for(JobStatusMessage msg: jobs) { - if(msg.getJobName().equals(name)) { + else if (name != null) { + for (JobStatusMessage msg: jobs) { + if (msg.getJobName().equals(name)) { status = msg; } } - if(status == null) { - throw new Exception("Could not cancel job - no job matched expected name = '" + name +"' in " + jobs); + if (status == null) { + throw new Exception("Could not cancel job - no job matched expected name = '" + name + "' in " + jobs); } } else { String jobNames = ""; - for(JobStatusMessage jsm: jobs) { + for (JobStatusMessage jsm: jobs) { jobNames += jsm.getJobName() + ", "; } throw new Exception("Could not cancel job - more than one running job: " + jobNames); } } - + if (status == null) { - throw new Exception("Could not cancel job - no running jobs"); + throw new Exception("Could not cancel job - no running jobs"); } else if (status.getJobState().isGloballyTerminalState()) { throw new Exception("Could not cancel job - job is not running any more"); } - + JobID jobId = status.getJobId(); - + Future response = jobManager.ask(new JobManagerMessages.CancelJob(jobId), askTimeout); try { Await.result(response, askTimeout); diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/PartitionValidatingMapper.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/PartitionValidatingMapper.java index e105e0183a219..29e469dc687d3 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/PartitionValidatingMapper.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/PartitionValidatingMapper.java @@ -23,14 +23,16 @@ import java.util.HashSet; import java.util.Set; - +/** + * {@link MapFunction} that verifies that he partitioning is identical. + */ public class PartitionValidatingMapper implements MapFunction { private static final long serialVersionUID = 1088381231244959088L; - + /* the partitions from which this function received data */ private final Set myPartitions = new HashSet<>(); - + private final int numPartitions; private final int maxPartitions; diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ThrottledMapper.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ThrottledMapper.java index 1d61229037784..040f15c954bb1 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ThrottledMapper.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ThrottledMapper.java @@ -23,10 +23,10 @@ /** * An identity map function that sleeps between elements, throttling the * processing speed. - * + * * @param The type mapped. */ -public class ThrottledMapper implements MapFunction { +public class ThrottledMapper implements MapFunction { private static final long serialVersionUID = 467008933767159126L; @@ -41,4 +41,4 @@ public T map(T value) throws Exception { Thread.sleep(this.sleep); return value; } -} \ No newline at end of file +} diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/Tuple2FlinkPartitioner.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/Tuple2FlinkPartitioner.java index e7fff523fdde6..6f2c4a10a5b60 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/Tuple2FlinkPartitioner.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/Tuple2FlinkPartitioner.java @@ -33,13 +33,13 @@ public class Tuple2FlinkPartitioner extends FlinkKafkaPartitioner next, byte[] key, byte[] value, String targetTopic, int[] partitions) { if (partitions.length != expectedPartitions) { throw new IllegalArgumentException("Expected " + expectedPartitions + " partitions"); } - + return next.f0; } } diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ValidatingExactlyOnceSink.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ValidatingExactlyOnceSink.java index 46e70fd6c6d93..5ace012708436 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ValidatingExactlyOnceSink.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ValidatingExactlyOnceSink.java @@ -22,6 +22,7 @@ import org.apache.flink.streaming.api.checkpoint.ListCheckpointed; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; import org.apache.flink.test.util.SuccessException; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -29,14 +30,17 @@ import java.util.Collections; import java.util.List; +/** + * A {@link RichSinkFunction} that verifies that no duplicate records are generated. + */ public class ValidatingExactlyOnceSink extends RichSinkFunction implements ListCheckpointed> { private static final Logger LOG = LoggerFactory.getLogger(ValidatingExactlyOnceSink.class); private static final long serialVersionUID = 1748426382527469932L; - + private final int numElementsTotal; - + private BitSet duplicateChecker = new BitSet(); // this is checkpointed private int numElements; // this is checkpointed @@ -45,11 +49,10 @@ public ValidatingExactlyOnceSink(int numElementsTotal) { this.numElementsTotal = numElementsTotal; } - @Override public void invoke(Integer value) throws Exception { numElements++; - + if (duplicateChecker.get(value)) { throw new Exception("Received a duplicate: " + value); } diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ZooKeeperStringSerializer.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ZooKeeperStringSerializer.java index 37ed4088a7634..9aa12073449e3 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ZooKeeperStringSerializer.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ZooKeeperStringSerializer.java @@ -18,9 +18,10 @@ package org.apache.flink.streaming.connectors.kafka.testutils; -import org.I0Itec.zkclient.serialize.ZkSerializer; import org.apache.flink.configuration.ConfigConstants; +import org.I0Itec.zkclient.serialize.ZkSerializer; + /** * Simple ZooKeeper serializer for Strings. */ diff --git a/flink-connectors/flink-connector-kafka-base/src/test/resources/log4j-test.properties b/flink-connectors/flink-connector-kafka-base/src/test/resources/log4j-test.properties index 16c226fc1bba1..a3fb2b02ed5d3 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/resources/log4j-test.properties +++ b/flink-connectors/flink-connector-kafka-base/src/test/resources/log4j-test.properties @@ -26,4 +26,3 @@ 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 -