From 2fb145cfb44c77cc2ce404cf9bb03a6ed3396058 Mon Sep 17 00:00:00 2001 From: radekg Date: Tue, 12 Jul 2016 13:19:01 -0400 Subject: [PATCH 1/9] [FLINK-4035] Add support for Kafka 0.10.x. --- .../flink-connector-kafka-0.10/pom.xml | 179 ++++++++++ .../kafka/FlinkKafkaConsumer010.java | 259 ++++++++++++++ .../kafka/FlinkKafkaProducer010.java | 137 ++++++++ .../kafka/Kafka010JsonTableSource.java | 71 ++++ .../connectors/kafka/Kafka010TableSource.java | 75 ++++ .../kafka/internal/Kafka010Fetcher.java | 312 +++++++++++++++++ .../src/main/resources/log4j.properties | 29 ++ .../connectors/kafka/Kafka010ITCase.java | 192 ++++++++++ .../kafka/Kafka010ProducerITCase.java | 33 ++ .../connectors/kafka/KafkaProducerTest.java | 119 +++++++ .../kafka/KafkaShortRetention010ITCase.java | 34 ++ .../kafka/KafkaTestEnvironmentImpl.java | 331 ++++++++++++++++++ .../src/test/resources/log4j-test.properties | 30 ++ .../src/test/resources/logback-test.xml | 30 ++ flink-streaming-connectors/pom.xml | 1 + 15 files changed, 1832 insertions(+) create mode 100644 flink-streaming-connectors/flink-connector-kafka-0.10/pom.xml create mode 100644 flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java create mode 100644 flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java create mode 100644 flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java create mode 100644 flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java create mode 100644 flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010Fetcher.java create mode 100644 flink-streaming-connectors/flink-connector-kafka-0.10/src/main/resources/log4j.properties create mode 100644 flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java create mode 100644 flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ProducerITCase.java create mode 100644 flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java create mode 100644 flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetention010ITCase.java create mode 100644 flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java create mode 100644 flink-streaming-connectors/flink-connector-kafka-0.10/src/test/resources/log4j-test.properties create mode 100644 flink-streaming-connectors/flink-connector-kafka-0.10/src/test/resources/logback-test.xml diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/pom.xml b/flink-streaming-connectors/flink-connector-kafka-0.10/pom.xml new file mode 100644 index 0000000000000..f2bcb11c5480d --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-0.10/pom.xml @@ -0,0 +1,179 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-streaming-connectors + 1.1-SNAPSHOT + .. + + + flink-connector-kafka-0.10_2.10 + flink-connector-kafka-0.10 + + jar + + + + 0.10.0.0 + + + + + + + + org.apache.flink + flink-streaming-java_2.10 + ${project.version} + provided + + + + org.apache.flink + flink-connector-kafka-base_2.10 + ${project.version} + + + org.apache.kafka + kafka_${scala.binary.version} + + + + + + org.apache.flink + flink-table_2.10 + ${project.version} + provided + + true + + + + org.apache.kafka + kafka-clients + ${kafka.version} + + + + + + org.apache.flink + flink-connector-kafka-base_2.10 + ${project.version} + + + + org.apache.kafka + kafka_${scala.binary.version} + + + test-jar + test + + + + + org.apache.kafka + kafka_${scala.binary.version} + ${kafka.version} + test + + + + org.apache.flink + flink-tests_2.10 + ${project.version} + test-jar + test + + + + org.apache.flink + flink-test-utils_2.10 + ${project.version} + test + + + + org.apache.flink + flink-runtime_2.10 + ${project.version} + test-jar + test + + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + **/KafkaTestEnvironmentImpl* + + + + + + + org.apache.maven.plugins + maven-source-plugin + + + attach-test-sources + + test-jar-no-fork + + + + **/KafkaTestEnvironmentImpl* + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + 1 + -Xms256m -Xmx1000m -Dlog4j.configuration=${log4j.configuration} -Dmvn.forkNumber=${surefire.forkNumber} -XX:-UseGCOverheadLimit + + + + + + diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java new file mode 100644 index 0000000000000..78ccd4ada3b8a --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java @@ -0,0 +1,259 @@ +/* + * 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.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.connectors.kafka.internal.Kafka010Fetcher; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.streaming.util.serialization.DeserializationSchema; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper; +import org.apache.flink.util.SerializedValue; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Properties; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * The Flink Kafka Consumer is a streaming data source that pulls a parallel data stream from + * Apache Kafka 0.10.x. The consumer can run in multiple parallel instances, each of which will pull + * data from one or more Kafka partitions. + * + *

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 FlinkKafkaConsumer010 extends FlinkKafkaConsumerBase { + + private static final long serialVersionUID = 2324564345203409112L; + + private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaConsumer010.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; + + /** 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 */ + private final long pollTimeout; + + // ------------------------------------------------------------------------ + + /** + * Creates a new Kafka streaming source consumer for Kafka 0.10.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 FlinkKafkaConsumer010(String topic, DeserializationSchema valueDeserializer, Properties props) { + this(Collections.singletonList(topic), valueDeserializer, props); + } + + /** + * Creates a new Kafka streaming source consumer for Kafka 0.10.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 FlinkKafkaConsumer010(String topic, KeyedDeserializationSchema deserializer, Properties props) { + this(Collections.singletonList(topic), deserializer, props); + } + + /** + * Creates a new Kafka streaming source consumer for Kafka 0.10.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 FlinkKafkaConsumer010(List topics, DeserializationSchema deserializer, Properties props) { + this(topics, new KeyedDeserializationSchemaWrapper<>(deserializer), props); + } + + /** + * Creates a new Kafka streaming source consumer for Kafka 0.10.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 FlinkKafkaConsumer010(List topics, KeyedDeserializationSchema deserializer, Properties props) { + super(deserializer); + + checkNotNull(topics, "topics"); + this.properties = checkNotNull(props, "props"); + setDeserializer(this.properties); + + // configure the polling timeout + try { + if (properties.containsKey(KEY_POLL_TIMEOUT)) { + this.pollTimeout = Long.parseLong(properties.getProperty(KEY_POLL_TIMEOUT)); + } else { + this.pollTimeout = DEFAULT_POLL_TIMEOUT; + } + } + catch (Exception e) { + throw new IllegalArgumentException("Cannot parse poll timeout for '" + KEY_POLL_TIMEOUT + '\'', e); + } + + // read the partitions that belong to the listed topics + final List partitions = new ArrayList<>(); + + try (KafkaConsumer consumer = new KafkaConsumer<>(this.properties)) { + for (final String topic: topics) { + // get partitions for each topic + List partitionsForTopic = consumer.partitionsFor(topic); + // for non existing topics, the list might be null. + if (partitionsForTopic != null) { + partitions.addAll(convertToFlinkKafkaTopicPartition(partitionsForTopic)); + } + } + } + + if (partitions.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: {}", partitions.size(), topics); + + if (LOG.isInfoEnabled()) { + logPartitionInfo(LOG, partitions); + } + + // register these partitions + setSubscribedPartitions(partitions); + } + + @Override + protected AbstractFetcher createFetcher( + SourceContext sourceContext, + List thisSubtaskPartitions, + SerializedValue> watermarksPeriodic, + SerializedValue> watermarksPunctuated, + StreamingRuntimeContext runtimeContext) throws Exception { + + boolean useMetrics = !Boolean.valueOf(properties.getProperty(KEY_DISABLE_METRICS, "false")); + + return new Kafka010Fetcher<>(sourceContext, thisSubtaskPartitions, + watermarksPeriodic, watermarksPunctuated, + runtimeContext, deserializer, + properties, pollTimeout, useMetrics); + + } + + // ------------------------------------------------------------------------ + // Utilities + // ------------------------------------------------------------------------ + + /** + * 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 + */ + private static List convertToFlinkKafkaTopicPartition(List partitions) { + checkNotNull(partitions); + + List ret = new ArrayList<>(partitions.size()); + for (PartitionInfo pi : partitions) { + ret.add(new KafkaTopicPartition(pi.topic(), pi.partition())); + } + return ret; + } + + /** + * Makes sure that the ByteArrayDeserializer is registered in the Kafka properties. + * + * @param props The Kafka properties to register the serializer in. + */ + private static void setDeserializer(Properties props) { + final String deSerName = ByteArrayDeserializer.class.getCanonicalName(); + + Object keyDeSer = props.get(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG); + Object valDeSer = props.get(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG); + + if (keyDeSer != null && !keyDeSer.equals(deSerName)) { + LOG.warn("Ignoring configured key DeSerializer ({})", ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG); + } + if (valDeSer != null && !valDeSer.equals(deSerName)) { + LOG.warn("Ignoring configured value DeSerializer ({})", ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG); + } + + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, deSerName); + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, deSerName); + } +} diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java new file mode 100644 index 0000000000000..49bce398a88d3 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java @@ -0,0 +1,137 @@ +/* + * 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 FlinkKafkaProducer010 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 FlinkKafkaProducer010(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 FlinkKafkaProducer010(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 FlinkKafkaProducer010(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 FlinkKafkaProducer010(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 FlinkKafkaProducer010(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 FlinkKafkaProducer010(String topicId, KeyedSerializationSchema serializationSchema, Properties producerConfig, KafkaPartitioner customPartitioner) { + super(topicId, serializationSchema, producerConfig, customPartitioner); + } + + @Override + protected void flush() { + if (this.producer != null) { + producer.flush(); + } + } +} diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java new file mode 100644 index 0000000000000..cda68cee251a5 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java @@ -0,0 +1,71 @@ +/* + * 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.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.table.Row; +import org.apache.flink.api.table.sources.StreamTableSource; +import org.apache.flink.streaming.util.serialization.DeserializationSchema; + +import java.util.Properties; + +/** + * Kafka {@link StreamTableSource} for Kafka 0.10. + */ +public class Kafka010JsonTableSource extends KafkaJsonTableSource { + + /** + * Creates a Kafka 0.10 JSON {@link StreamTableSource}. + * + * @param topic Kafka topic to consume. + * @param properties Properties for the Kafka consumer. + * @param fieldNames Row field names. + * @param fieldTypes Row field types. + */ + public Kafka010JsonTableSource( + String topic, + Properties properties, + String[] fieldNames, + TypeInformation[] fieldTypes) { + + super(topic, properties, fieldNames, fieldTypes); + } + + /** + * Creates a Kafka 0.10 JSON {@link StreamTableSource}. + * + * @param topic Kafka topic to consume. + * @param properties Properties for the Kafka consumer. + * @param fieldNames Row field names. + * @param fieldTypes Row field types. + */ + public Kafka010JsonTableSource( + String topic, + Properties properties, + String[] fieldNames, + Class[] fieldTypes) { + + super(topic, properties, fieldNames, fieldTypes); + } + + @Override + FlinkKafkaConsumerBase getKafkaConsumer(String topic, Properties properties, DeserializationSchema deserializationSchema) { + return new FlinkKafkaConsumer010<>(topic, deserializationSchema, properties); + } +} diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java new file mode 100644 index 0000000000000..cee1b90716d59 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java @@ -0,0 +1,75 @@ +/* + * 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.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.table.Row; +import org.apache.flink.api.table.sources.StreamTableSource; +import org.apache.flink.streaming.util.serialization.DeserializationSchema; + +import java.util.Properties; + +/** + * Kafka {@link StreamTableSource} for Kafka 0.10. + */ +public class Kafka010TableSource extends KafkaTableSource { + + /** + * Creates a Kafka 0.10 {@link StreamTableSource}. + * + * @param topic Kafka topic to consume. + * @param properties Properties for the Kafka consumer. + * @param deserializationSchema Deserialization schema to use for Kafka records. + * @param fieldNames Row field names. + * @param fieldTypes Row field types. + */ + public Kafka010TableSource( + String topic, + Properties properties, + DeserializationSchema deserializationSchema, + String[] fieldNames, + TypeInformation[] fieldTypes) { + + super(topic, properties, deserializationSchema, fieldNames, fieldTypes); + } + + /** + * Creates a Kafka 0.10 {@link StreamTableSource}. + * + * @param topic Kafka topic to consume. + * @param properties Properties for the Kafka consumer. + * @param deserializationSchema Deserialization schema to use for Kafka records. + * @param fieldNames Row field names. + * @param fieldTypes Row field types. + */ + public Kafka010TableSource( + String topic, + Properties properties, + DeserializationSchema deserializationSchema, + String[] fieldNames, + Class[] fieldTypes) { + + super(topic, properties, deserializationSchema, fieldNames, fieldTypes); + } + + @Override + FlinkKafkaConsumerBase getKafkaConsumer(String topic, Properties properties, DeserializationSchema deserializationSchema) { + return new FlinkKafkaConsumer010<>(topic, deserializationSchema, properties); + } +} diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010Fetcher.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010Fetcher.java new file mode 100644 index 0000000000000..70f530b96b1cf --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010Fetcher.java @@ -0,0 +1,312 @@ +/* + * 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.internal; + +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.connectors.kafka.internals.ExceptionProxy; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionState; +import org.apache.flink.streaming.connectors.kafka.internals.metrics.DefaultKafkaMetricAccumulator; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +import org.apache.flink.util.SerializedValue; + +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.TopicPartition; +import org.apache.kafka.common.errors.WakeupException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +/** + * A fetcher that fetches data from Kafka brokers via the Kafka 0.10 consumer API. + * + * @param The type of elements produced by the fetcher. + */ +public class Kafka010Fetcher extends AbstractFetcher implements Runnable { + + private static final Logger LOG = LoggerFactory.getLogger(Kafka010Fetcher.class); + + // ------------------------------------------------------------------------ + + /** The schema to convert between Kafka's byte messages, and Flink's objects */ + private final KeyedDeserializationSchema deserializer; + + /** The subtask's runtime context */ + private final RuntimeContext runtimeContext; + + /** The configuration for the Kafka consumer */ + private final Properties kafkaProperties; + + /** The maximum number of milliseconds to wait for a fetch batch */ + private final long pollTimeout; + + /** Flag whether to register Kafka metrics as Flink accumulators */ + private final boolean forwardKafkaMetrics; + + /** Mutex to guard against concurrent access to the non-threadsafe Kafka consumer */ + private final Object consumerLock = new Object(); + + /** Reference to the Kafka consumer, once it is created */ + private volatile KafkaConsumer consumer; + + /** Reference to the proxy, forwarding exceptions from the fetch thread to the main thread */ + private volatile ExceptionProxy errorHandler; + + /** Flag to mark the main work loop as alive */ + private volatile boolean running = true; + + // ------------------------------------------------------------------------ + + public Kafka010Fetcher( + SourceContext sourceContext, + List assignedPartitions, + SerializedValue> watermarksPeriodic, + SerializedValue> watermarksPunctuated, + StreamingRuntimeContext runtimeContext, + KeyedDeserializationSchema deserializer, + Properties kafkaProperties, + long pollTimeout, + boolean forwardKafkaMetrics) throws Exception + { + super(sourceContext, assignedPartitions, watermarksPeriodic, watermarksPunctuated, runtimeContext); + + this.deserializer = deserializer; + this.runtimeContext = runtimeContext; + this.kafkaProperties = kafkaProperties; + this.pollTimeout = pollTimeout; + this.forwardKafkaMetrics = forwardKafkaMetrics; + + // if checkpointing is enabled, we are not automatically committing to Kafka. + kafkaProperties.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, + Boolean.toString(!runtimeContext.isCheckpointingEnabled())); + } + + // ------------------------------------------------------------------------ + // Fetcher work methods + // ------------------------------------------------------------------------ + + @Override + public void runFetchLoop() throws Exception { + this.errorHandler = new ExceptionProxy(Thread.currentThread()); + + // rather than running the main fetch loop directly here, we spawn a dedicated thread + // this makes sure that no interrupt() call upon canceling reaches the Kafka consumer code + Thread runner = new Thread(this, "Kafka 0.10 Fetcher for " + runtimeContext.getTaskNameWithSubtasks()); + runner.setDaemon(true); + runner.start(); + + try { + runner.join(); + } catch (InterruptedException e) { + // may be the result of a wake-up after an exception. we ignore this here and only + // restore the interruption state + Thread.currentThread().interrupt(); + } + + // make sure we propagate any exception that occurred in the concurrent fetch thread, + // before leaving this method + this.errorHandler.checkAndThrowException(); + } + + @Override + public void cancel() { + // flag the main thread to exit + running = false; + + // NOTE: + // - We cannot interrupt the runner thread, because the Kafka consumer may + // deadlock when the thread is interrupted while in certain methods + // - We cannot call close() on the consumer, because it will actually throw + // an exception if a concurrent call is in progress + + // make sure the consumer finds out faster that we are shutting down + if (consumer != null) { + consumer.wakeup(); + } + } + + @Override + public void run() { + // This method initializes the KafkaConsumer and guarantees it is torn down properly. + // This is important, because the consumer has multi-threading issues, + // including concurrent 'close()' calls. + + final KafkaConsumer consumer; + try { + consumer = new KafkaConsumer<>(kafkaProperties); + } + catch (Throwable t) { + running = false; + errorHandler.reportError(t); + return; + } + + // from here on, the consumer will be closed properly + try { + consumer.assign(convertKafkaPartitions(subscribedPartitions())); + + // register Kafka metrics to Flink accumulators + if (forwardKafkaMetrics) { + Map metrics = consumer.metrics(); + if (metrics == null) { + // MapR's Kafka implementation returns null here. + LOG.info("Consumer implementation does not support metrics"); + } else { + // we have metrics, register them where possible + for (Map.Entry metric : metrics.entrySet()) { + String name = "KafkaConsumer-" + metric.getKey().name(); + DefaultKafkaMetricAccumulator kafkaAccumulator = + DefaultKafkaMetricAccumulator.createFor(metric.getValue()); + + // best effort: we only add the accumulator if available. + if (kafkaAccumulator != null) { + runtimeContext.addAccumulator(name, kafkaAccumulator); + } + } + } + } + + // seek the consumer to the initial offsets + for (KafkaTopicPartitionState partition : subscribedPartitions()) { + if (partition.isOffsetDefined()) { + consumer.seek(partition.getKafkaPartitionHandle(), partition.getOffset() + 1); + } + } + + // from now on, external operations may call the consumer + this.consumer = consumer; + + // main fetch loop + while (running) { + // get the next batch of records + final ConsumerRecords records; + synchronized (consumerLock) { + try { + records = consumer.poll(pollTimeout); + } + catch (WakeupException we) { + if (running) { + throw we; + } else { + continue; + } + } + } + + // get the records for each topic partition + for (KafkaTopicPartitionState partition : subscribedPartitions()) { + + List> partitionRecords = records.records(partition.getKafkaPartitionHandle()); + + for (ConsumerRecord record : partitionRecords) { + T value = deserializer.deserialize( + record.key(), record.value(), + record.topic(), record.partition(), record.offset()); + + if (deserializer.isEndOfStream(value)) { + // end of stream signaled + running = false; + break; + } + + // emit the actual record. this also update offset state atomically + // and deals with timestamps and watermark generation + emitRecord(value, partition, record.offset()); + } + } + } + // end main fetch loop + } + catch (Throwable t) { + if (running) { + running = false; + errorHandler.reportError(t); + } else { + LOG.debug("Stopped ConsumerThread threw exception", t); + } + } + finally { + try { + synchronized (consumerLock) { + consumer.close(); + } + } catch (Throwable t) { + LOG.warn("Error while closing Kafka 0.10 consumer", t); + } + } + } + + // ------------------------------------------------------------------------ + // Kafka 0.10 specific fetcher behavior + // ------------------------------------------------------------------------ + + @Override + public TopicPartition createKafkaPartitionHandle(KafkaTopicPartition partition) { + return new TopicPartition(partition.getTopic(), partition.getPartition()); + } + + @Override + public void commitSpecificOffsetsToKafka(Map offsets) throws Exception { + KafkaTopicPartitionState[] partitions = subscribedPartitions(); + Map offsetsToCommit = new HashMap<>(partitions.length); + + for (KafkaTopicPartitionState partition : partitions) { + Long offset = offsets.get(partition.getKafkaTopicPartition()); + if (offset != null) { + offsetsToCommit.put(partition.getKafkaPartitionHandle(), new OffsetAndMetadata(offset, "")); + } + } + + if (this.consumer != null) { + synchronized (consumerLock) { + this.consumer.commitSync(offsetsToCommit); + } + } + } + + // ------------------------------------------------------------------------ + // Utilities + // ------------------------------------------------------------------------ + + public static Collection convertKafkaPartitions(KafkaTopicPartitionState[] partitions) { + ArrayList result = new ArrayList<>(partitions.length); + for (KafkaTopicPartitionState p : partitions) { + result.add(p.getKafkaPartitionHandle()); + } + return result; + } +} diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/resources/log4j.properties b/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/resources/log4j.properties new file mode 100644 index 0000000000000..6bdfb48cd550f --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-0.10/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-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java new file mode 100644 index 0000000000000..54278532abc3c --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java @@ -0,0 +1,192 @@ +/* + * 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.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.junit.Test; + +import java.util.UUID; + +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + + +public class Kafka010ITCase extends KafkaConsumerTestBase { + + // ------------------------------------------------------------------------ + // Suite of Tests + // ------------------------------------------------------------------------ + + @Override + public String getExpectedKafkaVersion() { + return "0.10"; + } + + @Test(timeout = 60000) + public void testFailOnNoBroker() throws Exception { + runFailOnNoBrokerTest(); + } + + @Test(timeout = 60000) + public void testConcurrentProducerConsumerTopology() throws Exception { + runSimpleConcurrentProducerConsumerTopology(); + } + +// @Test(timeout = 60000) +// public void testPunctuatedExplicitWMConsumer() throws Exception { +// runExplicitPunctuatedWMgeneratingConsumerTest(false); +// } + +// @Test(timeout = 60000) +// public void testPunctuatedExplicitWMConsumerWithEmptyTopic() throws Exception { +// runExplicitPunctuatedWMgeneratingConsumerTest(true); +// } + + @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(); + } + + + // --- 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 { + runProduceConsumeMultipleTopics(); + } + + @Test(timeout = 60000) + public void testAllDeletes() throws Exception { + runAllDeletesTest(); + } + + @Test(timeout = 60000) + public void testMetricsAndEndOfStream() throws Exception { + runMetricsAndEndOfStreamTest(); + } + + @Test + public void testJsonTableSource() throws Exception { + String topic = UUID.randomUUID().toString(); + + // Names and types are determined in the actual test method of the + // base test class. + Kafka010JsonTableSource tableSource = new Kafka010JsonTableSource( + topic, + standardProps, + new String[] { + "long", + "string", + "boolean", + "double", + "missing-field"}, + new TypeInformation[] { + BasicTypeInfo.LONG_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.BOOLEAN_TYPE_INFO, + BasicTypeInfo.DOUBLE_TYPE_INFO, + BasicTypeInfo.LONG_TYPE_INFO }); + + // Don't fail on missing field, but set to null (default) + tableSource.setFailOnMissingField(false); + + runJsonTableSource(topic, tableSource); + } + + @Test + public void testJsonTableSourceWithFailOnMissingField() throws Exception { + String topic = UUID.randomUUID().toString(); + + // Names and types are determined in the actual test method of the + // base test class. + Kafka010JsonTableSource tableSource = new Kafka010JsonTableSource( + topic, + standardProps, + new String[] { + "long", + "string", + "boolean", + "double", + "missing-field"}, + new TypeInformation[] { + BasicTypeInfo.LONG_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.BOOLEAN_TYPE_INFO, + BasicTypeInfo.DOUBLE_TYPE_INFO, + BasicTypeInfo.LONG_TYPE_INFO }); + + // Don't fail on missing field, but set to null (default) + tableSource.setFailOnMissingField(true); + + try { + runJsonTableSource(topic, tableSource); + fail("Did not throw expected Exception"); + } catch (Exception e) { + Throwable rootCause = e.getCause().getCause().getCause(); + assertTrue("Unexpected root cause", rootCause instanceof IllegalStateException); + } + } + +} diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ProducerITCase.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ProducerITCase.java new file mode 100644 index 0000000000000..42b96828403e2 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ProducerITCase.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 Kafka010ProducerITCase extends KafkaProducerTestBase { + + @Test + public void testCustomPartitioning() { + runCustomPartitioningTest(); + } + +} diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java new file mode 100644 index 0000000000000..5f5ac636790a3 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java @@ -0,0 +1,119 @@ +/* + * 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.Collections; +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( + Collections.singletonList(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 + + FlinkKafkaProducer010 producerPropagating = new FlinkKafkaProducer010<>( + "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 + + FlinkKafkaProducer010 producerLogging = new FlinkKafkaProducer010<>( + "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.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetention010ITCase.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetention010ITCase.java new file mode 100644 index 0000000000000..1d36198d554f8 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetention010ITCase.java @@ -0,0 +1,34 @@ +/* + * 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 KafkaShortRetention010ITCase extends KafkaShortRetentionTestBase { + + @Test(timeout=60000) + public void testAutoOffsetReset() throws Exception { + runAutoOffsetResetTest(); + } + + @Test(timeout=60000) + public void testAutoOffsetResetNone() throws Exception { + runFailOnAutoOffsetResetNone(); + } +} diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java new file mode 100644 index 0000000000000..45f0478dafe3b --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java @@ -0,0 +1,331 @@ +/* + * 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.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.testutils.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.apache.kafka.common.requests.MetadataResponse; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +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.10 + */ +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 Properties additionalServerProperties; + + public String getBrokerConnectionString() { + return brokerConnectionString; + } + + @Override + public Properties getStandardProperties() { + return standardProps; + } + + @Override + public String getVersion() { + return "0.10"; + } + + @Override + public List getBrokers() { + return brokers; + } + + @Override + public FlinkKafkaConsumerBase getConsumer(List topics, KeyedDeserializationSchema readSchema, Properties props) { + return new FlinkKafkaConsumer010<>(topics, readSchema, props); + } + + @Override + public FlinkKafkaProducerBase getProducer(String topic, KeyedSerializationSchema serSchema, Properties props, KafkaPartitioner partitioner) { + FlinkKafkaProducer010 prod = new FlinkKafkaProducer010<>(topic, serSchema, props, partitioner); + prod.setFlushOnCheckpoint(true); + return prod; + } + + @Override + public void restartBroker(int leaderId) throws Exception { + brokers.set(leaderId, getKafkaServer(leaderId, tmpKafkaDirs.get(leaderId))); + } + + @Override + public int getLeaderToShutDown(String topic) throws Exception { + ZkUtils zkUtils = getZkUtils(); + try { + MetadataResponse.PartitionMetadata firstPart = null; + do { + if (firstPart != null) { + LOG.info("Unable to find leader. error code {}", firstPart.error().code()); + // not the first try. Sleep a bit + Thread.sleep(150); + } + + List partitionMetadata = AdminUtils.fetchTopicMetadataFromZk(topic, zkUtils).partitionMetadata(); + firstPart = partitionMetadata.get(0); + } + while (firstPart.error().code() != 0); + + return firstPart.leader().id(); + } finally { + zkUtils.close(); + } + } + + @Override + public int getBrokerId(KafkaServer server) { + return server.config().brokerId(); + } + + @Override + public void prepare(int numKafkaServers, Properties additionalServerProperties) { + this.additionalServerProperties = additionalServerProperties; + 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))); + + 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", "30000"); // 6 seconds is default. Seems to be too small for travis. + standardProps.setProperty("zookeeper.connection.timeout.ms", "30000"); + standardProps.setProperty("auto.offset.reset", "earliest"); // read from the beginning. (earliest is kafka 0.10 value) + standardProps.setProperty("fetch.message.max.bytes", "256"); // make a lot of fetches (MESSAGES MUST BE SMALLER!) + } + + @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(zookeeperConnectionString, Integer.valueOf(standardProps.getProperty("zookeeper.session.timeout.ms")), + Integer.valueOf(standardProps.getProperty("zookeeper.connection.timeout.ms")), new ZooKeeperStringSerializer()); + return ZkUtils.apply(creator, false); + } + + @Override + public void createTestTopic(String topic, int numberOfPartitions, int replicationFactor, Properties topicConfig) { + // create topic with one client + LOG.info("Creating topic {}", topic); + + ZkUtils zkUtils = getZkUtils(); + try { + AdminUtils.createTopic(zkUtils, topic, numberOfPartitions, replicationFactor, topicConfig, new kafka.admin.RackAwareMode.Enforced$()); + } 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(zookeeperConnectionString, Integer.valueOf(standardProps.getProperty("zookeeper.session.timeout.ms")), + Integer.valueOf(standardProps.getProperty("zookeeper.connection.timeout.ms")), new ZooKeeperStringSerializer()); + + AdminUtils.deleteTopic(zkUtils, topic); + + zk.close(); + } finally { + zkUtils.close(); + } + } + + /** + * Copied from com.github.sakserv.minicluster.KafkaLocalBrokerIntegrationTest (ASL licensed) + */ + protected KafkaServer getKafkaServer(int brokerId, File tmpFolder) throws Exception { + Properties kafkaProperties = new Properties(); + + // properties have to be Strings + kafkaProperties.put("advertised.host.name", KAFKA_HOST); + 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", "30000"); + kafkaProperties.put("zookeeper.connection.timeout.ms", "30000"); + if(additionalServerProperties != null) { + kafkaProperties.putAll(additionalServerProperties); + } + + 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.10/src/test/resources/log4j-test.properties b/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/resources/log4j-test.properties new file mode 100644 index 0000000000000..fbeb110350f5f --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/resources/log4j-test.properties @@ -0,0 +1,30 @@ +################################################################################ +# 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 +log4j.logger.org.apache.zookeeper=OFF, testlogger +log4j.logger.state.change.logger=OFF, testlogger +log4j.logger.kafka=OFF, testlogger diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/resources/logback-test.xml b/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/resources/logback-test.xml new file mode 100644 index 0000000000000..45b3b92f0cf65 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-0.10/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/pom.xml b/flink-streaming-connectors/pom.xml index 113109bcf1f9a..78e39cadb5151 100644 --- a/flink-streaming-connectors/pom.xml +++ b/flink-streaming-connectors/pom.xml @@ -40,6 +40,7 @@ under the License. flink-connector-kafka-base flink-connector-kafka-0.8 flink-connector-kafka-0.9 + flink-connector-kafka-0.10 flink-connector-elasticsearch flink-connector-elasticsearch2 flink-connector-rabbitmq From 656f82919819bd431622940a1793981202ded9e8 Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Tue, 9 Aug 2016 16:38:21 +0200 Subject: [PATCH 2/9] [FLINK-4035] Refactor the Kafka 0.10 connector to be based upon the 0.9 connector Add a test case for Kafka's new timestamp functionality and update the documentation. --- docs/dev/connectors/kafka.md | 67 ++++- docs/page/js/flink.js | 3 +- .../flink-connector-kafka-0.10/pom.xml | 37 ++- .../kafka/FlinkKafkaConsumer010.java | 121 +------- .../kafka/FlinkKafkaProducer010.java | 199 ++++++++----- .../kafka/Kafka010JsonTableSource.java | 2 +- .../connectors/kafka/Kafka010TableSource.java | 2 +- .../kafka/internal/Kafka010Fetcher.java | 269 ++---------------- .../connectors/kafka/Kafka010ITCase.java | 201 ++++++++++++- .../connectors/kafka/KafkaProducerTest.java | 56 ++-- .../kafka/KafkaTestEnvironmentImpl.java | 11 +- .../kafka/internals/SimpleConsumerThread.java | 2 +- .../kafka/KafkaTestEnvironmentImpl.java | 7 +- .../kafka/FlinkKafkaConsumer09.java | 4 +- .../kafka/FlinkKafkaProducer09.java | 2 +- .../kafka/internal/Kafka09Fetcher.java | 18 +- .../kafka/KafkaTestEnvironmentImpl.java | 6 +- .../kafka/FlinkKafkaConsumerBase.java | 5 +- .../kafka/FlinkKafkaProducerBase.java | 4 +- .../kafka/internals/AbstractFetcher.java | 36 +-- ...cPartitionStateWithPeriodicWatermarks.java | 4 +- ...artitionStateWithPunctuatedWatermarks.java | 4 +- .../kafka/KafkaConsumerTestBase.java | 200 ++++++------- .../kafka/KafkaProducerTestBase.java | 5 +- .../kafka/KafkaShortRetentionTestBase.java | 5 +- .../kafka/KafkaTestEnvironment.java | 7 +- .../AbstractFetcherTimestampsTest.java | 69 ++--- .../kafka/testutils/DataGenerators.java | 84 +++--- .../JobManagerCommunicationUtils.java | 15 +- 29 files changed, 738 insertions(+), 707 deletions(-) diff --git a/docs/dev/connectors/kafka.md b/docs/dev/connectors/kafka.md index d2221facbed4e..6f72cf450cf6a 100644 --- a/docs/dev/connectors/kafka.md +++ b/docs/dev/connectors/kafka.md @@ -46,14 +46,6 @@ For most users, the `FlinkKafkaConsumer08` (part of `flink-connector-kafka`) is - flink-connector-kafka - 0.9.1, 0.10 - FlinkKafkaConsumer082
- FlinkKafkaProducer - 0.8.x - Uses the SimpleConsumer API of Kafka internally. Offsets are committed to ZK by Flink. - - flink-connector-kafka-0.8{{ site.scala_version_suffix }} 1.0.0 FlinkKafkaConsumer08
@@ -61,7 +53,7 @@ For most users, the `FlinkKafkaConsumer08` (part of `flink-connector-kafka`) is 0.8.x Uses the SimpleConsumer API of Kafka internally. Offsets are committed to ZK by Flink. - + flink-connector-kafka-0.9{{ site.scala_version_suffix }} 1.0.0 FlinkKafkaConsumer09
@@ -69,6 +61,14 @@ For most users, the `FlinkKafkaConsumer08` (part of `flink-connector-kafka`) is 0.9.x Uses the new Consumer API Kafka. + + flink-connector-kafka-0.10{{ site.scala_version_suffix }} + 1.2.0 + FlinkKafkaConsumer010
+ FlinkKafkaProducer010 + 0.10.x + This connector supports Kafka messages with timestamps both for producing and consuming. + @@ -87,7 +87,6 @@ Note that the streaming connectors are currently not part of the binary distribu ### Installing Apache Kafka * Follow the instructions from [Kafka's quickstart](https://kafka.apache.org/documentation.html#quickstart) to download the code and launch a server (launching a Zookeeper and a Kafka server is required every time before starting the application). -* On 32 bit computers [this](http://stackoverflow.com/questions/22325364/unrecognized-vm-option-usecompressedoops-when-running-kafka-from-my-ubuntu-in) problem may occur. * If the Kafka and Zookeeper servers are running on a remote machine, then the `advertised.host.name` setting in the `config/server.properties` file must be set to the machine's IP address. ### Kafka Consumer @@ -256,17 +255,28 @@ records to partitions. Example: +
-
+
{% highlight java %} stream.addSink(new FlinkKafkaProducer08("localhost:9092", "my-topic", new SimpleStringSchema())); {% endhighlight %}
-
+
+{% highlight java %} +FlinkKafkaProducer010.writeToKafka(stream, "my-topic", new SimpleStringSchema(), properties); +{% endhighlight %} +
+
{% highlight scala %} stream.addSink(new FlinkKafkaProducer08[String]("localhost:9092", "my-topic", new SimpleStringSchema())) {% endhighlight %}
+
+{% highlight scala %} +FlinkKafkaProducer010.writeToKafka(stream, "my-topic", new SimpleStringSchema(), properties); +{% endhighlight %} +
You can also define a custom Kafka producer configuration for the KafkaSink with the constructor. Please refer to @@ -287,3 +297,36 @@ higher value. There is currently no transactional producer for Kafka, so Flink can not guarantee exactly-once delivery into a Kafka topic. + +### Using Kafka timestamps and Flink event time in Kafka 0.10 + +Since Apache Kafka 0.10., Kafka's messages can carry [timestamps](https://cwiki.apache.org/confluence/display/KAFKA/KIP-32+-+Add+timestamps+to+Kafka+message), indicating +the time the event has occurred (see ["event time" in Apache Flink](../event_time.html)) or the time when the message +has been written to the Kafka broker. + +The `FlinkKafkaConsumer010` will emit records with the timestamp attached, if the time characteristic in Flink is +set to `TimeCharacteristic.EventTime` (`StreamExecutionEnvironment.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)`). + +The Kafka consumer does not emit watermarks. To emit watermarks, the same mechanisms as described above in +"Kafka Consumers and Timestamp Extraction/Watermark Emission" using the `assignTimestampsAndWatermarks` method are applicable. + +There is no need to define a timestamp extractor when using the timestamps from Kafka. The `previousElementTimestamp` argument of +the `extractTimestamp()` method contains the timestamp carried by the Kafka message. + +A timestamp extractor for a Kafka consumer would look like this: +{% highlight java %} +public long extractTimestamp(Long element, long previousElementTimestamp) { + return previousElementTimestamp; +} +{% endhighlight %} + + + +The `FlinkKafkaProducer010` only emits the record timestamp, if `setWriteTimestampToKafka(true)` is set. + +{% highlight java %} +FlinkKafkaProducer010.FlinkKafkaProducer010Configuration config = FlinkKafkaProducer010.writeToKafka(streamWithTimestamps, topic, new SimpleStringSchema(), standardProps); +config.setWriteTimestampToKafka(true); +{% endhighlight %} + + diff --git a/docs/page/js/flink.js b/docs/page/js/flink.js index fdf972c8df397..885a8ffad49b9 100644 --- a/docs/page/js/flink.js +++ b/docs/page/js/flink.js @@ -42,6 +42,7 @@ function codeTabs() { var image = $(this).data("image"); var notabs = $(this).data("notabs"); var capitalizedLang = lang.substr(0, 1).toUpperCase() + lang.substr(1); + lang = lang.replace(/[^a-zA-Z0-9]/g, "_"); var id = "tab_" + lang + "_" + counter; $(this).attr("id", id); if (image != null && langImages[lang]) { @@ -99,9 +100,7 @@ function viewSolution() { // A script to fix internal hash links because we have an overlapping top bar. // Based on https://github.com/twitter/bootstrap/issues/193#issuecomment-2281510 function maybeScrollToHash() { - console.log("HERE"); if (window.location.hash && $(window.location.hash).length) { - console.log("HERE2", $(window.location.hash), $(window.location.hash).offset().top); var newTop = $(window.location.hash).offset().top - 57; $(window).scrollTop(newTop); } diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/pom.xml b/flink-streaming-connectors/flink-connector-kafka-0.10/pom.xml index f2bcb11c5480d..f42540800e4de 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.10/pom.xml +++ b/flink-streaming-connectors/flink-connector-kafka-0.10/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-streaming-connectors - 1.1-SNAPSHOT + 1.2-SNAPSHOT .. @@ -46,16 +46,10 @@ under the License. org.apache.flink - flink-streaming-java_2.10 - ${project.version} - provided - - - - org.apache.flink - flink-connector-kafka-base_2.10 + flink-connector-kafka-0.9_2.10 ${project.version} + org.apache.kafka kafka_${scala.binary.version} @@ -81,12 +75,27 @@ under the License. + + org.apache.flink + flink-connector-kafka-0.9_2.10 + ${project.version} + + + + org.apache.kafka + kafka_${scala.binary.version} + + + test-jar + test + + org.apache.flink flink-connector-kafka-base_2.10 ${project.version} - + org.apache.kafka kafka_${scala.binary.version} @@ -127,6 +136,14 @@ under the License. test + + org.apache.flink + flink-metrics-jmx + ${project.version} + test + + + diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java index 78ccd4ada3b8a..267ff572972e4 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java @@ -28,20 +28,10 @@ import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper; import org.apache.flink.util.SerializedValue; -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.common.PartitionInfo; -import org.apache.kafka.common.serialization.ByteArrayDeserializer; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Properties; -import static org.apache.flink.util.Preconditions.checkNotNull; /** * The Flink Kafka Consumer is a streaming data source that pulls a parallel data stream from @@ -64,30 +54,10 @@ * is constructed. That means that the client that submits the program needs to be able to * reach the Kafka brokers or ZooKeeper.

*/ -public class FlinkKafkaConsumer010 extends FlinkKafkaConsumerBase { +public class FlinkKafkaConsumer010 extends FlinkKafkaConsumer09 { private static final long serialVersionUID = 2324564345203409112L; - private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaConsumer010.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; - - /** 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 */ - private final long pollTimeout; // ------------------------------------------------------------------------ @@ -151,51 +121,7 @@ public FlinkKafkaConsumer010(List topics, DeserializationSchema deser * The properties that are used to configure both the fetcher and the offset handler. */ public FlinkKafkaConsumer010(List topics, KeyedDeserializationSchema deserializer, Properties props) { - super(deserializer); - - checkNotNull(topics, "topics"); - this.properties = checkNotNull(props, "props"); - setDeserializer(this.properties); - - // configure the polling timeout - try { - if (properties.containsKey(KEY_POLL_TIMEOUT)) { - this.pollTimeout = Long.parseLong(properties.getProperty(KEY_POLL_TIMEOUT)); - } else { - this.pollTimeout = DEFAULT_POLL_TIMEOUT; - } - } - catch (Exception e) { - throw new IllegalArgumentException("Cannot parse poll timeout for '" + KEY_POLL_TIMEOUT + '\'', e); - } - - // read the partitions that belong to the listed topics - final List partitions = new ArrayList<>(); - - try (KafkaConsumer consumer = new KafkaConsumer<>(this.properties)) { - for (final String topic: topics) { - // get partitions for each topic - List partitionsForTopic = consumer.partitionsFor(topic); - // for non existing topics, the list might be null. - if (partitionsForTopic != null) { - partitions.addAll(convertToFlinkKafkaTopicPartition(partitionsForTopic)); - } - } - } - - if (partitions.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: {}", partitions.size(), topics); - - if (LOG.isInfoEnabled()) { - logPartitionInfo(LOG, partitions); - } - - // register these partitions - setSubscribedPartitions(partitions); + super(topics, deserializer, props); } @Override @@ -212,48 +138,5 @@ public FlinkKafkaConsumer010(List topics, KeyedDeserializationSchema watermarksPeriodic, watermarksPunctuated, runtimeContext, deserializer, properties, pollTimeout, useMetrics); - - } - - // ------------------------------------------------------------------------ - // Utilities - // ------------------------------------------------------------------------ - - /** - * 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 - */ - private static List convertToFlinkKafkaTopicPartition(List partitions) { - checkNotNull(partitions); - - List ret = new ArrayList<>(partitions.size()); - for (PartitionInfo pi : partitions) { - ret.add(new KafkaTopicPartition(pi.topic(), pi.partition())); - } - return ret; - } - - /** - * Makes sure that the ByteArrayDeserializer is registered in the Kafka properties. - * - * @param props The Kafka properties to register the serializer in. - */ - private static void setDeserializer(Properties props) { - final String deSerName = ByteArrayDeserializer.class.getCanonicalName(); - - Object keyDeSer = props.get(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG); - Object valDeSer = props.get(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG); - - if (keyDeSer != null && !keyDeSer.equals(deSerName)) { - LOG.warn("Ignoring configured key DeSerializer ({})", ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG); - } - if (valDeSer != null && !valDeSer.equals(deSerName)) { - LOG.warn("Ignoring configured value DeSerializer ({})", ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG); - } - - props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, deSerName); - props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, deSerName); } } diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java index 49bce398a88d3..7e936674f629f 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java @@ -17,121 +17,182 @@ package org.apache.flink.streaming.connectors.kafka; +import org.apache.flink.api.java.typeutils.GenericTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.operators.StreamSink; import org.apache.flink.streaming.connectors.kafka.partitioner.FixedPartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema; import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper; import org.apache.flink.streaming.util.serialization.SerializationSchema; +import org.apache.kafka.clients.producer.ProducerRecord; import java.util.Properties; /** - * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.8. + * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.10.x * - * Please note that this producer does not have any reliability guarantees. - * - * @param Type of the messages to write into Kafka. + * Implementation note: This Producer wraps a Flink Kafka 0.9 Producer, overriding only + * the "processElement" / "invoke" method. */ -public class FlinkKafkaProducer010 extends FlinkKafkaProducerBase { +public class FlinkKafkaProducer010 extends StreamSink { - private static final long serialVersionUID = 1L; + /** + * Flag controlling whether we are writing the Flink record's timestamp into Kafka. + */ + private boolean writeTimestampToKafka = false; - // ------------------- Keyless serialization schema constructors ---------------------- + // ---------------------- "Constructors" for the producer ------------------ // /** * 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. + * @param inStream The stream to write to Kafka + * @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 FlinkKafkaProducer010(String brokerList, String topicId, SerializationSchema serializationSchema) { - this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), new FixedPartitioner()); + public static FlinkKafkaProducer010Configuration writeToKafka(DataStream inStream, + String topicId, + KeyedSerializationSchema serializationSchema, + Properties producerConfig) { + return writeToKafka(inStream, topicId, serializationSchema, producerConfig, 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 FlinkKafkaProducer010(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) + * @param inStream The stream to write to Kafka + * @param topicId ID of the Kafka topic. + * @param serializationSchema User defined (keyless) serialization schema. + * @param producerConfig Properties with the producer configuration. */ - public FlinkKafkaProducer010(String topicId, SerializationSchema serializationSchema, Properties producerConfig, KafkaPartitioner customPartitioner) { - this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner); - + public static FlinkKafkaProducer010Configuration writeToKafka(DataStream inStream, + String topicId, + SerializationSchema serializationSchema, + Properties producerConfig) { + return writeToKafka(inStream, topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FixedPartitioner()); } - // ------------------- 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 + * @param inStream The stream to write to Kafka + * @param topicId The name of the target topic + * @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 FlinkKafkaProducer010(String brokerList, String topicId, KeyedSerializationSchema serializationSchema) { - this(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), new FixedPartitioner()); + public static FlinkKafkaProducer010Configuration writeToKafka(DataStream inStream, + String topicId, + KeyedSerializationSchema serializationSchema, + Properties producerConfig, + KafkaPartitioner customPartitioner) { + GenericTypeInfo objectTypeInfo = new GenericTypeInfo<>(Object.class); + FlinkKafkaProducer010 kafkaProducer = new FlinkKafkaProducer010<>(topicId, serializationSchema, producerConfig, customPartitioner); + SingleOutputStreamOperator transformation = inStream.transform("FlinKafkaProducer 0.10.x", objectTypeInfo, kafkaProducer); + return new FlinkKafkaProducer010Configuration<>(transformation, kafkaProducer); } /** - * 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. + * Configuration object returned by the writeToKafka() call. */ - public FlinkKafkaProducer010(String topicId, KeyedSerializationSchema serializationSchema, Properties producerConfig) { - this(topicId, serializationSchema, producerConfig, new FixedPartitioner()); + public static class FlinkKafkaProducer010Configuration extends DataStreamSink { + + private final FlinkKafkaProducer09 wrapped09producer; + private final FlinkKafkaProducer010 producer; + + private FlinkKafkaProducer010Configuration(DataStream stream, FlinkKafkaProducer010 producer) { + super(stream, producer); + this.producer = producer; + this.wrapped09producer = (FlinkKafkaProducer09) producer.userFunction; + } + + /** + * Defines whether the producer should fail on errors, or only log them. + * If this is set to true, then exceptions will be only logged, if set to false, + * exceptions will be eventually thrown and cause the streaming program to + * fail (and enter recovery). + * + * @param logFailuresOnly The flag to indicate logging-only on exceptions. + */ + public void setLogFailuresOnly(boolean logFailuresOnly) { + this.wrapped09producer.setLogFailuresOnly(logFailuresOnly); + } + + /** + * If set to true, the Flink producer will wait for all outstanding messages in the Kafka buffers + * to be acknowledged by the Kafka producer on a checkpoint. + * This way, the producer can guarantee that messages in the Kafka buffers are part of the checkpoint. + * + * @param flush Flag indicating the flushing mode (true = flush on checkpoint) + */ + public void setFlushOnCheckpoint(boolean flush) { + this.wrapped09producer.setFlushOnCheckpoint(flush); + } + + /** + * If set to true, Flink will write the (event time) timestamp attached to each record into Kafka. + * Timestamps must be positive for Kafka to accept them. + * + * @param writeTimestampToKafka Flag indicating if Flink's internal timestamps are written to Kafka. + */ + public void setWriteTimestampToKafka(boolean writeTimestampToKafka) { + this.producer.writeTimestampToKafka = writeTimestampToKafka; + } } /** - * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to - * the topic. + * Create internal Kafka producer, and pass it as a UDF to the StreamSink. * - * @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 FlinkKafkaProducer010(String topicId, KeyedSerializationSchema serializationSchema, Properties producerConfig, KafkaPartitioner customPartitioner) { - super(topicId, serializationSchema, producerConfig, customPartitioner); + private FlinkKafkaProducer010(String topicId, KeyedSerializationSchema serializationSchema, Properties producerConfig, KafkaPartitioner customPartitioner) { + super(new FlinkKafkaProducer09<>(topicId, serializationSchema, producerConfig, customPartitioner)); } + /** + * This method contains the timestamp specific operations + */ @Override - protected void flush() { - if (this.producer != null) { - producer.flush(); + public void processElement(StreamRecord element) throws Exception { + // usually, we would call: userFunction.invoke(element.getValue()); + + final FlinkKafkaProducer09 internalProducer = (FlinkKafkaProducer09) userFunction; + final T next = element.getValue(); + + internalProducer.checkErroneous(); + + byte[] serializedKey = internalProducer.schema.serializeKey(next); + byte[] serializedValue = internalProducer.schema.serializeValue(next); + String targetTopic = internalProducer.schema.getTargetTopic(next); + if (targetTopic == null) { + targetTopic = internalProducer.defaultTopicId; + } + + Long timestamp = null; + if(this.writeTimestampToKafka) { + timestamp = element.getTimestamp(); + } + + ProducerRecord record; + if (internalProducer.partitioner == null) { + record = new ProducerRecord<>(targetTopic, null, timestamp, serializedKey, serializedValue); + } else { + record = new ProducerRecord<>(targetTopic, internalProducer.partitioner.partition(next, serializedKey, serializedValue, internalProducer.partitions.length), timestamp, serializedKey, serializedValue); + } + if (internalProducer.flushOnCheckpoint) { + synchronized (internalProducer.pendingRecordsLock) { + internalProducer.pendingRecords++; + } } + internalProducer.producer.send(record, internalProducer.callback); } + } diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java index cda68cee251a5..ddf1ad3b8166a 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java @@ -28,7 +28,7 @@ /** * Kafka {@link StreamTableSource} for Kafka 0.10. */ -public class Kafka010JsonTableSource extends KafkaJsonTableSource { +public class Kafka010JsonTableSource extends Kafka09JsonTableSource { /** * Creates a Kafka 0.10 JSON {@link StreamTableSource}. diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java index cee1b90716d59..732440b11a45f 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java @@ -28,7 +28,7 @@ /** * Kafka {@link StreamTableSource} for Kafka 0.10. */ -public class Kafka010TableSource extends KafkaTableSource { +public class Kafka010TableSource extends Kafka09TableSource { /** * Creates a Kafka 0.10 {@link StreamTableSource}. diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010Fetcher.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010Fetcher.java index 70f530b96b1cf..41c9ff637f412 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010Fetcher.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010Fetcher.java @@ -18,37 +18,20 @@ package org.apache.flink.streaming.connectors.kafka.internal; -import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; -import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; -import org.apache.flink.streaming.connectors.kafka.internals.ExceptionProxy; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionState; -import org.apache.flink.streaming.connectors.kafka.internals.metrics.DefaultKafkaMetricAccumulator; import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; import org.apache.flink.util.SerializedValue; -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.TopicPartition; -import org.apache.kafka.common.errors.WakeupException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.Properties; /** @@ -56,40 +39,7 @@ * * @param The type of elements produced by the fetcher. */ -public class Kafka010Fetcher extends AbstractFetcher implements Runnable { - - private static final Logger LOG = LoggerFactory.getLogger(Kafka010Fetcher.class); - - // ------------------------------------------------------------------------ - - /** The schema to convert between Kafka's byte messages, and Flink's objects */ - private final KeyedDeserializationSchema deserializer; - - /** The subtask's runtime context */ - private final RuntimeContext runtimeContext; - - /** The configuration for the Kafka consumer */ - private final Properties kafkaProperties; - - /** The maximum number of milliseconds to wait for a fetch batch */ - private final long pollTimeout; - - /** Flag whether to register Kafka metrics as Flink accumulators */ - private final boolean forwardKafkaMetrics; - - /** Mutex to guard against concurrent access to the non-threadsafe Kafka consumer */ - private final Object consumerLock = new Object(); - - /** Reference to the Kafka consumer, once it is created */ - private volatile KafkaConsumer consumer; - - /** Reference to the proxy, forwarding exceptions from the fetch thread to the main thread */ - private volatile ExceptionProxy errorHandler; - - /** Flag to mark the main work loop as alive */ - private volatile boolean running = true; - - // ------------------------------------------------------------------------ +public class Kafka010Fetcher extends Kafka09Fetcher { public Kafka010Fetcher( SourceContext sourceContext, @@ -100,213 +50,42 @@ public Kafka010Fetcher( KeyedDeserializationSchema deserializer, Properties kafkaProperties, long pollTimeout, - boolean forwardKafkaMetrics) throws Exception + boolean useMetrics) throws Exception { - super(sourceContext, assignedPartitions, watermarksPeriodic, watermarksPunctuated, runtimeContext); - - this.deserializer = deserializer; - this.runtimeContext = runtimeContext; - this.kafkaProperties = kafkaProperties; - this.pollTimeout = pollTimeout; - this.forwardKafkaMetrics = forwardKafkaMetrics; - - // if checkpointing is enabled, we are not automatically committing to Kafka. - kafkaProperties.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, - Boolean.toString(!runtimeContext.isCheckpointingEnabled())); - } - - // ------------------------------------------------------------------------ - // Fetcher work methods - // ------------------------------------------------------------------------ - - @Override - public void runFetchLoop() throws Exception { - this.errorHandler = new ExceptionProxy(Thread.currentThread()); - - // rather than running the main fetch loop directly here, we spawn a dedicated thread - // this makes sure that no interrupt() call upon canceling reaches the Kafka consumer code - Thread runner = new Thread(this, "Kafka 0.10 Fetcher for " + runtimeContext.getTaskNameWithSubtasks()); - runner.setDaemon(true); - runner.start(); - - try { - runner.join(); - } catch (InterruptedException e) { - // may be the result of a wake-up after an exception. we ignore this here and only - // restore the interruption state - Thread.currentThread().interrupt(); - } - - // make sure we propagate any exception that occurred in the concurrent fetch thread, - // before leaving this method - this.errorHandler.checkAndThrowException(); + super(sourceContext, assignedPartitions, watermarksPeriodic, watermarksPunctuated, runtimeContext, deserializer, kafkaProperties, pollTimeout, useMetrics); } @Override - public void cancel() { - // flag the main thread to exit - running = false; - - // NOTE: - // - We cannot interrupt the runner thread, because the Kafka consumer may - // deadlock when the thread is interrupted while in certain methods - // - We cannot call close() on the consumer, because it will actually throw - // an exception if a concurrent call is in progress - - // make sure the consumer finds out faster that we are shutting down - if (consumer != null) { - consumer.wakeup(); - } + protected void assignPartitionsToConsumer(KafkaConsumer consumer, List topicPartitions) { + consumer.assign(topicPartitions); } + /** + * Emit record Kafka-timestamp aware. + */ @Override - public void run() { - // This method initializes the KafkaConsumer and guarantees it is torn down properly. - // This is important, because the consumer has multi-threading issues, - // including concurrent 'close()' calls. - - final KafkaConsumer consumer; - try { - consumer = new KafkaConsumer<>(kafkaProperties); - } - catch (Throwable t) { - running = false; - errorHandler.reportError(t); - return; - } - - // from here on, the consumer will be closed properly - try { - consumer.assign(convertKafkaPartitions(subscribedPartitions())); - - // register Kafka metrics to Flink accumulators - if (forwardKafkaMetrics) { - Map metrics = consumer.metrics(); - if (metrics == null) { - // MapR's Kafka implementation returns null here. - LOG.info("Consumer implementation does not support metrics"); - } else { - // we have metrics, register them where possible - for (Map.Entry metric : metrics.entrySet()) { - String name = "KafkaConsumer-" + metric.getKey().name(); - DefaultKafkaMetricAccumulator kafkaAccumulator = - DefaultKafkaMetricAccumulator.createFor(metric.getValue()); - - // best effort: we only add the accumulator if available. - if (kafkaAccumulator != null) { - runtimeContext.addAccumulator(name, kafkaAccumulator); - } - } - } - } - - // seek the consumer to the initial offsets - for (KafkaTopicPartitionState partition : subscribedPartitions()) { - if (partition.isOffsetDefined()) { - consumer.seek(partition.getKafkaPartitionHandle(), partition.getOffset() + 1); - } - } - - // from now on, external operations may call the consumer - this.consumer = consumer; - - // main fetch loop - while (running) { - // get the next batch of records - final ConsumerRecords records; - synchronized (consumerLock) { - try { - records = consumer.poll(pollTimeout); - } - catch (WakeupException we) { - if (running) { - throw we; - } else { - continue; - } - } - } - - // get the records for each topic partition - for (KafkaTopicPartitionState partition : subscribedPartitions()) { - - List> partitionRecords = records.records(partition.getKafkaPartitionHandle()); - - for (ConsumerRecord record : partitionRecords) { - T value = deserializer.deserialize( - record.key(), record.value(), - record.topic(), record.partition(), record.offset()); - - if (deserializer.isEndOfStream(value)) { - // end of stream signaled - running = false; - break; - } - - // emit the actual record. this also update offset state atomically - // and deals with timestamps and watermark generation - emitRecord(value, partition, record.offset()); - } - } + protected void emitRecord(T record, KafkaTopicPartitionState partitionState, long offset, R kafkaRecord) throws Exception { + long timestamp = ((ConsumerRecord) kafkaRecord).timestamp(); + if (timestampWatermarkMode == NO_TIMESTAMPS_WATERMARKS) { + // fast path logic, in case there are no watermarks + + // emit the record, using the checkpoint lock to guarantee + // atomicity of record emission and offset state update + synchronized (checkpointLock) { + sourceContext.collectWithTimestamp(record, timestamp); + partitionState.setOffset(offset); } - // end main fetch loop } - catch (Throwable t) { - if (running) { - running = false; - errorHandler.reportError(t); - } else { - LOG.debug("Stopped ConsumerThread threw exception", t); - } + else if (timestampWatermarkMode == PERIODIC_WATERMARKS) { + emitRecordWithTimestampAndPeriodicWatermark(record, partitionState, offset, timestamp); } - finally { - try { - synchronized (consumerLock) { - consumer.close(); - } - } catch (Throwable t) { - LOG.warn("Error while closing Kafka 0.10 consumer", t); - } + else { + emitRecordWithTimestampAndPunctuatedWatermark(record, partitionState, offset, timestamp); } } - // ------------------------------------------------------------------------ - // Kafka 0.10 specific fetcher behavior - // ------------------------------------------------------------------------ - - @Override - public TopicPartition createKafkaPartitionHandle(KafkaTopicPartition partition) { - return new TopicPartition(partition.getTopic(), partition.getPartition()); - } - @Override - public void commitSpecificOffsetsToKafka(Map offsets) throws Exception { - KafkaTopicPartitionState[] partitions = subscribedPartitions(); - Map offsetsToCommit = new HashMap<>(partitions.length); - - for (KafkaTopicPartitionState partition : partitions) { - Long offset = offsets.get(partition.getKafkaTopicPartition()); - if (offset != null) { - offsetsToCommit.put(partition.getKafkaPartitionHandle(), new OffsetAndMetadata(offset, "")); - } - } - - if (this.consumer != null) { - synchronized (consumerLock) { - this.consumer.commitSync(offsetsToCommit); - } - } - } - - // ------------------------------------------------------------------------ - // Utilities - // ------------------------------------------------------------------------ - - public static Collection convertKafkaPartitions(KafkaTopicPartitionState[] partitions) { - ArrayList result = new ArrayList<>(partitions.length); - for (KafkaTopicPartitionState p : partitions) { - result.add(p.getKafkaPartitionHandle()); - } - return result; + protected String getFetcherName() { + return "Kafka 0.10 Fetcher"; } } diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java index 54278532abc3c..e221f745741c8 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java @@ -17,10 +17,43 @@ package org.apache.flink.streaming.connectors.kafka; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.typeutils.GenericTypeInfo; +import org.apache.flink.api.java.typeutils.TypeInfoParser; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.api.TimeCharacteristic; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.graph.StreamConfig; +import org.apache.flink.streaming.api.operators.ChainingStrategy; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.api.operators.StreamSink; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.StreamTask; +import org.apache.flink.streaming.runtime.tasks.StreamTaskState; +import org.apache.flink.streaming.util.serialization.DeserializationSchema; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +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.streaming.util.serialization.TypeInformationSerializationSchema; import org.junit.Test; +import javax.annotation.Nullable; +import java.io.ByteArrayInputStream; +import java.io.IOException; import java.util.UUID; import static org.junit.Assert.assertTrue; @@ -33,10 +66,6 @@ public class Kafka010ITCase extends KafkaConsumerTestBase { // Suite of Tests // ------------------------------------------------------------------------ - @Override - public String getExpectedKafkaVersion() { - return "0.10"; - } @Test(timeout = 60000) public void testFailOnNoBroker() throws Exception { @@ -124,7 +153,7 @@ public void testAllDeletes() throws Exception { @Test(timeout = 60000) public void testMetricsAndEndOfStream() throws Exception { - runMetricsAndEndOfStreamTest(); + runEndOfStreamTest(); } @Test @@ -189,4 +218,166 @@ public void testJsonTableSourceWithFailOnMissingField() throws Exception { } } + /** + * Kafka 0.10 specific test, ensuring Timestamps are properly written to and read from Kafka + */ + @Test(timeout = 60000) + public void testTimestamps() throws Exception { + + final String topic = "tstopic"; + createTestTopic(topic, 3, 1); + + // ---------- Produce an event time stream into Kafka ------------------- + + StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); + env.setParallelism(1); + env.getConfig().setRestartStrategy(RestartStrategies.noRestart()); + env.getConfig().disableSysoutLogging(); + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); + + DataStream streamWithTimestamps = env.addSource(new SourceFunction() { + boolean running = true; + + @Override + public void run(SourceContext ctx) throws Exception { + long i = 0; + while(running) { + ctx.collectWithTimestamp(i, i*2); + if(i++ == 1000L) { + running = false; + } + } + } + + @Override + public void cancel() { + running = false; + } + }); + + final TypeInformationSerializationSchema longSer = new TypeInformationSerializationSchema<>(TypeInfoParser.parse("Long"), env.getConfig()); + FlinkKafkaProducer010.FlinkKafkaProducer010Configuration prod = FlinkKafkaProducer010.writeToKafka(streamWithTimestamps, topic, new KeyedSerializationSchemaWrapper<>(longSer), standardProps, new KafkaPartitioner() { + @Override + public int partition(Long next, byte[] serializedKey, byte[] serializedValue, int numPartitions) { + return (int)(next % 3); + } + }); + prod.setParallelism(3); + prod.setWriteTimestampToKafka(true); + env.execute("Produce some"); + + // ---------- Consume stream from Kafka ------------------- + + env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); + env.setParallelism(1); + env.getConfig().setRestartStrategy(RestartStrategies.noRestart()); + env.getConfig().disableSysoutLogging(); + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); + + FlinkKafkaConsumer010 kafkaSource = new FlinkKafkaConsumer010<>(topic, new LimitedLongDeserializer(), standardProps); + kafkaSource.assignTimestampsAndWatermarks(new AssignerWithPunctuatedWatermarks() { + @Nullable + @Override + public Watermark checkAndGetNextWatermark(Long lastElement, long extractedTimestamp) { + if(lastElement % 10 == 0) { + return new Watermark(lastElement); + } + return null; + } + + @Override + public long extractTimestamp(Long element, long previousElementTimestamp) { + return previousElementTimestamp; + } + }); + + DataStream stream = env.addSource(kafkaSource); + GenericTypeInfo objectTypeInfo = new GenericTypeInfo<>(Object.class); + stream.transform("timestamp validating operator", objectTypeInfo, new TimestampValidatingOperator()).setParallelism(1); + + env.execute("Consume again"); + + deleteTestTopic(topic); + } + + private static class TimestampValidatingOperator extends StreamSink { + + public TimestampValidatingOperator() { + super(new SinkFunction() { + @Override + public void invoke(Long value) throws Exception { + throw new RuntimeException("Unexpected"); + } + }); + } + + long elCount = 0; + long wmCount = 0; + long lastWM = Long.MIN_VALUE; + + @Override + public void processElement(StreamRecord element) throws Exception { + elCount++; + if(element.getValue() * 2 != element.getTimestamp()) { + throw new RuntimeException("Invalid timestamp: " + element); + } + } + + @Override + public void processWatermark(Watermark mark) throws Exception { + wmCount++; + + if(lastWM <= mark.getTimestamp()) { + lastWM = mark.getTimestamp(); + } else { + throw new RuntimeException("Received watermark higher than the last one"); + } + + if( mark.getTimestamp() % 10 != 0 && mark.getTimestamp() != Long.MAX_VALUE ) { + throw new RuntimeException("Invalid watermark: " + mark.getTimestamp()); + } + } + + @Override + public void close() throws Exception { + super.close(); + if(elCount != 1000L) { + throw new RuntimeException("Wrong final element count " + elCount); + } + + if(wmCount <= 2) { + throw new RuntimeException("Almost no watermarks have been sent " + wmCount); + } + } + } + + private static class LimitedLongDeserializer implements KeyedDeserializationSchema { + + private final TypeInformation ti; + private final TypeSerializer ser; + long cnt = 0; + + public LimitedLongDeserializer() { + this.ti = TypeInfoParser.parse("Long"); + this.ser = ti.createSerializer(new ExecutionConfig()); + } + @Override + public TypeInformation getProducedType() { + return ti; + } + + @Override + public Long deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset) throws IOException { + cnt++; + DataInputView in = new DataInputViewStreamWrapper(new ByteArrayInputStream(message)); + Long e = ser.deserialize(in); + return e; + } + + @Override + public boolean isEndOfStream(Long nextElement) { + return cnt > 1000L; + } + } + } diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java index 5f5ac636790a3..9b46ab60596c8 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java @@ -18,8 +18,12 @@ package org.apache.flink.streaming.connectors.kafka; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.api.common.functions.RichFunction; +import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; import org.apache.flink.streaming.connectors.kafka.testutils.MockRuntimeContext; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema; +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper; import org.apache.flink.streaming.util.serialization.SimpleStringSchema; import org.apache.flink.util.TestLogger; @@ -38,6 +42,8 @@ import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; +import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; import java.util.Collections; import java.util.Properties; import java.util.concurrent.Future; @@ -61,7 +67,7 @@ public void testPropagateExceptions() { try { // mock kafka producer KafkaProducer kafkaProducerMock = mock(KafkaProducer.class); - + // partition setup when(kafkaProducerMock.partitionsFor(anyString())).thenReturn( Collections.singletonList(new PartitionInfo("mock_topic", 42, null, null, null))); @@ -76,21 +82,21 @@ public Future answer(InvocationOnMock invocation) throws Throwab return null; } }); - + // make sure the FlinkKafkaProducer instantiates our mock producer whenNew(KafkaProducer.class).withAnyArguments().thenReturn(kafkaProducerMock); - + // (1) producer that propagates errors - FlinkKafkaProducer010 producerPropagating = new FlinkKafkaProducer010<>( - "mock_topic", new SimpleStringSchema(), new Properties(), null); - producerPropagating.setRuntimeContext(new MockRuntimeContext(17, 3)); - producerPropagating.open(new Configuration()); - + FlinkKafkaProducer010 producerPropagating = newProducer(); + + ((RichFunction)producerPropagating.getUserFunction()).setRuntimeContext(new MockRuntimeContext(17, 3)); + producerPropagating.open(); + try { - producerPropagating.invoke("value"); - producerPropagating.invoke("value"); + producerPropagating.processElement(new StreamRecord<>("value", 1L)); + producerPropagating.processElement(new StreamRecord<>("value", 1L)); fail("This should fail with an exception"); } catch (Exception e) { @@ -98,22 +104,34 @@ public Future answer(InvocationOnMock invocation) throws Throwab assertNotNull(e.getCause().getMessage()); assertTrue(e.getCause().getMessage().contains("Test error")); } + // avoid reuse + producerPropagating = null; // (2) producer that only logs errors - FlinkKafkaProducer010 producerLogging = new FlinkKafkaProducer010<>( - "mock_topic", new SimpleStringSchema(), new Properties(), null); - producerLogging.setLogFailuresOnly(true); - - producerLogging.setRuntimeContext(new MockRuntimeContext(17, 3)); - producerLogging.open(new Configuration()); + FlinkKafkaProducer010 producerLogging = newProducer(); + ((FlinkKafkaProducer09)producerLogging.getUserFunction()).setLogFailuresOnly(true); - producerLogging.invoke("value"); - producerLogging.invoke("value"); + ((RichFunction)producerLogging.getUserFunction()).setRuntimeContext(new MockRuntimeContext(17, 3)); + producerLogging.open(); + + producerLogging.processElement(new StreamRecord<>("value", 1L)); + producerLogging.processElement(new StreamRecord<>("value", 1L)); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); } } + + private static FlinkKafkaProducer010 newProducer() { + try { + Class prodClass = FlinkKafkaProducer010.class; + Constructor ctor = prodClass.getDeclaredConstructor(String.class, KeyedSerializationSchema.class, Properties.class, KafkaPartitioner.class); + ctor.setAccessible(true); + return ctor.newInstance("mock_topic", new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()), new Properties(), null); + } catch (InstantiationException | IllegalAccessException | InvocationTargetException | NoSuchMethodException e) { + throw new RuntimeException("Error while creating producer", e); + } + } } diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java index 45f0478dafe3b..b3a8d73bc4ce0 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java @@ -28,6 +28,8 @@ import org.I0Itec.zkclient.ZkClient; import org.apache.commons.io.FileUtils; import org.apache.curator.test.TestingServer; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.connectors.kafka.testutils.ZooKeeperStringSerializer; import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; @@ -90,10 +92,13 @@ public FlinkKafkaConsumerBase getConsumer(List topics, KeyedDeser } @Override - public FlinkKafkaProducerBase getProducer(String topic, KeyedSerializationSchema serSchema, Properties props, KafkaPartitioner partitioner) { - FlinkKafkaProducer010 prod = new FlinkKafkaProducer010<>(topic, serSchema, props, partitioner); + public DataStreamSink produceIntoKafka(DataStream stream, String topic, KeyedSerializationSchema serSchema, Properties props, KafkaPartitioner partitioner) { + /*FlinkKafkaProducer010 prod = new FlinkKafkaProducer010<>(topic, serSchema, props, partitioner); prod.setFlushOnCheckpoint(true); - return prod; + return prod; */ + FlinkKafkaProducer010.FlinkKafkaProducer010Configuration sink = FlinkKafkaProducer010.writeToKafka(stream, topic, serSchema, props, partitioner); + sink.setFlushOnCheckpoint(true); + return sink; } @Override diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SimpleConsumerThread.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SimpleConsumerThread.java index 35e491acfd75e..7741c590f0a99 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SimpleConsumerThread.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SimpleConsumerThread.java @@ -376,7 +376,7 @@ else if (partitionsRemoved) { continue partitionsLoop; } - owner.emitRecord(value, currentPartition, offset); + owner.emitRecord(value, currentPartition, offset, msg); } else { // no longer running diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java index cbf3d06b7f199..a0d50029bf590 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java @@ -31,6 +31,9 @@ import org.apache.curator.framework.CuratorFrameworkFactory; import org.apache.curator.retry.ExponentialBackoffRetry; import org.apache.curator.test.TestingServer; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionLeader; import org.apache.flink.streaming.connectors.kafka.testutils.ZooKeeperStringSerializer; import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; @@ -101,10 +104,10 @@ public FlinkKafkaConsumerBase getConsumer(List topics, KeyedDeser } @Override - public FlinkKafkaProducerBase getProducer(String topic, KeyedSerializationSchema serSchema, Properties props, KafkaPartitioner partitioner) { + public DataStreamSink produceIntoKafka(DataStream stream, String topic, KeyedSerializationSchema serSchema, Properties props, KafkaPartitioner partitioner) { FlinkKafkaProducer08 prod = new FlinkKafkaProducer08<>(topic, serSchema, props, partitioner); prod.setFlushOnCheckpoint(true); - return prod; + return stream.addSink(prod); } @Override 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 index 8c3eaf86011d6..63e3b92939311 100644 --- 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 @@ -81,11 +81,11 @@ public class FlinkKafkaConsumer09 extends FlinkKafkaConsumerBase { // ------------------------------------------------------------------------ /** User-supplied properties for Kafka **/ - private final Properties properties; + protected final Properties properties; /** 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 */ - private final long pollTimeout; + protected final long pollTimeout; // ------------------------------------------------------------------------ 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 index eb3440aedccec..2a3e39da1c2a2 100644 --- 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 @@ -27,7 +27,7 @@ /** - * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.8. + * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.9. * * Please note that this producer does not have any reliability guarantees. * diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java index 9c861c9feff68..40ff2ff86b8c1 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java @@ -121,7 +121,7 @@ public void runFetchLoop() throws Exception { // rather than running the main fetch loop directly here, we spawn a dedicated thread // this makes sure that no interrupt() call upon canceling reaches the Kafka consumer code - Thread runner = new Thread(this, "Kafka 0.9 Fetcher for " + runtimeContext.getTaskNameWithSubtasks()); + Thread runner = new Thread(this, getFetcherName() + " for " + runtimeContext.getTaskNameWithSubtasks()); runner.setDaemon(true); runner.start(); @@ -173,7 +173,8 @@ public void run() { // from here on, the consumer will be closed properly try { - consumer.assign(convertKafkaPartitions(subscribedPartitions())); + assignPartitionsToConsumer(consumer, convertKafkaPartitions(subscribedPartitions())); + if (useMetrics) { final MetricGroup kafkaMetricGroup = runtimeContext.getMetricGroup().addGroup("KafkaConsumer"); @@ -236,7 +237,7 @@ public void run() { // emit the actual record. this also update offset state atomically // and deals with timestamps and watermark generation - emitRecord(value, partition, record.offset()); + emitRecord(value, partition, record.offset(), record); } } } @@ -261,6 +262,17 @@ public void run() { } } + /** + * Protected method to make the partition assignment pluggable, for different Kafka versions. + */ + protected void assignPartitionsToConsumer(KafkaConsumer consumer, List topicPartitions) { + consumer.assign(topicPartitions); + } + + protected String getFetcherName() { + return "Kafka 0.9 Fetcher"; + } + // ------------------------------------------------------------------------ // Kafka 0.9 specific fetcher behavior // ------------------------------------------------------------------------ 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 index 213ba4a31cb60..9d8fa9aec07eb 100644 --- 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 @@ -29,6 +29,8 @@ import org.I0Itec.zkclient.ZkClient; import org.apache.commons.io.FileUtils; import org.apache.curator.test.TestingServer; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.connectors.kafka.testutils.ZooKeeperStringSerializer; import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; @@ -94,10 +96,10 @@ public FlinkKafkaConsumerBase getConsumer(List topics, KeyedDeser } @Override - public FlinkKafkaProducerBase getProducer(String topic, KeyedSerializationSchema serSchema, Properties props, KafkaPartitioner partitioner) { + public DataStreamSink produceIntoKafka(DataStream stream, String topic, KeyedSerializationSchema serSchema, Properties props, KafkaPartitioner partitioner) { FlinkKafkaProducer09 prod = new FlinkKafkaProducer09<>(topic, serSchema, props, partitioner); prod.setFlushOnCheckpoint(true); - return prod; + return stream.addSink(prod); } @Override 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 index 2b2c52787d868..7b58ba15a7c97 100644 --- 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 @@ -20,6 +20,7 @@ import org.apache.commons.collections.map.LinkedMap; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.ClosureCleaner; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; import org.apache.flink.runtime.state.CheckpointListener; import org.apache.flink.streaming.api.checkpoint.CheckpointedAsynchronously; @@ -158,6 +159,7 @@ public FlinkKafkaConsumerBase assignTimestampsAndWatermarks(AssignerWithPunct throw new IllegalStateException("A periodic watermark emitter has already been set."); } try { + ClosureCleaner.clean(assigner, true); this.punctuatedWatermarkAssigner = new SerializedValue<>(assigner); return this; } catch (Exception e) { @@ -166,7 +168,7 @@ public FlinkKafkaConsumerBase assignTimestampsAndWatermarks(AssignerWithPunct } /** - * Specifies an {@link AssignerWithPunctuatedWatermarks} to emit watermarks in a punctuated manner. + * Specifies an {@link AssignerWithPunctuatedWatermarks} to emit watermarks in a periodic manner. * The watermark extractor will run per Kafka partition, watermarks will be merged across partitions * in the same way as in the Flink runtime, when streams are merged. * @@ -192,6 +194,7 @@ public FlinkKafkaConsumerBase assignTimestampsAndWatermarks(AssignerWithPerio throw new IllegalStateException("A punctuated watermark emitter has already been set."); } try { + ClosureCleaner.clean(assigner, true); this.periodicWatermarkAssigner = new SerializedValue<>(assigner); return this; } catch (Exception e) { diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBase.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBase.java index e63f03313f1dc..b6eb278649b6a 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBase.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBase.java @@ -107,7 +107,7 @@ public abstract class FlinkKafkaProducerBase extends RichSinkFunction im /** * If true, the producer will wait until all outstanding records have been send to the broker. */ - private boolean flushOnCheckpoint; + protected boolean flushOnCheckpoint; // -------------------------------- Runtime fields ------------------------------------------ @@ -139,7 +139,7 @@ public FlinkKafkaProducerBase(String defaultTopicId, KeyedSerializationSchema { - private static final int NO_TIMESTAMPS_WATERMARKS = 0; - private static final int PERIODIC_WATERMARKS = 1; - private static final int PUNCTUATED_WATERMARKS = 2; + protected static final int NO_TIMESTAMPS_WATERMARKS = 0; + protected static final int PERIODIC_WATERMARKS = 1; + protected static final int PUNCTUATED_WATERMARKS = 2; // ------------------------------------------------------------------------ /** The source context to emit records and watermarks to */ - private final SourceContext sourceContext; + protected final SourceContext sourceContext; /** The lock that guarantees that record emission and state updates are atomic, * from the view of taking a checkpoint */ - private final Object checkpointLock; + protected final Object checkpointLock; /** All partitions (and their state) that this fetcher is subscribed to */ private final KafkaTopicPartitionState[] allPartitions; /** The mode describing whether the fetcher also generates timestamps and watermarks */ - private final int timestampWatermarkMode; + protected final int timestampWatermarkMode; /** Flag whether to register metrics for the fetcher */ protected final boolean useMetrics; @@ -209,15 +209,14 @@ public void restoreOffsets(HashMap snapshotState) { // ------------------------------------------------------------------------ /** - * *

Implementation Note: This method is kept brief to be JIT inlining friendly. * That makes the fast path efficient, the extended paths are called as separate methods. - * * @param record The record to emit * @param partitionState The state of the Kafka partition from which the record was fetched - * @param offset The offset from which the record was fetched + * @param offset The offset of the record + * @param kafkaRecord The original Kafka record */ - protected final void emitRecord(T record, KafkaTopicPartitionState partitionState, long offset) { + protected void emitRecord(T record, KafkaTopicPartitionState partitionState, long offset, R kafkaRecord) throws Exception { if (timestampWatermarkMode == NO_TIMESTAMPS_WATERMARKS) { // fast path logic, in case there are no watermarks @@ -229,10 +228,10 @@ protected final void emitRecord(T record, KafkaTopicPartitionState partitio } } else if (timestampWatermarkMode == PERIODIC_WATERMARKS) { - emitRecordWithTimestampAndPeriodicWatermark(record, partitionState, offset); + emitRecordWithTimestampAndPeriodicWatermark(record, partitionState, offset, Long.MIN_VALUE); } else { - emitRecordWithTimestampAndPunctuatedWatermark(record, partitionState, offset); + emitRecordWithTimestampAndPunctuatedWatermark(record, partitionState, offset, Long.MIN_VALUE); } } @@ -240,8 +239,8 @@ else if (timestampWatermarkMode == PERIODIC_WATERMARKS) { * Record emission, if a timestamp will be attached from an assigner that is * also a periodic watermark generator. */ - private void emitRecordWithTimestampAndPeriodicWatermark( - T record, KafkaTopicPartitionState partitionState, long offset) + protected void emitRecordWithTimestampAndPeriodicWatermark( + T record, KafkaTopicPartitionState partitionState, long offset, long kafkaEventTimestamp) { @SuppressWarnings("unchecked") final KafkaTopicPartitionStateWithPeriodicWatermarks withWatermarksState = @@ -253,7 +252,7 @@ private void emitRecordWithTimestampAndPeriodicWatermark( final long timestamp; //noinspection SynchronizationOnLocalVariableOrMethodParameter synchronized (withWatermarksState) { - timestamp = withWatermarksState.getTimestampForRecord(record); + timestamp = withWatermarksState.getTimestampForRecord(record, kafkaEventTimestamp); } // emit the record with timestamp, using the usual checkpoint lock to guarantee @@ -268,8 +267,8 @@ private void emitRecordWithTimestampAndPeriodicWatermark( * Record emission, if a timestamp will be attached from an assigner that is * also a punctuated watermark generator. */ - private void emitRecordWithTimestampAndPunctuatedWatermark( - T record, KafkaTopicPartitionState partitionState, long offset) + protected void emitRecordWithTimestampAndPunctuatedWatermark( + T record, KafkaTopicPartitionState partitionState, long offset, long kafkaEventTimestamp) { @SuppressWarnings("unchecked") final KafkaTopicPartitionStateWithPunctuatedWatermarks withWatermarksState = @@ -277,7 +276,7 @@ private void emitRecordWithTimestampAndPunctuatedWatermark( // only one thread ever works on accessing timestamps and watermarks // from the punctuated extractor - final long timestamp = withWatermarksState.getTimestampForRecord(record); + final long timestamp = withWatermarksState.getTimestampForRecord(record, kafkaEventTimestamp); final Watermark newWatermark = withWatermarksState.checkAndGetNewWatermark(record, timestamp); // emit the record with timestamp, using the usual checkpoint lock to guarantee @@ -293,6 +292,7 @@ private void emitRecordWithTimestampAndPunctuatedWatermark( updateMinPunctuatedWatermark(newWatermark); } } + /** *Checks whether a new per-partition watermark is also a new cross-partition watermark. */ diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithPeriodicWatermarks.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithPeriodicWatermarks.java index 99c5d69ca517b..efdc73fc0f390 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithPeriodicWatermarks.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithPeriodicWatermarks.java @@ -49,8 +49,8 @@ public KafkaTopicPartitionStateWithPeriodicWatermarks( // ------------------------------------------------------------------------ - public long getTimestampForRecord (T record) { - return timestampsAndWatermarks.extractTimestamp(record, Long.MIN_VALUE); + public long getTimestampForRecord(T record, long kafkaEventTimestamp) { + return timestampsAndWatermarks.extractTimestamp(record, kafkaEventTimestamp); } public long getCurrentWatermarkTimestamp() { diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithPunctuatedWatermarks.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithPunctuatedWatermarks.java index b2659900d4d31..edf40ce671e26 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithPunctuatedWatermarks.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithPunctuatedWatermarks.java @@ -54,8 +54,8 @@ public KafkaTopicPartitionStateWithPunctuatedWatermarks( // ------------------------------------------------------------------------ - public long getTimestampForRecord(T record) { - return timestampsAndWatermarks.extractTimestamp(record, Long.MIN_VALUE); + public long getTimestampForRecord(T record, long kafkaEventTimestamp) { + return timestampsAndWatermarks.extractTimestamp(record, kafkaEventTimestamp); } @Nullable diff --git a/flink-streaming-connectors/flink-connector-kafka-base/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 index a87ff8ab1c8a3..4fc01aa484ff8 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/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 @@ -187,7 +187,7 @@ public void runFailOnNoBrokerTest() throws Exception { stream.print(); see.execute("No broker test"); } catch(RuntimeException re) { - if(kafkaServer.getVersion().equals("0.9")) { + if(kafkaServer.getVersion().equals("0.9") || kafkaServer.getVersion().equals("0.10")) { Assert.assertTrue("Wrong RuntimeException thrown: " + StringUtils.stringifyException(re), re.getClass().equals(TimeoutException.class) && re.getMessage().contains("Timeout expired while fetching topic metadata")); @@ -285,11 +285,7 @@ public void cancel() { List topics = new ArrayList<>(); topics.add(topic); topics.add(additionalEmptyTopic); - - Properties props = new Properties(); - props.putAll(standardProps); - props.putAll(secureProps); - FlinkKafkaConsumerBase> source = kafkaServer.getConsumer(topics, sourceSchema, props); + FlinkKafkaConsumerBase> source = kafkaServer.getConsumer(topics, sourceSchema, standardProps); DataStreamSource> consuming = env.addSource(source).setParallelism(parallelism); @@ -560,6 +556,7 @@ public void run() { assertTrue(failueCause.getMessage().contains("Job was cancelled")); if (generator.isAlive()) { + JobManagerCommunicationUtils.cancelCurrentJob(flink.getLeaderGateway(timeout), "String generator"); generator.shutdown(); generator.join(); } @@ -728,7 +725,6 @@ public void cancel() { Properties props = new Properties(); props.putAll(standardProps); props.putAll(secureProps); - stream.addSink(kafkaServer.getProducer("dummy", schema, props, null)); env.execute("Write to topics"); @@ -824,7 +820,7 @@ public void runJsonTableSource(String topic, KafkaTableSource kafkaTableSource) .createRemoteEnvironment("localhost", flinkPort); env.getConfig().disableSysoutLogging(); - env.addSource(new SourceFunction() { + DataStream stream = env.addSource(new SourceFunction() { @Override public void run(SourceContext ctx) throws Exception { for (int i = 0; i < numElements; i++) { @@ -835,11 +831,8 @@ public void run(SourceContext ctx) throws Exception { @Override public void cancel() { } - }).addSink(kafkaServer.getProducer( - topic, - new ByteArraySerializationSchema(), - props, - null)); + }); + kafkaServer.produceIntoKafka(stream,topic, new ByteArraySerializationSchema(), props, null); // Execute blocks env.execute(); @@ -1042,7 +1035,7 @@ public void cancel() { } }); - stream.addSink(kafkaServer.getProducer(topic, new KeyedSerializationSchemaWrapper<>(serSchema), producerProps, null)); + kafkaServer.produceIntoKafka(stream, topic, new KeyedSerializationSchemaWrapper<>(serSchema), producerProps, null); tryExecute(env, "big topology test"); deleteTestTopic(topic); @@ -1134,8 +1127,7 @@ public void cancel() { KeyedSerializationSchema> schema = new TypeInformationKeyValueSerializationSchema<>(Long.class, PojoValue.class, env.getConfig()); Properties producerProperties = FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings); producerProperties.setProperty("retries", "3"); - producerProperties.putAll(secureProps); - kvStream.addSink(kafkaServer.getProducer(topic, schema, producerProperties, null)); + kafkaServer.produceIntoKafka(kvStream, topic, schema, producerProperties, null); env.execute("Write KV to Kafka"); // ----------- Read the data again ------------------- @@ -1220,8 +1212,7 @@ public void cancel() { Properties producerProperties = FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings); producerProperties.setProperty("retries", "3"); producerProperties.putAll(secureProps); - - kvStream.addSink(kafkaServer.getProducer(topic, schema, producerProperties, null)); + kafkaServer.produceIntoKafka(kvStream, topic, schema, producerProperties, null); env.execute("Write deletes to Kafka"); @@ -1312,10 +1303,6 @@ public void run() { env1.getConfig().disableSysoutLogging(); env1.disableOperatorChaining(); // let the source read everything into the network buffers - Properties props = new Properties(); - props.putAll(standardProps); - props.putAll(secureProps); - TypeInformationSerializationSchema> schema = new TypeInformationSerializationSchema<>(TypeInfoParser.>parse("Tuple2"), env1.getConfig()); DataStream> fromKafka = env1.addSource(kafkaServer.getConsumer(topic, schema, props)); fromKafka.flatMap(new FlatMapFunction, Void>() { @@ -1342,7 +1329,7 @@ public void cancel() { } }); - fromGen.addSink(kafkaServer.getProducer(topic, new KeyedSerializationSchemaWrapper<>(schema), props, null)); + kafkaServer.produceIntoKafka(fromGen, topic, new KeyedSerializationSchemaWrapper<>(schema), props, null); env1.execute("Metrics test job"); } catch(Throwable t) { @@ -1457,7 +1444,6 @@ protected void readSequence(StreamExecutionEnvironment env, Properties cc, final TypeInformationSerializationSchema> deser = new TypeInformationSerializationSchema<>(intIntTupleType, env.getConfig()); - cc.putAll(secureProps); // create the consumer FlinkKafkaConsumerBase> consumer = kafkaServer.getConsumer(topicName, deser, cc); @@ -1562,9 +1548,7 @@ public void cancel() { producerProperties.setProperty("retries", "0"); producerProperties.putAll(secureProps); - stream.addSink(kafkaServer.getProducer( - topicName, serSchema, producerProperties, - new Tuple2Partitioner(parallelism))) + kafkaServer.produceIntoKafka(stream, topicName, serSchema, producerProperties, new Tuple2Partitioner(parallelism)) .setParallelism(parallelism); try { @@ -1861,86 +1845,88 @@ public void restoreState(Integer state) { // deleteTestTopic(topic); // } // } - - private void executeAndCatchException(StreamExecutionEnvironment env, String execName) throws Exception { - try { - tryExecutePropagateExceptions(env, execName); - } - catch (ProgramInvocationException | JobExecutionException e) { - // look for NotLeaderForPartitionException - Throwable cause = e.getCause(); - - // search for nested SuccessExceptions - int depth = 0; - while (cause != null && depth++ < 20) { - if (cause instanceof kafka.common.NotLeaderForPartitionException) { - throw (Exception) cause; - } - cause = cause.getCause(); - } - throw e; - } - } - - private void putDataInTopics(StreamExecutionEnvironment env, - Properties producerProperties, - final int elementsPerPartition, - Map topics, - TypeInformation> outputTypeInfo) { - if(topics.size() != 2) { - throw new RuntimeException("This method accepts two topics as arguments."); - } - - TypeInformationSerializationSchema> sinkSchema = - new TypeInformationSerializationSchema<>(outputTypeInfo, env.getConfig()); - - DataStream> stream = env - .addSource(new RichParallelSourceFunction>() { - private boolean running = true; - - @Override - public void run(SourceContext> ctx) throws InterruptedException { - int topic = 0; - int currentTs = 1; - - while (running && currentTs < elementsPerPartition) { - long timestamp = (currentTs % 10 == 0) ? -1L : currentTs; - ctx.collect(new Tuple2(timestamp, topic)); - currentTs++; - } - - Tuple2 toWrite2 = new Tuple2(-1L, topic); - ctx.collect(toWrite2); - } - - @Override - public void cancel() { - running = false; - } - }).setParallelism(1); - - List> topicsL = new ArrayList<>(topics.entrySet()); - stream.map(new MapFunction, Tuple2>() { - - @Override - public Tuple2 map(Tuple2 value) throws Exception { - return value; - } - }).setParallelism(1).addSink(kafkaServer.getProducer(topicsL.get(0).getKey(), - new KeyedSerializationSchemaWrapper<>(sinkSchema), producerProperties, null)).setParallelism(1); - - if(!topicsL.get(1).getValue()) { - stream.map(new MapFunction, Tuple2>() { - - @Override - public Tuple2 map(Tuple2 value) throws Exception { - long timestamp = (value.f0 == -1) ? -1L : 1000 + value.f0; - return new Tuple2(timestamp, 1); - } - }).setParallelism(1).addSink(kafkaServer.getProducer(topicsL.get(1).getKey(), - new KeyedSerializationSchemaWrapper<>(sinkSchema), producerProperties, null)).setParallelism(1); - } - } +// +// private void executeAndCatchException(StreamExecutionEnvironment env, String execName) throws Exception { +// try { +// tryExecutePropagateExceptions(env, execName); +// } +// catch (ProgramInvocationException | JobExecutionException e) { +// // look for NotLeaderForPartitionException +// Throwable cause = e.getCause(); +// +// // search for nested SuccessExceptions +// int depth = 0; +// while (cause != null && depth++ < 20) { +// if (cause instanceof kafka.common.NotLeaderForPartitionException) { +// throw (Exception) cause; +// } +// cause = cause.getCause(); +// } +// throw e; +// } +// } +// +// private void putDataInTopics(StreamExecutionEnvironment env, +// Properties producerProperties, +// final int elementsPerPartition, +// Map topics, +// TypeInformation> outputTypeInfo) { +// if(topics.size() != 2) { +// throw new RuntimeException("This method accepts two topics as arguments."); +// } +// +// TypeInformationSerializationSchema> sinkSchema = +// new TypeInformationSerializationSchema<>(outputTypeInfo, env.getConfig()); +// +// DataStream> stream = env +// .addSource(new RichParallelSourceFunction>() { +// private boolean running = true; +// +// @Override +// public void run(SourceContext> ctx) throws InterruptedException { +// int topic = 0; +// int currentTs = 1; +// +// while (running && currentTs < elementsPerPartition) { +// long timestamp = (currentTs % 10 == 0) ? -1L : currentTs; +// ctx.collect(new Tuple2(timestamp, topic)); +// currentTs++; +// } +// +// Tuple2 toWrite2 = new Tuple2(-1L, topic); +// ctx.collect(toWrite2); +// } +// +// @Override +// public void cancel() { +// running = false; +// } +// }).setParallelism(1); +// +// List> topicsL = new ArrayList<>(topics.entrySet()); +// +// stream = stream.map(new MapFunction, Tuple2>() { +// +// @Override +// public Tuple2 map(Tuple2 value) throws Exception { +// return value; +// } +// }).setParallelism(1); +// kafkaServer.produceIntoKafka(stream, topicsL.get(0).getKey(), +// new KeyedSerializationSchemaWrapper<>(sinkSchema), producerProperties, null).setParallelism(1); +// +// if(!topicsL.get(1).getValue()) { +// stream.map(new MapFunction, Tuple2>() { +// +// @Override +// public Tuple2 map(Tuple2 value) throws Exception { +// long timestamp = (value.f0 == -1) ? -1L : 1000 + value.f0; +// return new Tuple2<>(timestamp, 1); +// } +// }).setParallelism(1).addSink(kafkaServer.produceIntoKafka(topicsL.get(1).getKey(), +// new KeyedSerializationSchemaWrapper<>(sinkSchema), producerProperties, null)).setParallelism(1); +// } +// } private DataStreamSink> runPunctuatedComsumer(StreamExecutionEnvironment env, List topics, diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java index 5bcf406533e83..c925c8f188c85 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java @@ -109,11 +109,10 @@ public void cancel() { props.putAll(secureProps); // sink partitions into - stream.addSink(kafkaServer.getProducer(topic, + kafkaServer.produceIntoKafka(stream, topic, new KeyedSerializationSchemaWrapper<>(serSchema), props, - new CustomPartitioner(parallelism))) - .setParallelism(parallelism); + new CustomPartitioner(parallelism)).setParallelism(parallelism); // ------ consuming topology --------- diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java index 9236e78bb1836..f735cece6891e 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java @@ -31,6 +31,7 @@ import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper; import org.apache.flink.streaming.util.serialization.SimpleStringSchema; +import org.apache.flink.test.util.ForkableFlinkMiniCluster; import org.apache.flink.util.InstantiationUtil; import org.junit.AfterClass; @@ -165,12 +166,10 @@ public void cancel() { running = false; } }); - Properties props = new Properties(); props.putAll(standardProps); props.putAll(secureProps); - - stream.addSink(kafkaServer.getProducer(topic, new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()), props, null)); + kafkaServer.produceIntoKafka(stream, topic, new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()), props, null); // ----------- add consumer dataflow ---------- 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 index 6ecde719cc6a0..ded1fded47d88 100644 --- 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 @@ -18,6 +18,9 @@ package org.apache.flink.streaming.connectors.kafka; import kafka.server.KafkaServer; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; 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; @@ -76,7 +79,9 @@ public FlinkKafkaConsumerBase getConsumer(String topic, DeserializationSc public abstract FlinkKafkaConsumerBase getConsumer(List topics, KeyedDeserializationSchema readSchema, Properties props); - public abstract FlinkKafkaProducerBase getProducer(String topic, KeyedSerializationSchema serSchema, Properties props, KafkaPartitioner partitioner); + public abstract DataStreamSink produceIntoKafka(DataStream stream, String topic, + KeyedSerializationSchema serSchema, Properties props, + KafkaPartitioner partitioner); // -- leader failure simulation diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java index 8c68fbe7148e4..10bfc66c6bb10 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java @@ -28,6 +28,7 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.util.SerializedValue; +import org.apache.kafka.clients.consumer.ConsumerRecord; import org.junit.Test; import javax.annotation.Nullable; @@ -42,10 +43,11 @@ public class AbstractFetcherTimestampsTest { @Test public void testPunctuatedWatermarks() throws Exception { + final String testTopic = "test topic name"; List originalPartitions = Arrays.asList( - new KafkaTopicPartition("test topic name", 7), - new KafkaTopicPartition("test topic name", 13), - new KafkaTopicPartition("test topic name", 21)); + new KafkaTopicPartition(testTopic, 7), + new KafkaTopicPartition(testTopic, 13), + new KafkaTopicPartition(testTopic, 21)); TestSourceContext sourceContext = new TestSourceContext<>(); @@ -61,22 +63,22 @@ public void testPunctuatedWatermarks() throws Exception { // elements generate a watermark if the timestamp is a multiple of three // elements for partition 1 - fetcher.emitRecord(1L, part1, 1L); - fetcher.emitRecord(2L, part1, 2L); - fetcher.emitRecord(3L, part1, 3L); + fetcher.emitRecord(1L, part1, 1L, new ConsumerRecord(testTopic, 7, new byte[]{0}, 1L)); + fetcher.emitRecord(2L, part1, 2L, new ConsumerRecord(testTopic, 7, new byte[]{0}, 2L)); + fetcher.emitRecord(3L, part1, 3L, new ConsumerRecord(testTopic, 7, new byte[]{0}, 3L)); assertEquals(3L, sourceContext.getLatestElement().getValue().longValue()); assertEquals(3L, sourceContext.getLatestElement().getTimestamp()); assertFalse(sourceContext.hasWatermark()); // elements for partition 2 - fetcher.emitRecord(12L, part2, 1L); + fetcher.emitRecord(12L, part2, 1L, new ConsumerRecord(testTopic, 13, new byte[]{0}, 1L)); assertEquals(12L, sourceContext.getLatestElement().getValue().longValue()); assertEquals(12L, sourceContext.getLatestElement().getTimestamp()); assertFalse(sourceContext.hasWatermark()); // elements for partition 3 - fetcher.emitRecord(101L, part3, 1L); - fetcher.emitRecord(102L, part3, 2L); + fetcher.emitRecord(101L, part3, 1L, new ConsumerRecord(testTopic, 21, new byte[]{0}, 1L)); + fetcher.emitRecord(102L, part3, 2L, new ConsumerRecord(testTopic, 21, new byte[]{0}, 2L)); assertEquals(102L, sourceContext.getLatestElement().getValue().longValue()); assertEquals(102L, sourceContext.getLatestElement().getTimestamp()); @@ -85,25 +87,25 @@ public void testPunctuatedWatermarks() throws Exception { assertEquals(3L, sourceContext.getLatestWatermark().getTimestamp()); // advance partition 3 - fetcher.emitRecord(1003L, part3, 3L); - fetcher.emitRecord(1004L, part3, 4L); - fetcher.emitRecord(1005L, part3, 5L); + fetcher.emitRecord(1003L, part3, 3L, new ConsumerRecord(testTopic, 21, new byte[]{0}, 3L)); + fetcher.emitRecord(1004L, part3, 4L, new ConsumerRecord(testTopic, 21, new byte[]{0}, 4L)); + fetcher.emitRecord(1005L, part3, 5L, new ConsumerRecord(testTopic, 21, new byte[]{0}, 5L)); assertEquals(1005L, sourceContext.getLatestElement().getValue().longValue()); assertEquals(1005L, sourceContext.getLatestElement().getTimestamp()); // advance partition 1 beyond partition 2 - this bumps the watermark - fetcher.emitRecord(30L, part1, 4L); + fetcher.emitRecord(30L, part1, 4L, new ConsumerRecord(testTopic, 7, new byte[]{0}, 4L)); assertEquals(30L, sourceContext.getLatestElement().getValue().longValue()); assertEquals(30L, sourceContext.getLatestElement().getTimestamp()); assertTrue(sourceContext.hasWatermark()); assertEquals(12L, sourceContext.getLatestWatermark().getTimestamp()); // advance partition 2 again - this bumps the watermark - fetcher.emitRecord(13L, part2, 2L); + fetcher.emitRecord(13L, part2, 2L, new ConsumerRecord(testTopic, 13, new byte[]{0}, 2L)); assertFalse(sourceContext.hasWatermark()); - fetcher.emitRecord(14L, part2, 3L); + fetcher.emitRecord(14L, part2, 3L, new ConsumerRecord(testTopic, 13, new byte[]{0}, 3L)); assertFalse(sourceContext.hasWatermark()); - fetcher.emitRecord(15L, part2, 3L); + fetcher.emitRecord(15L, part2, 3L, new ConsumerRecord(testTopic, 13, new byte[]{0}, 3L)); assertTrue(sourceContext.hasWatermark()); assertEquals(15L, sourceContext.getLatestWatermark().getTimestamp()); } @@ -112,11 +114,12 @@ public void testPunctuatedWatermarks() throws Exception { public void testPeriodicWatermarks() throws Exception { ExecutionConfig config = new ExecutionConfig(); config.setAutoWatermarkInterval(10); - + + final String testTopic = "test topic name"; List originalPartitions = Arrays.asList( - new KafkaTopicPartition("test topic name", 7), - new KafkaTopicPartition("test topic name", 13), - new KafkaTopicPartition("test topic name", 21)); + new KafkaTopicPartition(testTopic, 7), + new KafkaTopicPartition(testTopic, 13), + new KafkaTopicPartition(testTopic, 21)); TestSourceContext sourceContext = new TestSourceContext<>(); @@ -132,20 +135,20 @@ public void testPeriodicWatermarks() throws Exception { // elements generate a watermark if the timestamp is a multiple of three // elements for partition 1 - fetcher.emitRecord(1L, part1, 1L); - fetcher.emitRecord(2L, part1, 2L); - fetcher.emitRecord(3L, part1, 3L); + fetcher.emitRecord(1L, part1, 1L, new ConsumerRecord(testTopic, 7, new byte[]{0}, 1L)); + fetcher.emitRecord(2L, part1, 2L, new ConsumerRecord(testTopic, 7, new byte[]{0}, 2L)); + fetcher.emitRecord(3L, part1, 3L, new ConsumerRecord(testTopic, 7, new byte[]{0}, 3L)); assertEquals(3L, sourceContext.getLatestElement().getValue().longValue()); assertEquals(3L, sourceContext.getLatestElement().getTimestamp()); // elements for partition 2 - fetcher.emitRecord(12L, part2, 1L); + fetcher.emitRecord(12L, part2, 1L, new ConsumerRecord(testTopic, 13, new byte[]{0}, 1L)); assertEquals(12L, sourceContext.getLatestElement().getValue().longValue()); assertEquals(12L, sourceContext.getLatestElement().getTimestamp()); // elements for partition 3 - fetcher.emitRecord(101L, part3, 1L); - fetcher.emitRecord(102L, part3, 2L); + fetcher.emitRecord(101L, part3, 1L, new ConsumerRecord(testTopic, 21, new byte[]{0}, 1L)); + fetcher.emitRecord(102L, part3, 2L, new ConsumerRecord(testTopic, 21, new byte[]{0}, 2L)); assertEquals(102L, sourceContext.getLatestElement().getValue().longValue()); assertEquals(102L, sourceContext.getLatestElement().getTimestamp()); @@ -153,14 +156,14 @@ public void testPeriodicWatermarks() throws Exception { assertEquals(3L, sourceContext.getLatestWatermark().getTimestamp()); // advance partition 3 - fetcher.emitRecord(1003L, part3, 3L); - fetcher.emitRecord(1004L, part3, 4L); - fetcher.emitRecord(1005L, part3, 5L); + fetcher.emitRecord(1003L, part3, 3L, new ConsumerRecord(testTopic, 21, new byte[]{0}, 3L)); + fetcher.emitRecord(1004L, part3, 4L, new ConsumerRecord(testTopic, 21, new byte[]{0}, 4L)); + fetcher.emitRecord(1005L, part3, 5L, new ConsumerRecord(testTopic, 21, new byte[]{0}, 5L)); assertEquals(1005L, sourceContext.getLatestElement().getValue().longValue()); assertEquals(1005L, sourceContext.getLatestElement().getTimestamp()); // advance partition 1 beyond partition 2 - this bumps the watermark - fetcher.emitRecord(30L, part1, 4L); + fetcher.emitRecord(30L, part1, 4L, new ConsumerRecord(testTopic, 7, new byte[]{0}, 4L)); assertEquals(30L, sourceContext.getLatestElement().getValue().longValue()); assertEquals(30L, sourceContext.getLatestElement().getTimestamp()); @@ -168,9 +171,9 @@ public void testPeriodicWatermarks() throws Exception { assertEquals(12L, sourceContext.getLatestWatermark().getTimestamp()); // advance partition 2 again - this bumps the watermark - fetcher.emitRecord(13L, part2, 2L); - fetcher.emitRecord(14L, part2, 3L); - fetcher.emitRecord(15L, part2, 3L); + fetcher.emitRecord(13L, part2, 2L, new ConsumerRecord(testTopic, 13, new byte[]{0}, 2L)); + fetcher.emitRecord(14L, part2, 3L, new ConsumerRecord(testTopic, 13, new byte[]{0}, 3L)); + fetcher.emitRecord(15L, part2, 3L, new ConsumerRecord(testTopic, 13, new byte[]{0}, 3L)); // this blocks until the periodic thread emitted the watermark long watermarkTs = sourceContext.getLatestWatermark().getTimestamp(); diff --git a/flink-streaming-connectors/flink-connector-kafka-base/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 index 58a5cc307e3c5..0f54409e8d38a 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/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 @@ -63,7 +63,7 @@ public void run(SourceContext> ctx) throws Exception { int partition = getRuntimeContext().getIndexOfThisSubtask(); while (running && cnt <= to) { - ctx.collect(new Tuple2(partition, cnt)); + ctx.collect(new Tuple2<>(partition, cnt)); cnt++; } } @@ -74,11 +74,11 @@ public void cancel() { } }); - stream.addSink(testServer.getProducer(topic, + testServer.produceIntoKafka(stream, topic, new KeyedSerializationSchemaWrapper<>(new TypeInformationSerializationSchema<>(resultType, env.getConfig())), FlinkKafkaProducerBase.getPropertiesFromBrokerList(testServer.getBrokerConnectionString()), new Tuple2Partitioner(numPartitions) - )); + ); env.execute("Data generator (Int, Int) stream to topic " + topic); } @@ -141,38 +141,40 @@ public void cancel() { if(secureProps != null) { props.putAll(testServer.getSecureProperties()); } - - stream - .rebalance() - .addSink(testServer.getProducer(topic, - new KeyedSerializationSchemaWrapper<>(new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, env.getConfig())), - props, - new KafkaPartitioner() { - @Override - public int partition(Integer next, byte[] serializedKey, byte[] serializedValue, int numPartitions) { - return next % numPartitions; - } - })); + + stream = stream.rebalance(); + testServer.produceIntoKafka(stream, topic, + new KeyedSerializationSchemaWrapper<>(new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, env.getConfig())), + props, + new KafkaPartitioner() { + @Override + public int partition(Integer next, byte[] serializedKey, byte[] serializedValue, int numPartitions) { + return next % numPartitions; + } + }); env.execute("Scrambles int sequence generator"); } // ------------------------------------------------------------------------ - public static class InfiniteStringsGenerator extends Thread { + public static class InfiniteStringsGenerator extends Thread implements Serializable{ - private final KafkaTestEnvironment server; + private transient KafkaTestEnvironment server; private final String topic; - + + private final int flinkPort; + private volatile Throwable error; private volatile boolean running = true; - public InfiniteStringsGenerator(KafkaTestEnvironment server, String topic) { + public InfiniteStringsGenerator(KafkaTestEnvironment server, String topic, int flinkPort) { this.server = server; this.topic = topic; + this.flinkPort = flinkPort; } @Override @@ -180,28 +182,36 @@ public void run() { // we manually feed data into the Kafka sink FlinkKafkaProducerBase producer = null; try { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); + DataStream stream = env.addSource(new SourceFunction() { + @Override + public void run(SourceContext ctx) throws Exception { + final StringBuilder bld = new StringBuilder(); + final Random rnd = new Random(); + while (running) { + bld.setLength(0); + int len = rnd.nextInt(100) + 1; + for (int i = 0; i < len; i++) { + bld.append((char) (rnd.nextInt(20) + 'a')); + } + + String next = bld.toString(); + ctx.collect(next); + } + } + + @Override + public void cancel() { + running = false; + } + }); + Properties producerProperties = FlinkKafkaProducerBase.getPropertiesFromBrokerList(server.getBrokerConnectionString()); producerProperties.setProperty("retries", "3"); - producer = server.getProducer(topic, + server.produceIntoKafka(stream, topic, new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()), producerProperties, new FixedPartitioner()); - producer.setRuntimeContext(new MockRuntimeContext(1,0)); - producer.open(new Configuration()); - - final StringBuilder bld = new StringBuilder(); - final Random rnd = new Random(); - - while (running) { - bld.setLength(0); - - int len = rnd.nextInt(100) + 1; - for (int i = 0; i < len; i++) { - bld.append((char) (rnd.nextInt(20) + 'a') ); - } - - String next = bld.toString(); - producer.invoke(next); - } + env.execute("String generator"); } catch (Throwable t) { this.error = t; diff --git a/flink-streaming-connectors/flink-connector-kafka-base/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 index 028045afd1201..143095663fcd2 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/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 @@ -54,6 +54,10 @@ public static void waitUntilNoJobIsRunning(ActorGateway jobManager) throws Excep } public static void cancelCurrentJob(ActorGateway jobManager) throws Exception { + cancelCurrentJob(jobManager, null); + } + + public static void cancelCurrentJob(ActorGateway jobManager, String name) throws Exception { JobStatusMessage status = null; for (int i = 0; i < 200; i++) { @@ -78,7 +82,16 @@ public static void cancelCurrentJob(ActorGateway jobManager) throws Exception { else if (jobs.size() == 1) { status = jobs.get(0); } - else { + else if(name != null) { + for(JobStatusMessage msg: jobs) { + if(msg.getJobName().equals(name)) { + status = msg; + } + } + if(status == null) { + throw new Exception("Could not cancel job - no job matched expected name = '" + name +"' in " + jobs); + } + } else { throw new Exception("Could not cancel job - more than one running job."); } } From 724c6ffd915a599007e0e795acb59dd1eb7483ca Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Tue, 23 Aug 2016 18:21:09 +0200 Subject: [PATCH 3/9] remove explicit exclusion --- .../flink-connector-kafka-0.10/pom.xml | 23 ++++++++----------- 1 file changed, 9 insertions(+), 14 deletions(-) diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/pom.xml b/flink-streaming-connectors/flink-connector-kafka-0.10/pom.xml index f42540800e4de..0b426b5ac61b0 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.10/pom.xml +++ b/flink-streaming-connectors/flink-connector-kafka-0.10/pom.xml @@ -37,7 +37,7 @@ under the License. - 0.10.0.0 + 0.10.0.1 @@ -48,13 +48,14 @@ under the License. org.apache.flink flink-connector-kafka-0.9_2.10 ${project.version} - - - - org.apache.kafka - kafka_${scala.binary.version} - - + + + + + + org.apache.kafka + kafka-clients + ${kafka.version} @@ -67,12 +68,6 @@ under the License. true - - org.apache.kafka - kafka-clients - ${kafka.version} - - From 75f17f500b26fc02ea5ddb272d95b4cd36bc19d5 Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Thu, 25 Aug 2016 10:02:26 +0200 Subject: [PATCH 4/9] turn the producer into a hybrid --- .../kafka/FlinkKafkaProducer010.java | 307 +++++++++++++++--- .../kafka/KafkaTestEnvironmentImpl.java | 8 +- 2 files changed, 258 insertions(+), 57 deletions(-) diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java index 7e936674f629f..9bb18a92b9a7b 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java @@ -17,10 +17,15 @@ package org.apache.flink.streaming.connectors.kafka; +import org.apache.flink.api.common.functions.IterationRuntimeContext; +import org.apache.flink.api.common.functions.RichFunction; +import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.java.typeutils.GenericTypeInfo; +import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.api.operators.StreamSink; import org.apache.flink.streaming.connectors.kafka.partitioner.FixedPartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; @@ -32,26 +37,47 @@ import java.util.Properties; +import static org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducerBase.getPropertiesFromBrokerList; + /** * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.10.x * - * Implementation note: This Producer wraps a Flink Kafka 0.9 Producer, overriding only - * the "processElement" / "invoke" method. + * Implementation note: This producer is a hybrid between a regular regular sink function (a) + * and a custom operator (b). + * + * For (a), the class implements the SinkFunction and RichFunction interfaces. + * For (b), it extends the StreamTask class. + * + * Details about approach (a): + * + * Pre Kafka 0.10 producers only follow approach (a), allowing users to use the producer using the + * DataStream.addSink() method. + * Since the APIs exposed in that variant do not allow accessing the the timestamp attached to the record + * the Kafka 0.10 producer has a section invocation option, approach (b). + * + * Details about approach (b): + * Kafka 0.10 supports writing the timestamp attached to a record to Kafka. When adding the + * FlinkKafkaProducer010 using the FlinkKafkaProducer010.writeToKafka() method, the Kafka producer + * can access the internal record timestamp of the record and write it to Kafka. + * + * All methods and constructors in this class are marked with the approach they are needed for. */ -public class FlinkKafkaProducer010 extends StreamSink { +public class FlinkKafkaProducer010 extends StreamSink implements SinkFunction, RichFunction { /** * Flag controlling whether we are writing the Flink record's timestamp into Kafka. */ private boolean writeTimestampToKafka = false; - // ---------------------- "Constructors" for the producer ------------------ // + // ---------------------- "Constructors" for timestamp writing ------------------ /** * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to * the topic. * + * This constructor allows writing timestamps to Kafka, it follow approach (b) (see above) + * * @param inStream The stream to write to Kafka * @param topicId ID of the Kafka topic. * @param serializationSchema User defined serialization schema supporting key/value messages @@ -69,6 +95,8 @@ public static FlinkKafkaProducer010Configuration writeToKafka(DataStream * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to * the topic. * + * This constructor allows writing timestamps to Kafka, it follow approach (b) (see above) + * * @param inStream The stream to write to Kafka * @param topicId ID of the Kafka topic. * @param serializationSchema User defined (keyless) serialization schema. @@ -84,6 +112,9 @@ public static FlinkKafkaProducer010Configuration writeToKafka(DataStream /** * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to * the topic. + * + * This constructor allows writing timestamps to Kafka, it follow approach (b) (see above) + * * @param inStream The stream to write to Kafka * @param topicId The name of the target topic * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages @@ -95,24 +126,239 @@ public static FlinkKafkaProducer010Configuration writeToKafka(DataStream< KeyedSerializationSchema serializationSchema, Properties producerConfig, KafkaPartitioner customPartitioner) { + GenericTypeInfo objectTypeInfo = new GenericTypeInfo<>(Object.class); FlinkKafkaProducer010 kafkaProducer = new FlinkKafkaProducer010<>(topicId, serializationSchema, producerConfig, customPartitioner); SingleOutputStreamOperator transformation = inStream.transform("FlinKafkaProducer 0.10.x", objectTypeInfo, kafkaProducer); return new FlinkKafkaProducer010Configuration<>(transformation, kafkaProducer); } + // ---------------------- Regular constructors w/o timestamp support ------------------ + + /** + * 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 FlinkKafkaProducer010(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 FlinkKafkaProducer010(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 FlinkKafkaProducer010(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 FlinkKafkaProducer010(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 FlinkKafkaProducer010(String topicId, KeyedSerializationSchema serializationSchema, Properties producerConfig) { + this(topicId, serializationSchema, producerConfig, new FixedPartitioner()); + } + + /** + * Create Kafka producer + * + * This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above) + */ + public FlinkKafkaProducer010(String topicId, KeyedSerializationSchema serializationSchema, Properties producerConfig, KafkaPartitioner customPartitioner) { + // We create a Kafka 09 producer instance here and only "override" (by intercepting) the + // invoke call. + super(new FlinkKafkaProducer09<>(topicId, serializationSchema, producerConfig, customPartitioner)); + } + + + // ----------------------------- Generic element processing --------------------------- + + private void invokeInternal(T next, long elementTimestamp) throws Exception { + + final FlinkKafkaProducerBase internalProducer = (FlinkKafkaProducerBase) userFunction; + + internalProducer.checkErroneous(); + + byte[] serializedKey = internalProducer.schema.serializeKey(next); + byte[] serializedValue = internalProducer.schema.serializeValue(next); + String targetTopic = internalProducer.schema.getTargetTopic(next); + if (targetTopic == null) { + targetTopic = internalProducer.defaultTopicId; + } + + Long timestamp = null; + if(this.writeTimestampToKafka) { + timestamp = elementTimestamp; + } + + ProducerRecord record; + if (internalProducer.partitioner == null) { + record = new ProducerRecord<>(targetTopic, null, timestamp, serializedKey, serializedValue); + } else { + record = new ProducerRecord<>(targetTopic, internalProducer.partitioner.partition(next, serializedKey, serializedValue, internalProducer.partitions.length), timestamp, serializedKey, serializedValue); + } + if (internalProducer.flushOnCheckpoint) { + synchronized (internalProducer.pendingRecordsLock) { + internalProducer.pendingRecords++; + } + } + internalProducer.producer.send(record, internalProducer.callback); + } + + + // ----------------- Helper methods implementing methods from SinkFunction and RichFunction (Approach (a)) ---- + + + // ---- Configuration setters + + /** + * Defines whether the producer should fail on errors, or only log them. + * If this is set to true, then exceptions will be only logged, if set to false, + * exceptions will be eventually thrown and cause the streaming program to + * fail (and enter recovery). + * + * Method is only accessible for approach (a) (see above) + * + * @param logFailuresOnly The flag to indicate logging-only on exceptions. + */ + public void setLogFailuresOnly(boolean logFailuresOnly) { + final FlinkKafkaProducerBase internalProducer = (FlinkKafkaProducerBase) userFunction; + internalProducer.setLogFailuresOnly(logFailuresOnly); + } + + /** + * If set to true, the Flink producer will wait for all outstanding messages in the Kafka buffers + * to be acknowledged by the Kafka producer on a checkpoint. + * This way, the producer can guarantee that messages in the Kafka buffers are part of the checkpoint. + * + * Method is only accessible for approach (a) (see above) + * + * @param flush Flag indicating the flushing mode (true = flush on checkpoint) + */ + public void setFlushOnCheckpoint(boolean flush) { + final FlinkKafkaProducerBase internalProducer = (FlinkKafkaProducerBase) userFunction; + internalProducer.setFlushOnCheckpoint(flush); + } + + /** + * This method is used for approach (a) (see above) + * + */ + @Override + public void open(Configuration parameters) throws Exception { + final FlinkKafkaProducerBase internalProducer = (FlinkKafkaProducerBase) userFunction; + internalProducer.open(parameters); + } + + /** + * This method is used for approach (a) (see above) + */ + @Override + public IterationRuntimeContext getIterationRuntimeContext() { + final FlinkKafkaProducerBase internalProducer = (FlinkKafkaProducerBase) userFunction; + return internalProducer.getIterationRuntimeContext(); + } + + /** + * This method is used for approach (a) (see above) + */ + @Override + public void setRuntimeContext(RuntimeContext t) { + final FlinkKafkaProducerBase internalProducer = (FlinkKafkaProducerBase) userFunction; + internalProducer.setRuntimeContext(t); + } + + /** + * Invoke method for using the Sink as DataStream.addSink() sink. + * + * This method is used for approach (a) (see above) + * + * @param value The input record. + */ + @Override + public void invoke(T value) throws Exception { + invokeInternal(value, Long.MAX_VALUE); + } + + + // ----------------- Helper methods and classes implementing methods from StreamSink (Approach (b)) ---- + + + /** + * Process method for using the sink with timestamp support. + * + * This method is used for approach (b) (see above) + */ + @Override + public void processElement(StreamRecord element) throws Exception { + invokeInternal(element.getValue(), element.getTimestamp()); + } + /** * Configuration object returned by the writeToKafka() call. */ public static class FlinkKafkaProducer010Configuration extends DataStreamSink { - private final FlinkKafkaProducer09 wrapped09producer; + private final FlinkKafkaProducerBase wrappedProducerBase; private final FlinkKafkaProducer010 producer; private FlinkKafkaProducer010Configuration(DataStream stream, FlinkKafkaProducer010 producer) { + //noinspection unchecked super(stream, producer); this.producer = producer; - this.wrapped09producer = (FlinkKafkaProducer09) producer.userFunction; + this.wrappedProducerBase = (FlinkKafkaProducerBase) producer.userFunction; } /** @@ -124,7 +370,7 @@ private FlinkKafkaProducer010Configuration(DataStream stream, FlinkKafkaProducer * @param logFailuresOnly The flag to indicate logging-only on exceptions. */ public void setLogFailuresOnly(boolean logFailuresOnly) { - this.wrapped09producer.setLogFailuresOnly(logFailuresOnly); + this.wrappedProducerBase.setLogFailuresOnly(logFailuresOnly); } /** @@ -135,7 +381,7 @@ public void setLogFailuresOnly(boolean logFailuresOnly) { * @param flush Flag indicating the flushing mode (true = flush on checkpoint) */ public void setFlushOnCheckpoint(boolean flush) { - this.wrapped09producer.setFlushOnCheckpoint(flush); + this.wrappedProducerBase.setFlushOnCheckpoint(flush); } /** @@ -149,50 +395,5 @@ public void setWriteTimestampToKafka(boolean writeTimestampToKafka) { } } - /** - * Create internal Kafka producer, and pass it as a UDF to the StreamSink. - * - */ - private FlinkKafkaProducer010(String topicId, KeyedSerializationSchema serializationSchema, Properties producerConfig, KafkaPartitioner customPartitioner) { - super(new FlinkKafkaProducer09<>(topicId, serializationSchema, producerConfig, customPartitioner)); - } - - /** - * This method contains the timestamp specific operations - */ - @Override - public void processElement(StreamRecord element) throws Exception { - // usually, we would call: userFunction.invoke(element.getValue()); - - final FlinkKafkaProducer09 internalProducer = (FlinkKafkaProducer09) userFunction; - final T next = element.getValue(); - internalProducer.checkErroneous(); - - byte[] serializedKey = internalProducer.schema.serializeKey(next); - byte[] serializedValue = internalProducer.schema.serializeValue(next); - String targetTopic = internalProducer.schema.getTargetTopic(next); - if (targetTopic == null) { - targetTopic = internalProducer.defaultTopicId; - } - - Long timestamp = null; - if(this.writeTimestampToKafka) { - timestamp = element.getTimestamp(); - } - - ProducerRecord record; - if (internalProducer.partitioner == null) { - record = new ProducerRecord<>(targetTopic, null, timestamp, serializedKey, serializedValue); - } else { - record = new ProducerRecord<>(targetTopic, internalProducer.partitioner.partition(next, serializedKey, serializedValue, internalProducer.partitions.length), timestamp, serializedKey, serializedValue); - } - if (internalProducer.flushOnCheckpoint) { - synchronized (internalProducer.pendingRecordsLock) { - internalProducer.pendingRecords++; - } - } - internalProducer.producer.send(record, internalProducer.callback); - } - } diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java index b3a8d73bc4ce0..aac2915ebf958 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java @@ -93,12 +93,12 @@ public FlinkKafkaConsumerBase getConsumer(List topics, KeyedDeser @Override public DataStreamSink produceIntoKafka(DataStream stream, String topic, KeyedSerializationSchema serSchema, Properties props, KafkaPartitioner partitioner) { - /*FlinkKafkaProducer010 prod = new FlinkKafkaProducer010<>(topic, serSchema, props, partitioner); + FlinkKafkaProducer010 prod = new FlinkKafkaProducer010<>(topic, serSchema, props, partitioner); prod.setFlushOnCheckpoint(true); - return prod; */ - FlinkKafkaProducer010.FlinkKafkaProducer010Configuration sink = FlinkKafkaProducer010.writeToKafka(stream, topic, serSchema, props, partitioner); + return stream.addSink(prod); + /* FlinkKafkaProducer010.FlinkKafkaProducer010Configuration sink = FlinkKafkaProducer010.writeToKafka(stream, topic, serSchema, props, partitioner); sink.setFlushOnCheckpoint(true); - return sink; + return sink; */ } @Override From 4dd10799cba2cdda74fbe663db110c27cd23d05b Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Thu, 25 Aug 2016 10:18:19 +0200 Subject: [PATCH 5/9] Change AbstractFetcher methods --- .../kafka/internal/Kafka010Fetcher.java | 9 +++- .../kafka/internals/SimpleConsumerThread.java | 2 +- .../kafka/internal/Kafka09Fetcher.java | 4 ++ .../kafka/internals/AbstractFetcher.java | 8 +-- .../AbstractFetcherTimestampsTest.java | 52 +++++++++---------- 5 files changed, 42 insertions(+), 33 deletions(-) diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010Fetcher.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010Fetcher.java index 41c9ff637f412..cd930daf4256f 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010Fetcher.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010Fetcher.java @@ -60,12 +60,17 @@ protected void assignPartitionsToConsumer(KafkaConsumer consumer consumer.assign(topicPartitions); } + @Override + protected void emitRecord(T record, KafkaTopicPartitionState partition, long offset, ConsumerRecord consumerRecord) throws Exception { + // pass timestamp + super.emitRecord(record, partition, offset, consumerRecord.timestamp()); + } + /** * Emit record Kafka-timestamp aware. */ @Override - protected void emitRecord(T record, KafkaTopicPartitionState partitionState, long offset, R kafkaRecord) throws Exception { - long timestamp = ((ConsumerRecord) kafkaRecord).timestamp(); + protected void emitRecord(T record, KafkaTopicPartitionState partitionState, long offset, long timestamp) throws Exception { if (timestampWatermarkMode == NO_TIMESTAMPS_WATERMARKS) { // fast path logic, in case there are no watermarks diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SimpleConsumerThread.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SimpleConsumerThread.java index 7741c590f0a99..6d403f6adaf63 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SimpleConsumerThread.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SimpleConsumerThread.java @@ -376,7 +376,7 @@ else if (partitionsRemoved) { continue partitionsLoop; } - owner.emitRecord(value, currentPartition, offset, msg); + owner.emitRecord(value, currentPartition, offset, Long.MAX_VALUE); } else { // no longer running diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java index 40ff2ff86b8c1..c4d069f0d1016 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java @@ -262,6 +262,10 @@ public void run() { } } + // Kafka09Fetcher ignores the timestamp. + protected void emitRecord(T record, KafkaTopicPartitionState partition, long offset, ConsumerRecord consumerRecord) throws Exception { + emitRecord(record, partition, offset, Long.MAX_VALUE); + } /** * Protected method to make the partition assignment pluggable, for different Kafka versions. */ diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java index 136e94e2cbdf5..0a7ce7cf24c40 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java @@ -214,9 +214,9 @@ public void restoreOffsets(HashMap snapshotState) { * @param record The record to emit * @param partitionState The state of the Kafka partition from which the record was fetched * @param offset The offset of the record - * @param kafkaRecord The original Kafka record + * @param timestamp The record's event-timestamp */ - protected void emitRecord(T record, KafkaTopicPartitionState partitionState, long offset, R kafkaRecord) throws Exception { + protected void emitRecord(T record, KafkaTopicPartitionState partitionState, long offset, long timestamp) throws Exception { if (timestampWatermarkMode == NO_TIMESTAMPS_WATERMARKS) { // fast path logic, in case there are no watermarks @@ -228,10 +228,10 @@ protected void emitRecord(T record, KafkaTopicPartitionState partitionS } } else if (timestampWatermarkMode == PERIODIC_WATERMARKS) { - emitRecordWithTimestampAndPeriodicWatermark(record, partitionState, offset, Long.MIN_VALUE); + emitRecordWithTimestampAndPeriodicWatermark(record, partitionState, offset, timestamp); } else { - emitRecordWithTimestampAndPunctuatedWatermark(record, partitionState, offset, Long.MIN_VALUE); + emitRecordWithTimestampAndPunctuatedWatermark(record, partitionState, offset, timestamp); } } diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java index 10bfc66c6bb10..1294d7305aa46 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java @@ -63,22 +63,22 @@ public void testPunctuatedWatermarks() throws Exception { // elements generate a watermark if the timestamp is a multiple of three // elements for partition 1 - fetcher.emitRecord(1L, part1, 1L, new ConsumerRecord(testTopic, 7, new byte[]{0}, 1L)); - fetcher.emitRecord(2L, part1, 2L, new ConsumerRecord(testTopic, 7, new byte[]{0}, 2L)); - fetcher.emitRecord(3L, part1, 3L, new ConsumerRecord(testTopic, 7, new byte[]{0}, 3L)); + fetcher.emitRecord(1L, part1, 1L, Long.MAX_VALUE); + fetcher.emitRecord(2L, part1, 2L, Long.MAX_VALUE); + fetcher.emitRecord(3L, part1, 3L, Long.MAX_VALUE); assertEquals(3L, sourceContext.getLatestElement().getValue().longValue()); assertEquals(3L, sourceContext.getLatestElement().getTimestamp()); assertFalse(sourceContext.hasWatermark()); // elements for partition 2 - fetcher.emitRecord(12L, part2, 1L, new ConsumerRecord(testTopic, 13, new byte[]{0}, 1L)); + fetcher.emitRecord(12L, part2, 1L, Long.MAX_VALUE); assertEquals(12L, sourceContext.getLatestElement().getValue().longValue()); assertEquals(12L, sourceContext.getLatestElement().getTimestamp()); assertFalse(sourceContext.hasWatermark()); // elements for partition 3 - fetcher.emitRecord(101L, part3, 1L, new ConsumerRecord(testTopic, 21, new byte[]{0}, 1L)); - fetcher.emitRecord(102L, part3, 2L, new ConsumerRecord(testTopic, 21, new byte[]{0}, 2L)); + fetcher.emitRecord(101L, part3, 1L, Long.MAX_VALUE); + fetcher.emitRecord(102L, part3, 2L, Long.MAX_VALUE); assertEquals(102L, sourceContext.getLatestElement().getValue().longValue()); assertEquals(102L, sourceContext.getLatestElement().getTimestamp()); @@ -87,25 +87,25 @@ public void testPunctuatedWatermarks() throws Exception { assertEquals(3L, sourceContext.getLatestWatermark().getTimestamp()); // advance partition 3 - fetcher.emitRecord(1003L, part3, 3L, new ConsumerRecord(testTopic, 21, new byte[]{0}, 3L)); - fetcher.emitRecord(1004L, part3, 4L, new ConsumerRecord(testTopic, 21, new byte[]{0}, 4L)); - fetcher.emitRecord(1005L, part3, 5L, new ConsumerRecord(testTopic, 21, new byte[]{0}, 5L)); + fetcher.emitRecord(1003L, part3, 3L, Long.MAX_VALUE); + fetcher.emitRecord(1004L, part3, 4L, Long.MAX_VALUE); + fetcher.emitRecord(1005L, part3, 5L, Long.MAX_VALUE); assertEquals(1005L, sourceContext.getLatestElement().getValue().longValue()); assertEquals(1005L, sourceContext.getLatestElement().getTimestamp()); // advance partition 1 beyond partition 2 - this bumps the watermark - fetcher.emitRecord(30L, part1, 4L, new ConsumerRecord(testTopic, 7, new byte[]{0}, 4L)); + fetcher.emitRecord(30L, part1, 4L, Long.MAX_VALUE); assertEquals(30L, sourceContext.getLatestElement().getValue().longValue()); assertEquals(30L, sourceContext.getLatestElement().getTimestamp()); assertTrue(sourceContext.hasWatermark()); assertEquals(12L, sourceContext.getLatestWatermark().getTimestamp()); // advance partition 2 again - this bumps the watermark - fetcher.emitRecord(13L, part2, 2L, new ConsumerRecord(testTopic, 13, new byte[]{0}, 2L)); + fetcher.emitRecord(13L, part2, 2L, Long.MAX_VALUE); assertFalse(sourceContext.hasWatermark()); - fetcher.emitRecord(14L, part2, 3L, new ConsumerRecord(testTopic, 13, new byte[]{0}, 3L)); + fetcher.emitRecord(14L, part2, 3L, Long.MAX_VALUE); assertFalse(sourceContext.hasWatermark()); - fetcher.emitRecord(15L, part2, 3L, new ConsumerRecord(testTopic, 13, new byte[]{0}, 3L)); + fetcher.emitRecord(15L, part2, 3L, Long.MAX_VALUE); assertTrue(sourceContext.hasWatermark()); assertEquals(15L, sourceContext.getLatestWatermark().getTimestamp()); } @@ -135,20 +135,20 @@ public void testPeriodicWatermarks() throws Exception { // elements generate a watermark if the timestamp is a multiple of three // elements for partition 1 - fetcher.emitRecord(1L, part1, 1L, new ConsumerRecord(testTopic, 7, new byte[]{0}, 1L)); - fetcher.emitRecord(2L, part1, 2L, new ConsumerRecord(testTopic, 7, new byte[]{0}, 2L)); - fetcher.emitRecord(3L, part1, 3L, new ConsumerRecord(testTopic, 7, new byte[]{0}, 3L)); + fetcher.emitRecord(1L, part1, 1L, Long.MAX_VALUE); + fetcher.emitRecord(2L, part1, 2L, Long.MAX_VALUE); + fetcher.emitRecord(3L, part1, 3L, Long.MAX_VALUE); assertEquals(3L, sourceContext.getLatestElement().getValue().longValue()); assertEquals(3L, sourceContext.getLatestElement().getTimestamp()); // elements for partition 2 - fetcher.emitRecord(12L, part2, 1L, new ConsumerRecord(testTopic, 13, new byte[]{0}, 1L)); + fetcher.emitRecord(12L, part2, 1L, Long.MAX_VALUE); assertEquals(12L, sourceContext.getLatestElement().getValue().longValue()); assertEquals(12L, sourceContext.getLatestElement().getTimestamp()); // elements for partition 3 - fetcher.emitRecord(101L, part3, 1L, new ConsumerRecord(testTopic, 21, new byte[]{0}, 1L)); - fetcher.emitRecord(102L, part3, 2L, new ConsumerRecord(testTopic, 21, new byte[]{0}, 2L)); + fetcher.emitRecord(101L, part3, 1L, Long.MAX_VALUE); + fetcher.emitRecord(102L, part3, 2L, Long.MAX_VALUE); assertEquals(102L, sourceContext.getLatestElement().getValue().longValue()); assertEquals(102L, sourceContext.getLatestElement().getTimestamp()); @@ -156,14 +156,14 @@ public void testPeriodicWatermarks() throws Exception { assertEquals(3L, sourceContext.getLatestWatermark().getTimestamp()); // advance partition 3 - fetcher.emitRecord(1003L, part3, 3L, new ConsumerRecord(testTopic, 21, new byte[]{0}, 3L)); - fetcher.emitRecord(1004L, part3, 4L, new ConsumerRecord(testTopic, 21, new byte[]{0}, 4L)); - fetcher.emitRecord(1005L, part3, 5L, new ConsumerRecord(testTopic, 21, new byte[]{0}, 5L)); + fetcher.emitRecord(1003L, part3, 3L, Long.MAX_VALUE); + fetcher.emitRecord(1004L, part3, 4L, Long.MAX_VALUE); + fetcher.emitRecord(1005L, part3, 5L, Long.MAX_VALUE); assertEquals(1005L, sourceContext.getLatestElement().getValue().longValue()); assertEquals(1005L, sourceContext.getLatestElement().getTimestamp()); // advance partition 1 beyond partition 2 - this bumps the watermark - fetcher.emitRecord(30L, part1, 4L, new ConsumerRecord(testTopic, 7, new byte[]{0}, 4L)); + fetcher.emitRecord(30L, part1, 4L, Long.MAX_VALUE); assertEquals(30L, sourceContext.getLatestElement().getValue().longValue()); assertEquals(30L, sourceContext.getLatestElement().getTimestamp()); @@ -171,9 +171,9 @@ public void testPeriodicWatermarks() throws Exception { assertEquals(12L, sourceContext.getLatestWatermark().getTimestamp()); // advance partition 2 again - this bumps the watermark - fetcher.emitRecord(13L, part2, 2L, new ConsumerRecord(testTopic, 13, new byte[]{0}, 2L)); - fetcher.emitRecord(14L, part2, 3L, new ConsumerRecord(testTopic, 13, new byte[]{0}, 3L)); - fetcher.emitRecord(15L, part2, 3L, new ConsumerRecord(testTopic, 13, new byte[]{0}, 3L)); + fetcher.emitRecord(13L, part2, 2L, Long.MAX_VALUE); + fetcher.emitRecord(14L, part2, 3L, Long.MAX_VALUE); + fetcher.emitRecord(15L, part2, 3L, Long.MAX_VALUE); // this blocks until the periodic thread emitted the watermark long watermarkTs = sourceContext.getLatestWatermark().getTimestamp(); From 1624021b6179ea6fe991efeab0ac5a49f95fd5cb Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Mon, 26 Sep 2016 14:58:30 +0200 Subject: [PATCH 6/9] post rebase cleanups --- .../connectors/kafka/Kafka010ITCase.java | 10 --- .../kafka/KafkaTestEnvironmentImpl.java | 71 ++++++++++++++++--- .../kafka/KafkaConsumerTestBase.java | 26 +++---- .../kafka/KafkaShortRetentionTestBase.java | 1 - .../kafka/testutils/DataGenerators.java | 5 +- .../JobManagerCommunicationUtils.java | 6 +- 6 files changed, 80 insertions(+), 39 deletions(-) diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java index e221f745741c8..edb3a7f0ac7bc 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java @@ -26,28 +26,18 @@ import org.apache.flink.api.java.typeutils.TypeInfoParser; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataInputViewStreamWrapper; -import org.apache.flink.metrics.MetricGroup; import org.apache.flink.streaming.api.TimeCharacteristic; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.api.functions.source.SourceFunction; -import org.apache.flink.streaming.api.graph.StreamConfig; -import org.apache.flink.streaming.api.operators.ChainingStrategy; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; -import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.operators.StreamSink; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.tasks.StreamTask; -import org.apache.flink.streaming.runtime.tasks.StreamTaskState; -import org.apache.flink.streaming.util.serialization.DeserializationSchema; import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; -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.streaming.util.serialization.TypeInformationSerializationSchema; import org.junit.Test; diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java index aac2915ebf958..3d1c756a9924f 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java @@ -66,6 +66,9 @@ public class KafkaTestEnvironmentImpl extends KafkaTestEnvironment { private String brokerConnectionString = ""; private Properties standardProps; private Properties additionalServerProperties; + private boolean secureMode = false; + // 6 seconds is default. Seems to be too small for travis. 30 seconds + private int zkTimeout = 30000; public String getBrokerConnectionString() { return brokerConnectionString; @@ -76,6 +79,22 @@ public Properties getStandardProperties() { return standardProps; } + @Override + public Properties getSecureProperties() { + Properties prop = new Properties(); + if(secureMode) { + prop.put("security.inter.broker.protocol", "SASL_PLAINTEXT"); + prop.put("security.protocol", "SASL_PLAINTEXT"); + prop.put("sasl.kerberos.service.name", "kafka"); + + //add special timeout for Travis + prop.setProperty("zookeeper.session.timeout.ms", String.valueOf(zkTimeout)); + prop.setProperty("zookeeper.connection.timeout.ms", String.valueOf(zkTimeout)); + prop.setProperty("metadata.fetch.timeout.ms","120000"); + } + return prop; + } + @Override public String getVersion() { return "0.10"; @@ -135,8 +154,21 @@ public int getBrokerId(KafkaServer server) { } @Override - public void prepare(int numKafkaServers, Properties additionalServerProperties) { + public boolean isSecureRunSupported() { + return true; + } + + @Override + public void prepare(int numKafkaServers, Properties additionalServerProperties, boolean secureMode) { + //increase the timeout since in Travis ZK connection takes long time for secure connection. + if(secureMode) { + //run only one kafka server to avoid multiple ZK connections from many instances - Travis timeout + numKafkaServers = 1; + zkTimeout = zkTimeout * 15; + } + this.additionalServerProperties = additionalServerProperties; + this.secureMode = secureMode; File tempDir = new File(System.getProperty("java.io.tmpdir")); tmpZkDir = new File(tempDir, "kafkaITcase-zk-dir-" + (UUID.randomUUID().toString())); @@ -156,9 +188,9 @@ public void prepare(int numKafkaServers, Properties additionalServerProperties) brokers = null; try { - LOG.info("Starting Zookeeper"); - zookeeper = new TestingServer(-1, tmpZkDir); + zookeeper = new TestingServer(- 1, tmpZkDir); zookeeperConnectionString = zookeeper.getConnectString(); + LOG.info("Starting Zookeeper with zookeeperConnectionString: {}", zookeeperConnectionString); LOG.info("Starting KafkaServer"); brokers = new ArrayList<>(numKafkaServers); @@ -166,8 +198,11 @@ public void prepare(int numKafkaServers, Properties additionalServerProperties) for (int i = 0; i < numKafkaServers; i++) { brokers.add(getKafkaServer(i, tmpKafkaDirs.get(i))); - SocketServer socketServer = brokers.get(i).socketServer(); - brokerConnectionString += hostAndPortToUrlString(KafkaTestEnvironment.KAFKA_HOST, brokers.get(i).socketServer().boundPort(SecurityProtocol.PLAINTEXT)) + ","; + if(secureMode) { + brokerConnectionString += hostAndPortToUrlString(KafkaTestEnvironment.KAFKA_HOST, brokers.get(i).socketServer().boundPort(SecurityProtocol.SASL_PLAINTEXT)) + ","; + } else { + brokerConnectionString += hostAndPortToUrlString(KafkaTestEnvironment.KAFKA_HOST, brokers.get(i).socketServer().boundPort(SecurityProtocol.PLAINTEXT)) + ","; + } } LOG.info("ZK and KafkaServer started."); @@ -182,8 +217,8 @@ public void prepare(int numKafkaServers, Properties additionalServerProperties) standardProps.setProperty("bootstrap.servers", brokerConnectionString); standardProps.setProperty("group.id", "flink-tests"); standardProps.setProperty("auto.commit.enable", "false"); - standardProps.setProperty("zookeeper.session.timeout.ms", "30000"); // 6 seconds is default. Seems to be too small for travis. - standardProps.setProperty("zookeeper.connection.timeout.ms", "30000"); + standardProps.setProperty("zookeeper.session.timeout.ms", String.valueOf(zkTimeout)); + standardProps.setProperty("zookeeper.connection.timeout.ms", String.valueOf(zkTimeout)); standardProps.setProperty("auto.offset.reset", "earliest"); // read from the beginning. (earliest is kafka 0.10 value) standardProps.setProperty("fetch.message.max.bytes", "256"); // make a lot of fetches (MESSAGES MUST BE SMALLER!) } @@ -249,7 +284,14 @@ public void createTestTopic(String topic, int numberOfPartitions, int replicatio final long deadline = System.currentTimeMillis() + 30000; do { try { - Thread.sleep(100); + if(secureMode) { + //increase wait time since in Travis ZK timeout occurs frequently + int wait = zkTimeout / 100; + LOG.info("waiting for {} msecs before the topic {} can be checked", wait, topic); + Thread.sleep(wait); + } else { + Thread.sleep(100); + } } catch (InterruptedException e) { // restore interrupted state } @@ -300,8 +342,8 @@ protected KafkaServer getKafkaServer(int brokerId, File tmpFolder) throws Except kafkaProperties.put("replica.fetch.max.bytes", String.valueOf(50 * 1024 * 1024)); // for CI stability, increase zookeeper session timeout - kafkaProperties.put("zookeeper.session.timeout.ms", "30000"); - kafkaProperties.put("zookeeper.connection.timeout.ms", "30000"); + kafkaProperties.put("zookeeper.session.timeout.ms", zkTimeout); + kafkaProperties.put("zookeeper.connection.timeout.ms", zkTimeout); if(additionalServerProperties != null) { kafkaProperties.putAll(additionalServerProperties); } @@ -311,6 +353,15 @@ protected KafkaServer getKafkaServer(int brokerId, File tmpFolder) throws Except for (int i = 1; i <= numTries; i++) { int kafkaPort = NetUtils.getAvailablePort(); kafkaProperties.put("port", Integer.toString(kafkaPort)); + + //to support secure kafka cluster + if(secureMode) { + LOG.info("Adding Kafka secure configurations"); + kafkaProperties.put("listeners", "SASL_PLAINTEXT://" + KAFKA_HOST + ":" + kafkaPort); + kafkaProperties.put("advertised.listeners", "SASL_PLAINTEXT://" + KAFKA_HOST + ":" + kafkaPort); + kafkaProperties.putAll(getSecureProperties()); + } + KafkaConfig kafkaConfig = new KafkaConfig(kafkaProperties); try { diff --git a/flink-streaming-connectors/flink-connector-kafka-base/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 index 4fc01aa484ff8..1e318fb4ce6c9 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/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 @@ -31,7 +31,6 @@ 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.MapFunction; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.restartstrategy.RestartStrategies; @@ -68,7 +67,6 @@ import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; import org.apache.flink.streaming.api.functions.source.RichSourceFunction; import org.apache.flink.streaming.api.functions.source.SourceFunction; -import org.apache.flink.streaming.api.functions.source.StatefulSequenceSource; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.watermark.Watermark; @@ -161,7 +159,6 @@ public void ensureNoJobIsLingering() throws Exception { * Test that ensures the KafkaConsumer is properly failing if the topic doesnt exist * and a wrong broker was specified * - * @throws Exception */ public void runFailOnNoBrokerTest() throws Exception { try { @@ -277,8 +274,7 @@ public void cancel() { Properties producerProperties = FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings); producerProperties.setProperty("retries", "3"); producerProperties.putAll(secureProps); - FlinkKafkaProducerBase> prod = kafkaServer.getProducer(topic, new KeyedSerializationSchemaWrapper<>(sinkSchema), producerProperties, null); - stream.addSink(prod); + kafkaServer.produceIntoKafka(stream, topic, new KeyedSerializationSchemaWrapper<>(sinkSchema), producerProperties, null); // ----------- add consumer dataflow ---------- @@ -502,7 +498,7 @@ public void runCancelingOnFullInputTest() throws Exception { // launch a producer thread DataGenerators.InfiniteStringsGenerator generator = - new DataGenerators.InfiniteStringsGenerator(kafkaServer, topic); + new DataGenerators.InfiniteStringsGenerator(kafkaServer, topic, flinkPort); generator.start(); // launch a consumer asynchronously @@ -525,7 +521,7 @@ public void run() { env.addSource(source).addSink(new DiscardingSink()); - env.execute(); + env.execute("Runner for CancelingOnFullInputTest"); } catch (Throwable t) { jobError.set(t); @@ -546,7 +542,7 @@ public void run() { } // cancel - JobManagerCommunicationUtils.cancelCurrentJob(flink.getLeaderGateway(timeout)); + JobManagerCommunicationUtils.cancelCurrentJob(flink.getLeaderGateway(timeout), "Runner for CancelingOnFullInputTest"); // wait for the program to be done and validate that we failed with the right exception runnerThread.join(); @@ -600,7 +596,7 @@ public void run() { env.addSource(source).addSink(new DiscardingSink()); - env.execute(); + env.execute("CancelingOnEmptyInputTest"); } catch (Throwable t) { LOG.error("Job Runner failed with exception", t); @@ -658,7 +654,7 @@ public void runFailOnDeployTest() throws Exception { .addSink(new DiscardingSink()); try { - env.execute(); + env.execute("test fail on deploy"); fail("this test should fail with an exception"); } catch (ProgramInvocationException e) { @@ -725,7 +721,7 @@ public void cancel() { Properties props = new Properties(); props.putAll(standardProps); props.putAll(secureProps); - stream.addSink(kafkaServer.getProducer("dummy", schema, props, null)); + kafkaServer.produceIntoKafka(stream, "dummy", schema, props, null); env.execute("Write to topics"); @@ -832,10 +828,10 @@ public void run(SourceContext ctx) throws Exception { public void cancel() { } }); - kafkaServer.produceIntoKafka(stream,topic, new ByteArraySerializationSchema(), props, null); + kafkaServer.produceIntoKafka(stream, topic, new ByteArraySerializationSchema(), props, null); // Execute blocks - env.execute(); + env.execute("runJsonTableSource: data gen"); // Register as table source StreamTableEnvironment tableEnvironment = StreamTableEnvironment.getTableEnvironment(env); @@ -1304,7 +1300,7 @@ public void run() { env1.disableOperatorChaining(); // let the source read everything into the network buffers TypeInformationSerializationSchema> schema = new TypeInformationSerializationSchema<>(TypeInfoParser.>parse("Tuple2"), env1.getConfig()); - DataStream> fromKafka = env1.addSource(kafkaServer.getConsumer(topic, schema, props)); + DataStream> fromKafka = env1.addSource(kafkaServer.getConsumer(topic, schema, standardProps)); fromKafka.flatMap(new FlatMapFunction, Void>() { @Override public void flatMap(Tuple2 value, Collector out) throws Exception {// no op @@ -1329,7 +1325,7 @@ public void cancel() { } }); - kafkaServer.produceIntoKafka(fromGen, topic, new KeyedSerializationSchemaWrapper<>(schema), props, null); + kafkaServer.produceIntoKafka(fromGen, topic, new KeyedSerializationSchemaWrapper<>(schema), standardProps, null); env1.execute("Metrics test job"); } catch(Throwable t) { diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java index f735cece6891e..5c03b78efb1d2 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java @@ -31,7 +31,6 @@ import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper; import org.apache.flink.streaming.util.serialization.SimpleStringSchema; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; import org.apache.flink.util.InstantiationUtil; import org.junit.AfterClass; diff --git a/flink-streaming-connectors/flink-connector-kafka-base/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 index 0f54409e8d38a..3f035fdc70f8e 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/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 @@ -23,10 +23,10 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TypeInfoParser; -import org.apache.flink.configuration.Configuration; 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.api.functions.source.SourceFunction; 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; @@ -35,6 +35,7 @@ import org.apache.flink.streaming.util.serialization.SimpleStringSchema; import org.apache.flink.streaming.util.serialization.TypeInformationSerializationSchema; +import java.io.Serializable; import java.util.Properties; import java.util.Random; @@ -158,7 +159,7 @@ public int partition(Integer next, byte[] serializedKey, byte[] serializedValue, // ------------------------------------------------------------------------ - public static class InfiniteStringsGenerator extends Thread implements Serializable{ + public static class InfiniteStringsGenerator extends Thread implements Serializable { private transient KafkaTestEnvironment server; diff --git a/flink-streaming-connectors/flink-connector-kafka-base/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 index 143095663fcd2..acdad5a2613bd 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/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 @@ -92,7 +92,11 @@ else if(name != null) { throw new Exception("Could not cancel job - no job matched expected name = '" + name +"' in " + jobs); } } else { - throw new Exception("Could not cancel job - more than one running job."); + String jobNames = ""; + for(JobStatusMessage jsm: jobs) { + jobNames += jsm.getJobName() + ", "; + } + throw new Exception("Could not cancel job - more than one running job: " + jobNames); } } From 488525ca0ef285dc95e89f8b75efcade8dc64b73 Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Mon, 26 Sep 2016 16:23:01 +0200 Subject: [PATCH 7/9] fix tests? --- .../connectors/kafka/KafkaProducerTest.java | 137 ------------------ .../kafka/KafkaConsumerTestBase.java | 6 +- 2 files changed, 5 insertions(+), 138 deletions(-) delete mode 100644 flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java deleted file mode 100644 index 9b46ab60596c8..0000000000000 --- a/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java +++ /dev/null @@ -1,137 +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.api.common.functions.RichFunction; -import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; -import org.apache.flink.streaming.connectors.kafka.testutils.MockRuntimeContext; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema; -import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper; -import org.apache.flink.streaming.util.serialization.SimpleStringSchema; -import org.apache.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.lang.reflect.Constructor; -import java.lang.reflect.InvocationTargetException; -import java.util.Collections; -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( - Collections.singletonList(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 - - - FlinkKafkaProducer010 producerPropagating = newProducer(); - - ((RichFunction)producerPropagating.getUserFunction()).setRuntimeContext(new MockRuntimeContext(17, 3)); - producerPropagating.open(); - - try { - producerPropagating.processElement(new StreamRecord<>("value", 1L)); - producerPropagating.processElement(new StreamRecord<>("value", 1L)); - fail("This should fail with an exception"); - } - catch (Exception e) { - assertNotNull(e.getCause()); - assertNotNull(e.getCause().getMessage()); - assertTrue(e.getCause().getMessage().contains("Test error")); - } - // avoid reuse - producerPropagating = null; - - // (2) producer that only logs errors - - FlinkKafkaProducer010 producerLogging = newProducer(); - ((FlinkKafkaProducer09)producerLogging.getUserFunction()).setLogFailuresOnly(true); - - ((RichFunction)producerLogging.getUserFunction()).setRuntimeContext(new MockRuntimeContext(17, 3)); - producerLogging.open(); - - producerLogging.processElement(new StreamRecord<>("value", 1L)); - producerLogging.processElement(new StreamRecord<>("value", 1L)); - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - private static FlinkKafkaProducer010 newProducer() { - try { - Class prodClass = FlinkKafkaProducer010.class; - Constructor ctor = prodClass.getDeclaredConstructor(String.class, KeyedSerializationSchema.class, Properties.class, KafkaPartitioner.class); - ctor.setAccessible(true); - return ctor.newInstance("mock_topic", new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()), new Properties(), null); - } catch (InstantiationException | IllegalAccessException | InvocationTargetException | NoSuchMethodException e) { - throw new RuntimeException("Error while creating producer", e); - } - } -} diff --git a/flink-streaming-connectors/flink-connector-kafka-base/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 index 1e318fb4ce6c9..a8bced97908b7 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/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 @@ -281,7 +281,10 @@ public void cancel() { List topics = new ArrayList<>(); topics.add(topic); topics.add(additionalEmptyTopic); - FlinkKafkaConsumerBase> source = kafkaServer.getConsumer(topics, sourceSchema, standardProps); + Properties props = new Properties(); + props.putAll(standardProps); + props.putAll(secureProps); + FlinkKafkaConsumerBase> source = kafkaServer.getConsumer(topics, sourceSchema, props); DataStreamSource> consuming = env.addSource(source).setParallelism(parallelism); @@ -1441,6 +1444,7 @@ protected void readSequence(StreamExecutionEnvironment env, Properties cc, new TypeInformationSerializationSchema<>(intIntTupleType, env.getConfig()); // create the consumer + cc.putAll(secureProps); FlinkKafkaConsumerBase> consumer = kafkaServer.getConsumer(topicName, deser, cc); DataStream> source = env From 29610a6992ea5575ba02002315ced3f454d22bb8 Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Mon, 26 Sep 2016 17:44:57 +0200 Subject: [PATCH 8/9] address comments --- docs/dev/connectors/kafka.md | 6 +-- .../kafka/FlinkKafkaProducer010.java | 37 +++++++------ .../kafka/internal/Kafka010Fetcher.java | 2 +- .../connectors/kafka/Kafka010ITCase.java | 2 +- .../kafka/KafkaTestEnvironmentImpl.java | 2 +- .../kafka/internals/SimpleConsumerThread.java | 2 +- .../kafka/internal/Kafka09Fetcher.java | 4 +- .../kafka/internals/AbstractFetcher.java | 8 ++- .../AbstractFetcherTimestampsTest.java | 52 +++++++++---------- 9 files changed, 60 insertions(+), 55 deletions(-) diff --git a/docs/dev/connectors/kafka.md b/docs/dev/connectors/kafka.md index 6f72cf450cf6a..9a360d47bfb85 100644 --- a/docs/dev/connectors/kafka.md +++ b/docs/dev/connectors/kafka.md @@ -264,7 +264,7 @@ stream.addSink(new FlinkKafkaProducer08("localhost:9092", "my-topic", ne
{% highlight java %} -FlinkKafkaProducer010.writeToKafka(stream, "my-topic", new SimpleStringSchema(), properties); +FlinkKafkaProducer010.writeToKafkaWithTimestamps(stream, "my-topic", new SimpleStringSchema(), properties); {% endhighlight %}
@@ -274,7 +274,7 @@ stream.addSink(new FlinkKafkaProducer08[String]("localhost:9092", "my-topic", ne
{% highlight scala %} -FlinkKafkaProducer010.writeToKafka(stream, "my-topic", new SimpleStringSchema(), properties); +FlinkKafkaProducer010.writeToKafkaWithTimestamps(stream, "my-topic", new SimpleStringSchema(), properties); {% endhighlight %}
@@ -325,7 +325,7 @@ public long extractTimestamp(Long element, long previousElementTimestamp) { The `FlinkKafkaProducer010` only emits the record timestamp, if `setWriteTimestampToKafka(true)` is set. {% highlight java %} -FlinkKafkaProducer010.FlinkKafkaProducer010Configuration config = FlinkKafkaProducer010.writeToKafka(streamWithTimestamps, topic, new SimpleStringSchema(), standardProps); +FlinkKafkaProducer010.FlinkKafkaProducer010Configuration config = FlinkKafkaProducer010.writeToKafkaWithTimestamps(streamWithTimestamps, topic, new SimpleStringSchema(), standardProps); config.setWriteTimestampToKafka(true); {% endhighlight %} diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java index 9bb18a92b9a7b..4a346bfb1a23c 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java @@ -54,11 +54,11 @@ * Pre Kafka 0.10 producers only follow approach (a), allowing users to use the producer using the * DataStream.addSink() method. * Since the APIs exposed in that variant do not allow accessing the the timestamp attached to the record - * the Kafka 0.10 producer has a section invocation option, approach (b). + * the Kafka 0.10 producer has a second invocation option, approach (b). * * Details about approach (b): * Kafka 0.10 supports writing the timestamp attached to a record to Kafka. When adding the - * FlinkKafkaProducer010 using the FlinkKafkaProducer010.writeToKafka() method, the Kafka producer + * FlinkKafkaProducer010 using the FlinkKafkaProducer010.writeToKafkaWithTimestamps() method, the Kafka producer * can access the internal record timestamp of the record and write it to Kafka. * * All methods and constructors in this class are marked with the approach they are needed for. @@ -83,11 +83,11 @@ public class FlinkKafkaProducer010 extends StreamSink implements SinkFunct * @param serializationSchema User defined serialization schema supporting key/value messages * @param producerConfig Properties with the producer configuration. */ - public static FlinkKafkaProducer010Configuration writeToKafka(DataStream inStream, - String topicId, - KeyedSerializationSchema serializationSchema, - Properties producerConfig) { - return writeToKafka(inStream, topicId, serializationSchema, producerConfig, new FixedPartitioner()); + public static FlinkKafkaProducer010Configuration writeToKafkaWithTimestamps(DataStream inStream, + String topicId, + KeyedSerializationSchema serializationSchema, + Properties producerConfig) { + return writeToKafkaWithTimestamps(inStream, topicId, serializationSchema, producerConfig, new FixedPartitioner()); } @@ -102,11 +102,11 @@ public static FlinkKafkaProducer010Configuration writeToKafka(DataStream * @param serializationSchema User defined (keyless) serialization schema. * @param producerConfig Properties with the producer configuration. */ - public static FlinkKafkaProducer010Configuration writeToKafka(DataStream inStream, - String topicId, - SerializationSchema serializationSchema, - Properties producerConfig) { - return writeToKafka(inStream, topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FixedPartitioner()); + public static FlinkKafkaProducer010Configuration writeToKafkaWithTimestamps(DataStream inStream, + String topicId, + SerializationSchema serializationSchema, + Properties producerConfig) { + return writeToKafkaWithTimestamps(inStream, topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FixedPartitioner()); } /** @@ -121,11 +121,11 @@ public static FlinkKafkaProducer010Configuration writeToKafka(DataStream * @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 static FlinkKafkaProducer010Configuration writeToKafka(DataStream inStream, - String topicId, - KeyedSerializationSchema serializationSchema, - Properties producerConfig, - KafkaPartitioner customPartitioner) { + public static FlinkKafkaProducer010Configuration writeToKafkaWithTimestamps(DataStream inStream, + String topicId, + KeyedSerializationSchema serializationSchema, + Properties producerConfig, + KafkaPartitioner customPartitioner) { GenericTypeInfo objectTypeInfo = new GenericTypeInfo<>(Object.class); FlinkKafkaProducer010 kafkaProducer = new FlinkKafkaProducer010<>(topicId, serializationSchema, producerConfig, customPartitioner); @@ -176,7 +176,6 @@ public FlinkKafkaProducer010(String topicId, SerializationSchema serializatio */ public FlinkKafkaProducer010(String topicId, SerializationSchema serializationSchema, Properties producerConfig, KafkaPartitioner customPartitioner) { this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner); - } // ------------------- Key/Value serialization schema constructors ---------------------- @@ -347,7 +346,7 @@ public void processElement(StreamRecord element) throws Exception { } /** - * Configuration object returned by the writeToKafka() call. + * Configuration object returned by the writeToKafkaWithTimestamps() call. */ public static class FlinkKafkaProducer010Configuration extends DataStreamSink { diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010Fetcher.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010Fetcher.java index cd930daf4256f..47bee2209c63f 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010Fetcher.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010Fetcher.java @@ -62,7 +62,7 @@ protected void assignPartitionsToConsumer(KafkaConsumer consumer @Override protected void emitRecord(T record, KafkaTopicPartitionState partition, long offset, ConsumerRecord consumerRecord) throws Exception { - // pass timestamp + // get timestamp from provided ConsumerRecord (only possible with kafka 0.10.x) super.emitRecord(record, partition, offset, consumerRecord.timestamp()); } diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java index edb3a7f0ac7bc..005ffa8ef2239 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java @@ -246,7 +246,7 @@ public void cancel() { }); final TypeInformationSerializationSchema longSer = new TypeInformationSerializationSchema<>(TypeInfoParser.parse("Long"), env.getConfig()); - FlinkKafkaProducer010.FlinkKafkaProducer010Configuration prod = FlinkKafkaProducer010.writeToKafka(streamWithTimestamps, topic, new KeyedSerializationSchemaWrapper<>(longSer), standardProps, new KafkaPartitioner() { + FlinkKafkaProducer010.FlinkKafkaProducer010Configuration prod = FlinkKafkaProducer010.writeToKafkaWithTimestamps(streamWithTimestamps, topic, new KeyedSerializationSchemaWrapper<>(longSer), standardProps, new KafkaPartitioner() { @Override public int partition(Long next, byte[] serializedKey, byte[] serializedValue, int numPartitions) { return (int)(next % 3); diff --git a/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java b/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java index 3d1c756a9924f..af6d254a4f161 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java @@ -115,7 +115,7 @@ public DataStreamSink produceIntoKafka(DataStream stream, String topic FlinkKafkaProducer010 prod = new FlinkKafkaProducer010<>(topic, serSchema, props, partitioner); prod.setFlushOnCheckpoint(true); return stream.addSink(prod); - /* FlinkKafkaProducer010.FlinkKafkaProducer010Configuration sink = FlinkKafkaProducer010.writeToKafka(stream, topic, serSchema, props, partitioner); + /* FlinkKafkaProducer010.FlinkKafkaProducer010Configuration sink = FlinkKafkaProducer010.writeToKafkaWithTimestamps(stream, topic, serSchema, props, partitioner); sink.setFlushOnCheckpoint(true); return sink; */ } diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SimpleConsumerThread.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SimpleConsumerThread.java index 6d403f6adaf63..130234858339c 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SimpleConsumerThread.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SimpleConsumerThread.java @@ -376,7 +376,7 @@ else if (partitionsRemoved) { continue partitionsLoop; } - owner.emitRecord(value, currentPartition, offset, Long.MAX_VALUE); + owner.emitRecord(value, currentPartition, offset, Long.MIN_VALUE); } else { // no longer running diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java index c4d069f0d1016..ec9d3f9973851 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java @@ -262,9 +262,9 @@ public void run() { } } - // Kafka09Fetcher ignores the timestamp. + // Kafka09Fetcher ignores the timestamp, Kafka010Fetcher is extracting the timestamp and passing it to the emitRecord() method. protected void emitRecord(T record, KafkaTopicPartitionState partition, long offset, ConsumerRecord consumerRecord) throws Exception { - emitRecord(record, partition, offset, Long.MAX_VALUE); + emitRecord(record, partition, offset, Long.MIN_VALUE); } /** * Protected method to make the partition assignment pluggable, for different Kafka versions. diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java index 0a7ce7cf24c40..3df87de420e67 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java @@ -34,6 +34,7 @@ import java.util.List; import java.util.Map; +import static org.apache.flink.util.Preconditions.checkElementIndex; import static org.apache.flink.util.Preconditions.checkNotNull; /** @@ -223,7 +224,12 @@ protected void emitRecord(T record, KafkaTopicPartitionState partitionState // emit the record, using the checkpoint lock to guarantee // atomicity of record emission and offset state update synchronized (checkpointLock) { - sourceContext.collect(record); + if(timestamp != Long.MIN_VALUE) { + // this case is true for Kafka 0.10 + sourceContext.collectWithTimestamp(record, timestamp); + } else { + sourceContext.collect(record); + } partitionState.setOffset(offset); } } diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java index 1294d7305aa46..3a8addf4cf278 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java @@ -63,22 +63,22 @@ public void testPunctuatedWatermarks() throws Exception { // elements generate a watermark if the timestamp is a multiple of three // elements for partition 1 - fetcher.emitRecord(1L, part1, 1L, Long.MAX_VALUE); - fetcher.emitRecord(2L, part1, 2L, Long.MAX_VALUE); - fetcher.emitRecord(3L, part1, 3L, Long.MAX_VALUE); + fetcher.emitRecord(1L, part1, 1L, Long.MIN_VALUE); + fetcher.emitRecord(2L, part1, 2L, Long.MIN_VALUE); + fetcher.emitRecord(3L, part1, 3L, Long.MIN_VALUE); assertEquals(3L, sourceContext.getLatestElement().getValue().longValue()); assertEquals(3L, sourceContext.getLatestElement().getTimestamp()); assertFalse(sourceContext.hasWatermark()); // elements for partition 2 - fetcher.emitRecord(12L, part2, 1L, Long.MAX_VALUE); + fetcher.emitRecord(12L, part2, 1L, Long.MIN_VALUE); assertEquals(12L, sourceContext.getLatestElement().getValue().longValue()); assertEquals(12L, sourceContext.getLatestElement().getTimestamp()); assertFalse(sourceContext.hasWatermark()); // elements for partition 3 - fetcher.emitRecord(101L, part3, 1L, Long.MAX_VALUE); - fetcher.emitRecord(102L, part3, 2L, Long.MAX_VALUE); + fetcher.emitRecord(101L, part3, 1L, Long.MIN_VALUE); + fetcher.emitRecord(102L, part3, 2L, Long.MIN_VALUE); assertEquals(102L, sourceContext.getLatestElement().getValue().longValue()); assertEquals(102L, sourceContext.getLatestElement().getTimestamp()); @@ -87,25 +87,25 @@ public void testPunctuatedWatermarks() throws Exception { assertEquals(3L, sourceContext.getLatestWatermark().getTimestamp()); // advance partition 3 - fetcher.emitRecord(1003L, part3, 3L, Long.MAX_VALUE); - fetcher.emitRecord(1004L, part3, 4L, Long.MAX_VALUE); - fetcher.emitRecord(1005L, part3, 5L, Long.MAX_VALUE); + fetcher.emitRecord(1003L, part3, 3L, Long.MIN_VALUE); + fetcher.emitRecord(1004L, part3, 4L, Long.MIN_VALUE); + fetcher.emitRecord(1005L, part3, 5L, Long.MIN_VALUE); assertEquals(1005L, sourceContext.getLatestElement().getValue().longValue()); assertEquals(1005L, sourceContext.getLatestElement().getTimestamp()); // advance partition 1 beyond partition 2 - this bumps the watermark - fetcher.emitRecord(30L, part1, 4L, Long.MAX_VALUE); + fetcher.emitRecord(30L, part1, 4L, Long.MIN_VALUE); assertEquals(30L, sourceContext.getLatestElement().getValue().longValue()); assertEquals(30L, sourceContext.getLatestElement().getTimestamp()); assertTrue(sourceContext.hasWatermark()); assertEquals(12L, sourceContext.getLatestWatermark().getTimestamp()); // advance partition 2 again - this bumps the watermark - fetcher.emitRecord(13L, part2, 2L, Long.MAX_VALUE); + fetcher.emitRecord(13L, part2, 2L, Long.MIN_VALUE); assertFalse(sourceContext.hasWatermark()); - fetcher.emitRecord(14L, part2, 3L, Long.MAX_VALUE); + fetcher.emitRecord(14L, part2, 3L, Long.MIN_VALUE); assertFalse(sourceContext.hasWatermark()); - fetcher.emitRecord(15L, part2, 3L, Long.MAX_VALUE); + fetcher.emitRecord(15L, part2, 3L, Long.MIN_VALUE); assertTrue(sourceContext.hasWatermark()); assertEquals(15L, sourceContext.getLatestWatermark().getTimestamp()); } @@ -135,20 +135,20 @@ public void testPeriodicWatermarks() throws Exception { // elements generate a watermark if the timestamp is a multiple of three // elements for partition 1 - fetcher.emitRecord(1L, part1, 1L, Long.MAX_VALUE); - fetcher.emitRecord(2L, part1, 2L, Long.MAX_VALUE); - fetcher.emitRecord(3L, part1, 3L, Long.MAX_VALUE); + fetcher.emitRecord(1L, part1, 1L, Long.MIN_VALUE); + fetcher.emitRecord(2L, part1, 2L, Long.MIN_VALUE); + fetcher.emitRecord(3L, part1, 3L, Long.MIN_VALUE); assertEquals(3L, sourceContext.getLatestElement().getValue().longValue()); assertEquals(3L, sourceContext.getLatestElement().getTimestamp()); // elements for partition 2 - fetcher.emitRecord(12L, part2, 1L, Long.MAX_VALUE); + fetcher.emitRecord(12L, part2, 1L, Long.MIN_VALUE); assertEquals(12L, sourceContext.getLatestElement().getValue().longValue()); assertEquals(12L, sourceContext.getLatestElement().getTimestamp()); // elements for partition 3 - fetcher.emitRecord(101L, part3, 1L, Long.MAX_VALUE); - fetcher.emitRecord(102L, part3, 2L, Long.MAX_VALUE); + fetcher.emitRecord(101L, part3, 1L, Long.MIN_VALUE); + fetcher.emitRecord(102L, part3, 2L, Long.MIN_VALUE); assertEquals(102L, sourceContext.getLatestElement().getValue().longValue()); assertEquals(102L, sourceContext.getLatestElement().getTimestamp()); @@ -156,14 +156,14 @@ public void testPeriodicWatermarks() throws Exception { assertEquals(3L, sourceContext.getLatestWatermark().getTimestamp()); // advance partition 3 - fetcher.emitRecord(1003L, part3, 3L, Long.MAX_VALUE); - fetcher.emitRecord(1004L, part3, 4L, Long.MAX_VALUE); - fetcher.emitRecord(1005L, part3, 5L, Long.MAX_VALUE); + fetcher.emitRecord(1003L, part3, 3L, Long.MIN_VALUE); + fetcher.emitRecord(1004L, part3, 4L, Long.MIN_VALUE); + fetcher.emitRecord(1005L, part3, 5L, Long.MIN_VALUE); assertEquals(1005L, sourceContext.getLatestElement().getValue().longValue()); assertEquals(1005L, sourceContext.getLatestElement().getTimestamp()); // advance partition 1 beyond partition 2 - this bumps the watermark - fetcher.emitRecord(30L, part1, 4L, Long.MAX_VALUE); + fetcher.emitRecord(30L, part1, 4L, Long.MIN_VALUE); assertEquals(30L, sourceContext.getLatestElement().getValue().longValue()); assertEquals(30L, sourceContext.getLatestElement().getTimestamp()); @@ -171,9 +171,9 @@ public void testPeriodicWatermarks() throws Exception { assertEquals(12L, sourceContext.getLatestWatermark().getTimestamp()); // advance partition 2 again - this bumps the watermark - fetcher.emitRecord(13L, part2, 2L, Long.MAX_VALUE); - fetcher.emitRecord(14L, part2, 3L, Long.MAX_VALUE); - fetcher.emitRecord(15L, part2, 3L, Long.MAX_VALUE); + fetcher.emitRecord(13L, part2, 2L, Long.MIN_VALUE); + fetcher.emitRecord(14L, part2, 3L, Long.MIN_VALUE); + fetcher.emitRecord(15L, part2, 3L, Long.MIN_VALUE); // this blocks until the periodic thread emitted the watermark long watermarkTs = sourceContext.getLatestWatermark().getTimestamp(); From ba39fc623e94946550f6a5dbcaaee1360a6534ec Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Mon, 26 Sep 2016 21:00:03 +0200 Subject: [PATCH 9/9] unused import --- .../streaming/connectors/kafka/internals/AbstractFetcher.java | 1 - .../kafka/internals/AbstractFetcherTimestampsTest.java | 1 - 2 files changed, 2 deletions(-) diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java index 3df87de420e67..b08364f66618c 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java @@ -34,7 +34,6 @@ import java.util.List; import java.util.Map; -import static org.apache.flink.util.Preconditions.checkElementIndex; import static org.apache.flink.util.Preconditions.checkNotNull; /** diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java index 3a8addf4cf278..2c89f2b5407a2 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java @@ -28,7 +28,6 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.util.SerializedValue; -import org.apache.kafka.clients.consumer.ConsumerRecord; import org.junit.Test; import javax.annotation.Nullable;