becketqin commented on a change in pull request #13574: URL: https://github.com/apache/flink/pull/13574#discussion_r502855189
########## File path: flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumerator.java ########## @@ -0,0 +1,370 @@ +/* + 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.connector.kafka.source.enumerator; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.connector.source.SplitsAssignment; +import org.apache.flink.connector.base.source.event.NoMoreSplitsEvent; +import org.apache.flink.connector.kafka.source.KafkaSourceOptions; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; +import org.apache.flink.connector.kafka.source.enumerator.subscriber.KafkaSubscriber; +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.function.ThrowingRunnable; + +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.ListConsumerGroupOffsetsOptions; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.OffsetAndTimestamp; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.ExecutionException; + +import static org.apache.flink.util.ComponentClosingUtils.closeWithTimeout; + +/** + * The enumerator class for Kafka source. + */ +@Internal +public class KafkaSourceEnumerator implements SplitEnumerator<KafkaPartitionSplit, KafkaSourceEnumState> { + private static final Logger LOG = LoggerFactory.getLogger(KafkaSourceEnumerator.class); + private final KafkaSubscriber subscriber; + private final OffsetsInitializer startingOffsetInitializer; + private final OffsetsInitializer stoppingOffsetInitializer; + private final Properties properties; + private final long partitionDiscoveryIntervalMs; + private final SplitEnumeratorContext<KafkaPartitionSplit> context; + + // The internal states of the enumerator. + /** This set is only accessed by the partition discovery callable in the callAsync() method, i.e worker thread. */ + private final Set<TopicPartition> discoveredPartitions; + /** The current assignment by reader id. Only accessed by the coordinator thread. */ + private final Map<Integer, Set<KafkaPartitionSplit>> readerIdToSplitAssignments; + /** The discovered and initialized partition splits that are waiting for owner reader to be ready. */ + private final Map<Integer, Set<KafkaPartitionSplit>> pendingPartitionSplitAssignment; + /** The consumer group id used for this KafkaSource. */ + private final String consumerGroupId; + + // Lazily instantiated or mutable fields. + private KafkaConsumer<byte[], byte[]> consumer; + private AdminClient adminClient; + private boolean noMoreNewPartitionSplits = false; + + public KafkaSourceEnumerator( + KafkaSubscriber subscriber, + OffsetsInitializer startingOffsetInitializer, + OffsetsInitializer stoppingOffsetInitializer, + Properties properties, + SplitEnumeratorContext<KafkaPartitionSplit> context) { + this(subscriber, startingOffsetInitializer, stoppingOffsetInitializer, properties, context, new HashMap<>()); + } + + public KafkaSourceEnumerator( + KafkaSubscriber subscriber, + OffsetsInitializer startingOffsetInitializer, + OffsetsInitializer stoppingOffsetInitializer, + Properties properties, + SplitEnumeratorContext<KafkaPartitionSplit> context, + Map<Integer, Set<KafkaPartitionSplit>> currentSplitsAssignments) { + this.subscriber = subscriber; + this.startingOffsetInitializer = startingOffsetInitializer; + this.stoppingOffsetInitializer = stoppingOffsetInitializer; + this.properties = properties; + this.context = context; + + this.discoveredPartitions = new HashSet<>(); + this.readerIdToSplitAssignments = new HashMap<>(currentSplitsAssignments); + this.readerIdToSplitAssignments.forEach((reader, splits) -> + splits.forEach(s -> discoveredPartitions.add(s.getTopicPartition()))); + this.pendingPartitionSplitAssignment = new HashMap<>(); + this.partitionDiscoveryIntervalMs = KafkaSourceOptions.getOption( + properties, + KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS, + Long::parseLong); + this.consumerGroupId = properties.getProperty(ConsumerConfig.GROUP_ID_CONFIG); + } + + @Override + public void start() { + consumer = getKafkaConsumer(); + adminClient = getKafkaAdminClient(); + if (partitionDiscoveryIntervalMs > 0) { + LOG.info("Starting the KafkaSourceEnumerator for consumer group {} " + + "with partition discovery interval of {} ms.", consumerGroupId, partitionDiscoveryIntervalMs); + context.callAsync( + this::discoverAndInitializePartitionSplit, + this::handlePartitionSplitChanges, + 0, + partitionDiscoveryIntervalMs); + } else { + LOG.info("Starting the KafkaSourceEnumerator for consumer group {} " + + "without periodic partition discovery.", consumerGroupId); + context.callAsync( + this::discoverAndInitializePartitionSplit, + this::handlePartitionSplitChanges); Review comment: Right, there is only one thread in `SplitEnumerator`, so everything is sequentialized. If a reader disconnected while the splits assignment is in progress, the registered readers will only get updated after the current split assignment has finished. ########## File path: flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReader.java ########## @@ -0,0 +1,398 @@ +/* + 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.connector.kafka.source.reader; + +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange; +import org.apache.flink.connector.kafka.source.KafkaSourceOptions; +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializer; +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit; +import org.apache.flink.util.Collector; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.Preconditions; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.WakeupException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Random; +import java.util.Set; +import java.util.StringJoiner; + +/** + * A {@link SplitReader} implementation that reads records from Kafka partitions. + * + * <p>The returned type are in the format of {@code tuple3(record, offset and timestamp}. + * + * @param <T> the type of the record to be emitted from the Source. + */ +public class KafkaPartitionSplitReader<T> implements SplitReader<Tuple3<T, Long, Long>, KafkaPartitionSplit> { + private static final Logger LOG = LoggerFactory.getLogger(KafkaPartitionSplitReader.class); + private static final long POLL_TIMEOUT = 10000L; + + private final KafkaConsumer<byte[], byte[]> consumer; + private final KafkaRecordDeserializer<T> deserializationSchema; + private final Map<TopicPartition, Long> stoppingOffsets; + private final SimpleCollector<T> collector; + private final String groupId; + + public KafkaPartitionSplitReader( + Properties props, + KafkaRecordDeserializer<T> deserializationSchema) { + Properties consumerProps = new Properties(); + consumerProps.putAll(props); + consumerProps.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, createConsumerClientId(props)); + this.consumer = new KafkaConsumer<>(consumerProps); + this.stoppingOffsets = new HashMap<>(); + this.deserializationSchema = deserializationSchema; + this.collector = new SimpleCollector<>(); + this.groupId = consumerProps.getProperty(ConsumerConfig.GROUP_ID_CONFIG); + } + + @Override + public RecordsWithSplitIds<Tuple3<T, Long, Long>> fetch() throws IOException { + KafkaPartitionSplitRecords<Tuple3<T, Long, Long>> recordsBySplits = new KafkaPartitionSplitRecords<>(); + ConsumerRecords<byte[], byte[]> consumerRecords; + try { + consumerRecords = consumer.poll(Duration.ofMillis(POLL_TIMEOUT)); + } catch (WakeupException we) { + return recordsBySplits; + } + + List<TopicPartition> finishedPartitions = new ArrayList<>(); + for (TopicPartition tp : consumerRecords.partitions()) { + long stoppingOffset = getStoppingOffset(tp); + String splitId = tp.toString(); + Collection<Tuple3<T, Long, Long>> recordsForSplit = recordsBySplits.recordsForSplit(splitId); + for (ConsumerRecord<byte[], byte[]> consumerRecord : consumerRecords.records(tp)) { + // Stop consuming from this partition if the offsets has reached the stopping offset. + // Note that there are two cases, either case finishes a split: + // 1. After processing a record with offset of "stoppingOffset - 1". The split reader + // should not continue fetching because the record with stoppingOffset may not exist. + // 2. Before processing a record whose offset is greater than or equals to the stopping + // offset. This should only happens when case 1 was not met due to log compaction or + // log retention. + // Case 2 is handled here. Case 1 is handled after the record is processed. + if (consumerRecord.offset() >= stoppingOffset) { + finishSplitAtRecord(tp, stoppingOffset, consumerRecord.offset(), + finishedPartitions, recordsBySplits); + break; + } + // Add the record to the partition collector. + try { + deserializationSchema.deserialize(consumerRecord, collector); + collector.getRecords().forEach(r -> + recordsForSplit.add(new Tuple3<>(r, + consumerRecord.offset(), + consumerRecord.timestamp()))); + // Finish the split because there might not be any message after this point. Keep polling + // will just block forever. + if (consumerRecord.offset() == stoppingOffset - 1) { + finishSplitAtRecord(tp, stoppingOffset, consumerRecord.offset(), + finishedPartitions, recordsBySplits); + } + } catch (Exception e) { + throw new IOException("Failed to deserialize consumer record due to", e); + } finally { + collector.reset(); + } + } + } + // Unassign the partitions that has finished. + if (!finishedPartitions.isEmpty()) { + unassignPartitions(finishedPartitions); + } + recordsBySplits.prepareForRead(); + return recordsBySplits; + } + + @Override + public void handleSplitsChanges(SplitsChange<KafkaPartitionSplit> splitsChange) { + // Get all the partition assignments and stopping offsets. + if (!(splitsChange instanceof SplitsAddition)) { + throw new UnsupportedOperationException(String.format( + "The SplitChange type of %s is not supported.", splitsChange.getClass())); + } + + // Assignment. + List<TopicPartition> newPartitionAssignments = new ArrayList<>(); + // Starting offsets. + Map<TopicPartition, Long> partitionsStartingFromSpecifiedOffsets = new HashMap<>(); + List<TopicPartition> partitionsStartingFromEarliest = new ArrayList<>(); + List<TopicPartition> partitionsStartingFromLatest = new ArrayList<>(); + // Stopping offsets. + List<TopicPartition> partitionsStoppingAtLatest = new ArrayList<>(); + Set<TopicPartition> partitionsStoppingAtCommitted = new HashSet<>(); + + // Parse the starting and stopping offsets. + splitsChange.splits().forEach(s -> { + newPartitionAssignments.add(s.getTopicPartition()); + parseStartingOffsets(s, partitionsStartingFromEarliest, partitionsStartingFromLatest, partitionsStartingFromSpecifiedOffsets); + parseStoppingOffsets(s, partitionsStoppingAtLatest, partitionsStoppingAtCommitted); + }); + + // Assign new partitions. + newPartitionAssignments.addAll(consumer.assignment()); + consumer.assign(newPartitionAssignments); + + // Seek on the newly assigned partitions to their stating offsets. + seekToStartingOffsets(partitionsStartingFromEarliest, partitionsStartingFromLatest, partitionsStartingFromSpecifiedOffsets); + // Setup the stopping offsets. + acquireAndSetStoppingOffsets(partitionsStoppingAtLatest, partitionsStoppingAtCommitted); + + // After acquiring the starting and stopping offsets, remove the empty splits if necessary. + removeEmptySplits(); + + maybeLogSplitChangesHandlingResult(splitsChange); + } + + @Override + public void wakeUp() { + consumer.wakeup(); + } + + // --------------- private helper method ---------------------- + + private void parseStartingOffsets( + KafkaPartitionSplit split, + List<TopicPartition> partitionsStartingFromEarliest, + List<TopicPartition> partitionsStartingFromLatest, + Map<TopicPartition, Long> partitionsStartingFromSpecifiedOffsets) { + TopicPartition tp = split.getTopicPartition(); + // Parse starting offsets. + if (split.getStartingOffset() == KafkaPartitionSplit.EARLIEST_OFFSET) { + partitionsStartingFromEarliest.add(tp); + } else if (split.getStartingOffset() == KafkaPartitionSplit.LATEST_OFFSET) { + partitionsStartingFromLatest.add(tp); + } else if (split.getStartingOffset() == KafkaPartitionSplit.COMMITTED_OFFSET) { + // Do nothing here, the consumer will first try to get the committed offsets of + // these partitions by default. + } else { + partitionsStartingFromSpecifiedOffsets.put(tp, split.getStartingOffset()); + } + } + + private void parseStoppingOffsets( + KafkaPartitionSplit split, + List<TopicPartition> partitionsStoppingAtLatest, + Set<TopicPartition> partitionsStoppingAtCommitted) { + TopicPartition tp = split.getTopicPartition(); + split.getStoppingOffset().ifPresent(stoppingOffset -> { + if (stoppingOffset >= 0) { + stoppingOffsets.put(tp, stoppingOffset); + } else if (stoppingOffset == KafkaPartitionSplit.LATEST_OFFSET) { + partitionsStoppingAtLatest.add(tp); + } else if (stoppingOffset == KafkaPartitionSplit.COMMITTED_OFFSET) { + partitionsStoppingAtCommitted.add(tp); + } else { + // This should not happen. + throw new FlinkRuntimeException(String.format( + "Invalid stopping offset %d for partition %s", stoppingOffset, tp)); + } + }); + } + + private void seekToStartingOffsets( + List<TopicPartition> partitionsStartingFromEarliest, + List<TopicPartition> partitionsStartingFromLatest, + Map<TopicPartition, Long> partitionsStartingFromSpecifiedOffsets) { + + if (!partitionsStartingFromEarliest.isEmpty()) { + LOG.trace("Seeking starting offsets to beginning: {}", partitionsStartingFromEarliest); + consumer.seekToBeginning(partitionsStartingFromEarliest); + } + + if (!partitionsStartingFromLatest.isEmpty()) { + LOG.trace("Seeking starting offsets to end: {}", partitionsStartingFromLatest); + consumer.seekToEnd(partitionsStartingFromLatest); + } + + if (!partitionsStartingFromSpecifiedOffsets.isEmpty()) { + LOG.trace("Seeking starting offsets to specified offsets: {}", partitionsStartingFromSpecifiedOffsets); + partitionsStartingFromSpecifiedOffsets.forEach(consumer::seek); + } + } + + private void acquireAndSetStoppingOffsets( + List<TopicPartition> partitionsStoppingAtLatest, + Set<TopicPartition> partitionsStoppingAtCommitted) { + Map<TopicPartition, Long> endOffset = consumer.endOffsets(partitionsStoppingAtLatest); + stoppingOffsets.putAll(endOffset); + consumer.committed(partitionsStoppingAtCommitted).forEach((tp, offsetAndMetadata) -> { + Preconditions.checkNotNull(offsetAndMetadata, String.format( + "Partition %s should stop at committed offset. " + + "But there is no committed offset of this partition for group %s", tp, groupId)); + stoppingOffsets.put(tp, offsetAndMetadata.offset()); + }); + } + + private void removeEmptySplits() { + List<TopicPartition> emptySplits = new ArrayList<>(); + // If none of the partitions have any records, + for (TopicPartition tp : consumer.assignment()) { + if (consumer.position(tp) >= getStoppingOffset(tp)) { + emptySplits.add(tp); + } + } + if (!emptySplits.isEmpty()) { + unassignPartitions(emptySplits); + } + } + + private void maybeLogSplitChangesHandlingResult(SplitsChange<KafkaPartitionSplit> splitsChange) { + if (LOG.isDebugEnabled()) { + StringJoiner splitsInfo = new StringJoiner(","); + for (KafkaPartitionSplit split : splitsChange.splits()) { + long startingOffset = consumer.position(split.getTopicPartition()); + long stoppingOffset = getStoppingOffset(split.getTopicPartition()); + splitsInfo.add(String.format("[%s, start:%d, stop: %d]", + split.getTopicPartition(), startingOffset, stoppingOffset)); + } + LOG.debug("SplitsChange handling result: {}", splitsInfo.toString()); + } + } + + private void unassignPartitions(Collection<TopicPartition> partitionsToUnassign) { + Collection<TopicPartition> newAssignment = new HashSet<>(consumer.assignment()); + newAssignment.removeAll(partitionsToUnassign); + consumer.assign(newAssignment); + } + + private String createConsumerClientId(Properties props) { + String prefix = props.getProperty(KafkaSourceOptions.CLIENT_ID_PREFIX.key()); + return prefix + "-" + new Random().nextLong(); Review comment: Yes, [FLINK-18044](https://issues.apache.org/jira/browse/FLINK-18044) was created for this. ########## File path: flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumerator.java ########## @@ -0,0 +1,370 @@ +/* + 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.connector.kafka.source.enumerator; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.connector.source.SplitsAssignment; +import org.apache.flink.connector.base.source.event.NoMoreSplitsEvent; +import org.apache.flink.connector.kafka.source.KafkaSourceOptions; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; +import org.apache.flink.connector.kafka.source.enumerator.subscriber.KafkaSubscriber; +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.function.ThrowingRunnable; + +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.ListConsumerGroupOffsetsOptions; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.OffsetAndTimestamp; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.ExecutionException; + +import static org.apache.flink.util.ComponentClosingUtils.closeWithTimeout; + +/** + * The enumerator class for Kafka source. + */ +@Internal +public class KafkaSourceEnumerator implements SplitEnumerator<KafkaPartitionSplit, KafkaSourceEnumState> { + private static final Logger LOG = LoggerFactory.getLogger(KafkaSourceEnumerator.class); + private final KafkaSubscriber subscriber; + private final OffsetsInitializer startingOffsetInitializer; + private final OffsetsInitializer stoppingOffsetInitializer; + private final Properties properties; + private final long partitionDiscoveryIntervalMs; + private final SplitEnumeratorContext<KafkaPartitionSplit> context; + + // The internal states of the enumerator. + /** This set is only accessed by the partition discovery callable in the callAsync() method, i.e worker thread. */ + private final Set<TopicPartition> discoveredPartitions; + /** The current assignment by reader id. Only accessed by the coordinator thread. */ + private final Map<Integer, Set<KafkaPartitionSplit>> readerIdToSplitAssignments; + /** The discovered and initialized partition splits that are waiting for owner reader to be ready. */ + private final Map<Integer, Set<KafkaPartitionSplit>> pendingPartitionSplitAssignment; + /** The consumer group id used for this KafkaSource. */ + private final String consumerGroupId; + + // Lazily instantiated or mutable fields. + private KafkaConsumer<byte[], byte[]> consumer; + private AdminClient adminClient; + private boolean noMoreNewPartitionSplits = false; + + public KafkaSourceEnumerator( + KafkaSubscriber subscriber, + OffsetsInitializer startingOffsetInitializer, + OffsetsInitializer stoppingOffsetInitializer, + Properties properties, + SplitEnumeratorContext<KafkaPartitionSplit> context) { + this(subscriber, startingOffsetInitializer, stoppingOffsetInitializer, properties, context, new HashMap<>()); + } + + public KafkaSourceEnumerator( + KafkaSubscriber subscriber, + OffsetsInitializer startingOffsetInitializer, + OffsetsInitializer stoppingOffsetInitializer, + Properties properties, + SplitEnumeratorContext<KafkaPartitionSplit> context, + Map<Integer, Set<KafkaPartitionSplit>> currentSplitsAssignments) { + this.subscriber = subscriber; + this.startingOffsetInitializer = startingOffsetInitializer; + this.stoppingOffsetInitializer = stoppingOffsetInitializer; + this.properties = properties; + this.context = context; + + this.discoveredPartitions = new HashSet<>(); + this.readerIdToSplitAssignments = new HashMap<>(currentSplitsAssignments); + this.readerIdToSplitAssignments.forEach((reader, splits) -> + splits.forEach(s -> discoveredPartitions.add(s.getTopicPartition()))); + this.pendingPartitionSplitAssignment = new HashMap<>(); + this.partitionDiscoveryIntervalMs = KafkaSourceOptions.getOption( + properties, + KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS, + Long::parseLong); + this.consumerGroupId = properties.getProperty(ConsumerConfig.GROUP_ID_CONFIG); + } + + @Override + public void start() { + consumer = getKafkaConsumer(); + adminClient = getKafkaAdminClient(); + if (partitionDiscoveryIntervalMs > 0) { + LOG.info("Starting the KafkaSourceEnumerator for consumer group {} " + + "with partition discovery interval of {} ms.", consumerGroupId, partitionDiscoveryIntervalMs); + context.callAsync( + this::discoverAndInitializePartitionSplit, + this::handlePartitionSplitChanges, + 0, + partitionDiscoveryIntervalMs); + } else { + LOG.info("Starting the KafkaSourceEnumerator for consumer group {} " + + "without periodic partition discovery.", consumerGroupId); + context.callAsync( + this::discoverAndInitializePartitionSplit, + this::handlePartitionSplitChanges); + } + } + + @Override + public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) { + + } + + @Override + public void addSplitsBack(List<KafkaPartitionSplit> splits, int subtaskId) { + addPartitionSplitChangeToPendingAssignments(splits); + assignPendingPartitionSplits(); + } + + @Override + public void addReader(int subtaskId) { + LOG.debug("Adding reader {} to KafkaSourceEnumerator for consumer group {}.", subtaskId, consumerGroupId); + assignPendingPartitionSplits(); + } + + @Override + public KafkaSourceEnumState snapshotState() throws Exception { + return new KafkaSourceEnumState(readerIdToSplitAssignments); + } + + @Override + public void close() throws IOException { + // When close the split enumerator, we need to make sure that the async calls are canceled + // before we can close the consumer and admin clients. + long closeTimeoutMs = KafkaSourceOptions.getOption( + properties, + KafkaSourceOptions.CLOSE_TIMEOUT_MS, + Long::parseLong); + close(closeTimeoutMs).ifPresent(t -> LOG.warn("Encountered error when closing KafkaSourceEnumerator", t)); + } + + @VisibleForTesting + Optional<Throwable> close(long timeoutMs) { + return closeWithTimeout( + "KafkaSourceEnumerator", + (ThrowingRunnable<Exception>) () -> { + consumer.close(); + adminClient.close(); + }, + timeoutMs); + } + + // ----------------- private methods ------------------- + + private PartitionSplitChange discoverAndInitializePartitionSplit() { + // Make a copy of the partitions to owners + KafkaSubscriber.PartitionChange partitionChange = + subscriber.getPartitionChanges(consumer, Collections.unmodifiableSet(discoveredPartitions)); + + Set<TopicPartition> newPartitions = Collections.unmodifiableSet(partitionChange.getNewPartitions()); + OffsetsInitializer.PartitionOffsetsRetriever offsetsRetriever = getOffsetsRetriever(); + + Map<TopicPartition, Long> startingOffsets = + startingOffsetInitializer.getPartitionOffsets(newPartitions, offsetsRetriever); + Map<TopicPartition, Long> stoppingOffsets = + stoppingOffsetInitializer.getPartitionOffsets(newPartitions, offsetsRetriever); + + Set<KafkaPartitionSplit> partitionSplits = new HashSet<>(newPartitions.size()); + for (TopicPartition tp : newPartitions) { + Long startingOffset = startingOffsets.get(tp); + long stoppingOffset = stoppingOffsets.getOrDefault(tp, KafkaPartitionSplit.NO_STOPPING_OFFSET); + partitionSplits.add(new KafkaPartitionSplit(tp, startingOffset, stoppingOffset)); + } + discoveredPartitions.addAll(newPartitions); + return new PartitionSplitChange(partitionSplits, partitionChange.getRemovedPartitions()); + } + + // This method should only be invoked in the coordinator executor thread. + private void handlePartitionSplitChanges(PartitionSplitChange partitionSplitChange, Throwable t) { + if (t != null) { + throw new FlinkRuntimeException("Failed to handle partition splits change due to ", t); + } + if (partitionDiscoveryIntervalMs < 0) { + LOG.debug(""); + noMoreNewPartitionSplits = true; + } + // TODO: Handle removed partitions. + addPartitionSplitChangeToPendingAssignments(partitionSplitChange.newPartitionSplits); + assignPendingPartitionSplits(); + } + + // This method should only be invoked in the coordinator executor thread. + private void addPartitionSplitChangeToPendingAssignments(Collection<KafkaPartitionSplit> newPartitionSplits) { + int numReaders = context.currentParallelism(); + for (KafkaPartitionSplit split : newPartitionSplits) { + // TODO: Implement a more sophisticated algorithm to reduce partition movement when parallelism changes. + int ownerReader = split.getTopicPartition().hashCode() % numReaders; Review comment: Reusing the `KafkaTopicPartitionAssigner` logic here sounds good. ########## File path: flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReader.java ########## @@ -0,0 +1,398 @@ +/* + 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.connector.kafka.source.reader; + +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange; +import org.apache.flink.connector.kafka.source.KafkaSourceOptions; +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializer; +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit; +import org.apache.flink.util.Collector; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.Preconditions; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.WakeupException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Random; +import java.util.Set; +import java.util.StringJoiner; + +/** + * A {@link SplitReader} implementation that reads records from Kafka partitions. + * + * <p>The returned type are in the format of {@code tuple3(record, offset and timestamp}. + * + * @param <T> the type of the record to be emitted from the Source. + */ +public class KafkaPartitionSplitReader<T> implements SplitReader<Tuple3<T, Long, Long>, KafkaPartitionSplit> { + private static final Logger LOG = LoggerFactory.getLogger(KafkaPartitionSplitReader.class); + private static final long POLL_TIMEOUT = 10000L; + + private final KafkaConsumer<byte[], byte[]> consumer; + private final KafkaRecordDeserializer<T> deserializationSchema; + private final Map<TopicPartition, Long> stoppingOffsets; + private final SimpleCollector<T> collector; + private final String groupId; + + public KafkaPartitionSplitReader( + Properties props, + KafkaRecordDeserializer<T> deserializationSchema) { + Properties consumerProps = new Properties(); + consumerProps.putAll(props); + consumerProps.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, createConsumerClientId(props)); + this.consumer = new KafkaConsumer<>(consumerProps); + this.stoppingOffsets = new HashMap<>(); + this.deserializationSchema = deserializationSchema; + this.collector = new SimpleCollector<>(); + this.groupId = consumerProps.getProperty(ConsumerConfig.GROUP_ID_CONFIG); + } + + @Override + public RecordsWithSplitIds<Tuple3<T, Long, Long>> fetch() throws IOException { + KafkaPartitionSplitRecords<Tuple3<T, Long, Long>> recordsBySplits = new KafkaPartitionSplitRecords<>(); + ConsumerRecords<byte[], byte[]> consumerRecords; + try { + consumerRecords = consumer.poll(Duration.ofMillis(POLL_TIMEOUT)); + } catch (WakeupException we) { + return recordsBySplits; + } + + List<TopicPartition> finishedPartitions = new ArrayList<>(); + for (TopicPartition tp : consumerRecords.partitions()) { + long stoppingOffset = getStoppingOffset(tp); + String splitId = tp.toString(); + Collection<Tuple3<T, Long, Long>> recordsForSplit = recordsBySplits.recordsForSplit(splitId); + for (ConsumerRecord<byte[], byte[]> consumerRecord : consumerRecords.records(tp)) { + // Stop consuming from this partition if the offsets has reached the stopping offset. + // Note that there are two cases, either case finishes a split: + // 1. After processing a record with offset of "stoppingOffset - 1". The split reader + // should not continue fetching because the record with stoppingOffset may not exist. + // 2. Before processing a record whose offset is greater than or equals to the stopping + // offset. This should only happens when case 1 was not met due to log compaction or + // log retention. + // Case 2 is handled here. Case 1 is handled after the record is processed. + if (consumerRecord.offset() >= stoppingOffset) { + finishSplitAtRecord(tp, stoppingOffset, consumerRecord.offset(), + finishedPartitions, recordsBySplits); + break; + } + // Add the record to the partition collector. + try { + deserializationSchema.deserialize(consumerRecord, collector); + collector.getRecords().forEach(r -> + recordsForSplit.add(new Tuple3<>(r, + consumerRecord.offset(), + consumerRecord.timestamp()))); + // Finish the split because there might not be any message after this point. Keep polling + // will just block forever. + if (consumerRecord.offset() == stoppingOffset - 1) { + finishSplitAtRecord(tp, stoppingOffset, consumerRecord.offset(), + finishedPartitions, recordsBySplits); + } + } catch (Exception e) { + throw new IOException("Failed to deserialize consumer record due to", e); + } finally { + collector.reset(); + } + } + } + // Unassign the partitions that has finished. + if (!finishedPartitions.isEmpty()) { + unassignPartitions(finishedPartitions); + } + recordsBySplits.prepareForRead(); + return recordsBySplits; + } + + @Override + public void handleSplitsChanges(SplitsChange<KafkaPartitionSplit> splitsChange) { + // Get all the partition assignments and stopping offsets. + if (!(splitsChange instanceof SplitsAddition)) { + throw new UnsupportedOperationException(String.format( + "The SplitChange type of %s is not supported.", splitsChange.getClass())); + } + + // Assignment. + List<TopicPartition> newPartitionAssignments = new ArrayList<>(); + // Starting offsets. + Map<TopicPartition, Long> partitionsStartingFromSpecifiedOffsets = new HashMap<>(); + List<TopicPartition> partitionsStartingFromEarliest = new ArrayList<>(); + List<TopicPartition> partitionsStartingFromLatest = new ArrayList<>(); + // Stopping offsets. + List<TopicPartition> partitionsStoppingAtLatest = new ArrayList<>(); + Set<TopicPartition> partitionsStoppingAtCommitted = new HashSet<>(); + + // Parse the starting and stopping offsets. + splitsChange.splits().forEach(s -> { + newPartitionAssignments.add(s.getTopicPartition()); + parseStartingOffsets(s, partitionsStartingFromEarliest, partitionsStartingFromLatest, partitionsStartingFromSpecifiedOffsets); + parseStoppingOffsets(s, partitionsStoppingAtLatest, partitionsStoppingAtCommitted); + }); + + // Assign new partitions. + newPartitionAssignments.addAll(consumer.assignment()); Review comment: The splitsChange only contains delta at this point. The `KafkaSubscriber` puts the current assignment in the removed partitions set, and that set is updated in `KafkaSubscriberUtils.updatePartitionChanges()`. ``` for (TopicPartitionInfo pi : partitionInfoList) { TopicPartition tp = new TopicPartition(topic, pi.partition()); // Initially the removedPartitions contains all of the currently assigned partitions. // If a partition exist in the Kafka cluster, but is not in the current assignment, then add it to the new partition. // If a partition exist in both Kafka cluster and the current assignment, simply remove it from the removed partitions, because the partition was not removed from the cluster, and it is not new partition either. // After the for loop, the partitions left in the removed partitions are // the partitions that exist in the current assignment, but no longer exist // in the Kafka cluster. So these partitions are essentially removed from // the Kafka cluster and should be unassigned. if (!removedPartitions.remove(tp)) { newPartitions.add(tp); } } ``` ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org