You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "ashwinpankaj (via GitHub)" <gi...@apache.org> on 2023/06/08 09:32:08 UTC

[GitHub] [kafka] ashwinpankaj commented on a diff in pull request #13801: KAFKA-15018: Failing offset flush for EOS when secondary offset store writes fails for tombstone records

ashwinpankaj commented on code in PR #13801:
URL: https://github.com/apache/kafka/pull/13801#discussion_r1222706232


##########
connect/runtime/src/main/java/org/apache/kafka/connect/storage/ConnectorOffsetBackingStore.java:
##########
@@ -279,10 +280,33 @@ public Future<Void> set(Map<ByteBuffer, ByteBuffer> values, Callback<Void> callb
             throw new IllegalStateException("At least one non-null offset store must be provided");
         }
 
+        boolean containsTombstones = values.entrySet()
+                .stream()
+                .anyMatch(offset -> offset.getValue() == null);
+
+        AtomicReference<Throwable> secondaryStoreTombstoneWriteError = new AtomicReference<>();
+
+        // If there are tombstone offsets, then the failure to write to secondary store will
+        // not be ignored. Also, for tombstone records, we first write to secondary store and
+        // then to primary stores.
+        if (secondaryStore != null && containsTombstones) {
+            secondaryStore.set(values, (secondaryWriteError, ignored) -> {
+                try (LoggingContext context = loggingContext()) {
+                    if (secondaryWriteError != null) {
+                        log.warn("Failed to write offsets with tombstone records to secondary backing store", secondaryWriteError);
+                        secondaryStoreTombstoneWriteError.compareAndSet(null, secondaryWriteError);
+                    } else {
+                        log.debug("Successfully flushed tombstone offsets to secondary backing store");
+                    }
+                }
+            });
+        }
+
         return primaryStore.set(values, (primaryWriteError, ignored) -> {
-            if (secondaryStore != null) {
+            // Secondary store writes have already happened for tombstone records

Review Comment:
   +1 we can ensure that the secondary write has already been attempted via a `CompletableFuture<Throwable>` set in the callback of `secondaryStore.set()` 



##########
connect/runtime/src/main/java/org/apache/kafka/connect/storage/ConnectorOffsetBackingStore.java:
##########
@@ -279,10 +280,33 @@ public Future<Void> set(Map<ByteBuffer, ByteBuffer> values, Callback<Void> callb
             throw new IllegalStateException("At least one non-null offset store must be provided");
         }
 
+        boolean containsTombstones = values.entrySet()
+                .stream()
+                .anyMatch(offset -> offset.getValue() == null);

Review Comment:
   We should do this only if `connectorStore.isPresent()` else we will always end up scanning the map for tombstones.



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