soarez commented on code in PR #15999: URL: https://github.com/apache/kafka/pull/15999#discussion_r1619486214
########## connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java: ########## @@ -73,6 +73,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.kafka.connect.mirror.MirrorConnectorConfig.OFFSET_SYNCS_TOPIC_CONFIG_PREFIX; +import static org.apache.kafka.connect.mirror.MirrorSourceConfig.OFFSET_SYNCS_CLIENT_ROLE_PREFIX; Review Comment: This constant is declared in MirrorConnectorConfig, so the static import to refer to that class directly ########## connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointConnectorTest.java: ########## @@ -50,6 +50,40 @@ public class MirrorCheckpointConnectorTest { private static final String CONSUMER_GROUP = "consumer-group-1"; private static final Map<String, ?> SOURCE_OFFSET = MirrorUtils.wrapOffset(0); + @Test + public void testEmitCheckpointsAndSyncGroupOffsetsBothDisabled() { + // disable the checkpoint emission + MirrorCheckpointConfig config = new MirrorCheckpointConfig( + makeProps("emit.checkpoints.enabled", "false", + "sync.group.offsets.enabled", "false")); + + Set<String> knownConsumerGroups = new HashSet<>(); + knownConsumerGroups.add(CONSUMER_GROUP); + // MirrorCheckpointConnector as minimum to run taskConfig() + MirrorCheckpointConnector connector = new MirrorCheckpointConnector(knownConsumerGroups, + config); + List<Map<String, String>> output = connector.taskConfigs(1); + // expect no task will be created + assertEquals(0, output.size(), "MirrorCheckpointConnector not disabled"); + } + + @Test + public void testEmitOffsetSyncsDisabled() { + // disable the checkpoint emission + MirrorCheckpointConfig config = new MirrorCheckpointConfig( + makeProps("emit.checkpoints.enabled", "false", + MirrorConnectorConfig.EMIT_OFFSET_SYNCS_ENABLED, "false")); + + Set<String> knownConsumerGroups = new HashSet<>(); + knownConsumerGroups.add(CONSUMER_GROUP); + // MirrorCheckpointConnector as minimum to run taskConfig() + MirrorCheckpointConnector connector = new MirrorCheckpointConnector(knownConsumerGroups, + config); + List<Map<String, String>> output = connector.taskConfigs(1); + // expect no task will be created + assertEquals(0, output.size(), "MirrorCheckpointConnector not disabled"); Review Comment: Should this be extracted into a common method? e.g. `assertMirrorCheckpointConnectorDisabled(config)` ########## connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorSourceTaskTest.java: ########## @@ -437,6 +437,55 @@ public void testSendSyncEvent() { verify(producer, times(5)).send(any(), any()); } + @Test + public void testDisableEmitOffsetSync() { + byte[] recordKey = "key".getBytes(); + byte[] recordValue = "value".getBytes(); + int maxOffsetLag = 50; + int recordPartition = 0; + int recordOffset = 0; + int metadataOffset = 100; + String topicName = "topic"; + String sourceClusterName = "sourceCluster"; + + RecordHeaders headers = new RecordHeaders(); + ReplicationPolicy replicationPolicy = new DefaultReplicationPolicy(); + + @SuppressWarnings("unchecked") + KafkaConsumer<byte[], byte[]> consumer = mock(KafkaConsumer.class); + @SuppressWarnings("unchecked") + KafkaProducer<byte[], byte[]> producer = mock(KafkaProducer.class); + MirrorSourceMetrics metrics = mock(MirrorSourceMetrics.class); + Semaphore outstandingOffsetSyncs = new Semaphore(1); + PartitionState partitionState = new PartitionState(maxOffsetLag); + Map<TopicPartition, PartitionState> partitionStates = new HashMap<>(); + + MirrorSourceTask mirrorSourceTask = new MirrorSourceTask(consumer, metrics, sourceClusterName, + replicationPolicy, maxOffsetLag, producer, outstandingOffsetSyncs, partitionStates, topicName, false); + + SourceRecord sourceRecord = mirrorSourceTask.convertRecord(new ConsumerRecord<>(topicName, recordPartition, + recordOffset, System.currentTimeMillis(), TimestampType.CREATE_TIME, recordKey.length, + recordValue.length, recordKey, recordValue, headers, Optional.empty())); + + TopicPartition sourceTopicPartition = MirrorUtils.unwrapPartition(sourceRecord.sourcePartition()); + partitionStates.put(sourceTopicPartition, partitionState); + RecordMetadata recordMetadata = new RecordMetadata(sourceTopicPartition, metadataOffset, 0, 0, 0, recordPartition); + + ArgumentCaptor<Callback> producerCallback = ArgumentCaptor.forClass(Callback.class); + when(producer.send(any(), producerCallback.capture())).thenAnswer(mockInvocation -> { + producerCallback.getValue().onCompletion(null, null); + return null; + }); + + mirrorSourceTask.commitRecord(sourceRecord, recordMetadata); + // No more syncs should take place; we've disabled emit offset-syncs + verifyNoInteractions(producer); Review Comment: We check for no interactions a couple of lines below, so this check seems redundant. ########## connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java: ########## @@ -257,6 +261,21 @@ public org.apache.kafka.common.config.Config validate(Map<String, String> props) ); } } + boolean offsetSyncsConfigured = configValues.stream() + .anyMatch(conf -> conf.name().startsWith(OFFSET_SYNCS_CLIENT_ROLE_PREFIX) || conf.name().startsWith(OFFSET_SYNCS_TOPIC_CONFIG_PREFIX)); + + if (!new MirrorSourceConfig(props).emitOffsetSyncEnabled() && offsetSyncsConfigured) { + ConfigValue emitOffsetSyncs = configValues.stream().filter(prop -> MirrorConnectorConfig.EMIT_OFFSET_SYNCS_ENABLED.equals(prop.name())) + .findAny() + .orElseGet(() -> { + ConfigValue result = new ConfigValue(MirrorConnectorConfig.EMIT_OFFSET_SYNCS_ENABLED); Review Comment: Since we're already importing some constants in `MirrorConnectorConfig`, maybe we should do the same for this one too? ########## connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorSourceTaskTest.java: ########## @@ -437,6 +437,55 @@ public void testSendSyncEvent() { verify(producer, times(5)).send(any(), any()); } + @Test + public void testDisableEmitOffsetSync() { + byte[] recordKey = "key".getBytes(); + byte[] recordValue = "value".getBytes(); + int maxOffsetLag = 50; + int recordPartition = 0; + int recordOffset = 0; + int metadataOffset = 100; + String topicName = "topic"; + String sourceClusterName = "sourceCluster"; + + RecordHeaders headers = new RecordHeaders(); + ReplicationPolicy replicationPolicy = new DefaultReplicationPolicy(); + + @SuppressWarnings("unchecked") + KafkaConsumer<byte[], byte[]> consumer = mock(KafkaConsumer.class); + @SuppressWarnings("unchecked") + KafkaProducer<byte[], byte[]> producer = mock(KafkaProducer.class); + MirrorSourceMetrics metrics = mock(MirrorSourceMetrics.class); + Semaphore outstandingOffsetSyncs = new Semaphore(1); + PartitionState partitionState = new PartitionState(maxOffsetLag); + Map<TopicPartition, PartitionState> partitionStates = new HashMap<>(); + + MirrorSourceTask mirrorSourceTask = new MirrorSourceTask(consumer, metrics, sourceClusterName, + replicationPolicy, maxOffsetLag, producer, outstandingOffsetSyncs, partitionStates, topicName, false); + + SourceRecord sourceRecord = mirrorSourceTask.convertRecord(new ConsumerRecord<>(topicName, recordPartition, + recordOffset, System.currentTimeMillis(), TimestampType.CREATE_TIME, recordKey.length, + recordValue.length, recordKey, recordValue, headers, Optional.empty())); + + TopicPartition sourceTopicPartition = MirrorUtils.unwrapPartition(sourceRecord.sourcePartition()); + partitionStates.put(sourceTopicPartition, partitionState); + RecordMetadata recordMetadata = new RecordMetadata(sourceTopicPartition, metadataOffset, 0, 0, 0, recordPartition); + + ArgumentCaptor<Callback> producerCallback = ArgumentCaptor.forClass(Callback.class); + when(producer.send(any(), producerCallback.capture())).thenAnswer(mockInvocation -> { + producerCallback.getValue().onCompletion(null, null); + return null; + }); + + mirrorSourceTask.commitRecord(sourceRecord, recordMetadata); Review Comment: Since this test suite is for `MirrorSourceTask`, and the offset sync is now handled by `OffsetSyncWriter`, perhaps that class should have its own test suite, and some of the logic being tested in `testSendSyncEvent` should move there? If the test constructor for `MirrorSourceTask` accepts `OffsetSyncWriter offsetSyncWriter` instead of `boolean emitOffsetSyncEnabled`, we could mock the writer in the tests in this suite, and verify interactions against the writer, not against the producer. ########## connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorSourceTaskTest.java: ########## @@ -437,6 +437,55 @@ public void testSendSyncEvent() { verify(producer, times(5)).send(any(), any()); } + @Test + public void testDisableEmitOffsetSync() { + byte[] recordKey = "key".getBytes(); + byte[] recordValue = "value".getBytes(); + int maxOffsetLag = 50; + int recordPartition = 0; + int recordOffset = 0; + int metadataOffset = 100; + String topicName = "topic"; + String sourceClusterName = "sourceCluster"; + + RecordHeaders headers = new RecordHeaders(); + ReplicationPolicy replicationPolicy = new DefaultReplicationPolicy(); + + @SuppressWarnings("unchecked") + KafkaConsumer<byte[], byte[]> consumer = mock(KafkaConsumer.class); + @SuppressWarnings("unchecked") + KafkaProducer<byte[], byte[]> producer = mock(KafkaProducer.class); + MirrorSourceMetrics metrics = mock(MirrorSourceMetrics.class); + Semaphore outstandingOffsetSyncs = new Semaphore(1); + PartitionState partitionState = new PartitionState(maxOffsetLag); + Map<TopicPartition, PartitionState> partitionStates = new HashMap<>(); + + MirrorSourceTask mirrorSourceTask = new MirrorSourceTask(consumer, metrics, sourceClusterName, + replicationPolicy, maxOffsetLag, producer, outstandingOffsetSyncs, partitionStates, topicName, false); + + SourceRecord sourceRecord = mirrorSourceTask.convertRecord(new ConsumerRecord<>(topicName, recordPartition, + recordOffset, System.currentTimeMillis(), TimestampType.CREATE_TIME, recordKey.length, + recordValue.length, recordKey, recordValue, headers, Optional.empty())); + + TopicPartition sourceTopicPartition = MirrorUtils.unwrapPartition(sourceRecord.sourcePartition()); + partitionStates.put(sourceTopicPartition, partitionState); + RecordMetadata recordMetadata = new RecordMetadata(sourceTopicPartition, metadataOffset, 0, 0, 0, recordPartition); + + ArgumentCaptor<Callback> producerCallback = ArgumentCaptor.forClass(Callback.class); + when(producer.send(any(), producerCallback.capture())).thenAnswer(mockInvocation -> { + producerCallback.getValue().onCompletion(null, null); + return null; + }); + + mirrorSourceTask.commitRecord(sourceRecord, recordMetadata); Review Comment: There's a lot in common here with `testSendSyncEvent` and the setup in the test is a bit involved. Do you think we could make this easier to read by extracting a common part of the test setup? -- 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