From 67c3b872cc0129b2f87b565e9fb92908926eac64 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Thu, 20 Apr 2017 17:17:43 +0800 Subject: [PATCH 1/2] [FLINK-4022] [kafka] Partition / topic discovery for FlinkKafkaConsumer --- .../kafka/FlinkKafkaConsumer010.java | 12 + .../internal/Kafka010PartitionDiscoverer.java | 38 + .../{ => internal}/Kafka010FetcherTest.java | 5 +- .../kafka/FlinkKafkaConsumer08.java | 173 +--- .../kafka/internals/Kafka08Fetcher.java | 29 +- .../internals/Kafka08PartitionDiscoverer.java | 290 ++++++ .../kafka/internals/PartitionInfoFetcher.java | 20 +- .../internals/PeriodicOffsetCommitter.java | 9 +- .../connectors/kafka/KafkaConsumer08Test.java | 58 +- .../kafka/KafkaTestEnvironmentImpl.java | 15 +- .../kafka/FlinkKafkaConsumer09.java | 59 +- .../kafka/internal/Kafka09Fetcher.java | 10 +- .../internal/Kafka09PartitionDiscoverer.java | 102 +++ .../kafka/internal/KafkaConsumerThread.java | 232 ++++- .../{ => internal}/Kafka09FetcherTest.java | 5 +- .../internal/KafkaConsumerThreadTest.java | 842 ++++++++++++++++++ .../kafka/FlinkKafkaConsumerBase.java | 375 ++++---- .../kafka/internals/AbstractFetcher.java | 255 ++++-- .../AbstractPartitionDiscoverer.java | 256 ++++++ .../internals/ClosableBlockingQueue.java | 0 .../internals/KafkaTopicPartitionState.java | 2 +- .../internals/KafkaTopicsDescriptor.java | 73 ++ .../FlinkKafkaConsumerBaseMigrationTest.java | 20 +- .../kafka/FlinkKafkaConsumerBaseTest.java | 39 +- .../KafkaConsumerPartitionAssignmentTest.java | 356 -------- .../kafka/KafkaConsumerTestBase.java | 7 +- .../kafka/internals/AbstractFetcherTest.java | 18 +- .../AbstractPartitionDiscovererTest.java | 430 +++++++++ .../internals/ClosableBlockingQueueTest.java | 0 29 files changed, 2750 insertions(+), 980 deletions(-) create mode 100644 flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010PartitionDiscoverer.java rename flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/{ => internal}/Kafka010FetcherTest.java (98%) create mode 100644 flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Kafka08PartitionDiscoverer.java create mode 100644 flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09PartitionDiscoverer.java rename flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/{ => internal}/Kafka09FetcherTest.java (98%) create mode 100644 flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerThreadTest.java create mode 100644 flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscoverer.java rename flink-connectors/{flink-connector-kafka-0.8 => flink-connector-kafka-base}/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueue.java (100%) create mode 100644 flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicsDescriptor.java delete mode 100644 flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerPartitionAssignmentTest.java create mode 100644 flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscovererTest.java rename flink-connectors/{flink-connector-kafka-0.8 => flink-connector-kafka-base}/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueueTest.java (100%) diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java index 9e06d6e33b0e6..dbd9ae1dbbd82 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java @@ -22,8 +22,11 @@ import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; import org.apache.flink.streaming.connectors.kafka.internal.Kafka010Fetcher; +import org.apache.flink.streaming.connectors.kafka.internal.Kafka010PartitionDiscoverer; import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractPartitionDiscoverer; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicsDescriptor; import org.apache.flink.streaming.util.serialization.DeserializationSchema; import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper; @@ -153,4 +156,13 @@ public FlinkKafkaConsumer010(List topics, KeyedDeserializationSchema pollTimeout, useMetrics); } + + @Override + protected AbstractPartitionDiscoverer createPartitionDiscoverer( + KafkaTopicsDescriptor topicsDescriptor, + int indexOfThisSubtask, + int numParallelSubtasks) { + + return new Kafka010PartitionDiscoverer(topicsDescriptor, indexOfThisSubtask, numParallelSubtasks, properties); + } } diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010PartitionDiscoverer.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010PartitionDiscoverer.java new file mode 100644 index 0000000000000..0c10f40406660 --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010PartitionDiscoverer.java @@ -0,0 +1,38 @@ +/* + * 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.streaming.connectors.kafka.internals.KafkaTopicsDescriptor; + +import java.util.Properties; + +/** + * A partition discoverer that can be used to discover topics and partitions metadata + * from Kafka brokers via the Kafka 0.10 high-level consumer API. + */ +public class Kafka010PartitionDiscoverer extends Kafka09PartitionDiscoverer { + + public Kafka010PartitionDiscoverer( + KafkaTopicsDescriptor topicsDescriptor, + int indexOfThisSubtask, + int numParallelSubtasks, + Properties kafkaProperties) { + + super(topicsDescriptor, indexOfThisSubtask, numParallelSubtasks, kafkaProperties); + } +} diff --git a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010FetcherTest.java b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010FetcherTest.java similarity index 98% rename from flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010FetcherTest.java rename to flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010FetcherTest.java index 2d0551dcac939..ad8d4c2724731 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010FetcherTest.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010FetcherTest.java @@ -16,16 +16,13 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.kafka; +package org.apache.flink.streaming.connectors.kafka.internal; import org.apache.flink.core.testutils.MultiShotLatch; import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext; import org.apache.flink.streaming.api.watermark.Watermark; -import org.apache.flink.streaming.connectors.kafka.internal.Handover; -import org.apache.flink.streaming.connectors.kafka.internal.Kafka010Fetcher; -import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionStateSentinel; import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService; diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java index 858a7903b6748..d748bafbb0cf6 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java @@ -17,44 +17,31 @@ package org.apache.flink.streaming.connectors.kafka; -import kafka.cluster.Broker; -import kafka.common.ErrorMapping; -import kafka.javaapi.PartitionMetadata; -import kafka.javaapi.TopicMetadata; -import kafka.javaapi.TopicMetadataRequest; -import kafka.javaapi.consumer.SimpleConsumer; - import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractPartitionDiscoverer; import org.apache.flink.streaming.connectors.kafka.internals.Kafka08Fetcher; +import org.apache.flink.streaming.connectors.kafka.internals.Kafka08PartitionDiscoverer; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; -import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionLeader; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicsDescriptor; 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.NetUtils; import org.apache.flink.util.PropertiesUtil; import org.apache.flink.util.SerializedValue; import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.common.Node; -import java.net.InetAddress; -import java.net.URL; -import java.net.UnknownHostException; -import java.nio.channels.ClosedChannelException; import java.util.Collections; -import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Properties; -import java.util.Random; -import static org.apache.flink.util.PropertiesUtil.getInt; import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.PropertiesUtil.getLong; /** * The Flink Kafka Consumer is a streaming data source that pulls a parallel data stream from @@ -175,9 +162,12 @@ public FlinkKafkaConsumer08(List topics, DeserializationSchema deseri * The properties that are used to configure both the fetcher and the offset handler. */ public FlinkKafkaConsumer08(List topics, KeyedDeserializationSchema deserializer, Properties props) { - super(topics, deserializer); + super( + topics, + null, + deserializer, + getLong(props, KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS, DEFAULT_PARTITION_DISCOVERY_INTERVAL_MILLIS)); - checkNotNull(topics, "topics"); this.kafkaProperties = checkNotNull(props, "props"); // validate the zookeeper properties @@ -215,22 +205,12 @@ public FlinkKafkaConsumer08(List topics, KeyedDeserializationSchema d } @Override - protected List getKafkaPartitions(List topics) { - // Connect to a broker to get the partitions for all topics - List partitionInfos = - KafkaTopicPartition.dropLeaderData(getPartitionsForTopic(topics, kafkaProperties)); - - if (partitionInfos.size() == 0) { - throw new RuntimeException( - "Unable to retrieve any partitions for the requested topics " + topics + - ". Please check previous log entries"); - } + protected AbstractPartitionDiscoverer createPartitionDiscoverer( + KafkaTopicsDescriptor topicsDescriptor, + int indexOfThisSubtask, + int numParallelSubtasks) { - if (LOG.isInfoEnabled()) { - logPartitionInfo(LOG, partitionInfos); - } - - return partitionInfos; + return new Kafka08PartitionDiscoverer(topicsDescriptor, indexOfThisSubtask, numParallelSubtasks, kafkaProperties); } @Override @@ -240,102 +220,9 @@ protected boolean getIsAutoCommitEnabled() { } // ------------------------------------------------------------------------ - // Kafka / ZooKeeper communication utilities + // Kafka / ZooKeeper configuration utilities // ------------------------------------------------------------------------ - /** - * Send request to Kafka to get partitions for topic. - * - * @param topics The name of the topics. - * @param properties The properties for the Kafka Consumer that is used to query the partitions for the topic. - */ - public static List getPartitionsForTopic(List topics, Properties properties) { - String seedBrokersConfString = properties.getProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG); - final int numRetries = getInt(properties, GET_PARTITIONS_RETRIES_KEY, DEFAULT_GET_PARTITIONS_RETRIES); - - checkNotNull(seedBrokersConfString, "Configuration property %s not set", ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG); - String[] seedBrokers = seedBrokersConfString.split(","); - List partitions = new ArrayList<>(); - - final String clientId = "flink-kafka-consumer-partition-lookup"; - final int soTimeout = getInt(properties, "socket.timeout.ms", 30000); - final int bufferSize = getInt(properties, "socket.receive.buffer.bytes", 65536); - - Random rnd = new Random(); - retryLoop: for (int retry = 0; retry < numRetries; retry++) { - // we pick a seed broker randomly to avoid overloading the first broker with all the requests when the - // parallel source instances start. Still, we try all available brokers. - int index = rnd.nextInt(seedBrokers.length); - brokersLoop: for (int arrIdx = 0; arrIdx < seedBrokers.length; arrIdx++) { - String seedBroker = seedBrokers[index]; - LOG.info("Trying to get topic metadata from broker {} in try {}/{}", seedBroker, retry, numRetries); - if (++index == seedBrokers.length) { - index = 0; - } - - URL brokerUrl = NetUtils.getCorrectHostnamePort(seedBroker); - SimpleConsumer consumer = null; - try { - consumer = new SimpleConsumer(brokerUrl.getHost(), brokerUrl.getPort(), soTimeout, bufferSize, clientId); - - TopicMetadataRequest req = new TopicMetadataRequest(topics); - kafka.javaapi.TopicMetadataResponse resp = consumer.send(req); - - List metaData = resp.topicsMetadata(); - - // clear in case we have an incomplete list from previous tries - partitions.clear(); - for (TopicMetadata item : metaData) { - if (item.errorCode() != ErrorMapping.NoError()) { - // warn and try more brokers - LOG.warn("Error while getting metadata from broker " + seedBroker + " to find partitions " + - "for " + topics.toString() + ". Error: " + ErrorMapping.exceptionFor(item.errorCode()).getMessage()); - continue brokersLoop; - } - if (!topics.contains(item.topic())) { - LOG.warn("Received metadata from topic " + item.topic() + " even though it was not requested. Skipping ..."); - continue brokersLoop; - } - for (PartitionMetadata part : item.partitionsMetadata()) { - Node leader = brokerToNode(part.leader()); - KafkaTopicPartition ktp = new KafkaTopicPartition(item.topic(), part.partitionId()); - KafkaTopicPartitionLeader pInfo = new KafkaTopicPartitionLeader(ktp, leader); - partitions.add(pInfo); - } - } - break retryLoop; // leave the loop through the brokers - } - catch (Exception e) { - //validates seed brokers in case of a ClosedChannelException - validateSeedBrokers(seedBrokers, e); - LOG.warn("Error communicating with broker {} to find partitions for {}. {} Message: {}", - seedBroker, topics, e.getClass().getName(), e.getMessage()); - LOG.debug("Detailed trace", e); - // we sleep a bit. Retrying immediately doesn't make sense in cases where Kafka is reorganizing the leader metadata - try { - Thread.sleep(500); - } catch (InterruptedException e1) { - // sleep shorter. - } - } finally { - if (consumer != null) { - consumer.close(); - } - } - } // brokers loop - } // retries loop - return partitions; - } - - /** - * Turn a broker instance into a node instance - * @param broker broker instance - * @return Node representing the given broker - */ - private static Node brokerToNode(Broker broker) { - return new Node(broker.id(), broker.host(), broker.port()); - } - /** * Validate the ZK configuration, checking for required parameters * @param props Properties to check @@ -366,36 +253,6 @@ protected static void validateZooKeeperConfig(Properties props) { } } - /** - * Validate that at least one seed broker is valid in case of a - * ClosedChannelException. - * - * @param seedBrokers - * array containing the seed brokers e.g. ["host1:port1", - * "host2:port2"] - * @param exception - * instance - */ - private static void validateSeedBrokers(String[] seedBrokers, Exception exception) { - if (!(exception instanceof ClosedChannelException)) { - return; - } - int unknownHosts = 0; - for (String broker : seedBrokers) { - URL brokerUrl = NetUtils.getCorrectHostnamePort(broker.trim()); - try { - InetAddress.getByName(brokerUrl.getHost()); - } catch (UnknownHostException e) { - unknownHosts++; - } - } - // throw meaningful exception if all the provided hosts are invalid - if (unknownHosts == seedBrokers.length) { - throw new IllegalArgumentException("All the servers provided in: '" - + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG + "' config are invalid. (unknown hosts)"); - } - } - /** * Check for invalid "auto.offset.reset" values. Should be called in constructor for eager checking before submitting * the job. Note that 'none' is also considered invalid, as we don't want to deliberately throw an exception diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Kafka08Fetcher.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Kafka08Fetcher.java index de201e5d4a494..4f29e8cb008b6 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Kafka08Fetcher.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Kafka08Fetcher.java @@ -73,9 +73,6 @@ public class Kafka08Fetcher extends AbstractFetcher { /** The subtask's runtime context */ private final RuntimeContext runtimeContext; - /** The queue of partitions that are currently not assigned to a broker connection */ - private final ClosableBlockingQueue> unassignedPartitionsQueue; - /** The behavior to use in case that an offset is not valid (any more) for a partition */ private final long invalidOffsetBehavior; @@ -91,7 +88,7 @@ public class Kafka08Fetcher extends AbstractFetcher { public Kafka08Fetcher( SourceContext sourceContext, - Map assignedPartitionsWithInitialOffsets, + Map seedPartitionsWithInitialOffsets, SerializedValue> watermarksPeriodic, SerializedValue> watermarksPunctuated, StreamingRuntimeContext runtimeContext, @@ -102,7 +99,7 @@ public Kafka08Fetcher( { super( sourceContext, - assignedPartitionsWithInitialOffsets, + seedPartitionsWithInitialOffsets, watermarksPeriodic, watermarksPunctuated, runtimeContext.getProcessingTimeService(), @@ -115,12 +112,6 @@ public Kafka08Fetcher( this.runtimeContext = runtimeContext; this.invalidOffsetBehavior = getInvalidOffsetBehavior(kafkaProperties); this.autoCommitInterval = autoCommitInterval; - this.unassignedPartitionsQueue = new ClosableBlockingQueue<>(); - - // initially, all these partitions are not assigned to a specific broker connection - for (KafkaTopicPartitionState partition : subscribedPartitionStates()) { - unassignedPartitionsQueue.add(partition); - } } // ------------------------------------------------------------------------ @@ -175,8 +166,11 @@ public void runFetchLoop() throws Exception { if (autoCommitInterval > 0) { LOG.info("Starting periodic offset committer, with commit interval of {}ms", autoCommitInterval); - periodicCommitter = new PeriodicOffsetCommitter(zookeeperOffsetHandler, - subscribedPartitionStates(), errorHandler, autoCommitInterval); + periodicCommitter = new PeriodicOffsetCommitter( + zookeeperOffsetHandler, + subscribedPartitionStates(), + errorHandler, + autoCommitInterval); periodicCommitter.setName("Periodic Kafka partition offset committer"); periodicCommitter.setDaemon(true); periodicCommitter.start(); @@ -196,7 +190,7 @@ public void runFetchLoop() throws Exception { // wait for max 5 seconds trying to get partitions to assign // if threads shut down, this poll returns earlier, because the threads inject the // special marker into the queue - List> partitionsToAssign = + List> partitionsToAssign = unassignedPartitionsQueue.getBatchBlocking(5000); partitionsToAssign.remove(MARKER); @@ -347,7 +341,7 @@ public void cancel() { // ------------------------------------------------------------------------ @Override - public TopicAndPartition createKafkaPartitionHandle(KafkaTopicPartition partition) { + protected TopicAndPartition createKafkaPartitionHandle(KafkaTopicPartition partition) { return new TopicAndPartition(partition.getTopic(), partition.getPartition()); } @@ -373,8 +367,7 @@ public void commitInternalOffsetsToKafka(Map offsets) } // Set committed offsets in topic partition state - KafkaTopicPartitionState[] partitions = subscribedPartitionStates(); - for (KafkaTopicPartitionState partition : partitions) { + for (KafkaTopicPartitionState partition : subscribedPartitionStates()) { Long offset = offsets.get(partition.getKafkaTopicPartition()); if (offset != null) { partition.setCommittedOffset(offset); @@ -398,7 +391,7 @@ private SimpleConsumerThread createAndStartSimpleConsumerThread( // seed thread with list of fetch partitions (otherwise it would shut down immediately again SimpleConsumerThread brokerThread = new SimpleConsumerThread<>( - this, errorHandler, kafkaConfig, leader, seedPartitions, unassignedPartitionsQueue, + this, errorHandler, kafkaConfig, leader, seedPartitions, unassignedPartitionsQueue, clonedDeserializer, invalidOffsetBehavior); brokerThread.setName(String.format("SimpleConsumer - %s - broker-%s (%s:%d)", diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Kafka08PartitionDiscoverer.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Kafka08PartitionDiscoverer.java new file mode 100644 index 0000000000000..a3c463835983d --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Kafka08PartitionDiscoverer.java @@ -0,0 +1,290 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kafka.internals; + +import kafka.cluster.Broker; +import kafka.common.ErrorMapping; +import kafka.javaapi.PartitionMetadata; +import kafka.javaapi.TopicMetadata; +import kafka.javaapi.TopicMetadataRequest; +import kafka.javaapi.consumer.SimpleConsumer; +import org.apache.flink.util.NetUtils; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.common.Node; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.net.InetAddress; +import java.net.URL; +import java.net.UnknownHostException; +import java.nio.channels.ClosedChannelException; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; +import java.util.Properties; + +import static org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08.DEFAULT_GET_PARTITIONS_RETRIES; +import static org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08.GET_PARTITIONS_RETRIES_KEY; +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.PropertiesUtil.getInt; + +/** + * A partition discoverer that can be used to discover topics and partitions metadata + * from Kafka brokers via the Kafka 0.8 low-level consumer API. + */ +public class Kafka08PartitionDiscoverer extends AbstractPartitionDiscoverer { + + private static final Logger LOG = LoggerFactory.getLogger(Kafka08PartitionDiscoverer.class); + + private static final String dummyClientId = "flink-kafka-consumer-partition-lookup"; + + /** All seed broker addresses. */ + private final String[] seedBrokerAddresses; + + /** Configuration for the Kafka client. */ + private final int numRetries; + private final int soTimeout; + private final int bufferSize; + + /** + * The current seed broker address to use. + * Each subtask starts with an assigned seed broker using round-robin assigning. + * If this subtask fails in any one of the fetch attempts, the next address in the seed brokers list + * will be used. + */ + private int currentContactSeedBrokerIndex; + + /** Low-level consumer used to fetch topics and partitions metadata. */ + private SimpleConsumer consumer; + + public Kafka08PartitionDiscoverer( + KafkaTopicsDescriptor topicsDescriptor, + int indexOfThisSubtask, + int numParallelSubtasks, + Properties kafkaProperties) { + + super(topicsDescriptor, indexOfThisSubtask, numParallelSubtasks); + + checkNotNull(kafkaProperties); + + String seedBrokersConfString = kafkaProperties.getProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG); + checkArgument(seedBrokersConfString != null && !seedBrokersConfString.isEmpty(), + "Configuration property %s not set", ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG); + + this.seedBrokerAddresses = seedBrokersConfString.split(","); + + // evenly distribute seed brokers across subtasks, to + // avoid too much pressure on a single broker on startup + this.currentContactSeedBrokerIndex = indexOfThisSubtask % seedBrokerAddresses.length; + + this.numRetries = getInt(kafkaProperties, GET_PARTITIONS_RETRIES_KEY, DEFAULT_GET_PARTITIONS_RETRIES); + this.soTimeout = getInt(kafkaProperties, "socket.timeout.ms", 30000); + this.bufferSize = getInt(kafkaProperties, "socket.receive.buffer.bytes", 65536); + } + + @Override + protected void initializeConnections() { + URL contactUrl = NetUtils.getCorrectHostnamePort(seedBrokerAddresses[currentContactSeedBrokerIndex]); + this.consumer = new SimpleConsumer(contactUrl.getHost(), contactUrl.getPort(), soTimeout, bufferSize, dummyClientId); + } + + @Override + protected List getAllTopics() { + List topics = new LinkedList<>(); + + retryLoop: for (int retry = 0; retry < numRetries; retry++) { + brokersLoop: for (int arrIdx = 0; arrIdx < seedBrokerAddresses.length; arrIdx++) { + LOG.info("Trying to get topic metadata from broker {} in try {}/{}", seedBrokerAddresses[currentContactSeedBrokerIndex], retry, numRetries); + + try { + // clear in case we have an incomplete list from previous tries + topics.clear(); + + for (TopicMetadata item : consumer.send(new TopicMetadataRequest(Collections.emptyList())).topicsMetadata()) { + if (item.errorCode() != ErrorMapping.NoError()) { + // warn and try more brokers + LOG.warn("Error while getting metadata from broker {} to find partitions for {}. Error: {}.", + seedBrokerAddresses[currentContactSeedBrokerIndex], topics.toString(), ErrorMapping.exceptionFor(item.errorCode()).getMessage()); + + useNextAddressAsNewContactSeedBroker(); + continue brokersLoop; + } + + topics.add(item.topic()); + } + break retryLoop; // leave the loop through the brokers + } + catch (Exception e) { + //validates seed brokers in case of a ClosedChannelException + validateSeedBrokers(seedBrokerAddresses, e); + LOG.warn("Error communicating with broker {} to find partitions for {}. {} Message: {}", + seedBrokerAddresses[currentContactSeedBrokerIndex], topics, e.getClass().getName(), e.getMessage()); + LOG.debug("Detailed trace", e); + + // we sleep a bit. Retrying immediately doesn't make sense in cases where Kafka is reorganizing the leader metadata + try { + Thread.sleep(500); + } catch (InterruptedException e1) { + // sleep shorter. + } + + useNextAddressAsNewContactSeedBroker(); + } + } // brokers loop + } // retries loop + + return topics; + } + + @Override + public List getAllPartitionsForTopics(List topics) { + return KafkaTopicPartition.dropLeaderData(getPartitionLeadersForTopics(topics)); + } + + @Override + protected void wakeupConnections() { + // nothing to do, as Kafka 0.8's SimpleConsumer does not support wakeup + } + + @Override + protected void closeConnections() throws Exception { + if (consumer != null) { + SimpleConsumer consumer = this.consumer; + consumer.close(); + + // de-reference the consumer to avoid closing multiple times + this.consumer = null; + } + } + + /** + * Send request to Kafka to get partitions for topics. + * + * @param topics The name of the topics. + */ + public List getPartitionLeadersForTopics(List topics) { + List partitions = new LinkedList<>(); + + retryLoop: for (int retry = 0; retry < numRetries; retry++) { + brokersLoop: for (int arrIdx = 0; arrIdx < seedBrokerAddresses.length; arrIdx++) { + LOG.info("Trying to get topic metadata from broker {} in try {}/{}", seedBrokerAddresses[currentContactSeedBrokerIndex], retry, numRetries); + + try { + // clear in case we have an incomplete list from previous tries + partitions.clear(); + + for (TopicMetadata item : consumer.send(new TopicMetadataRequest(topics)).topicsMetadata()) { + if (item.errorCode() != ErrorMapping.NoError()) { + // warn and try more brokers + LOG.warn("Error while getting metadata from broker {} to find partitions for {}. Error: {}.", + seedBrokerAddresses[currentContactSeedBrokerIndex], topics.toString(), ErrorMapping.exceptionFor(item.errorCode()).getMessage()); + + useNextAddressAsNewContactSeedBroker(); + continue brokersLoop; + } + + if (!topics.contains(item.topic())) { + LOG.warn("Received metadata from topic " + item.topic() + " even though it was not requested. Skipping ..."); + + useNextAddressAsNewContactSeedBroker(); + continue brokersLoop; + } + + for (PartitionMetadata part : item.partitionsMetadata()) { + Node leader = brokerToNode(part.leader()); + KafkaTopicPartition ktp = new KafkaTopicPartition(item.topic(), part.partitionId()); + KafkaTopicPartitionLeader pInfo = new KafkaTopicPartitionLeader(ktp, leader); + partitions.add(pInfo); + } + } + break retryLoop; // leave the loop through the brokers + } + catch (Exception e) { + //validates seed brokers in case of a ClosedChannelException + validateSeedBrokers(seedBrokerAddresses, e); + LOG.warn("Error communicating with broker {} to find partitions for {}. {} Message: {}", + seedBrokerAddresses[currentContactSeedBrokerIndex], topics, e.getClass().getName(), e.getMessage()); + LOG.debug("Detailed trace", e); + + // we sleep a bit. Retrying immediately doesn't make sense in cases where Kafka is reorganizing the leader metadata + try { + Thread.sleep(500); + } catch (InterruptedException e1) { + // sleep shorter. + } + + useNextAddressAsNewContactSeedBroker(); + } + } // brokers loop + } // retries loop + + return partitions; + } + + /** + * Re-establish broker connection using the next available seed broker address. + */ + private void useNextAddressAsNewContactSeedBroker() { + if (++currentContactSeedBrokerIndex == seedBrokerAddresses.length) { + currentContactSeedBrokerIndex = 0; + } + + URL newContactUrl = NetUtils.getCorrectHostnamePort(seedBrokerAddresses[currentContactSeedBrokerIndex]); + this.consumer = new SimpleConsumer(newContactUrl.getHost(), newContactUrl.getPort(), soTimeout, bufferSize, dummyClientId); + } + + /** + * Turn a broker instance into a node instance + * @param broker broker instance + * @return Node representing the given broker + */ + private static Node brokerToNode(Broker broker) { + return new Node(broker.id(), broker.host(), broker.port()); + } + + /** + * Validate that at least one seed broker is valid in case of a + * ClosedChannelException. + * + * @param seedBrokers + * array containing the seed brokers e.g. ["host1:port1", + * "host2:port2"] + * @param exception + * instance + */ + private static void validateSeedBrokers(String[] seedBrokers, Exception exception) { + if (!(exception instanceof ClosedChannelException)) { + return; + } + int unknownHosts = 0; + for (String broker : seedBrokers) { + URL brokerUrl = NetUtils.getCorrectHostnamePort(broker.trim()); + try { + InetAddress.getByName(brokerUrl.getHost()); + } catch (UnknownHostException e) { + unknownHosts++; + } + } + // throw meaningful exception if all the provided hosts are invalid + if (unknownHosts == seedBrokers.length) { + throw new IllegalArgumentException("All the servers provided in: '" + + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG + "' config are invalid. (unknown hosts)"); + } + } +} diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PartitionInfoFetcher.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PartitionInfoFetcher.java index d8d927d9ec16b..650b41ed775b0 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PartitionInfoFetcher.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PartitionInfoFetcher.java @@ -18,32 +18,38 @@ package org.apache.flink.streaming.connectors.kafka.internals; -import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08; - import java.util.List; import java.util.Properties; class PartitionInfoFetcher extends Thread { private final List topics; - private final Properties properties; + private final Kafka08PartitionDiscoverer partitionDiscoverer; private volatile List result; private volatile Throwable error; PartitionInfoFetcher(List topics, Properties properties) { + // we're only using partial functionality of the partition discoverer; the subtask id arguments doesn't matter + this.partitionDiscoverer = new Kafka08PartitionDiscoverer(new KafkaTopicsDescriptor(topics, null), 0, 1, properties); this.topics = topics; - this.properties = properties; } @Override public void run() { try { - result = FlinkKafkaConsumer08.getPartitionsForTopic(topics, properties); + partitionDiscoverer.open(); + result = partitionDiscoverer.getPartitionLeadersForTopics(topics); } catch (Throwable t) { this.error = t; + } finally { + try { + partitionDiscoverer.close(); + } catch (Exception e) { + throw new RuntimeException("Error while closing partition discoverer.", e); + } } } @@ -58,9 +64,11 @@ public List getPartitions() throws Exception { if (error != null) { throw new Exception("Failed to fetch partitions for topics " + topics.toString(), error); } + if (result != null) { return result; } + throw new Exception("Partition fetching failed"); } -} \ No newline at end of file +} diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PeriodicOffsetCommitter.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PeriodicOffsetCommitter.java index 27d90f2e98561..d3003135c3323 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PeriodicOffsetCommitter.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PeriodicOffsetCommitter.java @@ -18,7 +18,10 @@ package org.apache.flink.streaming.connectors.kafka.internals; +import kafka.common.TopicAndPartition; + import java.util.HashMap; +import java.util.List; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -31,7 +34,7 @@ public class PeriodicOffsetCommitter extends Thread { /** The ZooKeeper handler */ private final ZookeeperOffsetHandler offsetHandler; - private final KafkaTopicPartitionState[] partitionStates; + private final List> partitionStates; /** The proxy to forward exceptions to the main thread */ private final ExceptionProxy errorHandler; @@ -43,7 +46,7 @@ public class PeriodicOffsetCommitter extends Thread { private volatile boolean running = true; PeriodicOffsetCommitter(ZookeeperOffsetHandler offsetHandler, - KafkaTopicPartitionState[] partitionStates, + List> partitionStates, ExceptionProxy errorHandler, long commitInterval) { @@ -62,7 +65,7 @@ public void run() { Thread.sleep(commitInterval); // create copy a deep copy of the current offsets - HashMap offsetsToCommit = new HashMap<>(partitionStates.length); + HashMap offsetsToCommit = new HashMap<>(partitionStates.size()); for (KafkaTopicPartitionState partitionState : partitionStates) { offsetsToCommit.put(partitionState.getKafkaTopicPartition(), partitionState.getOffset()); } diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumer08Test.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumer08Test.java index 83cdd90a1287e..8be801948431f 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumer08Test.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumer08Test.java @@ -19,6 +19,7 @@ package org.apache.flink.streaming.connectors.kafka; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.Mockito.mock; @@ -27,25 +28,16 @@ import java.net.InetAddress; import java.net.URL; import java.net.UnknownHostException; -import java.util.Arrays; import java.util.Collections; -import java.util.List; -import java.util.Map; import java.util.Properties; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.configuration.Configuration; -import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; -import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; -import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; -import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; -import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.streaming.util.serialization.DeserializationSchema; -import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +import org.apache.flink.streaming.connectors.kafka.internals.Kafka08PartitionDiscoverer; import org.apache.flink.streaming.util.serialization.SimpleStringSchema; import org.apache.flink.util.NetUtils; -import org.apache.flink.util.SerializedValue; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.junit.Test; import org.junit.runner.RunWith; @@ -57,7 +49,7 @@ import org.powermock.modules.junit4.PowerMockRunner; @RunWith(PowerMockRunner.class) -@PrepareForTest(FlinkKafkaConsumer08.class) +@PrepareForTest(Kafka08PartitionDiscoverer.class) @PowerMockIgnore("javax.management.*") public class KafkaConsumer08Test { @@ -118,6 +110,7 @@ public void testCreateSourceWithoutCluster() { consumer.setRuntimeContext(mockRuntimeContext); consumer.open(new Configuration()); + fail(); } catch (Exception e) { @@ -146,21 +139,21 @@ public void testAllBoostrapServerHostsAreInvalid() { consumer.setRuntimeContext(mockRuntimeContext); consumer.open(new Configuration()); + fail(); - } catch (Exception e) { + } catch (Exception expected) { assertTrue("Exception should be thrown containing 'all bootstrap servers invalid' message!", - e.getMessage().contains("All the servers provided in: '" + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG + expected.getMessage().contains("All the servers provided in: '" + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG + "' config are invalid")); } } @Test - public void testAtLeastOneBootstrapServerHostIsValid() { + public void testAtLeastOneBootstrapServerHostIsValid() throws Exception { try { String zookeeperConnect = "localhost:56794"; String unknownHost = "foobar:11111"; - // we declare one valid bootstrap server, namely the one with - // 'localhost' + // we declare one valid bootstrap server, namely the one with 'localhost' String bootstrapServers = unknownHost + ", localhost:22222"; URL unknownHostURL = NetUtils.getCorrectHostnamePort(unknownHost); @@ -170,15 +163,17 @@ public void testAtLeastOneBootstrapServerHostIsValid() { String groupId = "non-existent-group"; Properties props = createKafkaProps(zookeeperConnect, bootstrapServers, groupId); - FlinkKafkaConsumer08 consumer = new FlinkKafkaConsumer08<>(Collections.singletonList("no op topic"), - new SimpleStringSchema(), props); + DummyFlinkKafkaConsumer consumer = new DummyFlinkKafkaConsumer( + "no op topic", + new SimpleStringSchema(), + props); consumer.open(new Configuration()); - fail(); + + // no exception should be thrown, because we have one valid bootstrap server; test passes if we reach here } catch (Exception e) { - // test is not failing because we have one valid boostrap server - assertTrue("The cause of the exception should not be 'all boostrap server are invalid'!", - !e.getMessage().contains("All the hosts provided in: " + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG - + " config are invalid")); + assertFalse("No exception should be thrown containing 'all bootstrap servers invalid' message!", + e.getMessage().contains("All the servers provided in: '" + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG + + "' config are invalid")); } } @@ -191,4 +186,21 @@ private Properties createKafkaProps(String zookeeperConnect, String bootstrapSer props.setProperty(FlinkKafkaConsumer08.GET_PARTITIONS_RETRIES_KEY, "1"); return props; } + + private static class DummyFlinkKafkaConsumer extends FlinkKafkaConsumer08 { + + private static final long serialVersionUID = -3939402845009972810L; + + public DummyFlinkKafkaConsumer(String topic, DeserializationSchema schema, Properties props) { + super(Collections.singletonList(topic), schema, props); + } + + @Override + public RuntimeContext getRuntimeContext() { + RuntimeContext mockRuntimeContext = mock(RuntimeContext.class); + when(mockRuntimeContext.getIndexOfThisSubtask()).thenReturn(0); + when(mockRuntimeContext.getNumberOfParallelSubtasks()).thenReturn(1); + return mockRuntimeContext; + } + } } diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java index 643ee8eee8143..341b734609c73 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java @@ -34,7 +34,9 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.operators.StreamSink; +import org.apache.flink.streaming.connectors.kafka.internals.Kafka08PartitionDiscoverer; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionLeader; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicsDescriptor; import org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler; import org.apache.flink.streaming.connectors.kafka.testutils.ZooKeeperStringSerializer; import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; @@ -282,6 +284,17 @@ public void createTestTopic(String topic, int numberOfPartitions, int replicatio AdminUtils.createTopic(creator, topic, numberOfPartitions, replicationFactor, topicConfig); creator.close(); + List topicList = Collections.singletonList(topic); + + // create a partition discoverer, to make sure that partitions for the test topic are created + Kafka08PartitionDiscoverer partitionDiscoverer = + new Kafka08PartitionDiscoverer(new KafkaTopicsDescriptor(topicList, null), 0, 1, standardProps); + try { + partitionDiscoverer.open(); + } catch (Exception e) { + throw new RuntimeException("Exception while opening partition discoverer.", e); + } + // validate that the topic has been created final long deadline = System.nanoTime() + 30_000_000_000L; do { @@ -291,7 +304,7 @@ public void createTestTopic(String topic, int numberOfPartitions, int replicatio catch (InterruptedException e) { // restore interrupted state } - List partitions = FlinkKafkaConsumer08.getPartitionsForTopic(Collections.singletonList(topic), standardProps); + List partitions = partitionDiscoverer.getPartitionLeadersForTopics(Collections.singletonList(topic)); if (partitions != null && partitions.size() > 0) { return; } diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java index d0284ce2b40ae..fe11cfdae63eb 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java @@ -22,8 +22,11 @@ import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; import org.apache.flink.streaming.connectors.kafka.internal.Kafka09Fetcher; +import org.apache.flink.streaming.connectors.kafka.internal.Kafka09PartitionDiscoverer; import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractPartitionDiscoverer; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicsDescriptor; import org.apache.flink.streaming.util.serialization.DeserializationSchema; import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper; @@ -31,20 +34,18 @@ 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.Map; import java.util.Properties; import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.PropertiesUtil.getLong; /** * The Flink Kafka Consumer is a streaming data source that pulls a parallel data stream from @@ -152,7 +153,7 @@ public FlinkKafkaConsumer09(List topics, DeserializationSchema deseri * The properties that are used to configure both the fetcher and the offset handler. */ public FlinkKafkaConsumer09(List topics, KeyedDeserializationSchema deserializer, Properties props) { - super(topics, deserializer); + super(topics, null, deserializer, getLong(props, KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS, DEFAULT_PARTITION_DISCOVERY_INTERVAL_MILLIS)); this.properties = checkNotNull(props, "props"); setDeserializer(this.properties); @@ -198,36 +199,12 @@ public FlinkKafkaConsumer09(List topics, KeyedDeserializationSchema d } @Override - protected List getKafkaPartitions(List topics) { - // 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)); - } - else{ - LOG.info("Unable to retrieve any partitions for the requested topic: {}", topic); - } - } - } - - 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); - } + protected AbstractPartitionDiscoverer createPartitionDiscoverer( + KafkaTopicsDescriptor topicsDescriptor, + int indexOfThisSubtask, + int numParallelSubtasks) { - return partitions; + return new Kafka09PartitionDiscoverer(topicsDescriptor, indexOfThisSubtask, numParallelSubtasks, properties); } @Override @@ -240,22 +217,6 @@ protected boolean getIsAutoCommitEnabled() { // 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. * diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java index 1c87542f8ae85..0d2b76c9817b2 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java @@ -102,7 +102,7 @@ public Kafka09Fetcher( LOG, handover, kafkaProperties, - subscribedPartitionStates(), + unassignedPartitionsQueue, kafkaMetricGroup, createCallBridge(), getFetcherName() + " for " + taskNameWithSubtasks, @@ -124,7 +124,7 @@ public void runFetchLoop() throws Exception { while (running) { // this blocks until we get the next records - // it automatically re-throws exceptions encountered in the fetcher thread + // it automatically re-throws exceptions encountered in the consumer thread final ConsumerRecords records = handover.pollNext(); // get the records for each topic partition @@ -212,8 +212,10 @@ public TopicPartition createKafkaPartitionHandle(KafkaTopicPartition partition) @Override public void commitInternalOffsetsToKafka(Map offsets) throws Exception { - KafkaTopicPartitionState[] partitions = subscribedPartitionStates(); - Map offsetsToCommit = new HashMap<>(partitions.length); + @SuppressWarnings("unchecked") + List> partitions = subscribedPartitionStates(); + + Map offsetsToCommit = new HashMap<>(partitions.size()); for (KafkaTopicPartitionState partition : partitions) { Long lastProcessedOffset = offsets.get(partition.getKafkaTopicPartition()); diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09PartitionDiscoverer.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09PartitionDiscoverer.java new file mode 100644 index 0000000000000..c1d9a706a1b94 --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09PartitionDiscoverer.java @@ -0,0 +1,102 @@ +/* + * 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.streaming.connectors.kafka.internals.AbstractPartitionDiscoverer; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicsDescriptor; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.PartitionInfo; + +import java.util.ArrayList; +import java.util.LinkedList; +import java.util.List; +import java.util.Properties; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A partition discoverer that can be used to discover topics and partitions metadata + * from Kafka brokers via the Kafka 0.9 high-level consumer API. + */ +public class Kafka09PartitionDiscoverer extends AbstractPartitionDiscoverer { + + private final Properties kafkaProperties; + + private KafkaConsumer kafkaConsumer; + + public Kafka09PartitionDiscoverer( + KafkaTopicsDescriptor topicsDescriptor, + int indexOfThisSubtask, + int numParallelSubtasks, + Properties kafkaProperties) { + + super(topicsDescriptor, indexOfThisSubtask, numParallelSubtasks); + this.kafkaProperties = checkNotNull(kafkaProperties); + } + + @Override + protected void initializeConnections() { + this.kafkaConsumer = new KafkaConsumer<>(kafkaProperties); + } + + @Override + protected List getAllTopics() throws WakeupException { + try { + return new ArrayList<>(kafkaConsumer.listTopics().keySet()); + } catch (org.apache.kafka.common.errors.WakeupException e) { + // rethrow our own wakeup exception + throw new WakeupException(); + } + } + + @Override + protected List getAllPartitionsForTopics(List topics) throws WakeupException { + List partitions = new LinkedList<>(); + + try { + for (String topic : topics) { + for (PartitionInfo partitionInfo : kafkaConsumer.partitionsFor(topic)) { + partitions.add(new KafkaTopicPartition(partitionInfo.topic(), partitionInfo.partition())); + } + } + } catch (org.apache.kafka.common.errors.WakeupException e) { + // rethrow our own wakeup exception + throw new WakeupException(); + } + + return partitions; + } + + @Override + protected void wakeupConnections() { + if (this.kafkaConsumer != null) { + this.kafkaConsumer.wakeup(); + } + } + + @Override + protected void closeConnections() throws Exception { + if (this.kafkaConsumer != null) { + this.kafkaConsumer.close(); + + // de-reference the consumer to avoid closing multiple times + this.kafkaConsumer = null; + } + } +} diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerThread.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerThread.java index cbe15511cdb06..c1e0528c42903 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerThread.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerThread.java @@ -18,7 +18,9 @@ package org.apache.flink.streaming.connectors.kafka.internal; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.connectors.kafka.internals.ClosableBlockingQueue; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionState; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionStateSentinel; import org.apache.flink.streaming.connectors.kafka.internals.metrics.KafkaMetricWrapper; @@ -34,6 +36,7 @@ import org.slf4j.Logger; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Properties; @@ -68,8 +71,8 @@ public class KafkaConsumerThread extends Thread { /** The configuration for the Kafka consumer */ private final Properties kafkaProperties; - /** The partitions that this consumer reads from */ - private final KafkaTopicPartitionState[] subscribedPartitionStates; + /** The queue of unassigned partitions that we need to assign to the Kafka consumer */ + private final ClosableBlockingQueue> unassignedPartitionsQueue; /** We get this from the outside to publish metrics. **/ private final MetricGroup kafkaMetricGroup; @@ -86,18 +89,26 @@ public class KafkaConsumerThread extends Thread { /** Reference to the Kafka consumer, once it is created */ private volatile KafkaConsumer consumer; + /** This lock is used to isolate the consumer for partition reassignment */ + private final Object consumerReassignmentLock; + + /** + * Flag to indicate whether an external operation ({@link #setOffsetsToCommit(Map)} or {@link #shutdown()}) + * had attempted to wakeup the consumer while it was isolated for partition reassignment. + */ + private volatile boolean hasBufferedWakeup; + /** Flag to mark the main work loop as alive */ private volatile boolean running; /** Flag tracking whether the latest commit request has completed */ private volatile boolean commitInProgress; - public KafkaConsumerThread( Logger log, Handover handover, Properties kafkaProperties, - KafkaTopicPartitionState[] subscribedPartitionStates, + ClosableBlockingQueue> unassignedPartitionsQueue, MetricGroup kafkaMetricGroup, KafkaConsumerCallBridge consumerCallBridge, String threadName, @@ -113,11 +124,12 @@ public KafkaConsumerThread( this.kafkaMetricGroup = checkNotNull(kafkaMetricGroup); this.consumerCallBridge = checkNotNull(consumerCallBridge); - this.subscribedPartitionStates = checkNotNull(subscribedPartitionStates); + this.unassignedPartitionsQueue = checkNotNull(unassignedPartitionsQueue); this.pollTimeout = pollTimeout; this.useMetrics = useMetrics; + this.consumerReassignmentLock = new Object(); this.nextOffsetsToCommit = new AtomicReference<>(); this.running = true; } @@ -137,9 +149,8 @@ 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); + this.consumer = getConsumer(kafkaProperties); } catch (Throwable t) { handover.reportError(t); @@ -148,12 +159,6 @@ public void run() { // from here on, the consumer is guaranteed to be closed properly try { - // The callback invoked by Kafka once an offset commit is complete - final OffsetCommitCallback offsetCommitCallback = new CommitCallback(); - - // tell the consumer which partitions to work with - consumerCallBridge.assignPartitions(consumer, convertKafkaPartitions(subscribedPartitionStates)); - // register Kafka's very own metrics in Flink's metric reporters if (useMetrics) { // register Kafka metrics to Flink @@ -174,33 +179,18 @@ public void run() { return; } - // offsets in the state may still be placeholder sentinel values if we are starting fresh, or the - // checkpoint / savepoint state we were restored with had not completely been replaced with actual offset - // values yet; replace those with actual offsets, according to what the sentinel value represent. - for (KafkaTopicPartitionState partition : subscribedPartitionStates) { - if (partition.getOffset() == KafkaTopicPartitionStateSentinel.EARLIEST_OFFSET) { - consumerCallBridge.seekPartitionToBeginning(consumer, partition.getKafkaPartitionHandle()); - partition.setOffset(consumer.position(partition.getKafkaPartitionHandle()) - 1); - } else if (partition.getOffset() == KafkaTopicPartitionStateSentinel.LATEST_OFFSET) { - consumerCallBridge.seekPartitionToEnd(consumer, partition.getKafkaPartitionHandle()); - partition.setOffset(consumer.position(partition.getKafkaPartitionHandle()) - 1); - } else if (partition.getOffset() == KafkaTopicPartitionStateSentinel.GROUP_OFFSET) { - // the KafkaConsumer by default will automatically seek the consumer position - // to the committed group offset, so we do not need to do it. - - partition.setOffset(consumer.position(partition.getKafkaPartitionHandle()) - 1); - } else { - consumer.seek(partition.getKafkaPartitionHandle(), partition.getOffset() + 1); - } - } - - // from now on, external operations may call the consumer - this.consumer = consumer; + // The callback invoked by Kafka once an offset commit is complete + final OffsetCommitCallback offsetCommitCallback = new CommitCallback(); - // the latest bulk of records. may carry across the loop if the thread is woken up + // the latest bulk of records. May carry across the loop if the thread is woken up // from blocking on the handover ConsumerRecords records = null; + // reused variable to hold found unassigned new partitions. + // found partitions are not carried across loops using this variable; + // they are carried across via re-adding them to the unassigned partitions queue + List> newPartitions; + // main fetch loop while (running) { @@ -219,6 +209,15 @@ public void run() { } } + try { + newPartitions = unassignedPartitionsQueue.pollBatch(); + if (newPartitions != null) { + reassignPartitions(newPartitions); + } + } catch (AbortedReassignmentException e) { + continue; + } + // get the next batch of records, unless we did not manage to hand the old batch over if (records == null) { try { @@ -271,9 +270,15 @@ public void shutdown() { // this wakes up the consumer if it is blocked handing over records handover.wakeupProducer(); - // this wakes up the consumer if it is blocked in a kafka poll - if (consumer != null) { - consumer.wakeup(); + // this wakes up the consumer if it is blocked in a kafka poll + synchronized (consumerReassignmentLock) { + if (consumer != null) { + consumer.wakeup(); + } else { + // the consumer is currently isolated for partition reassignment; + // set this flag so that the wakeup state is restored once the reassignment is complete + hasBufferedWakeup = true; + } } } @@ -297,25 +302,154 @@ public void setOffsetsToCommit(Map offsetsToC // if the consumer is blocked in a poll() or handover operation, wake it up to commit soon handover.wakeupProducer(); - if (consumer != null) { - consumer.wakeup(); + + synchronized (consumerReassignmentLock) { + if (consumer != null) { + consumer.wakeup(); + } else { + // the consumer is currently isolated for partition reassignment; + // set this flag so that the wakeup state is restored once the reassignment is complete + hasBufferedWakeup = true; + } } } + // ------------------------------------------------------------------------ + + /** + * Reestablishes the assigned partitions for the consumer. + * The reassigned partitions consists of the provided new partitions and whatever partitions + * was already previously assigned to the consumer. + * + *

The reassignment process is protected against wakeup calls, so that after + * this method returns, the consumer is either untouched or completely reassigned + * with the correct offset positions. + * + *

If the consumer was already woken-up prior to a reassignment resulting in an + * interruption any time during the reassignment, the consumer is guaranteed + * to roll back as if it was untouched. On the other hand, if there was an attempt + * to wakeup the consumer during the reassignment, the wakeup call is "buffered" + * until the reassignment completes. + * + *

This method is exposed for testing purposes. + */ + @VisibleForTesting + void reassignPartitions(List> newPartitions) throws Exception { + boolean reassignmentStarted = false; + + // since the reassignment may introduce several Kafka blocking calls that cannot be interrupted, + // the consumer needs to be isolated from external wakeup calls in setOffsetsToCommit() and shutdown() + // until the reassignment is complete. + final KafkaConsumer consumerTmp; + synchronized (consumerReassignmentLock) { + consumerTmp = this.consumer; + this.consumer = null; + } + + final Map oldPartitionAssignmentsToPosition = new HashMap<>(); + try { + for (TopicPartition oldPartition : consumerTmp.assignment()) { + oldPartitionAssignmentsToPosition.put(oldPartition, consumerTmp.position(oldPartition)); + } + + final List newPartitionAssignments = + new ArrayList<>(newPartitions.size() + oldPartitionAssignmentsToPosition.size()); + newPartitionAssignments.addAll(oldPartitionAssignmentsToPosition.keySet()); + newPartitionAssignments.addAll(convertKafkaPartitions(newPartitions)); + + // reassign with the new partitions + consumerCallBridge.assignPartitions(consumerTmp, newPartitionAssignments); + reassignmentStarted = true; + + // old partitions should be seeked to their previous position + for (Map.Entry oldPartitionToPosition : oldPartitionAssignmentsToPosition.entrySet()) { + consumerTmp.seek(oldPartitionToPosition.getKey(), oldPartitionToPosition.getValue()); + } + + // offsets in the state of new partitions may still be placeholder sentinel values if we are: + // (1) starting fresh, + // (2) checkpoint / savepoint state we were restored with had not completely + // been replaced with actual offset values yet, or + // (3) the partition was newly discovered after startup; + // replace those with actual offsets, according to what the sentinel value represent. + for (KafkaTopicPartitionState newPartitionState : newPartitions) { + if (newPartitionState.getOffset() == KafkaTopicPartitionStateSentinel.EARLIEST_OFFSET) { + consumerCallBridge.seekPartitionToBeginning(consumerTmp, newPartitionState.getKafkaPartitionHandle()); + newPartitionState.setOffset(consumerTmp.position(newPartitionState.getKafkaPartitionHandle()) - 1); + } else if (newPartitionState.getOffset() == KafkaTopicPartitionStateSentinel.LATEST_OFFSET) { + consumerCallBridge.seekPartitionToEnd(consumerTmp, newPartitionState.getKafkaPartitionHandle()); + newPartitionState.setOffset(consumerTmp.position(newPartitionState.getKafkaPartitionHandle()) - 1); + } else if (newPartitionState.getOffset() == KafkaTopicPartitionStateSentinel.GROUP_OFFSET) { + // the KafkaConsumer by default will automatically seek the consumer position + // to the committed group offset, so we do not need to do it. + + newPartitionState.setOffset(consumerTmp.position(newPartitionState.getKafkaPartitionHandle()) - 1); + } else { + consumerTmp.seek(newPartitionState.getKafkaPartitionHandle(), newPartitionState.getOffset() + 1); + } + } + } catch (WakeupException e) { + // a WakeupException may be thrown if the consumer was invoked wakeup() + // before it was isolated for the reassignment. In this case, we abort the + // reassignment and just re-expose the original consumer. + + synchronized (consumerReassignmentLock) { + this.consumer = consumerTmp; + + // if reassignment had already started and affected the consumer, + // we do a full roll back so that it is as if it was left untouched + if (reassignmentStarted) { + consumerCallBridge.assignPartitions( + this.consumer, new ArrayList<>(oldPartitionAssignmentsToPosition.keySet())); + + for (Map.Entry oldPartitionToPosition : oldPartitionAssignmentsToPosition.entrySet()) { + this.consumer.seek(oldPartitionToPosition.getKey(), oldPartitionToPosition.getValue()); + } + } + + // no need to restore the wakeup state in this case, + // since only the last wakeup call is effective anyways + hasBufferedWakeup = false; + + // re-add all new partitions back to the unassigned partitions queue to be picked up again + for (KafkaTopicPartitionState newPartition : newPartitions) { + unassignedPartitionsQueue.add(newPartition); + } + + // this signals the main fetch loop to continue through the loop + throw new AbortedReassignmentException(); + } + } + + // reassignment complete; expose the reassigned consumer + synchronized (consumerReassignmentLock) { + this.consumer = consumerTmp; + + // restore wakeup state for the consumer if necessary + if (hasBufferedWakeup) { + this.consumer.wakeup(); + hasBufferedWakeup = false; + } + } + } + + @VisibleForTesting + KafkaConsumer getConsumer(Properties kafkaProperties) { + return new KafkaConsumer<>(kafkaProperties); + } + // ------------------------------------------------------------------------ // Utilities // ------------------------------------------------------------------------ - private static List convertKafkaPartitions(KafkaTopicPartitionState[] partitions) { - ArrayList result = new ArrayList<>(partitions.length); + private static List convertKafkaPartitions(List> partitions) { + ArrayList result = new ArrayList<>(partitions.size()); for (KafkaTopicPartitionState p : partitions) { result.add(p.getKafkaPartitionHandle()); } return result; } - // ------------------------------------------------------------------------ - private class CommitCallback implements OffsetCommitCallback { @Override @@ -327,4 +461,12 @@ public void onComplete(Map offsets, Exception } } } + + /** + * Utility exception that serves as a signal for the main loop to continue through the loop + * if a reassignment attempt was aborted due to an pre-reassignment wakeup call on the consumer. + */ + private class AbortedReassignmentException extends Exception { + private static final long serialVersionUID = 1L; + } } diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09FetcherTest.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09FetcherTest.java similarity index 98% rename from flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09FetcherTest.java rename to flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09FetcherTest.java index 6e13db2935f9e..5e9eab29989ef 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09FetcherTest.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09FetcherTest.java @@ -16,16 +16,13 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.kafka; +package org.apache.flink.streaming.connectors.kafka.internal; import org.apache.flink.core.testutils.MultiShotLatch; import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext; import org.apache.flink.streaming.api.watermark.Watermark; -import org.apache.flink.streaming.connectors.kafka.internal.Handover; -import org.apache.flink.streaming.connectors.kafka.internal.Kafka09Fetcher; -import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionStateSentinel; import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService; diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerThreadTest.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerThreadTest.java new file mode 100644 index 0000000000000..67a44501fc04b --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerThreadTest.java @@ -0,0 +1,842 @@ +/* + * 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.core.testutils.MultiShotLatch; +import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.connectors.kafka.internals.ClosableBlockingQueue; +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.KafkaTopicPartitionStateSentinel; +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.TopicPartition; +import org.apache.kafka.common.errors.WakeupException; +import org.junit.Test; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; +import org.slf4j.Logger; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyListOf; +import static org.mockito.Matchers.anyLong; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +/** + * Unit tests for the {@link KafkaConsumerThread}. + */ +public class KafkaConsumerThreadTest { + + /** + * Tests reassignment works correctly in the case when: + * - the consumer initially had no assignments + * - new unassigned partitions already have defined offsets + * + * Setting a timeout because the test will not finish if there is logic error with + * the reassignment flow. + */ + @SuppressWarnings("unchecked") + @Test(timeout = 10000) + public void testReassigningPartitionsWithDefinedOffsetsWhenNoInitialAssignment() throws Exception { + final String testTopic = "test-topic"; + + // -------- new partitions with defined offsets -------- + + KafkaTopicPartitionState newPartition1 = new KafkaTopicPartitionState<>( + new KafkaTopicPartition(testTopic, 0), new TopicPartition(testTopic, 0)); + newPartition1.setOffset(23L); + + KafkaTopicPartitionState newPartition2 = new KafkaTopicPartitionState<>( + new KafkaTopicPartition(testTopic, 1), new TopicPartition(testTopic, 1)); + newPartition2.setOffset(31L); + + final List> newPartitions = new ArrayList<>(2); + newPartitions.add(newPartition1); + newPartitions.add(newPartition2); + + // -------- setup mock KafkaConsumer -------- + + // no initial assignment + final Map mockConsumerAssignmentsAndPositions = new LinkedHashMap<>(); + + final KafkaConsumer mockConsumer = createMockConsumer( + mockConsumerAssignmentsAndPositions, + Collections.emptyMap(), + false, + null, + null); + + // -------- setup new partitions to be polled from the unassigned partitions queue -------- + + final ClosableBlockingQueue> unassignedPartitionsQueue = + new ClosableBlockingQueue<>(); + + for (KafkaTopicPartitionState newPartition : newPartitions) { + unassignedPartitionsQueue.add(newPartition); + } + + // -------- start test -------- + + final TestKafkaConsumerThread testThread = + new TestKafkaConsumerThread(mockConsumer, unassignedPartitionsQueue, new Handover()); + testThread.start(); + + testThread.startPartitionReassignment(); + testThread.waitPartitionReassignmentComplete(); + + // verify that the consumer called assign() with all new partitions, and that positions are correctly advanced + + assertEquals(newPartitions.size(), mockConsumerAssignmentsAndPositions.size()); + + for (KafkaTopicPartitionState newPartition : newPartitions) { + assertTrue(mockConsumerAssignmentsAndPositions.containsKey(newPartition.getKafkaPartitionHandle())); + + // should be seeked to (offset in state + 1) because offsets in state represent the last processed record + assertEquals( + newPartition.getOffset() + 1, + mockConsumerAssignmentsAndPositions.get(newPartition.getKafkaPartitionHandle()).longValue()); + } + + assertEquals(0, unassignedPartitionsQueue.size()); + } + + /** + * Tests reassignment works correctly in the case when: + * - the consumer initially had no assignments + * - new unassigned partitions have undefined offsets (e.g. EARLIEST_OFFSET sentinel value) + * + * Setting a timeout because the test will not finish if there is logic error with + * the reassignment flow. + */ + @SuppressWarnings("unchecked") + @Test(timeout = 10000) + public void testReassigningPartitionsWithoutDefinedOffsetsWhenNoInitialAssignment() throws Exception { + final String testTopic = "test-topic"; + + // -------- new partitions with undefined offsets -------- + + KafkaTopicPartitionState newPartition1 = new KafkaTopicPartitionState<>( + new KafkaTopicPartition(testTopic, 0), new TopicPartition(testTopic, 0)); + newPartition1.setOffset(KafkaTopicPartitionStateSentinel.EARLIEST_OFFSET); + + KafkaTopicPartitionState newPartition2 = new KafkaTopicPartitionState<>( + new KafkaTopicPartition(testTopic, 1), new TopicPartition(testTopic, 1)); + newPartition2.setOffset(KafkaTopicPartitionStateSentinel.EARLIEST_OFFSET); + + final List> newPartitions = new ArrayList<>(2); + newPartitions.add(newPartition1); + newPartitions.add(newPartition2); + + // -------- setup mock KafkaConsumer -------- + + // no initial assignment + final Map mockConsumerAssignmentsAndPositions = new LinkedHashMap<>(); + + // mock retrieved values that should replace the EARLIEST_OFFSET sentinels + final Map mockRetrievedPositions = new HashMap<>(); + mockRetrievedPositions.put(newPartition1.getKafkaPartitionHandle(), 23L); + mockRetrievedPositions.put(newPartition2.getKafkaPartitionHandle(), 32L); + + final KafkaConsumer mockConsumer = createMockConsumer( + mockConsumerAssignmentsAndPositions, + mockRetrievedPositions, + false, + null, + null); + + // -------- setup new partitions to be polled from the unassigned partitions queue -------- + + final ClosableBlockingQueue> unassignedPartitionsQueue = + new ClosableBlockingQueue<>(); + + for (KafkaTopicPartitionState newPartition : newPartitions) { + unassignedPartitionsQueue.add(newPartition); + } + + // -------- start test -------- + + final TestKafkaConsumerThread testThread = + new TestKafkaConsumerThread(mockConsumer, unassignedPartitionsQueue, new Handover()); + testThread.start(); + + testThread.startPartitionReassignment(); + testThread.waitPartitionReassignmentComplete(); + + // the sentinel offset states should have been replaced with defined values according to the retrieved values + assertEquals(mockRetrievedPositions.get(newPartition1.getKafkaPartitionHandle()) - 1, newPartition1.getOffset()); + assertEquals(mockRetrievedPositions.get(newPartition2.getKafkaPartitionHandle()) - 1, newPartition2.getOffset()); + + // verify that the consumer called assign() with all new partitions, and that positions are correctly advanced + + assertEquals(newPartitions.size(), mockConsumerAssignmentsAndPositions.size()); + + for (KafkaTopicPartitionState newPartition : newPartitions) { + assertTrue(mockConsumerAssignmentsAndPositions.containsKey(newPartition.getKafkaPartitionHandle())); + + // should be seeked to (offset in state + 1) because offsets in state represent the last processed record + assertEquals( + newPartition.getOffset() + 1, + mockConsumerAssignmentsAndPositions.get(newPartition.getKafkaPartitionHandle()).longValue()); + } + + assertEquals(0, unassignedPartitionsQueue.size()); + } + + /** + * Tests reassignment works correctly in the case when: + * - the consumer already have some assignments + * - new unassigned partitions already have defined offsets + * + * Setting a timeout because the test will not finish if there is logic error with + * the reassignment flow. + */ + @SuppressWarnings("unchecked") + @Test(timeout = 10000) + public void testReassigningPartitionsWithDefinedOffsets() throws Exception { + final String testTopic = "test-topic"; + + // -------- old partitions -------- + + KafkaTopicPartitionState oldPartition1 = new KafkaTopicPartitionState<>( + new KafkaTopicPartition(testTopic, 0), new TopicPartition(testTopic, 0)); + oldPartition1.setOffset(23L); + + KafkaTopicPartitionState oldPartition2 = new KafkaTopicPartitionState<>( + new KafkaTopicPartition(testTopic, 1), new TopicPartition(testTopic, 1)); + oldPartition2.setOffset(32L); + + List> oldPartitions = new ArrayList<>(2); + oldPartitions.add(oldPartition1); + oldPartitions.add(oldPartition2); + + // -------- new partitions with defined offsets -------- + + KafkaTopicPartitionState newPartition = new KafkaTopicPartitionState<>( + new KafkaTopicPartition(testTopic, 2), new TopicPartition(testTopic, 2)); + newPartition.setOffset(29L); + + List> totalPartitions = new ArrayList<>(3); + totalPartitions.add(oldPartition1); + totalPartitions.add(oldPartition2); + totalPartitions.add(newPartition); + + // -------- setup mock KafkaConsumer -------- + + // has initial assignments + final Map mockConsumerAssignmentsAndPositions = new HashMap<>(); + for (KafkaTopicPartitionState oldPartition : oldPartitions) { + mockConsumerAssignmentsAndPositions.put(oldPartition.getKafkaPartitionHandle(), oldPartition.getOffset() + 1); + } + + final KafkaConsumer mockConsumer = createMockConsumer( + mockConsumerAssignmentsAndPositions, + Collections.emptyMap(), + false, + null, + null); + + // -------- setup new partitions to be polled from the unassigned partitions queue -------- + + final ClosableBlockingQueue> unassignedPartitionsQueue = + new ClosableBlockingQueue<>(); + + unassignedPartitionsQueue.add(newPartition); + + // -------- start test -------- + + final TestKafkaConsumerThread testThread = + new TestKafkaConsumerThread(mockConsumer, unassignedPartitionsQueue, new Handover()); + testThread.start(); + + testThread.startPartitionReassignment(); + testThread.waitPartitionReassignmentComplete(); + + // verify that the consumer called assign() with all new partitions, and that positions are correctly advanced + + assertEquals(totalPartitions.size(), mockConsumerAssignmentsAndPositions.size()); + + // old partitions should be re-seeked to their previous positions + for (KafkaTopicPartitionState partition : totalPartitions) { + assertTrue(mockConsumerAssignmentsAndPositions.containsKey(partition.getKafkaPartitionHandle())); + + // should be seeked to (offset in state + 1) because offsets in state represent the last processed record + assertEquals( + partition.getOffset() + 1, + mockConsumerAssignmentsAndPositions.get(partition.getKafkaPartitionHandle()).longValue()); + } + + assertEquals(0, unassignedPartitionsQueue.size()); + } + + /** + * Tests reassignment works correctly in the case when: + * - the consumer already have some assignments + * - new unassigned partitions have undefined offsets (e.g. EARLIEST_OFFSET sentinel value) + * + * Setting a timeout because the test will not finish if there is logic error with + * the reassignment flow. + */ + @SuppressWarnings("unchecked") + @Test(timeout = 10000) + public void testReassigningPartitionsWithoutDefinedOffsets() throws Exception { + final String testTopic = "test-topic"; + + // -------- old partitions -------- + + KafkaTopicPartitionState oldPartition1 = new KafkaTopicPartitionState<>( + new KafkaTopicPartition(testTopic, 0), new TopicPartition(testTopic, 0)); + oldPartition1.setOffset(23L); + + KafkaTopicPartitionState oldPartition2 = new KafkaTopicPartitionState<>( + new KafkaTopicPartition(testTopic, 1), new TopicPartition(testTopic, 1)); + oldPartition2.setOffset(32L); + + List> oldPartitions = new ArrayList<>(2); + oldPartitions.add(oldPartition1); + oldPartitions.add(oldPartition2); + + // -------- new partitions with undefined offsets -------- + + KafkaTopicPartitionState newPartition = new KafkaTopicPartitionState<>( + new KafkaTopicPartition(testTopic, 2), new TopicPartition(testTopic, 2)); + newPartition.setOffset(KafkaTopicPartitionStateSentinel.EARLIEST_OFFSET); + + List> totalPartitions = new ArrayList<>(3); + totalPartitions.add(oldPartition1); + totalPartitions.add(oldPartition2); + totalPartitions.add(newPartition); + + // -------- setup mock KafkaConsumer -------- + + // has initial assignments + final Map mockConsumerAssignmentsAndPositions = new HashMap<>(); + for (KafkaTopicPartitionState oldPartition : oldPartitions) { + mockConsumerAssignmentsAndPositions.put(oldPartition.getKafkaPartitionHandle(), oldPartition.getOffset() + 1); + } + + // mock retrieved values that should replace the EARLIEST_OFFSET sentinels + final Map mockRetrievedPositions = new HashMap<>(); + mockRetrievedPositions.put(newPartition.getKafkaPartitionHandle(), 30L); + + final KafkaConsumer mockConsumer = createMockConsumer( + mockConsumerAssignmentsAndPositions, + mockRetrievedPositions, + false, + null, + null); + + // -------- setup new partitions to be polled from the unassigned partitions queue -------- + + final ClosableBlockingQueue> unassignedPartitionsQueue = + new ClosableBlockingQueue<>(); + + unassignedPartitionsQueue.add(newPartition); + + // -------- start test -------- + + final TestKafkaConsumerThread testThread = + new TestKafkaConsumerThread(mockConsumer, unassignedPartitionsQueue, new Handover()); + testThread.start(); + + testThread.startPartitionReassignment(); + testThread.waitPartitionReassignmentComplete(); + + // the sentinel offset states should have been replaced with defined values according to the retrieved positions + assertEquals(mockRetrievedPositions.get(newPartition.getKafkaPartitionHandle()) - 1, newPartition.getOffset()); + + // verify that the consumer called assign() with all new partitions, and that positions are correctly advanced + + assertEquals(totalPartitions.size(), mockConsumerAssignmentsAndPositions.size()); + + // old partitions should be re-seeked to their previous positions + for (KafkaTopicPartitionState partition : totalPartitions) { + assertTrue(mockConsumerAssignmentsAndPositions.containsKey(partition.getKafkaPartitionHandle())); + + // should be seeked to (offset in state + 1) because offsets in state represent the last processed record + assertEquals( + partition.getOffset() + 1, + mockConsumerAssignmentsAndPositions.get(partition.getKafkaPartitionHandle()).longValue()); + } + + assertEquals(0, unassignedPartitionsQueue.size()); + } + + /** + * Tests reassignment works correctly in the case when: + * - the consumer already have some assignments + * - new unassigned partitions already have defined offsets + * - the consumer was woken up prior to the reassignment + * + * In this case, reassignment should not have occurred at all, and the consumer retains the original assignment. + * + * Setting a timeout because the test will not finish if there is logic error with + * the reassignment flow. + */ + @SuppressWarnings("unchecked") + @Test(timeout = 10000) + public void testReassigningPartitionsWithDefinedOffsetsWhenEarlyWakeup() throws Exception { + final String testTopic = "test-topic"; + + // -------- old partitions -------- + + KafkaTopicPartitionState oldPartition1 = new KafkaTopicPartitionState<>( + new KafkaTopicPartition(testTopic, 0), new TopicPartition(testTopic, 0)); + oldPartition1.setOffset(23L); + + KafkaTopicPartitionState oldPartition2 = new KafkaTopicPartitionState<>( + new KafkaTopicPartition(testTopic, 1), new TopicPartition(testTopic, 1)); + oldPartition2.setOffset(32L); + + List> oldPartitions = new ArrayList<>(2); + oldPartitions.add(oldPartition1); + oldPartitions.add(oldPartition2); + + // -------- new partitions with defined offsets -------- + + KafkaTopicPartitionState newPartition = new KafkaTopicPartitionState<>( + new KafkaTopicPartition(testTopic, 2), new TopicPartition(testTopic, 2)); + newPartition.setOffset(29L); + + // -------- setup mock KafkaConsumer -------- + + // initial assignments + final Map mockConsumerAssignmentsToPositions = new LinkedHashMap<>(); + for (KafkaTopicPartitionState oldPartition : oldPartitions) { + mockConsumerAssignmentsToPositions.put(oldPartition.getKafkaPartitionHandle(), oldPartition.getOffset() + 1); + } + + final KafkaConsumer mockConsumer = createMockConsumer( + mockConsumerAssignmentsToPositions, + Collections.emptyMap(), + true, + null, + null); + + // -------- setup new partitions to be polled from the unassigned partitions queue -------- + + final ClosableBlockingQueue> unassignedPartitionsQueue = + new ClosableBlockingQueue<>(); + + unassignedPartitionsQueue.add(newPartition); + + // -------- start test -------- + + final TestKafkaConsumerThread testThread = + new TestKafkaConsumerThread(mockConsumer, unassignedPartitionsQueue, new Handover()); + testThread.start(); + + // pause just before the reassignment so we can inject the wakeup + testThread.waitPartitionReassignmentInvoked(); + + testThread.setOffsetsToCommit(new HashMap()); + verify(mockConsumer, times(1)).wakeup(); + + testThread.startPartitionReassignment(); + testThread.waitPartitionReassignmentComplete(); + + // the consumer's assignment should have remained untouched + + assertEquals(oldPartitions.size(), mockConsumerAssignmentsToPositions.size()); + + for (KafkaTopicPartitionState oldPartition : oldPartitions) { + assertTrue(mockConsumerAssignmentsToPositions.containsKey(oldPartition.getKafkaPartitionHandle())); + assertEquals( + oldPartition.getOffset() + 1, + mockConsumerAssignmentsToPositions.get(oldPartition.getKafkaPartitionHandle()).longValue()); + } + + // the new partitions should have been re-added to the unassigned partitions queue + assertEquals(1, unassignedPartitionsQueue.size()); + } + + /** + * Tests reassignment works correctly in the case when: + * - the consumer has no initial assignments + * - new unassigned partitions have undefined offsets + * - the consumer was woken up prior to the reassignment + * + * In this case, reassignment should not have occurred at all, and the consumer retains the original assignment. + * + * Setting a timeout because the test will not finish if there is logic error with + * the reassignment flow. + */ + @SuppressWarnings("unchecked") + @Test(timeout = 10000) + public void testReassignPartitionsDefinedOffsetsWithoutInitialAssignmentsWhenEarlyWakeup() throws Exception { + final String testTopic = "test-topic"; + + // -------- new partitions with defined offsets -------- + + KafkaTopicPartitionState newPartition1 = new KafkaTopicPartitionState<>( + new KafkaTopicPartition(testTopic, 0), new TopicPartition(testTopic, 0)); + newPartition1.setOffset(KafkaTopicPartitionStateSentinel.EARLIEST_OFFSET); + + KafkaTopicPartitionState newPartition2 = new KafkaTopicPartitionState<>( + new KafkaTopicPartition(testTopic, 1), new TopicPartition(testTopic, 1)); + newPartition2.setOffset(KafkaTopicPartitionStateSentinel.EARLIEST_OFFSET); + + List> newPartitions = new ArrayList<>(2); + newPartitions.add(newPartition1); + newPartitions.add(newPartition2); + + // -------- setup mock KafkaConsumer -------- + + // no initial assignments + final Map mockConsumerAssignmentsAndPositions = new LinkedHashMap<>(); + + // mock retrieved values that should replace the EARLIEST_OFFSET sentinels + final Map mockRetrievedPositions = new HashMap<>(); + mockRetrievedPositions.put(newPartition1.getKafkaPartitionHandle(), 23L); + mockRetrievedPositions.put(newPartition2.getKafkaPartitionHandle(), 32L); + + final KafkaConsumer mockConsumer = createMockConsumer( + mockConsumerAssignmentsAndPositions, + mockRetrievedPositions, + true, + null, + null); + + // -------- setup new partitions to be polled from the unassigned partitions queue -------- + + final ClosableBlockingQueue> unassignedPartitionsQueue = + new ClosableBlockingQueue<>(); + + for (KafkaTopicPartitionState newPartition : newPartitions) { + unassignedPartitionsQueue.add(newPartition); + } + + // -------- start test -------- + + final TestKafkaConsumerThread testThread = + new TestKafkaConsumerThread(mockConsumer, unassignedPartitionsQueue, new Handover()); + testThread.start(); + + // pause just before the reassignment so we can inject the wakeup + testThread.waitPartitionReassignmentInvoked(); + + testThread.setOffsetsToCommit(new HashMap()); + + // make sure the consumer was actually woken up + verify(mockConsumer, times(1)).wakeup(); + + testThread.startPartitionReassignment(); + testThread.waitPartitionReassignmentComplete(); + + // the consumer's assignment should have remained untouched (in this case, empty) + assertEquals(0, mockConsumerAssignmentsAndPositions.size()); + + // the new partitions should have been re-added to the unassigned partitions queue + assertEquals(2, unassignedPartitionsQueue.size()); + } + + /** + * Tests reassignment works correctly in the case when: + * - the consumer has no initial assignments + * - new unassigned partitions have undefined offsets + * - the consumer was woken up during the reassignment + * + * In this case, reassignment should have completed, and the consumer is restored the wakeup call after the reassignment. + * + * Setting a timeout because the test will not finish if there is logic error with + * the reassignment flow. + */ + @SuppressWarnings("unchecked") + @Test(timeout = 10000) + public void testReassignPartitionsDefinedOffsetsWithoutInitialAssignmentsWhenWakeupMidway() throws Exception { + final String testTopic = "test-topic"; + + // -------- new partitions with defined offsets -------- + + KafkaTopicPartitionState newPartition1 = new KafkaTopicPartitionState<>( + new KafkaTopicPartition(testTopic, 0), new TopicPartition(testTopic, 0)); + newPartition1.setOffset(KafkaTopicPartitionStateSentinel.EARLIEST_OFFSET); + + KafkaTopicPartitionState newPartition2 = new KafkaTopicPartitionState<>( + new KafkaTopicPartition(testTopic, 1), new TopicPartition(testTopic, 1)); + newPartition2.setOffset(KafkaTopicPartitionStateSentinel.EARLIEST_OFFSET); + + List> newPartitions = new ArrayList<>(2); + newPartitions.add(newPartition1); + newPartitions.add(newPartition2); + + // -------- setup mock KafkaConsumer -------- + + // no initial assignments + final Map mockConsumerAssignmentsAndPositions = new LinkedHashMap<>(); + + // mock retrieved values that should replace the EARLIEST_OFFSET sentinels + final Map mockRetrievedPositions = new HashMap<>(); + mockRetrievedPositions.put(newPartition1.getKafkaPartitionHandle(), 23L); + mockRetrievedPositions.put(newPartition2.getKafkaPartitionHandle(), 32L); + + // these latches are used to pause midway the reassignment process + final OneShotLatch midAssignmentLatch = new OneShotLatch(); + final OneShotLatch continueAssigmentLatch = new OneShotLatch(); + + final KafkaConsumer mockConsumer = createMockConsumer( + mockConsumerAssignmentsAndPositions, + mockRetrievedPositions, + false, + midAssignmentLatch, + continueAssigmentLatch); + + // -------- setup new partitions to be polled from the unassigned partitions queue -------- + + final ClosableBlockingQueue> unassignedPartitionsQueue = + new ClosableBlockingQueue<>(); + + for (KafkaTopicPartitionState newPartition : newPartitions) { + unassignedPartitionsQueue.add(newPartition); + } + + // -------- start test -------- + + final TestKafkaConsumerThread testThread = + new TestKafkaConsumerThread(mockConsumer, unassignedPartitionsQueue, new Handover()); + testThread.start(); + + testThread.startPartitionReassignment(); + + // wait until the reassignment has started + midAssignmentLatch.await(); + + testThread.setOffsetsToCommit(new HashMap()); + + // the wakeup the setOffsetsToCommit() call should have been buffered, and not called on the consumer + verify(mockConsumer, never()).wakeup(); + + continueAssigmentLatch.trigger(); + + testThread.waitPartitionReassignmentComplete(); + + // verify that the consumer called assign() with all new partitions, and that positions are correctly advanced + + assertEquals(newPartitions.size(), mockConsumerAssignmentsAndPositions.size()); + + for (KafkaTopicPartitionState newPartition : newPartitions) { + assertTrue(mockConsumerAssignmentsAndPositions.containsKey(newPartition.getKafkaPartitionHandle())); + + // should be seeked to (offset in state + 1) because offsets in state represent the last processed record + assertEquals( + newPartition.getOffset() + 1, + mockConsumerAssignmentsAndPositions.get(newPartition.getKafkaPartitionHandle()).longValue()); + } + + // after the reassignment, the consumer should be restored the wakeup call + verify(mockConsumer, times(1)).wakeup(); + + assertEquals(0, unassignedPartitionsQueue.size()); + } + + /** + * A testable {@link KafkaConsumerThread} that injects multiple latches exactly before and after + * partition reassignment, so that tests are eligible to setup various conditions before the reassignment happens + * and inspect reqssignment results after it is completed. + */ + private static class TestKafkaConsumerThread extends KafkaConsumerThread { + + private final KafkaConsumer mockConsumer; + private final MultiShotLatch preReassignmentLatch = new MultiShotLatch(); + private final MultiShotLatch startReassignmentLatch = new MultiShotLatch(); + private final MultiShotLatch reassignmentCompleteLatch = new MultiShotLatch(); + private final MultiShotLatch postReassignmentLatch = new MultiShotLatch(); + + public TestKafkaConsumerThread( + KafkaConsumer mockConsumer, + ClosableBlockingQueue> unassignedPartitionsQueue, + Handover handover) { + + super( + mock(Logger.class), + handover, + new Properties(), + unassignedPartitionsQueue, + mock(MetricGroup.class), + new KafkaConsumerCallBridge(), + "test-kafka-consumer-thread", + 0, + false); + + this.mockConsumer = mockConsumer; + } + + public void waitPartitionReassignmentInvoked() throws InterruptedException { + preReassignmentLatch.await(); + } + + public void startPartitionReassignment() { + startReassignmentLatch.trigger(); + } + + public void waitPartitionReassignmentComplete() throws InterruptedException { + reassignmentCompleteLatch.await(); + } + + public void endPartitionReassignment() { + postReassignmentLatch.trigger(); + } + + @Override + KafkaConsumer getConsumer(Properties kafkaProperties) { + return mockConsumer; + } + + @Override + void reassignPartitions(List> newPartitions) throws Exception { + // triggers blocking calls on waitPartitionReassignmentInvoked() + preReassignmentLatch.trigger(); + + // waits for startPartitionReassignment() to be called + startReassignmentLatch.await(); + + try { + super.reassignPartitions(newPartitions); + } finally { + // triggers blocking calls on waitPartitionReassignmentComplete() + reassignmentCompleteLatch.trigger(); + + // waits for endPartitionReassignment() to be called + postReassignmentLatch.await(); + } + } + } + + @SuppressWarnings("unchecked") + private static KafkaConsumer createMockConsumer( + final Map mockConsumerAssignmentAndPosition, + final Map mockRetrievedPositions, + final boolean earlyWakeup, + final OneShotLatch midAssignmentLatch, + final OneShotLatch continueAssignmentLatch) { + + final KafkaConsumer mockConsumer = mock(KafkaConsumer.class); + when(mockConsumer.assignment()).thenAnswer(new Answer() { + @Override + public Object answer(InvocationOnMock invocationOnMock) throws Throwable { + if (midAssignmentLatch != null) { + midAssignmentLatch.trigger(); + } + + if (continueAssignmentLatch != null) { + continueAssignmentLatch.await(); + } + + return mockConsumerAssignmentAndPosition.keySet(); + } + }); + + when(mockConsumer.poll(anyLong())).thenReturn(mock(ConsumerRecords.class)); + + if (!earlyWakeup) { + when(mockConsumer.position(any(TopicPartition.class))).thenAnswer(new Answer() { + @Override + public Object answer(InvocationOnMock invocationOnMock) throws Throwable { + return mockConsumerAssignmentAndPosition.get(invocationOnMock.getArgumentAt(0, TopicPartition.class)); + } + }); + } else { + when(mockConsumer.position(any(TopicPartition.class))).thenThrow(new WakeupException()); + } + + doAnswer(new Answer() { + @Override + public Object answer(InvocationOnMock invocationOnMock) throws Throwable { + mockConsumerAssignmentAndPosition.clear(); + + List assignedPartitions = invocationOnMock.getArgumentAt(0, List.class); + for (TopicPartition assigned : assignedPartitions) { + mockConsumerAssignmentAndPosition.put(assigned, null); + } + return null; + } + }).when(mockConsumer).assign(anyListOf(TopicPartition.class)); + + doAnswer(new Answer() { + @Override + public Object answer(InvocationOnMock invocationOnMock) throws Throwable { + TopicPartition partition = invocationOnMock.getArgumentAt(0, TopicPartition.class); + long position = invocationOnMock.getArgumentAt(1, long.class); + + if (!mockConsumerAssignmentAndPosition.containsKey(partition)) { + throw new Exception("the current mock assignment does not contain partition " + partition); + } else { + mockConsumerAssignmentAndPosition.put(partition, position); + } + return null; + } + }).when(mockConsumer).seek(any(TopicPartition.class), anyLong()); + + doAnswer(new Answer() { + @Override + public Object answer(InvocationOnMock invocationOnMock) throws Throwable { + TopicPartition partition = invocationOnMock.getArgumentAt(0, TopicPartition.class); + + if (!mockConsumerAssignmentAndPosition.containsKey(partition)) { + throw new Exception("the current mock assignment does not contain partition " + partition); + } else { + Long mockRetrievedPosition = mockRetrievedPositions.get(partition); + if (mockRetrievedPosition == null) { + throw new Exception("mock consumer needed to retrieve a position, but no value was provided in the mock values for retrieval"); + } else { + mockConsumerAssignmentAndPosition.put(partition, mockRetrievedPositions.get(partition)); + } + } + return null; + } + }).when(mockConsumer).seekToBeginning(any(TopicPartition.class)); + + doAnswer(new Answer() { + @Override + public Object answer(InvocationOnMock invocationOnMock) throws Throwable { + TopicPartition partition = invocationOnMock.getArgumentAt(0, TopicPartition.class); + + if (!mockConsumerAssignmentAndPosition.containsKey(partition)) { + throw new Exception("the current mock assignment does not contain partition " + partition); + } else { + Long mockRetrievedPosition = mockRetrievedPositions.get(partition); + if (mockRetrievedPosition == null) { + throw new Exception("mock consumer needed to retrieve a position, but no value was provided in the mock values for retrieval"); + } else { + mockConsumerAssignmentAndPosition.put(partition, mockRetrievedPositions.get(partition)); + } + } + return null; + } + }).when(mockConsumer).seekToEnd(any(TopicPartition.class)); + + return mockConsumer; + } +} diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java index a35e7104ecbf6..4c58ea6d698c5 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java @@ -36,15 +36,15 @@ import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; -import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitModes; import org.apache.flink.streaming.connectors.kafka.config.StartupMode; import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractPartitionDiscoverer; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionStateSentinel; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicsDescriptor; import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; -import org.apache.flink.util.Preconditions; import org.apache.flink.util.SerializedValue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -53,6 +53,8 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; +import java.util.regex.Pattern; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -79,15 +81,22 @@ public abstract class FlinkKafkaConsumerBase extends RichParallelSourceFuncti /** The maximum number of pending non-committed checkpoints to track, to avoid memory leaks */ public static final int MAX_NUM_PENDING_CHECKPOINTS = 100; + /** The default interval to execute partition discovery, in milliseconds */ + public static final long DEFAULT_PARTITION_DISCOVERY_INTERVAL_MILLIS = 10000L; + /** Boolean configuration key to disable metrics tracking **/ public static final String KEY_DISABLE_METRICS = "flink.disable-metrics"; + /** Configuration key to define the consumer's partition discovery interval, in milliseconds */ + public static final String KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS = "flink.partition-discovery.interval-millis"; + // ------------------------------------------------------------------------ // configuration state, set on the client relevant for all subtasks // ------------------------------------------------------------------------ - private final List topics; - + /** Describes whether we are discovering partitions for fixed topics or a topic pattern */ + private final KafkaTopicsDescriptor topicsDescriptor; + /** The schema to convert between Kafka's byte messages, and Flink's objects */ protected final KeyedDeserializationSchema deserializer; @@ -119,6 +128,9 @@ public abstract class FlinkKafkaConsumerBase extends RichParallelSourceFuncti */ private OffsetCommitMode offsetCommitMode; + /** User configured value for discovery interval, in milliseconds */ + private final long discoveryIntervalMillis; + /** The startup mode for the consumer (default is {@link StartupMode#GROUP_OFFSETS}) */ private StartupMode startupMode = StartupMode.GROUP_OFFSETS; @@ -134,11 +146,17 @@ public abstract class FlinkKafkaConsumerBase extends RichParallelSourceFuncti /** The fetcher implements the connections to the Kafka brokers */ private transient volatile AbstractFetcher kafkaFetcher; - + + /** The partition discoverer, used to find new partitions */ + private transient volatile AbstractPartitionDiscoverer partitionDiscoverer; + /** The offsets to restore to, if the consumer restores state from a checkpoint */ private transient volatile HashMap restoredState; - - /** Flag indicating whether the consumer is still running **/ + + /** Reference to the thread that invoked run(). This is used to interrupt partition discovery on shutdown */ + private transient volatile Thread runThread; + + /** Flag indicating whether the consumer is still running */ private volatile boolean running = true; // ------------------------------------------------------------------------ @@ -146,13 +164,20 @@ public abstract class FlinkKafkaConsumerBase extends RichParallelSourceFuncti /** * Base constructor. * - * @param deserializer - * The deserializer to turn raw byte messages into Java/Scala objects. + * @param topics fixed list of topics to subscribe to (null, if using topic pattern) + * @param topicPattern the topic pattern to subscribe to (null, if using fixed topics) + * @param deserializer The deserializer to turn raw byte messages into Java/Scala objects. */ - public FlinkKafkaConsumerBase(List topics, KeyedDeserializationSchema deserializer) { - this.topics = checkNotNull(topics); - checkArgument(topics.size() > 0, "You have to define at least one topic."); + public FlinkKafkaConsumerBase( + List topics, + Pattern topicPattern, + KeyedDeserializationSchema deserializer, + long discoveryIntervalMillis) { + this.topicsDescriptor = new KafkaTopicsDescriptor(topics, topicPattern); this.deserializer = checkNotNull(deserializer, "valueDeserializer"); + + checkArgument(discoveryIntervalMillis >= 0); + this.discoveryIntervalMillis = discoveryIntervalMillis; } // ------------------------------------------------------------------------ @@ -324,54 +349,61 @@ public FlinkKafkaConsumerBase setStartFromSpecificOffsets(Map kafkaTopicPartitions = getKafkaPartitions(topics); - Preconditions.checkNotNull(kafkaTopicPartitions, "TopicPartitions must not be null."); + // create the partition discoverer + this.partitionDiscoverer = createPartitionDiscoverer( + topicsDescriptor, + getRuntimeContext().getIndexOfThisSubtask(), + getRuntimeContext().getNumberOfParallelSubtasks()); + this.partitionDiscoverer.open(); - subscribedPartitionsToStartOffsets = new HashMap<>(kafkaTopicPartitions.size()); + subscribedPartitionsToStartOffsets = new HashMap<>(); if (restoredState != null) { - for (KafkaTopicPartition kafkaTopicPartition : kafkaTopicPartitions) { - if (restoredState.containsKey(kafkaTopicPartition)) { - subscribedPartitionsToStartOffsets.put(kafkaTopicPartition, restoredState.get(kafkaTopicPartition)); - } + for (Map.Entry restoredStateEntry : restoredState.entrySet()) { + subscribedPartitionsToStartOffsets.put(restoredStateEntry.getKey(), restoredStateEntry.getValue()); + + // we also need to update what partitions the partition discoverer has seen already + // TODO Currently, after a restore, partition discovery will not work correctly + // TODO because the discoverer can not fully recover all globally seen records + partitionDiscoverer.checkAndSetDiscoveredPartition(restoredStateEntry.getKey()); } LOG.info("Consumer subtask {} will start reading {} partitions with offsets in restored state: {}", getRuntimeContext().getIndexOfThisSubtask(), subscribedPartitionsToStartOffsets.size(), subscribedPartitionsToStartOffsets); } else { - initializeSubscribedPartitionsToStartOffsets( - subscribedPartitionsToStartOffsets, - kafkaTopicPartitions, - getRuntimeContext().getIndexOfThisSubtask(), - getRuntimeContext().getNumberOfParallelSubtasks(), - startupMode, - specificStartupOffsets); + // use the partition discoverer to fetch the initial seed partitions, + // and set their initial offsets depending on the startup mode + for (KafkaTopicPartition seedPartition : partitionDiscoverer.discoverPartitions()) { + if (startupMode != StartupMode.SPECIFIC_OFFSETS) { + subscribedPartitionsToStartOffsets.put(seedPartition, startupMode.getStateSentinel()); + } else { + if (specificStartupOffsets == null) { + throw new IllegalArgumentException( + "Startup mode for the consumer set to " + StartupMode.SPECIFIC_OFFSETS + + ", but no specific offsets were specified"); + } + + Long specificOffset = specificStartupOffsets.get(seedPartition); + if (specificOffset != null) { + // since the specified offsets represent the next record to read, we subtract + // it by one so that the initial state of the consumer will be correct + subscribedPartitionsToStartOffsets.put(seedPartition, specificOffset - 1); + } else { + // default to group offset behaviour if the user-provided specific offsets + // do not contain a value for this partition + subscribedPartitionsToStartOffsets.put(seedPartition, KafkaTopicPartitionStateSentinel.GROUP_OFFSET); + } + } + } - if (subscribedPartitionsToStartOffsets.size() != 0) { + if (!subscribedPartitionsToStartOffsets.isEmpty()) { switch (startupMode) { case EARLIEST: LOG.info("Consumer subtask {} will start reading the following {} partitions from the earliest offsets: {}", @@ -414,6 +446,9 @@ public void open(Configuration configuration) { subscribedPartitionsToStartOffsets.size(), subscribedPartitionsToStartOffsets.keySet()); } + } else { + LOG.info("Consumer subtask {} initially has no partitions to read from.", + getRuntimeContext().getIndexOfThisSubtask()); } } } @@ -424,65 +459,122 @@ public void run(SourceContext sourceContext) throws Exception { throw new Exception("The partitions were not set for the consumer"); } - // we need only do work, if we actually have partitions assigned - if (!subscribedPartitionsToStartOffsets.isEmpty()) { - - // create the fetcher that will communicate with the Kafka brokers - final AbstractFetcher fetcher = createFetcher( - sourceContext, - subscribedPartitionsToStartOffsets, - periodicWatermarkAssigner, - punctuatedWatermarkAssigner, - (StreamingRuntimeContext) getRuntimeContext(), - offsetCommitMode); - - // publish the reference, for snapshot-, commit-, and cancel calls - // IMPORTANT: We can only do that now, because only now will calls to - // the fetchers 'snapshotCurrentState()' method return at least - // the restored offsets - this.kafkaFetcher = fetcher; - if (!running) { - return; - } - - // (3) run the fetcher' main work method - fetcher.runFetchLoop(); + this.runThread = Thread.currentThread(); + + // mark the subtask as temporarily idle if there are no initial seed partitions; + // once this subtask discovers some partitions and starts collecting records, the subtask's + // status will automatically be triggered back to be active. + if (subscribedPartitionsToStartOffsets.isEmpty()) { + sourceContext.markAsTemporarilyIdle(); } - else { - // this source never completes, so emit a Long.MAX_VALUE watermark - // to not block watermark forwarding - sourceContext.emitWatermark(new Watermark(Long.MAX_VALUE)); - - // wait until this is canceled - final Object waitLock = new Object(); - while (running) { + + // create the fetcher that will communicate with the Kafka brokers + final AbstractFetcher fetcher = createFetcher( + sourceContext, + subscribedPartitionsToStartOffsets, + periodicWatermarkAssigner, + punctuatedWatermarkAssigner, + (StreamingRuntimeContext) getRuntimeContext(), + offsetCommitMode); + + // publish the reference, for snapshot-, commit-, and cancel calls + // IMPORTANT: We can only do that now, because only now will calls to + // the fetchers 'snapshotCurrentState()' method return at least + // the restored offsets + this.kafkaFetcher = fetcher; + + if (!running) { + return; + } + + final AtomicReference fetcherErrorRef = new AtomicReference<>(); + Thread fetcherThread = new Thread(new Runnable() { + @Override + public void run() { try { - //noinspection SynchronizationOnLocalVariableOrMethodParameter - synchronized (waitLock) { - waitLock.wait(); - } + // run the fetcher' main work method + kafkaFetcher.runFetchLoop(); + } catch (Exception e) { + fetcherErrorRef.set(e); + } finally { + // calling cancel will also let the partition discovery loop escape + cancel(); } - catch (InterruptedException e) { - if (!running) { - // restore the interrupted state, and fall through the loop - Thread.currentThread().interrupt(); - } + } + }); + fetcherThread.start(); + + // --------------------- partition discovery loop --------------------- + + List discoveredPartitions; + + // throughout the loop, we always eagerly check if we are still running before + // performing the next operation, so that we can escape the loop as soon as possible + while (running) { + if (LOG.isDebugEnabled()) { + LOG.debug("Consumer subtask {} is trying to discover new partitions ..."); + } + + try { + discoveredPartitions = partitionDiscoverer.discoverPartitions(); + } catch (AbstractPartitionDiscoverer.WakeupException | AbstractPartitionDiscoverer.ClosedException e) { + // the partition discoverer may have been closed or woken up before or during the discovery; + // this would only happen if the consumer was canceled; simply escape the loop + break; + } + + // no need to add the discovered partitions if we were closed during the meantime + if (running && !discoveredPartitions.isEmpty()) { + fetcher.addDiscoveredPartitions(discoveredPartitions); + } + + // do not waste any time sleeping if we're not running anymore + if (running && discoveryIntervalMillis != 0) { + try { + Thread.sleep(discoveryIntervalMillis); + } catch (InterruptedException iex) { + // may be interrupted if the consumer was canceled midway; simply escape the loop + break; } } } + + // -------------------------------------------------------------------- + + // make sure that the partition discoverer is properly closed + partitionDiscoverer.close(); + + fetcherThread.join(); + + // rethrow any fetcher errors + final Exception fetcherError = fetcherErrorRef.get(); + if (fetcherError != null) { + throw new RuntimeException(fetcherError); + } } @Override public void cancel() { - // set ourselves as not running + // set ourselves as not running; + // this would let the main discovery loop escape as soon as possible running = false; - + + if (partitionDiscoverer != null) { + // we cannot close the discoverer here, as it is error-prone to concurrent access; + // only wakeup the discoverer, the discovery loop will clean itself up after it escapes + partitionDiscoverer.wakeup(); + } + + // the main discovery loop may currently be sleeping in-between + // consecutive discoveries; interrupt to shutdown faster + if (runThread != null) { + runThread.interrupt(); + } + // abort the fetcher, if there is one if (kafkaFetcher != null) { kafkaFetcher.cancel(); } - - // there will be an interrupt() call to the main thread anyways } @Override @@ -657,10 +749,22 @@ public void notifyCheckpointComplete(long checkpointId) throws Exception { StreamingRuntimeContext runtimeContext, OffsetCommitMode offsetCommitMode) throws Exception; - protected abstract List getKafkaPartitions(List topics); + /** + * Creates the partition discoverer that is used to find new partitions for this subtask. + * + * @param topicsDescriptor Descriptor that describes whether we are discovering partitions for fixed topics or a topic pattern. + * @param indexOfThisSubtask The index of this consumer subtask. + * @param numParallelSubtasks The total number of parallel consumer subtasks. + * + * @return The instantiated partition discoverer + */ + protected abstract AbstractPartitionDiscoverer createPartitionDiscoverer( + KafkaTopicsDescriptor topicsDescriptor, + int indexOfThisSubtask, + int numParallelSubtasks); protected abstract boolean getIsAutoCommitEnabled(); - + // ------------------------------------------------------------------------ // ResultTypeQueryable methods // ------------------------------------------------------------------------ @@ -671,94 +775,9 @@ public TypeInformation getProducedType() { } // ------------------------------------------------------------------------ - // Utilities + // Test utilities // ------------------------------------------------------------------------ - /** - * Initializes {@link FlinkKafkaConsumerBase#subscribedPartitionsToStartOffsets} with appropriate - * values. The method decides which partitions this consumer instance should subscribe to, and also - * sets the initial offset each subscribed partition should be started from based on the configured startup mode. - * - * @param subscribedPartitionsToStartOffsets to subscribedPartitionsToStartOffsets to initialize - * @param kafkaTopicPartitions the complete list of all Kafka partitions - * @param indexOfThisSubtask the index of this consumer instance - * @param numParallelSubtasks total number of parallel consumer instances - * @param startupMode the configured startup mode for the consumer - * @param specificStartupOffsets specific partition offsets to start from - * (only relevant if startupMode is {@link StartupMode#SPECIFIC_OFFSETS}) - * - * Note: This method is also exposed for testing. - */ - protected static void initializeSubscribedPartitionsToStartOffsets( - Map subscribedPartitionsToStartOffsets, - List kafkaTopicPartitions, - int indexOfThisSubtask, - int numParallelSubtasks, - StartupMode startupMode, - Map specificStartupOffsets) { - - for (int i = 0; i < kafkaTopicPartitions.size(); i++) { - if (i % numParallelSubtasks == indexOfThisSubtask) { - if (startupMode != StartupMode.SPECIFIC_OFFSETS) { - subscribedPartitionsToStartOffsets.put(kafkaTopicPartitions.get(i), startupMode.getStateSentinel()); - } else { - if (specificStartupOffsets == null) { - throw new IllegalArgumentException( - "Startup mode for the consumer set to " + StartupMode.SPECIFIC_OFFSETS + - ", but no specific offsets were specified"); - } - - KafkaTopicPartition partition = kafkaTopicPartitions.get(i); - - Long specificOffset = specificStartupOffsets.get(partition); - if (specificOffset != null) { - // since the specified offsets represent the next record to read, we subtract - // it by one so that the initial state of the consumer will be correct - subscribedPartitionsToStartOffsets.put(partition, specificOffset - 1); - } else { - subscribedPartitionsToStartOffsets.put(partition, KafkaTopicPartitionStateSentinel.GROUP_OFFSET); - } - } - } - } - } - - /** - * Logs the partition information in INFO level. - * - * @param logger The logger to log to. - * @param partitionInfos List of subscribed partitions - */ - protected static void logPartitionInfo(Logger logger, List partitionInfos) { - Map countPerTopic = new HashMap<>(); - for (KafkaTopicPartition partition : partitionInfos) { - Integer count = countPerTopic.get(partition.getTopic()); - if (count == null) { - count = 1; - } else { - count++; - } - countPerTopic.put(partition.getTopic(), count); - } - StringBuilder sb = new StringBuilder( - "Consumer is going to read the following topics (with number of partitions): "); - - for (Map.Entry e : countPerTopic.entrySet()) { - sb.append(e.getKey()).append(" (").append(e.getValue()).append("), "); - } - - logger.info(sb.toString()); - } - - @VisibleForTesting - void setSubscribedPartitions(List allSubscribedPartitions) { - checkNotNull(allSubscribedPartitions); - this.subscribedPartitionsToStartOffsets = new HashMap<>(); - for (KafkaTopicPartition partition : allSubscribedPartitions) { - this.subscribedPartitionsToStartOffsets.put(partition, null); - } - } - @VisibleForTesting Map getSubscribedPartitionsToStartOffsets() { return subscribedPartitionsToStartOffsets; diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java index 0b311a9bafab9..996339c81e1b0 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java @@ -31,6 +31,8 @@ import java.io.IOException; import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; import java.util.Map; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -62,7 +64,18 @@ public abstract class AbstractFetcher { protected final Object checkpointLock; /** All partitions (and their state) that this fetcher is subscribed to */ - private final KafkaTopicPartitionState[] subscribedPartitionStates; + private final List> subscribedPartitionStates; + + /** + * Queue of partitions that are not yet assigned to any Kafka clients for consuming. + * Kafka version-specific implementations of {@link AbstractFetcher#runFetchLoop()} + * should continuously poll this queue for unassigned partitions, and start consuming + * them accordingly. + * + * All partitions added to this queue are guaranteed to have been added + * to {@link #subscribedPartitionStates} already. + */ + protected final ClosableBlockingQueue> unassignedPartitionsQueue; /** The mode describing whether the fetcher also generates timestamps and watermarks */ protected final int timestampWatermarkMode; @@ -70,6 +83,23 @@ public abstract class AbstractFetcher { /** Flag whether to register metrics for the fetcher */ protected final boolean useMetrics; + /** + * Optional timestamp extractor / watermark generator that will be run per Kafka partition, + * to exploit per-partition timestamp characteristics. + * The assigner is kept in serialized form, to deserialize it into multiple copies. + */ + private final SerializedValue> watermarksPeriodic; + + /** + * Optional timestamp extractor / watermark generator that will be run per Kafka partition, + * to exploit per-partition timestamp characteristics. + * The assigner is kept in serialized form, to deserialize it into multiple copies. + */ + private final SerializedValue> watermarksPunctuated; + + /** User class loader used to deserialize watermark assigners. */ + private final ClassLoader userCodeClassLoader; + /** Only relevant for punctuated watermarks: The current cross partition watermark */ private volatile long maxWatermarkSoFar = Long.MIN_VALUE; @@ -77,7 +107,7 @@ public abstract class AbstractFetcher { protected AbstractFetcher( SourceContext sourceContext, - Map assignedPartitionsWithInitialOffsets, + Map seedPartitionsWithInitialOffsets, SerializedValue> watermarksPeriodic, SerializedValue> watermarksPunctuated, ProcessingTimeService processingTimeProvider, @@ -88,9 +118,12 @@ protected AbstractFetcher( this.sourceContext = checkNotNull(sourceContext); this.checkpointLock = sourceContext.getCheckpointLock(); this.useMetrics = useMetrics; - + this.userCodeClassLoader = checkNotNull(userCodeClassLoader); + // figure out what we watermark mode we will be using - + this.watermarksPeriodic = watermarksPeriodic; + this.watermarksPunctuated = watermarksPunctuated; + if (watermarksPeriodic == null) { if (watermarksPunctuated == null) { // simple case, no watermarks involved @@ -106,31 +139,69 @@ protected AbstractFetcher( } } - // create our partition state according to the timestamp/watermark mode - this.subscribedPartitionStates = initializeSubscribedPartitionStates( - assignedPartitionsWithInitialOffsets, + this.unassignedPartitionsQueue = new ClosableBlockingQueue<>(); + + // initialize subscribed partition states with seed partitions + this.subscribedPartitionStates = createPartitionStateHolders( + seedPartitionsWithInitialOffsets, timestampWatermarkMode, - watermarksPeriodic, watermarksPunctuated, + watermarksPeriodic, + watermarksPunctuated, userCodeClassLoader); - // check that all partition states have a defined offset + // check that all seed partition states have a defined offset for (KafkaTopicPartitionState partitionState : subscribedPartitionStates) { if (!partitionState.isOffsetDefined()) { - throw new IllegalArgumentException("The fetcher was assigned partitions with undefined initial offsets."); + throw new IllegalArgumentException("The fetcher was assigned seed partitions with undefined initial offsets."); } } - + + // all seed partitions are not assigned yet, so should be added to the unassigned partitions queue + for (KafkaTopicPartitionState partition : subscribedPartitionStates) { + unassignedPartitionsQueue.add(partition); + } + // if we have periodic watermarks, kick off the interval scheduler if (timestampWatermarkMode == PERIODIC_WATERMARKS) { - KafkaTopicPartitionStateWithPeriodicWatermarks[] parts = - (KafkaTopicPartitionStateWithPeriodicWatermarks[]) subscribedPartitionStates; - - PeriodicWatermarkEmitter periodicEmitter = - new PeriodicWatermarkEmitter(parts, sourceContext, processingTimeProvider, autoWatermarkInterval); + @SuppressWarnings("unchecked") + PeriodicWatermarkEmitter periodicEmitter = new PeriodicWatermarkEmitter( + subscribedPartitionStates, + sourceContext, + processingTimeProvider, + autoWatermarkInterval); + periodicEmitter.start(); } } + /** + * Adds a list of newly discovered partitions to the fetcher for consuming. + * + * This method creates the partition state holder for each new partition, using + * {@link KafkaTopicPartitionStateSentinel#EARLIEST_OFFSET} as the starting offset. + * It uses the earliest offset because there may be delay in discovering a partition + * after it was created and started receiving records. + * + * After the state representation for a partition is created, it is added to the + * unassigned partitions queue to await to be consumed. + * + * @param newPartitions discovered partitions to add + */ + public void addDiscoveredPartitions(List newPartitions) throws IOException, ClassNotFoundException { + List> newPartitionStates = createPartitionStateHolders( + newPartitions, + KafkaTopicPartitionStateSentinel.EARLIEST_OFFSET, + timestampWatermarkMode, + watermarksPeriodic, + watermarksPunctuated, + userCodeClassLoader); + + for (KafkaTopicPartitionState newPartitionState : newPartitionStates) { + subscribedPartitionStates.add(newPartitionState); + unassignedPartitionsQueue.add(newPartitionState); + } + } + // ------------------------------------------------------------------------ // Properties // ------------------------------------------------------------------------ @@ -140,7 +211,7 @@ protected AbstractFetcher( * * @return All subscribed partitions. */ - protected final KafkaTopicPartitionState[] subscribedPartitionStates() { + protected final List> subscribedPartitionStates() { return subscribedPartitionStates; } @@ -155,15 +226,6 @@ protected final KafkaTopicPartitionState[] subscribedPartitionStates() { // ------------------------------------------------------------------------ // Kafka version specifics // ------------------------------------------------------------------------ - - /** - * Creates the Kafka version specific representation of the given - * topic partition. - * - * @param partition The Flink representation of the Kafka topic partition. - * @return The specific Kafka representation of the Kafka topic partition. - */ - public abstract KPH createKafkaPartitionHandle(KafkaTopicPartition partition); /** * Commits the given partition offsets to the Kafka brokers (or to ZooKeeper for @@ -179,7 +241,16 @@ protected final KafkaTopicPartitionState[] subscribedPartitionStates() { * @throws Exception This method forwards exceptions. */ public abstract void commitInternalOffsetsToKafka(Map offsets) throws Exception; - + + /** + * Creates the Kafka version specific representation of the given + * topic partition. + * + * @param partition The Flink representation of the Kafka topic partition. + * @return The version-specific Kafka representation of the Kafka topic partition. + */ + protected abstract KPH createKafkaPartitionHandle(KafkaTopicPartition partition); + // ------------------------------------------------------------------------ // snapshot and restore the state // ------------------------------------------------------------------------ @@ -187,7 +258,7 @@ protected final KafkaTopicPartitionState[] subscribedPartitionStates() { /** * Takes a snapshot of the partition offsets. * - *

Important: This method mus be called under the checkpoint lock. + *

Important: This method must be called under the checkpoint lock. * * @return A map from partition to current offset. */ @@ -195,8 +266,8 @@ public HashMap snapshotCurrentState() { // this method assumes that the checkpoint lock is held assert Thread.holdsLock(checkpointLock); - HashMap state = new HashMap<>(subscribedPartitionStates.length); - for (KafkaTopicPartitionState partition : subscribedPartitionStates()) { + HashMap state = new HashMap<>(subscribedPartitionStates.size()); + for (KafkaTopicPartitionState partition : subscribedPartitionStates) { state.put(partition.getKafkaTopicPartition(), partition.getOffset()); } return state; @@ -368,80 +439,73 @@ private void updateMinPunctuatedWatermark(Watermark nextWatermark) { /** * Utility method that takes the topic partitions and creates the topic partition state - * holders. If a watermark generator per partition exists, this will also initialize those. + * holders, depending on the timestamp / watermark mode. */ - private KafkaTopicPartitionState[] initializeSubscribedPartitionStates( - Map assignedPartitionsToInitialOffsets, + private List> createPartitionStateHolders( + Map partitionsToInitialOffsets, int timestampWatermarkMode, SerializedValue> watermarksPeriodic, SerializedValue> watermarksPunctuated, - ClassLoader userCodeClassLoader) - throws IOException, ClassNotFoundException - { + ClassLoader userCodeClassLoader) throws IOException, ClassNotFoundException { + + List> partitionStates = new LinkedList<>(); + switch (timestampWatermarkMode) { - case NO_TIMESTAMPS_WATERMARKS: { - @SuppressWarnings("unchecked") - KafkaTopicPartitionState[] partitions = - (KafkaTopicPartitionState[]) new KafkaTopicPartitionState[assignedPartitionsToInitialOffsets.size()]; - - int pos = 0; - for (Map.Entry partition : assignedPartitionsToInitialOffsets.entrySet()) { + for (Map.Entry partitionEntry : partitionsToInitialOffsets.entrySet()) { // create the kafka version specific partition handle - KPH kafkaHandle = createKafkaPartitionHandle(partition.getKey()); - partitions[pos] = new KafkaTopicPartitionState<>(partition.getKey(), kafkaHandle); - partitions[pos].setOffset(partition.getValue()); + KPH kafkaHandle = createKafkaPartitionHandle(partitionEntry.getKey()); - pos++; + KafkaTopicPartitionState partitionState = + new KafkaTopicPartitionState<>(partitionEntry.getKey(), kafkaHandle); + partitionState.setOffset(partitionEntry.getValue()); + + partitionStates.add(partitionState); } - return partitions; + return partitionStates; } case PERIODIC_WATERMARKS: { - @SuppressWarnings("unchecked") - KafkaTopicPartitionStateWithPeriodicWatermarks[] partitions = - (KafkaTopicPartitionStateWithPeriodicWatermarks[]) - new KafkaTopicPartitionStateWithPeriodicWatermarks[assignedPartitionsToInitialOffsets.size()]; - - int pos = 0; - for (Map.Entry partition : assignedPartitionsToInitialOffsets.entrySet()) { - KPH kafkaHandle = createKafkaPartitionHandle(partition.getKey()); + for (Map.Entry partitionEntry : partitionsToInitialOffsets.entrySet()) { + KPH kafkaHandle = createKafkaPartitionHandle(partitionEntry.getKey()); AssignerWithPeriodicWatermarks assignerInstance = watermarksPeriodic.deserializeValue(userCodeClassLoader); - - partitions[pos] = new KafkaTopicPartitionStateWithPeriodicWatermarks<>( - partition.getKey(), kafkaHandle, assignerInstance); - partitions[pos].setOffset(partition.getValue()); - pos++; + KafkaTopicPartitionStateWithPeriodicWatermarks partitionState = + new KafkaTopicPartitionStateWithPeriodicWatermarks<>( + partitionEntry.getKey(), + kafkaHandle, + assignerInstance); + + partitionState.setOffset(partitionEntry.getValue()); + + partitionStates.add(partitionState); } - return partitions; + return partitionStates; } case PUNCTUATED_WATERMARKS: { - @SuppressWarnings("unchecked") - KafkaTopicPartitionStateWithPunctuatedWatermarks[] partitions = - (KafkaTopicPartitionStateWithPunctuatedWatermarks[]) - new KafkaTopicPartitionStateWithPunctuatedWatermarks[assignedPartitionsToInitialOffsets.size()]; - - int pos = 0; - for (Map.Entry partition : assignedPartitionsToInitialOffsets.entrySet()) { - KPH kafkaHandle = createKafkaPartitionHandle(partition.getKey()); + for (Map.Entry partitionEntry : partitionsToInitialOffsets.entrySet()) { + KPH kafkaHandle = createKafkaPartitionHandle(partitionEntry.getKey()); AssignerWithPunctuatedWatermarks assignerInstance = watermarksPunctuated.deserializeValue(userCodeClassLoader); - partitions[pos] = new KafkaTopicPartitionStateWithPunctuatedWatermarks<>( - partition.getKey(), kafkaHandle, assignerInstance); - partitions[pos].setOffset(partition.getValue()); + KafkaTopicPartitionStateWithPunctuatedWatermarks partitionState = + new KafkaTopicPartitionStateWithPunctuatedWatermarks<>( + partitionEntry.getKey(), + kafkaHandle, + assignerInstance); - pos++; + partitionState.setOffset(partitionEntry.getValue()); + + partitionStates.add(partitionState); } - return partitions; + return partitionStates; } default: // cannot happen, add this as a guard for the future @@ -449,6 +513,31 @@ private KafkaTopicPartitionState[] initializeSubscribedPartitionStates( } } + /** + * Shortcut variant of {@link #createPartitionStateHolders(Map, int, SerializedValue, SerializedValue, ClassLoader)} + * that uses the same offset for all partitions when creating their state holders. + */ + private List> createPartitionStateHolders( + List partitions, + long initialOffset, + int timestampWatermarkMode, + SerializedValue> watermarksPeriodic, + SerializedValue> watermarksPunctuated, + ClassLoader userCodeClassLoader) throws IOException, ClassNotFoundException { + + Map partitionsToInitialOffset = new HashMap<>(partitions.size()); + for (KafkaTopicPartition partition : partitions) { + partitionsToInitialOffset.put(partition, initialOffset); + } + + return createPartitionStateHolders( + partitionsToInitialOffset, + timestampWatermarkMode, + watermarksPeriodic, + watermarksPunctuated, + userCodeClassLoader); + } + // ------------------------- Metrics ---------------------------------- /** @@ -460,7 +549,7 @@ protected void addOffsetStateGauge(MetricGroup metricGroup) { // add current offsets to gage MetricGroup currentOffsets = metricGroup.addGroup("current-offsets"); MetricGroup committedOffsets = metricGroup.addGroup("committed-offsets"); - for (KafkaTopicPartitionState ktp: subscribedPartitionStates()) { + for (KafkaTopicPartitionState ktp : subscribedPartitionStates) { currentOffsets.gauge(ktp.getTopic() + "-" + ktp.getPartition(), new OffsetGauge(ktp, OffsetGaugeType.CURRENT_OFFSET)); committedOffsets.gauge(ktp.getTopic() + "-" + ktp.getPartition(), new OffsetGauge(ktp, OffsetGaugeType.COMMITTED_OFFSET)); } @@ -500,14 +589,14 @@ public Long getValue() { } } // ------------------------------------------------------------------------ - + /** * The periodic watermark emitter. In its given interval, it checks all partitions for * the current event time watermark, and possibly emits the next watermark. */ - private static class PeriodicWatermarkEmitter implements ProcessingTimeCallback { + private static class PeriodicWatermarkEmitter implements ProcessingTimeCallback { - private final KafkaTopicPartitionStateWithPeriodicWatermarks[] allPartitions; + private final List> allPartitions; private final SourceContext emitter; @@ -520,7 +609,7 @@ private static class PeriodicWatermarkEmitter implements ProcessingTimeCallback //------------------------------------------------- PeriodicWatermarkEmitter( - KafkaTopicPartitionStateWithPeriodicWatermarks[] allPartitions, + List> allPartitions, SourceContext emitter, ProcessingTimeService timerService, long autoWatermarkInterval) @@ -542,14 +631,14 @@ public void start() { public void onProcessingTime(long timestamp) throws Exception { long minAcrossAll = Long.MAX_VALUE; - for (KafkaTopicPartitionStateWithPeriodicWatermarks state : allPartitions) { - + for (KafkaTopicPartitionState state : allPartitions) { + // we access the current watermark for the periodic assigners under the state // lock, to prevent concurrent modification to any internal variables final long curr; //noinspection SynchronizationOnLocalVariableOrMethodParameter synchronized (state) { - curr = state.getCurrentWatermarkTimestamp(); + curr = ((KafkaTopicPartitionStateWithPeriodicWatermarks) state).getCurrentWatermarkTimestamp();; } minAcrossAll = Math.min(minAcrossAll, curr); diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscoverer.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscoverer.java new file mode 100644 index 0000000000000..2fc6e9ae59adc --- /dev/null +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscoverer.java @@ -0,0 +1,256 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kafka.internals; + +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Base class for all partition discoverers. + * + *

This partition discoverer base class implements the logic around bookkeeping + * discovered partitions, and using the information to determine whether or not there + * are new partitions that the consumer subtask should subscribe to. + * + *

Subclass implementations should simply implement the logic of using the version-specific + * Kafka clients to fetch topic and partition metadata. + * + *

Since Kafka clients are generally not thread-safe, partition discoverers should + * not be concurrently accessed. The only exception for this would be the {@link #wakeup()} + * call, which allows the discoverer to be interrupted during a {@link #discoverPartitions()} call. + */ +public abstract class AbstractPartitionDiscoverer { + + /** Describes whether we are discovering partitions for fixed topics or a topic pattern. */ + private final KafkaTopicsDescriptor topicsDescriptor; + + /** Index of the consumer subtask that this partition discoverer belongs to. */ + private final int indexOfThisSubtask; + + /** The total number of consumer subtasks. */ + private final int numParallelSubtasks; + + /** Flag to determine whether or not the discoverer is closed. */ + private volatile boolean closed = true; + + /** + * Flag to determine whether or not the discoverer had been woken up. + * When set to {@code true}, {@link #discoverPartitions()} would be interrupted as early as possible. + * Once interrupted, the flag is reset. + */ + private volatile boolean wakeup; + + /** + * Map of topics to they're largest discovered partition id seen by this subtask. + * This state may be updated whenever {@link AbstractPartitionDiscoverer#discoverPartitions()} or + * {@link AbstractPartitionDiscoverer#checkAndSetDiscoveredPartition(KafkaTopicPartition)} is called. + * + * This is used to remove old partitions from the fetched partition lists. It is sufficient + * to keep track of only the largest partition id because Kafka partition numbers are only + * allowed to be increased and has incremental ids. + */ + private final Map topicsToLargestDiscoveredPartitionId; + + public AbstractPartitionDiscoverer( + KafkaTopicsDescriptor topicsDescriptor, + int indexOfThisSubtask, + int numParallelSubtasks) { + + this.topicsDescriptor = checkNotNull(topicsDescriptor); + this.indexOfThisSubtask = indexOfThisSubtask; + this.numParallelSubtasks = numParallelSubtasks; + this.topicsToLargestDiscoveredPartitionId = new HashMap<>(); + } + + /** + * Opens the partition discoverer, initializing all required Kafka connections. + * + *

NOTE: thread-safety is not guaranteed. + */ + public void open() throws Exception { + closed = false; + initializeConnections(); + } + + /** + * Closes the partition discoverer, cleaning up all Kafka connections. + * + *

NOTE: thread-safety is not guaranteed. + */ + public void close() throws Exception { + closed = true; + closeConnections(); + } + + /** + * Interrupt an in-progress discovery attempt by throwing a {@link WakeupException}. + * If no attempt is in progress, the immediate next attempt will throw a {@link WakeupException}. + * + *

This method can be called concurrently from a different thread. + */ + public void wakeup() { + wakeup = true; + wakeupConnections(); + } + + /** + * Execute a partition discovery attempt for this subtask. + * This method lets the partition discoverer update what partitions it has discovered so far. + * + * @return List of discovered new partitions that this subtask should subscribe to. + */ + public List discoverPartitions() throws WakeupException, ClosedException { + if (!closed && !wakeup) { + try { + List newDiscoveredPartitions; + + // (1) get all possible partitions, based on whether we are subscribed to fixed topics or a topic patern + if (topicsDescriptor.isFixedTopics()) { + newDiscoveredPartitions = getAllPartitionsForTopics(topicsDescriptor.getFixedTopics()); + } else { + List matchedTopics = getAllTopics(); + + // retain topics that match the pattern + Iterator iter = matchedTopics.iterator(); + while (iter.hasNext()) { + if (!topicsDescriptor.getTopicPattern().matcher(iter.next()).matches()) { + iter.remove(); + } + } + + if (matchedTopics.size() != 0) { + // get partitions only for matched topics + newDiscoveredPartitions = getAllPartitionsForTopics(matchedTopics); + } else { + newDiscoveredPartitions = null; + } + } + + // (2) eliminate partition that are old partitions or should not be subscribed by this subtask + if (newDiscoveredPartitions == null || newDiscoveredPartitions.isEmpty()) { + throw new RuntimeException("Unable to retrieve any partitions with KafkaTopicsDescriptor: " + topicsDescriptor); + } else { + // sort so that we make sure the topicsToLargestDiscoveredPartitionId state is updated + // with incremental partition ids of the same topics (otherwise some partition ids may be skipped) + Collections.sort(newDiscoveredPartitions, new Comparator() { + @Override + public int compare(KafkaTopicPartition o1, KafkaTopicPartition o2) { + if (!o1.getTopic().equals(o2.getTopic())) { + return o1.getTopic().compareTo(o2.getTopic()); + } else { + return Integer.compare(o1.getPartition(), o2.getPartition()); + } + } + }); + + Iterator iter = newDiscoveredPartitions.iterator(); + KafkaTopicPartition nextPartition; + while (iter.hasNext()) { + nextPartition = iter.next(); + if (!checkAndSetDiscoveredPartition(nextPartition) || + !shouldAssignToThisSubtask(nextPartition, indexOfThisSubtask, numParallelSubtasks)) { + iter.remove(); + } + } + } + + return newDiscoveredPartitions; + } catch (WakeupException e) { + // the actual topic / partition metadata fetching methods + // may be woken up midway; reset the wakeup flag and rethrow + wakeup = false; + throw e; + } + } else if (!closed && wakeup) { + // may have been woken up before the method call + wakeup = false; + throw new WakeupException(); + } else { + throw new ClosedException(); + } + } + + /** + * Checks whether the given partition has been discovered yet. + * If it hasn't been discovered yet, this method lets the partition discoverer update what + * partitions it has discovered so far. + * + * @param partition the partition to check and set + * + * @return {@code true}, if the partition wasn't seen before, {@code false} otherwise + */ + public boolean checkAndSetDiscoveredPartition(KafkaTopicPartition partition) { + if (!topicsToLargestDiscoveredPartitionId.containsKey(partition.getTopic()) + || partition.getPartition() > topicsToLargestDiscoveredPartitionId.get(partition.getTopic())) { + + topicsToLargestDiscoveredPartitionId.put(partition.getTopic(), partition.getPartition()); + return true; + } + + return false; + } + + // ------------------------------------------------------------------------ + // Kafka version specifics + // ------------------------------------------------------------------------ + + /** Establish the required connections in order to fetch topics and partitions metadata. */ + protected abstract void initializeConnections() throws Exception; + + /** + * Attempt to eagerly wakeup from blocking calls to Kafka in {@link AbstractPartitionDiscoverer#getAllTopics()} + * and {@link AbstractPartitionDiscoverer#getAllPartitionsForTopics(List)}. + * + *

If the invocation indeed results in interrupting an actual blocking Kafka call, the implementations + * of {@link AbstractPartitionDiscoverer#getAllTopics()} and + * {@link AbstractPartitionDiscoverer#getAllPartitionsForTopics(List)} are responsible of throwing a + * {@link WakeupException}. + */ + protected abstract void wakeupConnections(); + + /** Close all established connections. */ + protected abstract void closeConnections() throws Exception; + + /** Fetch the list of all topics from Kafka. */ + protected abstract List getAllTopics() throws WakeupException; + + /** Fetch the list of all partitions for a specific topics list from Kafka. */ + protected abstract List getAllPartitionsForTopics(List topics) throws WakeupException; + + // ------------------------------------------------------------------------ + // Utilities + // ------------------------------------------------------------------------ + + private static boolean shouldAssignToThisSubtask(KafkaTopicPartition partition, int indexOfThisSubtask, int numParallelSubtasks) { + return Math.abs(partition.hashCode() % numParallelSubtasks) == indexOfThisSubtask; + } + + public static final class WakeupException extends Exception { + private static final long serialVersionUID = 1L; + } + + public static final class ClosedException extends Exception { + private static final long serialVersionUID = 1L; + } +} diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueue.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueue.java similarity index 100% rename from flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueue.java rename to flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueue.java diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionState.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionState.java index adfbf7952d2c3..a0076c139f849 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionState.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionState.java @@ -44,7 +44,7 @@ public class KafkaTopicPartitionState { private volatile long committedOffset; // ------------------------------------------------------------------------ - + public KafkaTopicPartitionState(KafkaTopicPartition partition, KPH kafkaPartitionHandle) { this.partition = partition; this.kafkaPartitionHandle = kafkaPartitionHandle; diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicsDescriptor.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicsDescriptor.java new file mode 100644 index 0000000000000..40440ada21bac --- /dev/null +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicsDescriptor.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kafka.internals; + +import javax.annotation.Nullable; +import java.io.Serializable; +import java.util.List; +import java.util.regex.Pattern; + +import static org.apache.flink.util.Preconditions.checkArgument; + +/** + * A Kafka Topics Descriptor describes how the consumer subscribes to Kafka topics - + * either a fixed list of topics, or a topic pattern. + */ +public class KafkaTopicsDescriptor implements Serializable { + + private static final long serialVersionUID = -3807227764764900975L; + + private final List fixedTopics; + private final Pattern topicPattern; + + public KafkaTopicsDescriptor(@Nullable List fixedTopics, @Nullable Pattern topicPattern) { + checkArgument((fixedTopics != null && topicPattern == null) || (fixedTopics == null && topicPattern != null), + "Exactly one of either fixedTopics or topicPattern must be specified."); + + if (fixedTopics != null) { + checkArgument(!fixedTopics.isEmpty(), "If subscribing to a fixed topics list, the supplied list cannot be empty."); + } + + this.fixedTopics = fixedTopics; + this.topicPattern = topicPattern; + } + + public boolean isFixedTopics() { + return fixedTopics != null; + } + + public boolean isTopicPattern() { + return topicPattern != null; + } + + public List getFixedTopics() { + return fixedTopics; + } + + public Pattern getTopicPattern() { + return topicPattern; + } + + @Override + public String toString() { + return (fixedTopics == null) + ? "Topic Regex Pattern (" + topicPattern.pattern() + ")" + : "Fixed Topics (" + fixedTopics + ")"; + } +} diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java index 9fc261e833b99..3702c80268167 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java @@ -24,8 +24,10 @@ import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractPartitionDiscoverer; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionStateSentinel; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicsDescriptor; import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness; import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; import org.apache.flink.util.SerializedValue; @@ -41,6 +43,7 @@ import java.util.List; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; /** * Tests for checking whether {@link FlinkKafkaConsumerBase} can restore from snapshots that were @@ -183,7 +186,7 @@ private static class DummyFlinkKafkaConsumer extends FlinkKafkaConsumerBase partitions) { - super(Arrays.asList("dummy-topic"), (KeyedDeserializationSchema< T >) mock(KeyedDeserializationSchema.class)); + super(Arrays.asList("dummy-topic"), null, (KeyedDeserializationSchema< T >) mock(KeyedDeserializationSchema.class), 0); this.partitions = partitions; } @@ -199,8 +202,19 @@ private static class DummyFlinkKafkaConsumer extends FlinkKafkaConsumerBase getKafkaPartitions(List topics) { - return partitions; + protected AbstractPartitionDiscoverer createPartitionDiscoverer( + KafkaTopicsDescriptor topicsDescriptor, + int indexOfThisSubtask, + int numParallelSubtasks) { + AbstractPartitionDiscoverer mockPartitionDiscoverer = mock(AbstractPartitionDiscoverer.class); + + try { + when(mockPartitionDiscoverer.discoverPartitions()).thenReturn(partitions); + } catch (Exception e) { + // ignore + } + + return mockPartitionDiscoverer; } @Override diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java index 4f5b2831ba976..b8c5abf4ee63c 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java @@ -28,11 +28,12 @@ import org.apache.flink.runtime.state.StateSnapshotContextSynchronousImpl; 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; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractPartitionDiscoverer; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicsDescriptor; import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; import org.apache.flink.util.SerializedValue; import org.junit.Assert; @@ -44,7 +45,6 @@ import java.lang.reflect.Field; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -188,32 +188,6 @@ public void checkRestoredNullCheckpointWhenFetcherNotReady() throws Exception { assertFalse(listState.get().iterator().hasNext()); } - /** - * Tests that on snapshots, states and offsets to commit to Kafka are correct - */ - @SuppressWarnings("unchecked") - @Test - public void checkUseFetcherWhenNoCheckpoint() throws Exception { - - FlinkKafkaConsumerBase consumer = getConsumer(null, new LinkedMap(), true); - List partitionList = new ArrayList<>(1); - partitionList.add(new KafkaTopicPartition("test", 0)); - consumer.setSubscribedPartitions(partitionList); - - OperatorStateStore operatorStateStore = mock(OperatorStateStore.class); - TestingListState listState = new TestingListState<>(); - when(operatorStateStore.getSerializableListState(Matchers.any(String.class))).thenReturn(listState); - - StateInitializationContext initializationContext = mock(StateInitializationContext.class); - - when(initializationContext.getOperatorStateStore()).thenReturn(operatorStateStore); - - // make the context signal that there is no restored state, then validate that - when(initializationContext.isRestored()).thenReturn(false); - consumer.initializeState(initializationContext); - consumer.run(mock(SourceFunction.SourceContext.class)); - } - @Test @SuppressWarnings("unchecked") public void testSnapshotStateWithCommitOnCheckpointsEnabled() throws Exception { @@ -498,7 +472,7 @@ private static class DummyFlinkKafkaConsumer extends FlinkKafkaConsumerBase) mock(KeyedDeserializationSchema.class)); + super(Arrays.asList("dummy-topic"), null, (KeyedDeserializationSchema < T >) mock(KeyedDeserializationSchema.class), 0); } @Override @@ -514,8 +488,11 @@ public DummyFlinkKafkaConsumer() { } @Override - protected List getKafkaPartitions(List topics) { - return Collections.emptyList(); + protected AbstractPartitionDiscoverer createPartitionDiscoverer( + KafkaTopicsDescriptor topicsDescriptor, + int indexOfThisSubtask, + int numParallelSubtasks) { + return mock(AbstractPartitionDiscoverer.class); } @Override diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerPartitionAssignmentTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerPartitionAssignmentTest.java deleted file mode 100644 index c24640dfd2a38..0000000000000 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerPartitionAssignmentTest.java +++ /dev/null @@ -1,356 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.streaming.connectors.kafka; - -import org.apache.flink.streaming.connectors.kafka.config.StartupMode; -import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; - -import org.junit.Test; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashSet; -import java.util.Map; -import java.util.HashMap; -import java.util.List; -import java.util.Set; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -/** - * Tests that the partition assignment is deterministic and stable. - */ -public class KafkaConsumerPartitionAssignmentTest { - - @Test - public void testPartitionsEqualConsumers() { - try { - List inPartitions = Arrays.asList( - new KafkaTopicPartition("test-topic", 4), - new KafkaTopicPartition("test-topic", 52), - new KafkaTopicPartition("test-topic", 17), - new KafkaTopicPartition("test-topic", 1)); - - for (int i = 0; i < inPartitions.size(); i++) { - Map subscribedPartitionsToStartOffsets = new HashMap<>(); - FlinkKafkaConsumerBase.initializeSubscribedPartitionsToStartOffsets( - subscribedPartitionsToStartOffsets, - inPartitions, - i, - inPartitions.size(), - StartupMode.GROUP_OFFSETS, - null); - - List subscribedPartitions = new ArrayList<>(subscribedPartitionsToStartOffsets.keySet()); - - assertEquals(1, subscribedPartitions.size()); - assertTrue(contains(inPartitions, subscribedPartitions.get(0).getPartition())); - } - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testMultiplePartitionsPerConsumers() { - try { - final int[] partitionIDs = {4, 52, 17, 1, 2, 3, 89, 42, 31, 127, 14}; - - final List partitions = new ArrayList<>(); - final Set allPartitions = new HashSet<>(); - - for (int p : partitionIDs) { - KafkaTopicPartition part = new KafkaTopicPartition("test-topic", p); - partitions.add(part); - allPartitions.add(part); - } - - final int numConsumers = 3; - final int minPartitionsPerConsumer = partitions.size() / numConsumers; - final int maxPartitionsPerConsumer = partitions.size() / numConsumers + 1; - - for (int i = 0; i < numConsumers; i++) { - Map subscribedPartitionsToStartOffsets = new HashMap<>(); - FlinkKafkaConsumerBase.initializeSubscribedPartitionsToStartOffsets( - subscribedPartitionsToStartOffsets, - partitions, - i, - numConsumers, - StartupMode.GROUP_OFFSETS, - null); - - List subscribedPartitions = new ArrayList<>(subscribedPartitionsToStartOffsets.keySet()); - - assertTrue(subscribedPartitions.size() >= minPartitionsPerConsumer); - assertTrue(subscribedPartitions.size() <= maxPartitionsPerConsumer); - - for (KafkaTopicPartition p : subscribedPartitions) { - // check that the element was actually contained - assertTrue(allPartitions.remove(p)); - } - } - - // all partitions must have been assigned - assertTrue(allPartitions.isEmpty()); - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testPartitionsFewerThanConsumers() { - try { - List inPartitions = Arrays.asList( - new KafkaTopicPartition("test-topic", 4), - new KafkaTopicPartition("test-topic", 52), - new KafkaTopicPartition("test-topic", 17), - new KafkaTopicPartition("test-topic", 1)); - - final Set allPartitions = new HashSet<>(); - allPartitions.addAll(inPartitions); - - final int numConsumers = 2 * inPartitions.size() + 3; - - for (int i = 0; i < numConsumers; i++) { - Map subscribedPartitionsToStartOffsets = new HashMap<>(); - FlinkKafkaConsumerBase.initializeSubscribedPartitionsToStartOffsets( - subscribedPartitionsToStartOffsets, - inPartitions, - i, - numConsumers, - StartupMode.GROUP_OFFSETS, - null); - - List subscribedPartitions = new ArrayList<>(subscribedPartitionsToStartOffsets.keySet()); - - assertTrue(subscribedPartitions.size() <= 1); - - for (KafkaTopicPartition p : subscribedPartitions) { - // check that the element was actually contained - assertTrue(allPartitions.remove(p)); - } - } - - // all partitions must have been assigned - assertTrue(allPartitions.isEmpty()); - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testAssignEmptyPartitions() { - try { - List ep = new ArrayList<>(); - Map subscribedPartitionsToStartOffsets = new HashMap<>(); - FlinkKafkaConsumerBase.initializeSubscribedPartitionsToStartOffsets( - subscribedPartitionsToStartOffsets, - ep, - 2, - 4, - StartupMode.GROUP_OFFSETS, - null); - assertTrue(subscribedPartitionsToStartOffsets.entrySet().isEmpty()); - - subscribedPartitionsToStartOffsets = new HashMap<>(); - FlinkKafkaConsumerBase.initializeSubscribedPartitionsToStartOffsets( - subscribedPartitionsToStartOffsets, - ep, - 0, - 1, - StartupMode.GROUP_OFFSETS, - null); - assertTrue(subscribedPartitionsToStartOffsets.entrySet().isEmpty()); - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testGrowingPartitionsRemainsStable() { - try { - final int[] newPartitionIDs = {4, 52, 17, 1, 2, 3, 89, 42, 31, 127, 14}; - List newPartitions = new ArrayList<>(); - - for (int p : newPartitionIDs) { - KafkaTopicPartition part = new KafkaTopicPartition("test-topic", p); - newPartitions.add(part); - } - - List initialPartitions = newPartitions.subList(0, 7); - - final Set allNewPartitions = new HashSet<>(newPartitions); - final Set allInitialPartitions = new HashSet<>(initialPartitions); - - final int numConsumers = 3; - final int minInitialPartitionsPerConsumer = initialPartitions.size() / numConsumers; - final int maxInitialPartitionsPerConsumer = initialPartitions.size() / numConsumers + 1; - final int minNewPartitionsPerConsumer = newPartitions.size() / numConsumers; - final int maxNewPartitionsPerConsumer = newPartitions.size() / numConsumers + 1; - - Map subscribedPartitionsToStartOffsets1 = new HashMap<>(); - Map subscribedPartitionsToStartOffsets2 = new HashMap<>(); - Map subscribedPartitionsToStartOffsets3 = new HashMap<>(); - - FlinkKafkaConsumerBase.initializeSubscribedPartitionsToStartOffsets( - subscribedPartitionsToStartOffsets1, - initialPartitions, - 0, - numConsumers, - StartupMode.GROUP_OFFSETS, - null); - - FlinkKafkaConsumerBase.initializeSubscribedPartitionsToStartOffsets( - subscribedPartitionsToStartOffsets2, - initialPartitions, - 1, - numConsumers, - StartupMode.GROUP_OFFSETS, - null); - - FlinkKafkaConsumerBase.initializeSubscribedPartitionsToStartOffsets( - subscribedPartitionsToStartOffsets3, - initialPartitions, - 2, - numConsumers, - StartupMode.GROUP_OFFSETS, - null); - - List subscribedPartitions1 = new ArrayList<>(subscribedPartitionsToStartOffsets1.keySet()); - List subscribedPartitions2 = new ArrayList<>(subscribedPartitionsToStartOffsets2.keySet()); - List subscribedPartitions3 = new ArrayList<>(subscribedPartitionsToStartOffsets3.keySet()); - - assertTrue(subscribedPartitions1.size() >= minInitialPartitionsPerConsumer); - assertTrue(subscribedPartitions1.size() <= maxInitialPartitionsPerConsumer); - assertTrue(subscribedPartitions2.size() >= minInitialPartitionsPerConsumer); - assertTrue(subscribedPartitions2.size() <= maxInitialPartitionsPerConsumer); - assertTrue(subscribedPartitions3.size() >= minInitialPartitionsPerConsumer); - assertTrue(subscribedPartitions3.size() <= maxInitialPartitionsPerConsumer); - - for (KafkaTopicPartition p : subscribedPartitions1) { - // check that the element was actually contained - assertTrue(allInitialPartitions.remove(p)); - } - - for (KafkaTopicPartition p : subscribedPartitions2) { - // check that the element was actually contained - assertTrue(allInitialPartitions.remove(p)); - } - - for (KafkaTopicPartition p : subscribedPartitions3) { - // check that the element was actually contained - assertTrue(allInitialPartitions.remove(p)); - } - - // all partitions must have been assigned - assertTrue(allInitialPartitions.isEmpty()); - - // grow the set of partitions and distribute anew - - subscribedPartitionsToStartOffsets1 = new HashMap<>(); - subscribedPartitionsToStartOffsets2 = new HashMap<>(); - subscribedPartitionsToStartOffsets3 = new HashMap<>(); - - FlinkKafkaConsumerBase.initializeSubscribedPartitionsToStartOffsets( - subscribedPartitionsToStartOffsets1, - newPartitions, - 0, - numConsumers, - StartupMode.GROUP_OFFSETS, - null); - - FlinkKafkaConsumerBase.initializeSubscribedPartitionsToStartOffsets( - subscribedPartitionsToStartOffsets2, - newPartitions, - 1, - numConsumers, - StartupMode.GROUP_OFFSETS, - null); - - FlinkKafkaConsumerBase.initializeSubscribedPartitionsToStartOffsets( - subscribedPartitionsToStartOffsets3, - newPartitions, - 2, - numConsumers, - StartupMode.GROUP_OFFSETS, - null); - - List subscribedPartitions1New = new ArrayList<>(subscribedPartitionsToStartOffsets1.keySet()); - List subscribedPartitions2New = new ArrayList<>(subscribedPartitionsToStartOffsets2.keySet()); - List subscribedPartitions3New = new ArrayList<>(subscribedPartitionsToStartOffsets3.keySet()); - - // new partitions must include all old partitions - - assertTrue(subscribedPartitions1New.size() > subscribedPartitions1.size()); - assertTrue(subscribedPartitions2New.size() > subscribedPartitions2.size()); - assertTrue(subscribedPartitions3New.size() > subscribedPartitions3.size()); - - assertTrue(subscribedPartitions1New.containsAll(subscribedPartitions1)); - assertTrue(subscribedPartitions2New.containsAll(subscribedPartitions2)); - assertTrue(subscribedPartitions3New.containsAll(subscribedPartitions3)); - - assertTrue(subscribedPartitions1New.size() >= minNewPartitionsPerConsumer); - assertTrue(subscribedPartitions1New.size() <= maxNewPartitionsPerConsumer); - assertTrue(subscribedPartitions2New.size() >= minNewPartitionsPerConsumer); - assertTrue(subscribedPartitions2New.size() <= maxNewPartitionsPerConsumer); - assertTrue(subscribedPartitions3New.size() >= minNewPartitionsPerConsumer); - assertTrue(subscribedPartitions3New.size() <= maxNewPartitionsPerConsumer); - - for (KafkaTopicPartition p : subscribedPartitions1New) { - // check that the element was actually contained - assertTrue(allNewPartitions.remove(p)); - } - for (KafkaTopicPartition p : subscribedPartitions2New) { - // check that the element was actually contained - assertTrue(allNewPartitions.remove(p)); - } - for (KafkaTopicPartition p : subscribedPartitions3New) { - // check that the element was actually contained - assertTrue(allNewPartitions.remove(p)); - } - - // all partitions must have been assigned - assertTrue(allNewPartitions.isEmpty()); - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - private boolean contains(List inPartitions, int partition) { - for (KafkaTopicPartition ktp : inPartitions) { - if (ktp.getPartition() == partition) { - return true; - } - } - return false; - } - -} diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java index ddac61c655ede..855ab16a8d9e6 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java @@ -27,7 +27,6 @@ import kafka.server.KafkaServer; import org.apache.commons.io.output.ByteArrayOutputStream; import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.functions.RichFlatMapFunction; @@ -145,7 +144,7 @@ public void ensureNoJobIsLingering() throws Exception { // ------------------------------------------------------------------------ /** - * Test that ensures the KafkaConsumer is properly failing if the topic doesnt exist + * Test that ensures the KafkaConsumer is properly failing if the topic doesn't exist * and a wrong broker was specified * * @throws Exception @@ -193,7 +192,7 @@ public void runFailOnNoBrokerTest() throws Exception { RuntimeException re = (RuntimeException) jee.getCause(); - assertTrue(re.getMessage().contains("Unable to retrieve any partitions for the requested topics [doesntexist]")); + assertTrue(re.getMessage().contains("Unable to retrieve any partitions")); } } } @@ -1678,7 +1677,7 @@ public void flatMap(Tuple2 value, Collector out) throws } }); - JobExecutionResult result = tryExecute(env1, "Consume " + ELEMENT_COUNT + " elements from Kafka"); + tryExecute(env1, "Consume " + ELEMENT_COUNT + " elements from Kafka"); deleteTestTopic(topic); } diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTest.java index c1a64c4bf20b3..4a53a73680c21 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTest.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTest.java @@ -61,7 +61,7 @@ public void testSkipCorruptedRecord() throws Exception { mock(TestProcessingTimeService.class), 0); - final KafkaTopicPartitionState partitionStateHolder = fetcher.subscribedPartitionStates()[0]; + final KafkaTopicPartitionState partitionStateHolder = fetcher.subscribedPartitionStates().get(0); fetcher.emitRecord(1L, partitionStateHolder, 1L); fetcher.emitRecord(2L, partitionStateHolder, 2L); @@ -92,7 +92,7 @@ public void testSkipCorruptedRecordWithPunctuatedWatermarks() throws Exception { processingTimeProvider, 0); - final KafkaTopicPartitionState partitionStateHolder = fetcher.subscribedPartitionStates()[0]; + final KafkaTopicPartitionState partitionStateHolder = fetcher.subscribedPartitionStates().get(0); // elements generate a watermark if the timestamp is a multiple of three fetcher.emitRecord(1L, partitionStateHolder, 1L); @@ -133,7 +133,7 @@ public void testSkipCorruptedRecordWithPeriodicWatermarks() throws Exception { processingTimeProvider, 10); - final KafkaTopicPartitionState partitionStateHolder = fetcher.subscribedPartitionStates()[0]; + final KafkaTopicPartitionState partitionStateHolder = fetcher.subscribedPartitionStates().get(0); // elements generate a watermark if the timestamp is a multiple of three fetcher.emitRecord(1L, partitionStateHolder, 1L); @@ -186,9 +186,9 @@ public void testPunctuatedWatermarks() throws Exception { processingTimeProvider, 0); - final KafkaTopicPartitionState part1 = fetcher.subscribedPartitionStates()[0]; - final KafkaTopicPartitionState part2 = fetcher.subscribedPartitionStates()[1]; - final KafkaTopicPartitionState part3 = fetcher.subscribedPartitionStates()[2]; + final KafkaTopicPartitionState part1 = fetcher.subscribedPartitionStates().get(0); + final KafkaTopicPartitionState part2 = fetcher.subscribedPartitionStates().get(1); + final KafkaTopicPartitionState part3 = fetcher.subscribedPartitionStates().get(2); // elements generate a watermark if the timestamp is a multiple of three @@ -260,9 +260,9 @@ public void testPeriodicWatermarks() throws Exception { processingTimeService, 10); - final KafkaTopicPartitionState part1 = fetcher.subscribedPartitionStates()[0]; - final KafkaTopicPartitionState part2 = fetcher.subscribedPartitionStates()[1]; - final KafkaTopicPartitionState part3 = fetcher.subscribedPartitionStates()[2]; + final KafkaTopicPartitionState part1 = fetcher.subscribedPartitionStates().get(0); + final KafkaTopicPartitionState part2 = fetcher.subscribedPartitionStates().get(1); + final KafkaTopicPartitionState part3 = fetcher.subscribedPartitionStates().get(2); // elements generate a watermark if the timestamp is a multiple of three diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscovererTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscovererTest.java new file mode 100644 index 0000000000000..10437e51956d2 --- /dev/null +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscovererTest.java @@ -0,0 +1,430 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kafka.internals; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.regex.Pattern; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.Matchers.anyInt; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +/** + * Tests that the partition assignment in the partition discoverer is + * deterministic and stable, with both fixed and growing partitions. + */ +@RunWith(Parameterized.class) +public class AbstractPartitionDiscovererTest { + + private static final String TEST_TOPIC = "test-topic"; + private static final String TEST_TOPIC_PATTERN = "^" + TEST_TOPIC + "[0-9]*$"; + + private final KafkaTopicsDescriptor topicsDescriptor; + + public AbstractPartitionDiscovererTest(KafkaTopicsDescriptor topicsDescriptor) { + this.topicsDescriptor = topicsDescriptor; + } + + @Parameterized.Parameters(name = "KafkaTopicsDescriptor = {0}") + @SuppressWarnings("unchecked") + public static Collection timeCharacteristic(){ + return Arrays.asList( + new KafkaTopicsDescriptor[]{new KafkaTopicsDescriptor(Collections.singletonList(TEST_TOPIC), null)}, + new KafkaTopicsDescriptor[]{new KafkaTopicsDescriptor(null, Pattern.compile(TEST_TOPIC_PATTERN))}); + } + + @Test + public void testPartitionsEqualConsumersFixedPartitions() throws Exception { + List mockGetAllPartitionsForTopicsReturn = Arrays.asList( + new KafkaTopicPartition(TEST_TOPIC, 0), + new KafkaTopicPartition(TEST_TOPIC, 1), + new KafkaTopicPartition(TEST_TOPIC, 2), + new KafkaTopicPartition(TEST_TOPIC, 3)); + + for (int i = 0; i < mockGetAllPartitionsForTopicsReturn.size(); i++) { + TestPartitionDiscoverer partitionDiscoverer = new TestPartitionDiscoverer( + topicsDescriptor, + i, + mockGetAllPartitionsForTopicsReturn.size(), + createMockGetAllTopicsSequenceFromFixedReturn(Collections.singletonList(TEST_TOPIC)), + createMockGetAllPartitionsFromTopicsSequenceFromFixedReturn(mockGetAllPartitionsForTopicsReturn)); + partitionDiscoverer.open(); + + List initialDiscovery = partitionDiscoverer.discoverPartitions(); + assertEquals(1, initialDiscovery.size()); + assertTrue(contains(mockGetAllPartitionsForTopicsReturn, initialDiscovery.get(0).getPartition())); + + // subsequent discoveries should not find anything + List secondDiscovery = partitionDiscoverer.discoverPartitions(); + List thirdDiscovery = partitionDiscoverer.discoverPartitions(); + assertEquals(0, secondDiscovery.size()); + assertEquals(0, thirdDiscovery.size()); + } + } + + @Test + public void testMultiplePartitionsPerConsumersFixedPartitions() { + try { + final int[] partitionIDs = {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10}; + + final List mockGetAllPartitionsForTopicsReturn = new ArrayList<>(); + final Set allPartitions = new HashSet<>(); + + for (int p : partitionIDs) { + KafkaTopicPartition part = new KafkaTopicPartition(TEST_TOPIC, p); + mockGetAllPartitionsForTopicsReturn.add(part); + allPartitions.add(part); + } + + final int numConsumers = 3; + final int minPartitionsPerConsumer = mockGetAllPartitionsForTopicsReturn.size() / numConsumers; + final int maxPartitionsPerConsumer = mockGetAllPartitionsForTopicsReturn.size() / numConsumers + 1; + + for (int i = 0; i < numConsumers; i++) { + TestPartitionDiscoverer partitionDiscoverer = new TestPartitionDiscoverer( + topicsDescriptor, + i, + numConsumers, + createMockGetAllTopicsSequenceFromFixedReturn(Collections.singletonList(TEST_TOPIC)), + createMockGetAllPartitionsFromTopicsSequenceFromFixedReturn(mockGetAllPartitionsForTopicsReturn)); + partitionDiscoverer.open(); + + List initialDiscovery = partitionDiscoverer.discoverPartitions(); + assertTrue(initialDiscovery.size() >= minPartitionsPerConsumer); + assertTrue(initialDiscovery.size() <= maxPartitionsPerConsumer); + + for (KafkaTopicPartition p : initialDiscovery) { + // check that the element was actually contained + assertTrue(allPartitions.remove(p)); + } + + // subsequent discoveries should not find anything + List secondDiscovery = partitionDiscoverer.discoverPartitions(); + List thirdDiscovery = partitionDiscoverer.discoverPartitions(); + assertEquals(0, secondDiscovery.size()); + assertEquals(0, thirdDiscovery.size()); + } + + // all partitions must have been assigned + assertTrue(allPartitions.isEmpty()); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void testPartitionsFewerThanConsumersFixedPartitions() { + try { + List mockGetAllPartitionsForTopicsReturn = Arrays.asList( + new KafkaTopicPartition(TEST_TOPIC, 0), + new KafkaTopicPartition(TEST_TOPIC, 1), + new KafkaTopicPartition(TEST_TOPIC, 2), + new KafkaTopicPartition(TEST_TOPIC, 3)); + + final Set allPartitions = new HashSet<>(); + allPartitions.addAll(mockGetAllPartitionsForTopicsReturn); + + final int numConsumers = 2 * mockGetAllPartitionsForTopicsReturn.size() + 3; + + for (int i = 0; i < numConsumers; i++) { + TestPartitionDiscoverer partitionDiscoverer = new TestPartitionDiscoverer( + topicsDescriptor, + i, + numConsumers, + createMockGetAllTopicsSequenceFromFixedReturn(Collections.singletonList(TEST_TOPIC)), + createMockGetAllPartitionsFromTopicsSequenceFromFixedReturn(mockGetAllPartitionsForTopicsReturn)); + partitionDiscoverer.open(); + + List initialDiscovery = partitionDiscoverer.discoverPartitions(); + assertTrue(initialDiscovery.size() <= 1); + + for (KafkaTopicPartition p : initialDiscovery) { + // check that the element was actually contained + assertTrue(allPartitions.remove(p)); + } + + // subsequent discoveries should not find anything + List secondDiscovery = partitionDiscoverer.discoverPartitions(); + List thirdDiscovery = partitionDiscoverer.discoverPartitions(); + assertEquals(0, secondDiscovery.size()); + assertEquals(0, thirdDiscovery.size()); + } + + // all partitions must have been assigned + assertTrue(allPartitions.isEmpty()); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void testGrowingPartitions() { + try { + final int[] newPartitionIDs = {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10}; + List allPartitions = new ArrayList<>(11); + + for (int p : newPartitionIDs) { + KafkaTopicPartition part = new KafkaTopicPartition(TEST_TOPIC, p); + allPartitions.add(part); + } + + // first discovery returns an initial subset of the partitions; second returns all partitions + List> mockGetAllPartitionsForTopicsReturnSequence = Arrays.asList( + new ArrayList<>(allPartitions.subList(0, 7)), + allPartitions); + + final Set allNewPartitions = new HashSet<>(allPartitions); + final Set allInitialPartitions = new HashSet<>(mockGetAllPartitionsForTopicsReturnSequence.get(0)); + + final int numConsumers = 3; + final int minInitialPartitionsPerConsumer = mockGetAllPartitionsForTopicsReturnSequence.get(0).size() / numConsumers; + final int maxInitialPartitionsPerConsumer = mockGetAllPartitionsForTopicsReturnSequence.get(0).size() / numConsumers + 1; + final int minNewPartitionsPerConsumer = allPartitions.size() / numConsumers; + final int maxNewPartitionsPerConsumer = allPartitions.size() / numConsumers + 1; + + TestPartitionDiscoverer partitionDiscovererSubtask0 = new TestPartitionDiscoverer( + topicsDescriptor, + 0, + numConsumers, + createMockGetAllTopicsSequenceFromFixedReturn(Collections.singletonList(TEST_TOPIC)), + deepClone(mockGetAllPartitionsForTopicsReturnSequence)); + partitionDiscovererSubtask0.open(); + + TestPartitionDiscoverer partitionDiscovererSubtask1 = new TestPartitionDiscoverer( + topicsDescriptor, + 1, + numConsumers, + createMockGetAllTopicsSequenceFromFixedReturn(Collections.singletonList(TEST_TOPIC)), + deepClone(mockGetAllPartitionsForTopicsReturnSequence)); + partitionDiscovererSubtask1.open(); + + TestPartitionDiscoverer partitionDiscovererSubtask2 = new TestPartitionDiscoverer( + topicsDescriptor, + 2, + numConsumers, + createMockGetAllTopicsSequenceFromFixedReturn(Collections.singletonList(TEST_TOPIC)), + deepClone(mockGetAllPartitionsForTopicsReturnSequence)); + partitionDiscovererSubtask2.open(); + + List initialDiscoverySubtask0 = partitionDiscovererSubtask0.discoverPartitions(); + List initialDiscoverySubtask1 = partitionDiscovererSubtask1.discoverPartitions(); + List initialDiscoverySubtask2 = partitionDiscovererSubtask2.discoverPartitions(); + + assertTrue(initialDiscoverySubtask0.size() >= minInitialPartitionsPerConsumer); + assertTrue(initialDiscoverySubtask0.size() <= maxInitialPartitionsPerConsumer); + assertTrue(initialDiscoverySubtask1.size() >= minInitialPartitionsPerConsumer); + assertTrue(initialDiscoverySubtask1.size() <= maxInitialPartitionsPerConsumer); + assertTrue(initialDiscoverySubtask2.size() >= minInitialPartitionsPerConsumer); + assertTrue(initialDiscoverySubtask2.size() <= maxInitialPartitionsPerConsumer); + + for (KafkaTopicPartition p : initialDiscoverySubtask0) { + // check that the element was actually contained + assertTrue(allInitialPartitions.remove(p)); + } + + for (KafkaTopicPartition p : initialDiscoverySubtask1) { + // check that the element was actually contained + assertTrue(allInitialPartitions.remove(p)); + } + + for (KafkaTopicPartition p : initialDiscoverySubtask2) { + // check that the element was actually contained + assertTrue(allInitialPartitions.remove(p)); + } + + // all partitions must have been assigned + assertTrue(allInitialPartitions.isEmpty()); + + + // now, execute discover again (should find the extra new partitions) + List secondDiscoverySubtask0 = partitionDiscovererSubtask0.discoverPartitions(); + List secondDiscoverySubtask1 = partitionDiscovererSubtask1.discoverPartitions(); + List secondDiscoverySubtask2 = partitionDiscovererSubtask2.discoverPartitions(); + + // new discovered partitions must not have been discovered before + assertTrue(Collections.disjoint(secondDiscoverySubtask0, initialDiscoverySubtask0)); + assertTrue(Collections.disjoint(secondDiscoverySubtask1, initialDiscoverySubtask1)); + assertTrue(Collections.disjoint(secondDiscoverySubtask2, initialDiscoverySubtask2)); + + assertTrue(secondDiscoverySubtask0.size() + initialDiscoverySubtask0.size() >= minNewPartitionsPerConsumer); + assertTrue(secondDiscoverySubtask0.size() + initialDiscoverySubtask0.size() <= maxNewPartitionsPerConsumer); + assertTrue(secondDiscoverySubtask1.size() + initialDiscoverySubtask1.size() >= minNewPartitionsPerConsumer); + assertTrue(secondDiscoverySubtask1.size() + initialDiscoverySubtask1.size() <= maxNewPartitionsPerConsumer); + assertTrue(secondDiscoverySubtask2.size() + initialDiscoverySubtask2.size() >= minNewPartitionsPerConsumer); + assertTrue(secondDiscoverySubtask2.size() + initialDiscoverySubtask2.size() <= maxNewPartitionsPerConsumer); + + // check that the two discoveries combined form all partitions + + for (KafkaTopicPartition p : initialDiscoverySubtask0) { + assertTrue(allNewPartitions.remove(p)); + } + + for (KafkaTopicPartition p : initialDiscoverySubtask1) { + assertTrue(allNewPartitions.remove(p)); + } + + for (KafkaTopicPartition p : initialDiscoverySubtask2) { + assertTrue(allNewPartitions.remove(p)); + } + + for (KafkaTopicPartition p : secondDiscoverySubtask0) { + assertTrue(allNewPartitions.remove(p)); + } + + for (KafkaTopicPartition p : secondDiscoverySubtask1) { + assertTrue(allNewPartitions.remove(p)); + } + + for (KafkaTopicPartition p : secondDiscoverySubtask2) { + assertTrue(allNewPartitions.remove(p)); + } + + // all partitions must have been assigned + assertTrue(allNewPartitions.isEmpty()); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + + private static class TestPartitionDiscoverer extends AbstractPartitionDiscoverer { + + private final KafkaTopicsDescriptor topicsDescriptor; + + private final List> mockGetAllTopicsReturnSequence; + private final List> mockGetAllPartitionsForTopicsReturnSequence; + + private int getAllTopicsInvokeCount = 0; + private int getAllPartitionsForTopicsInvokeCount = 0; + + public TestPartitionDiscoverer( + KafkaTopicsDescriptor topicsDescriptor, + int indexOfThisSubtask, + int numParallelSubtasks, + List> mockGetAllTopicsReturnSequence, + List> mockGetAllPartitionsForTopicsReturnSequence) { + + super(topicsDescriptor, indexOfThisSubtask, numParallelSubtasks); + + this.topicsDescriptor = topicsDescriptor; + this.mockGetAllTopicsReturnSequence = mockGetAllTopicsReturnSequence; + this.mockGetAllPartitionsForTopicsReturnSequence = mockGetAllPartitionsForTopicsReturnSequence; + } + + @Override + protected List getAllTopics() { + assertTrue(topicsDescriptor.isTopicPattern()); + return mockGetAllTopicsReturnSequence.get(getAllTopicsInvokeCount++); + } + + @Override + protected List getAllPartitionsForTopics(List topics) { + if (topicsDescriptor.isFixedTopics()) { + assertEquals(topicsDescriptor.getFixedTopics(), topics); + } else { + assertEquals(mockGetAllTopicsReturnSequence.get(getAllPartitionsForTopicsInvokeCount - 1), topics); + } + return mockGetAllPartitionsForTopicsReturnSequence.get(getAllPartitionsForTopicsInvokeCount++); + } + + @Override + protected void initializeConnections() { + // nothing to do + } + + @Override + protected void wakeupConnections() { + // nothing to do + } + + @Override + protected void closeConnections() { + // nothing to do + } + } + + private static List> createMockGetAllTopicsSequenceFromFixedReturn(final List fixed) { + @SuppressWarnings("unchecked") + List> mockSequence = mock(List.class); + when(mockSequence.get(anyInt())).thenAnswer(new Answer>() { + @Override + public List answer(InvocationOnMock invocationOnMock) throws Throwable { + return new ArrayList<>(fixed); + } + }); + + return mockSequence; + } + + private static List> createMockGetAllPartitionsFromTopicsSequenceFromFixedReturn(final List fixed) { + @SuppressWarnings("unchecked") + List> mockSequence = mock(List.class); + when(mockSequence.get(anyInt())).thenAnswer(new Answer>() { + @Override + public List answer(InvocationOnMock invocationOnMock) throws Throwable { + return new ArrayList<>(fixed); + } + }); + + return mockSequence; + } + + private boolean contains(List partitions, int partition) { + for (KafkaTopicPartition ktp : partitions) { + if (ktp.getPartition() == partition) { + return true; + } + } + + return false; + } + + private List> deepClone(List> toClone) { + List> clone = new ArrayList<>(toClone.size()); + for (List partitionsToClone : toClone) { + List clonePartitions = new ArrayList<>(partitionsToClone.size()); + clonePartitions.addAll(partitionsToClone); + + clone.add(clonePartitions); + } + + return clone; + } +} diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueueTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueueTest.java similarity index 100% rename from flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueueTest.java rename to flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueueTest.java From bf2dd78e7706b9aae08b75ff917878e3f2ceb68b Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Thu, 20 Apr 2017 23:08:39 +0800 Subject: [PATCH 2/2] [FLINK-4022] Migrate to union list state --- .../kafka/FlinkKafkaConsumerBase.java | 217 +++++++++++------- .../AbstractPartitionDiscoverer.java | 56 +++-- .../kafka/internals/KafkaTopicPartition.java | 16 ++ .../kafka/FlinkKafkaConsumerBaseTest.java | 23 +- 4 files changed, 192 insertions(+), 120 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java index 4c58ea6d698c5..6a95b985ea793 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java @@ -20,7 +20,9 @@ import org.apache.commons.collections.map.LinkedMap; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.state.OperatorStateStore; +import org.apache.flink.api.common.typeinfo.TypeHint; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.ClosureCleaner; import org.apache.flink.api.java.tuple.Tuple2; @@ -53,6 +55,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.TreeMap; import java.util.concurrent.atomic.AtomicReference; import java.util.regex.Pattern; @@ -90,6 +93,9 @@ public abstract class FlinkKafkaConsumerBase extends RichParallelSourceFuncti /** Configuration key to define the consumer's partition discovery interval, in milliseconds */ public static final String KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS = "flink.partition-discovery.interval-millis"; + /** State name of the consumer's partition offset states */ + private static final String OFFSETS_STATE_NAME = "topic-partition-offset-states"; + // ------------------------------------------------------------------------ // configuration state, set on the client relevant for all subtasks // ------------------------------------------------------------------------ @@ -113,8 +119,6 @@ public abstract class FlinkKafkaConsumerBase extends RichParallelSourceFuncti * The assigner is kept in serialized form, to deserialize it into multiple copies */ private SerializedValue> punctuatedWatermarkAssigner; - private transient ListState> offsetsStateForCheckpoint; - /** * User-set flag determining whether or not to commit on checkpoints. * Note: this flag does not represent the final offset commit mode. @@ -135,7 +139,7 @@ public abstract class FlinkKafkaConsumerBase extends RichParallelSourceFuncti private StartupMode startupMode = StartupMode.GROUP_OFFSETS; /** Specific startup offsets; only relevant when startup mode is {@link StartupMode#SPECIFIC_OFFSETS} */ - protected Map specificStartupOffsets; + private Map specificStartupOffsets; // ------------------------------------------------------------------------ // runtime state (used individually by each parallel subtask) @@ -150,8 +154,25 @@ public abstract class FlinkKafkaConsumerBase extends RichParallelSourceFuncti /** The partition discoverer, used to find new partitions */ private transient volatile AbstractPartitionDiscoverer partitionDiscoverer; - /** The offsets to restore to, if the consumer restores state from a checkpoint */ - private transient volatile HashMap restoredState; + /** + * The offsets to restore to, if the consumer restores state from a checkpoint. + * + *

This map will be populated either by the legacy {@link #restoreState(HashMap)} method + * or {@link #initializeState(FunctionInitializationContext)}. + * + *

Using a sorted map as the ordering is important when using restored state + * to seed the partition discoverer. + */ + private transient volatile TreeMap restoredState; + + /** Accessor for state in the operator state backend. */ + private transient ListState> unionOffsetStates; + + /** + * Flag indicating whether the consumer is restored from older state written with Flink 1.1 or 1.2. + * When the current run is restored from older state, partition discovery is disabled. + */ + private boolean restoredFromOldState; /** Reference to the thread that invoked run(). This is used to interrupt partition discovery on shutdown */ private transient volatile Thread runThread; @@ -367,12 +388,17 @@ public void open(Configuration configuration) throws Exception { if (restoredState != null) { for (Map.Entry restoredStateEntry : restoredState.entrySet()) { - subscribedPartitionsToStartOffsets.put(restoredStateEntry.getKey(), restoredStateEntry.getValue()); - - // we also need to update what partitions the partition discoverer has seen already - // TODO Currently, after a restore, partition discovery will not work correctly - // TODO because the discoverer can not fully recover all globally seen records - partitionDiscoverer.checkAndSetDiscoveredPartition(restoredStateEntry.getKey()); + if (!restoredFromOldState) { + // seed the partition discoverer with the union state while filtering out + // restored partitions that should not be subscribed by this subtask + if (partitionDiscoverer.setAndCheckDiscoveredPartition(restoredStateEntry.getKey())) { + subscribedPartitionsToStartOffsets.put(restoredStateEntry.getKey(), restoredStateEntry.getValue()); + } + } else { + // when restoring from older 1.1 / 1.2 state, the restored state would not be the union state; + // in this case, just use the restored state as the subscribed partitions + subscribedPartitionsToStartOffsets.put(restoredStateEntry.getKey(), restoredStateEntry.getValue()); + } } LOG.info("Consumer subtask {} will start reading {} partitions with offsets in restored state: {}", @@ -487,69 +513,83 @@ public void run(SourceContext sourceContext) throws Exception { return; } - final AtomicReference fetcherErrorRef = new AtomicReference<>(); - Thread fetcherThread = new Thread(new Runnable() { - @Override - public void run() { - try { - // run the fetcher' main work method - kafkaFetcher.runFetchLoop(); - } catch (Exception e) { - fetcherErrorRef.set(e); - } finally { - // calling cancel will also let the partition discovery loop escape - cancel(); + // depending on whether we were restored with the current state version (1.3), + // remaining logic branches off into 2 paths: + // 1) New state - main fetcher loop executed as separate thread, with this + // thread running the partition discovery loop + // 2) Old state - partition discovery is disabled, simply going into the main fetcher loop + + if (!restoredFromOldState) { + final AtomicReference fetcherErrorRef = new AtomicReference<>(); + Thread fetcherThread = new Thread(new Runnable() { + @Override + public void run() { + try { + // run the fetcher' main work method + kafkaFetcher.runFetchLoop(); + } catch (Exception e) { + fetcherErrorRef.set(e); + } finally { + // calling cancel will also let the partition discovery loop escape + cancel(); + } } - } - }); - fetcherThread.start(); + }); + fetcherThread.start(); - // --------------------- partition discovery loop --------------------- + // --------------------- partition discovery loop --------------------- - List discoveredPartitions; + List discoveredPartitions; - // throughout the loop, we always eagerly check if we are still running before - // performing the next operation, so that we can escape the loop as soon as possible - while (running) { - if (LOG.isDebugEnabled()) { - LOG.debug("Consumer subtask {} is trying to discover new partitions ..."); - } + // throughout the loop, we always eagerly check if we are still running before + // performing the next operation, so that we can escape the loop as soon as possible - try { - discoveredPartitions = partitionDiscoverer.discoverPartitions(); - } catch (AbstractPartitionDiscoverer.WakeupException | AbstractPartitionDiscoverer.ClosedException e) { - // the partition discoverer may have been closed or woken up before or during the discovery; - // this would only happen if the consumer was canceled; simply escape the loop - break; - } - - // no need to add the discovered partitions if we were closed during the meantime - if (running && !discoveredPartitions.isEmpty()) { - fetcher.addDiscoveredPartitions(discoveredPartitions); - } + while (running) { + if (LOG.isDebugEnabled()) { + LOG.debug("Consumer subtask {} is trying to discover new partitions ..."); + } - // do not waste any time sleeping if we're not running anymore - if (running && discoveryIntervalMillis != 0) { try { - Thread.sleep(discoveryIntervalMillis); - } catch (InterruptedException iex) { - // may be interrupted if the consumer was canceled midway; simply escape the loop + discoveredPartitions = partitionDiscoverer.discoverPartitions(); + } catch (AbstractPartitionDiscoverer.WakeupException | AbstractPartitionDiscoverer.ClosedException e) { + // the partition discoverer may have been closed or woken up before or during the discovery; + // this would only happen if the consumer was canceled; simply escape the loop break; } + + // no need to add the discovered partitions if we were closed during the meantime + if (running && !discoveredPartitions.isEmpty()) { + fetcher.addDiscoveredPartitions(discoveredPartitions); + } + + // do not waste any time sleeping if we're not running anymore + if (running && discoveryIntervalMillis != 0) { + try { + Thread.sleep(discoveryIntervalMillis); + } catch (InterruptedException iex) { + // may be interrupted if the consumer was canceled midway; simply escape the loop + break; + } + } } - } - // -------------------------------------------------------------------- + // -------------------------------------------------------------------- - // make sure that the partition discoverer is properly closed - partitionDiscoverer.close(); + // make sure that the partition discoverer is properly closed + partitionDiscoverer.close(); - fetcherThread.join(); + fetcherThread.join(); + + // rethrow any fetcher errors + final Exception fetcherError = fetcherErrorRef.get(); + if (fetcherError != null) { + throw new RuntimeException(fetcherError); + } + } else { + // won't be using the discoverer + partitionDiscoverer.close(); - // rethrow any fetcher errors - final Exception fetcherError = fetcherErrorRef.get(); - if (fetcherError != null) { - throw new RuntimeException(fetcherError); + fetcher.runFetchLoop(); } } @@ -595,22 +635,30 @@ public void close() throws Exception { public void initializeState(FunctionInitializationContext context) throws Exception { OperatorStateStore stateStore = context.getOperatorStateStore(); - offsetsStateForCheckpoint = stateStore.getSerializableListState(DefaultOperatorStateBackend.DEFAULT_OPERATOR_STATE_NAME); - if (context.isRestored()) { - if (restoredState == null) { - restoredState = new HashMap<>(); - for (Tuple2 kafkaOffset : offsetsStateForCheckpoint.get()) { - restoredState.put(kafkaOffset.f0, kafkaOffset.f1); - } + ListState> oldRoundRobinListState = + stateStore.getSerializableListState(DefaultOperatorStateBackend.DEFAULT_OPERATOR_STATE_NAME); - LOG.info("Setting restore state in the FlinkKafkaConsumer."); - if (LOG.isDebugEnabled()) { - LOG.debug("Using the following offsets: {}", restoredState); - } - } else if (restoredState.isEmpty()) { - restoredState = null; + this.unionOffsetStates = stateStore.getUnionListState(new ListStateDescriptor<>( + OFFSETS_STATE_NAME, + TypeInformation.of(new TypeHint>() {}))); + + if (context.isRestored() && !restoredFromOldState) { + restoredState = new TreeMap<>(new KafkaTopicPartition.Comparator()); + + // migrate from 1.2 state, if there is any + for (Tuple2 kafkaOffset : oldRoundRobinListState.get()) { + restoredFromOldState = true; + unionOffsetStates.add(kafkaOffset); } + oldRoundRobinListState.clear(); + + // populate actual holder for restored state + for (Tuple2 kafkaOffset : unionOffsetStates.get()) { + restoredState.put(kafkaOffset.f0, kafkaOffset.f1); + } + + LOG.info("Setting restore state in the FlinkKafkaConsumer: {}", restoredState); } else { LOG.info("No restore state for FlinkKafkaConsumer."); } @@ -621,15 +669,14 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { if (!running) { LOG.debug("snapshotState() called on closed source"); } else { - - offsetsStateForCheckpoint.clear(); + unionOffsetStates.clear(); final AbstractFetcher fetcher = this.kafkaFetcher; if (fetcher == null) { // the fetcher has not yet been initialized, which means we need to return the // originally restored offsets or the assigned partitions for (Map.Entry subscribedPartition : subscribedPartitionsToStartOffsets.entrySet()) { - offsetsStateForCheckpoint.add(Tuple2.of(subscribedPartition.getKey(), subscribedPartition.getValue())); + unionOffsetStates.add(Tuple2.of(subscribedPartition.getKey(), subscribedPartition.getValue())); } if (offsetCommitMode == OffsetCommitMode.ON_CHECKPOINTS) { @@ -647,7 +694,7 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { } for (Map.Entry kafkaTopicPartitionLongEntry : currentOffsets.entrySet()) { - offsetsStateForCheckpoint.add( + unionOffsetStates.add( Tuple2.of(kafkaTopicPartitionLongEntry.getKey(), kafkaTopicPartitionLongEntry.getValue())); } } @@ -663,14 +710,16 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { @Override public void restoreState(HashMap restoredOffsets) { - LOG.info("{} (taskIdx={}) restoring offsets from an older version.", - getClass().getSimpleName(), getRuntimeContext().getIndexOfThisSubtask()); + LOG.info("{} (taskIdx={}) restoring offsets from an older version: {}", + getClass().getSimpleName(), getRuntimeContext().getIndexOfThisSubtask(), restoredOffsets); - restoredState = restoredOffsets.isEmpty() ? null : restoredOffsets; + restoredFromOldState = true; - if (LOG.isDebugEnabled()) { - LOG.debug("{} (taskIdx={}) restored offsets from an older Flink version: {}", - getClass().getSimpleName(), getRuntimeContext().getIndexOfThisSubtask(), restoredState); + if (restoredOffsets.isEmpty()) { + restoredState = null; + } else { + restoredState = new TreeMap<>(new KafkaTopicPartition.Comparator()); + restoredState.putAll(restoredOffsets); } } @@ -784,7 +833,7 @@ Map getSubscribedPartitionsToStartOffsets() { } @VisibleForTesting - HashMap getRestoredState() { + TreeMap getRestoredState() { return restoredState; } } diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscoverer.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscoverer.java index 2fc6e9ae59adc..c689d5cb0de48 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscoverer.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscoverer.java @@ -17,8 +17,6 @@ package org.apache.flink.streaming.connectors.kafka.internals; -import java.util.Collections; -import java.util.Comparator; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -64,7 +62,7 @@ public abstract class AbstractPartitionDiscoverer { /** * Map of topics to they're largest discovered partition id seen by this subtask. * This state may be updated whenever {@link AbstractPartitionDiscoverer#discoverPartitions()} or - * {@link AbstractPartitionDiscoverer#checkAndSetDiscoveredPartition(KafkaTopicPartition)} is called. + * {@link AbstractPartitionDiscoverer#setAndCheckDiscoveredPartition(KafkaTopicPartition)} is called. * * This is used to remove old partitions from the fetched partition lists. It is sufficient * to keep track of only the largest partition id because Kafka partition numbers are only @@ -153,23 +151,13 @@ public List discoverPartitions() throws WakeupException, Cl } else { // sort so that we make sure the topicsToLargestDiscoveredPartitionId state is updated // with incremental partition ids of the same topics (otherwise some partition ids may be skipped) - Collections.sort(newDiscoveredPartitions, new Comparator() { - @Override - public int compare(KafkaTopicPartition o1, KafkaTopicPartition o2) { - if (!o1.getTopic().equals(o2.getTopic())) { - return o1.getTopic().compareTo(o2.getTopic()); - } else { - return Integer.compare(o1.getPartition(), o2.getPartition()); - } - } - }); + KafkaTopicPartition.sort(newDiscoveredPartitions); Iterator iter = newDiscoveredPartitions.iterator(); KafkaTopicPartition nextPartition; while (iter.hasNext()) { nextPartition = iter.next(); - if (!checkAndSetDiscoveredPartition(nextPartition) || - !shouldAssignToThisSubtask(nextPartition, indexOfThisSubtask, numParallelSubtasks)) { + if (!setAndCheckDiscoveredPartition(nextPartition)) { iter.remove(); } } @@ -192,20 +180,25 @@ public int compare(KafkaTopicPartition o1, KafkaTopicPartition o2) { } /** - * Checks whether the given partition has been discovered yet. - * If it hasn't been discovered yet, this method lets the partition discoverer update what - * partitions it has discovered so far. + * Sets a partition as discovered. Partitions are considered as new + * if its partition id is larger than all partition ids previously + * seen for the topic it belongs to. Therefore, for a set of + * discovered partitions, the order that this method is invoked with + * each partition is important. + * + * If the partition is indeed newly discovered, this method also returns + * whether the new partition should be subscribed by this subtask. * - * @param partition the partition to check and set + * @param partition the partition to set and check * - * @return {@code true}, if the partition wasn't seen before, {@code false} otherwise + * @return {@code true}, if the partition wasn't seen before and should + * be subscribed by this subtask; {@code false} otherwise */ - public boolean checkAndSetDiscoveredPartition(KafkaTopicPartition partition) { - if (!topicsToLargestDiscoveredPartitionId.containsKey(partition.getTopic()) - || partition.getPartition() > topicsToLargestDiscoveredPartitionId.get(partition.getTopic())) { - + public boolean setAndCheckDiscoveredPartition(KafkaTopicPartition partition) { + if (isUndiscoveredPartition(partition)) { topicsToLargestDiscoveredPartitionId.put(partition.getTopic(), partition.getPartition()); - return true; + + return shouldAssignToThisSubtask(partition, indexOfThisSubtask, numParallelSubtasks); } return false; @@ -242,10 +235,6 @@ public boolean checkAndSetDiscoveredPartition(KafkaTopicPartition partition) { // Utilities // ------------------------------------------------------------------------ - private static boolean shouldAssignToThisSubtask(KafkaTopicPartition partition, int indexOfThisSubtask, int numParallelSubtasks) { - return Math.abs(partition.hashCode() % numParallelSubtasks) == indexOfThisSubtask; - } - public static final class WakeupException extends Exception { private static final long serialVersionUID = 1L; } @@ -253,4 +242,13 @@ public static final class WakeupException extends Exception { public static final class ClosedException extends Exception { private static final long serialVersionUID = 1L; } + + private boolean isUndiscoveredPartition(KafkaTopicPartition partition) { + return !topicsToLargestDiscoveredPartitionId.containsKey(partition.getTopic()) + || partition.getPartition() > topicsToLargestDiscoveredPartitionId.get(partition.getTopic()); + } + + private static boolean shouldAssignToThisSubtask(KafkaTopicPartition partition, int indexOfThisSubtask, int numParallelSubtasks) { + return Math.abs(partition.hashCode() % numParallelSubtasks) == indexOfThisSubtask; + } } diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java index c68fe28a9b172..467afee19ca56 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java @@ -19,6 +19,7 @@ import java.io.Serializable; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; @@ -117,4 +118,19 @@ public static List dropLeaderData(List { + @Override + public int compare(KafkaTopicPartition p1, KafkaTopicPartition p2) { + if (!p1.getTopic().equals(p2.getTopic())) { + return p1.getTopic().compareTo(p2.getTopic()); + } else { + return Integer.compare(p1.getPartition(), p2.getPartition()); + } + } + } + + public static void sort(List partitions) { + Collections.sort(partitions, new Comparator()); + } } diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java index b8c5abf4ee63c..0d5a0013259ec 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java @@ -129,8 +129,10 @@ public void checkRestoredCheckpointWhenFetcherNotReady() throws Exception { listState.add(Tuple2.of(new KafkaTopicPartition("def", 7), 987654321L)); FlinkKafkaConsumerBase consumer = getConsumer(null, new LinkedMap(), true); - - when(operatorStateStore.getSerializableListState(Matchers.any(String.class))).thenReturn(listState); + // mock old 1.2 state (empty) + when(operatorStateStore.getSerializableListState(Matchers.any(String.class))).thenReturn(new TestingListState()); + // mock 1.3 state + when(operatorStateStore.getUnionListState(Matchers.any(ListStateDescriptor.class))).thenReturn(listState); StateInitializationContext initializationContext = mock(StateInitializationContext.class); @@ -172,7 +174,10 @@ public void checkRestoredNullCheckpointWhenFetcherNotReady() throws Exception { OperatorStateStore operatorStateStore = mock(OperatorStateStore.class); TestingListState listState = new TestingListState<>(); - when(operatorStateStore.getSerializableListState(Matchers.any(String.class))).thenReturn(listState); + // mock old 1.2 state (empty) + when(operatorStateStore.getSerializableListState(Matchers.any(String.class))).thenReturn(new TestingListState()); + // mock 1.3 state + when(operatorStateStore.getUnionListState(Matchers.any(ListStateDescriptor.class))).thenReturn(listState); StateInitializationContext initializationContext = mock(StateInitializationContext.class); @@ -225,8 +230,10 @@ public void testSnapshotStateWithCommitOnCheckpointsEnabled() throws Exception { OperatorStateStore backend = mock(OperatorStateStore.class); TestingListState listState = new TestingListState<>(); - - when(backend.getSerializableListState(Matchers.any(String.class))).thenReturn(listState); + // mock old 1.2 state (empty) + when(backend.getSerializableListState(Matchers.any(String.class))).thenReturn(new TestingListState()); + // mock 1.3 state + when(backend.getUnionListState(Matchers.any(ListStateDescriptor.class))).thenReturn(listState); StateInitializationContext initializationContext = mock(StateInitializationContext.class); @@ -353,8 +360,10 @@ public void testSnapshotStateWithCommitOnCheckpointsDisabled() throws Exception OperatorStateStore backend = mock(OperatorStateStore.class); TestingListState listState = new TestingListState<>(); - - when(backend.getSerializableListState(Matchers.any(String.class))).thenReturn(listState); + // mock old 1.2 state (empty) + when(backend.getSerializableListState(Matchers.any(String.class))).thenReturn(new TestingListState()); + // mock 1.3 state + when(backend.getUnionListState(Matchers.any(ListStateDescriptor.class))).thenReturn(listState); StateInitializationContext initializationContext = mock(StateInitializationContext.class);