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


##########
connect/runtime/src/main/java/org/apache/kafka/connect/storage/ConnectorOffsetBackingStore.java:
##########
@@ -279,8 +284,59 @@ 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(s)-containing 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 EOS is disabled, we wait for the 
same duration as `offset.commit.timeout.ms`
+                // and throw that exception which would allow the offset 
commit to fail.
+                if (exactlyOnce) {
+                    secondaryWriteFuture.get();
+                } else {
+                    secondaryWriteFuture.get(offsetFlushTimeoutMs, 
TimeUnit.MILLISECONDS);
+                }
+            } catch (InterruptedException e) {
+                log.warn("{} Flush of tombstone(s)-containing to secondary 
store interrupted, cancelling", this);

Review Comment:
   I thought about this. The only reason I feel we should include it is that 
this goes against what the KIP says it will do in case of secondary write 
failure which could confuse users in case they aren't aware of this PR. I am ok 
to drop it or we could modify the KIP with this one behavioural change. 



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