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


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

Review Comment:
   Can we also add a sentence or two explaining the reasoning here (wording can 
be borrowed from https://issues.apache.org/jira/browse/KAFKA-15018)? 



##########
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<>();

Review Comment:
   This doesn't need to be an `AtomicReference` anymore



##########
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
+                // 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 offsets to 
secondary store interrupted, cancelling", this);

Review Comment:
   Can we unify the language used in the log lines here and the subsequent 
catch blocks? `tombstone(s)-containing offsets`  / `offsets with tombstones` 
and `secondary store` / `secondary storage`. Also, since most users won't be 
aware of what "primary" and "secondary" stores are, I'd prefer using something 
like worker / global offsets topic instead. As per 
https://kafka.apache.org/coding-guide.html, logging is part of our "UI" 😄 



##########
connect/runtime/src/test/java/org/apache/kafka/connect/storage/OffsetStorageWriterTest.java:
##########
@@ -192,6 +212,251 @@ public void testCancelAfterAwaitFlush() throws Exception {
         flushFuture.get(1000, TimeUnit.MILLISECONDS);
     }
 
+    @Test
+    public void 
testFlushFailureWhenWriteToSecondaryStoreFailsForTombstoneOffsets() throws 
Exception {
+
+        KafkaOffsetBackingStore connectorStore = 
setupOffsetBackingStoreWithProducer("topic1", false);
+        KafkaOffsetBackingStore workerStore = 
setupOffsetBackingStoreWithProducer("topic2", true);
+
+        extractKeyValue(OFFSET_KEY, OFFSET_KEY_SERIALIZED, null, null);
+
+        ConnectorOffsetBackingStore offsetBackingStore = 
ConnectorOffsetBackingStore.withConnectorAndWorkerStores(
+                () -> LoggingContext.forConnector("source-connector"),
+                workerStore,
+                connectorStore,
+                "offsets-topic",
+                mock(TopicAdmin.class));
+
+        OffsetStorageWriter offsetStorageWriter = new 
OffsetStorageWriter(offsetBackingStore, NAMESPACE, keyConverter, 
valueConverter);
+
+        offsetStorageWriter.offset(OFFSET_KEY, OFFSET_VALUE);
+        assertTrue(offsetStorageWriter.beginFlush(1000L, 
TimeUnit.MILLISECONDS));
+        Future<Void> flushFuture = offsetStorageWriter.doFlush((error, result) 
-> {
+            assertEquals(PRODUCE_EXCEPTION, error);
+            assertNull(result);
+        });
+        assertThrows(ExecutionException.class, () -> flushFuture.get(1000L, 
TimeUnit.MILLISECONDS));
+    }
+
+    @Test
+    public void 
testFlushSuccessWhenWritesSucceedToBothPrimaryAndSecondaryStoresForTombstoneOffsets()
 throws Exception {
+
+        KafkaOffsetBackingStore connectorStore = 
setupOffsetBackingStoreWithProducer("topic1", false);
+        KafkaOffsetBackingStore workerStore = 
setupOffsetBackingStoreWithProducer("topic2", false);
+
+        extractKeyValue(OFFSET_KEY, OFFSET_KEY_SERIALIZED, null, null);
+
+        ConnectorOffsetBackingStore offsetBackingStore = 
ConnectorOffsetBackingStore.withConnectorAndWorkerStores(
+                () -> LoggingContext.forConnector("source-connector"),
+                workerStore,
+                connectorStore,
+                "offsets-topic",
+                mock(TopicAdmin.class));
+
+        OffsetStorageWriter offsetStorageWriter = new 
OffsetStorageWriter(offsetBackingStore, NAMESPACE, keyConverter, 
valueConverter);
+
+        offsetStorageWriter.offset(OFFSET_KEY, OFFSET_VALUE);
+        assertTrue(offsetStorageWriter.beginFlush(1000L, 
TimeUnit.MILLISECONDS));
+        offsetStorageWriter.doFlush((error, result) -> {
+            assertNull(error);
+            assertNull(result);
+        }).get(1000L, TimeUnit.MILLISECONDS);
+    }
+
+    @Test
+    public void 
testFlushSuccessWhenWriteToSecondaryStoreFailsForNonTombstoneOffsets() throws 
Exception {
+
+        KafkaOffsetBackingStore connectorStore = 
setupOffsetBackingStoreWithProducer("topic1", false);
+        KafkaOffsetBackingStore workerStore = 
setupOffsetBackingStoreWithProducer("topic2", true);
+
+        extractKeyValue(OFFSET_KEY, OFFSET_KEY_SERIALIZED, OFFSET_VALUE, 
OFFSET_VALUE_SERIALIZED);
+
+        ConnectorOffsetBackingStore offsetBackingStore = 
ConnectorOffsetBackingStore.withConnectorAndWorkerStores(
+                () -> LoggingContext.forConnector("source-connector"),
+                workerStore,
+                connectorStore,
+                "offsets-topic",
+                mock(TopicAdmin.class));
+
+        OffsetStorageWriter offsetStorageWriter = new 
OffsetStorageWriter(offsetBackingStore, NAMESPACE, keyConverter, 
valueConverter);
+
+        offsetStorageWriter.offset(OFFSET_KEY, OFFSET_VALUE);
+        assertTrue(offsetStorageWriter.beginFlush(1000L, 
TimeUnit.MILLISECONDS));
+        offsetStorageWriter.doFlush((error, result) -> {
+            assertNull(error);
+            assertNull(result);
+        }).get(1000L, TimeUnit.MILLISECONDS);
+    }
+
+    @Test
+    public void testFlushSuccessWhenWritesToPrimaryAndSecondaryStoreSucceeds() 
throws Exception {
+
+        KafkaOffsetBackingStore connectorStore = 
setupOffsetBackingStoreWithProducer("topic1", false);
+        KafkaOffsetBackingStore workerStore = 
setupOffsetBackingStoreWithProducer("topic2", false);
+
+        extractKeyValue(OFFSET_KEY, OFFSET_KEY_SERIALIZED, OFFSET_VALUE, 
OFFSET_VALUE_SERIALIZED);
+
+        ConnectorOffsetBackingStore offsetBackingStore = 
ConnectorOffsetBackingStore.withConnectorAndWorkerStores(
+                () -> LoggingContext.forConnector("source-connector"),
+                workerStore,
+                connectorStore,
+                "offsets-topic",
+                mock(TopicAdmin.class));
+
+        OffsetStorageWriter offsetStorageWriter = new 
OffsetStorageWriter(offsetBackingStore, NAMESPACE, keyConverter, 
valueConverter);
+
+        offsetStorageWriter.offset(OFFSET_KEY, OFFSET_VALUE);
+        assertTrue(offsetStorageWriter.beginFlush(1000L, 
TimeUnit.MILLISECONDS));
+        offsetStorageWriter.doFlush((error, result) -> {
+            assertNull(error);
+            assertNull(result);
+        }).get(1000L, TimeUnit.MILLISECONDS);
+    }
+
+    @Test
+    public void 
testFlushFailureWhenWritesToPrimaryStoreFailsAndSecondarySucceeds() throws 
Exception {
+
+        KafkaOffsetBackingStore connectorStore = 
setupOffsetBackingStoreWithProducer("topic1", true);
+        KafkaOffsetBackingStore workerStore = 
setupOffsetBackingStoreWithProducer("topic2", false);
+
+        extractKeyValue(OFFSET_KEY, OFFSET_KEY_SERIALIZED, OFFSET_VALUE, 
OFFSET_VALUE_SERIALIZED);
+
+        ConnectorOffsetBackingStore offsetBackingStore = 
ConnectorOffsetBackingStore.withConnectorAndWorkerStores(
+                () -> LoggingContext.forConnector("source-connector"),
+                workerStore,
+                connectorStore,
+                "offsets-topic",
+                mock(TopicAdmin.class));
+
+        OffsetStorageWriter offsetStorageWriter = new 
OffsetStorageWriter(offsetBackingStore, NAMESPACE, keyConverter, 
valueConverter);
+
+        offsetStorageWriter.offset(OFFSET_KEY, OFFSET_VALUE);
+        assertTrue(offsetStorageWriter.beginFlush(1000L, 
TimeUnit.MILLISECONDS));
+        Future<Void> flushFuture = offsetStorageWriter.doFlush((error, result) 
-> {
+            assertEquals(PRODUCE_EXCEPTION, error);
+            assertNull(result);
+        });
+        assertThrows(ExecutionException.class, () -> flushFuture.get(1000L, 
TimeUnit.MILLISECONDS));
+    }
+
+    @Test
+    public void 
testFlushFailureWhenWritesToPrimaryStoreFailsAndSecondarySucceedsForTombstoneRecords()
 throws Exception {
+
+        KafkaOffsetBackingStore connectorStore = 
setupOffsetBackingStoreWithProducer("topic1", true);
+        KafkaOffsetBackingStore workerStore = 
setupOffsetBackingStoreWithProducer("topic2", false);
+
+        extractKeyValue(OFFSET_KEY, OFFSET_KEY_SERIALIZED, null, null);
+
+        ConnectorOffsetBackingStore offsetBackingStore = 
ConnectorOffsetBackingStore.withConnectorAndWorkerStores(
+                () -> LoggingContext.forConnector("source-connector"),
+                workerStore,
+                connectorStore,
+                "offsets-topic",
+                mock(TopicAdmin.class));
+
+        OffsetStorageWriter offsetStorageWriter = new 
OffsetStorageWriter(offsetBackingStore, NAMESPACE, keyConverter, 
valueConverter);
+
+        offsetStorageWriter.offset(OFFSET_KEY, OFFSET_VALUE);
+        assertTrue(offsetStorageWriter.beginFlush(1000L, 
TimeUnit.MILLISECONDS));
+        Future<Void> flushFuture = offsetStorageWriter.doFlush((error, result) 
-> {
+            assertEquals(PRODUCE_EXCEPTION, error);
+            assertNull(result);
+        });
+        assertThrows(ExecutionException.class, () -> flushFuture.get(1000L, 
TimeUnit.MILLISECONDS));
+    }
+
+    @Test
+    public void 
testFlushSuccessWhenWritesToPrimaryStoreSucceedsWithNoSecondaryStore() throws 
Exception {
+
+        KafkaOffsetBackingStore connectorStore = 
setupOffsetBackingStoreWithProducer("topic1", false);
+
+        extractKeyValue(OFFSET_KEY, OFFSET_KEY_SERIALIZED, OFFSET_VALUE, 
OFFSET_VALUE_SERIALIZED);
+
+        ConnectorOffsetBackingStore offsetBackingStore = 
ConnectorOffsetBackingStore.withOnlyConnectorStore(
+                () -> LoggingContext.forConnector("source-connector"),
+                connectorStore,
+                "offsets-topic",
+                mock(TopicAdmin.class));
+
+        OffsetStorageWriter offsetStorageWriter = new 
OffsetStorageWriter(offsetBackingStore, NAMESPACE, keyConverter, 
valueConverter);
+
+        offsetStorageWriter.offset(OFFSET_KEY, OFFSET_VALUE);
+        assertTrue(offsetStorageWriter.beginFlush(1000L, 
TimeUnit.MILLISECONDS));
+        offsetStorageWriter.doFlush((error, result) -> {
+            assertNull(error);
+            assertNull(result);
+        }).get(1000L, TimeUnit.MILLISECONDS);
+    }
+
+    @Test
+    public void 
testFlushFailureWhenWritesToPrimaryStoreFailsWithNoSecondaryStore() throws 
Exception {
+
+        KafkaOffsetBackingStore connectorStore = 
setupOffsetBackingStoreWithProducer("topic1", true);
+
+        extractKeyValue(OFFSET_KEY, OFFSET_KEY_SERIALIZED, OFFSET_VALUE, 
OFFSET_VALUE_SERIALIZED);
+
+        ConnectorOffsetBackingStore offsetBackingStore = 
ConnectorOffsetBackingStore.withOnlyConnectorStore(
+                () -> LoggingContext.forConnector("source-connector"),
+                connectorStore,
+                "offsets-topic",
+                mock(TopicAdmin.class));
+
+        OffsetStorageWriter offsetStorageWriter = new 
OffsetStorageWriter(offsetBackingStore, NAMESPACE, keyConverter, 
valueConverter);
+
+        offsetStorageWriter.offset(OFFSET_KEY, OFFSET_VALUE);
+        assertTrue(offsetStorageWriter.beginFlush(1000L, 
TimeUnit.MILLISECONDS));
+        Future<Void> flushFuture = offsetStorageWriter.doFlush((error, result) 
-> {
+            assertEquals(PRODUCE_EXCEPTION, error);
+            assertNull(result);
+        });
+        assertThrows(ExecutionException.class, () -> flushFuture.get(1000L, 
TimeUnit.MILLISECONDS));
+    }
+
+    @SuppressWarnings("unchecked")
+    private KafkaOffsetBackingStore setupOffsetBackingStoreWithProducer(String 
topic, boolean throwingProducer) {
+        KafkaOffsetBackingStore offsetBackingStore = new 
KafkaOffsetBackingStore(() -> mock(TopicAdmin.class), () -> "connect",  
mock(Converter.class));
+        MockConsumer<byte[], byte[]> consumer = new 
MockConsumer<>(OffsetResetStrategy.EARLIEST);
+        Node noNode = Node.noNode();
+        Node[] nodes = new Node[]{noNode};
+        consumer.updatePartitions(topic, Collections.singletonList(new 
PartitionInfo(topic, 0, noNode, nodes, nodes)));
+        KafkaBasedLog<byte[], byte[]> kafkaBasedLog = new 
KafkaBasedLog<byte[], byte[]>(
+                topic, new HashMap<>(), new HashMap<>(),
+                () -> mock(TopicAdmin.class), mock(Callback.class), new 
MockTime(), null) {
+            @Override
+            protected Producer<byte[], byte[]> createProducer() {
+                return createMockProducer(throwingProducer);
+            }
+
+            @Override
+            protected Consumer<byte[], byte[]> createConsumer() {
+                return consumer;
+            }
+        };
+        kafkaBasedLog.start();
+        offsetBackingStore.offsetLog = kafkaBasedLog;
+        return offsetBackingStore;
+    }
+
+
+    private Producer<byte[], byte[]> createMockProducer(boolean 
throwingProducer) {
+        if (throwingProducer) {
+            return new MockProducer<byte[], byte[]>() {
+                @Override
+                public synchronized Future<RecordMetadata> send(final 
ProducerRecord<byte[], byte[]> record, final 
org.apache.kafka.clients.producer.Callback callback) {
+                    callback.onCompletion(null, PRODUCE_EXCEPTION);
+                    return null;
+                }
+            };
+        }
+        return new MockProducer<byte[], byte[]>() {
+            @Override
+            public synchronized Future<RecordMetadata> send(final 
ProducerRecord<byte[], byte[]> record, final 
org.apache.kafka.clients.producer.Callback callback) {
+                callback.onCompletion(null, null);
+                return null;
+            }
+        };

Review Comment:
   nit: can be simplified
   ```suggestion
           return new MockProducer<byte[], byte[]>() {
               @Override
               public synchronized Future<RecordMetadata> send(final 
ProducerRecord<byte[], byte[]> record, final 
org.apache.kafka.clients.producer.Callback callback) {
                   if (throwingProducer) {
                       callback.onCompletion(null, PRODUCE_EXCEPTION);
                   } else {
                       callback.onCompletion(null, null);
                   }
                   return null;
               }
           };
   ```



##########
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
+                // 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 offsets to 
secondary store interrupted, cancelling", this);

Review Comment:
   Can we unify the language used in the log lines here and the subsequent 
catch blocks? `tombstone(s)-containing offsets`  / `offsets with tombstones` 
and `secondary store` / `secondary storage`. Also, since most users won't be 
aware of what "primary" and "secondary" stores are, I'd prefer using something 
like worker / global offsets topic instead. As per 
https://kafka.apache.org/coding-guide.html, logging is part of our "UI" 😄 



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

Review Comment:
   nit: Let's make it explicit that the secondary store write is intentionally 
synchronous here?



##########
connect/runtime/src/test/java/org/apache/kafka/connect/storage/OffsetStorageWriterTest.java:
##########
@@ -233,4 +496,10 @@ private void expectStore(Map<String, Object> key, byte[] 
keySerialized,
         });
     }
 
+    private void extractKeyValue(Map<String, Object> key, byte[] 
keySerialized, Map<String, Object> value, byte[] valueSerialized) {

Review Comment:
   WDYT about `mockKeyValueConversion` instead? `extractKeyValue` doesn't seem 
to convey what the method is actually doing IMO.



##########
connect/runtime/src/test/java/org/apache/kafka/connect/storage/OffsetStorageWriterTest.java:
##########
@@ -192,6 +212,251 @@ public void testCancelAfterAwaitFlush() throws Exception {
         flushFuture.get(1000, TimeUnit.MILLISECONDS);
     }
 
+    @Test
+    public void 
testFlushFailureWhenWriteToSecondaryStoreFailsForTombstoneOffsets() throws 
Exception {
+
+        KafkaOffsetBackingStore connectorStore = 
setupOffsetBackingStoreWithProducer("topic1", false);
+        KafkaOffsetBackingStore workerStore = 
setupOffsetBackingStoreWithProducer("topic2", true);
+
+        extractKeyValue(OFFSET_KEY, OFFSET_KEY_SERIALIZED, null, null);
+
+        ConnectorOffsetBackingStore offsetBackingStore = 
ConnectorOffsetBackingStore.withConnectorAndWorkerStores(
+                () -> LoggingContext.forConnector("source-connector"),
+                workerStore,
+                connectorStore,
+                "offsets-topic",
+                mock(TopicAdmin.class));
+
+        OffsetStorageWriter offsetStorageWriter = new 
OffsetStorageWriter(offsetBackingStore, NAMESPACE, keyConverter, 
valueConverter);
+
+        offsetStorageWriter.offset(OFFSET_KEY, OFFSET_VALUE);
+        assertTrue(offsetStorageWriter.beginFlush(1000L, 
TimeUnit.MILLISECONDS));
+        Future<Void> flushFuture = offsetStorageWriter.doFlush((error, result) 
-> {
+            assertEquals(PRODUCE_EXCEPTION, error);
+            assertNull(result);
+        });
+        assertThrows(ExecutionException.class, () -> flushFuture.get(1000L, 
TimeUnit.MILLISECONDS));
+    }
+
+    @Test
+    public void 
testFlushSuccessWhenWritesSucceedToBothPrimaryAndSecondaryStoresForTombstoneOffsets()
 throws Exception {
+
+        KafkaOffsetBackingStore connectorStore = 
setupOffsetBackingStoreWithProducer("topic1", false);
+        KafkaOffsetBackingStore workerStore = 
setupOffsetBackingStoreWithProducer("topic2", false);
+
+        extractKeyValue(OFFSET_KEY, OFFSET_KEY_SERIALIZED, null, null);
+
+        ConnectorOffsetBackingStore offsetBackingStore = 
ConnectorOffsetBackingStore.withConnectorAndWorkerStores(
+                () -> LoggingContext.forConnector("source-connector"),
+                workerStore,
+                connectorStore,
+                "offsets-topic",
+                mock(TopicAdmin.class));
+
+        OffsetStorageWriter offsetStorageWriter = new 
OffsetStorageWriter(offsetBackingStore, NAMESPACE, keyConverter, 
valueConverter);
+
+        offsetStorageWriter.offset(OFFSET_KEY, OFFSET_VALUE);
+        assertTrue(offsetStorageWriter.beginFlush(1000L, 
TimeUnit.MILLISECONDS));
+        offsetStorageWriter.doFlush((error, result) -> {
+            assertNull(error);
+            assertNull(result);
+        }).get(1000L, TimeUnit.MILLISECONDS);
+    }
+
+    @Test
+    public void 
testFlushSuccessWhenWriteToSecondaryStoreFailsForNonTombstoneOffsets() throws 
Exception {
+
+        KafkaOffsetBackingStore connectorStore = 
setupOffsetBackingStoreWithProducer("topic1", false);
+        KafkaOffsetBackingStore workerStore = 
setupOffsetBackingStoreWithProducer("topic2", true);
+
+        extractKeyValue(OFFSET_KEY, OFFSET_KEY_SERIALIZED, OFFSET_VALUE, 
OFFSET_VALUE_SERIALIZED);
+
+        ConnectorOffsetBackingStore offsetBackingStore = 
ConnectorOffsetBackingStore.withConnectorAndWorkerStores(
+                () -> LoggingContext.forConnector("source-connector"),
+                workerStore,
+                connectorStore,
+                "offsets-topic",
+                mock(TopicAdmin.class));
+
+        OffsetStorageWriter offsetStorageWriter = new 
OffsetStorageWriter(offsetBackingStore, NAMESPACE, keyConverter, 
valueConverter);
+
+        offsetStorageWriter.offset(OFFSET_KEY, OFFSET_VALUE);
+        assertTrue(offsetStorageWriter.beginFlush(1000L, 
TimeUnit.MILLISECONDS));
+        offsetStorageWriter.doFlush((error, result) -> {
+            assertNull(error);
+            assertNull(result);
+        }).get(1000L, TimeUnit.MILLISECONDS);
+    }
+
+    @Test
+    public void testFlushSuccessWhenWritesToPrimaryAndSecondaryStoreSucceeds() 
throws Exception {
+
+        KafkaOffsetBackingStore connectorStore = 
setupOffsetBackingStoreWithProducer("topic1", false);
+        KafkaOffsetBackingStore workerStore = 
setupOffsetBackingStoreWithProducer("topic2", false);
+
+        extractKeyValue(OFFSET_KEY, OFFSET_KEY_SERIALIZED, OFFSET_VALUE, 
OFFSET_VALUE_SERIALIZED);
+
+        ConnectorOffsetBackingStore offsetBackingStore = 
ConnectorOffsetBackingStore.withConnectorAndWorkerStores(
+                () -> LoggingContext.forConnector("source-connector"),
+                workerStore,
+                connectorStore,
+                "offsets-topic",
+                mock(TopicAdmin.class));
+
+        OffsetStorageWriter offsetStorageWriter = new 
OffsetStorageWriter(offsetBackingStore, NAMESPACE, keyConverter, 
valueConverter);
+
+        offsetStorageWriter.offset(OFFSET_KEY, OFFSET_VALUE);
+        assertTrue(offsetStorageWriter.beginFlush(1000L, 
TimeUnit.MILLISECONDS));
+        offsetStorageWriter.doFlush((error, result) -> {
+            assertNull(error);
+            assertNull(result);
+        }).get(1000L, TimeUnit.MILLISECONDS);
+    }
+
+    @Test
+    public void 
testFlushFailureWhenWritesToPrimaryStoreFailsAndSecondarySucceeds() throws 
Exception {
+
+        KafkaOffsetBackingStore connectorStore = 
setupOffsetBackingStoreWithProducer("topic1", true);
+        KafkaOffsetBackingStore workerStore = 
setupOffsetBackingStoreWithProducer("topic2", false);
+
+        extractKeyValue(OFFSET_KEY, OFFSET_KEY_SERIALIZED, OFFSET_VALUE, 
OFFSET_VALUE_SERIALIZED);
+
+        ConnectorOffsetBackingStore offsetBackingStore = 
ConnectorOffsetBackingStore.withConnectorAndWorkerStores(
+                () -> LoggingContext.forConnector("source-connector"),
+                workerStore,
+                connectorStore,
+                "offsets-topic",
+                mock(TopicAdmin.class));
+
+        OffsetStorageWriter offsetStorageWriter = new 
OffsetStorageWriter(offsetBackingStore, NAMESPACE, keyConverter, 
valueConverter);
+
+        offsetStorageWriter.offset(OFFSET_KEY, OFFSET_VALUE);
+        assertTrue(offsetStorageWriter.beginFlush(1000L, 
TimeUnit.MILLISECONDS));
+        Future<Void> flushFuture = offsetStorageWriter.doFlush((error, result) 
-> {
+            assertEquals(PRODUCE_EXCEPTION, error);
+            assertNull(result);
+        });
+        assertThrows(ExecutionException.class, () -> flushFuture.get(1000L, 
TimeUnit.MILLISECONDS));
+    }
+
+    @Test
+    public void 
testFlushFailureWhenWritesToPrimaryStoreFailsAndSecondarySucceedsForTombstoneRecords()
 throws Exception {
+
+        KafkaOffsetBackingStore connectorStore = 
setupOffsetBackingStoreWithProducer("topic1", true);
+        KafkaOffsetBackingStore workerStore = 
setupOffsetBackingStoreWithProducer("topic2", false);
+
+        extractKeyValue(OFFSET_KEY, OFFSET_KEY_SERIALIZED, null, null);
+
+        ConnectorOffsetBackingStore offsetBackingStore = 
ConnectorOffsetBackingStore.withConnectorAndWorkerStores(
+                () -> LoggingContext.forConnector("source-connector"),
+                workerStore,
+                connectorStore,
+                "offsets-topic",
+                mock(TopicAdmin.class));
+
+        OffsetStorageWriter offsetStorageWriter = new 
OffsetStorageWriter(offsetBackingStore, NAMESPACE, keyConverter, 
valueConverter);
+
+        offsetStorageWriter.offset(OFFSET_KEY, OFFSET_VALUE);
+        assertTrue(offsetStorageWriter.beginFlush(1000L, 
TimeUnit.MILLISECONDS));
+        Future<Void> flushFuture = offsetStorageWriter.doFlush((error, result) 
-> {
+            assertEquals(PRODUCE_EXCEPTION, error);
+            assertNull(result);
+        });
+        assertThrows(ExecutionException.class, () -> flushFuture.get(1000L, 
TimeUnit.MILLISECONDS));
+    }
+
+    @Test
+    public void 
testFlushSuccessWhenWritesToPrimaryStoreSucceedsWithNoSecondaryStore() throws 
Exception {
+
+        KafkaOffsetBackingStore connectorStore = 
setupOffsetBackingStoreWithProducer("topic1", false);
+
+        extractKeyValue(OFFSET_KEY, OFFSET_KEY_SERIALIZED, OFFSET_VALUE, 
OFFSET_VALUE_SERIALIZED);
+
+        ConnectorOffsetBackingStore offsetBackingStore = 
ConnectorOffsetBackingStore.withOnlyConnectorStore(
+                () -> LoggingContext.forConnector("source-connector"),
+                connectorStore,
+                "offsets-topic",
+                mock(TopicAdmin.class));
+
+        OffsetStorageWriter offsetStorageWriter = new 
OffsetStorageWriter(offsetBackingStore, NAMESPACE, keyConverter, 
valueConverter);
+
+        offsetStorageWriter.offset(OFFSET_KEY, OFFSET_VALUE);
+        assertTrue(offsetStorageWriter.beginFlush(1000L, 
TimeUnit.MILLISECONDS));
+        offsetStorageWriter.doFlush((error, result) -> {
+            assertNull(error);
+            assertNull(result);
+        }).get(1000L, TimeUnit.MILLISECONDS);
+    }
+
+    @Test
+    public void 
testFlushFailureWhenWritesToPrimaryStoreFailsWithNoSecondaryStore() throws 
Exception {
+
+        KafkaOffsetBackingStore connectorStore = 
setupOffsetBackingStoreWithProducer("topic1", true);
+
+        extractKeyValue(OFFSET_KEY, OFFSET_KEY_SERIALIZED, OFFSET_VALUE, 
OFFSET_VALUE_SERIALIZED);
+
+        ConnectorOffsetBackingStore offsetBackingStore = 
ConnectorOffsetBackingStore.withOnlyConnectorStore(
+                () -> LoggingContext.forConnector("source-connector"),
+                connectorStore,
+                "offsets-topic",
+                mock(TopicAdmin.class));
+
+        OffsetStorageWriter offsetStorageWriter = new 
OffsetStorageWriter(offsetBackingStore, NAMESPACE, keyConverter, 
valueConverter);
+
+        offsetStorageWriter.offset(OFFSET_KEY, OFFSET_VALUE);
+        assertTrue(offsetStorageWriter.beginFlush(1000L, 
TimeUnit.MILLISECONDS));
+        Future<Void> flushFuture = offsetStorageWriter.doFlush((error, result) 
-> {
+            assertEquals(PRODUCE_EXCEPTION, error);
+            assertNull(result);
+        });
+        assertThrows(ExecutionException.class, () -> flushFuture.get(1000L, 
TimeUnit.MILLISECONDS));
+    }
+
+    @SuppressWarnings("unchecked")
+    private KafkaOffsetBackingStore setupOffsetBackingStoreWithProducer(String 
topic, boolean throwingProducer) {
+        KafkaOffsetBackingStore offsetBackingStore = new 
KafkaOffsetBackingStore(() -> mock(TopicAdmin.class), () -> "connect",  
mock(Converter.class));
+        MockConsumer<byte[], byte[]> consumer = new 
MockConsumer<>(OffsetResetStrategy.EARLIEST);
+        Node noNode = Node.noNode();
+        Node[] nodes = new Node[]{noNode};
+        consumer.updatePartitions(topic, Collections.singletonList(new 
PartitionInfo(topic, 0, noNode, nodes, nodes)));
+        KafkaBasedLog<byte[], byte[]> kafkaBasedLog = new 
KafkaBasedLog<byte[], byte[]>(
+                topic, new HashMap<>(), new HashMap<>(),
+                () -> mock(TopicAdmin.class), mock(Callback.class), new 
MockTime(), null) {
+            @Override
+            protected Producer<byte[], byte[]> createProducer() {
+                return createMockProducer(throwingProducer);
+            }
+
+            @Override
+            protected Consumer<byte[], byte[]> createConsumer() {
+                return consumer;
+            }
+        };
+        kafkaBasedLog.start();
+        offsetBackingStore.offsetLog = kafkaBasedLog;
+        return offsetBackingStore;
+    }
+
+

Review Comment:
   ```suggestion
   ```
   nit: stray blank line



##########
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:
   What's the rationale behind not having a timeout here for the worker global 
offset store write here? It seems safer to have the same time bound here as the 
non-EoS case and fail the task in case the write times out instead of the task 
getting stuck indefinitely while attempting to commit a transaction (where the 
offset flush occurs for exactly-once enabled source tasks).



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