You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2021/09/28 17:10:13 UTC

[GitHub] [kafka] rhauch commented on a change in pull request #11323: KAFKA-12226: Commit source task offsets without blocking on batch delivery

rhauch commented on a change in pull request #11323:
URL: https://github.com/apache/kafka/pull/11323#discussion_r717740096



##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SubmittedRecords.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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 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.
+ * Note that this class is not thread-safe.
+ */
+class SubmittedRecords {
+    private 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.
+     * @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

Review comment:
       Nit on indentation:
   ```suggestion
        * @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
   ```

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SubmittedRecords.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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 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.
+ * Note that this class is not thread-safe.
+ */
+class SubmittedRecords {
+    private 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.
+     * @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.

Review comment:
       Should this mention that it's only possible to remove the `SubmittedRecord` most recently submitted via `submit(...)`?

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SubmittedRecords.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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 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.
+ * Note that this class is not thread-safe.

Review comment:
       WDYT about putting more design context here, such as this class tracking the source offsets for each of the source partitions _in the same order in which the records were returned by the source task's `poll()` method_. And that this class returns the committable offset for any source partition as the offset from the latest submitted record with that source partition that was acknowledged by the producer.
   
   I think it's important that we define the semantics clearly.
   
   Also:
   ```suggestion
    * Used to track source records that have been (or are about to be) dispatched to a producer and their accompanying
    * source offsets.
    * Note that this class is not thread-safe, though a {@link SubmittedRecord} can be
    * {@link SubmittedRecord#ack() acknowledged} from a different thread.
   ```

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SubmittedRecords.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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 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.
+ * Note that this class is not thread-safe.
+ */
+class SubmittedRecords {
+    private 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.
+     * @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) {
+        records.get(record.partition())
+                .removeLastOccurrence(record);
+    }
+
+    /**
+     * Clear out any acknowledged records 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;
+    }
+
+    // Note that this will return null if either there are no committable offsets for the given deque, or the latest
+    // committable offset is itself null. The caller is responsible for distinguishing between the two cases.
+    private Map<String, Object> committableOffset(Deque<SubmittedRecord> queuedRecords) {
+        Map<String, Object> result = null;
+        while (canCommitHead(queuedRecords)) {
+            result = queuedRecords.poll().offset();
+        }
+        return result;
+    }
+
+    private boolean canCommitHead(Deque<SubmittedRecord> queuedRecords) {
+        return queuedRecords.peek() != null && queuedRecords.peek().acked();
+    }
+
+    static class SubmittedRecord {
+        private final Map<String, Object> partition;
+        private final Map<String, Object> offset;
+        private volatile boolean acked;
+
+        public SubmittedRecord(Map<String, Object> partition, Map<String, Object> offset) {
+            this.partition = partition;
+            this.offset = offset;
+            this.acked = false;
+        }
+
+        /**
+         * Acknowledge this record; signals that its offset may be safely committed.

Review comment:
       Maybe add:
   ```suggestion
            * Acknowledge this record; signals that its offset may be safely committed.
            * This is safe to be called from a different thread than what called {@link SubmittedRecords#submit(SourceRecord)}.
   ```

##########
File path: connect/runtime/src/test/java/org/apache/kafka/connect/runtime/SubmittedRecordsTest.java
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.junit.Before;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.apache.kafka.connect.runtime.SubmittedRecords.SubmittedRecord;
+import static org.junit.Assert.assertEquals;
+
+public class SubmittedRecordsTest {
+
+    private static final Map<String, Object> PARTITION1 = Collections.singletonMap("subreddit", "apachekafka");
+    private static final Map<String, Object> PARTITION2 = Collections.singletonMap("subreddit", "pcj");
+    private static final Map<String, Object> PARTITION3 = Collections.singletonMap("subreddit", "asdfqweoicus");
+
+    private AtomicInteger offset;
+
+    SubmittedRecords submittedRecords;
+
+    @Before
+    public void setup() {
+        submittedRecords = new SubmittedRecords();
+        offset = new AtomicInteger();
+    }
+
+    @Test
+    public void testNoRecords() {
+        assertEquals(Collections.emptyMap(), submittedRecords.committableOffsets());
+        assertEquals(Collections.emptyMap(), submittedRecords.committableOffsets());
+        assertEquals(Collections.emptyMap(), submittedRecords.committableOffsets());
+    }
+
+    @Test
+    public void testNoCommittedRecords() {
+        for (int i = 0; i < 3; i++) {
+            for (Map<String, Object> partition : Arrays.asList(PARTITION1, PARTITION2, PARTITION3)) {
+                submittedRecords.submit(partition, newOffset());
+            }
+        }
+        assertEquals(Collections.emptyMap(), submittedRecords.committableOffsets());
+        assertEquals(Collections.emptyMap(), submittedRecords.committableOffsets());
+    }
+
+    @Test
+    public void testSingleAck() {
+        Map<String, Object> offset = newOffset();
+
+        SubmittedRecord submittedRecord = submittedRecords.submit(PARTITION1, offset);
+        // Record has been submitted but not yet acked; cannot commit offsets for it yet
+        assertEquals(Collections.emptyMap(), submittedRecords.committableOffsets());
+
+        submittedRecord.ack();
+        // Record has been acked; can commit offsets for it
+        assertEquals(Collections.singletonMap(PARTITION1, offset), submittedRecords.committableOffsets());
+
+        // Old offsets should be wiped
+        assertEquals(Collections.emptyMap(), submittedRecords.committableOffsets());
+    }
+
+    @Test
+    public void testMultipleAcksAcrossMultiplePartitions() {
+        Map<String, Object> partition1Offset1 = newOffset();
+        Map<String, Object> partition1Offset2 = newOffset();
+        Map<String, Object> partition2Offset1 = newOffset();
+        Map<String, Object> partition2Offset2 = newOffset();
+
+        SubmittedRecord partition1Record1 = submittedRecords.submit(PARTITION1, partition1Offset1);
+        SubmittedRecord partition1Record2 = submittedRecords.submit(PARTITION1, partition1Offset2);
+        SubmittedRecord partition2Record1 = submittedRecords.submit(PARTITION2, partition2Offset1);
+        SubmittedRecord partition2Record2 = submittedRecords.submit(PARTITION2, partition2Offset2);
+
+        // No records ack'd yet; can't commit any offsets
+        assertEquals(Collections.emptyMap(), submittedRecords.committableOffsets());
+
+        partition1Record2.ack();
+        // One record has been ack'd, but a record that comes before it and corresponds to the same source partition hasn't been
+        assertEquals(Collections.emptyMap(), submittedRecords.committableOffsets());
+
+        partition2Record1.ack();
+        // We can commit the first offset for the second partition
+        assertEquals(Collections.singletonMap(PARTITION2, partition2Offset1), submittedRecords.committableOffsets());
+
+        // No new offsets to commit
+        assertEquals(Collections.emptyMap(), submittedRecords.committableOffsets());
+
+        partition1Record1.ack();
+        partition2Record2.ack();
+        // We can commit new offsets for both partitions now
+        Map<Map<String, Object>, Map<String, Object>> expectedOffsets = new HashMap<>();
+        expectedOffsets.put(PARTITION1, partition1Offset2);
+        expectedOffsets.put(PARTITION2, partition2Offset2);
+        assertEquals(expectedOffsets, submittedRecords.committableOffsets());
+
+        // No new offsets to commit
+        assertEquals(Collections.emptyMap(), submittedRecords.committableOffsets());
+    }
+
+    @Test
+    public void testRemove() {
+        SubmittedRecord submittedRecord = submittedRecords.submit(PARTITION1, newOffset());
+        submittedRecords.remove(submittedRecord);
+
+        // Even if SubmittedRecords::remove is broken, we haven't ack'd anything yet, so there should be no committable offsets
+        assertEquals(Collections.emptyMap(), submittedRecords.committableOffsets());
+
+        submittedRecord.ack();
+        // Even though the record has somehow been acknowledged, it should not be counted when collecting committable offsets
+        assertEquals(Collections.emptyMap(), submittedRecords.committableOffsets());
+    }

Review comment:
       This is the only test method for `remove(...)`, but as mentioned above the `remove(...)` method does nothing when it is called with any previously-submitted record other than the most recently submitted record. It'd be good to add another test the case of trying to remove a previously-submitted record that wasn't the most recently submitted. And to verify that, it might be useful for `SubmittedRecords.remove(...)` to return a boolean as to whether it was successfully removed.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java
##########
@@ -570,22 +512,21 @@ public boolean commitOffsets() {
             // could look a little confusing.
         } catch (InterruptedException e) {
             log.warn("{} Flush of offsets interrupted, cancelling", this);
-            finishFailedFlush();
+            offsetWriter.cancelFlush();
             recordCommitFailure(time.milliseconds() - started, e);
             return false;
         } catch (ExecutionException e) {
             log.error("{} Flush of offsets threw an unexpected exception: ", this, e);
-            finishFailedFlush();
+            offsetWriter.cancelFlush();
             recordCommitFailure(time.milliseconds() - started, e);
             return false;
         } catch (TimeoutException e) {
             log.error("{} Timed out waiting to flush offsets to storage", this);
-            finishFailedFlush();
+            offsetWriter.cancelFlush();

Review comment:
       I wonder if it would be worth improving this log message slightly, to something like:
   > Timed out waiting to flush offsets to storage; will try again on next flush interval with new offsets
   
   Strictly speaking, it's unrelated to the changes made in this PR. But for users seeing this in the log it would be helpful to know that despite it being an error that should be looked into, the next flush interval will attempt to commit all (potentially-updated) offsets.

##########
File path: connect/runtime/src/test/java/org/apache/kafka/connect/runtime/SubmittedRecordsTest.java
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.junit.Before;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.apache.kafka.connect.runtime.SubmittedRecords.SubmittedRecord;
+import static org.junit.Assert.assertEquals;
+
+public class SubmittedRecordsTest {
+
+    private static final Map<String, Object> PARTITION1 = Collections.singletonMap("subreddit", "apachekafka");
+    private static final Map<String, Object> PARTITION2 = Collections.singletonMap("subreddit", "pcj");

Review comment:
       Maybe we could use a different value here.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java
##########
@@ -94,14 +95,9 @@
     private final TopicCreation topicCreation;
 
     private List<SourceRecord> toSend;
-    private boolean lastSendFailed; // Whether the last send failed *synchronously*, i.e. never made it into the producer's RecordAccumulator
-    // Use IdentityHashMap to ensure correctness with duplicate records. This is a HashMap because
-    // there is no IdentityHashSet.
-    private IdentityHashMap<ProducerRecord<byte[], byte[]>, ProducerRecord<byte[], byte[]>> outstandingMessages;
-    // A second buffer is used while an offset flush is running
-    private IdentityHashMap<ProducerRecord<byte[], byte[]>, ProducerRecord<byte[], byte[]>> outstandingMessagesBacklog;
-    private boolean flushing;
-    private CountDownLatch stopRequestedLatch;
+    private volatile Map<Map<String, Object>, Map<String, Object>> offsets;

Review comment:
       WDYT about renaming this as `committableOffsets` to be more clear where this is (and should be) used?

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SubmittedRecords.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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 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.
+ * Note that this class is not thread-safe.
+ */
+class SubmittedRecords {
+    private 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.
+     * @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) {
+        records.get(record.partition())
+                .removeLastOccurrence(record);
+    }
+
+    /**
+     * Clear out any acknowledged records 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);

Review comment:
       WDYT about adding a unit test that verifies that deques are indeed removed when they are empty? Might require adding some protected method or two in this class, but I think it'd be worth it.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java
##########
@@ -246,6 +240,7 @@ public void execute() {
                 }
 
                 maybeThrowProducerSendException();
+                updateCommittableOffsets();

Review comment:
       Per our original discussion, we agreed that it'd be better if the `WorkerSourceTask` thread iterates over the submitted records to find those that have been acked and accumulate the acked source partition and source offsets. The benefit is that the offset thread that periodically commits offsets for all source tasks only has to grab the committed offsets and then commit them.
   
   This is a nice place to do this work. 💯 

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SubmittedRecords.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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 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.
+ * Note that this class is not thread-safe.
+ */
+class SubmittedRecords {
+    private 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.
+     * @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) {
+        records.get(record.partition())
+                .removeLastOccurrence(record);
+    }
+
+    /**
+     * Clear out any acknowledged records and return the latest offset for each source partition that can be committed.

Review comment:
       The first sentence of the JavaDoc is a bit misleading, since this method doesn't remove _all_ acknowledged records:
   ```suggestion
        * Clear out any acknowledged records at the head of the deques and return the latest offset for each source partition that can be committed.
   ```

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SubmittedRecords.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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 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.
+ * Note that this class is not thread-safe.
+ */
+class SubmittedRecords {
+    private 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.
+     * @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) {
+        records.get(record.partition())
+                .removeLastOccurrence(record);
+    }
+
+    /**
+     * Clear out any acknowledged records 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

Review comment:
       This is essentially unbounded, meaning we could exhaust memory if the source task keeps producing records to a Kafka partition that remains offline for an extended period of time. That is already the case with the prior behavior, so this is no worse.
   
   Might be worth mentioning in the PR description.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SubmittedRecords.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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 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.
+ * Note that this class is not thread-safe.
+ */
+class SubmittedRecords {
+    private 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.

Review comment:
       Maybe add:
   ```suggestion
        * 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.
        * 
   ```

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java
##########
@@ -291,6 +286,13 @@ private void maybeThrowProducerSendException() {
         }
     }
 
+    private void updateCommittableOffsets() {
+        Map<Map<String, Object>, Map<String, Object>> newOffsets = submittedRecords.committableOffsets();
+        synchronized (this) {
+            offsets.putAll(newOffsets);
+        }

Review comment:
       Can we avoid this synchronized block if the `newOffsets` is empty?




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org