yashmayya commented on code in PR #13801: URL: https://github.com/apache/kafka/pull/13801#discussion_r1233861544
########## connect/runtime/src/main/java/org/apache/kafka/connect/storage/ConnectorOffsetBackingStore.java: ########## @@ -140,6 +142,9 @@ public static ConnectorOffsetBackingStore withOnlyConnectorStore( private final Optional<KafkaOffsetBackingStore> connectorStore; private final Optional<TopicAdmin> connectorStoreAdmin; + private boolean isEOSEnabled; Review Comment: ```suggestion private boolean exactlyOnce; ``` nit: similar to https://github.com/apache/kafka/blob/9b7f7e0fa09121e204540b46bd3d7340a0175e7d/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStore.java#L152 ########## 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"); + } + } + }); Review Comment: Why do we need to do this if we're anyway doing a get on the returned future immediately? Any producer error will be wrapped in an `ExecutionException` and thrown when we call get on the future returned from `KafkaOffsetBackingStore` (the same exception will also be used to complete the passed callback exceptionally) - https://github.com/apache/kafka/blob/546b912b831d46208499c6e1f4f785db6ddd9aa3/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStore.java#L344-L414 ########## 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` Review Comment: nit: I haven't seen the term ALOS used anywhere else in Connect. I'm presuming it means at least once delivery semantics? The Connect framework doesn't guarantee at least once semantics for all source connectors - depending on the specific connector plugin implementation, we could also have at most once delivery semantics. We could just phrase this as "If exactly-once support is disabled" instead. ########## 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(); + } else { + secondaryWriteFuture.get(offsetFlushTimeoutMs, TimeUnit.MILLISECONDS); + } + } catch (InterruptedException e) { + log.warn("{} Flush of tombstone offsets to secondary store interrupted, cancelling", this); + secondaryStoreTombstoneWriteError.compareAndSet(null, e); + } catch (ExecutionException e) { + log.error("{} Flush of tombstone offsets to secondary store threw an unexpected exception: ", this, e); + secondaryStoreTombstoneWriteError.compareAndSet(null, e); + } catch (TimeoutException e) { + log.error("{} Timed out waiting to flush offsets with tombstones to secondary storage ", this); + secondaryStoreTombstoneWriteError.compareAndSet(null, e); + } catch (Exception e) { + log.error("{} Got Exception when trying to flush tombstone offsets to secondary storage", this); Review Comment: Let's log the exception as well? ########## 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(); + } else { + secondaryWriteFuture.get(offsetFlushTimeoutMs, TimeUnit.MILLISECONDS); + } + } catch (InterruptedException e) { + log.warn("{} Flush of tombstone offsets to secondary store interrupted, cancelling", this); + secondaryStoreTombstoneWriteError.compareAndSet(null, e); + } catch (ExecutionException e) { + log.error("{} Flush of tombstone offsets to secondary store threw an unexpected exception: ", this, e); + secondaryStoreTombstoneWriteError.compareAndSet(null, e); + } catch (TimeoutException e) { + log.error("{} Timed out waiting to flush offsets with tombstones to secondary storage ", this); + secondaryStoreTombstoneWriteError.compareAndSet(null, e); + } catch (Exception e) { + log.error("{} Got Exception when trying to flush tombstone offsets to secondary storage", this); + secondaryStoreTombstoneWriteError.compareAndSet(null, e); + } + Throwable writeError = secondaryStoreTombstoneWriteError.get(); + if (writeError != null) { + FutureCallback<Void> failedWriteCallback = new FutureCallback<>(callback); + failedWriteCallback.onCompletion(writeError, null); + return failedWriteCallback; + } + } + return primaryStore.set(values, (primaryWriteError, ignored) -> { - if (secondaryStore != null) { + // Secondary store writes have already happened for tombstone records Review Comment: ```suggestion // Secondary store writes have already happened if the offsets batch contains tombstone records ``` nit: the existing comment could be interpreted as - `only tombstone records in the offsets batch have been written to the secondary store`, which is not the case. ########## 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"); Review Comment: This log line (and multiple subsequent others) seems to indicate that the whole offsets batch consists of tombstone records which need not necessarily be the case since we're only checking whether the batch contains at least one tombstone record. Maybe we could reword it to something like - `Successfully flushed tombstone(s)-containing offsets batch to secondary backing store` instead, WDYT? ########## 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(); + } else { + secondaryWriteFuture.get(offsetFlushTimeoutMs, TimeUnit.MILLISECONDS); + } + } catch (InterruptedException e) { + log.warn("{} Flush of tombstone offsets to secondary store interrupted, cancelling", this); + secondaryStoreTombstoneWriteError.compareAndSet(null, e); + } catch (ExecutionException e) { + log.error("{} Flush of tombstone offsets to secondary store threw an unexpected exception: ", this, e); + secondaryStoreTombstoneWriteError.compareAndSet(null, e); + } catch (TimeoutException e) { + log.error("{} Timed out waiting to flush offsets with tombstones to secondary storage ", this); + secondaryStoreTombstoneWriteError.compareAndSet(null, e); + } catch (Exception e) { + log.error("{} Got Exception when trying to flush tombstone offsets to secondary storage", this); + secondaryStoreTombstoneWriteError.compareAndSet(null, e); + } + Throwable writeError = secondaryStoreTombstoneWriteError.get(); + if (writeError != null) { + FutureCallback<Void> failedWriteCallback = new FutureCallback<>(callback); + failedWriteCallback.onCompletion(writeError, null); + return failedWriteCallback; + } + } + return primaryStore.set(values, (primaryWriteError, ignored) -> { - if (secondaryStore != null) { + // Secondary store writes have already happened for tombstone records + if (secondaryStore != null && !containsTombstones) { if (primaryWriteError != null) { - log.trace("Skipping offsets write to secondary store because primary write has failed", primaryWriteError); + log.trace("Skipping offsets write to secondary store for non tombstone offsets because primary write has failed", primaryWriteError); Review Comment: This log line would look pretty confusing to most users, maybe we can just leave it as is since the `non tombstone offsets` bit isn't too relevant here? ########## connect/runtime/src/test/java/org/apache/kafka/connect/storage/OffsetStorageWriterTest.java: ########## @@ -192,6 +197,220 @@ public void testCancelAfterAwaitFlush() throws Exception { flushFuture.get(1000, TimeUnit.MILLISECONDS); } + @Test Review Comment: I think it makes more sense to directly test the `ConnectorOffsetBackingStore::set` functionality in a `ConnectorOffsetBackingStore`-specific test class rather than indirectly testing through `OffsetStorageWriter::doFlush`? -- 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