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



##########
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:
       @C0urante, right now we have no visibility into the number or size of 
deques. We can't add a metric without a KIP, but WDYT about adding some DEBUG 
and/or TRACE log messages here? The benefit of here rather than in the 
WorderSourceTask is that it would be much easier to enable DEBUG or TRACE for 
only these log messages. One disadvantage is that this `committableOffsets()` 
method is calls once per iteration in the `WorkerSourceTask.execute()` method.
   
   I guess an alternative might be to add a method (e.g., `toString()`?) that 
output this information, and then put the log messages in 
`WorkerSinkTask.commitOffsets()`.
   
   Thoughts?




-- 
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


Reply via email to