yashmayya commented on code in PR #13801: URL: https://github.com/apache/kafka/pull/13801#discussion_r1259168414
########## connect/runtime/src/main/java/org/apache/kafka/connect/storage/ConnectorOffsetBackingStore.java: ########## @@ -279,8 +284,51 @@ 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<>(); + FutureCallback<Void> secondaryWriteFuture = new FutureCallback<>(); + secondaryStore.set(values, secondaryWriteFuture); + 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 Review Comment: > Well the rationale it the same that EOS waits indefinitely for the commits to happen Prior to this change, we just waited for the producer transaction commit to complete since we didn't care about the writes to the secondary store (and the transaction would contain all the data records as well as offset records written to the connector's primary store). However, that would be bounded by the producer's `transaction.timeout.ms` right? -- 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