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