C0urante commented on a change in pull request #11323: URL: https://github.com/apache/kafka/pull/11323#discussion_r740365960
########## File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SubmittedRecords.java ########## @@ -0,0 +1,162 @@ +/* + * 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.kafka.connect.runtime; + +import org.apache.kafka.connect.source.SourceRecord; +import org.apache.kafka.connect.source.SourceTask; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Deque; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.Map; + +/** + * Used to track source records that have been (or are about to be) dispatched to a producer and their accompanying + * source offsets. Records are tracked in the order in which they are submitted, which should match the order they were + * returned from {@link SourceTask#poll()}. The latest-eligible offsets for each source partition can be retrieved via + * {@link #committableOffsets()}, where every record up to and including the record for each returned offset has been + * either {@link SubmittedRecord#ack() acknowledged} or {@link #removeLastOccurrence(SubmittedRecord) removed}. + * Note that this class is not thread-safe, though a {@link SubmittedRecord} can be + * {@link SubmittedRecord#ack() acknowledged} from a different thread. + */ +class SubmittedRecords { + + private static final Logger log = LoggerFactory.getLogger(SubmittedRecords.class); + + // Visible for testing + final Map<Map<String, Object>, Deque<SubmittedRecord>> records; + + public SubmittedRecords() { + this.records = new HashMap<>(); + } + + /** + * Enqueue a new source record before dispatching it to a producer. + * The returned {@link SubmittedRecord} should either be {@link SubmittedRecord#ack() acknowledged} in the + * producer callback, or {@link #removeLastOccurrence(SubmittedRecord) removed} if the record could not be successfully + * sent to the producer. + * + * @param record the record about to be dispatched; may not be null but may have a null + * {@link SourceRecord#sourcePartition()} and/or {@link SourceRecord#sourceOffset()} + * @return a {@link SubmittedRecord} that can be either {@link SubmittedRecord#ack() acknowledged} once ack'd by + * the producer, or {@link #removeLastOccurrence removed} if synchronously rejected by the producer + */ + @SuppressWarnings("unchecked") + public SubmittedRecord submit(SourceRecord record) { + return submit((Map<String, Object>) record.sourcePartition(), (Map<String, Object>) record.sourceOffset()); + } + + // Convenience method for testing + SubmittedRecord submit(Map<String, Object> partition, Map<String, Object> offset) { + SubmittedRecord result = new SubmittedRecord(partition, offset); + records.computeIfAbsent(result.partition(), p -> new LinkedList<>()) + .add(result); + return result; + } + + /** + * Remove a source record and do not take it into account any longer when tracking offsets. + * Useful if the record has been synchronously rejected by the producer. + * If multiple instances of the same {@link SubmittedRecord} have been submitted already, only the first one found + * (traversing from the end of the deque backward) will be removed. + * @param record the {@link #submit previously-submitted} record to stop tracking; may not be null + * @return whether an instance of the record was removed + */ + public boolean removeLastOccurrence(SubmittedRecord record) { + Deque<SubmittedRecord> deque = records.get(record.partition()); + if (deque == null) { + log.warn("Attempted to remove record from submitted queue for partition {}, but no records with that partition appear to have been submitted", record.partition()); + return false; + } + boolean result = deque.removeLastOccurrence(record); + if (deque.isEmpty()) { + records.remove(record.partition()); + } + if (!result) { + log.warn("Attempted to remove record from submitted queue for partition {}, but the record has not been submitted or has already been removed", record.partition()); + } + return result; + } + + /** + * Clear out any acknowledged records at the head of the deques and return the latest offset for each source partition that can be committed. + * Note that this may take some time to complete if a large number of records has built up, which may occur if a + * Kafka partition is offline and all records targeting that partition go unacknowledged while records targeting + * other partitions continue to be dispatched to the producer and sent successfully + * @return the latest-possible offsets to commit for each source partition; may be empty but never null + */ + public Map<Map<String, Object>, Map<String, Object>> committableOffsets() { + Map<Map<String, Object>, Map<String, Object>> result = new HashMap<>(); + records.forEach((partition, queuedRecords) -> { + if (canCommitHead(queuedRecords)) { + Map<String, Object> offset = committableOffset(queuedRecords); + result.put(partition, offset); + } + }); + // Clear out all empty deques from the map to keep it from growing indefinitely + records.values().removeIf(Deque::isEmpty); + return result; + } Review comment: I agree with your concerns about excess logging if a message is added to the `WorkerSourceTask::execute` loop. Since we're removing [this log message](https://github.com/apache/kafka/blob/af8100b94fda4a27511797233e9845078ae8a69f/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java#L510) in this PR, I wonder if we can replace it with something similar? I think users may want to know how many total pending (i.e., unacked) messages there are, how many deques there are, and the number of messages in the largest deque (which may be useful for identifying "stuck" topic partitions). I'll take a shot at this; LMKWYT. ########## File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java ########## @@ -474,9 +479,24 @@ public boolean commitOffsets() { long timeout = started + commitTimeoutMs; Map<Map<String, Object>, Map<String, Object>> offsetsToCommit; + SubmittedRecords.Pending pendingMetadataForCommit; synchronized (this) { offsetsToCommit = this.committableOffsets; this.committableOffsets = new HashMap<>(); + pendingMetadataForCommit = this.pendingRecordsMetadata; + this.pendingRecordsMetadata = null; + } + + if (pendingMetadataForCommit != null) { + log.info("There are currently {} pending messages spread across {} source partitions whose offsets will not be committed. " + + "The source partition with the most pending messages is {}, with {} pending messages", + pendingMetadataForCommit.totalPendingMessages(), + pendingMetadataForCommit.numDeques(), + pendingMetadataForCommit.largestDequePartition(), + pendingMetadataForCommit.largestDequeSize() + ); + } else { + log.info("There are currently no pending messages for this offset commit; all messages since the last commit have been acknowledged"); Review comment: 👍 SGTM. I've updated the PR accordingly. One nit: the "flushing <n> outstanding messages for offset commit" message actually refers to the number of unacked messages in the current batch; this has tripped up many of my colleagues who see "flushing 0 outstanding messages" and think their source connector isn't producing any data when all it really means is that its producers are keeping up with the throughput of its tasks very well. I think both pieces of information (number of acked and unacked messages) are useful here so I've included both in the latest draft. ########## File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java ########## @@ -474,9 +479,24 @@ public boolean commitOffsets() { long timeout = started + commitTimeoutMs; Map<Map<String, Object>, Map<String, Object>> offsetsToCommit; + SubmittedRecords.Pending pendingMetadataForCommit; synchronized (this) { offsetsToCommit = this.committableOffsets; this.committableOffsets = new HashMap<>(); + pendingMetadataForCommit = this.pendingRecordsMetadata; + this.pendingRecordsMetadata = null; + } + + if (pendingMetadataForCommit != null) { + log.info("There are currently {} pending messages spread across {} source partitions whose offsets will not be committed. " + + "The source partition with the most pending messages is {}, with {} pending messages", + pendingMetadataForCommit.totalPendingMessages(), + pendingMetadataForCommit.numDeques(), + pendingMetadataForCommit.largestDequePartition(), + pendingMetadataForCommit.largestDequeSize() + ); + } else { + log.info("There are currently no pending messages for this offset commit; all messages since the last commit have been acknowledged"); Review comment: 👍 SGTM. I've updated the PR accordingly. One nit: the "flushing <n> outstanding messages for offset commit" message actually refers to the number of unacked messages in the current batch and not the number of acknowledged messages for which offsets will be committed; this has tripped up many of my colleagues who see "flushing 0 outstanding messages" and think their source connector isn't producing any data when all it really means is that its producers are keeping up with the throughput of its tasks very well. I think both pieces of information (number of acked and unacked messages) are useful here so I've included both in the latest draft. -- 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. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org