C0urante commented on a change in pull request #11323: URL: https://github.com/apache/kafka/pull/11323#discussion_r726477234
########## File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SubmittedRecords.java ########## @@ -0,0 +1,156 @@ +/* + * 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()}, the latest-eligible offsets for each source partition can be retrieved, where every + * record up to and including the record for each returned offset has been either + * {@link SubmittedRecord#ack() acknowledged} or {@link #remove(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 #remove(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 #remove 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. + * @param record the {@link #submit previously-submitted} record to stop tracking; may not be null + */ + public void remove(SubmittedRecord record) { + Deque<SubmittedRecord> deque = records.get(record.partition()); + if (deque == null) { + log.warn("Attempted to remove record for partition {}, but no records with that partition are present", record.partition()); Review comment: Happy to make the change, although I personally doubt that any amount of rewording would make this message meaningful to anyone who isn't actively reviewing or already familiar with the source code. It's such an internal-facing detail; if I were a user, the addition of "from submitted queue" and "appear to have been submitted" wouldn't really change the fact that I would have no idea of how Connect does offset calculation for source tasks or why this might be a problem. -- 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