From 1c2b0b7c61f9d46474601083c81f5b5731bddff5 Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Wed, 16 Dec 2015 17:29:42 +0100 Subject: [PATCH] [FLINK-3058] Add support for Kafka 0.9.0.0 For adding Kafka 0.9.0.0 support, this commit changes the following: - Split up of the kafka connector into a flink-connector-kafka-(base|0.9|0.8) with different dependencies - The base package contains common test cases, classes and implementations (the producer for 0.9 and 0.8 relies on exactly the same code) - the 0.8 package contains a kafka connector implementation against the SimpleConsumer (low level) API of Kafka 0.8. There are some additional tests for the ZK offset committing - The 0.9 package relies on the new Consumer API of Kafka 0.9.0.0 - Support for metrics for all producers and the 0.9 consumer through Flink's accumulators. --- .gitignore | 1 + docs/apis/streaming/connectors/kafka.md | 67 ++- docs/apis/streaming/index.md | 4 +- .../api/java/typeutils/TypeExtractor.java | 4 +- .../webmonitor/RuntimeMonitorHandler.java | 6 + .../pom.xml | 31 +- .../kafka/FlinkKafkaConsumer08.java} | 439 ++++----------- .../kafka/FlinkKafkaConsumer081.java | 33 +- .../kafka/FlinkKafkaConsumer082.java | 39 ++ .../connectors/kafka/FlinkKafkaProducer.java | 64 +++ .../kafka/FlinkKafkaProducer08.java | 128 +++++ .../connectors/kafka/internals/Fetcher.java | 0 .../kafka/internals/LegacyFetcher.java | 16 +- .../kafka/internals/OffsetHandler.java | 0 .../kafka/internals/PartitionerWrapper.java | 0 .../internals/ZookeeperOffsetHandler.java | 7 +- .../connectors/kafka/Kafka08ITCase.java | 266 ++++++++++ .../kafka/Kafka08ProducerITCase.java | 33 ++ .../connectors/kafka/KafkaConsumerTest.java | 21 +- .../kafka/KafkaLocalSystemTime.java | 0 .../connectors/kafka/KafkaProducerTest.java | 10 +- .../kafka/KafkaTestEnvironmentImpl.java} | 354 ++++++------- .../internals/ZookeeperOffsetHandlerTest.java | 17 +- .../src/test/resources/log4j-test.properties | 0 .../src/test/resources/logback-test.xml | 0 .../flink-connector-kafka-0.9/pom.xml | 131 +++++ .../kafka/FlinkKafkaConsumer09.java | 501 ++++++++++++++++++ .../kafka/FlinkKafkaProducer09.java | 130 +++++ .../kafka/examples/ReadFromKafka.java | 56 ++ .../kafka/examples/WriteIntoKafka.java | 70 +++ .../src/main/resources/log4j.properties | 29 + .../connectors/kafka/Kafka09ITCase.java} | 65 +-- .../kafka/Kafka09ProducerITCase.java | 33 ++ .../connectors/kafka/KafkaProducerTest.java | 115 ++++ .../kafka/KafkaTestEnvironmentImpl.java | 340 ++++++++++++ .../src/test/resources/log4j-test.properties | 29 + .../src/test/resources/logback-test.xml | 30 ++ .../flink-connector-kafka-base/pom.xml | 169 ++++++ .../kafka/FlinkKafkaConsumerBase.java | 225 ++++++++ .../kafka/FlinkKafkaProducerBase.java} | 157 ++---- .../kafka/internals/KafkaTopicPartition.java | 8 + .../internals/KafkaTopicPartitionLeader.java | 0 .../internals/ZooKeeperStringSerializer.java | 0 .../metrics/AvgKafkaMetricAccumulator.java | 141 +++++ .../DefaultKafkaMetricAccumulator.java | 159 ++++++ .../metrics/MaxKafkaMetricAccumulator.java | 57 ++ .../metrics/MinKafkaMetricAccumulator.java | 57 ++ .../kafka/partitioner/FixedPartitioner.java | 4 +- .../kafka/partitioner/KafkaPartitioner.java | 9 +- .../KeyedDeserializationSchema.java | 6 +- .../KeyedDeserializationSchemaWrapper.java | 2 +- .../KeyedSerializationSchema.java | 0 .../KeyedSerializationSchemaWrapper.java | 0 ...nformationKeyValueSerializationSchema.java | 2 +- .../KafkaConsumerPartitionAssignmentTest.java | 22 +- .../kafka/KafkaConsumerTestBase.java | 424 ++++++--------- .../kafka/KafkaProducerTestBase.java} | 34 +- .../connectors/kafka/KafkaTestBase.java | 170 ++++++ .../kafka/KafkaTestEnvironment.java | 83 +++ .../kafka/TestFixedPartitioner.java | 30 +- .../kafka/testutils/DataGenerators.java | 49 +- .../kafka/testutils/DiscardingSink.java | 0 .../testutils/FailingIdentityMapper.java | 0 .../JobManagerCommunicationUtils.java | 0 .../kafka/testutils/MockRuntimeContext.java | 2 +- .../testutils/PartitionValidatingMapper.java | 0 .../kafka/testutils/ThrottledMapper.java | 0 .../kafka/testutils/Tuple2Partitioner.java | 8 +- .../testutils/ValidatingExactlyOnceSink.java | 7 +- .../src/test/resources/log4j-test.properties | 29 + .../src/test/resources/logback-test.xml | 30 ++ .../kafka/FlinkKafkaConsumer082.java | 85 --- flink-streaming-connectors/pom.xml | 4 +- .../api/environment/CheckpointConfig.java | 2 +- .../TypeInformationSerializationSchema.java | 1 - ...EventTimeAllWindowCheckpointingITCase.java | 29 +- .../EventTimeWindowCheckpointingITCase.java | 29 +- .../WindowCheckpointingITCase.java | 27 +- .../flink/test/util}/SuccessException.java | 2 +- .../org/apache/flink/test/util/TestUtils.java | 52 ++ pom.xml | 1 + 81 files changed, 3894 insertions(+), 1291 deletions(-) rename flink-streaming-connectors/{flink-connector-kafka => flink-connector-kafka-0.8}/pom.xml (85%) rename flink-streaming-connectors/{flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer.java => flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java} (60%) rename flink-streaming-connectors/{flink-connector-kafka => flink-connector-kafka-0.8}/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer081.java (51%) create mode 100644 flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer082.java create mode 100644 flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java create mode 100644 flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer08.java rename flink-streaming-connectors/{flink-connector-kafka => flink-connector-kafka-0.8}/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Fetcher.java (100%) rename flink-streaming-connectors/{flink-connector-kafka => flink-connector-kafka-0.8}/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java (98%) rename flink-streaming-connectors/{flink-connector-kafka => flink-connector-kafka-0.8}/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/OffsetHandler.java (100%) rename flink-streaming-connectors/{flink-connector-kafka => flink-connector-kafka-0.8}/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PartitionerWrapper.java (100%) rename flink-streaming-connectors/{flink-connector-kafka => flink-connector-kafka-0.8}/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java (96%) create mode 100644 flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java create mode 100644 flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ProducerITCase.java rename flink-streaming-connectors/{flink-connector-kafka => flink-connector-kafka-0.8}/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTest.java (83%) rename flink-streaming-connectors/{flink-connector-kafka => flink-connector-kafka-0.8}/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaLocalSystemTime.java (100%) rename flink-streaming-connectors/{flink-connector-kafka => flink-connector-kafka-0.8}/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java (94%) rename flink-streaming-connectors/{flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java => flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java} (56%) rename flink-streaming-connectors/{flink-connector-kafka => flink-connector-kafka-0.8}/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandlerTest.java (77%) rename flink-streaming-connectors/{flink-connector-kafka => flink-connector-kafka-0.8}/src/test/resources/log4j-test.properties (100%) rename flink-streaming-connectors/{flink-connector-kafka => flink-connector-kafka-0.8}/src/test/resources/logback-test.xml (100%) create mode 100644 flink-streaming-connectors/flink-connector-kafka-0.9/pom.xml create mode 100644 flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java create mode 100644 flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer09.java create mode 100644 flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/examples/ReadFromKafka.java create mode 100644 flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/examples/WriteIntoKafka.java create mode 100644 flink-streaming-connectors/flink-connector-kafka-0.9/src/main/resources/log4j.properties rename flink-streaming-connectors/{flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java => flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ITCase.java} (75%) create mode 100644 flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ProducerITCase.java create mode 100644 flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java create mode 100644 flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java create mode 100644 flink-streaming-connectors/flink-connector-kafka-0.9/src/test/resources/log4j-test.properties create mode 100644 flink-streaming-connectors/flink-connector-kafka-0.9/src/test/resources/logback-test.xml create mode 100644 flink-streaming-connectors/flink-connector-kafka-base/pom.xml create mode 100644 flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java rename flink-streaming-connectors/{flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java => flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBase.java} (63%) rename flink-streaming-connectors/{flink-connector-kafka => flink-connector-kafka-base}/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java (92%) rename flink-streaming-connectors/{flink-connector-kafka => flink-connector-kafka-base}/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionLeader.java (100%) rename flink-streaming-connectors/{flink-connector-kafka => flink-connector-kafka-base}/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZooKeeperStringSerializer.java (100%) create mode 100644 flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/AvgKafkaMetricAccumulator.java create mode 100644 flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/DefaultKafkaMetricAccumulator.java create mode 100644 flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/MaxKafkaMetricAccumulator.java create mode 100644 flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/MinKafkaMetricAccumulator.java rename flink-streaming-connectors/{flink-connector-kafka => flink-connector-kafka-base}/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FixedPartitioner.java (92%) rename flink-streaming-connectors/{flink-connector-kafka => flink-connector-kafka-base}/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/KafkaPartitioner.java (87%) rename {flink-streaming-java => flink-streaming-connectors/flink-connector-kafka-base}/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchema.java (91%) rename {flink-streaming-java => flink-streaming-connectors/flink-connector-kafka-base}/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchemaWrapper.java (97%) rename {flink-streaming-java => flink-streaming-connectors/flink-connector-kafka-base}/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchema.java (100%) rename {flink-streaming-java => flink-streaming-connectors/flink-connector-kafka-base}/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchemaWrapper.java (100%) rename {flink-streaming-java => flink-streaming-connectors/flink-connector-kafka-base}/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationKeyValueSerializationSchema.java (98%) rename flink-streaming-connectors/{flink-connector-kafka => flink-connector-kafka-base}/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerPartitionAssignmentTest.java (91%) rename flink-streaming-connectors/{flink-connector-kafka => flink-connector-kafka-base}/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java (76%) rename flink-streaming-connectors/{flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerITCase.java => flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java} (87%) create mode 100644 flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java create mode 100644 flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java rename flink-streaming-connectors/{flink-connector-kafka => flink-connector-kafka-base}/src/test/java/org/apache/flink/streaming/connectors/kafka/TestFixedPartitioner.java (63%) rename flink-streaming-connectors/{flink-connector-kafka => flink-connector-kafka-base}/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java (75%) rename flink-streaming-connectors/{flink-connector-kafka => flink-connector-kafka-base}/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DiscardingSink.java (100%) rename flink-streaming-connectors/{flink-connector-kafka => flink-connector-kafka-base}/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FailingIdentityMapper.java (100%) rename flink-streaming-connectors/{flink-connector-kafka => flink-connector-kafka-base}/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/JobManagerCommunicationUtils.java (100%) rename flink-streaming-connectors/{flink-connector-kafka => flink-connector-kafka-base}/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockRuntimeContext.java (99%) rename flink-streaming-connectors/{flink-connector-kafka => flink-connector-kafka-base}/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/PartitionValidatingMapper.java (100%) rename flink-streaming-connectors/{flink-connector-kafka => flink-connector-kafka-base}/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ThrottledMapper.java (100%) rename flink-streaming-connectors/{flink-connector-kafka => flink-connector-kafka-base}/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/Tuple2Partitioner.java (85%) rename flink-streaming-connectors/{flink-connector-kafka => flink-connector-kafka-base}/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ValidatingExactlyOnceSink.java (89%) create mode 100644 flink-streaming-connectors/flink-connector-kafka-base/src/test/resources/log4j-test.properties create mode 100644 flink-streaming-connectors/flink-connector-kafka-base/src/test/resources/logback-test.xml delete mode 100644 flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer082.java rename {flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils => flink-tests/src/test/java/org/apache/flink/test/util}/SuccessException.java (94%) create mode 100644 flink-tests/src/test/java/org/apache/flink/test/util/TestUtils.java diff --git a/.gitignore b/.gitignore index 8e30de7b246ef..a73a9d34b2cc9 100644 --- a/.gitignore +++ b/.gitignore @@ -23,3 +23,4 @@ flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/gen flink-runtime-web/web-dashboard/node_modules/ flink-runtime-web/web-dashboard/bower_components/ atlassian-ide-plugin.xml +out/ diff --git a/docs/apis/streaming/connectors/kafka.md b/docs/apis/streaming/connectors/kafka.md index 9d9d6ba744918..f0757cecf10d3 100644 --- a/docs/apis/streaming/connectors/kafka.md +++ b/docs/apis/streaming/connectors/kafka.md @@ -34,14 +34,15 @@ exactly-once processing semantics. To achieve that, Flink does not purely rely o offset tracking, but tracks and checkpoints these offsets internally as well. Please pick a package (maven artifact id) and class name for your use-case and environment. -For most users, the `FlinkKafkaConsumer082` (part of `flink-connector-kafka`) is appropriate. +For most users, the `FlinkKafkaConsumer08` (part of `flink-connector-kafka`) is appropriate. - + @@ -50,17 +51,27 @@ For most users, the `FlinkKafkaConsumer082` (part of `flink-connector-kafka`) is - - + + - - - - - + + + + + + + + + + + +
Maven Dependency Supported sinceClass nameConsumer and
+ Producer Class name
Kafka version Notes
flink-connector-kafka 0.9.1, 0.10FlinkKafkaConsumer0810.8.1FlinkKafkaConsumer082
+ FlinkKafkaProducer
0.8.x Uses the SimpleConsumer API of Kafka internally. Offsets are committed to ZK by Flink.
flink-connector-kafka0.9.1, 0.10FlinkKafkaConsumer0820.8.2
flink-connector-kafka-0.81.0.0FlinkKafkaConsumer08
+ FlinkKafkaProducer08
0.8.x Uses the SimpleConsumer API of Kafka internally. Offsets are committed to ZK by Flink.
flink-connector-kafka-0.91.0.0FlinkKafkaConsumer09
+ FlinkKafkaProducer09
0.9.xUses the new Consumer API Kafka.
@@ -69,7 +80,7 @@ Then, import the connector in your maven project: {% highlight xml %} org.apache.flink - flink-connector-kafka + flink-connector-kafka-0.8 {{site.version }} {% endhighlight %} @@ -84,14 +95,16 @@ Note that the streaming connectors are currently not part of the binary distribu #### Kafka Consumer -The standard `FlinkKafkaConsumer082` is a Kafka consumer providing access to one topic. It takes the following parameters to the constructor: +Flink's Kafka consumer is called `FlinkKafkaConsumer08` (or `09`). It provides access to one or more Kafka topics. + +The constructor accepts the following arguments: -1. The topic name -2. A DeserializationSchema +1. The topic name / list of topic names +2. A DeserializationSchema / KeyedDeserializationSchema for deserializing the data from Kafka 3. Properties for the Kafka consumer. The following properties are required: - "bootstrap.servers" (comma separated list of Kafka brokers) - - "zookeeper.connect" (comma separated list of Zookeeper servers) + - "zookeeper.connect" (comma separated list of Zookeeper servers) (**only required for Kafka 0.8**) - "group.id" the id of the consumer group Example: @@ -101,10 +114,11 @@ Example: {% highlight java %} Properties properties = new Properties(); properties.setProperty("bootstrap.servers", "localhost:9092"); +// only required for Kafka 0.8 properties.setProperty("zookeeper.connect", "localhost:2181"); properties.setProperty("group.id", "test"); DataStream stream = env - .addSource(new FlinkKafkaConsumer082<>("topic", new SimpleStringSchema(), properties)) + .addSource(new FlinkKafkaConsumer08<>("topic", new SimpleStringSchema(), properties)) .print(); {% endhighlight %} @@ -112,15 +126,28 @@ DataStream stream = env {% highlight scala %} val properties = new Properties(); properties.setProperty("bootstrap.servers", "localhost:9092"); +// only required for Kafka 0.8 properties.setProperty("zookeeper.connect", "localhost:2181"); properties.setProperty("group.id", "test"); stream = env - .addSource(new FlinkKafkaConsumer082[String]("topic", new SimpleStringSchema(), properties)) + .addSource(new FlinkKafkaConsumer08[String]("topic", new SimpleStringSchema(), properties)) .print {% endhighlight %} + +##### The `DeserializationSchema` + +The `FlinkKafkaConsumer08` needs to know how to turn the data in Kafka into Java objects. The +`DeserializationSchema` allows users to specify such a schema. The `T deserialize(byte[] message)` +method gets called for each Kafka message, passing the value from Kafka. +For accessing both the key and value of the Kafka message, the `KeyedDeserializationSchema` has +the following deserialize method ` T deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset)`. + +For convenience, Flink provides a `TypeInformationSerializationSchema` (and `TypeInformationKeyValueSerializationSchema`) +which creates a schema based on a Flink `TypeInformation`. + #### Kafka Consumers and Fault Tolerance With Flink's checkpointing enabled, the Flink Kafka Consumer will consume records from a topic and periodically checkpoint all @@ -155,20 +182,20 @@ If checkpointing is not enabled, the Kafka consumer will periodically commit the #### Kafka Producer -The `FlinkKafkaProducer` writes data to a Kafka topic. The producer can specify a custom partitioner that assigns -recors to partitions. +The `FlinkKafkaProducer08` writes data to a Kafka topic. The producer can specify a custom partitioner that assigns +records to partitions. Example:
{% highlight java %} -stream.addSink(new FlinkKafkaProducer("localhost:9092", "my-topic", new SimpleStringSchema())); +stream.addSink(new FlinkKafkaProducer08("localhost:9092", "my-topic", new SimpleStringSchema())); {% endhighlight %}
{% highlight scala %} -stream.addSink(new FlinkKafkaProducer[String]("localhost:9092", "my-topic", new SimpleStringSchema())) +stream.addSink(new FlinkKafkaProducer08[String]("localhost:9092", "my-topic", new SimpleStringSchema())) {% endhighlight %}
diff --git a/docs/apis/streaming/index.md b/docs/apis/streaming/index.md index 06c0014c99b3d..9d2481c07a544 100644 --- a/docs/apis/streaming/index.md +++ b/docs/apis/streaming/index.md @@ -1630,7 +1630,7 @@ Collection-based: Custom: - `addSource` - Attache a new source function. For example, to read from Apache Kafka you can use - `addSource(new FlinkKafkaConsumer082<>(...))`. See [connectors]({{ site.baseurl }}/apis/streaming/connectors/) for more details. + `addSource(new FlinkKafkaConsumer08<>(...))`. See [connectors]({{ site.baseurl }}/apis/streaming/connectors/) for more details. @@ -1682,7 +1682,7 @@ Collection-based: Custom: - `addSource` - Attache a new source function. For example, to read from Apache Kafka you can use - `addSource(new FlinkKafkaConsumer082<>(...))`. See [connectors]({{ site.baseurl }}/apis/streaming/connectors/) for more details. + `addSource(new FlinkKafkaConsumer08<>(...))`. See [connectors]({{ site.baseurl }}/apis/streaming/connectors/) for more details. diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java index 1a4ccae16a2b7..ddb4a488223b3 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java @@ -820,7 +820,7 @@ private static void validateInputType(Type t, TypeInformation inType) { validateInfo(typeHierarchy, t, inType); } catch(InvalidTypesException e) { - throw new InvalidTypesException("Input mismatch: " + e.getMessage()); + throw new InvalidTypesException("Input mismatch: " + e.getMessage(), e); } } @@ -840,7 +840,7 @@ private static void validateInputType(Class baseClass, Class clazz, int in validateInfo(typeHierarchy, inType, inTypeInfo); } catch(InvalidTypesException e) { - throw new InvalidTypesException("Input mismatch: " + e.getMessage()); + throw new InvalidTypesException("Input mismatch: " + e.getMessage(), e); } } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java index 414343473e2da..c304abb6faa8d 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java @@ -36,6 +36,8 @@ import org.apache.flink.runtime.webmonitor.handlers.RequestHandler; import org.apache.flink.util.ExceptionUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import scala.Option; import scala.Tuple2; import scala.concurrent.Await; @@ -58,6 +60,8 @@ @ChannelHandler.Sharable public class RuntimeMonitorHandler extends SimpleChannelInboundHandler { + private static final Logger LOG = LoggerFactory.getLogger(RuntimeMonitorHandler.class); + private static final Charset ENCODING = Charset.forName("UTF-8"); public static final String WEB_MONITOR_ADDRESS_KEY = "web.monitor.address"; @@ -143,12 +147,14 @@ private void respondAsLeader(ChannelHandlerContext ctx, Routed routed, ActorGate : Unpooled.wrappedBuffer(e.getMessage().getBytes(ENCODING)); response = new DefaultFullHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.NOT_FOUND, message); response.headers().set(HttpHeaders.Names.CONTENT_TYPE, "text/plain"); + LOG.warn("Error while handling request", e); } catch (Exception e) { byte[] bytes = ExceptionUtils.stringifyException(e).getBytes(ENCODING); response = new DefaultFullHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.INTERNAL_SERVER_ERROR, Unpooled.wrappedBuffer(bytes)); response.headers().set(HttpHeaders.Names.CONTENT_TYPE, "text/plain"); + LOG.warn("Error while handling request", e); } response.headers().set(HttpHeaders.Names.CONTENT_ENCODING, "utf-8"); diff --git a/flink-streaming-connectors/flink-connector-kafka/pom.xml b/flink-streaming-connectors/flink-connector-kafka-0.8/pom.xml similarity index 85% rename from flink-streaming-connectors/flink-connector-kafka/pom.xml rename to flink-streaming-connectors/flink-connector-kafka-0.8/pom.xml index 7bd9bcbf83e1f..aae484746ec7f 100644 --- a/flink-streaming-connectors/flink-connector-kafka/pom.xml +++ b/flink-streaming-connectors/flink-connector-kafka-0.8/pom.xml @@ -30,8 +30,8 @@ under the License. .. - flink-connector-kafka - flink-connector-kafka + flink-connector-kafka-0.8 + flink-connector-kafka-0.8 jar @@ -42,6 +42,21 @@ under the License. + + org.apache.flink + flink-connector-kafka-base + ${project.version} + + + + org.apache.flink + flink-connector-kafka-base + ${project.version} + test-jar + test + + + org.apache.flink flink-streaming-java @@ -92,9 +107,6 @@ under the License. - - - com.google.guava guava @@ -121,9 +133,16 @@ under the License. test + + org.apache.flink + flink-tests + ${project.version} + test-jar + test + + - diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java similarity index 60% rename from flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer.java rename to flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java index 69ed9bfaa18ec..543e0ff1a5e2c 100644 --- a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java @@ -23,14 +23,8 @@ import kafka.javaapi.TopicMetadata; import kafka.javaapi.TopicMetadataRequest; import kafka.javaapi.consumer.SimpleConsumer; -import org.apache.commons.collections.map.LinkedMap; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.typeutils.ResultTypeQueryable; import org.apache.flink.configuration.Configuration; -import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier; -import org.apache.flink.streaming.api.checkpoint.CheckpointedAsynchronously; -import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.connectors.kafka.internals.Fetcher; @@ -52,40 +46,26 @@ import java.io.IOException; import java.net.URL; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Properties; import java.util.Random; -import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; /** * The Flink Kafka Consumer is a streaming data source that pulls a parallel data stream from - * Apache Kafka. The consumer can run in multiple parallel instances, each of which will pull + * Apache Kafka 0.8.x. The consumer can run in multiple parallel instances, each of which will pull * data from one or more Kafka partitions. * *

The Flink Kafka Consumer participates in checkpointing and guarantees that no data is lost * during a failure, and that the computation processes elements "exactly once". * (Note: These guarantees naturally assume that Kafka itself does not loose any data.)

* - *

To support a variety of Kafka brokers, protocol versions, and offset committing approaches, - * the Flink Kafka Consumer can be parametrized with a fetcher and an offset handler.

- * - *

Fetcher

- * - *

The fetcher is responsible to pull data from Kafka. Because Kafka has undergone a change in - * protocols and APIs, there are currently two fetchers available:

- * - *
    - *
  • {@link FetcherType#NEW_HIGH_LEVEL}: A fetcher based on the new Kafka consumer API. - * This fetcher is generally more robust, but works only with later versions of - * Kafka (> 0.8.2).
  • - * - *
  • {@link FetcherType#LEGACY_LOW_LEVEL}: A fetcher based on the old low-level consumer API. - * This fetcher is works also with older versions of Kafka (0.8.1). The fetcher interprets - * the old Kafka consumer properties, like: + *

    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: *

      *
    • socket.timeout.ms
    • *
    • socket.receive.buffer.bytes
    • @@ -96,91 +76,39 @@ * *
    * - *

    Offset handler

    + *

    Offset handling

    * - *

    Offsets whose records have been read and are checkpointed will be committed back to Kafka / ZooKeeper + *

    Offsets whose records have been read and are checkpointed will be committed back to ZooKeeper * by the offset handler. In addition, the offset handler finds the point where the source initially * starts reading from the stream, when the streaming job is started.

    - * - *

    Currently, the source offers two different offset handlers exist:

    - *
      - *
    • {@link OffsetStore#KAFKA}: Use this offset handler when the Kafka brokers are managing the offsets, - * and hence offsets need to be committed the Kafka brokers, rather than to ZooKeeper. - * Note that this offset handler works only on new versions of Kafka (0.8.2.x +) and - * with the {@link FetcherType#NEW_HIGH_LEVEL} fetcher.
    • - * - *
    • {@link OffsetStore#FLINK_ZOOKEEPER}: Use this offset handler when the offsets are managed - * by ZooKeeper, as in older versions of Kafka (0.8.1.x)
    • - *
    - * + * *

    Please note that Flink snapshots the offsets internally as part of its distributed checkpoints. The offsets * committed to Kafka / ZooKeeper are only to bring the outside view of progress in sync with Flink's view * of the progress. That way, monitoring and other jobs can get a view of how far the Flink Kafka consumer * has consumed a topic.

    + * + *

    If checkpointing is disabled, the consumer will periodically commit the current offset + * to Zookeeper.

    + * + *

    When using a Kafka topic to send data between Flink jobs, we recommend using the + * {@see TypeInformationSerializationSchema} and {@see TypeInformationKeyValueSerializationSchema}.

    * *

    NOTE: The implementation currently accesses partition metadata when the consumer * is constructed. That means that the client that submits the program needs to be able to * reach the Kafka brokers or ZooKeeper.

    */ -public class FlinkKafkaConsumer extends RichParallelSourceFunction - implements CheckpointNotifier, CheckpointedAsynchronously>, ResultTypeQueryable { - - /** - * The offset store defines how acknowledged offsets are committed back to Kafka. Different - * options include letting Flink periodically commit to ZooKeeper, or letting Kafka manage the - * offsets (new Kafka versions only). - */ - public enum OffsetStore { - - /** - * Let Flink manage the offsets. Flink will periodically commit them to Zookeeper (usually after - * successful checkpoints), in the same structure as Kafka 0.8.2.x - * - *

    Use this mode when using the source with Kafka 0.8.1.x brokers.

    - */ - FLINK_ZOOKEEPER, - - /** - * Use the mechanisms in Kafka to commit offsets. Depending on the Kafka configuration, different - * mechanism will be used (broker coordinator, zookeeper) - */ - KAFKA - } - - /** - * The fetcher type defines which code paths to use to pull data from teh Kafka broker. - */ - public enum FetcherType { - - /** - * The legacy fetcher uses Kafka's old low-level consumer API. - * - *

    Use this fetcher for Kafka 0.8.1 brokers.

    - */ - LEGACY_LOW_LEVEL, - - /** - * This fetcher uses a backport of the new consumer API to pull data from the Kafka broker. - * It is the fetcher that will be maintained in the future, and it already - * handles certain failure cases with less overhead than the legacy fetcher. - * - *

    This fetcher works only Kafka 0.8.2 and 0.8.3 (and future versions).

    - */ - NEW_HIGH_LEVEL - } +public class FlinkKafkaConsumer08 extends FlinkKafkaConsumerBase { // ------------------------------------------------------------------------ private static final long serialVersionUID = -6272159445203409112L; - private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaConsumer.class); + private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaConsumer08.class); /** Magic number to define an unset offset. Negative offsets are not used by Kafka (invalid), * and we pick a number that is probably (hopefully) not used by Kafka as a magic number for anything else. */ public static final long OFFSET_NOT_SET = -915623761776L; - /** The maximum number of pending non-committed checkpoints to track, to avoid memory leaks */ - public static final int MAX_NUM_PENDING_CHECKPOINTS = 100; /** Configuration key for the number of retries for getting the partition info */ public static final String GET_PARTITIONS_RETRIES_KEY = "flink.get-partitions.retry"; @@ -189,14 +117,7 @@ public enum FetcherType { public static final int DEFAULT_GET_PARTITIONS_RETRIES = 3; - // ------ Configuration of the Consumer ------- - - /** The offset store where this consumer commits safe offsets */ - private final OffsetStore offsetStore; - - /** The type of fetcher to be used to pull data from Kafka */ - private final FetcherType fetcherType; /** List of partitions (including topics and leaders) to consume */ private final List partitionInfos; @@ -204,14 +125,8 @@ public enum FetcherType { /** The properties to parametrize the Kafka consumer and ZooKeeper client */ private final Properties props; - /** The schema to convert between Kafka#s byte messages, and Flink's objects */ - private final KeyedDeserializationSchema deserializer; - // ------ Runtime State ------- - - /** Data for pending but uncommitted checkpoints */ - private final LinkedMap pendingCheckpoints = new LinkedMap(); /** The fetcher used to pull data from the Kafka brokers */ private transient Fetcher fetcher; @@ -222,84 +137,80 @@ public enum FetcherType { /** The partitions actually handled by this consumer at runtime */ private transient List subscribedPartitions; - /** The offsets of the last returned elements */ - private transient HashMap lastOffsets; - /** The latest offsets that have been committed to Kafka or ZooKeeper. These are never * newer then the last offsets (Flink's internal view is fresher) */ private transient HashMap committedOffsets; - - /** The offsets to restore to, if the consumer restores state from a checkpoint */ - private transient HashMap restoreToOffset; - - private volatile boolean running = true; - + // ------------------------------------------------------------------------ + /** + * Creates a new Kafka streaming source consumer for Kafka 0.8.x + * + * @param topic + * The name of the topic that should be consumed. + * @param valueDeserializer + * The de-/serializer used to convert between Kafka's byte messages and Flink's objects. + * @param props + * The properties used to configure the Kafka consumer client, and the ZooKeeper client. + */ + public FlinkKafkaConsumer08(String topic, DeserializationSchema valueDeserializer, Properties props) { + this(Collections.singletonList(topic), valueDeserializer, props); + } /** - * Creates a new Flink Kafka Consumer, using the given type of fetcher and offset handler. + * Creates a new Kafka streaming source consumer for Kafka 0.8.x * - *

    To determine which kink of fetcher and offset handler to use, please refer to the docs - * at the beginning of this class.

    + * This constructor allows passing a {@see KeyedDeserializationSchema} for reading key/value + * pairs, offsets, and topic names from Kafka. * * @param topic - * The Kafka topic to read from. + * The name of the topic that should be consumed. + * @param deserializer + * The keyed de-/serializer used to convert between Kafka's byte messages and Flink's objects. + * @param props + * The properties used to configure the Kafka consumer client, and the ZooKeeper client. + */ + public FlinkKafkaConsumer08(String topic, KeyedDeserializationSchema deserializer, Properties props) { + this(Collections.singletonList(topic), deserializer, props); + } + + /** + * Creates a new Kafka streaming source consumer for Kafka 0.8.x + * + * This constructor allows passing multiple topics to the consumer. + * + * @param topics + * The Kafka topics to read from. * @param deserializer - * The deserializer to turn raw byte messages (without key) into Java/Scala objects. + * The de-/serializer used to convert between Kafka's byte messages and Flink's objects. * @param props * The properties that are used to configure both the fetcher and the offset handler. - * @param offsetStore - * The type of offset store to use (Kafka / ZooKeeper) - * @param fetcherType - * The type of fetcher to use (new high-level API, old low-level API). */ - public FlinkKafkaConsumer(List topic, DeserializationSchema deserializer, Properties props, - OffsetStore offsetStore, FetcherType fetcherType) { - this(topic, new KeyedDeserializationSchemaWrapper<>(deserializer), - props, offsetStore, fetcherType); + public FlinkKafkaConsumer08(List topics, DeserializationSchema deserializer, Properties props) { + this(topics, new KeyedDeserializationSchemaWrapper<>(deserializer), props); } /** - * Creates a new Flink Kafka Consumer, using the given type of fetcher and offset handler. - * - *

    To determine which kink of fetcher and offset handler to use, please refer to the docs - * at the beginning of this class.

    + * Creates a new Kafka streaming source consumer for Kafka 0.8.x + * + * This constructor allows passing multiple topics and a key/value deserialization schema. * * @param topics * The Kafka topics to read from. * @param deserializer - * The deserializer to turn raw byte messages into Java/Scala objects. + * The keyed de-/serializer used to convert between Kafka's byte messages and Flink's objects. * @param props * The properties that are used to configure both the fetcher and the offset handler. - * @param offsetStore - * The type of offset store to use (Kafka / ZooKeeper) - * @param fetcherType - * The type of fetcher to use (new high-level API, old low-level API). */ - public FlinkKafkaConsumer(List topics, KeyedDeserializationSchema deserializer, Properties props, - OffsetStore offsetStore, FetcherType fetcherType) { - this.offsetStore = checkNotNull(offsetStore); - this.fetcherType = checkNotNull(fetcherType); - - if (fetcherType == FetcherType.NEW_HIGH_LEVEL) { - throw new UnsupportedOperationException("The fetcher for Kafka 0.8.3 / 0.9.0 is not yet " + - "supported in Flink"); - } - if (offsetStore == OffsetStore.KAFKA && fetcherType == FetcherType.LEGACY_LOW_LEVEL) { - throw new IllegalArgumentException( - "The Kafka offset handler cannot be used together with the old low-level fetcher."); - } - + public FlinkKafkaConsumer08(List topics, KeyedDeserializationSchema deserializer, Properties props) { + super(deserializer, props); + checkNotNull(topics, "topics"); this.props = checkNotNull(props, "props"); - this.deserializer = checkNotNull(deserializer, "valueDeserializer"); // validate the zookeeper properties - if (offsetStore == OffsetStore.FLINK_ZOOKEEPER) { - validateZooKeeperConfig(props); - } - + validateZooKeeperConfig(props); + // Connect to a broker to get the partitions for all topics this.partitionInfos = getPartitionsForTopic(topics, props); @@ -309,21 +220,7 @@ public FlinkKafkaConsumer(List topics, KeyedDeserializationSchema des } if (LOG.isInfoEnabled()) { - Map countPerTopic = new HashMap<>(); - for (KafkaTopicPartitionLeader partition : partitionInfos) { - Integer count = countPerTopic.get(partition.getTopicPartition().getTopic()); - if (count == null) { - count = 1; - } else { - count++; - } - countPerTopic.put(partition.getTopicPartition().getTopic(), count); - } - StringBuilder sb = new StringBuilder(); - for (Map.Entry e : countPerTopic.entrySet()) { - sb.append(e.getKey()).append(" (").append(e.getValue()).append("), "); - } - LOG.info("Consumer is going to read the following topics (with number of partitions): ", sb.toString()); + logPartitionInfo(KafkaTopicPartition.convertToPartitionInfo(partitionInfos)); } } @@ -349,31 +246,16 @@ public void open(Configuration parameters) throws Exception { // we leave the fetcher as null, if we have no partitions if (subscribedPartitions.isEmpty()) { LOG.info("Kafka consumer {} has no partitions (empty source)", thisConsumerIndex); + this.fetcher = null; // fetcher remains null return; } // create fetcher - switch (fetcherType){ - case NEW_HIGH_LEVEL: - throw new UnsupportedOperationException("Currently unsupported"); - case LEGACY_LOW_LEVEL: - fetcher = new LegacyFetcher(this.subscribedPartitions, props, getRuntimeContext().getTaskName()); - break; - default: - throw new RuntimeException("Requested unknown fetcher " + fetcher); - } + fetcher = new LegacyFetcher(this.subscribedPartitions, props, getRuntimeContext().getTaskName()); // offset handling - switch (offsetStore){ - case FLINK_ZOOKEEPER: - offsetHandler = new ZookeeperOffsetHandler(props); - break; - case KAFKA: - throw new Exception("Kafka offset handler cannot work with legacy fetcher"); - default: - throw new RuntimeException("Requested unknown offset store " + offsetStore); - } - + offsetHandler = new ZookeeperOffsetHandler(props); + committedOffsets = new HashMap<>(); // seek to last known pos, from restore request @@ -390,12 +272,12 @@ public void open(Configuration parameters) throws Exception { fetcher.seek(restorePartition.getKey(), restorePartition.getValue() + 1); } // initialize offsets with restored state - this.lastOffsets = restoreToOffset; + this.offsetsState = restoreToOffset; restoreToOffset = null; } else { // start with empty offsets - lastOffsets = new HashMap<>(); + offsetsState = new HashMap<>(); // no restore request. Let the offset handler take care of the initial offset seeking offsetHandler.seekFetcherToInitialOffsets(subscribedPartitions, fetcher); @@ -422,7 +304,7 @@ public void run(SourceContext sourceContext) throws Exception { } try { - fetcher.run(sourceContext, deserializer, lastOffsets); + fetcher.run(sourceContext, deserializer, offsetsState); } finally { if (offsetCommitter != null) { offsetCommitter.close(); @@ -496,129 +378,60 @@ public void close() throws Exception { super.close(); } - @Override - public TypeInformation getProducedType() { - return deserializer.getProducedType(); - } - // ------------------------------------------------------------------------ // Checkpoint and restore // ------------------------------------------------------------------------ - @Override - public HashMap snapshotState(long checkpointId, long checkpointTimestamp) throws Exception { - if (lastOffsets == null) { - LOG.debug("snapshotState() requested on not yet opened source; returning null."); - return null; - } - if (!running) { - LOG.debug("snapshotState() called on closed source"); - return null; - } - - if (LOG.isDebugEnabled()) { - LOG.debug("Snapshotting state. Offsets: {}, checkpoint id: {}, timestamp: {}", - KafkaTopicPartition.toString(lastOffsets), checkpointId, checkpointTimestamp); + /** + * Utility method to commit offsets. + * + * @param toCommit the offsets to commit + * @throws Exception + */ + protected void commitOffsets(HashMap toCommit) throws Exception { + Map offsetsToCommit = new HashMap<>(); + for (KafkaTopicPartitionLeader tp : this.subscribedPartitions) { + Long offset = toCommit.get(tp.getTopicPartition()); + if(offset == null) { + // There was no data ever consumed from this topic, that's why there is no entry + // for this topicPartition in the map. + continue; + } + Long lastCommitted = this.committedOffsets.get(tp.getTopicPartition()); + if (lastCommitted == null) { + lastCommitted = OFFSET_NOT_SET; + } + if (offset != OFFSET_NOT_SET) { + if (offset > lastCommitted) { + offsetsToCommit.put(tp.getTopicPartition(), offset); + this.committedOffsets.put(tp.getTopicPartition(), offset); + LOG.debug("Committing offset {} for partition {}", offset, tp.getTopicPartition()); + } else { + LOG.debug("Ignoring offset {} for partition {} because it is already committed", offset, tp.getTopicPartition()); + } + } } - // the use of clone() is okay here is okay, we just need a new map, the keys are not changed - @SuppressWarnings("unchecked") - HashMap currentOffsets = (HashMap) lastOffsets.clone(); - - // the map cannot be asynchronously updated, because only one checkpoint call can happen - // on this function at a time: either snapshotState() or notifyCheckpointComplete() - pendingCheckpoints.put(checkpointId, currentOffsets); - - while (pendingCheckpoints.size() > MAX_NUM_PENDING_CHECKPOINTS) { - pendingCheckpoints.remove(0); + if (LOG.isDebugEnabled() && offsetsToCommit.size() > 0) { + LOG.debug("Committing offsets {} to Zookeeper", KafkaTopicPartition.toString(offsetsToCommit)); } - return currentOffsets; + this.offsetHandler.commit(offsetsToCommit); } - @Override - public void restoreState(HashMap restoredOffsets) { - restoreToOffset = restoredOffsets; - } - - @Override - public void notifyCheckpointComplete(long checkpointId) throws Exception { - if (fetcher == null) { - LOG.debug("notifyCheckpointComplete() called on uninitialized source"); - return; - } - if (!running) { - LOG.debug("notifyCheckpointComplete() called on closed source"); - return; - } - - // only one commit operation must be in progress - if (LOG.isDebugEnabled()) { - LOG.debug("Committing offsets externally for checkpoint {}", checkpointId); - } - - try { - HashMap checkpointOffsets; - - // the map may be asynchronously updates when snapshotting state, so we synchronize - synchronized (pendingCheckpoints) { - final int posInMap = pendingCheckpoints.indexOf(checkpointId); - if (posInMap == -1) { - LOG.warn("Received confirmation for unknown checkpoint id {}", checkpointId); - return; - } - - //noinspection unchecked - checkpointOffsets = (HashMap) pendingCheckpoints.remove(posInMap); - - // remove older checkpoints in map - for (int i = 0; i < posInMap; i++) { - pendingCheckpoints.remove(0); - } - } - if (checkpointOffsets == null || checkpointOffsets.size() == 0) { - LOG.info("Checkpoint state was empty."); - return; - } - commitOffsets(checkpointOffsets, this); - } - catch (Exception e) { - if (running) { - throw e; - } - // else ignore exception if we are no longer running - } - } - // ------------------------------------------------------------------------ // Miscellaneous utilities // ------------------------------------------------------------------------ - protected static List assignPartitions(List partitions, int numConsumers, int consumerIndex) { - checkArgument(numConsumers > 0); - checkArgument(consumerIndex < numConsumers); - - List partitionsToSub = new ArrayList<>(); - - for (int i = 0; i < partitions.size(); i++) { - if (i % numConsumers == consumerIndex) { - partitionsToSub.add(partitions.get(i)); - } - } - return partitionsToSub; - } - /** * Thread to periodically commit the current read offset into Zookeeper. */ private static class PeriodicOffsetCommitter extends Thread { - private final long commitInterval; - private final FlinkKafkaConsumer consumer; - + private final FlinkKafkaConsumer08 consumer; private volatile boolean running = true; - public PeriodicOffsetCommitter(long commitInterval, FlinkKafkaConsumer consumer) { + public PeriodicOffsetCommitter(long commitInterval, FlinkKafkaConsumer08 consumer) { this.commitInterval = commitInterval; this.consumer = consumer; } @@ -626,15 +439,16 @@ public PeriodicOffsetCommitter(long commitInterval, FlinkKafkaConsumer consum @Override public void run() { try { + while (running) { try { Thread.sleep(commitInterval); // ------------ commit current offsets ---------------- // create copy of current offsets - //noinspection unchecked - HashMap currentOffsets = (HashMap) consumer.lastOffsets.clone(); - commitOffsets(currentOffsets, this.consumer); + @SuppressWarnings("unchecked") + HashMap currentOffsets = (HashMap) consumer.offsetsState.clone(); + consumer.commitOffsets(currentOffsets); } catch (InterruptedException e) { if (running) { // throw unexpected interruption @@ -655,45 +469,7 @@ public void close() { } - /** - * Utility method to commit offsets. - * - * @param toCommit the offsets to commit - * @param consumer consumer reference - * @param message type - * @throws Exception - */ - private static void commitOffsets(HashMap toCommit, FlinkKafkaConsumer consumer) throws Exception { - Map offsetsToCommit = new HashMap<>(); - for (KafkaTopicPartitionLeader tp : consumer.subscribedPartitions) { - Long offset = toCommit.get(tp.getTopicPartition()); - if(offset == null) { - // There was no data ever consumed from this topic, that's why there is no entry - // for this topicPartition in the map. - continue; - } - Long lastCommitted = consumer.committedOffsets.get(tp.getTopicPartition()); - if (lastCommitted == null) { - lastCommitted = OFFSET_NOT_SET; - } - if (offset != OFFSET_NOT_SET) { - if (offset > lastCommitted) { - offsetsToCommit.put(tp.getTopicPartition(), offset); - consumer.committedOffsets.put(tp.getTopicPartition(), offset); - LOG.debug("Committing offset {} for partition {}", offset, tp.getTopicPartition()); - } else { - LOG.debug("Ignoring offset {} for partition {} because it is already committed", offset, tp.getTopicPartition()); - } - } - } - if (LOG.isDebugEnabled() && offsetsToCommit.size() > 0) { - LOG.debug("Committing offsets {} to offset store: {}", KafkaTopicPartition.toString(offsetsToCommit), consumer.offsetStore); - } - - consumer.offsetHandler.commit(offsetsToCommit); - } - // ------------------------------------------------------------------------ // Kafka / ZooKeeper communication utilities // ------------------------------------------------------------------------ @@ -763,7 +539,8 @@ public static List getPartitionsForTopic(final ListThe following additional configuration values are available:

    - *
      - *
    • socket.timeout.ms
    • - *
    • socket.receive.buffer.bytes
    • - *
    • fetch.message.max.bytes
    • - *
    • auto.offset.reset with the values "latest", "earliest" (unlike 0.8.2 behavior)
    • - *
    • fetch.wait.max.ms
    • - *
    - * - * @param The type of elements produced by this consumer. + * THIS CLASS IS DEPRECATED. Use FlinkKafkaConsumer08 instead. */ -public class FlinkKafkaConsumer081 extends FlinkKafkaConsumer { +@Deprecated +public class FlinkKafkaConsumer081 extends FlinkKafkaConsumer08 { private static final long serialVersionUID = -5649906773771949146L; /** - * Creates a new Kafka 0.8.1.x streaming source consumer. - * - * @param topic - * The name of the topic that should be consumed. - * @param valueDeserializer - * The de-/serializer used to convert between Kafka's byte messages and Flink's objects. - * @param props - * The properties used to configure the Kafka consumer client, and the ZooKeeper client. + * THIS CONSTRUCTOR IS DEPRECATED. Use FlinkKafkaConsumer08 instead. */ + @Deprecated public FlinkKafkaConsumer081(String topic, DeserializationSchema valueDeserializer, Properties props) { - super(Collections.singletonList(topic), valueDeserializer, props, OffsetStore.FLINK_ZOOKEEPER, FetcherType.LEGACY_LOW_LEVEL); + super(topic, valueDeserializer, props); } } + diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer082.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer082.java new file mode 100644 index 0000000000000..0520336a585f3 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer082.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.streaming.connectors.kafka; + +import org.apache.flink.streaming.util.serialization.DeserializationSchema; + +import java.util.Properties; + +/** + * THIS CLASS IS DEPRECATED. Use FlinkKafkaConsumer08 instead. + */ +@Deprecated +public class FlinkKafkaConsumer082 extends FlinkKafkaConsumer08 { + + private static final long serialVersionUID = -5649906773771949146L; + + /** + * THIS CONSTRUCTOR IS DEPRECATED. Use FlinkKafkaConsumer08 instead. + */ + @Deprecated + public FlinkKafkaConsumer082(String topic, DeserializationSchema valueDeserializer, Properties props) { + super(topic, valueDeserializer, props); + } +} + diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java new file mode 100644 index 0000000000000..1c2e0b7de8bf0 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kafka; + +import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema; +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper; +import org.apache.flink.streaming.util.serialization.SerializationSchema; +import java.util.Properties; + + +/** + * THIS CLASS IS DEPRECATED. Use FlinkKafkaProducer08 instead. + */ +@Deprecated +public class FlinkKafkaProducer extends FlinkKafkaProducer08 { + + @Deprecated + public FlinkKafkaProducer(String brokerList, String topicId, SerializationSchema serializationSchema) { + super(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), null); + } + + @Deprecated + public FlinkKafkaProducer(String topicId, SerializationSchema serializationSchema, Properties producerConfig) { + super(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, null); + } + + @Deprecated + public FlinkKafkaProducer(String topicId, SerializationSchema serializationSchema, Properties producerConfig, KafkaPartitioner customPartitioner) { + super(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner); + + } + + @Deprecated + public FlinkKafkaProducer(String brokerList, String topicId, KeyedSerializationSchema serializationSchema) { + super(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), null); + } + + @Deprecated + public FlinkKafkaProducer(String topicId, KeyedSerializationSchema serializationSchema, Properties producerConfig) { + super(topicId, serializationSchema, producerConfig, null); + } + + @Deprecated + public FlinkKafkaProducer(String topicId, KeyedSerializationSchema serializationSchema, Properties producerConfig, KafkaPartitioner customPartitioner) { + super(topicId, serializationSchema, producerConfig, customPartitioner); + } + +} \ No newline at end of file diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer08.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer08.java new file mode 100644 index 0000000000000..4975f9ab4744c --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer08.java @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kafka; + +import org.apache.flink.streaming.connectors.kafka.partitioner.FixedPartitioner; +import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema; +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper; +import org.apache.flink.streaming.util.serialization.SerializationSchema; + +import java.util.Properties; + + +/** + * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.8. + * + * Please note that this producer does not have any reliability guarantees. + * + * @param Type of the messages to write into Kafka. + */ +public class FlinkKafkaProducer08 extends FlinkKafkaProducerBase { + + private static final long serialVersionUID = 1L; + + // ------------------- Keyless serialization schema constructors ---------------------- + /** + * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to + * the topic. + * + * @param brokerList + * Comma separated addresses of the brokers + * @param topicId + * ID of the Kafka topic. + * @param serializationSchema + * User defined (keyless) serialization schema. + */ + public FlinkKafkaProducer08(String brokerList, String topicId, SerializationSchema serializationSchema) { + this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), new FixedPartitioner()); + } + + /** + * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to + * the topic. + * + * @param topicId + * ID of the Kafka topic. + * @param serializationSchema + * User defined (keyless) serialization schema. + * @param producerConfig + * Properties with the producer configuration. + */ + public FlinkKafkaProducer08(String topicId, SerializationSchema serializationSchema, Properties producerConfig) { + this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FixedPartitioner()); + } + + /** + * The main constructor for creating a FlinkKafkaProducer. + * + * @param topicId The topic to write data to + * @param serializationSchema A (keyless) serializable serialization schema for turning user objects into a kafka-consumable byte[] + * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument. + * @param customPartitioner A serializable partitioner for assining messages to Kafka partitions. + */ + public FlinkKafkaProducer08(String topicId, SerializationSchema serializationSchema, Properties producerConfig, KafkaPartitioner customPartitioner) { + this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner); + + } + + // ------------------- Key/Value serialization schema constructors ---------------------- + + /** + * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to + * the topic. + * + * @param brokerList + * Comma separated addresses of the brokers + * @param topicId + * ID of the Kafka topic. + * @param serializationSchema + * User defined serialization schema supporting key/value messages + */ + public FlinkKafkaProducer08(String brokerList, String topicId, KeyedSerializationSchema serializationSchema) { + this(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), new FixedPartitioner()); + } + + /** + * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to + * the topic. + * + * @param topicId + * ID of the Kafka topic. + * @param serializationSchema + * User defined serialization schema supporting key/value messages + * @param producerConfig + * Properties with the producer configuration. + */ + public FlinkKafkaProducer08(String topicId, KeyedSerializationSchema serializationSchema, Properties producerConfig) { + this(topicId, serializationSchema, producerConfig, new FixedPartitioner()); + } + + /** + * The main constructor for creating a FlinkKafkaProducer. + * + * @param topicId The topic to write data to + * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages + * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument. + * @param customPartitioner A serializable partitioner for assining messages to Kafka partitions. + */ + public FlinkKafkaProducer08(String topicId, KeyedSerializationSchema serializationSchema, Properties producerConfig, KafkaPartitioner customPartitioner) { + super(topicId, serializationSchema, producerConfig, customPartitioner); + } + +} diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Fetcher.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Fetcher.java similarity index 100% rename from flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Fetcher.java rename to flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Fetcher.java diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java similarity index 98% rename from flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java rename to flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java index b51ad6166924f..9fec52d3e9671 100644 --- a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java @@ -29,7 +29,7 @@ import kafka.message.MessageAndOffset; import org.apache.flink.streaming.api.functions.source.SourceFunction; -import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08; import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; import org.apache.flink.util.StringUtils; @@ -90,7 +90,7 @@ public LegacyFetcher(List partitions, Properties prop //this.topic = checkNotNull(topic, "The topic cannot be null"); this.partitionsToRead = new HashMap<>(); for (KafkaTopicPartitionLeader p: partitions) { - partitionsToRead.put(p, FlinkKafkaConsumer.OFFSET_NOT_SET); + partitionsToRead.put(p, FlinkKafkaConsumer08.OFFSET_NOT_SET); } this.taskName = taskName; this.error = new AtomicReference<>(); @@ -370,7 +370,7 @@ public void run() { List partitionsToGetOffsetsFor = new ArrayList<>(); for (FetchPartition fp : partitions) { - if (fp.nextOffsetToRead == FlinkKafkaConsumer.OFFSET_NOT_SET) { + if (fp.nextOffsetToRead == FlinkKafkaConsumer08.OFFSET_NOT_SET) { // retrieve the offset from the consumer partitionsToGetOffsetsFor.add(fp); } @@ -383,7 +383,7 @@ public void run() { // Now, the actual work starts :-) int offsetOutOfRangeCount = 0; - while (running) { + fetchLoop: while (running) { FetchRequestBuilder frb = new FetchRequestBuilder(); frb.clientId(clientId); frb.maxWait(maxWait); @@ -471,7 +471,11 @@ public void run() { keyPayload.get(keyBytes); } - final T value = deserializer.deserialize(keyBytes, valueBytes, fp.topic, offset); + final T value = deserializer.deserialize(keyBytes, valueBytes, fp.topic, fp.partition, offset); + if(deserializer.isEndOfStream(value)) { + running = false; + break fetchLoop; // leave running loop + } synchronized (sourceContext.getCheckpointLock()) { sourceContext.collect(value); offsetsState.put(topicPartition, offset); @@ -586,7 +590,7 @@ private static class PartitionInfoFetcher extends Thread { @Override public void run() { try { - result = FlinkKafkaConsumer.getPartitionsForTopic(topics, properties); + result = FlinkKafkaConsumer08.getPartitionsForTopic(topics, properties); } catch (Throwable t) { this.error = t; diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/OffsetHandler.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/OffsetHandler.java similarity index 100% rename from flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/OffsetHandler.java rename to flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/OffsetHandler.java diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PartitionerWrapper.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PartitionerWrapper.java similarity index 100% rename from flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PartitionerWrapper.java rename to flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PartitionerWrapper.java diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java similarity index 96% rename from flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java rename to flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java index f72117d444941..1eca4dd540c73 100644 --- a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java @@ -24,7 +24,7 @@ import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; import org.apache.curator.retry.ExponentialBackoffRetry; -import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.slf4j.Logger; @@ -35,11 +35,14 @@ import java.util.Map; import java.util.Properties; +/** + * Handler for committing Kafka offsets to Zookeeper and to retrieve them again. + */ public class ZookeeperOffsetHandler implements OffsetHandler { private static final Logger LOG = LoggerFactory.getLogger(ZookeeperOffsetHandler.class); - private static final long OFFSET_NOT_SET = FlinkKafkaConsumer.OFFSET_NOT_SET; + private static final long OFFSET_NOT_SET = FlinkKafkaConsumer08.OFFSET_NOT_SET; private final String groupId; diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java new file mode 100644 index 0000000000000..26e31f59987ed --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java @@ -0,0 +1,266 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kafka; + +import org.apache.curator.framework.CuratorFramework; +import org.apache.flink.api.java.functions.FlatMapIterator; +import org.apache.flink.api.java.operators.DataSource; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler; +import org.junit.Test; + +import java.util.Iterator; +import java.util.Properties; + +import static org.junit.Assert.assertTrue; + + +public class Kafka08ITCase extends KafkaConsumerTestBase { + + // ------------------------------------------------------------------------ + // Suite of Tests + // ------------------------------------------------------------------------ + + @Test(timeout = 60000) + public void testCheckpointing() throws Exception { + runCheckpointingTest(); + } + + @Test(timeout = 60000) + public void testFailOnNoBroker() throws Exception { + runFailOnNoBrokerTest(); + } + + + @Test(timeout = 60000) + public void testConcurrentProducerConsumerTopology() throws Exception { + runSimpleConcurrentProducerConsumerTopology(); + } + + @Test(timeout = 60000) + public void testKeyValueSupport() throws Exception { + runKeyValueTest(); + } + + // --- canceling / failures --- + + @Test(timeout = 60000) + public void testCancelingEmptyTopic() throws Exception { + runCancelingOnEmptyInputTest(); + } + + @Test(timeout = 60000) + public void testCancelingFullTopic() throws Exception { + runCancelingOnFullInputTest(); + } + + @Test(timeout = 60000) + public void testFailOnDeploy() throws Exception { + runFailOnDeployTest(); + } + + @Test(timeout = 60000) + public void testInvalidOffset() throws Exception { + final String topic = "invalidOffsetTopic"; + final int parallelism = 1; + + // create topic + createTestTopic(topic, parallelism, 1); + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); + + // write 20 messages into topic: + writeSequence(env, topic, 20, parallelism); + + // set invalid offset: + CuratorFramework curatorClient = ((KafkaTestEnvironmentImpl)kafkaServer).createCuratorClient(); + ZookeeperOffsetHandler.setOffsetInZooKeeper(curatorClient, standardCC.groupId(), topic, 0, 1234); + curatorClient.close(); + + // read from topic + final int valuesCount = 20; + final int startFrom = 0; + readSequence(env, standardCC.props().props(), parallelism, topic, valuesCount, startFrom); + + deleteTestTopic(topic); + } + + // --- source to partition mappings and exactly once --- + + @Test(timeout = 60000) + public void testOneToOneSources() throws Exception { + runOneToOneExactlyOnceTest(); + } + + @Test(timeout = 60000) + public void testOneSourceMultiplePartitions() throws Exception { + runOneSourceMultiplePartitionsExactlyOnceTest(); + } + + @Test(timeout = 60000) + public void testMultipleSourcesOnePartition() throws Exception { + runMultipleSourcesOnePartitionExactlyOnceTest(); + } + + // --- broker failure --- + + @Test(timeout = 60000) + public void testBrokerFailure() throws Exception { + runBrokerFailureTest(); + } + + // --- special executions --- + + @Test(timeout = 60000) + public void testBigRecordJob() throws Exception { + runBigRecordTestTopology(); + } + + @Test(timeout = 60000) + public void testMultipleTopics() throws Exception { + runConsumeMultipleTopics(); + } + + @Test(timeout = 60000) + public void testAllDeletes() throws Exception { + runAllDeletesTest(); + } + + @Test(timeout=60000) + public void testMetricsAndEndOfStream() throws Exception { + runMetricsAndEndOfStreamTest(); + } + + + /** + * Tests that offsets are properly committed to ZooKeeper and initial offsets are read from ZooKeeper. + * + * This test is only applicable if the Flink Kafka Consumer uses the ZooKeeperOffsetHandler. + */ + @Test(timeout = 60000) + public void testOffsetInZookeeper() throws Exception { + final String topicName = "testOffsetInZK"; + final int parallelism = 3; + + createTestTopic(topicName, parallelism, 1); + + StreamExecutionEnvironment env1 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); + env1.getConfig().disableSysoutLogging(); + env1.enableCheckpointing(50); + env1.setNumberOfExecutionRetries(0); + env1.setParallelism(parallelism); + + StreamExecutionEnvironment env2 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); + env2.getConfig().disableSysoutLogging(); + env2.enableCheckpointing(50); + env2.setNumberOfExecutionRetries(0); + env2.setParallelism(parallelism); + + StreamExecutionEnvironment env3 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); + env3.getConfig().disableSysoutLogging(); + env3.enableCheckpointing(50); + env3.setNumberOfExecutionRetries(0); + env3.setParallelism(parallelism); + + // write a sequence from 0 to 99 to each of the 3 partitions. + writeSequence(env1, topicName, 100, parallelism); + + readSequence(env2, standardProps, parallelism, topicName, 100, 0); + + CuratorFramework curatorClient = ((KafkaTestEnvironmentImpl)kafkaServer).createCuratorClient(); + + long o1 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorClient, standardCC.groupId(), topicName, 0); + long o2 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorClient, standardCC.groupId(), topicName, 1); + long o3 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorClient, standardCC.groupId(), topicName, 2); + + LOG.info("Got final offsets from zookeeper o1={}, o2={}, o3={}", o1, o2, o3); + + assertTrue(o1 == FlinkKafkaConsumer08.OFFSET_NOT_SET || (o1 >= 0 && o1 <= 100)); + assertTrue(o2 == FlinkKafkaConsumer08.OFFSET_NOT_SET || (o2 >= 0 && o2 <= 100)); + assertTrue(o3 == FlinkKafkaConsumer08.OFFSET_NOT_SET || (o3 >= 0 && o3 <= 100)); + + LOG.info("Manipulating offsets"); + + // set the offset to 50 for the three partitions + ZookeeperOffsetHandler.setOffsetInZooKeeper(curatorClient, standardCC.groupId(), topicName, 0, 49); + ZookeeperOffsetHandler.setOffsetInZooKeeper(curatorClient, standardCC.groupId(), topicName, 1, 49); + ZookeeperOffsetHandler.setOffsetInZooKeeper(curatorClient, standardCC.groupId(), topicName, 2, 49); + + curatorClient.close(); + + // create new env + readSequence(env3, standardProps, parallelism, topicName, 50, 50); + + deleteTestTopic(topicName); + } + + @Test(timeout = 60000) + public void testOffsetAutocommitTest() throws Exception { + final String topicName = "testOffsetAutocommit"; + final int parallelism = 3; + + createTestTopic(topicName, parallelism, 1); + + StreamExecutionEnvironment env1 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); + env1.getConfig().disableSysoutLogging(); + env1.setNumberOfExecutionRetries(0); + env1.setParallelism(parallelism); + + StreamExecutionEnvironment env2 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); + // NOTE: We are not enabling the checkpointing! + env2.getConfig().disableSysoutLogging(); + env2.setNumberOfExecutionRetries(0); + env2.setParallelism(parallelism); + + + // write a sequence from 0 to 99 to each of the 3 partitions. + writeSequence(env1, topicName, 100, parallelism); + + + // the readSequence operation sleeps for 20 ms between each record. + // setting a delay of 25*20 = 500 for the commit interval makes + // sure that we commit roughly 3-4 times while reading, however + // at least once. + Properties readProps = new Properties(); + readProps.putAll(standardProps); + readProps.setProperty("auto.commit.interval.ms", "500"); + + // read so that the offset can be committed to ZK + readSequence(env2, readProps, parallelism, topicName, 100, 0); + + // get the offset + CuratorFramework curatorFramework = ((KafkaTestEnvironmentImpl)kafkaServer).createCuratorClient(); + + long o1 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorFramework, standardCC.groupId(), topicName, 0); + long o2 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorFramework, standardCC.groupId(), topicName, 1); + long o3 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorFramework, standardCC.groupId(), topicName, 2); + + LOG.info("Got final offsets from zookeeper o1={}, o2={}, o3={}", o1, o2, o3); + + // ensure that the offset has been committed + assertTrue("Offset of o1=" + o1 + " was not in range", o1 > 0 && o1 <= 100); + assertTrue("Offset of o2=" + o2 + " was not in range", o2 > 0 && o2 <= 100); + assertTrue("Offset of o3=" + o3 + " was not in range", o3 > 0 && o3 <= 100); + + deleteTestTopic(topicName); + } +} diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ProducerITCase.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ProducerITCase.java new file mode 100644 index 0000000000000..fc137198c3272 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ProducerITCase.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kafka; + + +import org.junit.Test; + + +@SuppressWarnings("serial") +public class Kafka08ProducerITCase extends KafkaProducerTestBase { + + @Test + public void testCustomPartitioning() { + runCustomPartitioningTest(); + } + +} diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTest.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTest.java similarity index 83% rename from flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTest.java rename to flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTest.java index efae9228acfa4..113ad714883a9 100644 --- a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTest.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTest.java @@ -20,7 +20,6 @@ import org.apache.commons.collections.map.LinkedMap; -import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.streaming.util.serialization.SimpleStringSchema; import org.apache.kafka.clients.consumer.ConsumerConfig; @@ -44,7 +43,7 @@ public void testValidateZooKeeperConfig() { // empty Properties emptyProperties = new Properties(); try { - FlinkKafkaConsumer.validateZooKeeperConfig(emptyProperties); + FlinkKafkaConsumer08.validateZooKeeperConfig(emptyProperties); fail("should fail with an exception"); } catch (IllegalArgumentException e) { @@ -55,7 +54,7 @@ public void testValidateZooKeeperConfig() { Properties noConnect = new Properties(); noConnect.put(ConsumerConfig.GROUP_ID_CONFIG, "flink-test-group"); try { - FlinkKafkaConsumer.validateZooKeeperConfig(noConnect); + FlinkKafkaConsumer08.validateZooKeeperConfig(noConnect); fail("should fail with an exception"); } catch (IllegalArgumentException e) { @@ -66,7 +65,7 @@ public void testValidateZooKeeperConfig() { Properties noGroup = new Properties(); noGroup.put("zookeeper.connect", "localhost:47574"); try { - FlinkKafkaConsumer.validateZooKeeperConfig(noGroup); + FlinkKafkaConsumer08.validateZooKeeperConfig(noGroup); fail("should fail with an exception"); } catch (IllegalArgumentException e) { @@ -82,15 +81,15 @@ public void testValidateZooKeeperConfig() { @Test public void testSnapshot() { try { - Field offsetsField = FlinkKafkaConsumer.class.getDeclaredField("lastOffsets"); - Field runningField = FlinkKafkaConsumer.class.getDeclaredField("running"); - Field mapField = FlinkKafkaConsumer.class.getDeclaredField("pendingCheckpoints"); + Field offsetsField = FlinkKafkaConsumerBase.class.getDeclaredField("offsetsState"); + Field runningField = FlinkKafkaConsumerBase.class.getDeclaredField("running"); + Field mapField = FlinkKafkaConsumerBase.class.getDeclaredField("pendingCheckpoints"); offsetsField.setAccessible(true); runningField.setAccessible(true); mapField.setAccessible(true); - FlinkKafkaConsumer consumer = mock(FlinkKafkaConsumer.class); + FlinkKafkaConsumer08 consumer = mock(FlinkKafkaConsumer08.class); when(consumer.snapshotState(anyLong(), anyLong())).thenCallRealMethod(); @@ -125,7 +124,7 @@ public void testSnapshot() { assertEquals(checkpointCopy, checkpoint); assertTrue(map.size() > 0); - assertTrue(map.size() <= FlinkKafkaConsumer.MAX_NUM_PENDING_CHECKPOINTS); + assertTrue(map.size() <= FlinkKafkaConsumer08.MAX_NUM_PENDING_CHECKPOINTS); } } catch (Exception e) { @@ -143,9 +142,7 @@ public void testCreateSourceWithoutCluster() { props.setProperty("bootstrap.servers", "localhost:11111, localhost:22222"); props.setProperty("group.id", "non-existent-group"); - new FlinkKafkaConsumer<>(Collections.singletonList("no op topic"), new SimpleStringSchema(), props, - FlinkKafkaConsumer.OffsetStore.FLINK_ZOOKEEPER, - FlinkKafkaConsumer.FetcherType.LEGACY_LOW_LEVEL); + new FlinkKafkaConsumer08<>(Collections.singletonList("no op topic"), new SimpleStringSchema(), props); } catch (Exception e) { e.printStackTrace(); diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaLocalSystemTime.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaLocalSystemTime.java similarity index 100% rename from flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaLocalSystemTime.java rename to flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaLocalSystemTime.java diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java similarity index 94% rename from flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java rename to flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java index 531b2194b299e..8602ffec2f375 100644 --- a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java @@ -47,7 +47,7 @@ import static org.junit.Assert.*; @RunWith(PowerMockRunner.class) -@PrepareForTest(FlinkKafkaProducer.class) +@PrepareForTest(FlinkKafkaProducerBase.class) public class KafkaProducerTest extends TestLogger { @Test @@ -76,8 +76,8 @@ public Future answer(InvocationOnMock invocation) throws Throwab whenNew(KafkaProducer.class).withAnyArguments().thenReturn(kafkaProducerMock); // (1) producer that propagates errors - - FlinkKafkaProducer producerPropagating = new FlinkKafkaProducer( + + FlinkKafkaProducer08 producerPropagating = new FlinkKafkaProducer08<>( "mock_topic", new SimpleStringSchema(), new Properties(), null); producerPropagating.setRuntimeContext(new MockRuntimeContext(17, 3)); @@ -95,8 +95,8 @@ public Future answer(InvocationOnMock invocation) throws Throwab } // (2) producer that only logs errors - - FlinkKafkaProducer producerLogging = new FlinkKafkaProducer( + + FlinkKafkaProducer08 producerLogging = new FlinkKafkaProducer08<>( "mock_topic", new SimpleStringSchema(), new Properties(), null); producerLogging.setLogFailuresOnly(true); diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java similarity index 56% rename from flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java rename to flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java index c9ac75b059ce5..348b75de5ba29 100644 --- a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java @@ -15,127 +15,145 @@ * limitations under the License. */ + package org.apache.flink.streaming.connectors.kafka; import kafka.admin.AdminUtils; +import kafka.api.PartitionMetadata; import kafka.common.KafkaException; import kafka.consumer.ConsumerConfig; import kafka.network.SocketServer; import kafka.server.KafkaConfig; import kafka.server.KafkaServer; - import org.I0Itec.zkclient.ZkClient; - import org.apache.commons.io.FileUtils; import org.apache.curator.RetryPolicy; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; import org.apache.curator.retry.ExponentialBackoffRetry; import org.apache.curator.test.TestingServer; -import org.apache.flink.client.program.ProgramInvocationException; -import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.client.JobExecutionException; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionLeader; import org.apache.flink.streaming.connectors.kafka.internals.ZooKeeperStringSerializer; -import org.apache.flink.streaming.connectors.kafka.testutils.SuccessException; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; +import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema; import org.apache.flink.util.NetUtils; -import org.apache.flink.util.TestLogger; - -import org.junit.AfterClass; -import org.junit.BeforeClass; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; - -import scala.concurrent.duration.FiniteDuration; +import scala.collection.Seq; import java.io.File; -import java.io.IOException; import java.net.BindException; import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Properties; import java.util.UUID; -import java.util.concurrent.TimeUnit; import static org.apache.flink.util.NetUtils.hostAndPortToUrlString; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; /** - * The base for the Kafka tests. It brings up: - *
      - *
    • A ZooKeeper mini cluster
    • - *
    • Three Kafka Brokers (mini clusters)
    • - *
    • A Flink mini cluster
    • - *
    - * - *

    Code in this test is based on the following GitHub repository: - * - * https://github.com/sakserv/hadoop-mini-clusters (ASL licensed), - * as per commit bc6b2b2d5f6424d5f377aa6c0871e82a956462ef

    + * An implementation of the KafkaServerProvider for Kafka 0.8 */ -@SuppressWarnings("serial") -public abstract class KafkaTestBase extends TestLogger { +public class KafkaTestEnvironmentImpl extends KafkaTestEnvironment { + + protected static final Logger LOG = LoggerFactory.getLogger(KafkaTestEnvironmentImpl.class); + private File tmpZkDir; + private File tmpKafkaParent; + private List tmpKafkaDirs; + private List brokers; + private TestingServer zookeeper; + private String zookeeperConnectionString; + private String brokerConnectionString = ""; + private Properties standardProps; + private ConsumerConfig standardCC; + + + public String getBrokerConnectionString() { + return brokerConnectionString; + } - protected static final Logger LOG = LoggerFactory.getLogger(KafkaTestBase.class); - - protected static final int NUMBER_OF_KAFKA_SERVERS = 3; - protected static String zookeeperConnectionString; + @Override + public ConsumerConfig getStandardConsumerConfig() { + return standardCC; + } - protected static File tmpZkDir; + @Override + public Properties getStandardProperties() { + return standardProps; + } - protected static File tmpKafkaParent; + @Override + public String getVersion() { + return "0.8"; + } - protected static TestingServer zookeeper; - protected static List brokers; - protected static String brokerConnectionStrings = ""; + @Override + public List getBrokers() { + return brokers; + } - protected static ConsumerConfig standardCC; - protected static Properties standardProps; - - protected static ForkableFlinkMiniCluster flink; + @Override + public FlinkKafkaConsumerBase getConsumer(List topics, KeyedDeserializationSchema readSchema, Properties props) { + return new FlinkKafkaConsumer08<>(topics, readSchema, props); + } - protected static int flinkPort; + @Override + public FlinkKafkaProducerBase getProducer(String topic, KeyedSerializationSchema serSchema, Properties props, KafkaPartitioner partitioner) { + return new FlinkKafkaProducer08(topic, serSchema, props, partitioner); + } - protected static FiniteDuration timeout = new FiniteDuration(10, TimeUnit.SECONDS); + @Override + public void restartBroker(int leaderId) throws Exception { + brokers.set(leaderId, getKafkaServer(leaderId, tmpKafkaDirs.get(leaderId), KAFKA_HOST, zookeeperConnectionString)); + } - protected static List tmpKafkaDirs; + @Override + public int getLeaderToShutDown(String topic) throws Exception { + ZkClient zkClient = createZkClient(); + PartitionMetadata firstPart = null; + do { + if (firstPart != null) { + LOG.info("Unable to find leader. error code {}", firstPart.errorCode()); + // not the first try. Sleep a bit + Thread.sleep(150); + } - protected static String kafkaHost = "localhost"; + Seq partitionMetadata = AdminUtils.fetchTopicMetadataFromZk(topic, zkClient).partitionsMetadata(); + firstPart = partitionMetadata.head(); + } + while (firstPart.errorCode() != 0); + zkClient.close(); - // ------------------------------------------------------------------------ - // Setup and teardown of the mini clusters - // ------------------------------------------------------------------------ - - @BeforeClass - public static void prepare() throws IOException { - LOG.info("-------------------------------------------------------------------------"); - LOG.info(" Starting KafkaITCase "); - LOG.info("-------------------------------------------------------------------------"); - - LOG.info("Starting KafkaITCase.prepare()"); - + return firstPart.leader().get().id(); + } + + @Override + public int getBrokerId(KafkaServer server) { + return server.socketServer().brokerId(); + } + + + @Override + public void prepare(int numKafkaServers) { File tempDir = new File(System.getProperty("java.io.tmpdir")); - + tmpZkDir = new File(tempDir, "kafkaITcase-zk-dir-" + (UUID.randomUUID().toString())); assertTrue("cannot create zookeeper temp dir", tmpZkDir.mkdirs()); tmpKafkaParent = new File(tempDir, "kafkaITcase-kafka-dir*" + (UUID.randomUUID().toString())); assertTrue("cannot create kafka temp dir", tmpKafkaParent.mkdirs()); - tmpKafkaDirs = new ArrayList<>(NUMBER_OF_KAFKA_SERVERS); - for (int i = 0; i < NUMBER_OF_KAFKA_SERVERS; i++) { + tmpKafkaDirs = new ArrayList<>(numKafkaServers); + for (int i = 0; i < numKafkaServers; i++) { File tmpDir = new File(tmpKafkaParent, "server-" + i); assertTrue("cannot create kafka temp dir", tmpDir.mkdir()); tmpKafkaDirs.add(tmpDir); } - + zookeeper = null; brokers = null; @@ -145,14 +163,14 @@ public static void prepare() throws IOException { zookeeperConnectionString = zookeeper.getConnectString(); LOG.info("Starting KafkaServer"); - brokers = new ArrayList<>(NUMBER_OF_KAFKA_SERVERS); - - for (int i = 0; i < NUMBER_OF_KAFKA_SERVERS; i++) { - brokers.add(getKafkaServer(i, tmpKafkaDirs.get(i), kafkaHost, zookeeperConnectionString)); + brokers = new ArrayList<>(numKafkaServers); + + for (int i = 0; i < numKafkaServers; i++) { + brokers.add(getKafkaServer(i, tmpKafkaDirs.get(i), KafkaTestEnvironment.KAFKA_HOST, zookeeperConnectionString)); SocketServer socketServer = brokers.get(i).socketServer(); - + String host = socketServer.host() == null ? "localhost" : socketServer.host(); - brokerConnectionStrings += hostAndPortToUrlString(host, socketServer.port()) + ","; + brokerConnectionString += hostAndPortToUrlString(host, socketServer.port()) + ","; } LOG.info("ZK and KafkaServer started."); @@ -163,54 +181,30 @@ public static void prepare() throws IOException { } standardProps = new Properties(); - standardProps.setProperty("zookeeper.connect", zookeeperConnectionString); - standardProps.setProperty("bootstrap.servers", brokerConnectionStrings); + standardProps.setProperty("bootstrap.servers", brokerConnectionString); standardProps.setProperty("group.id", "flink-tests"); standardProps.setProperty("auto.commit.enable", "false"); standardProps.setProperty("zookeeper.session.timeout.ms", "12000"); // 6 seconds is default. Seems to be too small for travis. standardProps.setProperty("zookeeper.connection.timeout.ms", "20000"); standardProps.setProperty("auto.offset.reset", "earliest"); // read from the beginning. standardProps.setProperty("fetch.message.max.bytes", "256"); // make a lot of fetches (MESSAGES MUST BE SMALLER!) - + Properties consumerConfigProps = new Properties(); consumerConfigProps.putAll(standardProps); consumerConfigProps.setProperty("auto.offset.reset", "smallest"); standardCC = new ConsumerConfig(consumerConfigProps); - - // start also a re-usable Flink mini cluster - - Configuration flinkConfig = new Configuration(); - flinkConfig.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1); - flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 8); - flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 16); - flinkConfig.setString(ConfigConstants.EXECUTION_RETRY_DELAY_KEY, "0 s"); - - flink = new ForkableFlinkMiniCluster(flinkConfig, false); - flink.start(); - - flinkPort = flink.getLeaderRPCPort(); } - @AfterClass - public static void shutDownServices() { - - LOG.info("-------------------------------------------------------------------------"); - LOG.info(" Shut down KafkaITCase "); - LOG.info("-------------------------------------------------------------------------"); - - flinkPort = -1; - if (flink != null) { - flink.shutdown(); - } - + @Override + public void shutdown() { for (KafkaServer broker : brokers) { if (broker != null) { broker.shutdown(); } } brokers.clear(); - + if (zookeeper != null) { try { zookeeper.stop(); @@ -220,9 +214,9 @@ public static void shutDownServices() { } zookeeper = null; } - + // clean up the temp spaces - + if (tmpKafkaParent != null && tmpKafkaParent.exists()) { try { FileUtils.deleteDirectory(tmpKafkaParent); @@ -239,10 +233,59 @@ public static void shutDownServices() { // ignore } } + } - LOG.info("-------------------------------------------------------------------------"); - LOG.info(" KafkaITCase finished"); - LOG.info("-------------------------------------------------------------------------"); + @Override + public void createTestTopic(String topic, int numberOfPartitions, int replicationFactor) { + // create topic with one client + Properties topicConfig = new Properties(); + LOG.info("Creating topic {}", topic); + + ZkClient creator = createZkClient(); + + AdminUtils.createTopic(creator, topic, numberOfPartitions, replicationFactor, topicConfig); + creator.close(); + + // validate that the topic has been created + final long deadline = System.currentTimeMillis() + 30000; + do { + try { + Thread.sleep(100); + } + catch (InterruptedException e) { + // restore interrupted state + } + List partitions = FlinkKafkaConsumer08.getPartitionsForTopic(Collections.singletonList(topic), standardProps); + if (partitions != null && partitions.size() > 0) { + return; + } + } + while (System.currentTimeMillis() < deadline); + fail ("Test topic could not be created"); + } + + @Override + public void deleteTestTopic(String topic) { + LOG.info("Deleting topic {}", topic); + + ZkClient zk = createZkClient(); + AdminUtils.deleteTopic(zk, topic); + zk.close(); + } + + private ZkClient createZkClient() { + return new ZkClient(standardCC.zkConnect(), standardCC.zkSessionTimeoutMs(), + standardCC.zkConnectionTimeoutMs(), new ZooKeeperStringSerializer()); + } + + /** + * Only for the 0.8 server we need access to the zk client. + */ + public CuratorFramework createCuratorClient() { + RetryPolicy retryPolicy = new ExponentialBackoffRetry(100, 10); + CuratorFramework curatorClient = CuratorFrameworkFactory.newClient(standardProps.getProperty("zookeeper.connect"), retryPolicy); + curatorClient.start(); + return curatorClient; } /** @@ -251,6 +294,7 @@ public static void shutDownServices() { protected static KafkaServer getKafkaServer(int brokerId, File tmpFolder, String kafkaHost, String zookeeperConnectionString) throws Exception { + LOG.info("Starting broker with id {}", brokerId); Properties kafkaProperties = new Properties(); // properties have to be Strings @@ -260,13 +304,13 @@ protected static KafkaServer getKafkaServer(int brokerId, File tmpFolder, kafkaProperties.put("zookeeper.connect", zookeeperConnectionString); kafkaProperties.put("message.max.bytes", String.valueOf(50 * 1024 * 1024)); kafkaProperties.put("replica.fetch.max.bytes", String.valueOf(50 * 1024 * 1024)); - + // for CI stability, increase zookeeper session timeout kafkaProperties.put("zookeeper.session.timeout.ms", "20000"); final int numTries = 5; - - for (int i = 1; i <= numTries; i++) { + + for (int i = 1; i <= numTries; i++) { int kafkaPort = NetUtils.getAvailablePort(); kafkaProperties.put("port", Integer.toString(kafkaPort)); KafkaConfig kafkaConfig = new KafkaConfig(kafkaProperties); @@ -286,102 +330,8 @@ protected static KafkaServer getKafkaServer(int brokerId, File tmpFolder, } } } - - throw new Exception("Could not start Kafka after " + numTries + " retries due to port conflicts."); - } - // ------------------------------------------------------------------------ - // Execution utilities - // ------------------------------------------------------------------------ - - protected CuratorFramework createZookeeperClient() { - RetryPolicy retryPolicy = new ExponentialBackoffRetry(100, 10); - CuratorFramework curatorClient = CuratorFrameworkFactory.newClient(standardProps.getProperty("zookeeper.connect"), retryPolicy); - curatorClient.start(); - return curatorClient; - } - - protected static void tryExecute(StreamExecutionEnvironment see, String name) throws Exception { - try { - see.execute(name); - } - catch (ProgramInvocationException | JobExecutionException root) { - Throwable cause = root.getCause(); - - // search for nested SuccessExceptions - int depth = 0; - while (!(cause instanceof SuccessException)) { - if (cause == null || depth++ == 20) { - root.printStackTrace(); - fail("Test failed: " + root.getMessage()); - } - else { - cause = cause.getCause(); - } - } - } - } - - protected static void tryExecutePropagateExceptions(StreamExecutionEnvironment see, String name) throws Exception { - try { - see.execute(name); - } - catch (ProgramInvocationException | JobExecutionException root) { - Throwable cause = root.getCause(); - - // search for nested SuccessExceptions - int depth = 0; - while (!(cause instanceof SuccessException)) { - if (cause == null || depth++ == 20) { - throw root; - } - else { - cause = cause.getCause(); - } - } - } - } - - - - protected static void createTestTopic(String topic, int numberOfPartitions, int replicationFactor) { - - // create topic with one client - Properties topicConfig = new Properties(); - LOG.info("Creating topic {}", topic); - - ZkClient creator = new ZkClient(standardCC.zkConnect(), standardCC.zkSessionTimeoutMs(), - standardCC.zkConnectionTimeoutMs(), new ZooKeeperStringSerializer()); - - AdminUtils.createTopic(creator, topic, numberOfPartitions, replicationFactor, topicConfig); - creator.close(); - - // validate that the topic has been created - final long deadline = System.currentTimeMillis() + 30000; - do { - try { - Thread.sleep(100); - } - catch (InterruptedException e) { - // restore interrupted state - } - List partitions = FlinkKafkaConsumer.getPartitionsForTopic(Collections.singletonList(topic), standardProps); - if (partitions != null && partitions.size() > 0) { - return; - } - } - while (System.currentTimeMillis() < deadline); - fail ("Test topic could not be created"); + throw new Exception("Could not start Kafka after " + numTries + " retries due to port conflicts."); } - - protected static void deleteTestTopic(String topic) { - LOG.info("Deleting topic {}", topic); - - ZkClient zk = new ZkClient(standardCC.zkConnect(), standardCC.zkSessionTimeoutMs(), - standardCC.zkConnectionTimeoutMs(), new ZooKeeperStringSerializer()); - AdminUtils.deleteTopic(zk, topic); - - zk.close(); - } } diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandlerTest.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandlerTest.java similarity index 77% rename from flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandlerTest.java rename to flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandlerTest.java index 8d16da0a46c8a..c99e133cd80a1 100644 --- a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandlerTest.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandlerTest.java @@ -18,17 +18,12 @@ package org.apache.flink.streaming.connectors.kafka.internals; -import kafka.admin.AdminUtils; - -import org.I0Itec.zkclient.ZkClient; import org.apache.curator.framework.CuratorFramework; +import org.apache.flink.streaming.connectors.kafka.KafkaTestEnvironmentImpl; import org.apache.flink.streaming.connectors.kafka.KafkaTestBase; -import org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler; import org.junit.Test; -import java.util.Properties; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; @@ -42,15 +37,9 @@ public void runOffsetManipulationinZooKeeperTest() { final long offset = (long) (Math.random() * Long.MAX_VALUE); - CuratorFramework curatorFramework = createZookeeperClient(); + CuratorFramework curatorFramework = ((KafkaTestEnvironmentImpl)kafkaServer ).createCuratorClient(); + kafkaServer.createTestTopic(topicName, 3, 2); - { - ZkClient zkClient = new ZkClient(standardCC.zkConnect(), standardCC.zkSessionTimeoutMs(), - standardCC.zkConnectionTimeoutMs(), new ZooKeeperStringSerializer()); - AdminUtils.createTopic(zkClient, topicName, 3, 2, new Properties()); - zkClient.close(); - } - ZookeeperOffsetHandler.setOffsetInZooKeeper(curatorFramework, groupId, topicName, 0, offset); long fetchedOffset = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorFramework, groupId, topicName, 0); diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/resources/log4j-test.properties b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/resources/log4j-test.properties similarity index 100% rename from flink-streaming-connectors/flink-connector-kafka/src/test/resources/log4j-test.properties rename to flink-streaming-connectors/flink-connector-kafka-0.8/src/test/resources/log4j-test.properties diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/resources/logback-test.xml b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/resources/logback-test.xml similarity index 100% rename from flink-streaming-connectors/flink-connector-kafka/src/test/resources/logback-test.xml rename to flink-streaming-connectors/flink-connector-kafka-0.8/src/test/resources/logback-test.xml diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/pom.xml b/flink-streaming-connectors/flink-connector-kafka-0.9/pom.xml new file mode 100644 index 0000000000000..b3c974954bea3 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-0.9/pom.xml @@ -0,0 +1,131 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-streaming-connectors-parent + 1.0-SNAPSHOT + .. + + + flink-connector-kafka-0.9 + flink-connector-kafka-0.9 + + jar + + + + 0.9.0.0 + + + + + + org.apache.flink + flink-connector-kafka-base + ${project.version} + + + org.apache.kafka + kafka_${scala.binary.version} + + + + + + org.apache.flink + flink-connector-kafka-base + ${project.version} + + + + org.apache.kafka + kafka_${scala.binary.version} + + + test-jar + test + + + + + org.apache.kafka + kafka-clients + ${kafka.version} + + + + + org.apache.kafka + kafka_${scala.binary.version} + ${kafka.version} + test + + + + org.apache.flink + flink-tests + ${project.version} + test-jar + test + + + + org.apache.flink + flink-streaming-java + ${project.version} + + + + com.google.guava + guava + ${guava.version} + + + + + org.apache.flink + flink-test-utils + ${project.version} + test + + + + + + + + + org.apache.maven.plugins + maven-failsafe-plugin + + + 1 + + + + + + diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java new file mode 100644 index 0000000000000..9faa24956e9f7 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java @@ -0,0 +1,501 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kafka; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.streaming.connectors.kafka.internals.metrics.DefaultKafkaMetricAccumulator; +import org.apache.flink.streaming.util.serialization.DeserializationSchema; + +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.Metric; +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.WakeupException; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import static com.google.common.base.Preconditions.checkNotNull; + +/** + * The Flink Kafka Consumer is a streaming data source that pulls a parallel data stream from + * Apache Kafka 0.9.x. The consumer can run in multiple parallel instances, each of which will pull + * data from one or more Kafka partitions. + * + *

    The Flink Kafka Consumer participates in checkpointing and guarantees that no data is lost + * during a failure, and that the computation processes elements "exactly once". + * (Note: These guarantees naturally assume that Kafka itself does not loose any data.)

    + * + *

    Please note that Flink snapshots the offsets internally as part of its distributed checkpoints. The offsets + * committed to Kafka / ZooKeeper are only to bring the outside view of progress in sync with Flink's view + * of the progress. That way, monitoring and other jobs can get a view of how far the Flink Kafka consumer + * has consumed a topic.

    + * + *

    Please refer to Kafka's documentation for the available configuration properties: + * http://kafka.apache.org/documentation.html#newconsumerconfigs

    + * + *

    NOTE: The implementation currently accesses partition metadata when the consumer + * is constructed. That means that the client that submits the program needs to be able to + * reach the Kafka brokers or ZooKeeper.

    + */ +public class FlinkKafkaConsumer09 extends FlinkKafkaConsumerBase { + + // ------------------------------------------------------------------------ + + private static final long serialVersionUID = 2324564345203409112L; + + private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaConsumer09.class); + + /** Configuration key to change the polling timeout **/ + public static final String KEY_POLL_TIMEOUT = "flink.poll-timeout"; + + /** Boolean configuration key to disable metrics tracking **/ + public static final String KEY_DISABLE_METRICS = "flink.disable-metrics"; + + /** + * From Kafka's Javadoc: The time, in milliseconds, spent waiting in poll if data is not + * available. If 0, returns immediately with any records that are available now. + */ + public static final long DEFAULT_POLL_TIMEOUT = 100L; + + /** User-supplied properties for Kafka **/ + private final Properties properties; + /** Ordered list of all partitions available in all subscribed partitions **/ + private final List partitionInfos; + + // ------ Runtime State ------- + + /** The partitions actually handled by this consumer at runtime */ + private transient List subscribedPartitions; + /** For performance reasons, we are keeping two representations of the subscribed partitions **/ + private transient List subscribedPartitionsAsFlink; + /** The Kafka Consumer instance**/ + private transient KafkaConsumer consumer; + /** The thread running Kafka's consumer **/ + private transient ConsumerThread consumerThread; + /** Exception set from the ConsumerThread */ + private transient Throwable consumerThreadException; + /** If the consumer doesn't have a Kafka partition assigned at runtime, it'll block on this waitThread **/ + private transient Thread waitThread; + + + // ------------------------------------------------------------------------ + + /** + * Creates a new Kafka streaming source consumer for Kafka 0.9.x + * + * @param topic + * The name of the topic that should be consumed. + * @param valueDeserializer + * The de-/serializer used to convert between Kafka's byte messages and Flink's objects. + * @param props + * The properties used to configure the Kafka consumer client, and the ZooKeeper client. + */ + public FlinkKafkaConsumer09(String topic, DeserializationSchema valueDeserializer, Properties props) { + this(Collections.singletonList(topic), valueDeserializer, props); + } + + /** + * Creates a new Kafka streaming source consumer for Kafka 0.9.x + * + * This constructor allows passing a {@see KeyedDeserializationSchema} for reading key/value + * pairs, offsets, and topic names from Kafka. + * + * @param topic + * The name of the topic that should be consumed. + * @param deserializer + * The keyed de-/serializer used to convert between Kafka's byte messages and Flink's objects. + * @param props + * The properties used to configure the Kafka consumer client, and the ZooKeeper client. + */ + public FlinkKafkaConsumer09(String topic, KeyedDeserializationSchema deserializer, Properties props) { + this(Collections.singletonList(topic), deserializer, props); + } + + /** + * Creates a new Kafka streaming source consumer for Kafka 0.9.x + * + * This constructor allows passing multiple topics to the consumer. + * + * @param topics + * The Kafka topics to read from. + * @param deserializer + * The de-/serializer used to convert between Kafka's byte messages and Flink's objects. + * @param props + * The properties that are used to configure both the fetcher and the offset handler. + */ + public FlinkKafkaConsumer09(List topics, DeserializationSchema deserializer, Properties props) { + this(topics, new KeyedDeserializationSchemaWrapper<>(deserializer), props); + } + + /** + * Creates a new Kafka streaming source consumer for Kafka 0.9.x + * + * This constructor allows passing multiple topics and a key/value deserialization schema. + * + * @param topics + * The Kafka topics to read from. + * @param deserializer + * The keyed de-/serializer used to convert between Kafka's byte messages and Flink's objects. + * @param props + * The properties that are used to configure both the fetcher and the offset handler. + */ + public FlinkKafkaConsumer09(List topics, KeyedDeserializationSchema deserializer, Properties props) { + super(deserializer, props); + checkNotNull(topics, "topics"); + this.properties = checkNotNull(props, "props"); + setDeserializer(this.properties); + KafkaConsumer consumer = null; + try { + consumer = new KafkaConsumer<>(this.properties); + this.partitionInfos = new ArrayList<>(); + for (final String topic: topics) { + // get partitions for each topic + List partitionsForTopic = null; + for(int tri = 0; tri < 10; tri++) { + LOG.info("Trying to get partitions for topic {}", topic); + try { + partitionsForTopic = consumer.partitionsFor(topic); + if(partitionsForTopic != null && partitionsForTopic.size() > 0) { + break; // it worked + } + } catch (NullPointerException npe) { + // workaround for KAFKA-2880: Fetcher.getTopicMetadata NullPointerException when broker cannot be reached + // we ignore the NPE. + } + // create a new consumer + consumer.close(); + try { + Thread.sleep(1000); + } catch (InterruptedException e) { + } + consumer = new KafkaConsumer<>(properties); + } + // for non existing topics, the list might be null. + if(partitionsForTopic != null) { + partitionInfos.addAll(convertToFlinkKafkaTopicPartition(partitionsForTopic)); + } + } + } finally { + if(consumer != null) { + consumer.close(); + } + } + if(partitionInfos.isEmpty()) { + throw new RuntimeException("Unable to retrieve any partitions for the requested topics " + topics); + } + + // we now have a list of partitions which is the same for all parallel consumer instances. + LOG.info("Got {} partitions from these topics: {}", partitionInfos.size(), topics); + + if (LOG.isInfoEnabled()) { + logPartitionInfo(partitionInfos); + } + } + + + /** + * Converts a list of Kafka PartitionInfo's to Flink's KafkaTopicPartition (which are serializable) + * @param partitions A list of Kafka PartitionInfos. + * @return A list of KafkaTopicPartitions + */ + public static List convertToFlinkKafkaTopicPartition(List partitions) { + checkNotNull(partitions, "The given list of partitions was null"); + List ret = new ArrayList<>(partitions.size()); + for(PartitionInfo pi: partitions) { + ret.add(new KafkaTopicPartition(pi.topic(), pi.partition())); + } + return ret; + } + + public static List convertToKafkaTopicPartition(List partitions) { + List ret = new ArrayList<>(partitions.size()); + for(KafkaTopicPartition ktp: partitions) { + ret.add(new TopicPartition(ktp.getTopic(), ktp.getPartition())); + } + return ret; + } + + // ------------------------------------------------------------------------ + // Source life cycle + // ------------------------------------------------------------------------ + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + + final int numConsumers = getRuntimeContext().getNumberOfParallelSubtasks(); + final int thisConsumerIndex = getRuntimeContext().getIndexOfThisSubtask(); + + // pick which partitions we work on + this.subscribedPartitionsAsFlink = assignPartitions(this.partitionInfos, numConsumers, thisConsumerIndex); + if(this.subscribedPartitionsAsFlink.isEmpty()) { + LOG.info("This consumer doesn't have any partitions assigned"); + this.offsetsState = null; + return; + } else { + StreamingRuntimeContext streamingRuntimeContext = (StreamingRuntimeContext) getRuntimeContext(); + // if checkpointing is enabled, we are not automatically committing to Kafka. + properties.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, Boolean.toString(!streamingRuntimeContext.isCheckpointingEnabled())); + this.consumer = new KafkaConsumer<>(properties); + } + subscribedPartitions = convertToKafkaTopicPartition(subscribedPartitionsAsFlink); + + this.consumer.assign(this.subscribedPartitions); + + // register Kafka metrics to Flink accumulators + if(!Boolean.getBoolean(properties.getProperty(KEY_DISABLE_METRICS, "false"))) { + Map metrics = this.consumer.metrics(); + if(metrics == null) { + // MapR's Kafka implementation returns null here. + LOG.info("Consumer implementation does not support metrics"); + } else { + for (Map.Entry metric : metrics.entrySet()) { + String name = "consumer-" + metric.getKey().name(); + DefaultKafkaMetricAccumulator kafkaAccumulator = DefaultKafkaMetricAccumulator.createFor(metric.getValue()); + // best effort: we only add the accumulator if available. + if (kafkaAccumulator != null) { + getRuntimeContext().addAccumulator(name, kafkaAccumulator); + } + } + } + } + + // check if we need to explicitly seek to a specific offset (restore case) + if(restoreToOffset != null) { + // we are in a recovery scenario + for(Map.Entry offset: restoreToOffset.entrySet()) { + // seek all offsets to the right position + this.consumer.seek(new TopicPartition(offset.getKey().getTopic(), offset.getKey().getPartition()), offset.getValue() + 1); + } + this.offsetsState = restoreToOffset; + } else { + this.offsetsState = new HashMap<>(); + } + } + + + + @Override + public void run(SourceContext sourceContext) throws Exception { + if(consumer != null) { + consumerThread = new ConsumerThread<>(this, sourceContext); + consumerThread.start(); + // wait for the consumer to stop + while(consumerThread.isAlive()) { + if(consumerThreadException != null) { + throw new RuntimeException("ConsumerThread threw an exception", consumerThreadException); + } + try { + consumerThread.join(50); + } catch (InterruptedException ie) { + consumerThread.shutdown(); + } + } + // check again for an exception + if(consumerThreadException != null) { + throw new RuntimeException("ConsumerThread threw an exception", consumerThreadException); + } + } else { + // this source never completes, so emit a Long.MAX_VALUE watermark + // to not block watermark forwarding + if (getRuntimeContext().getExecutionConfig().areTimestampsEnabled()) { + sourceContext.emitWatermark(new Watermark(Long.MAX_VALUE)); + } + + final Object waitLock = new Object(); + this.waitThread = Thread.currentThread(); + while (running) { + // wait until we are canceled + try { + //noinspection SynchronizationOnLocalVariableOrMethodParameter + synchronized (waitLock) { + waitLock.wait(); + } + } + catch (InterruptedException e) { + // do nothing, check our "running" status + } + } + } + // close the context after the work was done. this can actually only + // happen when the fetcher decides to stop fetching + sourceContext.close(); + } + + @Override + public void cancel() { + // set ourselves as not running + running = false; + if(this.consumerThread != null) { + this.consumerThread.shutdown(); + } else { + // the consumer thread is not running, so we have to interrupt our own thread + if(waitThread != null) { + waitThread.interrupt(); + } + } + } + + @Override + public void close() throws Exception { + cancel(); + super.close(); + } + + // ------------------------------------------------------------------------ + // Checkpoint and restore + // ------------------------------------------------------------------------ + + + @Override + protected void commitOffsets(HashMap checkpointOffsets) { + Map kafkaCheckpointOffsets = convertToCommitMap(checkpointOffsets); + synchronized (this.consumer) { + this.consumer.commitSync(kafkaCheckpointOffsets); + } + } + + public static Map convertToCommitMap(HashMap checkpointOffsets) { + Map ret = new HashMap<>(checkpointOffsets.size()); + for(Map.Entry partitionOffset: checkpointOffsets.entrySet()) { + ret.put(new TopicPartition(partitionOffset.getKey().getTopic(), partitionOffset.getKey().getPartition()), + new OffsetAndMetadata(partitionOffset.getValue(), "")); + } + return ret; + } + + // ------------------------------------------------------------------------ + // Miscellaneous utilities + // ------------------------------------------------------------------------ + + + protected static void setDeserializer(Properties props) { + if (!props.contains(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG)) { + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getCanonicalName()); + } else { + LOG.warn("Overwriting the '{}' is not recommended", ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG); + } + + if (!props.contains(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG)) { + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getCanonicalName()); + } else { + LOG.warn("Overwriting the '{}' is not recommended", ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG); + } + } + + /** + * We use a separate thread for executing the KafkaConsumer.poll(timeout) call because Kafka is not + * handling interrupts properly. On an interrupt (which happens automatically by Flink if the task + * doesn't react to cancel() calls), the poll() method might never return. + * On cancel, we'll wakeup the .poll() call and wait for it to return + */ + private static class ConsumerThread extends Thread { + private final FlinkKafkaConsumer09 flinkKafkaConsumer; + private final SourceContext sourceContext; + private boolean running = true; + + public ConsumerThread(FlinkKafkaConsumer09 flinkKafkaConsumer, SourceContext sourceContext) { + this.flinkKafkaConsumer = flinkKafkaConsumer; + this.sourceContext = sourceContext; + } + + @Override + public void run() { + try { + long pollTimeout = Long.parseLong(flinkKafkaConsumer.properties.getProperty(KEY_POLL_TIMEOUT, Long.toString(DEFAULT_POLL_TIMEOUT))); + pollLoop: while (running) { + ConsumerRecords records; + //noinspection SynchronizeOnNonFinalField + synchronized (flinkKafkaConsumer.consumer) { + try { + records = flinkKafkaConsumer.consumer.poll(pollTimeout); + } catch (WakeupException we) { + if (running) { + throw we; + } + // leave loop + continue; + } + } + // get the records for each topic partition + for (int i = 0; i < flinkKafkaConsumer.subscribedPartitions.size(); i++) { + TopicPartition partition = flinkKafkaConsumer.subscribedPartitions.get(i); + KafkaTopicPartition flinkPartition = flinkKafkaConsumer.subscribedPartitionsAsFlink.get(i); + List> partitionRecords = records.records(partition); + //noinspection ForLoopReplaceableByForEach + for (int j = 0; j < partitionRecords.size(); j++) { + ConsumerRecord record = partitionRecords.get(j); + T value = flinkKafkaConsumer.deserializer.deserialize(record.key(), record.value(), record.topic(), record.partition(),record.offset()); + if(flinkKafkaConsumer.deserializer.isEndOfStream(value)) { + // end of stream signaled + running = false; + break pollLoop; + } + synchronized (sourceContext.getCheckpointLock()) { + sourceContext.collect(value); + flinkKafkaConsumer.offsetsState.put(flinkPartition, record.offset()); + } + } + } + } + } catch(Throwable t) { + if(running) { + this.flinkKafkaConsumer.stopWithError(t); + } else { + LOG.debug("Stopped ConsumerThread threw exception", t); + } + } finally { + try { + flinkKafkaConsumer.consumer.close(); + } catch(Throwable t) { + LOG.warn("Error while closing consumer", t); + } + } + } + + /** + * Try to shutdown the thread + */ + public void shutdown() { + this.running = false; + this.flinkKafkaConsumer.consumer.wakeup(); + } + } + + private void stopWithError(Throwable t) { + this.consumerThreadException = t; + } +} diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer09.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer09.java new file mode 100644 index 0000000000000..6f7f68703a6a5 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer09.java @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kafka; + +import org.apache.flink.streaming.connectors.kafka.partitioner.FixedPartitioner; +import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema; +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper; +import org.apache.flink.streaming.util.serialization.SerializationSchema; + +import java.util.Properties; + + +/** + * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.8. + * + * Please note that this producer does not have any reliability guarantees. + * + * @param Type of the messages to write into Kafka. + */ +public class FlinkKafkaProducer09 extends FlinkKafkaProducerBase { + + private static final long serialVersionUID = 1L; + + // ------------------- Keyless serialization schema constructors ---------------------- + + /** + * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to + * the topic. + * + * @param brokerList + * Comma separated addresses of the brokers + * @param topicId + * ID of the Kafka topic. + * @param serializationSchema + * User defined (keyless) serialization schema. + */ + public FlinkKafkaProducer09(String brokerList, String topicId, SerializationSchema serializationSchema) { + this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), new FixedPartitioner()); + } + + /** + * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to + * the topic. + * + * @param topicId + * ID of the Kafka topic. + * @param serializationSchema + * User defined (keyless) serialization schema. + * @param producerConfig + * Properties with the producer configuration. + */ + public FlinkKafkaProducer09(String topicId, SerializationSchema serializationSchema, Properties producerConfig) { + this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FixedPartitioner()); + } + + /** + * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to + * the topic. + * + * @param topicId The topic to write data to + * @param serializationSchema A (keyless) serializable serialization schema for turning user objects into a kafka-consumable byte[] + * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument. + * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions (when passing null, we'll use Kafka's partitioner) + */ + public FlinkKafkaProducer09(String topicId, SerializationSchema serializationSchema, Properties producerConfig, KafkaPartitioner customPartitioner) { + this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner); + + } + + // ------------------- Key/Value serialization schema constructors ---------------------- + + /** + * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to + * the topic. + * + * @param brokerList + * Comma separated addresses of the brokers + * @param topicId + * ID of the Kafka topic. + * @param serializationSchema + * User defined serialization schema supporting key/value messages + */ + public FlinkKafkaProducer09(String brokerList, String topicId, KeyedSerializationSchema serializationSchema) { + this(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), new FixedPartitioner()); + } + + /** + * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to + * the topic. + * + * @param topicId + * ID of the Kafka topic. + * @param serializationSchema + * User defined serialization schema supporting key/value messages + * @param producerConfig + * Properties with the producer configuration. + */ + public FlinkKafkaProducer09(String topicId, KeyedSerializationSchema serializationSchema, Properties producerConfig) { + this(topicId, serializationSchema, producerConfig, new FixedPartitioner()); + } + + /** + * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to + * the topic. + * + * @param topicId The topic to write data to + * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages + * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument. + * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. + */ + public FlinkKafkaProducer09(String topicId, KeyedSerializationSchema serializationSchema, Properties producerConfig, KafkaPartitioner customPartitioner) { + super(topicId, serializationSchema, producerConfig, customPartitioner); + } +} diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/examples/ReadFromKafka.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/examples/ReadFromKafka.java new file mode 100644 index 0000000000000..643da66b11294 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/examples/ReadFromKafka.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kafka.examples; + +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer09; +import org.apache.flink.streaming.util.serialization.SimpleStringSchema; + + +/** + * Read Strings from Kafka and print them to standard out. + * Note: On a cluster, DataStream.print() will print to the TaskManager's .out file! + * + * Please pass the following arguments to run the example: + * --topic test --bootstrap.servers localhost:9092 --group.id myconsumer + * + */ +public class ReadFromKafka { + + public static void main(String[] args) throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.getConfig().disableSysoutLogging(); + env.setNumberOfExecutionRetries(4); + env.enableCheckpointing(5000); + env.setParallelism(2); + + ParameterTool parameterTool = ParameterTool.fromArgs(args); + + DataStream messageStream = env + .addSource(new FlinkKafkaConsumer09<>( + parameterTool.getRequired("topic"), + new SimpleStringSchema(), + parameterTool.getProperties())); + + messageStream.print(); + + env.execute("Read from Kafka example"); + } +} diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/examples/WriteIntoKafka.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/examples/WriteIntoKafka.java new file mode 100644 index 0000000000000..fbe53fae16f40 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/examples/WriteIntoKafka.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kafka.examples; + +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer09; +import org.apache.flink.streaming.util.serialization.SimpleStringSchema; + + +/** + * Generate a String every 500 ms and write it into a Kafka topic + * + * Please pass the following arguments to run the example: + * --topic test --bootstrap.servers localhost:9092 + * + */ +public class WriteIntoKafka { + + public static void main(String[] args) throws Exception { + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(); + env.getConfig().disableSysoutLogging(); + env.setNumberOfExecutionRetries(4); + env.setParallelism(2); + + ParameterTool parameterTool = ParameterTool.fromArgs(args); + + // very simple data generator + DataStream messageStream = env.addSource(new SourceFunction() { + public boolean running = true; + + @Override + public void run(SourceContext ctx) throws Exception { + long i = 0; + while(this.running) { + ctx.collect("Element - " + i++); + Thread.sleep(500); + } + } + + @Override + public void cancel() { + running = false; + } + }); + + // write data into Kafka + messageStream.addSink(new FlinkKafkaProducer09<>(parameterTool.getRequired("topic"), new SimpleStringSchema(), parameterTool.getProperties())); + + env.execute("Write into Kafka example"); + } +} diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/resources/log4j.properties b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/resources/log4j.properties new file mode 100644 index 0000000000000..6bdfb48cd550f --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/resources/log4j.properties @@ -0,0 +1,29 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +log4j.rootLogger=INFO, testlogger + +log4j.appender.testlogger=org.apache.log4j.ConsoleAppender +log4j.appender.testlogger.target = System.err +log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout +log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n + +# suppress the irrelevant (wrong) warnings from the netty channel handler +log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger + + diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ITCase.java similarity index 75% rename from flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java rename to flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ITCase.java index 07e650ad1d57f..55abaaa9da12a 100644 --- a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ITCase.java @@ -17,47 +17,27 @@ package org.apache.flink.streaming.connectors.kafka; -import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; - import org.junit.Test; -import java.util.List; -import java.util.Properties; +public class Kafka09ITCase extends KafkaConsumerTestBase { -public class KafkaITCase extends KafkaConsumerTestBase { - - @Override - protected FlinkKafkaConsumer getConsumer(List topics, KeyedDeserializationSchema deserializationSchema, Properties props) { - return new FlinkKafkaConsumer082<>(topics, deserializationSchema, props); - } - // ------------------------------------------------------------------------ // Suite of Tests // ------------------------------------------------------------------------ - - @Test + + @Test(timeout = 60000) public void testCheckpointing() throws Exception { runCheckpointingTest(); } - @Test() + @Test(timeout = 60000) public void testFailOnNoBroker() throws Exception { runFailOnNoBrokerTest(); } - @Test - public void testOffsetInZookeeper() throws Exception { - runOffsetInZookeeperValidationTest(); - } - - @Test - public void testOffsetAutocommitTest() throws Exception { - runOffsetAutocommitTest(); - } - - @Test + @Test(timeout = 60000) public void testConcurrentProducerConsumerTopology() throws Exception { runSimpleConcurrentProducerConsumerTopology(); } @@ -68,66 +48,67 @@ public void testKeyValueSupport() throws Exception { } // --- canceling / failures --- - - @Test + + @Test(timeout = 60000) public void testCancelingEmptyTopic() throws Exception { runCancelingOnEmptyInputTest(); } - @Test + @Test(timeout = 60000) public void testCancelingFullTopic() throws Exception { runCancelingOnFullInputTest(); } - @Test + @Test(timeout = 60000) public void testFailOnDeploy() throws Exception { runFailOnDeployTest(); } - @Test - public void testInvalidOffset() throws Exception { - runInvalidOffsetTest(); - } // --- source to partition mappings and exactly once --- - - @Test + + @Test(timeout = 60000) public void testOneToOneSources() throws Exception { runOneToOneExactlyOnceTest(); } - @Test + @Test(timeout = 60000) public void testOneSourceMultiplePartitions() throws Exception { runOneSourceMultiplePartitionsExactlyOnceTest(); } - @Test + @Test(timeout = 60000) public void testMultipleSourcesOnePartition() throws Exception { runMultipleSourcesOnePartitionExactlyOnceTest(); } // --- broker failure --- - @Test + @Test(timeout = 60000) public void testBrokerFailure() throws Exception { runBrokerFailureTest(); } // --- special executions --- - - @Test + + @Test(timeout = 60000) public void testBigRecordJob() throws Exception { runBigRecordTestTopology(); } - @Test + @Test(timeout = 60000) public void testMultipleTopics() throws Exception { runConsumeMultipleTopics(); } - @Test + @Test(timeout = 60000) public void testAllDeletes() throws Exception { runAllDeletesTest(); } + @Test(timeout = 60000) + public void testMetricsAndEndOfStream() throws Exception { + runMetricsAndEndOfStreamTest(); + } + } diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ProducerITCase.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ProducerITCase.java new file mode 100644 index 0000000000000..1288347e28ac5 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ProducerITCase.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kafka; + + +import org.junit.Test; + + +@SuppressWarnings("serial") +public class Kafka09ProducerITCase extends KafkaProducerTestBase { + + @Test + public void testCustomPartitioning() { + runCustomPartitioningTest(); + } + +} diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java new file mode 100644 index 0000000000000..a2c4f736dfe88 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kafka; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.connectors.kafka.testutils.MockRuntimeContext; +import org.apache.flink.streaming.util.serialization.SimpleStringSchema; +import org.apache.flink.util.TestLogger; +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.PartitionInfo; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.util.Arrays; +import java.util.Properties; +import java.util.concurrent.Future; + +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.powermock.api.mockito.PowerMockito.whenNew; + +@RunWith(PowerMockRunner.class) +@PrepareForTest(FlinkKafkaProducerBase.class) +public class KafkaProducerTest extends TestLogger { + + @Test + @SuppressWarnings("unchecked") + public void testPropagateExceptions() { + try { + // mock kafka producer + KafkaProducer kafkaProducerMock = mock(KafkaProducer.class); + + // partition setup + when(kafkaProducerMock.partitionsFor(anyString())).thenReturn( + Arrays.asList(new PartitionInfo("mock_topic", 42, null, null, null))); + + // failure when trying to send an element + when(kafkaProducerMock.send(any(ProducerRecord.class), any(Callback.class))) + .thenAnswer(new Answer>() { + @Override + public Future answer(InvocationOnMock invocation) throws Throwable { + Callback callback = (Callback) invocation.getArguments()[1]; + callback.onCompletion(null, new Exception("Test error")); + return null; + } + }); + + // make sure the FlinkKafkaProducer instantiates our mock producer + whenNew(KafkaProducer.class).withAnyArguments().thenReturn(kafkaProducerMock); + + // (1) producer that propagates errors + + FlinkKafkaProducer09 producerPropagating = new FlinkKafkaProducer09<>( + "mock_topic", new SimpleStringSchema(), new Properties(), null); + + producerPropagating.setRuntimeContext(new MockRuntimeContext(17, 3)); + producerPropagating.open(new Configuration()); + + try { + producerPropagating.invoke("value"); + producerPropagating.invoke("value"); + fail("This should fail with an exception"); + } + catch (Exception e) { + assertNotNull(e.getCause()); + assertNotNull(e.getCause().getMessage()); + assertTrue(e.getCause().getMessage().contains("Test error")); + } + + // (2) producer that only logs errors + + FlinkKafkaProducer09 producerLogging = new FlinkKafkaProducer09<>( + "mock_topic", new SimpleStringSchema(), new Properties(), null); + producerLogging.setLogFailuresOnly(true); + + producerLogging.setRuntimeContext(new MockRuntimeContext(17, 3)); + producerLogging.open(new Configuration()); + + producerLogging.invoke("value"); + producerLogging.invoke("value"); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } +} diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java new file mode 100644 index 0000000000000..0855ba66ac10d --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java @@ -0,0 +1,340 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package org.apache.flink.streaming.connectors.kafka; + +import kafka.admin.AdminUtils; +import kafka.common.KafkaException; +import kafka.consumer.ConsumerConfig; +import kafka.api.PartitionMetadata; +import kafka.network.SocketServer; +import kafka.server.KafkaConfig; +import kafka.server.KafkaServer; +import kafka.utils.SystemTime$; +import kafka.utils.ZkUtils; +import org.I0Itec.zkclient.ZkClient; +import org.apache.commons.io.FileUtils; +import org.apache.curator.test.TestingServer; +import org.apache.flink.streaming.connectors.kafka.internals.ZooKeeperStringSerializer; +import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema; +import org.apache.flink.util.NetUtils; +import org.apache.kafka.common.protocol.SecurityProtocol; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.collection.Seq; + +import java.io.File; +import java.net.BindException; +import java.util.ArrayList; +import java.util.List; +import java.util.Properties; +import java.util.UUID; + +import static org.apache.flink.util.NetUtils.hostAndPortToUrlString; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +/** + * An implementation of the KafkaServerProvider for Kafka 0.9 + */ +public class KafkaTestEnvironmentImpl extends KafkaTestEnvironment { + + protected static final Logger LOG = LoggerFactory.getLogger(KafkaTestEnvironmentImpl.class); + private File tmpZkDir; + private File tmpKafkaParent; + private List tmpKafkaDirs; + private List brokers; + private TestingServer zookeeper; + private String zookeeperConnectionString; + private String brokerConnectionString = ""; + private Properties standardProps; + private ConsumerConfig standardCC; + + + public String getBrokerConnectionString() { + return brokerConnectionString; + } + + @Override + public ConsumerConfig getStandardConsumerConfig() { + return standardCC; + } + + @Override + public Properties getStandardProperties() { + return standardProps; + } + + @Override + public String getVersion() { + return "0.9"; + } + + @Override + public List getBrokers() { + return brokers; + } + + @Override + public FlinkKafkaConsumerBase getConsumer(List topics, KeyedDeserializationSchema readSchema, Properties props) { + return new FlinkKafkaConsumer09<>(topics, readSchema, props); + } + + @Override + public FlinkKafkaProducerBase getProducer(String topic, KeyedSerializationSchema serSchema, Properties props, KafkaPartitioner partitioner) { + return new FlinkKafkaProducer09<>(topic, serSchema, props, partitioner); + } + + @Override + public void restartBroker(int leaderId) throws Exception { + brokers.set(leaderId, getKafkaServer(leaderId, tmpKafkaDirs.get(leaderId), KAFKA_HOST, zookeeperConnectionString)); + } + + @Override + public int getLeaderToShutDown(String topic) throws Exception { + ZkUtils zkUtils = getZkUtils(); + try { + PartitionMetadata firstPart = null; + do { + if (firstPart != null) { + LOG.info("Unable to find leader. error code {}", firstPart.errorCode()); + // not the first try. Sleep a bit + Thread.sleep(150); + } + + Seq partitionMetadata = AdminUtils.fetchTopicMetadataFromZk(topic, zkUtils).partitionsMetadata(); + firstPart = partitionMetadata.head(); + } + while (firstPart.errorCode() != 0); + + return firstPart.leader().get().id(); + } finally { + zkUtils.close(); + } + } + + @Override + public int getBrokerId(KafkaServer server) { + return server.config().brokerId(); + } + + @Override + public void prepare(int numKafkaServers) { + File tempDir = new File(System.getProperty("java.io.tmpdir")); + + tmpZkDir = new File(tempDir, "kafkaITcase-zk-dir-" + (UUID.randomUUID().toString())); + assertTrue("cannot create zookeeper temp dir", tmpZkDir.mkdirs()); + + tmpKafkaParent = new File(tempDir, "kafkaITcase-kafka-dir*" + (UUID.randomUUID().toString())); + assertTrue("cannot create kafka temp dir", tmpKafkaParent.mkdirs()); + + tmpKafkaDirs = new ArrayList<>(numKafkaServers); + for (int i = 0; i < numKafkaServers; i++) { + File tmpDir = new File(tmpKafkaParent, "server-" + i); + assertTrue("cannot create kafka temp dir", tmpDir.mkdir()); + tmpKafkaDirs.add(tmpDir); + } + + zookeeper = null; + brokers = null; + + try { + LOG.info("Starting Zookeeper"); + zookeeper = new TestingServer(-1, tmpZkDir); + zookeeperConnectionString = zookeeper.getConnectString(); + + LOG.info("Starting KafkaServer"); + brokers = new ArrayList<>(numKafkaServers); + + for (int i = 0; i < numKafkaServers; i++) { + brokers.add(getKafkaServer(i, tmpKafkaDirs.get(i), KafkaTestEnvironment.KAFKA_HOST, zookeeperConnectionString)); + + SocketServer socketServer = brokers.get(i).socketServer(); + brokerConnectionString += hostAndPortToUrlString(KafkaTestEnvironment.KAFKA_HOST, brokers.get(i).socketServer().boundPort(SecurityProtocol.PLAINTEXT)) + ","; + } + + LOG.info("ZK and KafkaServer started."); + } + catch (Throwable t) { + t.printStackTrace(); + fail("Test setup failed: " + t.getMessage()); + } + + standardProps = new Properties(); + standardProps.setProperty("zookeeper.connect", zookeeperConnectionString); + standardProps.setProperty("bootstrap.servers", brokerConnectionString); + standardProps.setProperty("group.id", "flink-tests"); + standardProps.setProperty("auto.commit.enable", "false"); + standardProps.setProperty("zookeeper.session.timeout.ms", "12000"); // 6 seconds is default. Seems to be too small for travis. + standardProps.setProperty("zookeeper.connection.timeout.ms", "20000"); + standardProps.setProperty("auto.offset.reset", "earliest"); // read from the beginning. + standardProps.setProperty("fetch.message.max.bytes", "256"); // make a lot of fetches (MESSAGES MUST BE SMALLER!) + + Properties consumerConfigProps = new Properties(); + consumerConfigProps.putAll(standardProps); + consumerConfigProps.setProperty("auto.offset.reset", "smallest"); + standardCC = new ConsumerConfig(consumerConfigProps); + } + + @Override + public void shutdown() { + for (KafkaServer broker : brokers) { + if (broker != null) { + broker.shutdown(); + } + } + brokers.clear(); + + if (zookeeper != null) { + try { + zookeeper.stop(); + } + catch (Exception e) { + LOG.warn("ZK.stop() failed", e); + } + zookeeper = null; + } + + // clean up the temp spaces + + if (tmpKafkaParent != null && tmpKafkaParent.exists()) { + try { + FileUtils.deleteDirectory(tmpKafkaParent); + } + catch (Exception e) { + // ignore + } + } + if (tmpZkDir != null && tmpZkDir.exists()) { + try { + FileUtils.deleteDirectory(tmpZkDir); + } + catch (Exception e) { + // ignore + } + } + } + + public ZkUtils getZkUtils() { + ZkClient creator = new ZkClient(standardCC.zkConnect(), standardCC.zkSessionTimeoutMs(), + standardCC.zkConnectionTimeoutMs(), new ZooKeeperStringSerializer()); + return ZkUtils.apply(creator, false); + } + + @Override + public void createTestTopic(String topic, int numberOfPartitions, int replicationFactor) { + // create topic with one client + Properties topicConfig = new Properties(); + LOG.info("Creating topic {}", topic); + + ZkUtils zkUtils = getZkUtils(); + try { + AdminUtils.createTopic(zkUtils, topic, numberOfPartitions, replicationFactor, topicConfig); + } finally { + zkUtils.close(); + } + + // validate that the topic has been created + final long deadline = System.currentTimeMillis() + 30000; + do { + try { + Thread.sleep(100); + } catch (InterruptedException e) { + // restore interrupted state + } + // we could use AdminUtils.topicExists(zkUtils, topic) here, but it's results are + // not always correct. + + // create a new ZK utils connection + ZkUtils checkZKConn = getZkUtils(); + if(AdminUtils.topicExists(checkZKConn, topic)) { + checkZKConn.close(); + return; + } + checkZKConn.close(); + } + while (System.currentTimeMillis() < deadline); + fail("Test topic could not be created"); + } + + @Override + public void deleteTestTopic(String topic) { + ZkUtils zkUtils = getZkUtils(); + try { + LOG.info("Deleting topic {}", topic); + + ZkClient zk = new ZkClient(standardCC.zkConnect(), standardCC.zkSessionTimeoutMs(), + standardCC.zkConnectionTimeoutMs(), new ZooKeeperStringSerializer()); + + AdminUtils.deleteTopic(zkUtils, topic); + + zk.close(); + } finally { + zkUtils.close(); + } + } + + /** + * Copied from com.github.sakserv.minicluster.KafkaLocalBrokerIntegrationTest (ASL licensed) + */ + protected static KafkaServer getKafkaServer(int brokerId, File tmpFolder, + String kafkaHost, + String zookeeperConnectionString) throws Exception { + Properties kafkaProperties = new Properties(); + + // properties have to be Strings + kafkaProperties.put("advertised.host.name", kafkaHost); + kafkaProperties.put("broker.id", Integer.toString(brokerId)); + kafkaProperties.put("log.dir", tmpFolder.toString()); + kafkaProperties.put("zookeeper.connect", zookeeperConnectionString); + kafkaProperties.put("message.max.bytes", String.valueOf(50 * 1024 * 1024)); + kafkaProperties.put("replica.fetch.max.bytes", String.valueOf(50 * 1024 * 1024)); + + // for CI stability, increase zookeeper session timeout + kafkaProperties.put("zookeeper.session.timeout.ms", "20000"); + + final int numTries = 5; + + for (int i = 1; i <= numTries; i++) { + int kafkaPort = NetUtils.getAvailablePort(); + kafkaProperties.put("port", Integer.toString(kafkaPort)); + KafkaConfig kafkaConfig = new KafkaConfig(kafkaProperties); + + try { + scala.Option stringNone = scala.Option.apply(null); + KafkaServer server = new KafkaServer(kafkaConfig, SystemTime$.MODULE$, stringNone); + server.startup(); + return server; + } + catch (KafkaException e) { + if (e.getCause() instanceof BindException) { + // port conflict, retry... + LOG.info("Port conflict when starting Kafka Broker. Retrying..."); + } + else { + throw e; + } + } + } + + throw new Exception("Could not start Kafka after " + numTries + " retries due to port conflicts."); + } + +} diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/resources/log4j-test.properties b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/resources/log4j-test.properties new file mode 100644 index 0000000000000..6bdfb48cd550f --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/resources/log4j-test.properties @@ -0,0 +1,29 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +log4j.rootLogger=INFO, testlogger + +log4j.appender.testlogger=org.apache.log4j.ConsoleAppender +log4j.appender.testlogger.target = System.err +log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout +log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n + +# suppress the irrelevant (wrong) warnings from the netty channel handler +log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger + + diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/resources/logback-test.xml b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/resources/logback-test.xml new file mode 100644 index 0000000000000..45b3b92f0cf65 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/resources/logback-test.xml @@ -0,0 +1,30 @@ + + + + + + %d{HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n + + + + + + + + \ No newline at end of file diff --git a/flink-streaming-connectors/flink-connector-kafka-base/pom.xml b/flink-streaming-connectors/flink-connector-kafka-base/pom.xml new file mode 100644 index 0000000000000..354d3538eb30b --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-base/pom.xml @@ -0,0 +1,169 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-streaming-connectors-parent + 1.0-SNAPSHOT + .. + + + flink-connector-kafka-base + flink-connector-kafka-base + + jar + + + + 0.8.2.0 + + + + + + org.apache.flink + flink-streaming-java + ${project.version} + + + + org.apache.kafka + kafka_${scala.binary.version} + ${kafka.version} + + + com.sun.jmx + jmxri + + + com.sun.jdmk + jmxtools + + + log4j + log4j + + + org.slf4j + slf4j-simple + + + net.sf.jopt-simple + jopt-simple + + + org.scala-lang + scala-reflect + + + org.scala-lang + scala-compiler + + + com.yammer.metrics + metrics-annotation + + + org.xerial.snappy + snappy-java + + + + + + + com.101tec + zkclient + 0.7 + jar + + + + com.google.guava + guava + ${guava.version} + + + + org.apache.curator + curator-test + ${curator.version} + test + + + + org.apache.flink + flink-tests + ${project.version} + test-jar + test + + + + org.apache.flink + flink-test-utils + ${project.version} + test + + + + + + + + com.101tec + zkclient + 0.7 + + + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + + + + org.apache.maven.plugins + maven-failsafe-plugin + + + 1 + + + + + + diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java new file mode 100644 index 0000000000000..3c3658686a8aa --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java @@ -0,0 +1,225 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kafka; + +import org.apache.commons.collections.map.LinkedMap; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier; +import org.apache.flink.streaming.api.checkpoint.CheckpointedAsynchronously; +import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; + + +public abstract class FlinkKafkaConsumerBase extends RichParallelSourceFunction + implements CheckpointNotifier, CheckpointedAsynchronously>, ResultTypeQueryable { + + // ------------------------------------------------------------------------ + + private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaConsumerBase.class); + + private static final long serialVersionUID = -6272159445203409112L; + + /** The maximum number of pending non-committed checkpoints to track, to avoid memory leaks */ + public static final int MAX_NUM_PENDING_CHECKPOINTS = 100; + + + /** The schema to convert between Kafka#s byte messages, and Flink's objects */ + protected final KeyedDeserializationSchema deserializer; + + // ------ Runtime State ------- + + /** Data for pending but uncommitted checkpoints */ + protected final LinkedMap pendingCheckpoints = new LinkedMap(); + + /** The offsets of the last returned elements */ + protected transient HashMap offsetsState; + + /** The offsets to restore to, if the consumer restores state from a checkpoint */ + protected transient HashMap restoreToOffset; + + /** Flag indicating whether the consumer is still running **/ + protected volatile boolean running = true; + + // ------------------------------------------------------------------------ + + + /** + * Creates a new Flink Kafka Consumer, using the given type of fetcher and offset handler. + * + *

    To determine which kink of fetcher and offset handler to use, please refer to the docs + * at the beginnign of this class.

    + * + * @param deserializer + * The deserializer to turn raw byte messages into Java/Scala objects. + * @param props + * The properties that are used to configure both the fetcher and the offset handler. + */ + public FlinkKafkaConsumerBase(KeyedDeserializationSchema deserializer, Properties props) { + this.deserializer = checkNotNull(deserializer, "valueDeserializer"); + } + + // ------------------------------------------------------------------------ + // Checkpoint and restore + // ------------------------------------------------------------------------ + + @Override + public HashMap snapshotState(long checkpointId, long checkpointTimestamp) throws Exception { + if (offsetsState == null) { + LOG.debug("snapshotState() requested on not yet opened source; returning null."); + return null; + } + if (!running) { + LOG.debug("snapshotState() called on closed source"); + return null; + } + + if (LOG.isDebugEnabled()) { + LOG.debug("Snapshotting state. Offsets: {}, checkpoint id: {}, timestamp: {}", + KafkaTopicPartition.toString(offsetsState), checkpointId, checkpointTimestamp); + } + + // the use of clone() is okay here is okay, we just need a new map, the keys are not changed + //noinspection unchecked + HashMap currentOffsets = (HashMap) offsetsState.clone(); + + // the map cannot be asynchronously updated, because only one checkpoint call can happen + // on this function at a time: either snapshotState() or notifyCheckpointComplete() + pendingCheckpoints.put(checkpointId, currentOffsets); + + while (pendingCheckpoints.size() > MAX_NUM_PENDING_CHECKPOINTS) { + pendingCheckpoints.remove(0); + } + + return currentOffsets; + } + + @Override + public void restoreState(HashMap restoredOffsets) { + LOG.info("Setting restore state in Kafka"); + restoreToOffset = restoredOffsets; + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + if (offsetsState == null) { + LOG.debug("notifyCheckpointComplete() called on uninitialized source"); + return; + } + if (!running) { + LOG.debug("notifyCheckpointComplete() called on closed source"); + return; + } + + // only one commit operation must be in progress + if (LOG.isDebugEnabled()) { + LOG.debug("Committing offsets externally for checkpoint {}", checkpointId); + } + + try { + HashMap checkpointOffsets; + + // the map may be asynchronously updates when snapshotting state, so we synchronize + synchronized (pendingCheckpoints) { + final int posInMap = pendingCheckpoints.indexOf(checkpointId); + if (posInMap == -1) { + LOG.warn("Received confirmation for unknown checkpoint id {}", checkpointId); + return; + } + + //noinspection unchecked + checkpointOffsets = (HashMap) pendingCheckpoints.remove(posInMap); + + + // remove older checkpoints in map + for (int i = 0; i < posInMap; i++) { + pendingCheckpoints.remove(0); + } + } + if (checkpointOffsets == null || checkpointOffsets.size() == 0) { + LOG.debug("Checkpoint state was empty."); + return; + } + commitOffsets(checkpointOffsets); + } + catch (Exception e) { + if (running) { + throw e; + } + // else ignore exception if we are no longer running + } + } + + protected abstract void commitOffsets(HashMap checkpointOffsets) throws Exception; + + + @Override + public TypeInformation getProducedType() { + return deserializer.getProducedType(); + } + + protected static List assignPartitions(List partitions, int numConsumers, int consumerIndex) { + checkArgument(numConsumers > 0); + checkArgument(consumerIndex < numConsumers); + + List partitionsToSub = new ArrayList<>(); + + for (int i = 0; i < partitions.size(); i++) { + if (i % numConsumers == consumerIndex) { + partitionsToSub.add(partitions.get(i)); + } + } + return partitionsToSub; + } + + /** + * Method to log partition information. + * @param partitionInfos List of subscribed partitions + */ + public static void logPartitionInfo(List partitionInfos) { + Map countPerTopic = new HashMap<>(); + for (KafkaTopicPartition partition : partitionInfos) { + Integer count = countPerTopic.get(partition.getTopic()); + if (count == null) { + count = 1; + } else { + count++; + } + countPerTopic.put(partition.getTopic(), count); + } + StringBuilder sb = new StringBuilder(); + for (Map.Entry e : countPerTopic.entrySet()) { + sb.append(e.getKey()).append(" (").append(e.getValue()).append("), "); + } + LOG.info("Consumer is going to read the following topics (with number of partitions): ", sb.toString()); + } + + +} diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBase.java similarity index 63% rename from flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java rename to flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBase.java index 7e01b54f42c50..ebc02c997bdcd 100644 --- a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBase.java @@ -23,11 +23,9 @@ import org.apache.flink.api.java.ClosureCleaner; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; -import org.apache.flink.streaming.connectors.kafka.partitioner.FixedPartitioner; +import org.apache.flink.streaming.connectors.kafka.internals.metrics.DefaultKafkaMetricAccumulator; import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema; -import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper; -import org.apache.flink.streaming.util.serialization.SerializationSchema; import org.apache.flink.util.NetUtils; import org.apache.kafka.clients.producer.Callback; @@ -35,6 +33,8 @@ import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.Metric; +import org.apache.kafka.common.MetricName; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.serialization.ByteArraySerializer; @@ -42,6 +42,7 @@ import org.slf4j.LoggerFactory; import java.util.List; +import java.util.Map; import java.util.Properties; @@ -52,130 +53,60 @@ * * @param Type of the messages to write into Kafka. */ -public class FlinkKafkaProducer extends RichSinkFunction { +public abstract class FlinkKafkaProducerBase extends RichSinkFunction { - private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaProducer.class); + private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaProducerBase.class); private static final long serialVersionUID = 1L; + /** + * Configuration key for disabling the metrics reporting + */ + public static final String KEY_DISABLE_METRICS = "flink.disable-metrics"; + /** * Array with the partition ids of the given topicId * The size of this array is the number of partitions */ - private final int[] partitions; + protected final int[] partitions; /** * User defined properties for the Producer */ - private final Properties producerConfig; + protected final Properties producerConfig; /** * The name of the topic this producer is writing data to */ - private final String topicId; + protected final String topicId; /** * (Serializable) SerializationSchema for turning objects used with Flink into * byte[] for Kafka. */ - private final KeyedSerializationSchema schema; + protected final KeyedSerializationSchema schema; /** * User-provided partitioner for assigning an object to a Kafka partition. */ - private final KafkaPartitioner partitioner; + protected final KafkaPartitioner partitioner; /** * Flag indicating whether to accept failures (and log them), or to fail on failures */ - private boolean logFailuresOnly; + protected boolean logFailuresOnly; // -------------------------------- Runtime fields ------------------------------------------ /** KafkaProducer instance */ - private transient KafkaProducer producer; + protected transient KafkaProducer producer; /** The callback than handles error propagation or logging callbacks */ - private transient Callback callback; + protected transient Callback callback; /** Errors encountered in the async producer are stored here */ - private transient volatile Exception asyncException; - - // ------------------- Keyless serialization schema constructors ---------------------- - /** - * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to - * the topic. - * - * @param brokerList - * Comma separated addresses of the brokers - * @param topicId - * ID of the Kafka topic. - * @param serializationSchema - * User defined (keyless) serialization schema. - */ - public FlinkKafkaProducer(String brokerList, String topicId, SerializationSchema serializationSchema) { - this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), null); - } - - /** - * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to - * the topic. - * - * @param topicId - * ID of the Kafka topic. - * @param serializationSchema - * User defined (keyless) serialization schema. - * @param producerConfig - * Properties with the producer configuration. - */ - public FlinkKafkaProducer(String topicId, SerializationSchema serializationSchema, Properties producerConfig) { - this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, null); - } - - /** - * The main constructor for creating a FlinkKafkaProducer. - * - * @param topicId The topic to write data to - * @param serializationSchema A (keyless) serializable serialization schema for turning user objects into a kafka-consumable byte[] - * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument. - * @param customPartitioner A serializable partitioner for assining messages to Kafka partitions. - */ - public FlinkKafkaProducer(String topicId, SerializationSchema serializationSchema, Properties producerConfig, KafkaPartitioner customPartitioner) { - this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner); - - } - - // ------------------- Key/Value serialization schema constructors ---------------------- + protected transient volatile Exception asyncException; - /** - * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to - * the topic. - * - * @param brokerList - * Comma separated addresses of the brokers - * @param topicId - * ID of the Kafka topic. - * @param serializationSchema - * User defined serialization schema supporting key/value messages - */ - public FlinkKafkaProducer(String brokerList, String topicId, KeyedSerializationSchema serializationSchema) { - this(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), null); - } - - /** - * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to - * the topic. - * - * @param topicId - * ID of the Kafka topic. - * @param serializationSchema - * User defined serialization schema supporting key/value messages - * @param producerConfig - * Properties with the producer configuration. - */ - public FlinkKafkaProducer(String topicId, KeyedSerializationSchema serializationSchema, Properties producerConfig) { - this(topicId, serializationSchema, producerConfig, null); - } /** * The main constructor for creating a FlinkKafkaProducer. @@ -183,9 +114,9 @@ public FlinkKafkaProducer(String topicId, KeyedSerializationSchema serializa * @param topicId The topic to write data to * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument. - * @param customPartitioner A serializable partitioner for assining messages to Kafka partitions. + * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner */ - public FlinkKafkaProducer(String topicId, KeyedSerializationSchema serializationSchema, Properties producerConfig, KafkaPartitioner customPartitioner) { + public FlinkKafkaProducerBase(String topicId, KeyedSerializationSchema serializationSchema, Properties producerConfig, KafkaPartitioner customPartitioner) { Preconditions.checkNotNull(topicId, "TopicID not set"); Preconditions.checkNotNull(serializationSchema, "serializationSchema not set"); Preconditions.checkNotNull(producerConfig, "producerConfig not set"); @@ -223,11 +154,7 @@ public FlinkKafkaProducer(String topicId, KeyedSerializationSchema serializa getPartitionsProd.close(); } - if (customPartitioner == null) { - this.partitioner = new FixedPartitioner(); - } else { - this.partitioner = customPartitioner; - } + this.partitioner = customPartitioner; } // ---------------------------------- Properties -------------------------- @@ -254,11 +181,32 @@ public void open(Configuration configuration) { producer = new org.apache.kafka.clients.producer.KafkaProducer<>(this.producerConfig); RuntimeContext ctx = getRuntimeContext(); - partitioner.open(ctx.getIndexOfThisSubtask(), ctx.getNumberOfParallelSubtasks(), partitions); + if(partitioner != null) { + partitioner.open(ctx.getIndexOfThisSubtask(), ctx.getNumberOfParallelSubtasks(), partitions); + } LOG.info("Starting FlinkKafkaProducer ({}/{}) to produce into topic {}", ctx.getIndexOfThisSubtask(), ctx.getNumberOfParallelSubtasks(), topicId); - + + // register Kafka metrics to Flink accumulators + if(!Boolean.getBoolean(producerConfig.getProperty(KEY_DISABLE_METRICS, "false"))) { + Map metrics = this.producer.metrics(); + + if(metrics == null) { + // MapR's Kafka implementation returns null here. + LOG.info("Producer implementation does not support metrics"); + } else { + for(Map.Entry metric: metrics.entrySet()) { + String name = "producer-" + metric.getKey().name(); + DefaultKafkaMetricAccumulator kafkaAccumulator = DefaultKafkaMetricAccumulator.createFor(metric.getValue()); + // best effort: we only add the accumulator if available. + if(kafkaAccumulator != null) { + getRuntimeContext().addAccumulator(name, kafkaAccumulator); + } + } + } + } + if (logFailuresOnly) { callback = new Callback() { @@ -295,10 +243,13 @@ public void invoke(IN next) throws Exception { byte[] serializedKey = schema.serializeKey(next); byte[] serializedValue = schema.serializeValue(next); - ProducerRecord record = new ProducerRecord<>(topicId, - partitioner.partition(next, partitions.length), - serializedKey, serializedValue); - + ProducerRecord record; + if(partitioner == null) { + record = new ProducerRecord<>(topicId, serializedKey, serializedValue); + } else { + record = new ProducerRecord<>(topicId, partitioner.partition(next, serializedKey, serializedValue, partitions.length), serializedKey, serializedValue); + } + producer.send(record, callback); } @@ -316,7 +267,7 @@ public void close() throws Exception { // ----------------------------------- Utilities -------------------------- - private void checkErroneous() throws Exception { + protected void checkErroneous() throws Exception { Exception e = asyncException; if (e != null) { // prevent double throwing diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java similarity index 92% rename from flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java rename to flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java index f269aa32d34f7..1168b2734322d 100644 --- a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java @@ -121,4 +121,12 @@ public boolean isContained(Map map) { } return false; } + + public static List convertToPartitionInfo(List partitionInfos) { + List ret = new ArrayList<>(partitionInfos.size()); + for(KafkaTopicPartitionLeader ktpl: partitionInfos) { + ret.add(ktpl.getTopicPartition()); + } + return ret; + } } diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionLeader.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionLeader.java similarity index 100% rename from flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionLeader.java rename to flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionLeader.java diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZooKeeperStringSerializer.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZooKeeperStringSerializer.java similarity index 100% rename from flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZooKeeperStringSerializer.java rename to flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZooKeeperStringSerializer.java diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/AvgKafkaMetricAccumulator.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/AvgKafkaMetricAccumulator.java new file mode 100644 index 0000000000000..a0387116cd263 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/AvgKafkaMetricAccumulator.java @@ -0,0 +1,141 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.streaming.connectors.kafka.internals.metrics; + +import org.apache.flink.api.common.accumulators.Accumulator; +import org.apache.kafka.common.metrics.KafkaMetric; +import org.apache.kafka.common.metrics.Measurable; +import org.apache.kafka.common.metrics.stats.Avg; +import org.apache.kafka.common.metrics.stats.SampledStat; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import java.lang.reflect.Field; +import java.util.List; + +public class AvgKafkaMetricAccumulator extends DefaultKafkaMetricAccumulator { + private static final Logger LOG = LoggerFactory.getLogger(AvgKafkaMetricAccumulator.class); + + /** The last sum/count before the serialization **/ + private AvgSumCount lastSumCount; + + public AvgKafkaMetricAccumulator(KafkaMetric kafkaMetric) { + super(kafkaMetric); + } + + @Override + public void merge(Accumulator other) { + if(!(other instanceof AvgKafkaMetricAccumulator)) { + throw new RuntimeException("Trying to merge incompatible accumulators: "+this+" with "+other); + } + AvgKafkaMetricAccumulator otherMetric = (AvgKafkaMetricAccumulator) other; + + AvgSumCount thisAvg; + if(this.lastSumCount == null) { + Measurable thisMeasurable = DefaultKafkaMetricAccumulator.getMeasurableFromKafkaMetric(this.kafkaMetric); + if (!(thisMeasurable instanceof Avg)) { + throw new RuntimeException("Must be of type Avg"); + } + thisAvg = getAvgSumCount((Avg) thisMeasurable); + } else { + thisAvg = this.lastSumCount; + } + + AvgSumCount otherAvg; + if(otherMetric.lastSumCount == null) { + Measurable otherMeasurable = DefaultKafkaMetricAccumulator.getMeasurableFromKafkaMetric(otherMetric.kafkaMetric); + if(!(otherMeasurable instanceof Avg) ) { + throw new RuntimeException("Must be of type Avg"); + } + otherAvg = getAvgSumCount((Avg) otherMeasurable); + } else { + otherAvg = otherMetric.lastSumCount; + } + + thisAvg.count += otherAvg.count; + thisAvg.sum += otherAvg.sum; + this.mergedValue = thisAvg.sum / thisAvg.count; + } + + @Override + public Accumulator clone() { + AvgKafkaMetricAccumulator clone = new AvgKafkaMetricAccumulator(kafkaMetric); + clone.lastSumCount = this.lastSumCount; + clone.isMerged = this.isMerged; + clone.mergedValue = this.mergedValue; + return clone; + } + + // ------------ Utilities + + private static class AvgSumCount implements Serializable { + double sum; + long count; + + @Override + public String toString() { + return "AvgSumCount{" + + "sum=" + sum + + ", count=" + count + + ", avg="+(sum/count)+"}"; + } + } + + /** + * Extracts sum and count from Avg using reflection + * + * @param avg Avg SampledStat from Kafka + * @return A KV pair with the average's sum and count + */ + private static AvgSumCount getAvgSumCount(Avg avg) { + try { + Field samplesField = SampledStat.class.getDeclaredField("samples"); + Field sampleValue = Class.forName("org.apache.kafka.common.metrics.stats.SampledStat$Sample").getDeclaredField("value"); + Field sampleEventCount = Class.forName("org.apache.kafka.common.metrics.stats.SampledStat$Sample").getDeclaredField("eventCount"); + samplesField.setAccessible(true); + sampleValue.setAccessible(true); + sampleEventCount.setAccessible(true); + List samples = (List) samplesField.get(avg); + AvgSumCount res = new AvgSumCount(); + for(int i = 0; i < samples.size(); i++) { + res.sum += (double)sampleValue.get(samples.get(i)); + res.count += (long)sampleEventCount.get(samples.get(i)); + } + return res; + } catch(Throwable t) { + throw new RuntimeException("Unable to extract sum and count from Avg using reflection. " + + "You can turn off the metrics from Flink's Kafka connector if this issue persists.", t); + } + } + + private void writeObject(ObjectOutputStream out) throws IOException { + Measurable thisMeasurable = DefaultKafkaMetricAccumulator.getMeasurableFromKafkaMetric(this.kafkaMetric); + if(!(thisMeasurable instanceof Avg) ) { + throw new RuntimeException("Must be of type Avg"); + } + this.lastSumCount = getAvgSumCount((Avg) thisMeasurable); + out.defaultWriteObject(); + } + + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + in.defaultReadObject(); + } +} diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/DefaultKafkaMetricAccumulator.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/DefaultKafkaMetricAccumulator.java new file mode 100644 index 0000000000000..06b79300c7bd8 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/DefaultKafkaMetricAccumulator.java @@ -0,0 +1,159 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kafka.internals.metrics; + +import org.apache.flink.api.common.accumulators.Accumulator; +import org.apache.kafka.common.Metric; +import org.apache.kafka.common.metrics.KafkaMetric; +import org.apache.kafka.common.metrics.Measurable; +import org.apache.kafka.common.metrics.stats.Avg; +import org.apache.kafka.common.metrics.stats.Max; +import org.apache.kafka.common.metrics.stats.Min; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import java.lang.reflect.Field; + +public class DefaultKafkaMetricAccumulator implements Accumulator, Serializable { + + private static final Logger LOG = LoggerFactory.getLogger(DefaultKafkaMetricAccumulator.class); + + protected boolean isMerged = false; + protected double mergedValue; + protected transient KafkaMetric kafkaMetric; + + + public static DefaultKafkaMetricAccumulator createFor(Metric metric) { + if(!(metric instanceof KafkaMetric)) { + return null; + } + KafkaMetric kafkaMetric = (KafkaMetric) metric; + Measurable measurable = getMeasurableFromKafkaMetric(kafkaMetric); + if(measurable == null) { + return null; + } + if (measurable instanceof Max) { + return new MaxKafkaMetricAccumulator(kafkaMetric); + } else if (measurable instanceof Min) { + return new MinKafkaMetricAccumulator(kafkaMetric); + } else if (measurable instanceof Avg) { + return new AvgKafkaMetricAccumulator(kafkaMetric); + } else { + // fallback accumulator. works for Rate, Total, Count. + return new DefaultKafkaMetricAccumulator(kafkaMetric); + } + } + + /** + * This utility method is using reflection to get the Measurable from the KafkaMetric. + * Since Kafka 0.9, Kafka is exposing the Measurable properly, but Kafka 0.8.2 does not yet expose it. + * + * @param kafkaMetric the metric to extract the field form + * @return Measurable type (or null in case of an error) + */ + protected static Measurable getMeasurableFromKafkaMetric(KafkaMetric kafkaMetric) { + try { + Field measurableField = kafkaMetric.getClass().getDeclaredField("measurable"); + measurableField.setAccessible(true); + return (Measurable) measurableField.get(kafkaMetric); + } catch (Throwable e) { + LOG.warn("Unable to initialize Kafka metric: " + kafkaMetric, e); + return null; + } + } + + + DefaultKafkaMetricAccumulator(KafkaMetric kafkaMetric) { + this.kafkaMetric = kafkaMetric; + } + + @Override + public void add(Void value) { + // noop + } + + @Override + public Double getLocalValue() { + if(isMerged && kafkaMetric == null) { + return mergedValue; + } + return kafkaMetric.value(); + } + + @Override + public void resetLocal() { + // noop + } + + @Override + public void merge(Accumulator other) { + if(!(other instanceof DefaultKafkaMetricAccumulator)) { + throw new RuntimeException("Trying to merge incompatible accumulators"); + } + DefaultKafkaMetricAccumulator otherMetric = (DefaultKafkaMetricAccumulator) other; + if(this.isMerged) { + if(otherMetric.isMerged) { + this.mergedValue += otherMetric.mergedValue; + } else { + this.mergedValue += otherMetric.getLocalValue(); + } + } else { + this.isMerged = true; + if(otherMetric.isMerged) { + this.mergedValue = this.getLocalValue() + otherMetric.mergedValue; + } else { + this.mergedValue = this.getLocalValue() + otherMetric.getLocalValue(); + } + + } + } + + @Override + public Accumulator clone() { + DefaultKafkaMetricAccumulator clone = new DefaultKafkaMetricAccumulator(this.kafkaMetric); + clone.isMerged = this.isMerged; + clone.mergedValue = this.mergedValue; + return clone; + } + + @Override + public String toString() { + if(isMerged) { + return Double.toString(mergedValue); + } + if(kafkaMetric == null) { + return "null"; + } + return Double.toString(kafkaMetric.value()); + } + + // -------- custom serialization methods + private void writeObject(ObjectOutputStream out) throws IOException { + this.isMerged = true; + this.mergedValue = kafkaMetric.value(); + out.defaultWriteObject(); + } + + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + in.defaultReadObject(); + } +} \ No newline at end of file diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/MaxKafkaMetricAccumulator.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/MaxKafkaMetricAccumulator.java new file mode 100644 index 0000000000000..c1770ff4eda60 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/MaxKafkaMetricAccumulator.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.streaming.connectors.kafka.internals.metrics; + +import org.apache.flink.api.common.accumulators.Accumulator; +import org.apache.kafka.common.metrics.KafkaMetric; + + +public class MaxKafkaMetricAccumulator extends DefaultKafkaMetricAccumulator { + public MaxKafkaMetricAccumulator(KafkaMetric kafkaMetric) { + super(kafkaMetric); + } + + @Override + public void merge(Accumulator other) { + if(!(other instanceof MaxKafkaMetricAccumulator)) { + throw new RuntimeException("Trying to merge incompatible accumulators"); + } + MaxKafkaMetricAccumulator otherMetric = (MaxKafkaMetricAccumulator) other; + if(this.isMerged) { + if(otherMetric.isMerged) { + this.mergedValue = Math.max(this.mergedValue, otherMetric.mergedValue); + } else { + this.mergedValue = Math.max(this.mergedValue, otherMetric.getLocalValue()); + } + } else { + this.isMerged = true; + if(otherMetric.isMerged) { + this.mergedValue = Math.max(this.getLocalValue(), otherMetric.mergedValue); + } else { + this.mergedValue = Math.max(this.getLocalValue(), otherMetric.getLocalValue()); + } + } + } + + @Override + public Accumulator clone() { + MaxKafkaMetricAccumulator clone = new MaxKafkaMetricAccumulator(this.kafkaMetric); + clone.isMerged = this.isMerged; + clone.mergedValue = this.mergedValue; + return clone; + } +} diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/MinKafkaMetricAccumulator.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/MinKafkaMetricAccumulator.java new file mode 100644 index 0000000000000..479489346844e --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/MinKafkaMetricAccumulator.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.streaming.connectors.kafka.internals.metrics; + +import org.apache.flink.api.common.accumulators.Accumulator; +import org.apache.kafka.common.metrics.KafkaMetric; + +public class MinKafkaMetricAccumulator extends DefaultKafkaMetricAccumulator { + + public MinKafkaMetricAccumulator(KafkaMetric kafkaMetric) { + super(kafkaMetric); + } + + @Override + public void merge(Accumulator other) { + if(!(other instanceof MinKafkaMetricAccumulator)) { + throw new RuntimeException("Trying to merge incompatible accumulators"); + } + MinKafkaMetricAccumulator otherMetric = (MinKafkaMetricAccumulator) other; + if(this.isMerged) { + if(otherMetric.isMerged) { + this.mergedValue = Math.min(this.mergedValue, otherMetric.mergedValue); + } else { + this.mergedValue = Math.min(this.mergedValue, otherMetric.getLocalValue()); + } + } else { + this.isMerged = true; + if(otherMetric.isMerged) { + this.mergedValue = Math.min(this.getLocalValue(), otherMetric.mergedValue); + } else { + this.mergedValue = Math.min(this.getLocalValue(), otherMetric.getLocalValue()); + } + } + } + + @Override + public Accumulator clone() { + MinKafkaMetricAccumulator clone = new MinKafkaMetricAccumulator(this.kafkaMetric); + clone.isMerged = this.isMerged; + clone.mergedValue = this.mergedValue; + return clone; + } +} diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FixedPartitioner.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FixedPartitioner.java similarity index 92% rename from flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FixedPartitioner.java rename to flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FixedPartitioner.java index 61735e98d9587..d9dcfc1ed6cb5 100644 --- a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FixedPartitioner.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FixedPartitioner.java @@ -51,7 +51,7 @@ * * */ -public class FixedPartitioner extends KafkaPartitioner implements Serializable { +public class FixedPartitioner extends KafkaPartitioner implements Serializable { private static final long serialVersionUID = 1627268846962918126L; int targetPartition = -1; @@ -71,7 +71,7 @@ public void open(int parallelInstanceId, int parallelInstances, int[] partitions } @Override - public int partition(Object element, int numPartitions) { + public int partition(T next, byte[] serializedKey, byte[] serializedValue, int numPartitions) { if (targetPartition == -1) { throw new RuntimeException("The partitioner has not been initialized properly"); } diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/KafkaPartitioner.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/KafkaPartitioner.java similarity index 87% rename from flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/KafkaPartitioner.java rename to flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/KafkaPartitioner.java index 55519f07d4558..038f414713637 100644 --- a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/KafkaPartitioner.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/KafkaPartitioner.java @@ -17,16 +17,13 @@ package org.apache.flink.streaming.connectors.kafka.partitioner; -import kafka.producer.Partitioner; - import java.io.Serializable; /** - * Extended Kafka Partitioner. * It contains a open() method which is called on each parallel instance. - * Partitioners have to be serializable! + * Partitioners must be serializable! */ -public abstract class KafkaPartitioner implements Partitioner, Serializable { +public abstract class KafkaPartitioner implements Serializable { private static final long serialVersionUID = -1974260817778593473L; @@ -39,4 +36,6 @@ public abstract class KafkaPartitioner implements Partitioner, Serializable { public void open(int parallelInstanceId, int parallelInstances, int[] partitions) { // overwrite this method if needed. } + + public abstract int partition(T next, byte[] serializedKey, byte[] serializedValue, int numPartitions); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchema.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchema.java similarity index 91% rename from flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchema.java rename to flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchema.java index 80bea8de1fe0f..01e72cae9032d 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchema.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchema.java @@ -36,10 +36,10 @@ public interface KeyedDeserializationSchema extends Serializable, ResultTypeQ * * @param messageKey the key as a byte array (null if no key has been set) * @param message The message, as a byte array. (null if the message was empty or deleted) - * @param offset the offset of the message in the original source (for example the Kafka offset) - * @return The deserialized message as an object. + * @param partition The partition the message has originated from + * @param offset the offset of the message in the original source (for example the Kafka offset) @return The deserialized message as an object. */ - T deserialize(byte[] messageKey, byte[] message, String topic, long offset) throws IOException; + T deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset) throws IOException; /** * Method to decide whether the element signals the end of the stream. If diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchemaWrapper.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchemaWrapper.java similarity index 97% rename from flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchemaWrapper.java rename to flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchemaWrapper.java index 8d9cf5dcee2dc..4b9dba2c4e031 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchemaWrapper.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchemaWrapper.java @@ -35,7 +35,7 @@ public KeyedDeserializationSchemaWrapper(DeserializationSchema deserializatio this.deserializationSchema = deserializationSchema; } @Override - public T deserialize(byte[] messageKey, byte[] message, String topic, long offset) throws IOException { + public T deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset) throws IOException { return deserializationSchema.deserialize(message); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchema.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchema.java similarity index 100% rename from flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchema.java rename to flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchema.java diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchemaWrapper.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchemaWrapper.java similarity index 100% rename from flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchemaWrapper.java rename to flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchemaWrapper.java diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationKeyValueSerializationSchema.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationKeyValueSerializationSchema.java similarity index 98% rename from flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationKeyValueSerializationSchema.java rename to flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationKeyValueSerializationSchema.java index 250012fa2e479..a35c01e600cd5 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationKeyValueSerializationSchema.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationKeyValueSerializationSchema.java @@ -79,7 +79,7 @@ public TypeInformationKeyValueSerializationSchema(Class keyClass, Class va @Override - public Tuple2 deserialize(byte[] messageKey, byte[] message, String topic, long offset) throws IOException { + public Tuple2 deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset) throws IOException { K key = null; if(messageKey != null) { key = keySerializer.deserialize(new ByteArrayInputView(messageKey)); diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerPartitionAssignmentTest.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerPartitionAssignmentTest.java similarity index 91% rename from flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerPartitionAssignmentTest.java rename to flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerPartitionAssignmentTest.java index c4b026b4747f0..e86d51a7944cb 100644 --- a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerPartitionAssignmentTest.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerPartitionAssignmentTest.java @@ -50,7 +50,7 @@ public void testPartitionsEqualConsumers() { inPartitions.add(new KafkaTopicPartitionLeader(new KafkaTopicPartition("test-topic", 1), fake)); for (int i = 0; i < inPartitions.size(); i++) { - List parts = FlinkKafkaConsumer.assignPartitions( + List parts = FlinkKafkaConsumerBase.assignPartitions( inPartitions, inPartitions.size(), i); assertNotNull(parts); @@ -92,7 +92,7 @@ public void testMultiplePartitionsPerConsumers() { final int maxPartitionsPerConsumer = partitions.size() / numConsumers + 1; for (int i = 0; i < numConsumers; i++) { - List parts = FlinkKafkaConsumer.assignPartitions(partitions, numConsumers, i); + List parts = FlinkKafkaConsumerBase.assignPartitions(partitions, numConsumers, i); assertNotNull(parts); assertTrue(parts.size() >= minPartitionsPerConsumer); @@ -128,7 +128,7 @@ public void testPartitionsFewerThanConsumers() { final int numConsumers = 2 * inPartitions.size() + 3; for (int i = 0; i < numConsumers; i++) { - List parts = FlinkKafkaConsumer.assignPartitions(inPartitions, numConsumers, i); + List parts = FlinkKafkaConsumerBase.assignPartitions(inPartitions, numConsumers, i); assertNotNull(parts); assertTrue(parts.size() <= 1); @@ -152,11 +152,11 @@ public void testPartitionsFewerThanConsumers() { public void testAssignEmptyPartitions() { try { List ep = new ArrayList<>(); - List parts1 = FlinkKafkaConsumer.assignPartitions(ep, 4, 2); + List parts1 = FlinkKafkaConsumerBase.assignPartitions(ep, 4, 2); assertNotNull(parts1); assertTrue(parts1.isEmpty()); - List parts2 = FlinkKafkaConsumer.assignPartitions(ep, 1, 0); + List parts2 = FlinkKafkaConsumerBase.assignPartitions(ep, 1, 0); assertNotNull(parts2); assertTrue(parts2.isEmpty()); } @@ -188,11 +188,11 @@ public void testGrowingPartitionsRemainsStable() { final int minNewPartitionsPerConsumer = newPartitions.size() / numConsumers; final int maxNewPartitionsPerConsumer = newPartitions.size() / numConsumers + 1; - List parts1 = FlinkKafkaConsumer.assignPartitions( + List parts1 = FlinkKafkaConsumerBase.assignPartitions( initialPartitions, numConsumers, 0); - List parts2 = FlinkKafkaConsumer.assignPartitions( + List parts2 = FlinkKafkaConsumerBase.assignPartitions( initialPartitions, numConsumers, 1); - List parts3 = FlinkKafkaConsumer.assignPartitions( + List parts3 = FlinkKafkaConsumerBase.assignPartitions( initialPartitions, numConsumers, 2); assertNotNull(parts1); @@ -224,11 +224,11 @@ public void testGrowingPartitionsRemainsStable() { // grow the set of partitions and distribute anew - List parts1new = FlinkKafkaConsumer.assignPartitions( + List parts1new = FlinkKafkaConsumerBase.assignPartitions( newPartitions, numConsumers, 0); - List parts2new = FlinkKafkaConsumer.assignPartitions( + List parts2new = FlinkKafkaConsumerBase.assignPartitions( newPartitions, numConsumers, 1); - List parts3new = FlinkKafkaConsumer.assignPartitions( + List parts3new = FlinkKafkaConsumerBase.assignPartitions( newPartitions, numConsumers, 2); // new partitions must include all old partitions diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java similarity index 76% rename from flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java rename to flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java index d75a15c0aced3..1bd01a20039e8 100644 --- a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java @@ -18,8 +18,6 @@ package org.apache.flink.streaming.connectors.kafka; -import kafka.admin.AdminUtils; -import kafka.api.PartitionMetadata; import kafka.consumer.Consumer; import kafka.consumer.ConsumerConfig; import kafka.consumer.ConsumerIterator; @@ -28,11 +26,10 @@ import kafka.message.MessageAndMetadata; import kafka.server.KafkaServer; -import org.I0Itec.zkclient.ZkClient; import org.apache.commons.collections.map.LinkedMap; -import org.apache.curator.framework.CuratorFramework; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.common.functions.RichMapFunction; @@ -41,15 +38,12 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.api.java.typeutils.TypeInfoParser; import org.apache.flink.api.java.typeutils.runtime.ByteArrayInputView; import org.apache.flink.client.program.ProgramInvocationException; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException; -import org.apache.flink.runtime.util.DataOutputSerializer; import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier; import org.apache.flink.streaming.api.checkpoint.Checkpointed; import org.apache.flink.streaming.api.datastream.DataStream; @@ -61,37 +55,32 @@ import org.apache.flink.streaming.api.functions.source.RichSourceFunction; import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; -import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionLeader; -import org.apache.flink.streaming.connectors.kafka.internals.ZooKeeperStringSerializer; -import org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler; import org.apache.flink.streaming.connectors.kafka.testutils.DataGenerators; import org.apache.flink.streaming.connectors.kafka.testutils.DiscardingSink; import org.apache.flink.streaming.connectors.kafka.testutils.FailingIdentityMapper; import org.apache.flink.streaming.connectors.kafka.testutils.JobManagerCommunicationUtils; import org.apache.flink.streaming.connectors.kafka.testutils.MockRuntimeContext; import org.apache.flink.streaming.connectors.kafka.testutils.PartitionValidatingMapper; -import org.apache.flink.streaming.connectors.kafka.testutils.SuccessException; import org.apache.flink.streaming.connectors.kafka.testutils.ThrottledMapper; import org.apache.flink.streaming.connectors.kafka.testutils.Tuple2Partitioner; import org.apache.flink.streaming.connectors.kafka.testutils.ValidatingExactlyOnceSink; import org.apache.flink.streaming.util.serialization.DeserializationSchema; +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper; import org.apache.flink.streaming.util.serialization.SimpleStringSchema; import org.apache.flink.streaming.util.serialization.TypeInformationKeyValueSerializationSchema; import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; -import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper; import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema; import org.apache.flink.streaming.util.serialization.TypeInformationSerializationSchema; +import org.apache.flink.test.util.SuccessException; import org.apache.flink.testutils.junit.RetryOnException; import org.apache.flink.testutils.junit.RetryRule; import org.apache.flink.util.Collector; -import org.apache.flink.util.NetUtils; import org.apache.flink.util.StringUtils; import org.apache.kafka.clients.producer.ProducerConfig; import org.junit.Assert; import org.junit.Rule; -import scala.collection.Seq; import java.io.IOException; import java.lang.reflect.Field; @@ -108,6 +97,7 @@ import java.util.UUID; import java.util.concurrent.atomic.AtomicReference; +import static org.apache.flink.test.util.TestUtils.tryExecute; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; @@ -122,27 +112,6 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase { @Rule public RetryRule retryRule = new RetryRule(); - // ------------------------------------------------------------------------ - // Required methods by the abstract test base - // ------------------------------------------------------------------------ - - protected abstract FlinkKafkaConsumer getConsumer( - List topics, KeyedDeserializationSchema deserializationSchema, Properties props); - - protected FlinkKafkaConsumer getConsumer( - List topics, DeserializationSchema deserializationSchema, Properties props) { - return getConsumer(topics, new KeyedDeserializationSchemaWrapper(deserializationSchema), props); - } - - protected FlinkKafkaConsumer getConsumer( - String topic, DeserializationSchema deserializationSchema, Properties props) { - return getConsumer(Collections.singletonList(topic), new KeyedDeserializationSchemaWrapper(deserializationSchema), props); - } - - protected FlinkKafkaConsumer getConsumer( - String topic, KeyedDeserializationSchema deserializationSchema, Properties props) { - return getConsumer(Collections.singletonList(topic), deserializationSchema, props); - } // ------------------------------------------------------------------------ // Suite of Tests @@ -172,12 +141,13 @@ public void runFailOnNoBrokerTest() throws Exception { properties.setProperty("bootstrap.servers", "localhost:80"); properties.setProperty("zookeeper.connect", "localhost:80"); properties.setProperty("group.id", "test"); - FlinkKafkaConsumer source = getConsumer("doesntexist", new SimpleStringSchema(), properties); + FlinkKafkaConsumerBase source = kafkaServer.getConsumer("doesntexist", new SimpleStringSchema(), properties); DataStream stream = see.addSource(source); stream.print(); see.execute("No broker test"); } catch(RuntimeException re){ Assert.assertTrue("Wrong RuntimeException thrown: " + StringUtils.stringifyException(re), + re.getClass().equals(RuntimeException.class) && re.getMessage().contains("Unable to retrieve any partitions for the requested topics [doesntexist]")); } } @@ -187,8 +157,8 @@ public void runFailOnNoBrokerTest() throws Exception { public void runCheckpointingTest() throws Exception { createTestTopic("testCheckpointing", 1, 1); - FlinkKafkaConsumer source = getConsumer("testCheckpointing", new SimpleStringSchema(), standardProps); - Field pendingCheckpointsField = FlinkKafkaConsumer.class.getDeclaredField("pendingCheckpoints"); + FlinkKafkaConsumerBase source = kafkaServer.getConsumer("testCheckpointing", new SimpleStringSchema(), standardProps); + Field pendingCheckpointsField = FlinkKafkaConsumerBase.class.getDeclaredField("pendingCheckpoints"); pendingCheckpointsField.setAccessible(true); LinkedMap pendingCheckpoints = (LinkedMap) pendingCheckpointsField.get(source); @@ -225,7 +195,7 @@ public void runCheckpointingTest() throws Exception { for (int i = 100; i < 600; i++) { source.snapshotState(i, 15 * i); } - Assert.assertEquals(FlinkKafkaConsumer.MAX_NUM_PENDING_CHECKPOINTS, pendingCheckpoints.size()); + Assert.assertEquals(FlinkKafkaConsumerBase.MAX_NUM_PENDING_CHECKPOINTS, pendingCheckpoints.size()); // commit only the second last source.notifyCheckpointComplete(598); @@ -243,116 +213,7 @@ public void runCheckpointingTest() throws Exception { deleteTestTopic("testCheckpointing"); } - /** - * Tests that offsets are properly committed to ZooKeeper and initial offsets are read from ZooKeeper. - * - * This test is only applicable if the Flink Kafka Consumer uses the ZooKeeperOffsetHandler. - */ - public void runOffsetInZookeeperValidationTest() throws Exception { - final String topicName = "testOffsetInZK"; - final int parallelism = 3; - - createTestTopic(topicName, parallelism, 1); - - StreamExecutionEnvironment env1 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); - env1.getConfig().disableSysoutLogging(); - env1.enableCheckpointing(50); - env1.setNumberOfExecutionRetries(0); - env1.setParallelism(parallelism); - - StreamExecutionEnvironment env2 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); - env2.getConfig().disableSysoutLogging(); - env2.enableCheckpointing(50); - env2.setNumberOfExecutionRetries(0); - env2.setParallelism(parallelism); - - StreamExecutionEnvironment env3 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); - env3.getConfig().disableSysoutLogging(); - env3.enableCheckpointing(50); - env3.setNumberOfExecutionRetries(0); - env3.setParallelism(parallelism); - - // write a sequence from 0 to 99 to each of the 3 partitions. - writeSequence(env1, topicName, 100, parallelism); - - readSequence(env2, standardProps, parallelism, topicName, 100, 0); - - CuratorFramework zkClient = createZookeeperClient(); - - long o1 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(zkClient, standardCC.groupId(), topicName, 0); - long o2 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(zkClient, standardCC.groupId(), topicName, 1); - long o3 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(zkClient, standardCC.groupId(), topicName, 2); - - LOG.info("Got final offsets from zookeeper o1={}, o2={}, o3={}", o1, o2, o3); - - assertTrue(o1 == FlinkKafkaConsumer.OFFSET_NOT_SET || (o1 >= 0 && o1 <= 100)); - assertTrue(o2 == FlinkKafkaConsumer.OFFSET_NOT_SET || (o2 >= 0 && o2 <= 100)); - assertTrue(o3 == FlinkKafkaConsumer.OFFSET_NOT_SET || (o3 >= 0 && o3 <= 100)); - - LOG.info("Manipulating offsets"); - - // set the offset to 50 for the three partitions - ZookeeperOffsetHandler.setOffsetInZooKeeper(zkClient, standardCC.groupId(), topicName, 0, 49); - ZookeeperOffsetHandler.setOffsetInZooKeeper(zkClient, standardCC.groupId(), topicName, 1, 49); - ZookeeperOffsetHandler.setOffsetInZooKeeper(zkClient, standardCC.groupId(), topicName, 2, 49); - - zkClient.close(); - - // create new env - readSequence(env3, standardProps, parallelism, topicName, 50, 50); - - deleteTestTopic(topicName); - } - - public void runOffsetAutocommitTest() throws Exception { - final String topicName = "testOffsetAutocommit"; - final int parallelism = 3; - createTestTopic(topicName, parallelism, 1); - - StreamExecutionEnvironment env1 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); - env1.getConfig().disableSysoutLogging(); - env1.setNumberOfExecutionRetries(0); - env1.setParallelism(parallelism); - - StreamExecutionEnvironment env2 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); - // NOTE: We are not enabling the checkpointing! - env2.getConfig().disableSysoutLogging(); - env2.setNumberOfExecutionRetries(0); - env2.setParallelism(parallelism); - - - // write a sequence from 0 to 99 to each of the 3 partitions. - writeSequence(env1, topicName, 100, parallelism); - - - // the readSequence operation sleeps for 20 ms between each record. - // setting a delay of 25*20 = 500 for the commit interval makes - // sure that we commit roughly 3-4 times while reading, however - // at least once. - Properties readProps = new Properties(); - readProps.putAll(standardProps); - readProps.setProperty("auto.commit.interval.ms", "500"); - - // read so that the offset can be committed to ZK - readSequence(env2, readProps, parallelism, topicName, 100, 0); - - // get the offset - CuratorFramework zkClient = createZookeeperClient(); - - long o1 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(zkClient, standardCC.groupId(), topicName, 0); - long o2 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(zkClient, standardCC.groupId(), topicName, 1); - long o3 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(zkClient, standardCC.groupId(), topicName, 2); - - LOG.info("Got final offsets from zookeeper o1={}, o2={}, o3={}", o1, o2, o3); - - // ensure that the offset has been committed - assertTrue("Offset of o1=" + o1 + " was not in range", o1 > 0 && o1 <= 100); - assertTrue("Offset of o2=" + o2 + " was not in range", o2 > 0 && o2 <= 100); - assertTrue("Offset of o3=" + o3 + " was not in range", o3 > 0 && o3 <= 100); - - deleteTestTopic(topicName); - } /** * Ensure Kafka is working on both producer and consumer side. @@ -429,14 +290,14 @@ public void cancel() { running = false; } }); - stream.addSink(new FlinkKafkaProducer<>(brokerConnectionStrings, topic, sinkSchema)); + stream.addSink(kafkaServer.getProducer(topic, new KeyedSerializationSchemaWrapper>(sinkSchema), FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings), null)); // ----------- add consumer dataflow ---------- List topics = new ArrayList<>(); topics.add(topic); topics.add(additionalEmptyTopic); - FlinkKafkaConsumer> source = getConsumer(topics, sourceSchema, standardProps); + FlinkKafkaConsumerBase> source = kafkaServer.getConsumer(topics, sourceSchema, standardProps); DataStreamSource> consuming = env.addSource(source).setParallelism(parallelism); @@ -494,6 +355,7 @@ public void close() throws Exception { deleteTestTopic(topic); } + /** * Tests the proper consumption when having a 1:1 correspondence between kafka partitions and * Flink sources. @@ -510,7 +372,7 @@ public void runOneToOneExactlyOnceTest() throws Exception { DataGenerators.generateRandomizedIntegerSequence( StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort), - brokerConnectionStrings, + kafkaServer, topic, parallelism, numElementsPerPartition, true); // run the topology that fails and recovers @@ -524,7 +386,7 @@ public void runOneToOneExactlyOnceTest() throws Exception { env.setNumberOfExecutionRetries(3); env.getConfig().disableSysoutLogging(); - FlinkKafkaConsumer kafkaSource = getConsumer(topic, schema, standardProps); + FlinkKafkaConsumerBase kafkaSource = kafkaServer.getConsumer(topic, schema, standardProps); env .addSource(kafkaSource) @@ -555,7 +417,7 @@ public void runOneSourceMultiplePartitionsExactlyOnceTest() throws Exception { DataGenerators.generateRandomizedIntegerSequence( StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort), - brokerConnectionStrings, + kafkaServer, topic, numPartitions, numElementsPerPartition, true); // run the topology that fails and recovers @@ -569,7 +431,7 @@ public void runOneSourceMultiplePartitionsExactlyOnceTest() throws Exception { env.setNumberOfExecutionRetries(3); env.getConfig().disableSysoutLogging(); - FlinkKafkaConsumer kafkaSource = getConsumer(topic, schema, standardProps); + FlinkKafkaConsumerBase kafkaSource = kafkaServer.getConsumer(topic, schema, standardProps); env .addSource(kafkaSource) @@ -600,7 +462,7 @@ public void runMultipleSourcesOnePartitionExactlyOnceTest() throws Exception { DataGenerators.generateRandomizedIntegerSequence( StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort), - brokerConnectionStrings, + kafkaServer, topic, numPartitions, numElementsPerPartition, true); // run the topology that fails and recovers @@ -615,7 +477,7 @@ public void runMultipleSourcesOnePartitionExactlyOnceTest() throws Exception { env.getConfig().disableSysoutLogging(); env.setBufferTimeout(0); - FlinkKafkaConsumer kafkaSource = getConsumer(topic, schema, standardProps); + FlinkKafkaConsumerBase kafkaSource = kafkaServer.getConsumer(topic, schema, standardProps); env .addSource(kafkaSource) @@ -642,7 +504,7 @@ public void runCancelingOnFullInputTest() throws Exception { // launch a producer thread DataGenerators.InfiniteStringsGenerator generator = - new DataGenerators.InfiniteStringsGenerator(brokerConnectionStrings, topic); + new DataGenerators.InfiniteStringsGenerator(kafkaServer, topic); generator.start(); // launch a consumer asynchronously @@ -658,7 +520,7 @@ public void run() { env.enableCheckpointing(100); env.getConfig().disableSysoutLogging(); - FlinkKafkaConsumer source = getConsumer(topic, new SimpleStringSchema(), standardProps); + FlinkKafkaConsumerBase source = kafkaServer.getConsumer(topic, new SimpleStringSchema(), standardProps); env.addSource(source).addSink(new DiscardingSink()); @@ -674,7 +536,13 @@ public void run() { runnerThread.start(); // wait a bit before canceling - Thread.sleep(2000); + Thread.sleep(8000); + + Throwable failueCause = jobError.get(); + if(failueCause != null) { + failueCause.printStackTrace(); + Assert.fail("Test failed prematurely with: " + failueCause.getMessage()); + } // cancel JobManagerCommunicationUtils.cancelCurrentJob(flink.getLeaderGateway(timeout)); @@ -682,7 +550,7 @@ public void run() { // wait for the program to be done and validate that we failed with the right exception runnerThread.join(); - Throwable failueCause = jobError.get(); + failueCause = jobError.get(); assertNotNull("program did not fail properly due to canceling", failueCause); assertTrue(failueCause.getMessage().contains("Job was cancelled")); @@ -723,13 +591,14 @@ public void run() { env.enableCheckpointing(100); env.getConfig().disableSysoutLogging(); - FlinkKafkaConsumer source = getConsumer(topic, new SimpleStringSchema(), standardProps); + FlinkKafkaConsumerBase source = kafkaServer.getConsumer(topic, new SimpleStringSchema(), standardProps); env.addSource(source).addSink(new DiscardingSink()); env.execute(); } catch (Throwable t) { + LOG.error("Job Runner failed with exception", t); error.set(t); } } @@ -739,15 +608,20 @@ public void run() { runnerThread.start(); // wait a bit before canceling - Thread.sleep(2000); + Thread.sleep(8000); + Throwable failueCause = error.get(); + if(failueCause != null) { + failueCause.printStackTrace(); + Assert.fail("Test failed prematurely with: " + failueCause.getMessage()); + } // cancel JobManagerCommunicationUtils.cancelCurrentJob(flink.getLeaderGateway(timeout)); // wait for the program to be done and validate that we failed with the right exception runnerThread.join(); - Throwable failueCause = error.get(); + failueCause = error.get(); assertNotNull("program did not fail properly due to canceling", failueCause); assertTrue(failueCause.getMessage().contains("Job was cancelled")); @@ -769,7 +643,7 @@ public void runFailOnDeployTest() throws Exception { env.setParallelism(12); // needs to be more that the mini cluster has slots env.getConfig().disableSysoutLogging(); - FlinkKafkaConsumer kafkaSource = getConsumer(topic, schema, standardProps); + FlinkKafkaConsumerBase kafkaSource = kafkaServer.getConsumer(topic, schema, standardProps); env .addSource(kafkaSource) @@ -800,30 +674,6 @@ public void runFailOnDeployTest() throws Exception { deleteTestTopic(topic); } - public void runInvalidOffsetTest() throws Exception { - final String topic = "invalidOffsetTopic"; - final int parallelism = 1; - - // create topic - createTestTopic(topic, parallelism, 1); - - final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); - - // write 20 messages into topic: - writeSequence(env, topic, 20, parallelism); - - // set invalid offset: - CuratorFramework zkClient = createZookeeperClient(); - ZookeeperOffsetHandler.setOffsetInZooKeeper(zkClient, standardCC.groupId(), topic, 0, 1234); - - // read from topic - final int valuesCount = 20; - final int startFrom = 0; - readSequence(env, standardCC.props().props(), parallelism, topic, valuesCount, startFrom); - - deleteTestTopic(topic); - } - public void runConsumeMultipleTopics() throws java.lang.Exception { final int NUM_TOPICS = 5; final int NUM_ELEMENTS = 20; @@ -842,12 +692,8 @@ public void runConsumeMultipleTopics() throws java.lang.Exception { writeSequence(env, topic, NUM_ELEMENTS, i + 1); } - // validate getPartitionsForTopic method - List topicPartitions = FlinkKafkaConsumer082.getPartitionsForTopic(topics, standardProps); - Assert.assertEquals((NUM_TOPICS * (NUM_TOPICS + 1))/2, topicPartitions.size()); - KeyedDeserializationSchema> readSchema = new Tuple2WithTopicDeserializationSchema(env.getConfig()); - DataStreamSource> stream = env.addSource(getConsumer(topics, readSchema, standardProps)); + DataStreamSource> stream = env.addSource(kafkaServer.getConsumer(topics, readSchema, standardProps)); stream.flatMap(new FlatMapFunction, Integer>() { Map countPerTopic = new HashMap<>(NUM_TOPICS); @@ -894,7 +740,7 @@ public Tuple2WithTopicDeserializationSchema(ExecutionConfig ec) { } @Override - public Tuple3 deserialize(byte[] messageKey, byte[] message, String topic, long offset) throws IOException { + public Tuple3 deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset) throws IOException { Tuple2 t2 = (Tuple2) ts.deserialize(new ByteArrayInputView(message)); return new Tuple3<>(t2.f0, t2.f1, topic); } @@ -913,6 +759,7 @@ public TypeInformation> getProducedType() { /** * 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 { @@ -942,7 +789,7 @@ public void runBigRecordTestTopology() throws Exception { consumerProps.setProperty("max.partition.fetch.bytes", Integer.toString(1024 * 1024 * 40)); // for the new fetcher consumerProps.setProperty("queued.max.message.chunks", "1"); - FlinkKafkaConsumer> source = getConsumer(topic, serSchema, consumerProps); + FlinkKafkaConsumerBase> source = kafkaServer.getConsumer(topic, serSchema, consumerProps); DataStreamSource> consuming = env.addSource(source); consuming.addSink(new SinkFunction>() { @@ -1007,12 +854,10 @@ public void cancel() { } }); - stream.addSink(new FlinkKafkaProducer<>(topic, deserSchema, producerProps)); + stream.addSink(kafkaServer.getProducer(topic, new KeyedSerializationSchemaWrapper<>(serSchema), producerProps, null)); tryExecute(env, "big topology test"); - deleteTestTopic(topic); - } @@ -1029,36 +874,13 @@ public void runBrokerFailureTest() throws Exception { DataGenerators.generateRandomizedIntegerSequence( StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort), - brokerConnectionStrings, + kafkaServer, topic, parallelism, numElementsPerPartition, true); // find leader to shut down - PartitionMetadata firstPart = null; - { - ZkClient zkClient = new ZkClient(standardCC.zkConnect(), standardCC.zkSessionTimeoutMs(), - standardCC.zkConnectionTimeoutMs(), new ZooKeeperStringSerializer()); - - do { - if (firstPart != null) { - LOG.info("Unable to find leader. error code {}", firstPart.errorCode()); - // not the first try. Sleep a bit - Thread.sleep(150); - } + int leaderId = kafkaServer.getLeaderToShutDown(topic); - Seq partitionMetadata = AdminUtils.fetchTopicMetadataFromZk(topic, zkClient).partitionsMetadata(); - firstPart = partitionMetadata.head(); - } - while (firstPart.errorCode() != 0); - zkClient.close(); - } - - final kafka.cluster.Broker leaderToShutDown = firstPart.leader().get(); - final String leaderToShutDownConnection = - NetUtils.hostAndPortToUrlString(leaderToShutDown.host(), leaderToShutDown.port()); - - - final int leaderIdToShutDown = firstPart.leader().get().id(); - LOG.info("Leader to shutdown {}", leaderToShutDown); + LOG.info("Leader to shutdown {}", leaderId); // run the topology that fails and recovers @@ -1073,20 +895,19 @@ public void runBrokerFailureTest() throws Exception { env.getConfig().disableSysoutLogging(); - FlinkKafkaConsumer kafkaSource = getConsumer(topic, schema, standardProps); + FlinkKafkaConsumerBase kafkaSource = kafkaServer.getConsumer(topic, schema, standardProps); env .addSource(kafkaSource) .map(new PartitionValidatingMapper(parallelism, 1)) - .map(new BrokerKillingMapper(leaderToShutDownConnection, failAfterElements)) + .map(new BrokerKillingMapper(leaderId, failAfterElements)) .addSink(new ValidatingExactlyOnceSink(totalElements)).setParallelism(1); BrokerKillingMapper.killedLeaderBefore = false; tryExecute(env, "One-to-one exactly once test"); // start a new broker: - brokers.set(leaderIdToShutDown, getKafkaServer(leaderIdToShutDown, tmpKafkaDirs.get(leaderIdToShutDown), kafkaHost, zookeeperConnectionString)); - + kafkaServer.restartBroker(leaderId); } public void runKeyValueTest() throws Exception { @@ -1098,7 +919,7 @@ public void runKeyValueTest() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); env.setParallelism(1); - env.setNumberOfExecutionRetries(3); + env.setNumberOfExecutionRetries(0); env.getConfig().disableSysoutLogging(); DataStream> kvStream = env.addSource(new SourceFunction>() { @@ -1121,21 +942,20 @@ public void cancel() { }); KeyedSerializationSchema> schema = new TypeInformationKeyValueSerializationSchema<>(Long.class, PojoValue.class, env.getConfig()); - kvStream.addSink(new FlinkKafkaProducer<>(topic, schema, - FlinkKafkaProducer.getPropertiesFromBrokerList(brokerConnectionStrings))); + kvStream.addSink(kafkaServer.getProducer(topic, schema, FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings), null)); env.execute("Write KV to Kafka"); // ----------- Read the data again ------------------- env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); env.setParallelism(1); - env.setNumberOfExecutionRetries(3); + env.setNumberOfExecutionRetries(0); env.getConfig().disableSysoutLogging(); KeyedDeserializationSchema> readSchema = new TypeInformationKeyValueSerializationSchema<>(Long.class, PojoValue.class, env.getConfig()); - DataStream> fromKafka = env.addSource(getConsumer(topic, readSchema, standardProps)); + DataStream> fromKafka = env.addSource(kafkaServer.getConsumer(topic, readSchema, standardProps)); fromKafka.flatMap(new RichFlatMapFunction, Object>() { long counter = 0; @Override @@ -1143,7 +963,7 @@ public void flatMap(Tuple2 value, Collector out) throws // the elements should be in order. Assert.assertTrue("Wrong value " + value.f1.lat, value.f1.lat == counter ); if (value.f1.lat % 2 == 0) { - Assert.assertNull("key was not null", value.f0); + assertNull("key was not null", value.f0); } else { Assert.assertTrue("Wrong value " + value.f0, value.f0 == counter); } @@ -1167,6 +987,11 @@ public static class PojoValue { public PojoValue() {} } + + /** + * Test delete behavior and metrics for producer + * @throws Exception + */ public void runAllDeletesTest() throws Exception { final String topic = "alldeletestest"; createTestTopic(topic, 1, 1); @@ -1176,7 +1001,7 @@ public void runAllDeletesTest() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); env.setParallelism(1); - env.setNumberOfExecutionRetries(3); + env.setNumberOfExecutionRetries(0); env.getConfig().disableSysoutLogging(); DataStream> kvStream = env.addSource(new SourceFunction>() { @@ -1196,18 +1021,23 @@ public void cancel() { TypeInformationKeyValueSerializationSchema schema = new TypeInformationKeyValueSerializationSchema<>(byte[].class, PojoValue.class, env.getConfig()); - kvStream.addSink(new FlinkKafkaProducer<>(topic, schema, - FlinkKafkaProducer.getPropertiesFromBrokerList(brokerConnectionStrings))); - env.execute("Write deletes to Kafka"); + kvStream.addSink(kafkaServer.getProducer(topic, schema, FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings), null)); + + JobExecutionResult result = env.execute("Write deletes to Kafka"); + + Map accuResults = result.getAllAccumulatorResults(); + // there are 37 accumulator results in Kafka 0.9 + // and 34 in Kafka 0.8 + Assert.assertTrue("Not enough accumulators from Kafka Producer: " + accuResults.size(), accuResults.size() > 33); // ----------- Read the data again ------------------- env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); env.setParallelism(1); - env.setNumberOfExecutionRetries(3); + env.setNumberOfExecutionRetries(0); env.getConfig().disableSysoutLogging(); - DataStream> fromKafka = env.addSource(getConsumer(topic, schema, standardProps)); + DataStream> fromKafka = env.addSource(kafkaServer.getConsumer(topic, schema, standardProps)); fromKafka.flatMap(new RichFlatMapFunction, Object>() { long counter = 0; @@ -1228,14 +1058,86 @@ public void flatMap(Tuple2 value, Collector out) thro deleteTestTopic(topic); } + /** + * Test that ensures that DeserializationSchema.isEndOfStream() is properly evaluated + * and that the metrics for the consumer are properly reported. + * + * @throws Exception + */ + public void runMetricsAndEndOfStreamTest() throws Exception { + final String topic = "testEndOfStream"; + createTestTopic(topic, 1, 1); + final int ELEMENT_COUNT = 300; + + // write some data + StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); + env.setParallelism(1); + env.setNumberOfExecutionRetries(0); + env.getConfig().disableSysoutLogging(); + + writeSequence(env, topic, ELEMENT_COUNT, 1); + + // read using custom schema + final StreamExecutionEnvironment env1 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); + env1.setParallelism(1); + env1.setNumberOfExecutionRetries(0); + env1.getConfig().disableSysoutLogging(); + + DataStream> fromKafka = env.addSource(kafkaServer.getConsumer(topic, new FixedNumberDeserializationSchema(ELEMENT_COUNT), standardProps)); + fromKafka.flatMap(new FlatMapFunction, Void>() { + @Override + public void flatMap(Tuple2 value, Collector out) throws Exception { + // noop ;) + } + }); + + JobExecutionResult result = tryExecute(env, "Consume " + ELEMENT_COUNT + " elements from Kafka"); + + Map accuResults = result.getAllAccumulatorResults(); + // kafka 0.9 consumer: 39 results + if(kafkaServer.getVersion().equals("0.9")) { + Assert.assertTrue("Not enough accumulators from Kafka Consumer: " + accuResults.size(), accuResults.size() > 38); + } + + deleteTestTopic(topic); + } + + public static class FixedNumberDeserializationSchema implements DeserializationSchema> { + final int finalCount; + int count = 0; + TypeInformation> ti = TypeInfoParser.parse("Tuple2"); + TypeSerializer> ser = ti.createSerializer(new ExecutionConfig()); + + public FixedNumberDeserializationSchema(int finalCount) { + this.finalCount = finalCount; + } + + @Override + public Tuple2 deserialize(byte[] message) throws IOException { + return ser.deserialize(new ByteArrayInputView(message)); + } + + @Override + public boolean isEndOfStream(Tuple2 nextElement) { + return ++count >= finalCount; + } + + @Override + public TypeInformation> getProducedType() { + return ti; + } + } + + // ------------------------------------------------------------------------ // Reading writing test data sets // ------------------------------------------------------------------------ - private void readSequence(StreamExecutionEnvironment env, Properties cc, + protected void readSequence(StreamExecutionEnvironment env, Properties cc, final int sourceParallelism, final String topicName, final int valuesCount, final int startFrom) throws Exception { + env.getCheckpointConfig().setCheckpointTimeout(5000); // set timeout for checkpoints to 5 seconds final int finalCount = valuesCount * sourceParallelism; @@ -1245,7 +1147,7 @@ private void readSequence(StreamExecutionEnvironment env, Properties cc, new TypeInformationSerializationSchema<>(intIntTupleType, env.getConfig()); // create the consumer - FlinkKafkaConsumer> consumer = getConsumer(topicName, deser, cc); + FlinkKafkaConsumerBase> consumer = kafkaServer.getConsumer(topicName, deser, cc); DataStream> source = env .addSource(consumer).setParallelism(sourceParallelism) @@ -1261,6 +1163,7 @@ private void readSequence(StreamExecutionEnvironment env, Properties cc, public void flatMap(Tuple2 value, Collector out) throws Exception { values[value.f1 - startFrom]++; count++; + LOG.info("Received message {}, total {} messages", value, count); // verify if we've seen everything if (count == finalCount) { @@ -1283,7 +1186,7 @@ public void flatMap(Tuple2 value, Collector out) thro LOG.info("Successfully read sequence for verification"); } - private static void writeSequence(StreamExecutionEnvironment env, String topicName, final int numElements, int parallelism) throws Exception { + protected void writeSequence(StreamExecutionEnvironment env, String topicName, final int numElements, int parallelism) throws Exception { LOG.info("\n===================================\n== Writing sequence of "+numElements+" into "+topicName+" with p="+parallelism+"\n==================================="); TypeInformation> resultType = TypeInfoParser.parse("Tuple2"); @@ -1309,11 +1212,10 @@ public void cancel() { } }).setParallelism(parallelism); - stream.addSink(new FlinkKafkaProducer<>(topicName, - new TypeInformationSerializationSchema<>(resultType, env.getConfig()), - FlinkKafkaProducer.getPropertiesFromBrokerList(brokerConnectionStrings), - new Tuple2Partitioner(parallelism) - )).setParallelism(parallelism); + stream.addSink(kafkaServer.getProducer(topicName, + new KeyedSerializationSchemaWrapper<>(new TypeInformationSerializationSchema<>(resultType, env.getConfig())), + FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings), + new Tuple2Partitioner(parallelism))).setParallelism(parallelism); env.execute("Write sequence"); @@ -1395,7 +1297,7 @@ public static class BrokerKillingMapper extends RichMapFunction public static volatile boolean killedLeaderBefore; public static volatile boolean hasBeenCheckpointedBeforeFailure; - private final String leaderToShutDown; + private final int shutdownBrokerId; private final int failCount; private int numElementsTotal; @@ -1403,8 +1305,8 @@ public static class BrokerKillingMapper extends RichMapFunction private boolean hasBeenCheckpointed; - public BrokerKillingMapper(String leaderToShutDown, int failCount) { - this.leaderToShutDown = leaderToShutDown; + public BrokerKillingMapper(int shutdownBrokerId, int failCount) { + this.shutdownBrokerId = shutdownBrokerId; this.failCount = failCount; } @@ -1423,28 +1325,22 @@ public T map(T value) throws Exception { if (failer && numElementsTotal >= failCount) { // shut down a Kafka broker KafkaServer toShutDown = null; - for (KafkaServer kafkaServer : brokers) { - String connectionUrl = - NetUtils.hostAndPortToUrlString( - kafkaServer.config().advertisedHostName(), - kafkaServer.config().advertisedPort()); - if (leaderToShutDown.equals(connectionUrl)) { - toShutDown = kafkaServer; + for (KafkaServer server : kafkaServer.getBrokers()) { + + if (kafkaServer.getBrokerId(server) == shutdownBrokerId) { + toShutDown = server; break; } } if (toShutDown == null) { StringBuilder listOfBrokers = new StringBuilder(); - for (KafkaServer kafkaServer : brokers) { - listOfBrokers.append( - NetUtils.hostAndPortToUrlString( - kafkaServer.config().advertisedHostName(), - kafkaServer.config().advertisedPort())); + for (KafkaServer server : kafkaServer.getBrokers()) { + listOfBrokers.append(kafkaServer.getBrokerId(server)); listOfBrokers.append(" ; "); } - throw new Exception("Cannot find broker to shut down: " + leaderToShutDown + throw new Exception("Cannot find broker to shut down: " + shutdownBrokerId + " ; available brokers: " + listOfBrokers.toString()); } else { diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerITCase.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java similarity index 87% rename from flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerITCase.java rename to flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java index f4c1899ba0dd8..228f3ac01788e 100644 --- a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerITCase.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java @@ -27,19 +27,19 @@ import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; -import org.apache.flink.streaming.connectors.kafka.testutils.SuccessException; +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper; import org.apache.flink.streaming.util.serialization.TypeInformationSerializationSchema; +import org.apache.flink.test.util.SuccessException; -import org.junit.Test; import java.io.Serializable; -import java.util.Collections; +import static org.apache.flink.test.util.TestUtils.tryExecute; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; @SuppressWarnings("serial") -public class KafkaProducerITCase extends KafkaTestBase { +public abstract class KafkaProducerTestBase extends KafkaTestBase { /** @@ -58,8 +58,7 @@ public class KafkaProducerITCase extends KafkaTestBase { * * The final sink validates that there are no duplicates and that all partitions are present. */ - @Test - public void testCustomPartitioning() { + public void runCustomPartitioningTest() { try { LOG.info("Starting KafkaProducerITCase.testCustomPartitioning()"); @@ -104,15 +103,15 @@ public void cancel() { .setParallelism(1); // sink partitions into - stream.addSink(new FlinkKafkaProducer<>(topic, serSchema, FlinkKafkaProducer.getPropertiesFromBrokerList(brokerConnectionStrings), new CustomPartitioner(parallelism))) + stream.addSink(kafkaServer.getProducer(topic, + new KeyedSerializationSchemaWrapper<>(serSchema), + FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings), + new CustomPartitioner(parallelism))) .setParallelism(parallelism); // ------ consuming topology --------- - FlinkKafkaConsumer> source = - new FlinkKafkaConsumer<>(Collections.singletonList(topic), deserSchema, standardProps, - FlinkKafkaConsumer.OffsetStore.FLINK_ZOOKEEPER, - FlinkKafkaConsumer.FetcherType.LEGACY_LOW_LEVEL); + FlinkKafkaConsumerBase> source = kafkaServer.getConsumer(topic, deserSchema, standardProps); env.addSource(source).setParallelism(parallelism) @@ -164,8 +163,7 @@ public void invoke(Integer value) throws Exception { fail(e.getMessage()); } } - - + // ------------------------------------------------------------------------ public static class CustomPartitioner extends KafkaPartitioner implements Serializable { @@ -176,13 +174,13 @@ public CustomPartitioner(int expectedPartitions) { this.expectedPartitions = expectedPartitions; } + @Override - public int partition(Object key, int numPartitions) { - @SuppressWarnings("unchecked") - Tuple2 tuple = (Tuple2) key; - + public int partition(Object next, byte[] serializedKey, byte[] serializedValue, int numPartitions) { + Tuple2 tuple = (Tuple2) next; + assertEquals(expectedPartitions, numPartitions); - + return (int) (tuple.f0 % numPartitions); } } diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java new file mode 100644 index 0000000000000..73cd2f9e72cc8 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java @@ -0,0 +1,170 @@ +/* + * 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.consumer.ConsumerConfig; + + +import org.apache.flink.client.program.ProgramInvocationException; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.util.ForkableFlinkMiniCluster; +import org.apache.flink.test.util.SuccessException; +import org.apache.flink.util.InstantiationUtil; +import org.apache.flink.util.TestLogger; + + +import org.junit.AfterClass; +import org.junit.BeforeClass; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import scala.concurrent.duration.FiniteDuration; + +import java.io.IOException; +import java.util.Properties; +import java.util.concurrent.TimeUnit; + + +/** + * The base for the Kafka tests. It brings up: + *
      + *
    • A ZooKeeper mini cluster
    • + *
    • Three Kafka Brokers (mini clusters)
    • + *
    • A Flink mini cluster
    • + *
    + * + *

    Code in this test is based on the following GitHub repository: + * + * https://github.com/sakserv/hadoop-mini-clusters (ASL licensed), + * as per commit bc6b2b2d5f6424d5f377aa6c0871e82a956462ef

    + */ +@SuppressWarnings("serial") +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 String brokerConnectionStrings; + + protected static ConsumerConfig standardCC; + protected static Properties standardProps; + + protected static ForkableFlinkMiniCluster flink; + + protected static int flinkPort; + + protected static FiniteDuration timeout = new FiniteDuration(10, TimeUnit.SECONDS); + + protected static KafkaTestEnvironment kafkaServer; + + // ------------------------------------------------------------------------ + // Setup and teardown of the mini clusters + // ------------------------------------------------------------------------ + + @BeforeClass + public static void prepare() throws IOException, ClassNotFoundException { + LOG.info("-------------------------------------------------------------------------"); + LOG.info(" Starting KafkaITCase "); + LOG.info("-------------------------------------------------------------------------"); + + + + // dynamically load the implementation for the test + Class clazz = Class.forName("org.apache.flink.streaming.connectors.kafka.KafkaTestEnvironmentImpl"); + kafkaServer = (KafkaTestEnvironment) InstantiationUtil.instantiate(clazz); + + LOG.info("Starting KafkaITCase.prepare() for Kafka " + kafkaServer.getVersion()); + + kafkaServer.prepare(NUMBER_OF_KAFKA_SERVERS); + + standardProps = kafkaServer.getStandardProperties(); + standardCC = kafkaServer.getStandardConsumerConfig(); + brokerConnectionStrings = kafkaServer.getBrokerConnectionString(); + + // start also a re-usable Flink mini cluster + Configuration flinkConfig = new Configuration(); + flinkConfig.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1); + flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 8); + flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 16); + flinkConfig.setString(ConfigConstants.EXECUTION_RETRY_DELAY_KEY, "0 s"); + + flink = new ForkableFlinkMiniCluster(flinkConfig, false); + flink.start(); + + flinkPort = flink.getLeaderRPCPort(); + } + + @AfterClass + public static void shutDownServices() { + + LOG.info("-------------------------------------------------------------------------"); + LOG.info(" Shut down KafkaITCase "); + LOG.info("-------------------------------------------------------------------------"); + + flinkPort = -1; + if (flink != null) { + flink.shutdown(); + } + + kafkaServer.shutdown(); + + LOG.info("-------------------------------------------------------------------------"); + LOG.info(" KafkaITCase finished"); + LOG.info("-------------------------------------------------------------------------"); + } + + + + // ------------------------------------------------------------------------ + // Execution utilities + // ------------------------------------------------------------------------ + + + protected static void tryExecutePropagateExceptions(StreamExecutionEnvironment see, String name) throws Exception { + try { + see.execute(name); + } + catch (ProgramInvocationException | JobExecutionException root) { + Throwable cause = root.getCause(); + + // search for nested SuccessExceptions + int depth = 0; + while (!(cause instanceof SuccessException)) { + if (cause == null || depth++ == 20) { + throw root; + } + else { + cause = cause.getCause(); + } + } + } + } + + protected static void createTestTopic(String topic, int numberOfPartitions, int replicationFactor) { + kafkaServer.createTestTopic(topic, numberOfPartitions, replicationFactor); + } + + protected static void deleteTestTopic(String topic) { + kafkaServer.deleteTestTopic(topic); + } +} diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java new file mode 100644 index 0000000000000..40be8a1d2bbb1 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java @@ -0,0 +1,83 @@ +/* + * 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.consumer.ConsumerConfig; +import kafka.server.KafkaServer; +import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; +import org.apache.flink.streaming.util.serialization.DeserializationSchema; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper; +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema; + +import java.net.UnknownHostException; +import java.util.Collections; +import java.util.List; +import java.util.Properties; + +/** + * Abstract class providing a Kafka test environment + */ +public abstract class KafkaTestEnvironment { + + protected static final String KAFKA_HOST = "localhost"; + + public abstract void prepare(int numKafkaServers); + + public abstract void shutdown(); + + public abstract void deleteTestTopic(String topic); + + public abstract void createTestTopic(String topic, int numberOfPartitions, int replicationFactor); + + public abstract ConsumerConfig getStandardConsumerConfig(); + + public abstract Properties getStandardProperties(); + + public abstract String getBrokerConnectionString(); + + public abstract String getVersion(); + + public abstract List getBrokers(); + + // -- consumer / producer instances: + public FlinkKafkaConsumerBase getConsumer(List topics, DeserializationSchema deserializationSchema, Properties props) { + return getConsumer(topics, new KeyedDeserializationSchemaWrapper(deserializationSchema), props); + } + + public FlinkKafkaConsumerBase getConsumer(String topic, KeyedDeserializationSchema readSchema, Properties props) { + return getConsumer(Collections.singletonList(topic), readSchema, props); + } + + public FlinkKafkaConsumerBase getConsumer(String topic, DeserializationSchema deserializationSchema, Properties props) { + return getConsumer(Collections.singletonList(topic), deserializationSchema, props); + } + + public abstract FlinkKafkaConsumerBase getConsumer(List topics, KeyedDeserializationSchema readSchema, Properties props); + + public abstract FlinkKafkaProducerBase getProducer(String topic, KeyedSerializationSchema serSchema, Properties props, KafkaPartitioner partitioner); + + + // -- leader failure simulation + + public abstract void restartBroker(int leaderId) throws Exception; + + public abstract int getLeaderToShutDown(String topic) throws Exception; + + public abstract int getBrokerId(KafkaServer server); +} diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/TestFixedPartitioner.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/TestFixedPartitioner.java similarity index 63% rename from flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/TestFixedPartitioner.java rename to flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/TestFixedPartitioner.java index 75fdd461ae823..5dab05a921283 100644 --- a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/TestFixedPartitioner.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/TestFixedPartitioner.java @@ -35,22 +35,22 @@ public class TestFixedPartitioner { */ @Test public void testMoreFlinkThanBrokers() { - FixedPartitioner part = new FixedPartitioner(); + FixedPartitioner part = new FixedPartitioner<>(); int[] partitions = new int[]{0}; part.open(0, 4, partitions); - Assert.assertEquals(0, part.partition("abc1", partitions.length)); + Assert.assertEquals(0, part.partition("abc1", null, null, partitions.length)); part.open(1, 4, partitions); - Assert.assertEquals(0, part.partition("abc2", partitions.length)); + Assert.assertEquals(0, part.partition("abc2", null, null, partitions.length)); part.open(2, 4, partitions); - Assert.assertEquals(0, part.partition("abc3", partitions.length)); - Assert.assertEquals(0, part.partition("abc3", partitions.length)); // check if it is changing ;) + Assert.assertEquals(0, part.partition("abc3", null, null, partitions.length)); + Assert.assertEquals(0, part.partition("abc3", null, null, partitions.length)); // check if it is changing ;) part.open(3, 4, partitions); - Assert.assertEquals(0, part.partition("abc4", partitions.length)); + Assert.assertEquals(0, part.partition("abc4", null, null, partitions.length)); } /** @@ -67,16 +67,16 @@ public void testMoreFlinkThanBrokers() { */ @Test public void testFewerPartitions() { - FixedPartitioner part = new FixedPartitioner(); + FixedPartitioner part = new FixedPartitioner<>(); int[] partitions = new int[]{0, 1, 2, 3, 4}; part.open(0, 2, partitions); - Assert.assertEquals(0, part.partition("abc1", partitions.length)); - Assert.assertEquals(0, part.partition("abc1", partitions.length)); + Assert.assertEquals(0, part.partition("abc1", null, null, partitions.length)); + Assert.assertEquals(0, part.partition("abc1", null, null, partitions.length)); part.open(1, 2, partitions); - Assert.assertEquals(1, part.partition("abc1", partitions.length)); - Assert.assertEquals(1, part.partition("abc1", partitions.length)); + Assert.assertEquals(1, part.partition("abc1", null, null, partitions.length)); + Assert.assertEquals(1, part.partition("abc1", null, null, partitions.length)); } /* @@ -87,17 +87,17 @@ public void testFewerPartitions() { */ @Test public void testMixedCase() { - FixedPartitioner part = new FixedPartitioner(); + FixedPartitioner part = new FixedPartitioner<>(); int[] partitions = new int[]{0,1}; part.open(0, 3, partitions); - Assert.assertEquals(0, part.partition("abc1", partitions.length)); + Assert.assertEquals(0, part.partition("abc1", null, null, partitions.length)); part.open(1, 3, partitions); - Assert.assertEquals(1, part.partition("abc1", partitions.length)); + Assert.assertEquals(1, part.partition("abc1", null, null, partitions.length)); part.open(2, 3, partitions); - Assert.assertEquals(0, part.partition("abc1", partitions.length)); + Assert.assertEquals(0, part.partition("abc1", null, null, partitions.length)); } diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java similarity index 75% rename from flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java rename to flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java index 22c6cfb50e45b..cc237b6ebe7af 100644 --- a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java @@ -26,8 +26,11 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; -import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducerBase; +import org.apache.flink.streaming.connectors.kafka.KafkaTestEnvironment; +import org.apache.flink.streaming.connectors.kafka.partitioner.FixedPartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper; import org.apache.flink.streaming.util.serialization.SimpleStringSchema; import org.apache.flink.streaming.util.serialization.TypeInformationSerializationSchema; @@ -37,9 +40,9 @@ public class DataGenerators { public static void generateLongStringTupleSequence(StreamExecutionEnvironment env, - String brokerConnection, String topic, - int numPartitions, - final int from, final int to) throws Exception { + KafkaTestEnvironment testServer, String topic, + int numPartitions, + final int from, final int to) throws Exception { TypeInformation> resultType = TypeInfoParser.parse("Tuple2"); @@ -69,9 +72,9 @@ public void cancel() { } }); - stream.addSink(new FlinkKafkaProducer<>(topic, - new TypeInformationSerializationSchema<>(resultType, env.getConfig()), - FlinkKafkaProducer.getPropertiesFromBrokerList(brokerConnection), + stream.addSink(testServer.getProducer(topic, + new KeyedSerializationSchemaWrapper<>(new TypeInformationSerializationSchema<>(resultType, env.getConfig())), + FlinkKafkaProducerBase.getPropertiesFromBrokerList(testServer.getBrokerConnectionString()), new Tuple2Partitioner(numPartitions) )); @@ -81,10 +84,10 @@ public void cancel() { // ------------------------------------------------------------------------ public static void generateRandomizedIntegerSequence(StreamExecutionEnvironment env, - String brokerConnection, String topic, - final int numPartitions, - final int numElements, - final boolean randomizeOrder) throws Exception { + KafkaTestEnvironment testServer, String topic, + final int numPartitions, + final int numElements, + final boolean randomizeOrder) throws Exception { env.setParallelism(numPartitions); env.getConfig().disableSysoutLogging(); env.setNumberOfExecutionRetries(0); @@ -132,13 +135,13 @@ public void cancel() { stream .rebalance() - .addSink(new FlinkKafkaProducer<>(topic, - new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, env.getConfig()), - FlinkKafkaProducer.getPropertiesFromBrokerList(brokerConnection), - new KafkaPartitioner() { + .addSink(testServer.getProducer(topic, + new KeyedSerializationSchemaWrapper<>(new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, env.getConfig())), + FlinkKafkaProducerBase.getPropertiesFromBrokerList(testServer.getBrokerConnectionString()), + new KafkaPartitioner() { @Override - public int partition(Object key, int numPartitions) { - return ((Integer) key) % numPartitions; + public int partition(Integer next, byte[] serializedKey, byte[] serializedValue, int numPartitions) { + return next % numPartitions; } })); @@ -149,7 +152,7 @@ public int partition(Object key, int numPartitions) { public static class InfiniteStringsGenerator extends Thread { - private final String kafkaConnectionString; + private final KafkaTestEnvironment server; private final String topic; @@ -158,17 +161,19 @@ public static class InfiniteStringsGenerator extends Thread { private volatile boolean running = true; - public InfiniteStringsGenerator(String kafkaConnectionString, String topic) { - this.kafkaConnectionString = kafkaConnectionString; + public InfiniteStringsGenerator(KafkaTestEnvironment server, String topic) { + this.server = server; this.topic = topic; } @Override public void run() { // we manually feed data into the Kafka sink - FlinkKafkaProducer producer = null; + FlinkKafkaProducerBase producer = null; try { - producer = new FlinkKafkaProducer<>(kafkaConnectionString, topic, new SimpleStringSchema()); + producer = server.getProducer(topic, + new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()), + FlinkKafkaProducerBase.getPropertiesFromBrokerList(server.getBrokerConnectionString()), new FixedPartitioner()); producer.setRuntimeContext(new MockRuntimeContext(1,0)); producer.open(new Configuration()); diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DiscardingSink.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DiscardingSink.java similarity index 100% rename from flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DiscardingSink.java rename to flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DiscardingSink.java diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FailingIdentityMapper.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FailingIdentityMapper.java similarity index 100% rename from flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FailingIdentityMapper.java rename to flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FailingIdentityMapper.java diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/JobManagerCommunicationUtils.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/JobManagerCommunicationUtils.java similarity index 100% rename from flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/JobManagerCommunicationUtils.java rename to flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/JobManagerCommunicationUtils.java diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockRuntimeContext.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockRuntimeContext.java similarity index 99% rename from flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockRuntimeContext.java rename to flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockRuntimeContext.java index b8afe3a83a377..50c57abe1fa69 100644 --- a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockRuntimeContext.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockRuntimeContext.java @@ -97,7 +97,7 @@ public ClassLoader getUserCodeClassLoader() { @Override public void addAccumulator(String name, Accumulator accumulator) { - throw new UnsupportedOperationException(); + // noop } @Override diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/PartitionValidatingMapper.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/PartitionValidatingMapper.java similarity index 100% rename from flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/PartitionValidatingMapper.java rename to flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/PartitionValidatingMapper.java diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ThrottledMapper.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ThrottledMapper.java similarity index 100% rename from flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ThrottledMapper.java rename to flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ThrottledMapper.java diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/Tuple2Partitioner.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/Tuple2Partitioner.java similarity index 85% rename from flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/Tuple2Partitioner.java rename to flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/Tuple2Partitioner.java index b762e21815edd..0844412c1cbf2 100644 --- a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/Tuple2Partitioner.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/Tuple2Partitioner.java @@ -27,7 +27,7 @@ * Special partitioner that uses the first field of a 2-tuple as the partition, * and that expects a specific number of partitions. */ -public class Tuple2Partitioner extends KafkaPartitioner implements Serializable { +public class Tuple2Partitioner extends KafkaPartitioner> implements Serializable { private static final long serialVersionUID = 1L; @@ -39,13 +39,13 @@ public Tuple2Partitioner(int expectedPartitions) { } @Override - public int partition(Object key, int numPartitions) { + public int partition(Tuple2 next, byte[] serializedKey, byte[] serializedValue, int numPartitions) { if (numPartitions != expectedPartitions) { throw new IllegalArgumentException("Expected " + expectedPartitions + " partitions"); } @SuppressWarnings("unchecked") - Tuple2 element = (Tuple2) key; - + Tuple2 element = next; + return element.f0; } } \ No newline at end of file diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ValidatingExactlyOnceSink.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ValidatingExactlyOnceSink.java similarity index 89% rename from flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ValidatingExactlyOnceSink.java rename to flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ValidatingExactlyOnceSink.java index f3cc4fa792845..78135618520b3 100644 --- a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ValidatingExactlyOnceSink.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ValidatingExactlyOnceSink.java @@ -20,13 +20,14 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.streaming.api.checkpoint.Checkpointed; -import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; +import org.apache.flink.test.util.SuccessException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.BitSet; -public class ValidatingExactlyOnceSink implements SinkFunction, Checkpointed> { +public class ValidatingExactlyOnceSink extends RichSinkFunction implements Checkpointed> { private static final Logger LOG = LoggerFactory.getLogger(ValidatingExactlyOnceSink.class); @@ -49,7 +50,7 @@ public void invoke(Integer value) throws Exception { numElements++; if (duplicateChecker.get(value)) { - throw new Exception("Received a duplicate"); + throw new Exception("Received a duplicate: " + value); } duplicateChecker.set(value); if (numElements == numElementsTotal) { diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/resources/log4j-test.properties b/flink-streaming-connectors/flink-connector-kafka-base/src/test/resources/log4j-test.properties new file mode 100644 index 0000000000000..6bdfb48cd550f --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/resources/log4j-test.properties @@ -0,0 +1,29 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +log4j.rootLogger=INFO, testlogger + +log4j.appender.testlogger=org.apache.log4j.ConsoleAppender +log4j.appender.testlogger.target = System.err +log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout +log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n + +# suppress the irrelevant (wrong) warnings from the netty channel handler +log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger + + diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/resources/logback-test.xml b/flink-streaming-connectors/flink-connector-kafka-base/src/test/resources/logback-test.xml new file mode 100644 index 0000000000000..45b3b92f0cf65 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/resources/logback-test.xml @@ -0,0 +1,30 @@ + + + + + + %d{HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n + + + + + + + + \ No newline at end of file diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer082.java b/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer082.java deleted file mode 100644 index adc42de9f7524..0000000000000 --- a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer082.java +++ /dev/null @@ -1,85 +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 org.apache.flink.streaming.util.serialization.DeserializationSchema; -import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; - -import java.util.Collections; -import java.util.List; -import java.util.Properties; - -/** - * Creates a Kafka consumer compatible with reading from Kafka 0.8.2.x brokers. - * The consumer will internally use the old low-level Kafka API, and manually commit offsets - * partition offsets to ZooKeeper. - * - * Once Kafka released the new consumer with Kafka 0.8.3 Flink might use the 0.8.3 consumer API - * also against Kafka 0.8.2 installations. - * - * @param The type of elements produced by this consumer. - */ -public class FlinkKafkaConsumer082 extends FlinkKafkaConsumer { - - private static final long serialVersionUID = -8450689820627198228L; - - /** - * Creates a new Kafka 0.8.2.x streaming source consumer. - * - * @param topic - * The name of the topic that should be consumed. - * @param valueDeserializer - * The de-/serializer used to convert between Kafka's byte messages and Flink's objects. - * @param props - * The properties used to configure the Kafka consumer client, and the ZooKeeper client. - */ - public FlinkKafkaConsumer082(String topic, DeserializationSchema valueDeserializer, Properties props) { - super(Collections.singletonList(topic), valueDeserializer, props, OffsetStore.FLINK_ZOOKEEPER, FetcherType.LEGACY_LOW_LEVEL); - } - - - //----- key-value deserializer constructor - - /** - * Creates a new Kafka 0.8.2.x streaming source consumer. - * - * This constructor allows passing a {@see KeyedDeserializationSchema} for reading key/value - * pairs from Kafka. - * - * @param topic - * The name of the topic that should be consumed. - * @param deserializer - * The de-/serializer used to convert between Kafka's byte messages and Flink's objects. - * @param props - * The properties used to configure the Kafka consumer client, and the ZooKeeper client. - */ - public FlinkKafkaConsumer082(String topic, KeyedDeserializationSchema deserializer, Properties props) { - super(Collections.singletonList(topic), deserializer, props, OffsetStore.FLINK_ZOOKEEPER, FetcherType.LEGACY_LOW_LEVEL); - } - - //----- topic list constructors - - - public FlinkKafkaConsumer082(List topics, DeserializationSchema valueDeserializer, Properties props) { - super(topics, valueDeserializer, props, OffsetStore.FLINK_ZOOKEEPER, FetcherType.LEGACY_LOW_LEVEL); - } - - public FlinkKafkaConsumer082(List topics, KeyedDeserializationSchema deserializer, Properties props) { - super(topics, deserializer, props, OffsetStore.FLINK_ZOOKEEPER, FetcherType.LEGACY_LOW_LEVEL); - } -} diff --git a/flink-streaming-connectors/pom.xml b/flink-streaming-connectors/pom.xml index 1b829f29833b7..dead48156ba35 100644 --- a/flink-streaming-connectors/pom.xml +++ b/flink-streaming-connectors/pom.xml @@ -37,7 +37,9 @@ under the License. flink-connector-flume - flink-connector-kafka + flink-connector-kafka-base + flink-connector-kafka-0.8 + flink-connector-kafka-0.9 flink-connector-elasticsearch flink-connector-rabbitmq flink-connector-twitter diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/CheckpointConfig.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/CheckpointConfig.java index 0320d6b63f33f..409304a817f5a 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/CheckpointConfig.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/CheckpointConfig.java @@ -132,7 +132,7 @@ public long getCheckpointTimeout() { * @param checkpointTimeout The checkpoint timeout, in milliseconds. */ public void setCheckpointTimeout(long checkpointTimeout) { - if (checkpointInterval <= 0) { + if (checkpointTimeout <= 0) { throw new IllegalArgumentException("Checkpoint timeout must be larger than zero"); } this.checkpointTimeout = checkpointTimeout; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationSerializationSchema.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationSerializationSchema.java index ce9c9caa87bca..6577be8eefda4 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationSerializationSchema.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationSerializationSchema.java @@ -30,7 +30,6 @@ * A serialization and deserialization schema that uses Flink's serialization stack to * transform typed from and to byte arrays. * - * @see TypeInformationKeyValueSerializationSchema for a serialization schema supporting Key Value pairs. * * @param The type to be serialized. */ diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java index 3493f182ddbb7..304dcb5062d50 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java @@ -21,10 +21,8 @@ import org.apache.flink.api.common.functions.RichReduceFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple4; -import org.apache.flink.client.program.ProgramInvocationException; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.streaming.api.TimeCharacteristic; import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier; import org.apache.flink.streaming.api.checkpoint.Checkpointed; @@ -36,6 +34,7 @@ import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.streaming.api.windowing.windows.TimeWindow; import org.apache.flink.test.util.ForkableFlinkMiniCluster; +import org.apache.flink.test.util.SuccessException; import org.apache.flink.util.Collector; import org.apache.flink.util.TestLogger; import org.junit.AfterClass; @@ -45,6 +44,7 @@ import java.util.HashMap; import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static org.apache.flink.test.util.TestUtils.tryExecute; import static org.junit.Assert.*; /** @@ -577,27 +577,6 @@ public void restoreState(HashMap state) { // Utilities // ------------------------------------------------------------------------ - public static void tryExecute(StreamExecutionEnvironment env, String jobName) throws Exception { - try { - env.execute(jobName); - } - catch (ProgramInvocationException | JobExecutionException root) { - Throwable cause = root.getCause(); - - // search for nested SuccessExceptions - int depth = 0; - while (!(cause instanceof SuccessException)) { - if (cause == null || depth++ == 20) { - root.printStackTrace(); - fail("Test failed: " + root.getMessage()); - } - else { - cause = cause.getCause(); - } - } - } - } - public static class IntType { public int value; @@ -606,8 +585,4 @@ public IntType() {} public IntType(int value) { this.value = value; } } - - static final class SuccessException extends Exception { - private static final long serialVersionUID = -9218191172606739598L; - } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java index 3e0c0f3c9afff..81e8f0af5089f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java @@ -23,10 +23,8 @@ import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple4; -import org.apache.flink.client.program.ProgramInvocationException; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.streaming.api.TimeCharacteristic; import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier; import org.apache.flink.streaming.api.checkpoint.Checkpointed; @@ -38,6 +36,7 @@ import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.streaming.api.windowing.windows.TimeWindow; import org.apache.flink.test.util.ForkableFlinkMiniCluster; +import org.apache.flink.test.util.SuccessException; import org.apache.flink.util.Collector; import org.apache.flink.util.TestLogger; import org.junit.AfterClass; @@ -47,6 +46,7 @@ import java.util.HashMap; import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static org.apache.flink.test.util.TestUtils.tryExecute; import static org.junit.Assert.*; /** @@ -728,27 +728,6 @@ public void restoreState(HashMap state) { // Utilities // ------------------------------------------------------------------------ - public static void tryExecute(StreamExecutionEnvironment env, String jobName) throws Exception { - try { - env.execute(jobName); - } - catch (ProgramInvocationException | JobExecutionException root) { - Throwable cause = root.getCause(); - - // search for nested SuccessExceptions - int depth = 0; - while (!(cause instanceof SuccessException)) { - if (cause == null || depth++ == 20) { - root.printStackTrace(); - fail("Test failed: " + root.getMessage()); - } - else { - cause = cause.getCause(); - } - } - } - } - public static class IntType { public int value; @@ -757,8 +736,4 @@ public IntType() {} public IntType(int value) { this.value = value; } } - - static final class SuccessException extends Exception { - private static final long serialVersionUID = -9218191172606739598L; - } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java index e4ebfa071f4df..500d7d33c0c85 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java @@ -36,6 +36,7 @@ import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.streaming.api.windowing.windows.TimeWindow; import org.apache.flink.test.util.ForkableFlinkMiniCluster; +import org.apache.flink.test.util.SuccessException; import org.apache.flink.util.Collector; import org.apache.flink.util.TestLogger; import org.junit.AfterClass; @@ -50,6 +51,7 @@ import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static org.apache.flink.test.util.TestUtils.tryExecute; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -491,27 +493,6 @@ public static Collection timeCharacteristic(){ // Utilities // ------------------------------------------------------------------------ - public static void tryExecute(StreamExecutionEnvironment env, String jobName) throws Exception { - try { - env.execute(jobName); - } - catch (ProgramInvocationException | JobExecutionException root) { - Throwable cause = root.getCause(); - - // search for nested SuccessExceptions - int depth = 0; - while (!(cause instanceof SuccessException)) { - if (cause == null || depth++ == 20) { - root.printStackTrace(); - fail("Test failed: " + root.getMessage()); - } - else { - cause = cause.getCause(); - } - } - } - } - public static class IntType { public int value; @@ -520,8 +501,4 @@ public IntType() {} public IntType(int value) { this.value = value; } } - - static final class SuccessException extends Exception { - private static final long serialVersionUID = -9218191172606739598L; - } } diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/SuccessException.java b/flink-tests/src/test/java/org/apache/flink/test/util/SuccessException.java similarity index 94% rename from flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/SuccessException.java rename to flink-tests/src/test/java/org/apache/flink/test/util/SuccessException.java index 12e34608de7b5..22ac02b76535e 100644 --- a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/SuccessException.java +++ b/flink-tests/src/test/java/org/apache/flink/test/util/SuccessException.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.kafka.testutils; +package org.apache.flink.test.util; /** * Exception that is thrown to terminate a program and indicate success. diff --git a/flink-tests/src/test/java/org/apache/flink/test/util/TestUtils.java b/flink-tests/src/test/java/org/apache/flink/test/util/TestUtils.java new file mode 100644 index 0000000000000..86b5002a030de --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/util/TestUtils.java @@ -0,0 +1,52 @@ +/* + * 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.test.util; + +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.client.program.ProgramInvocationException; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; + +import static org.junit.Assert.fail; + +public class TestUtils { + public static JobExecutionResult tryExecute(StreamExecutionEnvironment see, String name) throws Exception { + try { + return see.execute(name); + } + catch (ProgramInvocationException | JobExecutionException root) { + Throwable cause = root.getCause(); + + // search for nested SuccessExceptions + int depth = 0; + while (!(cause instanceof SuccessException)) { + if (cause == null || depth++ == 20) { + root.printStackTrace(); + fail("Test failed: " + root.getMessage()); + } + else { + cause = cause.getCause(); + } + } + } + return null; + } + + +} diff --git a/pom.xml b/pom.xml index 8ae24dd8924d4..bad12da328c12 100644 --- a/pom.xml +++ b/pom.xml @@ -822,6 +822,7 @@ under the License. **/*.iml flink-quickstart/**/testArtifact/goal.txt + out/** **/target/** docs/_site/** **/scalastyle-output.xml