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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to