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-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to