curcur commented on a change in pull request #11725: URL: https://github.com/apache/flink/pull/11725#discussion_r426140306
########## File path: flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaShuffleFetcher.java ########## @@ -0,0 +1,379 @@ +/* + * 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.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.metrics.MetricGroup; +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.watermark.Watermark; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaCommitCallback; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionState; +import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.SerializedValue; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnull; + +import java.io.Serializable; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Properties; + +import static org.apache.flink.streaming.connectors.kafka.shuffle.FlinkKafkaShuffleProducer.KafkaSerializer.TAG_REC_WITHOUT_TIMESTAMP; +import static org.apache.flink.streaming.connectors.kafka.shuffle.FlinkKafkaShuffleProducer.KafkaSerializer.TAG_REC_WITH_TIMESTAMP; +import static org.apache.flink.streaming.connectors.kafka.shuffle.FlinkKafkaShuffleProducer.KafkaSerializer.TAG_WATERMARK; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * Fetch data from Kafka for Kafka Shuffle. + */ +@Internal +public class KafkaShuffleFetcher<T> extends AbstractFetcher<T, TopicPartition> { + private static final Logger LOG = LoggerFactory.getLogger(KafkaShuffleFetcher.class); + + /** The handler to check and generate watermarks from fetched records. **/ + private final WatermarkHandler watermarkHandler; + + /** The schema to convert between Kafka's byte messages, and Flink's objects. */ + private final KafkaShuffleElementDeserializer<T> deserializer; + + /** Serializer to serialize record. */ + private final TypeSerializer<T> serializer; + + /** The handover of data and exceptions between the consumer thread and the task thread. */ + private final Handover handover; + + /** The thread that runs the actual KafkaConsumer and hand the record batches to this fetcher. */ + private final KafkaConsumerThread consumerThread; + + /** Flag to mark the main work loop as alive. */ + private volatile boolean running = true; + + public KafkaShuffleFetcher( + SourceFunction.SourceContext<T> sourceContext, + Map<KafkaTopicPartition, Long> assignedPartitionsWithInitialOffsets, + SerializedValue<AssignerWithPeriodicWatermarks<T>> watermarksPeriodic, + SerializedValue<AssignerWithPunctuatedWatermarks<T>> watermarksPunctuated, + ProcessingTimeService processingTimeProvider, + long autoWatermarkInterval, + ClassLoader userCodeClassLoader, + String taskNameWithSubtasks, + TypeSerializer<T> serializer, + Properties kafkaProperties, + long pollTimeout, + MetricGroup subtaskMetricGroup, + MetricGroup consumerMetricGroup, + boolean useMetrics, + int producerParallelism) throws Exception { + super( + sourceContext, + assignedPartitionsWithInitialOffsets, + watermarksPeriodic, + watermarksPunctuated, + processingTimeProvider, + autoWatermarkInterval, + userCodeClassLoader, + consumerMetricGroup, + useMetrics); + this.deserializer = new KafkaShuffleElementDeserializer<>(); + this.serializer = serializer; + this.handover = new Handover(); + this.consumerThread = new KafkaConsumerThread( + LOG, + handover, + kafkaProperties, + unassignedPartitionsQueue, + getFetcherName() + " for " + taskNameWithSubtasks, + pollTimeout, + useMetrics, + consumerMetricGroup, + subtaskMetricGroup); + this.watermarkHandler = new WatermarkHandler(producerParallelism); + } + + // ------------------------------------------------------------------------ + // Fetcher work methods + // ------------------------------------------------------------------------ + + @Override + public void runFetchLoop() throws Exception { + try { + final Handover handover = this.handover; + + // kick off the actual Kafka consumer + consumerThread.start(); + + while (running) { + // this blocks until we get the next records + // it automatically re-throws exceptions encountered in the consumer thread + final ConsumerRecords<byte[], byte[]> records = handover.pollNext(); + + // get the records for each topic partition + for (KafkaTopicPartitionState<TopicPartition> partition : subscribedPartitionStates()) { + List<ConsumerRecord<byte[], byte[]>> partitionRecords = + records.records(partition.getKafkaPartitionHandle()); + + for (ConsumerRecord<byte[], byte[]> record : partitionRecords) { + final KafkaShuffleElement<T> element = deserializer.deserialize(serializer, record); + + // TODO: do we need to check the end of stream if reaching the end watermark? + + if (element.isRecord()) { + // timestamp is inherent from upstream + // If using ProcessTime, timestamp is going to be ignored (upstream does not include timestamp as well) + // If using IngestionTime, timestamp is going to be overwritten + // If using EventTime, timestamp is going to be used + synchronized (checkpointLock) { + KafkaShuffleRecord<T> elementAsRecord = element.asRecord(); + sourceContext.collectWithTimestamp( + elementAsRecord.value, + elementAsRecord.timestamp == null ? record.timestamp() : elementAsRecord.timestamp); + partition.setOffset(record.offset()); + } + } else if (element.isWatermark()) { + final KafkaShuffleWatermark watermark = element.asWatermark(); + Optional<Watermark> newWatermark = watermarkHandler.checkAndGetNewWatermark(watermark); + newWatermark.ifPresent(sourceContext::emitWatermark); + } + } + } + } + } + finally { + // this signals the consumer thread that no more work is to be done + consumerThread.shutdown(); + } + + // on a clean exit, wait for the runner thread + try { + consumerThread.join(); + } + catch (InterruptedException e) { + // may be the result of a wake-up interruption after an exception. + // we ignore this here and only restore the interruption state + Thread.currentThread().interrupt(); + } + } + + @Override + public void cancel() { + // flag the main thread to exit. A thread interrupt will come anyways. + running = false; + handover.close(); + consumerThread.shutdown(); + } + + @Override + protected TopicPartition createKafkaPartitionHandle(KafkaTopicPartition partition) { + return new TopicPartition(partition.getTopic(), partition.getPartition()); + } + + @Override + protected void doCommitInternalOffsetsToKafka( + Map<KafkaTopicPartition, Long> offsets, + @Nonnull KafkaCommitCallback commitCallback) throws Exception { + @SuppressWarnings("unchecked") + List<KafkaTopicPartitionState<TopicPartition>> partitions = subscribedPartitionStates(); + + Map<TopicPartition, OffsetAndMetadata> offsetsToCommit = new HashMap<>(partitions.size()); + + for (KafkaTopicPartitionState<TopicPartition> partition : partitions) { + Long lastProcessedOffset = offsets.get(partition.getKafkaTopicPartition()); + if (lastProcessedOffset != null) { + checkState(lastProcessedOffset >= 0, "Illegal offset value to commit"); + + // committed offsets through the KafkaConsumer need to be 1 more than the last processed offset. + // This does not affect Flink's checkpoints/saved state. + long offsetToCommit = lastProcessedOffset + 1; + + offsetsToCommit.put(partition.getKafkaPartitionHandle(), new OffsetAndMetadata(offsetToCommit)); + partition.setCommittedOffset(offsetToCommit); + } + } + + // record the work to be committed by the main consumer thread and make sure the consumer notices that + consumerThread.setOffsetsToCommit(offsetsToCommit, commitCallback); + } + + private String getFetcherName() { + return "Kafka Shuffle Fetcher"; + } + + /** + * An element in a KafkaShuffle. Can be a record or a Watermark. + */ + @VisibleForTesting + public abstract static class KafkaShuffleElement<T> { + + public boolean isRecord() { + return getClass() == KafkaShuffleRecord.class; + } + + public boolean isWatermark() { + return getClass() == KafkaShuffleWatermark.class; + } + + public KafkaShuffleRecord<T> asRecord() { + return (KafkaShuffleRecord<T>) this; + } + + public KafkaShuffleWatermark asWatermark() { + return (KafkaShuffleWatermark) this; + } + } + + /** + * A watermark element in a KafkaShuffle. It includes + * - subtask index where the watermark is coming from + * - watermark timestamp + */ + @VisibleForTesting + public static class KafkaShuffleWatermark<T> extends KafkaShuffleElement<T> { Review comment: Yes, good catch. ---------------------------------------------------------------- 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