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/10/01 16:24:40 UTC

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

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



##########
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:
       That's not actually the case; the use of [`Deque::removeLastOccurrence`](https://docs.oracle.com/javase/8/docs/api/java/util/Deque.html#removeLastOccurrence-java.lang.Object-) is different from [`Deque::removeLast`](https://docs.oracle.com/javase/8/docs/api/java/util/Deque.html#removeLast--) and [`Deque::pollLast`](https://docs.oracle.com/javase/8/docs/api/java/util/Deque.html#pollLast--) in that it can and will remove an element from anywhere in the deque (as opposed to just the element at the very end) if it `equals` the one passed to `removeLastOccurrence`. The reason `removeLastOccurrence` is used here instead of the more-common [`remove`](https://docs.oracle.com/javase/8/docs/api/java/util/Deque.html#remove-java.lang.Object-) is that, at least with the current use of `SubmittedRecords` by the `WorkerSourceTask`, any calls to `SubmittedRecords::remove` will pass in the most-recently-submitted record, which will then be at the deque it was inserted into, so there's a (poten
 tially-substantial) performance gain.
   
   I wonder if it might be helpful to alter this method:
   - By renaming to `removeLastOccurrence`
   - By clarifying in the Javadoc that, if multiple instances of the same `SubmittedRecord` have been submitted already, only the first one found (traversing from the end of the deque backward) will be removed




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