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


##########
connect/runtime/src/main/java/org/apache/kafka/connect/storage/ConnectorOffsetBackingStore.java:
##########
@@ -279,10 +284,61 @@ 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.containsValue(null);
+
+        // 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) {
+            AtomicReference<Throwable> secondaryStoreTombstoneWriteError = new 
AtomicReference<>();
+            Future<Void> secondaryWriteFuture = 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");
+                    }
+                }
+            });
+            try {
+                // For EOS, there is no timeout for offset commit and it is 
allowed to take as much time as needed for
+                // commits. We still need to wait because we want to fail the 
offset commit for cases when
+                // tombstone records fail to be written to the secondary 
store. Note that while commitTransaction
+                // already waits for all records to be sent and ack'ed, in 
this case we do need to add an explicit
+                // blocking call. In case of ALOS, we wait for the same 
duration as `offset.commit.timeout.ms`
+                // and throw that exception which would allow the offset 
commit to fail.
+                if (isEOSEnabled) {
+                    secondaryWriteFuture.get();

Review Comment:
   While in the comments I have mentioned the reasoning of using an explicit 
`get()` call here, one thing to note is that I couldn't test the scenario where 
the secondary store write fails and the doFlush callback is able to see the 
error. This is because the `MockProducer#send` method throws any exception that 
you pass on to it w/o being able to create/execute the callback for it. This 
makes the `set` method itself to throw the error which is not what's going to 
happen from what I have understood.
   
   I could have ideally extended the send() in `MockProducer` to handle the 
case properly, but I didn't want to make any assumptions around it. If the 
reasoning stated above seems fine, I think it should be ok to do what I have 
done here.



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