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/20 17:10:54 UTC

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

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



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

Review comment:
       The choice of having a map of deques here has some interesting implications. 
   
   For cases with multiple partitions, a failure to deliver a produced record within a batch would not stop committing offsets associated with subsequent records (that have been produced and ack'ed successfully) as long as the partition of these records is different than the record that hasn't been ack'ed and eventually will fail to be produced. And that's a valid choice, however the disadvantage I see here is mainly that this new logic doesn't map always to the existing behavior. The existing code, that is based on two queues of outstanding messages, once a record is not ack'ed does not commit offsets of subsequent successful records. 
   
   To match the existing behavior with this new and improved non-blocking implementation seems to me that we'd only need to use a common global `Deque` instead of this map and stop removing elements once the first non-ack'ed and non-failed with a retriable exception submitted record was encountered (like you do for a single entry and a single deque now). 
   
   To summarize a comparison of the two options: 
   * with global deque we preserve the current semantics and behavior, we make fewer allocations of collections and maps, the code is a bit simpler. 
   * with map of deques we avoid few duplicates by committing offsets of successfully produced records within a batch
   
   With the above in mind, I wonder if using a single deque is preferable here, most importantly to preserve the existing behavior while getting the benefit of unblocking the offset commit in the presence of failures. Wdyt @C0urante @rhauch ?




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