tombentley commented on code in PR #11782:
URL: https://github.com/apache/kafka/pull/11782#discussion_r908189222
##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java:
##########
@@ -174,8 +174,6 @@ protected void producerSendFailed(
);
commitTaskRecord(preTransformRecord, null);
} else {
- log.error("{} failed to send record to {}: ",
WorkerSourceTask.this, topic, e);
- log.trace("{} Failed record: {}", WorkerSourceTask.this,
preTransformRecord);
producerSendException.compareAndSet(null, e);
Review Comment:
Doesn't this mean we're potentially losing info about the topic which might
be relevant to understanding the exception?
##########
connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExactlyOnceSourceIntegrationTest.java:
##########
@@ -0,0 +1,997 @@
+/*
+ * 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,
+ recordNum >= recordsProduced);
+ }
+
+ /**
+ * 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 "interval" transaction boundary is used with a
connector-specific override.
+ */
+ @Test
+ public void testIntervalBoundary() 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, INTERVAL.toString());
+ props.put(TRANSACTION_BOUNDARY_INTERVAL_CONFIG, "10000");
+ 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,
+ recordNum >= recordsProduced);
+ }
+
+ /**
+ * 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 "connector" transaction boundary is used with a
connector that defines transactions whose
+ * size correspond to successive elements of the Fibonacci sequence, where
transactions with an even number of
+ * records are aborted, and those with an odd number of records are
committed.
Review Comment:
Nice
##########
connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExactlyOnceSourceIntegrationTest.java:
##########
@@ -0,0 +1,997 @@
+/*
+ * 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,
+ recordNum >= recordsProduced);
+ }
+
+ /**
+ * 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 "interval" transaction boundary is used with a
connector-specific override.
+ */
+ @Test
+ public void testIntervalBoundary() 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, INTERVAL.toString());
+ props.put(TRANSACTION_BOUNDARY_INTERVAL_CONFIG, "10000");
+ 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,
+ recordNum >= recordsProduced);
+ }
+
+ /**
+ * 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 "connector" transaction boundary is used with a
connector that defines transactions whose
+ * size correspond to successive elements of the Fibonacci sequence, where
transactions with an even number of
+ * records are aborted, and those with an odd number of records are
committed.
+ */
+ @Test
+ public void testConnectorBoundary() throws Exception {
+ String offsetsTopic = "exactly-once-source-cluster-offsets";
+ workerProps.put(DistributedConfig.OFFSET_STORAGE_TOPIC_CONFIG,
offsetsTopic);
+ 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, CONNECTOR.toString());
+ props.put("custom.transaction.boundary.support", "supported");
+ 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));
+
+ Map<String, Object> consumerProps = new HashMap<>();
+ consumerProps.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG,
"read_committed");
+ consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
+ // consume all records from the source topic or fail, to ensure that
they were correctly produced
+ ConsumerRecords<byte[], byte[]> sourceRecords = connect.kafka()
+ .consume(
+ recordsProduced,
+ TimeUnit.MINUTES.toMillis(1),
+ consumerProps,
+ "test-topic");
+ assertTrue("Not enough records produced by source connector. Expected
at least: " + recordsProduced + " + but got " + sourceRecords.count(),
+ sourceRecords.count() >= recordsProduced);
+
+ // also consume from the cluster's offsets topic to verify that the
expected offsets (which should correspond to the connector's
+ // custom transaction boundaries) were committed
+ List<Long> expectedOffsetSeqnos = new ArrayList<>();
+ long lastExpectedOffsetSeqno = 1;
+ long nextExpectedOffsetSeqno = 1;
+ while (nextExpectedOffsetSeqno <= recordsProduced) {
+ expectedOffsetSeqnos.add(nextExpectedOffsetSeqno);
+ nextExpectedOffsetSeqno += lastExpectedOffsetSeqno;
+ lastExpectedOffsetSeqno = nextExpectedOffsetSeqno -
lastExpectedOffsetSeqno;
+ }
+ ConsumerRecords<byte[], byte[]> offsetRecords = connect.kafka()
+ .consume(
+ expectedOffsetSeqnos.size(),
+ TimeUnit.MINUTES.toMillis(1),
+ consumerProps,
+ offsetsTopic
+ );
+
+ List<Long> actualOffsetSeqnos = new ArrayList<>();
+ offsetRecords.forEach(record ->
actualOffsetSeqnos.add(parseAndAssertOffsetForSingleTask(record)));
+
+ assertEquals("Committed offsets should match connector-defined
transaction boundaries",
+ expectedOffsetSeqnos, actualOffsetSeqnos.subList(0,
expectedOffsetSeqnos.size()));
+
+ List<Long> expectedRecordSeqnos = LongStream.range(1, recordsProduced
+ 1).boxed().collect(Collectors.toList());
+ long priorBoundary = 1;
+ long nextBoundary = 2;
+ while (priorBoundary <
expectedRecordSeqnos.get(expectedRecordSeqnos.size() - 1)) {
+ if (nextBoundary % 2 == 0) {
+ for (long i = priorBoundary + 1; i < nextBoundary + 1; i++) {
+ expectedRecordSeqnos.remove(i);
+ }
+ }
+ nextBoundary += priorBoundary;
+ priorBoundary = nextBoundary - priorBoundary;
+ }
+ List<Long> actualRecordSeqnos = new ArrayList<>();
+ sourceRecords.forEach(record ->
actualRecordSeqnos.add(parseAndAssertValueForSingleTask(record)));
+ Collections.sort(actualRecordSeqnos);
+ assertEquals("Committed records should exclude connector-aborted
transactions",
+ expectedRecordSeqnos, actualRecordSeqnos.subList(0,
expectedRecordSeqnos.size()));
+ }
+
+ /**
+ * Brings up a one-node cluster, then intentionally fences out the
transactional producer used by the leader
+ * for writes to the config topic to simulate a zombie leader being active
in the cluster. The leader should
+ * automatically recover, verify that it is still the leader, and then
succeed to create a connector when the
+ * user resends the request.
Review Comment:
```suggestion
* user resends the request.
```
##########
connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExactlyOnceSourceIntegrationTest.java:
##########
@@ -0,0 +1,997 @@
+/*
+ * 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.network.ListenerName;
+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.security.auth.SecurityProtocol;
+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,
+ recordNum >= recordsProduced);
+ }
+
+ /**
+ * 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 "interval" transaction boundary is used with a
connector-specific override.
+ */
+ @Test
+ public void testIntervalBoundary() 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, INTERVAL.toString());
+ props.put(TRANSACTION_BOUNDARY_INTERVAL_CONFIG, "10000");
+ 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,
+ recordNum >= recordsProduced);
+ }
+
+ /**
+ * 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 "connector" transaction boundary is used with a
connector that defines transactions whose
+ * size correspond to successive elements of the Fibonacci sequence, where
transactions with an even number of
+ * records are aborted, and those with an odd number of records are
committed.
+ */
+ @Test
+ public void testConnectorBoundary() throws Exception {
+ String offsetsTopic = "exactly-once-source-cluster-offsets";
+ workerProps.put(DistributedConfig.OFFSET_STORAGE_TOPIC_CONFIG,
offsetsTopic);
+ 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, CONNECTOR.toString());
+ props.put("custom.transaction.boundary.support", "supported");
+ 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));
+
+ Map<String, Object> consumerProps = new HashMap<>();
+ consumerProps.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG,
"read_committed");
+ consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
+ // consume all records from the source topic or fail, to ensure that
they were correctly produced
+ ConsumerRecords<byte[], byte[]> sourceRecords = connect.kafka()
+ .consume(
+ recordsProduced,
+ TimeUnit.MINUTES.toMillis(1),
+ consumerProps,
+ "test-topic");
+ assertTrue("Not enough records produced by source connector. Expected
at least: " + recordsProduced + " + but got " + sourceRecords.count(),
+ sourceRecords.count() >= recordsProduced);
+
+ // also consume from the cluster's offsets topic to verify that the
expected offsets (which should correspond to the connector's
+ // custom transaction boundaries) were committed
+ List<Long> expectedOffsetSeqnos = new ArrayList<>();
+ long lastExpectedOffsetSeqno = 1;
+ long nextExpectedOffsetSeqno = 1;
+ while (nextExpectedOffsetSeqno <= recordsProduced) {
+ expectedOffsetSeqnos.add(nextExpectedOffsetSeqno);
+ nextExpectedOffsetSeqno += lastExpectedOffsetSeqno;
+ lastExpectedOffsetSeqno = nextExpectedOffsetSeqno -
lastExpectedOffsetSeqno;
+ }
+ ConsumerRecords<byte[], byte[]> offsetRecords = connect.kafka()
+ .consume(
+ expectedOffsetSeqnos.size(),
+ TimeUnit.MINUTES.toMillis(1),
+ consumerProps,
+ offsetsTopic
+ );
+
+ List<Long> actualOffsetSeqnos = new ArrayList<>();
+ offsetRecords.forEach(record ->
actualOffsetSeqnos.add(parseAndAssertOffsetForSingleTask(record)));
+
+ assertEquals("Committed offsets should match connector-defined
transaction boundaries",
+ expectedOffsetSeqnos, actualOffsetSeqnos.subList(0,
expectedOffsetSeqnos.size()));
+
+ List<Long> expectedRecordSeqnos = LongStream.range(1, recordsProduced
+ 1).boxed().collect(Collectors.toList());
+ long priorBoundary = 1;
+ long nextBoundary = 2;
+ while (priorBoundary <
expectedRecordSeqnos.get(expectedRecordSeqnos.size() - 1)) {
+ if (nextBoundary % 2 == 0) {
+ for (long i = priorBoundary + 1; i < nextBoundary + 1; i++) {
+ expectedRecordSeqnos.remove(i);
+ }
+ }
+ nextBoundary += priorBoundary;
+ priorBoundary = nextBoundary - priorBoundary;
+ }
+ List<Long> actualRecordSeqnos = new ArrayList<>();
+ sourceRecords.forEach(record ->
actualRecordSeqnos.add(parseAndAssertValueForSingleTask(record)));
+ Collections.sort(actualRecordSeqnos);
+ assertEquals("Committed records should exclude connector-aborted
transactions",
+ expectedRecordSeqnos, actualRecordSeqnos.subList(0,
expectedRecordSeqnos.size()));
+ }
+
+ /**
+ * Brings up a one-node cluster, then intentionally fences out the
transactional producer used by the leader
+ * for writes to the config topic to simulate a zombie leader being active
in the cluster. The leader should
+ * automatically recover, verify that it is still the leader, and then
succeed to create a connector when the
+ * user resends the request.
+ */
+ @Test
+ public void testFencedLeaderRecovery() throws Exception {
+ // Much slower offset commit interval; should never be triggered
during this test
+ workerProps.put(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG,
"600000");
+ 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);
+
+ // make sure the worker is actually up (otherwise, it may fence out
our simulated zombie leader, instead of the other way around)
+ assertEquals(404,
connect.requestGet(connect.endpointForResource("connectors/nonexistent")).getStatus());
+
+ // fence out the leader of the cluster
+
transactionalProducer(DistributedConfig.transactionalProducerId(CLUSTER_GROUP_ID)).initTransactions();
+
+ // start a source connector--should fail the first time
+ assertThrows(ConnectRestException.class, () ->
connect.configureConnector(CONNECTOR_NAME, props));
+
+ // if at first you don't succeed, then spam the worker with rest
requests until it gives in to your demands
+ 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,
+ recordNum >= recordsProduced);
+ }
+
+ /**
+ * A moderately-complex green-path test that ensures the worker can start
up and run tasks for a source
+ * connector that gets reconfigured, and will fence out potential zombie
tasks for older generations before
+ * bringing up new task instances.
+ */
+ @Test
+ public void testConnectorReconfiguration() throws Exception {
+ // Much slower offset commit interval; should never be triggered
during this test
+ workerProps.put(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG,
"600000");
+ 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(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("messages.per.poll", Integer.toString(recordsProduced));
+
+ // expect all records to be consumed and committed by the connector
+ connectorHandle.expectedRecords(recordsProduced);
+ connectorHandle.expectedCommits(recordsProduced);
+
+ StartAndStopLatch connectorStart = connectorAndTaskStart(3);
+ props.put(TASKS_MAX_CONFIG, "3");
+ // start a source connector
+ connect.configureConnector(CONNECTOR_NAME, props);
+ assertConnectorStarted(connectorStart);
+
+ assertProducersAreFencedOnReconfiguration(3, 5, topic, props);
+ assertProducersAreFencedOnReconfiguration(5, 1, topic, props);
+ assertProducersAreFencedOnReconfiguration(1, 5, topic, props);
+ assertProducersAreFencedOnReconfiguration(5, 3, topic, props);
+
+ // Do a final sanity check to make sure that the final generation of
tasks is able to run
+ 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,
+ recordNum >= recordsProduced);
+ }
+
+ @Test
+ public void testConnectorFailsOnInabilityToFence() throws Exception {
Review Comment:
I feel the missing Javadoc on this one!
##########
connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExactlyOnceSourceIntegrationTest.java:
##########
@@ -0,0 +1,997 @@
+/*
+ * 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,
+ recordNum >= recordsProduced);
+ }
+
+ /**
+ * 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 "interval" transaction boundary is used with a
connector-specific override.
+ */
+ @Test
+ public void testIntervalBoundary() 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, INTERVAL.toString());
+ props.put(TRANSACTION_BOUNDARY_INTERVAL_CONFIG, "10000");
+ 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,
+ recordNum >= recordsProduced);
+ }
+
+ /**
+ * 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 "connector" transaction boundary is used with a
connector that defines transactions whose
+ * size correspond to successive elements of the Fibonacci sequence, where
transactions with an even number of
+ * records are aborted, and those with an odd number of records are
committed.
+ */
+ @Test
+ public void testConnectorBoundary() throws Exception {
+ String offsetsTopic = "exactly-once-source-cluster-offsets";
+ workerProps.put(DistributedConfig.OFFSET_STORAGE_TOPIC_CONFIG,
offsetsTopic);
+ 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, CONNECTOR.toString());
+ props.put("custom.transaction.boundary.support", "supported");
+ 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));
+
+ Map<String, Object> consumerProps = new HashMap<>();
+ consumerProps.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG,
"read_committed");
+ consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
+ // consume all records from the source topic or fail, to ensure that
they were correctly produced
+ ConsumerRecords<byte[], byte[]> sourceRecords = connect.kafka()
+ .consume(
+ recordsProduced,
+ TimeUnit.MINUTES.toMillis(1),
+ consumerProps,
+ "test-topic");
+ assertTrue("Not enough records produced by source connector. Expected
at least: " + recordsProduced + " + but got " + sourceRecords.count(),
+ sourceRecords.count() >= recordsProduced);
+
+ // also consume from the cluster's offsets topic to verify that the
expected offsets (which should correspond to the connector's
+ // custom transaction boundaries) were committed
+ List<Long> expectedOffsetSeqnos = new ArrayList<>();
+ long lastExpectedOffsetSeqno = 1;
+ long nextExpectedOffsetSeqno = 1;
+ while (nextExpectedOffsetSeqno <= recordsProduced) {
+ expectedOffsetSeqnos.add(nextExpectedOffsetSeqno);
+ nextExpectedOffsetSeqno += lastExpectedOffsetSeqno;
+ lastExpectedOffsetSeqno = nextExpectedOffsetSeqno -
lastExpectedOffsetSeqno;
+ }
+ ConsumerRecords<byte[], byte[]> offsetRecords = connect.kafka()
+ .consume(
+ expectedOffsetSeqnos.size(),
+ TimeUnit.MINUTES.toMillis(1),
+ consumerProps,
+ offsetsTopic
+ );
+
+ List<Long> actualOffsetSeqnos = new ArrayList<>();
+ offsetRecords.forEach(record ->
actualOffsetSeqnos.add(parseAndAssertOffsetForSingleTask(record)));
+
+ assertEquals("Committed offsets should match connector-defined
transaction boundaries",
+ expectedOffsetSeqnos, actualOffsetSeqnos.subList(0,
expectedOffsetSeqnos.size()));
+
+ List<Long> expectedRecordSeqnos = LongStream.range(1, recordsProduced
+ 1).boxed().collect(Collectors.toList());
+ long priorBoundary = 1;
+ long nextBoundary = 2;
+ while (priorBoundary <
expectedRecordSeqnos.get(expectedRecordSeqnos.size() - 1)) {
+ if (nextBoundary % 2 == 0) {
+ for (long i = priorBoundary + 1; i < nextBoundary + 1; i++) {
+ expectedRecordSeqnos.remove(i);
+ }
+ }
+ nextBoundary += priorBoundary;
+ priorBoundary = nextBoundary - priorBoundary;
+ }
+ List<Long> actualRecordSeqnos = new ArrayList<>();
+ sourceRecords.forEach(record ->
actualRecordSeqnos.add(parseAndAssertValueForSingleTask(record)));
+ Collections.sort(actualRecordSeqnos);
+ assertEquals("Committed records should exclude connector-aborted
transactions",
+ expectedRecordSeqnos, actualRecordSeqnos.subList(0,
expectedRecordSeqnos.size()));
+ }
+
+ /**
+ * Brings up a one-node cluster, then intentionally fences out the
transactional producer used by the leader
+ * for writes to the config topic to simulate a zombie leader being active
in the cluster. The leader should
+ * automatically recover, verify that it is still the leader, and then
succeed to create a connector when the
+ * user resends the request.
+ */
+ @Test
+ public void testFencedLeaderRecovery() throws Exception {
+ // Much slower offset commit interval; should never be triggered
during this test
+ workerProps.put(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG,
"600000");
+ 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);
+
+ // make sure the worker is actually up (otherwise, it may fence out
our simulated zombie leader, instead of the other way around)
+ assertEquals(404,
connect.requestGet(connect.endpointForResource("connectors/nonexistent")).getStatus());
+
+ // fence out the leader of the cluster
+
transactionalProducer(DistributedConfig.transactionalProducerId(CLUSTER_GROUP_ID)).initTransactions();
+
+ // start a source connector--should fail the first time
+ assertThrows(ConnectRestException.class, () ->
connect.configureConnector(CONNECTOR_NAME, props));
+
+ // if at first you don't succeed, then spam the worker with rest
requests until it gives in to your demands
Review Comment:
This doesn't really explain why this second call is necessary.
##########
connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExactlyOnceSourceIntegrationTest.java:
##########
@@ -0,0 +1,997 @@
+/*
+ * 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,
+ recordNum >= recordsProduced);
+ }
+
+ /**
+ * 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 "interval" transaction boundary is used with a
connector-specific override.
+ */
+ @Test
+ public void testIntervalBoundary() 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, INTERVAL.toString());
+ props.put(TRANSACTION_BOUNDARY_INTERVAL_CONFIG, "10000");
+ 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,
+ recordNum >= recordsProduced);
+ }
+
+ /**
+ * 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 "connector" transaction boundary is used with a
connector that defines transactions whose
+ * size correspond to successive elements of the Fibonacci sequence, where
transactions with an even number of
+ * records are aborted, and those with an odd number of records are
committed.
+ */
+ @Test
+ public void testConnectorBoundary() throws Exception {
+ String offsetsTopic = "exactly-once-source-cluster-offsets";
+ workerProps.put(DistributedConfig.OFFSET_STORAGE_TOPIC_CONFIG,
offsetsTopic);
+ 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, CONNECTOR.toString());
+ props.put("custom.transaction.boundary.support", "supported");
+ 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));
+
+ Map<String, Object> consumerProps = new HashMap<>();
+ consumerProps.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG,
"read_committed");
+ consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
+ // consume all records from the source topic or fail, to ensure that
they were correctly produced
+ ConsumerRecords<byte[], byte[]> sourceRecords = connect.kafka()
+ .consume(
+ recordsProduced,
+ TimeUnit.MINUTES.toMillis(1),
+ consumerProps,
+ "test-topic");
+ assertTrue("Not enough records produced by source connector. Expected
at least: " + recordsProduced + " + but got " + sourceRecords.count(),
+ sourceRecords.count() >= recordsProduced);
+
+ // also consume from the cluster's offsets topic to verify that the
expected offsets (which should correspond to the connector's
+ // custom transaction boundaries) were committed
+ List<Long> expectedOffsetSeqnos = new ArrayList<>();
+ long lastExpectedOffsetSeqno = 1;
+ long nextExpectedOffsetSeqno = 1;
+ while (nextExpectedOffsetSeqno <= recordsProduced) {
+ expectedOffsetSeqnos.add(nextExpectedOffsetSeqno);
+ nextExpectedOffsetSeqno += lastExpectedOffsetSeqno;
+ lastExpectedOffsetSeqno = nextExpectedOffsetSeqno -
lastExpectedOffsetSeqno;
+ }
+ ConsumerRecords<byte[], byte[]> offsetRecords = connect.kafka()
+ .consume(
+ expectedOffsetSeqnos.size(),
+ TimeUnit.MINUTES.toMillis(1),
+ consumerProps,
+ offsetsTopic
+ );
+
+ List<Long> actualOffsetSeqnos = new ArrayList<>();
+ offsetRecords.forEach(record ->
actualOffsetSeqnos.add(parseAndAssertOffsetForSingleTask(record)));
+
+ assertEquals("Committed offsets should match connector-defined
transaction boundaries",
+ expectedOffsetSeqnos, actualOffsetSeqnos.subList(0,
expectedOffsetSeqnos.size()));
+
+ List<Long> expectedRecordSeqnos = LongStream.range(1, recordsProduced
+ 1).boxed().collect(Collectors.toList());
+ long priorBoundary = 1;
+ long nextBoundary = 2;
+ while (priorBoundary <
expectedRecordSeqnos.get(expectedRecordSeqnos.size() - 1)) {
+ if (nextBoundary % 2 == 0) {
+ for (long i = priorBoundary + 1; i < nextBoundary + 1; i++) {
+ expectedRecordSeqnos.remove(i);
+ }
+ }
+ nextBoundary += priorBoundary;
+ priorBoundary = nextBoundary - priorBoundary;
+ }
+ List<Long> actualRecordSeqnos = new ArrayList<>();
+ sourceRecords.forEach(record ->
actualRecordSeqnos.add(parseAndAssertValueForSingleTask(record)));
+ Collections.sort(actualRecordSeqnos);
+ assertEquals("Committed records should exclude connector-aborted
transactions",
+ expectedRecordSeqnos, actualRecordSeqnos.subList(0,
expectedRecordSeqnos.size()));
+ }
+
+ /**
+ * Brings up a one-node cluster, then intentionally fences out the
transactional producer used by the leader
+ * for writes to the config topic to simulate a zombie leader being active
in the cluster. The leader should
+ * automatically recover, verify that it is still the leader, and then
succeed to create a connector when the
+ * user resends the request.
+ */
+ @Test
+ public void testFencedLeaderRecovery() throws Exception {
+ // Much slower offset commit interval; should never be triggered
during this test
+ workerProps.put(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG,
"600000");
+ 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);
+
+ // make sure the worker is actually up (otherwise, it may fence out
our simulated zombie leader, instead of the other way around)
+ assertEquals(404,
connect.requestGet(connect.endpointForResource("connectors/nonexistent")).getStatus());
+
+ // fence out the leader of the cluster
+
transactionalProducer(DistributedConfig.transactionalProducerId(CLUSTER_GROUP_ID)).initTransactions();
+
+ // start a source connector--should fail the first time
+ assertThrows(ConnectRestException.class, () ->
connect.configureConnector(CONNECTOR_NAME, props));
+
+ // if at first you don't succeed, then spam the worker with rest
requests until it gives in to your demands
+ 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,
+ recordNum >= recordsProduced);
+ }
+
+ /**
+ * A moderately-complex green-path test that ensures the worker can start
up and run tasks for a source
+ * connector that gets reconfigured, and will fence out potential zombie
tasks for older generations before
+ * bringing up new task instances.
+ */
+ @Test
+ public void testConnectorReconfiguration() throws Exception {
+ // Much slower offset commit interval; should never be triggered
during this test
+ workerProps.put(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG,
"600000");
+ 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(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("messages.per.poll", Integer.toString(recordsProduced));
+
+ // expect all records to be consumed and committed by the connector
+ connectorHandle.expectedRecords(recordsProduced);
+ connectorHandle.expectedCommits(recordsProduced);
+
+ StartAndStopLatch connectorStart = connectorAndTaskStart(3);
+ props.put(TASKS_MAX_CONFIG, "3");
+ // start a source connector
+ connect.configureConnector(CONNECTOR_NAME, props);
+ assertConnectorStarted(connectorStart);
+
+ assertProducersAreFencedOnReconfiguration(3, 5, topic, props);
+ assertProducersAreFencedOnReconfiguration(5, 1, topic, props);
+ assertProducersAreFencedOnReconfiguration(1, 5, topic, props);
+ assertProducersAreFencedOnReconfiguration(5, 3, topic, props);
+
+ // Do a final sanity check to make sure that the final generation of
tasks is able to run
+ 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,
+ recordNum >= recordsProduced);
+ }
+
+ @Test
+ public void testTasksFailOnInabilityToFence() throws Exception {
+ brokerProps.put("authorizer.class.name",
"kafka.security.authorizer.AclAuthorizer");
+ brokerProps.put("sasl.enabled.mechanisms", "PLAIN");
+ brokerProps.put("sasl.mechanism.inter.broker.protocol", "PLAIN");
+ brokerProps.put("security.inter.broker.protocol", "SASL_PLAINTEXT");
+ brokerProps.put("listeners", "SASL_PLAINTEXT://localhost:0");
+ brokerProps.put("listener.name.sasl_plaintext.plain.sasl.jaas.config",
+ "org.apache.kafka.common.security.plain.PlainLoginModule
required "
+ + "username=\"super\" "
+ + "password=\"super_pwd\" "
+ + "user_connector=\"connector_pwd\" "
+ + "user_super=\"super_pwd\";");
+ brokerProps.put("super.users", "User:super");
+
+ Map<String, String> superUserClientConfig = new HashMap<>();
+ superUserClientConfig.put("sasl.mechanism", "PLAIN");
+ superUserClientConfig.put("security.protocol", "SASL_PLAINTEXT");
+ superUserClientConfig.put("sasl.jaas.config",
+ "org.apache.kafka.common.security.plain.PlainLoginModule
required "
+ + "username=\"super\" "
+ + "password=\"super_pwd\";");
+ // Give the worker super-user privileges
+ workerProps.putAll(superUserClientConfig);
+
+ final String globalOffsetsTopic = "connect-worker-offsets-topic";
+ workerProps.put(DistributedConfig.OFFSET_STORAGE_TOPIC_CONFIG,
globalOffsetsTopic);
+
+ startConnect();
+
+ String topic = "test-topic";
+ Admin admin =
connect.kafka().createAdminClient(Utils.mkProperties(superUserClientConfig));
+ admin.createTopics(Collections.singleton(new NewTopic(topic, 3,
(short) 1))).all().get();
+
+ Map<String, String> props = new HashMap<>();
+ int tasksMax = 2; // Use two tasks since single-task connectors don't
require zombie fencing
+ props.put(CONNECTOR_CLASS_CONFIG,
MonitorableSourceConnector.class.getName());
+ 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(TASKS_MAX_CONFIG, Integer.toString(tasksMax));
+ // Give the connectors' consumer and producer super-user privileges
+ superUserClientConfig.forEach((property, value) -> {
+ props.put(CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX + property,
value);
+ props.put(CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX + property,
value);
+ });
+ // But limit its admin client's privileges
+ props.put(CONNECTOR_CLIENT_ADMIN_OVERRIDES_PREFIX + "sasl.mechanism",
"PLAIN");
+ props.put(CONNECTOR_CLIENT_ADMIN_OVERRIDES_PREFIX +
"security.protocol", "SASL_PLAINTEXT");
+ props.put(CONNECTOR_CLIENT_ADMIN_OVERRIDES_PREFIX + "sasl.jaas.config",
+ "org.apache.kafka.common.security.plain.PlainLoginModule
required "
+ + "username=\"connector\" "
+ + "password=\"connector_pwd\";");
+ // Grant the connector's admin permissions to access the topics for
its records and offsets
+ // Intentionally leave out permissions required for fencing
+ admin.createAcls(Arrays.asList(
+ new AclBinding(
+ new ResourcePattern(ResourceType.TOPIC, topic,
PatternType.LITERAL),
+ new AccessControlEntry("User:connector", "*",
AclOperation.ALL, AclPermissionType.ALLOW)
+ ),
+ new AclBinding(
+ new ResourcePattern(ResourceType.TOPIC,
globalOffsetsTopic, PatternType.LITERAL),
+ new AccessControlEntry("User:connector", "*",
AclOperation.ALL, AclPermissionType.ALLOW)
+ )
+ )).all().get();
+
+ StartAndStopLatch connectorStart = connectorAndTaskStart(tasksMax);
+
+ log.info("Bringing up connector with fresh slate; fencing should not
be necessary");
+ connect.configureConnector(CONNECTOR_NAME, props);
+ assertConnectorStarted(connectorStart);
+ // Verify that the connector and its tasks have been able to start
successfully
+
connect.assertions().assertConnectorAndExactlyNumTasksAreRunning(CONNECTOR_NAME,
tasksMax, "Connector and task should have started successfully");
+
+ log.info("Reconfiguring connector; fencing should be necessary, and
tasks should fail to start");
+ props.put("message.in.a.bottle", "19e184427ac45bd34c8588a4e771aa1a");
+ connect.configureConnector(CONNECTOR_NAME, props);
+
+ // Verify that the task has failed, and that the failure is visible to
users via the REST API
+
connect.assertions().assertConnectorIsRunningAndTasksHaveFailed(CONNECTOR_NAME,
tasksMax, "Task should have failed on startup");
+
+ // Now grant the necessary permissions for fencing to the connector's
admin
+ admin.createAcls(Arrays.asList(
+ new AclBinding(
+ new ResourcePattern(ResourceType.TRANSACTIONAL_ID,
Worker.taskTransactionalId(CLUSTER_GROUP_ID, CONNECTOR_NAME, 0),
PatternType.LITERAL),
+ new AccessControlEntry("User:connector", "*",
AclOperation.ALL, AclPermissionType.ALLOW)
+ ),
+ new AclBinding(
+ new ResourcePattern(ResourceType.TRANSACTIONAL_ID,
Worker.taskTransactionalId(CLUSTER_GROUP_ID, CONNECTOR_NAME, 1),
PatternType.LITERAL),
+ new AccessControlEntry("User:connector", "*",
AclOperation.ALL, AclPermissionType.ALLOW)
+ )
+ ));
+
+ log.info("Restarting connector after tweaking its ACLs; fencing should
succeed this time");
+ connect.restartConnectorAndTasks(CONNECTOR_NAME, false, true, false);
+ // Verify that the connector and its tasks have been able to restart
successfully
+
connect.assertions().assertConnectorAndExactlyNumTasksAreRunning(CONNECTOR_NAME,
tasksMax, "Connector and task should have restarted successfully");
+ }
+
+ @Test
+ public void testSeparateOffsetsTopic() throws Exception {
+ final String globalOffsetsTopic = "connect-worker-offsets-topic";
+ workerProps.put(DistributedConfig.OFFSET_STORAGE_TOPIC_CONFIG,
globalOffsetsTopic);
+
+ startConnect();
+ EmbeddedKafkaCluster connectorTargetedCluster = new
EmbeddedKafkaCluster(1, brokerProps);
+ try (Closeable clusterShutdown = connectorTargetedCluster::stop) {
+ connectorTargetedCluster.start();
+ String topic = "test-topic";
+ connectorTargetedCluster.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));
+ props.put(CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX +
BOOTSTRAP_SERVERS_CONFIG, connectorTargetedCluster.bootstrapServers());
+ props.put(CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX +
BOOTSTRAP_SERVERS_CONFIG, connectorTargetedCluster.bootstrapServers());
+ props.put(CONNECTOR_CLIENT_ADMIN_OVERRIDES_PREFIX +
BOOTSTRAP_SERVERS_CONFIG, connectorTargetedCluster.bootstrapServers());
+ String offsetsTopic = CONNECTOR_NAME + "-offsets";
+ props.put(OFFSETS_TOPIC_CONFIG, offsetsTopic);
+
+ // 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 = connectorTargetedCluster
+ .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,
+ recordNum >= recordsProduced);
+
+ // also consume from the connector's dedicated offsets topic; just
need to read one offset record
+ ConsumerRecord<byte[], byte[]> offsetRecord =
connectorTargetedCluster
+ .consume(
+ 1,
+ TimeUnit.MINUTES.toMillis(1),
+
Collections.singletonMap(ConsumerConfig.ISOLATION_LEVEL_CONFIG,
"read_committed"),
+ offsetsTopic
+ ).iterator().next();
+ long seqno = parseAndAssertOffsetForSingleTask(offsetRecord);
+ assertEquals("Offset commits should occur on connector-defined
poll boundaries, which happen every " + recordsProduced + " records",
+ 0, seqno % recordsProduced);
+
+ // also consume from the cluster's global offsets topic; again,
just need to read one offset record
+ offsetRecord = connect.kafka()
+ .consume(
+ 1,
+ TimeUnit.MINUTES.toMillis(1),
+ globalOffsetsTopic
+ ).iterator().next();
+ seqno = parseAndAssertOffsetForSingleTask(offsetRecord);
+ assertEquals("Offset commits should occur on connector-defined
poll boundaries, which happen every " + recordsProduced + " records",
+ 0, seqno % recordsProduced);
+
+ // Shut down the whole cluster
+ connect.workers().forEach(connect::removeWorker);
+ // Reconfigure the cluster with exactly-once support disabled
+ workerProps.put(EXACTLY_ONCE_SOURCE_SUPPORT_CONFIG, "disabled");
+
+ // Establish new expectations for records+offsets
+ connectorHandle.expectedRecords(recordsProduced);
+ connectorHandle.expectedCommits(recordsProduced);
+
+ // Restart the whole cluster
+ for (int i = 0; i < DEFAULT_NUM_WORKERS; i++) {
+ connect.addWorker();
+ }
+
+ // And perform a basic sanity check that the cluster is able to
come back up, our connector and its task are able to resume running,
+ // and the task is still able to produce source records and commit
offsets
+
connect.assertions().assertAtLeastNumWorkersAreUp(DEFAULT_NUM_WORKERS, "cluster
did not restart in time");
+ connect.assertions().assertConnectorAndExactlyNumTasksAreRunning(
+ CONNECTOR_NAME,
+ 1,
+ "connector and tasks did not resume running after cluster
restart in time"
+ );
+
+ 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));
+
+ // Delete the connector before shutting down the Kafka cluster
it's targeting
+ connect.deleteConnector(CONNECTOR_NAME);
+ }
+ }
+
+ @Test
+ public void testPotentialDeadlockWhenProducingToOffsetsTopic() throws
Exception {
Review Comment:
Could we have a comment explaining the potential deadlock?
--
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]