C0urante commented on code in PR #11782: URL: https://github.com/apache/kafka/pull/11782#discussion_r912235126
########## connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExactlyOnceSourceIntegrationTest.java: ########## @@ -0,0 +1,1034 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.integration; + +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.acl.AccessControlEntry; +import org.apache.kafka.common.acl.AclBinding; +import org.apache.kafka.common.acl.AclOperation; +import org.apache.kafka.common.acl.AclPermissionType; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.errors.ProducerFencedException; +import org.apache.kafka.common.resource.PatternType; +import org.apache.kafka.common.resource.ResourcePattern; +import org.apache.kafka.common.resource.ResourceType; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.connect.connector.Task; +import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.json.JsonConverter; +import org.apache.kafka.connect.json.JsonConverterConfig; +import org.apache.kafka.connect.runtime.Worker; +import org.apache.kafka.connect.runtime.WorkerConfig; +import org.apache.kafka.connect.runtime.distributed.DistributedConfig; +import org.apache.kafka.connect.runtime.rest.entities.ConfigInfo; +import org.apache.kafka.connect.runtime.rest.entities.ConfigInfos; +import org.apache.kafka.connect.runtime.rest.errors.ConnectRestException; +import org.apache.kafka.connect.source.SourceConnector; +import org.apache.kafka.connect.source.SourceRecord; +import org.apache.kafka.connect.source.SourceTask; +import org.apache.kafka.connect.storage.StringConverter; +import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster; +import org.apache.kafka.connect.util.clusters.EmbeddedConnectClusterAssertions; +import org.apache.kafka.connect.util.clusters.EmbeddedKafkaCluster; +import org.apache.kafka.test.IntegrationTest; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.LongStream; + +import static org.apache.kafka.clients.producer.ProducerConfig.BOOTSTRAP_SERVERS_CONFIG; +import static org.apache.kafka.clients.producer.ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG; +import static org.apache.kafka.clients.producer.ProducerConfig.TRANSACTIONAL_ID_CONFIG; +import static org.apache.kafka.connect.integration.MonitorableSourceConnector.TOPIC_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLIENT_ADMIN_OVERRIDES_PREFIX; +import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX; +import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX; +import static org.apache.kafka.connect.runtime.ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.NAME_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.TASKS_MAX_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.SourceConnectorConfig.EXACTLY_ONCE_SUPPORT_CONFIG; +import static org.apache.kafka.connect.runtime.SourceConnectorConfig.OFFSETS_TOPIC_CONFIG; +import static org.apache.kafka.connect.runtime.SourceConnectorConfig.TRANSACTION_BOUNDARY_CONFIG; +import static org.apache.kafka.connect.runtime.SourceConnectorConfig.TRANSACTION_BOUNDARY_INTERVAL_CONFIG; +import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.EXACTLY_ONCE_SOURCE_SUPPORT_CONFIG; +import static org.apache.kafka.connect.source.SourceTask.TransactionBoundary.CONNECTOR; +import static org.apache.kafka.connect.source.SourceTask.TransactionBoundary.INTERVAL; +import static org.apache.kafka.connect.source.SourceTask.TransactionBoundary.POLL; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; + +@Category(IntegrationTest.class) +public class ExactlyOnceSourceIntegrationTest { + + private static final Logger log = LoggerFactory.getLogger(ExactlyOnceSourceIntegrationTest.class); + private static final String CLUSTER_GROUP_ID = "exactly-once-source-integration-test"; + private static final String CONNECTOR_NAME = "exactlyOnceQuestionMark"; + + private static final int SOURCE_TASK_PRODUCE_TIMEOUT_MS = 30_000; + private static final int DEFAULT_NUM_WORKERS = 3; + + private Properties brokerProps; + private Map<String, String> workerProps; + private EmbeddedConnectCluster.Builder connectBuilder; + private EmbeddedConnectCluster connect; + private ConnectorHandle connectorHandle; + + @Before + public void setup() { + workerProps = new HashMap<>(); + workerProps.put(EXACTLY_ONCE_SOURCE_SUPPORT_CONFIG, "enabled"); + workerProps.put(DistributedConfig.GROUP_ID_CONFIG, CLUSTER_GROUP_ID); + + brokerProps = new Properties(); + brokerProps.put("transaction.state.log.replication.factor", "1"); + brokerProps.put("transaction.state.log.min.isr", "1"); + + // build a Connect cluster backed by Kafka and Zk + connectBuilder = new EmbeddedConnectCluster.Builder() + .numWorkers(DEFAULT_NUM_WORKERS) + .numBrokers(1) + .workerProps(workerProps) + .brokerProps(brokerProps); + + // get a handle to the connector + connectorHandle = RuntimeHandles.get().connectorHandle(CONNECTOR_NAME); + } + + private void startConnect() { + connect = connectBuilder.build(); + connect.start(); + } + + @After + public void close() { + try { + // stop all Connect, Kafka and Zk threads. + connect.stop(); + } finally { + // Clear the handle for the connector. Fun fact: if you don't do this, your tests become quite flaky. + RuntimeHandles.get().deleteConnector(CONNECTOR_NAME); + } + } + + /** + * A simple test for the pre-flight validation API for connectors to provide their own delivery guarantees. + */ + @Test + public void testPreflightValidation() { + connectBuilder.numWorkers(1); + startConnect(); + + Map<String, String> props = new HashMap<>(); + props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getName()); + props.put(TASKS_MAX_CONFIG, "1"); + props.put(TOPIC_CONFIG, "topic"); + props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + props.put(VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + props.put(NAME_CONFIG, CONNECTOR_NAME); + + // Test out the "exactly.once.support" property + props.put(EXACTLY_ONCE_SUPPORT_CONFIG, "required"); + + // Connector will return null from SourceConnector::exactlyOnceSupport + props.put("exactly.once.support.level", "null"); + ConfigInfos validation = connect.validateConnectorConfig(MonitorableSourceConnector.class.getSimpleName(), props); + assertEquals("Preflight validation should have exactly one error", 1, validation.errorCount()); + ConfigInfo propertyValidation = findConfigInfo(EXACTLY_ONCE_SUPPORT_CONFIG, validation); + assertFalse("Preflight validation for exactly-once support property should have at least one error message", + propertyValidation.configValue().errors().isEmpty()); + + // Connector will return UNSUPPORTED from SourceConnector::exactlyOnceSupport + props.put("exactly.once.support.level", "unsupported"); + validation = connect.validateConnectorConfig(MonitorableSourceConnector.class.getSimpleName(), props); + assertEquals("Preflight validation should have exactly one error", 1, validation.errorCount()); + propertyValidation = findConfigInfo(EXACTLY_ONCE_SUPPORT_CONFIG, validation); + assertFalse("Preflight validation for exactly-once support property should have at least one error message", + propertyValidation.configValue().errors().isEmpty()); + + // Connector will throw an exception from SourceConnector::exactlyOnceSupport + props.put("exactly.once.support.level", "fail"); + validation = connect.validateConnectorConfig(MonitorableSourceConnector.class.getSimpleName(), props); + assertEquals("Preflight validation should have exactly one error", 1, validation.errorCount()); + propertyValidation = findConfigInfo(EXACTLY_ONCE_SUPPORT_CONFIG, validation); + assertFalse("Preflight validation for exactly-once support property should have at least one error message", + propertyValidation.configValue().errors().isEmpty()); + + // Connector will return SUPPORTED from SourceConnector::exactlyOnceSupport + props.put("exactly.once.support.level", "supported"); + validation = connect.validateConnectorConfig(MonitorableSourceConnector.class.getSimpleName(), props); + assertEquals("Preflight validation should have zero errors", 0, validation.errorCount()); + + // Test out the transaction boundary definition property + props.put(TRANSACTION_BOUNDARY_CONFIG, CONNECTOR.toString()); + + // Connector will return null from SourceConnector::canDefineTransactionBoundaries + props.put("custom.transaction.boundary.support", "null"); + validation = connect.validateConnectorConfig(MonitorableSourceConnector.class.getSimpleName(), props); + assertEquals("Preflight validation should have exactly one error", 1, validation.errorCount()); + propertyValidation = findConfigInfo(TRANSACTION_BOUNDARY_CONFIG, validation); + assertFalse("Preflight validation for transaction boundary property should have at least one error message", + propertyValidation.configValue().errors().isEmpty()); + + // Connector will return UNSUPPORTED from SourceConnector::canDefineTransactionBoundaries + props.put("custom.transaction.boundary.support", "unsupported"); + validation = connect.validateConnectorConfig(MonitorableSourceConnector.class.getSimpleName(), props); + assertEquals("Preflight validation should have exactly one error", 1, validation.errorCount()); + propertyValidation = findConfigInfo(TRANSACTION_BOUNDARY_CONFIG, validation); + assertFalse("Preflight validation for transaction boundary property should have at least one error message", + propertyValidation.configValue().errors().isEmpty()); + + // Connector will throw an exception from SourceConnector::canDefineTransactionBoundaries + props.put("custom.transaction.boundary.support", "null"); + validation = connect.validateConnectorConfig(MonitorableSourceConnector.class.getSimpleName(), props); + assertEquals("Preflight validation should have exactly one error", 1, validation.errorCount()); + propertyValidation = findConfigInfo(TRANSACTION_BOUNDARY_CONFIG, validation); + assertFalse("Preflight validation for transaction boundary property should have at least one error message", + propertyValidation.configValue().errors().isEmpty()); + + // Connector will return SUPPORTED from SourceConnector::canDefineTransactionBoundaries + props.put("custom.transaction.boundary.support", "supported"); + validation = connect.validateConnectorConfig(MonitorableSourceConnector.class.getSimpleName(), props); + assertEquals("Preflight validation should have zero errors", 0, validation.errorCount()); + } + + /** + * A simple green-path test that ensures the worker can start up a source task with exactly-once support enabled + * and write some records to Kafka that will be visible to a downstream consumer using the "READ_COMMITTED" + * isolation level. The "poll" transaction boundary is used. + */ + @Test + public void testPollBoundary() throws Exception { + // Much slower offset commit interval; should never be triggered during this test + workerProps.put(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG, "600000"); + connectBuilder.numWorkers(1); + startConnect(); + + String topic = "test-topic"; + connect.kafka().createTopic(topic, 3); + + int recordsProduced = 100; + + Map<String, String> props = new HashMap<>(); + props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getName()); + props.put(TASKS_MAX_CONFIG, "1"); + props.put(TOPIC_CONFIG, topic); + props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + props.put(VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + props.put(NAME_CONFIG, CONNECTOR_NAME); + props.put(TRANSACTION_BOUNDARY_CONFIG, POLL.toString()); + props.put("messages.per.poll", Integer.toString(recordsProduced)); + + // expect all records to be consumed and committed by the connector + connectorHandle.expectedRecords(recordsProduced); + connectorHandle.expectedCommits(recordsProduced); + + // start a source connector + connect.configureConnector(CONNECTOR_NAME, props); + + log.info("Waiting for records to be provided to worker by task"); + // wait for the connector tasks to produce enough records + connectorHandle.awaitRecords(SOURCE_TASK_PRODUCE_TIMEOUT_MS); + + log.info("Waiting for records to be committed to Kafka by worker"); + // wait for the connector tasks to commit enough records + connectorHandle.awaitCommits(TimeUnit.MINUTES.toMillis(1)); + + // consume all records from the source topic or fail, to ensure that they were correctly produced + int recordNum = connect.kafka() + .consume( + recordsProduced, + TimeUnit.MINUTES.toMillis(1), + Collections.singletonMap(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"), + "test-topic") + .count(); + assertTrue("Not enough records produced by source connector. Expected at least: " + recordsProduced + " + but got " + recordNum, Review Comment: It actually is a surprising amount of work to verify sequence numbers since records can be spread across partitions and, as a result, polled out of order by a consumer. We could reduce the number of topic partitions that we write to in these cases to 1, or place an arbitrary limitation on the number of records produced by each `MonitorableSourceTask` instance before it stops producing, or shut down the connector and do a poll for the end offsets of the topic and read from beginning up to that point to get absolutely everything that's in the topic. But the first option would not be worth the decrease in coverage IMO and the last two are pretty complex and come with their own edge cases. I do realize that system tests are pretty uncommon (I hardly run them myself), but do we really think there's risk of a regression that would be caught by these integration tests that wouldn't be caught by all the accompanying unit tests that we have? If there's a simpler way to do things let me know, too; I could be missing an easy win here. -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
