From 1fb8bd9c44749bc0045dce0e156fea04652f0524 Mon Sep 17 00:00:00 2001 From: Lev Zemlyanov Date: Fri, 19 Mar 2021 07:03:36 -0700 Subject: [PATCH] KAFKA-10070: parameterize Connect unit tests to remove code duplication (#10299) Reviewers: Mickael Maison , Konstantine Karantasis --- checkstyle/suppressions.xml | 2 +- .../runtime/ErrorHandlingTaskTest.java | 56 +- ...rrorHandlingTaskWithTopicCreationTest.java | 658 ------- .../connect/runtime/WorkerSourceTaskTest.java | 459 ++++- ...WorkerSourceTaskWithTopicCreationTest.java | 1469 ---------------- .../kafka/connect/runtime/WorkerTest.java | 21 +- .../runtime/WorkerWithTopicCreationTest.java | 1510 ----------------- .../kafka/connect/util/ParameterizedTest.java | 83 + 8 files changed, 572 insertions(+), 3686 deletions(-) delete mode 100644 connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskWithTopicCreationTest.java delete mode 100644 connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskWithTopicCreationTest.java delete mode 100644 connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerWithTopicCreationTest.java create mode 100644 connect/runtime/src/test/java/org/apache/kafka/connect/util/ParameterizedTest.java diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index 242fef65e98..5473b035ea9 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -137,7 +137,7 @@ + files="(DistributedHerder|KafkaBasedLog|WorkerSourceTaskWithTopicCreation|WorkerSourceTask)Test.java"/> diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java index daa9eddac9c..33ebd3df391 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java @@ -39,6 +39,7 @@ import org.apache.kafka.connect.runtime.errors.ErrorReporter; import org.apache.kafka.connect.runtime.errors.LogReporter; import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator; import org.apache.kafka.connect.runtime.errors.ToleranceType; +import org.apache.kafka.connect.runtime.errors.WorkerErrantRecordReporter; import org.apache.kafka.connect.runtime.isolation.PluginClassLoader; import org.apache.kafka.connect.runtime.isolation.Plugins; import org.apache.kafka.connect.runtime.standalone.StandaloneConfig; @@ -56,7 +57,9 @@ import org.apache.kafka.connect.storage.StringConverter; import org.apache.kafka.connect.transforms.Transformation; import org.apache.kafka.connect.transforms.util.SimpleConfig; import org.apache.kafka.connect.util.ConnectorTaskId; +import org.apache.kafka.connect.util.ParameterizedTest; import org.apache.kafka.connect.util.TopicAdmin; +import org.apache.kafka.connect.util.TopicCreationGroup; import org.easymock.Capture; import org.easymock.EasyMock; import org.easymock.IExpectationSetters; @@ -69,14 +72,17 @@ import org.powermock.api.easymock.annotation.Mock; import org.powermock.core.classloader.annotations.PowerMockIgnore; import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; +import org.powermock.modules.junit4.PowerMockRunnerDelegate; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.time.Duration; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.Map; +import java.util.Set; import java.util.concurrent.Executor; import static java.util.Collections.emptyMap; @@ -87,10 +93,16 @@ import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_C import static org.apache.kafka.connect.runtime.ConnectorConfig.KEY_CONVERTER_CLASS_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.TOPIC_CREATION_GROUPS_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.DEFAULT_TOPIC_CREATION_PREFIX; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.INCLUDE_REGEX_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.PARTITIONS_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.REPLICATION_FACTOR_CONFIG; import static org.apache.kafka.connect.runtime.WorkerConfig.TOPIC_CREATION_ENABLE_CONFIG; import static org.junit.Assert.assertEquals; @RunWith(PowerMockRunner.class) +@PowerMockRunnerDelegate(ParameterizedTest.class) @PrepareForTest({WorkerSinkTask.class, WorkerSourceTask.class}) @PowerMockIgnore("javax.management.*") public class ErrorHandlingTaskTest { @@ -156,10 +168,21 @@ public class ErrorHandlingTaskTest { @SuppressWarnings("unused") @Mock private StatusBackingStore statusBackingStore; + @Mock + private WorkerErrantRecordReporter workerErrantRecordReporter; + private ErrorHandlingMetrics errorHandlingMetrics; - // when this test becomes parameterized, this variable will be a test parameter - public boolean enableTopicCreation = false; + private boolean enableTopicCreation; + + @ParameterizedTest.Parameters + public static Collection parameters() { + return Arrays.asList(false, true); + } + + public ErrorHandlingTaskTest(boolean enableTopicCreation) { + this.enableTopicCreation = enableTopicCreation; + } @Before public void setup() { @@ -189,6 +212,10 @@ public class ErrorHandlingTaskTest { 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(TOPIC_CREATION_GROUPS_CONFIG, String.join(",", "foo", "bar")); + props.put(DEFAULT_TOPIC_CREATION_PREFIX + REPLICATION_FACTOR_CONFIG, String.valueOf(1)); + props.put(DEFAULT_TOPIC_CREATION_PREFIX + PARTITIONS_CONFIG, String.valueOf(1)); + props.put(SourceConnectorConfig.TOPIC_CREATION_PREFIX + "foo" + "." + INCLUDE_REGEX_CONFIG, topic); return props; } @@ -514,9 +541,16 @@ public class ErrorHandlingTaskTest { private void expectTopicCreation(String topic) { if (workerConfig.topicCreationEnable()) { EasyMock.expect(admin.describeTopics(topic)).andReturn(Collections.emptyMap()); - Capture newTopicCapture = EasyMock.newCapture(); - EasyMock.expect(admin.createTopic(EasyMock.capture(newTopicCapture))).andReturn(true); + + if (enableTopicCreation) { + Set created = Collections.singleton(topic); + Set existing = Collections.emptySet(); + TopicAdmin.TopicCreationResponse response = new TopicAdmin.TopicCreationResponse(created, existing); + EasyMock.expect(admin.createOrFindTopics(EasyMock.capture(newTopicCapture))).andReturn(response); + } else { + EasyMock.expect(admin.createTopic(EasyMock.capture(newTopicCapture))).andReturn(true); + } } } @@ -533,7 +567,7 @@ public class ErrorHandlingTaskTest { taskId, sinkTask, statusListener, initialState, workerConfig, ClusterConfigState.EMPTY, metrics, converter, converter, headerConverter, sinkTransforms, consumer, pluginLoader, time, - retryWithToleranceOperator, null, statusBackingStore); + retryWithToleranceOperator, workerErrantRecordReporter, statusBackingStore); } private void createSourceTask(TargetState initialState, RetryWithToleranceOperator retryWithToleranceOperator) { @@ -559,12 +593,12 @@ public class ErrorHandlingTaskTest { TransformationChain sourceTransforms = new TransformationChain<>(singletonList(new FaultyPassthrough()), retryWithToleranceOperator); workerSourceTask = PowerMock.createPartialMock( - WorkerSourceTask.class, new String[]{"commitOffsets", "isStopping"}, - taskId, sourceTask, statusListener, initialState, converter, converter, headerConverter, sourceTransforms, - producer, admin, null, - offsetReader, offsetWriter, workerConfig, - ClusterConfigState.EMPTY, metrics, pluginLoader, time, retryWithToleranceOperator, - statusBackingStore, (Executor) Runnable::run); + WorkerSourceTask.class, new String[]{"commitOffsets", "isStopping"}, + taskId, sourceTask, statusListener, initialState, converter, converter, headerConverter, sourceTransforms, + producer, admin, TopicCreationGroup.configuredGroups(sourceConfig), + offsetReader, offsetWriter, workerConfig, + ClusterConfigState.EMPTY, metrics, pluginLoader, time, retryWithToleranceOperator, + statusBackingStore, (Executor) Runnable::run); } private ConsumerRecords records(ConsumerRecord record) { diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskWithTopicCreationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskWithTopicCreationTest.java deleted file mode 100644 index 5cdfab96d22..00000000000 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskWithTopicCreationTest.java +++ /dev/null @@ -1,658 +0,0 @@ -/* - * 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.runtime; - -import org.apache.kafka.clients.admin.NewTopic; -import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.clients.producer.KafkaProducer; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.config.ConfigDef; -import org.apache.kafka.common.utils.MockTime; -import org.apache.kafka.common.utils.Time; -import org.apache.kafka.connect.connector.ConnectRecord; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.SchemaBuilder; -import org.apache.kafka.connect.data.Struct; -import org.apache.kafka.connect.errors.RetriableException; -import org.apache.kafka.connect.integration.MonitorableSourceConnector; -import org.apache.kafka.connect.json.JsonConverter; -import org.apache.kafka.connect.runtime.distributed.ClusterConfigState; -import org.apache.kafka.connect.runtime.errors.ErrorHandlingMetrics; -import org.apache.kafka.connect.runtime.errors.ErrorReporter; -import org.apache.kafka.connect.runtime.errors.LogReporter; -import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator; -import org.apache.kafka.connect.runtime.errors.ToleranceType; -import org.apache.kafka.connect.runtime.errors.WorkerErrantRecordReporter; -import org.apache.kafka.connect.runtime.isolation.PluginClassLoader; -import org.apache.kafka.connect.runtime.isolation.Plugins; -import org.apache.kafka.connect.runtime.standalone.StandaloneConfig; -import org.apache.kafka.connect.sink.SinkConnector; -import org.apache.kafka.connect.sink.SinkRecord; -import org.apache.kafka.connect.sink.SinkTask; -import org.apache.kafka.connect.source.SourceRecord; -import org.apache.kafka.connect.source.SourceTask; -import org.apache.kafka.connect.storage.Converter; -import org.apache.kafka.connect.storage.HeaderConverter; -import org.apache.kafka.connect.storage.OffsetStorageReaderImpl; -import org.apache.kafka.connect.storage.OffsetStorageWriter; -import org.apache.kafka.connect.storage.StatusBackingStore; -import org.apache.kafka.connect.storage.StringConverter; -import org.apache.kafka.connect.transforms.Transformation; -import org.apache.kafka.connect.transforms.util.SimpleConfig; -import org.apache.kafka.connect.util.ConnectorTaskId; -import org.apache.kafka.connect.util.TopicAdmin; -import org.apache.kafka.connect.util.TopicCreationGroup; -import org.easymock.Capture; -import org.easymock.EasyMock; -import org.easymock.IExpectationSetters; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.powermock.api.easymock.PowerMock; -import org.powermock.api.easymock.annotation.Mock; -import org.powermock.core.classloader.annotations.PowerMockIgnore; -import org.powermock.core.classloader.annotations.PrepareForTest; -import org.powermock.modules.junit4.PowerMockRunner; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.time.Duration; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.Executor; - -import static java.util.Collections.emptyMap; -import static java.util.Collections.singletonList; -import static org.apache.kafka.common.utils.Time.SYSTEM; -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.KEY_CONVERTER_CLASS_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.TOPIC_CREATION_GROUPS_CONFIG; -import static org.apache.kafka.connect.runtime.TopicCreationConfig.DEFAULT_TOPIC_CREATION_PREFIX; -import static org.apache.kafka.connect.runtime.TopicCreationConfig.INCLUDE_REGEX_CONFIG; -import static org.apache.kafka.connect.runtime.TopicCreationConfig.PARTITIONS_CONFIG; -import static org.apache.kafka.connect.runtime.TopicCreationConfig.REPLICATION_FACTOR_CONFIG; -import static org.apache.kafka.connect.runtime.WorkerConfig.TOPIC_CREATION_ENABLE_CONFIG; -import static org.junit.Assert.assertEquals; - -@RunWith(PowerMockRunner.class) -@PrepareForTest({WorkerSinkTask.class, WorkerSourceTask.class}) -@PowerMockIgnore("javax.management.*") -public class ErrorHandlingTaskWithTopicCreationTest { - - private static final String TOPIC = "test"; - private static final int PARTITION1 = 12; - private static final int PARTITION2 = 13; - private static final long FIRST_OFFSET = 45; - - @Mock Plugins plugins; - - private static final Map TASK_PROPS = new HashMap<>(); - - static { - TASK_PROPS.put(SinkConnector.TOPICS_CONFIG, TOPIC); - TASK_PROPS.put(TaskConfig.TASK_CLASS_CONFIG, TestSinkTask.class.getName()); - } - - public static final long OPERATOR_RETRY_TIMEOUT_MILLIS = 60000; - public static final long OPERATOR_RETRY_MAX_DELAY_MILLIS = 5000; - public static final ToleranceType OPERATOR_TOLERANCE_TYPE = ToleranceType.ALL; - - private static final TaskConfig TASK_CONFIG = new TaskConfig(TASK_PROPS); - - private ConnectorTaskId taskId = new ConnectorTaskId("job", 0); - private TargetState initialState = TargetState.STARTED; - private Time time; - private MockConnectMetrics metrics; - @SuppressWarnings("unused") - @Mock - private SinkTask sinkTask; - @SuppressWarnings("unused") - @Mock - private SourceTask sourceTask; - private Capture sinkTaskContext = EasyMock.newCapture(); - private WorkerConfig workerConfig; - private SourceConnectorConfig sourceConfig; - @Mock - private PluginClassLoader pluginLoader; - @SuppressWarnings("unused") - @Mock - private HeaderConverter headerConverter; - private WorkerSinkTask workerSinkTask; - private WorkerSourceTask workerSourceTask; - @SuppressWarnings("unused") - @Mock - private KafkaConsumer consumer; - @SuppressWarnings("unused") - @Mock - private KafkaProducer producer; - @SuppressWarnings("unused") - @Mock private TopicAdmin admin; - - @Mock - OffsetStorageReaderImpl offsetReader; - @Mock - OffsetStorageWriter offsetWriter; - - private Capture rebalanceListener = EasyMock.newCapture(); - @SuppressWarnings("unused") - @Mock - private TaskStatus.Listener statusListener; - @SuppressWarnings("unused") - @Mock private StatusBackingStore statusBackingStore; - - @Mock - private WorkerErrantRecordReporter workerErrantRecordReporter; - - private ErrorHandlingMetrics errorHandlingMetrics; - - // when this test becomes parameterized, this variable will be a test parameter - public boolean enableTopicCreation = true; - - @Before - public void setup() { - time = new MockTime(0, 0, 0); - metrics = new MockConnectMetrics(); - Map workerProps = new HashMap<>(); - workerProps.put("key.converter", "org.apache.kafka.connect.json.JsonConverter"); - workerProps.put("value.converter", "org.apache.kafka.connect.json.JsonConverter"); - workerProps.put("internal.key.converter", "org.apache.kafka.connect.json.JsonConverter"); - workerProps.put("internal.value.converter", "org.apache.kafka.connect.json.JsonConverter"); - workerProps.put("internal.key.converter.schemas.enable", "false"); - workerProps.put("internal.value.converter.schemas.enable", "false"); - workerProps.put("offset.storage.file.filename", "/tmp/connect.offsets"); - workerProps.put(TOPIC_CREATION_ENABLE_CONFIG, String.valueOf(enableTopicCreation)); - pluginLoader = PowerMock.createMock(PluginClassLoader.class); - workerConfig = new StandaloneConfig(workerProps); - sourceConfig = new SourceConnectorConfig(plugins, sourceConnectorPropsWithGroups(TOPIC), true); - errorHandlingMetrics = new ErrorHandlingMetrics(taskId, metrics); - } - - private Map sourceConnectorPropsWithGroups(String topic) { - // setup up props for the source connector - Map props = new HashMap<>(); - props.put("name", "foo-connector"); - props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getSimpleName()); - props.put(TASKS_MAX_CONFIG, String.valueOf(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(TOPIC_CREATION_GROUPS_CONFIG, String.join(",", "foo", "bar")); - props.put(DEFAULT_TOPIC_CREATION_PREFIX + REPLICATION_FACTOR_CONFIG, String.valueOf(1)); - props.put(DEFAULT_TOPIC_CREATION_PREFIX + PARTITIONS_CONFIG, String.valueOf(1)); - props.put(SourceConnectorConfig.TOPIC_CREATION_PREFIX + "foo" + "." + INCLUDE_REGEX_CONFIG, topic); - return props; - } - - @After - public void tearDown() { - if (metrics != null) { - metrics.stop(); - } - } - - @Test - public void testSinkTasksCloseErrorReporters() throws Exception { - ErrorReporter reporter = EasyMock.mock(ErrorReporter.class); - - RetryWithToleranceOperator retryWithToleranceOperator = operator(); - retryWithToleranceOperator.metrics(errorHandlingMetrics); - retryWithToleranceOperator.reporters(singletonList(reporter)); - - createSinkTask(initialState, retryWithToleranceOperator); - - expectInitializeTask(); - reporter.close(); - EasyMock.expectLastCall(); - sinkTask.stop(); - EasyMock.expectLastCall(); - - consumer.close(); - EasyMock.expectLastCall(); - - PowerMock.replayAll(); - - workerSinkTask.initialize(TASK_CONFIG); - workerSinkTask.initializeAndStart(); - workerSinkTask.close(); - - PowerMock.verifyAll(); - } - - @Test - public void testSourceTasksCloseErrorReporters() { - ErrorReporter reporter = EasyMock.mock(ErrorReporter.class); - - RetryWithToleranceOperator retryWithToleranceOperator = operator(); - retryWithToleranceOperator.metrics(errorHandlingMetrics); - retryWithToleranceOperator.reporters(singletonList(reporter)); - - createSourceTask(initialState, retryWithToleranceOperator); - - expectClose(); - - reporter.close(); - EasyMock.expectLastCall(); - - PowerMock.replayAll(); - - workerSourceTask.initialize(TASK_CONFIG); - workerSourceTask.close(); - - PowerMock.verifyAll(); - } - - @Test - public void testCloseErrorReportersExceptionPropagation() { - ErrorReporter reporterA = EasyMock.mock(ErrorReporter.class); - ErrorReporter reporterB = EasyMock.mock(ErrorReporter.class); - - RetryWithToleranceOperator retryWithToleranceOperator = operator(); - retryWithToleranceOperator.metrics(errorHandlingMetrics); - retryWithToleranceOperator.reporters(Arrays.asList(reporterA, reporterB)); - - createSourceTask(initialState, retryWithToleranceOperator); - - expectClose(); - - // Even though the reporters throw exceptions, they should both still be closed. - reporterA.close(); - EasyMock.expectLastCall().andThrow(new RuntimeException()); - - reporterB.close(); - EasyMock.expectLastCall().andThrow(new RuntimeException()); - - PowerMock.replayAll(); - - workerSourceTask.initialize(TASK_CONFIG); - workerSourceTask.close(); - - PowerMock.verifyAll(); - } - - @Test - public void testErrorHandlingInSinkTasks() throws Exception { - Map reportProps = new HashMap<>(); - reportProps.put(ConnectorConfig.ERRORS_LOG_ENABLE_CONFIG, "true"); - reportProps.put(ConnectorConfig.ERRORS_LOG_INCLUDE_MESSAGES_CONFIG, "true"); - LogReporter reporter = new LogReporter(taskId, connConfig(reportProps), errorHandlingMetrics); - - RetryWithToleranceOperator retryWithToleranceOperator = operator(); - retryWithToleranceOperator.metrics(errorHandlingMetrics); - retryWithToleranceOperator.reporters(singletonList(reporter)); - createSinkTask(initialState, retryWithToleranceOperator); - - expectInitializeTask(); - expectTaskGetTopic(true); - - // valid json - ConsumerRecord record1 = new ConsumerRecord<>(TOPIC, PARTITION1, FIRST_OFFSET, null, "{\"a\": 10}".getBytes()); - // bad json - ConsumerRecord record2 = new ConsumerRecord<>(TOPIC, PARTITION2, FIRST_OFFSET, null, "{\"a\" 10}".getBytes()); - - EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andReturn(records(record1)); - EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andReturn(records(record2)); - - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall().times(2); - - PowerMock.replayAll(); - - workerSinkTask.initialize(TASK_CONFIG); - workerSinkTask.initializeAndStart(); - workerSinkTask.iteration(); - - workerSinkTask.iteration(); - - // two records were consumed from Kafka - assertSinkMetricValue("sink-record-read-total", 2.0); - // only one was written to the task - assertSinkMetricValue("sink-record-send-total", 1.0); - // one record completely failed (converter issues) - assertErrorHandlingMetricValue("total-record-errors", 1.0); - // 2 failures in the transformation, and 1 in the converter - assertErrorHandlingMetricValue("total-record-failures", 3.0); - // one record completely failed (converter issues), and thus was skipped - assertErrorHandlingMetricValue("total-records-skipped", 1.0); - - PowerMock.verifyAll(); - } - - private RetryWithToleranceOperator operator() { - return new RetryWithToleranceOperator(OPERATOR_RETRY_TIMEOUT_MILLIS, OPERATOR_RETRY_MAX_DELAY_MILLIS, OPERATOR_TOLERANCE_TYPE, SYSTEM); - } - - @Test - public void testErrorHandlingInSourceTasks() throws Exception { - Map reportProps = new HashMap<>(); - reportProps.put(ConnectorConfig.ERRORS_LOG_ENABLE_CONFIG, "true"); - reportProps.put(ConnectorConfig.ERRORS_LOG_INCLUDE_MESSAGES_CONFIG, "true"); - LogReporter reporter = new LogReporter(taskId, connConfig(reportProps), errorHandlingMetrics); - - RetryWithToleranceOperator retryWithToleranceOperator = operator(); - retryWithToleranceOperator.metrics(errorHandlingMetrics); - retryWithToleranceOperator.reporters(singletonList(reporter)); - createSourceTask(initialState, retryWithToleranceOperator); - - // valid json - Schema valSchema = SchemaBuilder.struct().field("val", Schema.INT32_SCHEMA).build(); - Struct struct1 = new Struct(valSchema).put("val", 1234); - SourceRecord record1 = new SourceRecord(emptyMap(), emptyMap(), TOPIC, PARTITION1, valSchema, struct1); - Struct struct2 = new Struct(valSchema).put("val", 6789); - SourceRecord record2 = new SourceRecord(emptyMap(), emptyMap(), TOPIC, PARTITION1, valSchema, struct2); - - EasyMock.expect(workerSourceTask.isStopping()).andReturn(false); - EasyMock.expect(workerSourceTask.isStopping()).andReturn(false); - EasyMock.expect(workerSourceTask.isStopping()).andReturn(true); - - EasyMock.expect(workerSourceTask.commitOffsets()).andReturn(true); - - offsetWriter.offset(EasyMock.anyObject(), EasyMock.anyObject()); - EasyMock.expectLastCall().times(2); - sourceTask.initialize(EasyMock.anyObject()); - EasyMock.expectLastCall(); - - sourceTask.start(EasyMock.anyObject()); - EasyMock.expectLastCall(); - - EasyMock.expect(sourceTask.poll()).andReturn(singletonList(record1)); - EasyMock.expect(sourceTask.poll()).andReturn(singletonList(record2)); - expectTopicDoesNotExist(TOPIC); - EasyMock.expect(producer.send(EasyMock.anyObject(), EasyMock.anyObject())).andReturn(null).times(2); - - PowerMock.replayAll(); - - workerSourceTask.initialize(TASK_CONFIG); - workerSourceTask.execute(); - - // two records were consumed from Kafka - assertSourceMetricValue("source-record-poll-total", 2.0); - // only one was written to the task - assertSourceMetricValue("source-record-write-total", 0.0); - // one record completely failed (converter issues) - assertErrorHandlingMetricValue("total-record-errors", 0.0); - // 2 failures in the transformation, and 1 in the converter - assertErrorHandlingMetricValue("total-record-failures", 4.0); - // one record completely failed (converter issues), and thus was skipped - assertErrorHandlingMetricValue("total-records-skipped", 0.0); - - PowerMock.verifyAll(); - } - - private ConnectorConfig connConfig(Map connProps) { - Map props = new HashMap<>(); - props.put(ConnectorConfig.NAME_CONFIG, "test"); - props.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, SinkTask.class.getName()); - props.putAll(connProps); - return new ConnectorConfig(plugins, props); - } - - @Test - public void testErrorHandlingInSourceTasksWthBadConverter() throws Exception { - Map reportProps = new HashMap<>(); - reportProps.put(ConnectorConfig.ERRORS_LOG_ENABLE_CONFIG, "true"); - reportProps.put(ConnectorConfig.ERRORS_LOG_INCLUDE_MESSAGES_CONFIG, "true"); - LogReporter reporter = new LogReporter(taskId, connConfig(reportProps), errorHandlingMetrics); - - RetryWithToleranceOperator retryWithToleranceOperator = operator(); - retryWithToleranceOperator.metrics(errorHandlingMetrics); - retryWithToleranceOperator.reporters(singletonList(reporter)); - createSourceTask(initialState, retryWithToleranceOperator, badConverter()); - - // valid json - Schema valSchema = SchemaBuilder.struct().field("val", Schema.INT32_SCHEMA).build(); - Struct struct1 = new Struct(valSchema).put("val", 1234); - SourceRecord record1 = new SourceRecord(emptyMap(), emptyMap(), TOPIC, PARTITION1, valSchema, struct1); - Struct struct2 = new Struct(valSchema).put("val", 6789); - SourceRecord record2 = new SourceRecord(emptyMap(), emptyMap(), TOPIC, PARTITION1, valSchema, struct2); - - EasyMock.expect(workerSourceTask.isStopping()).andReturn(false); - EasyMock.expect(workerSourceTask.isStopping()).andReturn(false); - EasyMock.expect(workerSourceTask.isStopping()).andReturn(true); - - EasyMock.expect(workerSourceTask.commitOffsets()).andReturn(true); - - offsetWriter.offset(EasyMock.anyObject(), EasyMock.anyObject()); - EasyMock.expectLastCall().times(2); - sourceTask.initialize(EasyMock.anyObject()); - EasyMock.expectLastCall(); - - sourceTask.start(EasyMock.anyObject()); - EasyMock.expectLastCall(); - - EasyMock.expect(sourceTask.poll()).andReturn(singletonList(record1)); - EasyMock.expect(sourceTask.poll()).andReturn(singletonList(record2)); - expectTopicDoesNotExist(TOPIC); - EasyMock.expect(producer.send(EasyMock.anyObject(), EasyMock.anyObject())).andReturn(null).times(2); - - PowerMock.replayAll(); - - workerSourceTask.initialize(TASK_CONFIG); - workerSourceTask.execute(); - - // two records were consumed from Kafka - assertSourceMetricValue("source-record-poll-total", 2.0); - // only one was written to the task - assertSourceMetricValue("source-record-write-total", 0.0); - // one record completely failed (converter issues) - assertErrorHandlingMetricValue("total-record-errors", 0.0); - // 2 failures in the transformation, and 1 in the converter - assertErrorHandlingMetricValue("total-record-failures", 8.0); - // one record completely failed (converter issues), and thus was skipped - assertErrorHandlingMetricValue("total-records-skipped", 0.0); - - PowerMock.verifyAll(); - } - - private void assertSinkMetricValue(String name, double expected) { - ConnectMetrics.MetricGroup sinkTaskGroup = workerSinkTask.sinkTaskMetricsGroup().metricGroup(); - double measured = metrics.currentMetricValueAsDouble(sinkTaskGroup, name); - assertEquals(expected, measured, 0.001d); - } - - private void assertSourceMetricValue(String name, double expected) { - ConnectMetrics.MetricGroup sinkTaskGroup = workerSourceTask.sourceTaskMetricsGroup().metricGroup(); - double measured = metrics.currentMetricValueAsDouble(sinkTaskGroup, name); - assertEquals(expected, measured, 0.001d); - } - - private void assertErrorHandlingMetricValue(String name, double expected) { - ConnectMetrics.MetricGroup sinkTaskGroup = errorHandlingMetrics.metricGroup(); - double measured = metrics.currentMetricValueAsDouble(sinkTaskGroup, name); - assertEquals(expected, measured, 0.001d); - } - - private void expectInitializeTask() throws Exception { - consumer.subscribe(EasyMock.eq(singletonList(TOPIC)), EasyMock.capture(rebalanceListener)); - PowerMock.expectLastCall(); - - sinkTask.initialize(EasyMock.capture(sinkTaskContext)); - PowerMock.expectLastCall(); - sinkTask.start(TASK_PROPS); - PowerMock.expectLastCall(); - } - - private void expectTaskGetTopic(boolean anyTimes) { - final Capture connectorCapture = EasyMock.newCapture(); - final Capture topicCapture = EasyMock.newCapture(); - IExpectationSetters expect = EasyMock.expect(statusBackingStore.getTopic( - EasyMock.capture(connectorCapture), - EasyMock.capture(topicCapture))); - if (anyTimes) { - expect.andStubAnswer(() -> new TopicStatus( - topicCapture.getValue(), - new ConnectorTaskId(connectorCapture.getValue(), 0), - Time.SYSTEM.milliseconds())); - } else { - expect.andAnswer(() -> new TopicStatus( - topicCapture.getValue(), - new ConnectorTaskId(connectorCapture.getValue(), 0), - Time.SYSTEM.milliseconds())); - } - if (connectorCapture.hasCaptured() && topicCapture.hasCaptured()) { - assertEquals("job", connectorCapture.getValue()); - assertEquals(TOPIC, topicCapture.getValue()); - } - } - - private void expectClose() { - producer.close(EasyMock.anyObject(Duration.class)); - EasyMock.expectLastCall(); - - admin.close(EasyMock.anyObject(Duration.class)); - EasyMock.expectLastCall(); - } - - private void expectTopicDoesNotExist(String topic) { - if (workerConfig.topicCreationEnable()) { - EasyMock.expect(admin.describeTopics(topic)).andReturn(Collections.emptyMap()); - - Capture newTopicCapture = EasyMock.newCapture(); - Set created = Collections.singleton(topic); - Set existing = Collections.emptySet(); - TopicAdmin.TopicCreationResponse response = new TopicAdmin.TopicCreationResponse(created, existing); - EasyMock.expect(admin.createOrFindTopics(EasyMock.capture(newTopicCapture))).andReturn(response); - } - } - - private void createSinkTask(TargetState initialState, RetryWithToleranceOperator retryWithToleranceOperator) { - JsonConverter converter = new JsonConverter(); - Map oo = workerConfig.originalsWithPrefix("value.converter."); - oo.put("converter.type", "value"); - oo.put("schemas.enable", "false"); - converter.configure(oo); - - TransformationChain sinkTransforms = new TransformationChain<>(singletonList(new FaultyPassthrough()), retryWithToleranceOperator); - - workerSinkTask = new WorkerSinkTask( - taskId, sinkTask, statusListener, initialState, workerConfig, - ClusterConfigState.EMPTY, metrics, converter, converter, - headerConverter, sinkTransforms, consumer, pluginLoader, time, - retryWithToleranceOperator, workerErrantRecordReporter, statusBackingStore); - } - - private void createSourceTask(TargetState initialState, RetryWithToleranceOperator retryWithToleranceOperator) { - JsonConverter converter = new JsonConverter(); - Map oo = workerConfig.originalsWithPrefix("value.converter."); - oo.put("converter.type", "value"); - oo.put("schemas.enable", "false"); - converter.configure(oo); - - createSourceTask(initialState, retryWithToleranceOperator, converter); - } - - private Converter badConverter() { - FaultyConverter converter = new FaultyConverter(); - Map oo = workerConfig.originalsWithPrefix("value.converter."); - oo.put("converter.type", "value"); - oo.put("schemas.enable", "false"); - converter.configure(oo); - return converter; - } - - private void createSourceTask(TargetState initialState, RetryWithToleranceOperator retryWithToleranceOperator, Converter converter) { - TransformationChain sourceTransforms = new TransformationChain<>(singletonList(new FaultyPassthrough()), retryWithToleranceOperator); - - workerSourceTask = PowerMock.createPartialMock( - WorkerSourceTask.class, new String[]{"commitOffsets", "isStopping"}, - taskId, sourceTask, statusListener, initialState, converter, converter, headerConverter, sourceTransforms, - producer, admin, TopicCreationGroup.configuredGroups(sourceConfig), - offsetReader, offsetWriter, workerConfig, - ClusterConfigState.EMPTY, metrics, pluginLoader, time, retryWithToleranceOperator, - statusBackingStore, (Executor) Runnable::run); - } - - private ConsumerRecords records(ConsumerRecord record) { - return new ConsumerRecords<>(Collections.singletonMap( - new TopicPartition(record.topic(), record.partition()), singletonList(record))); - } - - private abstract static class TestSinkTask extends SinkTask { - } - - static class FaultyConverter extends JsonConverter { - private static final Logger log = LoggerFactory.getLogger(FaultyConverter.class); - private int invocations = 0; - - public byte[] fromConnectData(String topic, Schema schema, Object value) { - if (value == null) { - return super.fromConnectData(topic, schema, null); - } - invocations++; - if (invocations % 3 == 0) { - log.debug("Succeeding record: {} where invocations={}", value, invocations); - return super.fromConnectData(topic, schema, value); - } else { - log.debug("Failing record: {} at invocations={}", value, invocations); - throw new RetriableException("Bad invocations " + invocations + " for mod 3"); - } - } - } - - static class FaultyPassthrough> implements Transformation { - - private static final Logger log = LoggerFactory.getLogger(FaultyPassthrough.class); - - private static final String MOD_CONFIG = "mod"; - private static final int MOD_CONFIG_DEFAULT = 3; - - public static final ConfigDef CONFIG_DEF = new ConfigDef() - .define(MOD_CONFIG, ConfigDef.Type.INT, MOD_CONFIG_DEFAULT, ConfigDef.Importance.MEDIUM, "Pass records without failure only if timestamp % mod == 0"); - - private int mod = MOD_CONFIG_DEFAULT; - - private int invocations = 0; - - @Override - public R apply(R record) { - invocations++; - if (invocations % mod == 0) { - log.debug("Succeeding record: {} where invocations={}", record, invocations); - return record; - } else { - log.debug("Failing record: {} at invocations={}", record, invocations); - throw new RetriableException("Bad invocations " + invocations + " for mod " + mod); - } - } - - @Override - public ConfigDef config() { - return CONFIG_DEF; - } - - @Override - public void close() { - log.info("Shutting down transform"); - } - - @Override - public void configure(Map configs) { - final SimpleConfig config = new SimpleConfig(CONFIG_DEF, configs); - mod = Math.max(config.getInt(MOD_CONFIG), 2); - log.info("Configuring {}. Setting mod to {}", this.getClass(), mod); - } - } -} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java index 0f9e6562fb9..47098dcb0de 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java @@ -16,13 +16,18 @@ */ package org.apache.kafka.connect.runtime; +import java.util.Collection; import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.admin.TopicDescription; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.InvalidRecordException; +import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.TopicPartitionInfo; +import org.apache.kafka.common.errors.InvalidTopicException; import org.apache.kafka.common.errors.TopicAuthorizationException; import org.apache.kafka.common.header.Header; import org.apache.kafka.common.header.Headers; @@ -31,6 +36,7 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaAndValue; import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.errors.RetriableException; import org.apache.kafka.connect.header.ConnectHeaders; import org.apache.kafka.connect.integration.MonitorableSourceConnector; import org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroup; @@ -50,8 +56,10 @@ import org.apache.kafka.connect.storage.StatusBackingStore; import org.apache.kafka.connect.storage.StringConverter; import org.apache.kafka.connect.util.Callback; import org.apache.kafka.connect.util.ConnectorTaskId; +import org.apache.kafka.connect.util.ParameterizedTest; import org.apache.kafka.connect.util.ThreadedTest; import org.apache.kafka.connect.util.TopicAdmin; +import org.apache.kafka.connect.util.TopicCreationGroup; import org.easymock.Capture; import org.easymock.EasyMock; import org.easymock.IAnswer; @@ -64,6 +72,7 @@ import org.powermock.api.easymock.annotation.Mock; import org.powermock.api.easymock.annotation.MockStrict; import org.powermock.core.classloader.annotations.PowerMockIgnore; import org.powermock.modules.junit4.PowerMockRunner; +import org.powermock.modules.junit4.PowerMockRunnerDelegate; import org.powermock.reflect.Whitebox; import java.nio.ByteBuffer; @@ -74,6 +83,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -87,6 +97,12 @@ import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_C import static org.apache.kafka.connect.runtime.ConnectorConfig.KEY_CONVERTER_CLASS_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.TOPIC_CREATION_GROUPS_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.DEFAULT_TOPIC_CREATION_PREFIX; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.EXCLUDE_REGEX_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.INCLUDE_REGEX_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.PARTITIONS_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.REPLICATION_FACTOR_CONFIG; import static org.apache.kafka.connect.runtime.WorkerConfig.TOPIC_CREATION_ENABLE_CONFIG; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -97,8 +113,10 @@ import static org.junit.Assert.assertTrue; @PowerMockIgnore({"javax.management.*", "org.apache.log4j.*"}) @RunWith(PowerMockRunner.class) +@PowerMockRunnerDelegate(ParameterizedTest.class) public class WorkerSourceTaskTest extends ThreadedTest { private static final String TOPIC = "topic"; + private static final String OTHER_TOPIC = "other-topic"; private static final Map PARTITION = Collections.singletonMap("key", "partition".getBytes()); private static final Map OFFSET = Collections.singletonMap("key", 12); @@ -146,29 +164,42 @@ public class WorkerSourceTaskTest extends ThreadedTest { new SourceRecord(PARTITION, OFFSET, "topic", null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD) ); - // when this test becomes parameterized, this variable will be a test parameter - public boolean enableTopicCreation = false; + private boolean enableTopicCreation; + + @ParameterizedTest.Parameters + public static Collection parameters() { + return Arrays.asList(false, true); + } + + public WorkerSourceTaskTest(boolean enableTopicCreation) { + this.enableTopicCreation = enableTopicCreation; + } @Override public void setup() { super.setup(); - Map workerProps = new HashMap<>(); - workerProps.put("key.converter", "org.apache.kafka.connect.json.JsonConverter"); - workerProps.put("value.converter", "org.apache.kafka.connect.json.JsonConverter"); - workerProps.put("internal.key.converter", "org.apache.kafka.connect.json.JsonConverter"); - workerProps.put("internal.value.converter", "org.apache.kafka.connect.json.JsonConverter"); - workerProps.put("internal.key.converter.schemas.enable", "false"); - workerProps.put("internal.value.converter.schemas.enable", "false"); - workerProps.put("offset.storage.file.filename", "/tmp/connect.offsets"); - workerProps.put(TOPIC_CREATION_ENABLE_CONFIG, String.valueOf(enableTopicCreation)); + Map workerProps = workerProps(); plugins = new Plugins(workerProps); config = new StandaloneConfig(workerProps); - sourceConfig = new SourceConnectorConfig(plugins, sourceConnectorProps(TOPIC), true); + sourceConfig = new SourceConnectorConfig(plugins, sourceConnectorPropsWithGroups(TOPIC), true); producerCallbacks = EasyMock.newCapture(); metrics = new MockConnectMetrics(); } - private Map sourceConnectorProps(String topic) { + private Map workerProps() { + Map props = new HashMap<>(); + props.put("key.converter", "org.apache.kafka.connect.json.JsonConverter"); + props.put("value.converter", "org.apache.kafka.connect.json.JsonConverter"); + props.put("internal.key.converter", "org.apache.kafka.connect.json.JsonConverter"); + props.put("internal.value.converter", "org.apache.kafka.connect.json.JsonConverter"); + props.put("internal.key.converter.schemas.enable", "false"); + props.put("internal.value.converter.schemas.enable", "false"); + props.put("offset.storage.file.filename", "/tmp/connect.offsets"); + props.put(TOPIC_CREATION_ENABLE_CONFIG, String.valueOf(enableTopicCreation)); + return props; + } + + private Map sourceConnectorPropsWithGroups(String topic) { // setup up props for the source connector Map props = new HashMap<>(); props.put("name", "foo-connector"); @@ -177,6 +208,12 @@ public class WorkerSourceTaskTest extends ThreadedTest { 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(TOPIC_CREATION_GROUPS_CONFIG, String.join(",", "foo", "bar")); + props.put(DEFAULT_TOPIC_CREATION_PREFIX + REPLICATION_FACTOR_CONFIG, String.valueOf(1)); + props.put(DEFAULT_TOPIC_CREATION_PREFIX + PARTITIONS_CONFIG, String.valueOf(1)); + props.put(SourceConnectorConfig.TOPIC_CREATION_PREFIX + "foo" + "." + INCLUDE_REGEX_CONFIG, topic); + props.put(SourceConnectorConfig.TOPIC_CREATION_PREFIX + "bar" + "." + INCLUDE_REGEX_CONFIG, ".*"); + props.put(SourceConnectorConfig.TOPIC_CREATION_PREFIX + "bar" + "." + EXCLUDE_REGEX_CONFIG, topic); return props; } @@ -195,9 +232,9 @@ public class WorkerSourceTaskTest extends ThreadedTest { private void createWorkerTask(TargetState initialState, Converter keyConverter, Converter valueConverter, HeaderConverter headerConverter) { workerTask = new WorkerSourceTask(taskId, sourceTask, statusListener, initialState, keyConverter, valueConverter, headerConverter, - transformationChain, producer, admin, null, - offsetReader, offsetWriter, config, clusterConfigState, metrics, plugins.delegatingLoader(), Time.SYSTEM, - RetryWithToleranceOperatorTest.NOOP_OPERATOR, statusBackingStore, Runnable::run); + transformationChain, producer, admin, TopicCreationGroup.configuredGroups(sourceConfig), + offsetReader, offsetWriter, config, clusterConfigState, metrics, plugins.delegatingLoader(), Time.SYSTEM, + RetryWithToleranceOperatorTest.NOOP_OPERATOR, statusBackingStore, Runnable::run); } @Test @@ -631,6 +668,29 @@ public class WorkerSourceTaskTest extends ThreadedTest { assertThrows(ConnectException.class, () -> Whitebox.invokeMethod(workerTask, "sendRecords")); } + @Test + public void testSendRecordsProducerSendFailsImmediately() { + if (!enableTopicCreation) + // should only test with topic creation enabled + return; + + createWorkerTask(); + + SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + + expectPreliminaryCalls(); + expectTopicCreation(TOPIC); + + EasyMock.expect(producer.send(EasyMock.anyObject(), EasyMock.anyObject())) + .andThrow(new KafkaException("Producer closed while send in progress", new InvalidTopicException(TOPIC))); + + PowerMock.replayAll(); + + Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); + assertThrows(ConnectException.class, () -> Whitebox.invokeMethod(workerTask, "sendRecords")); + } + @Test public void testSendRecordsTaskCommitRecordFail() throws Exception { createWorkerTask(); @@ -777,11 +837,11 @@ public class WorkerSourceTaskTest extends ThreadedTest { createWorkerTask(); List records = new ArrayList<>(); - records.add(new SourceRecord(PARTITION, OFFSET, "topic", null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD, null, connectHeaders)); + records.add(new SourceRecord(PARTITION, OFFSET, TOPIC, null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD, null, connectHeaders)); expectTopicCreation(TOPIC); - Capture> sent = expectSendRecord(true, false, true, true, true, headers); + Capture> sent = expectSendRecord(TOPIC, true, false, true, true, true, headers); PowerMock.replayAll(); @@ -819,8 +879,8 @@ public class WorkerSourceTaskTest extends ThreadedTest { expectTopicCreation(TOPIC); - Capture> sentRecordA = expectSendRecord(false, false, true, true, false, null); - Capture> sentRecordB = expectSendRecord(false, false, true, true, false, null); + Capture> sentRecordA = expectSendRecord(TOPIC, false, false, true, true, false, null); + Capture> sentRecordB = expectSendRecord(TOPIC, false, false, true, true, false, null); PowerMock.replayAll(); @@ -844,6 +904,335 @@ public class WorkerSourceTaskTest extends ThreadedTest { PowerMock.verifyAll(); } + @Test + public void testTopicCreateWhenTopicExists() throws Exception { + if (!enableTopicCreation) + // should only test with topic creation enabled + return; + + createWorkerTask(); + + SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + + expectPreliminaryCalls(); + TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, null, Collections.emptyList(), Collections.emptyList()); + TopicDescription topicDesc = new TopicDescription(TOPIC, false, Collections.singletonList(topicPartitionInfo)); + EasyMock.expect(admin.describeTopics(TOPIC)).andReturn(Collections.singletonMap(TOPIC, topicDesc)); + + expectSendRecordTaskCommitRecordSucceed(false, false); + expectSendRecordTaskCommitRecordSucceed(false, false); + + PowerMock.replayAll(); + + Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); + Whitebox.invokeMethod(workerTask, "sendRecords"); + } + + @Test + public void testSendRecordsTopicDescribeRetries() throws Exception { + if (!enableTopicCreation) + // should only test with topic creation enabled + return; + + createWorkerTask(); + + SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + + expectPreliminaryCalls(); + // First round - call to describe the topic times out + EasyMock.expect(admin.describeTopics(TOPIC)) + .andThrow(new RetriableException(new TimeoutException("timeout"))); + + // Second round - calls to describe and create succeed + expectTopicCreation(TOPIC); + // Exactly two records are sent + expectSendRecordTaskCommitRecordSucceed(false, false); + expectSendRecordTaskCommitRecordSucceed(false, false); + + PowerMock.replayAll(); + + Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); + Whitebox.invokeMethod(workerTask, "sendRecords"); + assertEquals(true, Whitebox.getInternalState(workerTask, "lastSendFailed")); + assertEquals(Arrays.asList(record1, record2), Whitebox.getInternalState(workerTask, "toSend")); + + // Next they all succeed + Whitebox.invokeMethod(workerTask, "sendRecords"); + assertEquals(false, Whitebox.getInternalState(workerTask, "lastSendFailed")); + assertNull(Whitebox.getInternalState(workerTask, "toSend")); + } + + @Test + public void testSendRecordsTopicCreateRetries() throws Exception { + if (!enableTopicCreation) + // should only test with topic creation enabled + return; + + createWorkerTask(); + + SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + + // First call to describe the topic times out + expectPreliminaryCalls(); + EasyMock.expect(admin.describeTopics(TOPIC)).andReturn(Collections.emptyMap()); + Capture newTopicCapture = EasyMock.newCapture(); + EasyMock.expect(admin.createOrFindTopics(EasyMock.capture(newTopicCapture))) + .andThrow(new RetriableException(new TimeoutException("timeout"))); + + // Second round + expectTopicCreation(TOPIC); + expectSendRecordTaskCommitRecordSucceed(false, false); + expectSendRecordTaskCommitRecordSucceed(false, false); + + PowerMock.replayAll(); + + Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); + Whitebox.invokeMethod(workerTask, "sendRecords"); + assertEquals(true, Whitebox.getInternalState(workerTask, "lastSendFailed")); + assertEquals(Arrays.asList(record1, record2), Whitebox.getInternalState(workerTask, "toSend")); + + // Next they all succeed + Whitebox.invokeMethod(workerTask, "sendRecords"); + assertEquals(false, Whitebox.getInternalState(workerTask, "lastSendFailed")); + assertNull(Whitebox.getInternalState(workerTask, "toSend")); + } + + @Test + public void testSendRecordsTopicDescribeRetriesMidway() throws Exception { + if (!enableTopicCreation) + // should only test with topic creation enabled + return; + + createWorkerTask(); + + // Differentiate only by Kafka partition so we can reuse conversion expectations + SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record3 = new SourceRecord(PARTITION, OFFSET, OTHER_TOPIC, 3, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + + // First round + expectPreliminaryCalls(OTHER_TOPIC); + expectTopicCreation(TOPIC); + expectSendRecordTaskCommitRecordSucceed(false, false); + expectSendRecordTaskCommitRecordSucceed(false, false); + + // First call to describe the topic times out + EasyMock.expect(admin.describeTopics(OTHER_TOPIC)) + .andThrow(new RetriableException(new TimeoutException("timeout"))); + + // Second round + expectTopicCreation(OTHER_TOPIC); + expectSendRecord(OTHER_TOPIC, false, true, true, true, true, emptyHeaders()); + + PowerMock.replayAll(); + + // Try to send 3, make first pass, second fail. Should save last two + Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2, record3)); + Whitebox.invokeMethod(workerTask, "sendRecords"); + assertEquals(true, Whitebox.getInternalState(workerTask, "lastSendFailed")); + assertEquals(Arrays.asList(record3), Whitebox.getInternalState(workerTask, "toSend")); + + // Next they all succeed + Whitebox.invokeMethod(workerTask, "sendRecords"); + assertEquals(false, Whitebox.getInternalState(workerTask, "lastSendFailed")); + assertNull(Whitebox.getInternalState(workerTask, "toSend")); + + PowerMock.verifyAll(); + } + + @Test + public void testSendRecordsTopicCreateRetriesMidway() throws Exception { + if (!enableTopicCreation) + // should only test with topic creation enabled + return; + + createWorkerTask(); + + // Differentiate only by Kafka partition so we can reuse conversion expectations + SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record3 = new SourceRecord(PARTITION, OFFSET, OTHER_TOPIC, 3, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + + // First round + expectPreliminaryCalls(OTHER_TOPIC); + expectTopicCreation(TOPIC); + expectSendRecordTaskCommitRecordSucceed(false, false); + expectSendRecordTaskCommitRecordSucceed(false, false); + + EasyMock.expect(admin.describeTopics(OTHER_TOPIC)).andReturn(Collections.emptyMap()); + // First call to create the topic times out + Capture newTopicCapture = EasyMock.newCapture(); + EasyMock.expect(admin.createOrFindTopics(EasyMock.capture(newTopicCapture))) + .andThrow(new RetriableException(new TimeoutException("timeout"))); + + // Second round + expectTopicCreation(OTHER_TOPIC); + expectSendRecord(OTHER_TOPIC, false, true, true, true, true, emptyHeaders()); + + PowerMock.replayAll(); + + // Try to send 3, make first pass, second fail. Should save last two + Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2, record3)); + Whitebox.invokeMethod(workerTask, "sendRecords"); + assertEquals(true, Whitebox.getInternalState(workerTask, "lastSendFailed")); + assertEquals(Arrays.asList(record3), Whitebox.getInternalState(workerTask, "toSend")); + + // Next they all succeed + Whitebox.invokeMethod(workerTask, "sendRecords"); + assertEquals(false, Whitebox.getInternalState(workerTask, "lastSendFailed")); + assertNull(Whitebox.getInternalState(workerTask, "toSend")); + + PowerMock.verifyAll(); + } + + @Test + public void testTopicDescribeFails() { + if (!enableTopicCreation) + // should only test with topic creation enabled + return; + + createWorkerTask(); + + SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + + expectPreliminaryCalls(); + EasyMock.expect(admin.describeTopics(TOPIC)) + .andThrow(new ConnectException(new TopicAuthorizationException("unauthorized"))); + + PowerMock.replayAll(); + + Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); + assertThrows(ConnectException.class, () -> Whitebox.invokeMethod(workerTask, "sendRecords")); + } + + @Test + public void testTopicCreateFails() throws Exception { + if (!enableTopicCreation) + // should only test with topic creation enabled + return; + + createWorkerTask(); + + SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + + expectPreliminaryCalls(); + EasyMock.expect(admin.describeTopics(TOPIC)).andReturn(Collections.emptyMap()); + + Capture newTopicCapture = EasyMock.newCapture(); + EasyMock.expect(admin.createOrFindTopics(EasyMock.capture(newTopicCapture))) + .andThrow(new ConnectException(new TopicAuthorizationException("unauthorized"))); + + PowerMock.replayAll(); + + Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); + assertThrows(ConnectException.class, () -> Whitebox.invokeMethod(workerTask, "sendRecords")); + assertTrue(newTopicCapture.hasCaptured()); + } + + @Test + public void testTopicCreateFailsWithExceptionWhenCreateReturnsTopicNotCreatedOrFound() throws Exception { + if (!enableTopicCreation) + // should only test with topic creation enabled + return; + + createWorkerTask(); + + SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + + expectPreliminaryCalls(); + EasyMock.expect(admin.describeTopics(TOPIC)).andReturn(Collections.emptyMap()); + + Capture newTopicCapture = EasyMock.newCapture(); + EasyMock.expect(admin.createOrFindTopics(EasyMock.capture(newTopicCapture))).andReturn(TopicAdmin.EMPTY_CREATION); + + PowerMock.replayAll(); + + Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); + assertThrows(ConnectException.class, () -> Whitebox.invokeMethod(workerTask, "sendRecords")); + assertTrue(newTopicCapture.hasCaptured()); + } + + @Test + public void testTopicCreateSucceedsWhenCreateReturnsExistingTopicFound() throws Exception { + if (!enableTopicCreation) + // should only test with topic creation enabled + return; + + createWorkerTask(); + + SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + + expectPreliminaryCalls(); + EasyMock.expect(admin.describeTopics(TOPIC)).andReturn(Collections.emptyMap()); + + Capture newTopicCapture = EasyMock.newCapture(); + EasyMock.expect(admin.createOrFindTopics(EasyMock.capture(newTopicCapture))).andReturn(foundTopic(TOPIC)); + + expectSendRecordTaskCommitRecordSucceed(false, false); + expectSendRecordTaskCommitRecordSucceed(false, false); + + PowerMock.replayAll(); + + Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); + Whitebox.invokeMethod(workerTask, "sendRecords"); + } + + @Test + public void testTopicCreateSucceedsWhenCreateReturnsNewTopicFound() throws Exception { + if (!enableTopicCreation) + // should only test with topic creation enabled + return; + + createWorkerTask(); + + SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + + expectPreliminaryCalls(); + EasyMock.expect(admin.describeTopics(TOPIC)).andReturn(Collections.emptyMap()); + + Capture newTopicCapture = EasyMock.newCapture(); + EasyMock.expect(admin.createOrFindTopics(EasyMock.capture(newTopicCapture))).andReturn(createdTopic(TOPIC)); + + expectSendRecordTaskCommitRecordSucceed(false, false); + expectSendRecordTaskCommitRecordSucceed(false, false); + + PowerMock.replayAll(); + + Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); + Whitebox.invokeMethod(workerTask, "sendRecords"); + } + + private TopicAdmin.TopicCreationResponse createdTopic(String topic) { + Set created = Collections.singleton(topic); + Set existing = Collections.emptySet(); + return new TopicAdmin.TopicCreationResponse(created, existing); + } + + private TopicAdmin.TopicCreationResponse foundTopic(String topic) { + Set created = Collections.emptySet(); + Set existing = Collections.singleton(topic); + return new TopicAdmin.TopicCreationResponse(created, existing); + } + + private void expectPreliminaryCalls() { + expectPreliminaryCalls(TOPIC); + } + + private void expectPreliminaryCalls(String topic) { + expectConvertHeadersAndKeyValue(topic, true, emptyHeaders()); + expectApplyTransformationChain(false); + offsetWriter.offset(PARTITION, OFFSET); + PowerMock.expectLastCall(); + } + private CountDownLatch expectEmptyPolls(int minimum, final AtomicInteger count) throws InterruptedException { final CountDownLatch latch = new CountDownLatch(minimum); // Note that we stub these to allow any number of calls because the thread will continue to @@ -889,9 +1278,9 @@ public class WorkerSourceTaskTest extends ThreadedTest { PowerMock.expectLastCall(); EasyMock.expect( - producer.send(EasyMock.anyObject(ProducerRecord.class), - EasyMock.anyObject(org.apache.kafka.clients.producer.Callback.class))) - .andThrow(error); + producer.send(EasyMock.anyObject(ProducerRecord.class), + EasyMock.anyObject(org.apache.kafka.clients.producer.Callback.class))) + .andThrow(error); } private Capture> expectSendRecordAnyTimes() throws InterruptedException { @@ -903,22 +1292,23 @@ public class WorkerSourceTaskTest extends ThreadedTest { } private Capture> expectSendRecordProducerCallbackFail() throws InterruptedException { - return expectSendRecord(false, false, false, false, true, emptyHeaders()); + return expectSendRecord(TOPIC, false, false, false, false, true, emptyHeaders()); } private Capture> expectSendRecordTaskCommitRecordSucceed(boolean anyTimes, boolean isRetry) throws InterruptedException { - return expectSendRecord(anyTimes, isRetry, true, true, true, emptyHeaders()); + return expectSendRecord(TOPIC, anyTimes, isRetry, true, true, true, emptyHeaders()); } private Capture> expectSendRecordTaskCommitRecordFail(boolean anyTimes, boolean isRetry) throws InterruptedException { - return expectSendRecord(anyTimes, isRetry, true, false, true, emptyHeaders()); + return expectSendRecord(TOPIC, anyTimes, isRetry, true, false, true, emptyHeaders()); } private Capture> expectSendRecord(boolean anyTimes, boolean isRetry, boolean succeed) throws InterruptedException { - return expectSendRecord(anyTimes, isRetry, succeed, true, true, emptyHeaders()); + return expectSendRecord(TOPIC, anyTimes, isRetry, succeed, true, true, emptyHeaders()); } private Capture> expectSendRecord( + String topic, boolean anyTimes, boolean isRetry, boolean sendSuccess, @@ -927,7 +1317,7 @@ public class WorkerSourceTaskTest extends ThreadedTest { Headers headers ) throws InterruptedException { if (isMockedConverters) { - expectConvertHeadersAndKeyValue(anyTimes, headers); + expectConvertHeadersAndKeyValue(topic, anyTimes, headers); } expectApplyTransformationChain(anyTimes); @@ -976,23 +1366,23 @@ public class WorkerSourceTaskTest extends ThreadedTest { } private void expectConvertHeadersAndKeyValue(boolean anyTimes) { - expectConvertHeadersAndKeyValue(anyTimes, emptyHeaders()); + expectConvertHeadersAndKeyValue(TOPIC, anyTimes, emptyHeaders()); } - private void expectConvertHeadersAndKeyValue(boolean anyTimes, Headers headers) { + private void expectConvertHeadersAndKeyValue(String topic, boolean anyTimes, Headers headers) { for (Header header : headers) { - IExpectationSetters convertHeaderExpect = EasyMock.expect(headerConverter.fromConnectHeader(TOPIC, header.key(), Schema.STRING_SCHEMA, new String(header.value()))); + IExpectationSetters convertHeaderExpect = EasyMock.expect(headerConverter.fromConnectHeader(topic, header.key(), Schema.STRING_SCHEMA, new String(header.value()))); if (anyTimes) convertHeaderExpect.andStubReturn(header.value()); else convertHeaderExpect.andReturn(header.value()); } - IExpectationSetters convertKeyExpect = EasyMock.expect(keyConverter.fromConnectData(TOPIC, headers, KEY_SCHEMA, KEY)); + IExpectationSetters convertKeyExpect = EasyMock.expect(keyConverter.fromConnectData(topic, headers, KEY_SCHEMA, KEY)); if (anyTimes) convertKeyExpect.andStubReturn(SERIALIZED_KEY); else convertKeyExpect.andReturn(SERIALIZED_KEY); - IExpectationSetters convertValueExpect = EasyMock.expect(valueConverter.fromConnectData(TOPIC, headers, RECORD_SCHEMA, RECORD)); + IExpectationSetters convertValueExpect = EasyMock.expect(valueConverter.fromConnectData(topic, headers, RECORD_SCHEMA, RECORD)); if (anyTimes) convertValueExpect.andStubReturn(SERIALIZED_RECORD); else @@ -1128,9 +1518,8 @@ public class WorkerSourceTaskTest extends ThreadedTest { private void expectTopicCreation(String topic) { if (config.topicCreationEnable()) { EasyMock.expect(admin.describeTopics(topic)).andReturn(Collections.emptyMap()); - Capture newTopicCapture = EasyMock.newCapture(); - EasyMock.expect(admin.createTopic(EasyMock.capture(newTopicCapture))).andReturn(true); + EasyMock.expect(admin.createOrFindTopics(EasyMock.capture(newTopicCapture))).andReturn(createdTopic(topic)); } } } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskWithTopicCreationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskWithTopicCreationTest.java deleted file mode 100644 index d26faa40707..00000000000 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskWithTopicCreationTest.java +++ /dev/null @@ -1,1469 +0,0 @@ -/* - * 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.runtime; - -import org.apache.kafka.clients.admin.NewTopic; -import org.apache.kafka.clients.admin.TopicDescription; -import org.apache.kafka.clients.producer.KafkaProducer; -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.clients.producer.RecordMetadata; -import org.apache.kafka.common.InvalidRecordException; -import org.apache.kafka.common.KafkaException; -import org.apache.kafka.common.MetricName; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.TopicPartitionInfo; -import org.apache.kafka.common.errors.InvalidTopicException; -import org.apache.kafka.common.errors.TopicAuthorizationException; -import org.apache.kafka.common.header.Header; -import org.apache.kafka.common.header.Headers; -import org.apache.kafka.common.header.internals.RecordHeaders; -import org.apache.kafka.common.utils.Time; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.SchemaAndValue; -import org.apache.kafka.connect.errors.ConnectException; -import org.apache.kafka.connect.errors.RetriableException; -import org.apache.kafka.connect.header.ConnectHeaders; -import org.apache.kafka.connect.integration.MonitorableSourceConnector; -import org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroup; -import org.apache.kafka.connect.runtime.WorkerSourceTask.SourceTaskMetricsGroup; -import org.apache.kafka.connect.runtime.distributed.ClusterConfigState; -import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperatorTest; -import org.apache.kafka.connect.runtime.isolation.Plugins; -import org.apache.kafka.connect.runtime.standalone.StandaloneConfig; -import org.apache.kafka.connect.source.SourceRecord; -import org.apache.kafka.connect.source.SourceTask; -import org.apache.kafka.connect.source.SourceTaskContext; -import org.apache.kafka.connect.storage.CloseableOffsetStorageReader; -import org.apache.kafka.connect.storage.Converter; -import org.apache.kafka.connect.storage.HeaderConverter; -import org.apache.kafka.connect.storage.OffsetStorageWriter; -import org.apache.kafka.connect.storage.StatusBackingStore; -import org.apache.kafka.connect.storage.StringConverter; -import org.apache.kafka.connect.util.Callback; -import org.apache.kafka.connect.util.ConnectorTaskId; -import org.apache.kafka.connect.util.ThreadedTest; -import org.apache.kafka.connect.util.TopicAdmin; -import org.apache.kafka.connect.util.TopicCreationGroup; -import org.easymock.Capture; -import org.easymock.EasyMock; -import org.easymock.IAnswer; -import org.easymock.IExpectationSetters; -import org.junit.After; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.powermock.api.easymock.PowerMock; -import org.powermock.api.easymock.annotation.Mock; -import org.powermock.api.easymock.annotation.MockStrict; -import org.powermock.core.classloader.annotations.PowerMockIgnore; -import org.powermock.modules.junit4.PowerMockRunner; -import org.powermock.reflect.Whitebox; - -import java.nio.ByteBuffer; -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.Set; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -import java.util.concurrent.atomic.AtomicInteger; - -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.KEY_CONVERTER_CLASS_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.TOPIC_CREATION_GROUPS_CONFIG; -import static org.apache.kafka.connect.runtime.TopicCreationConfig.DEFAULT_TOPIC_CREATION_PREFIX; -import static org.apache.kafka.connect.runtime.TopicCreationConfig.EXCLUDE_REGEX_CONFIG; -import static org.apache.kafka.connect.runtime.TopicCreationConfig.INCLUDE_REGEX_CONFIG; -import static org.apache.kafka.connect.runtime.TopicCreationConfig.PARTITIONS_CONFIG; -import static org.apache.kafka.connect.runtime.TopicCreationConfig.REPLICATION_FACTOR_CONFIG; -import static org.apache.kafka.connect.runtime.WorkerConfig.TOPIC_CREATION_ENABLE_CONFIG; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertThrows; -import static org.junit.Assert.assertTrue; - -@PowerMockIgnore({"javax.management.*", - "org.apache.log4j.*"}) -@RunWith(PowerMockRunner.class) -public class WorkerSourceTaskWithTopicCreationTest extends ThreadedTest { - private static final String TOPIC = "topic"; - private static final String OTHER_TOPIC = "other-topic"; - private static final Map PARTITION = Collections.singletonMap("key", "partition".getBytes()); - private static final Map OFFSET = Collections.singletonMap("key", 12); - - // Connect-format data - private static final Schema KEY_SCHEMA = Schema.INT32_SCHEMA; - private static final Integer KEY = -1; - private static final Schema RECORD_SCHEMA = Schema.INT64_SCHEMA; - private static final Long RECORD = 12L; - // Serialized data. The actual format of this data doesn't matter -- we just want to see that the right version - // is used in the right place. - private static final byte[] SERIALIZED_KEY = "converted-key".getBytes(); - private static final byte[] SERIALIZED_RECORD = "converted-record".getBytes(); - - private ExecutorService executor = Executors.newSingleThreadExecutor(); - private ConnectorTaskId taskId = new ConnectorTaskId("job", 0); - private ConnectorTaskId taskId1 = new ConnectorTaskId("job", 1); - private WorkerConfig config; - private SourceConnectorConfig sourceConfig; - private Plugins plugins; - private MockConnectMetrics metrics; - @Mock private SourceTask sourceTask; - @Mock private Converter keyConverter; - @Mock private Converter valueConverter; - @Mock private HeaderConverter headerConverter; - @Mock private TransformationChain transformationChain; - @Mock private KafkaProducer producer; - @Mock private TopicAdmin admin; - @Mock private CloseableOffsetStorageReader offsetReader; - @Mock private OffsetStorageWriter offsetWriter; - @Mock private ClusterConfigState clusterConfigState; - private WorkerSourceTask workerTask; - @Mock private Future sendFuture; - @MockStrict private TaskStatus.Listener statusListener; - @Mock private StatusBackingStore statusBackingStore; - - private Capture producerCallbacks; - - private static final Map TASK_PROPS = new HashMap<>(); - static { - TASK_PROPS.put(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName()); - } - private static final TaskConfig TASK_CONFIG = new TaskConfig(TASK_PROPS); - - private static final List RECORDS = Arrays.asList( - new SourceRecord(PARTITION, OFFSET, TOPIC, null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD) - ); - - // when this test becomes parameterized, this variable will be a test parameter - public boolean enableTopicCreation = true; - - @Override - public void setup() { - super.setup(); - Map workerProps = workerProps(); - plugins = new Plugins(workerProps); - config = new StandaloneConfig(workerProps); - sourceConfig = new SourceConnectorConfig(plugins, sourceConnectorPropsWithGroups(TOPIC), true); - producerCallbacks = EasyMock.newCapture(); - metrics = new MockConnectMetrics(); - } - - private Map workerProps() { - Map props = new HashMap<>(); - props.put("key.converter", "org.apache.kafka.connect.json.JsonConverter"); - props.put("value.converter", "org.apache.kafka.connect.json.JsonConverter"); - props.put("internal.key.converter", "org.apache.kafka.connect.json.JsonConverter"); - props.put("internal.value.converter", "org.apache.kafka.connect.json.JsonConverter"); - props.put("internal.key.converter.schemas.enable", "false"); - props.put("internal.value.converter.schemas.enable", "false"); - props.put("offset.storage.file.filename", "/tmp/connect.offsets"); - props.put(TOPIC_CREATION_ENABLE_CONFIG, String.valueOf(enableTopicCreation)); - return props; - } - - private Map sourceConnectorPropsWithGroups(String topic) { - // setup up props for the source connector - Map props = new HashMap<>(); - props.put("name", "foo-connector"); - props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getSimpleName()); - props.put(TASKS_MAX_CONFIG, String.valueOf(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(TOPIC_CREATION_GROUPS_CONFIG, String.join(",", "foo", "bar")); - props.put(DEFAULT_TOPIC_CREATION_PREFIX + REPLICATION_FACTOR_CONFIG, String.valueOf(1)); - props.put(DEFAULT_TOPIC_CREATION_PREFIX + PARTITIONS_CONFIG, String.valueOf(1)); - props.put(SourceConnectorConfig.TOPIC_CREATION_PREFIX + "foo" + "." + INCLUDE_REGEX_CONFIG, topic); - props.put(SourceConnectorConfig.TOPIC_CREATION_PREFIX + "bar" + "." + INCLUDE_REGEX_CONFIG, ".*"); - props.put(SourceConnectorConfig.TOPIC_CREATION_PREFIX + "bar" + "." + EXCLUDE_REGEX_CONFIG, topic); - return props; - } - - @After - public void tearDown() { - if (metrics != null) metrics.stop(); - } - - private void createWorkerTask() { - createWorkerTask(TargetState.STARTED); - } - - private void createWorkerTask(TargetState initialState) { - createWorkerTask(initialState, keyConverter, valueConverter, headerConverter); - } - - private void createWorkerTask(TargetState initialState, Converter keyConverter, Converter valueConverter, HeaderConverter headerConverter) { - workerTask = new WorkerSourceTask(taskId, sourceTask, statusListener, initialState, keyConverter, valueConverter, headerConverter, - transformationChain, producer, admin, TopicCreationGroup.configuredGroups(sourceConfig), - offsetReader, offsetWriter, config, clusterConfigState, metrics, plugins.delegatingLoader(), Time.SYSTEM, - RetryWithToleranceOperatorTest.NOOP_OPERATOR, statusBackingStore, Runnable::run); - } - - @Test - public void testStartPaused() throws Exception { - final CountDownLatch pauseLatch = new CountDownLatch(1); - - createWorkerTask(TargetState.PAUSED); - - statusListener.onPause(taskId); - EasyMock.expectLastCall().andAnswer(() -> { - pauseLatch.countDown(); - return null; - }); - - expectClose(); - - statusListener.onShutdown(taskId); - EasyMock.expectLastCall(); - - PowerMock.replayAll(); - - workerTask.initialize(TASK_CONFIG); - Future taskFuture = executor.submit(workerTask); - - assertTrue(pauseLatch.await(5, TimeUnit.SECONDS)); - workerTask.stop(); - assertTrue(workerTask.awaitStop(1000)); - - taskFuture.get(); - - PowerMock.verifyAll(); - } - - @Test - public void testPause() throws Exception { - createWorkerTask(); - - sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class)); - EasyMock.expectLastCall(); - sourceTask.start(TASK_PROPS); - EasyMock.expectLastCall(); - statusListener.onStartup(taskId); - EasyMock.expectLastCall(); - - AtomicInteger count = new AtomicInteger(0); - CountDownLatch pollLatch = expectPolls(10, count); - // In this test, we don't flush, so nothing goes any further than the offset writer - - expectTopicCreation(TOPIC); - - statusListener.onPause(taskId); - EasyMock.expectLastCall(); - - sourceTask.stop(); - EasyMock.expectLastCall(); - expectOffsetFlush(true); - - statusListener.onShutdown(taskId); - EasyMock.expectLastCall(); - - expectClose(); - - PowerMock.replayAll(); - - workerTask.initialize(TASK_CONFIG); - Future taskFuture = executor.submit(workerTask); - assertTrue(awaitLatch(pollLatch)); - - workerTask.transitionTo(TargetState.PAUSED); - - int priorCount = count.get(); - Thread.sleep(100); - - // since the transition is observed asynchronously, the count could be off by one loop iteration - assertTrue(count.get() - priorCount <= 1); - - workerTask.stop(); - assertTrue(workerTask.awaitStop(1000)); - - taskFuture.get(); - - PowerMock.verifyAll(); - } - - @Test - public void testPollsInBackground() throws Exception { - createWorkerTask(); - - sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class)); - EasyMock.expectLastCall(); - sourceTask.start(TASK_PROPS); - EasyMock.expectLastCall(); - statusListener.onStartup(taskId); - EasyMock.expectLastCall(); - - final CountDownLatch pollLatch = expectPolls(10); - // In this test, we don't flush, so nothing goes any further than the offset writer - - expectTopicCreation(TOPIC); - - sourceTask.stop(); - EasyMock.expectLastCall(); - expectOffsetFlush(true); - - statusListener.onShutdown(taskId); - EasyMock.expectLastCall(); - - expectClose(); - - PowerMock.replayAll(); - - workerTask.initialize(TASK_CONFIG); - Future taskFuture = executor.submit(workerTask); - - assertTrue(awaitLatch(pollLatch)); - workerTask.stop(); - assertTrue(workerTask.awaitStop(1000)); - - taskFuture.get(); - assertPollMetrics(10); - - PowerMock.verifyAll(); - } - - @Test - public void testFailureInPoll() throws Exception { - createWorkerTask(); - - sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class)); - EasyMock.expectLastCall(); - sourceTask.start(TASK_PROPS); - EasyMock.expectLastCall(); - statusListener.onStartup(taskId); - EasyMock.expectLastCall(); - - final CountDownLatch pollLatch = new CountDownLatch(1); - final RuntimeException exception = new RuntimeException(); - EasyMock.expect(sourceTask.poll()).andAnswer(() -> { - pollLatch.countDown(); - throw exception; - }); - - statusListener.onFailure(taskId, exception); - EasyMock.expectLastCall(); - - sourceTask.stop(); - EasyMock.expectLastCall(); - expectOffsetFlush(true); - - expectClose(); - - PowerMock.replayAll(); - - workerTask.initialize(TASK_CONFIG); - Future taskFuture = executor.submit(workerTask); - - assertTrue(awaitLatch(pollLatch)); - workerTask.stop(); - assertTrue(workerTask.awaitStop(1000)); - - taskFuture.get(); - assertPollMetrics(0); - - PowerMock.verifyAll(); - } - - @Test - public void testPollReturnsNoRecords() throws Exception { - // Test that the task handles an empty list of records - createWorkerTask(); - - sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class)); - EasyMock.expectLastCall(); - sourceTask.start(TASK_PROPS); - EasyMock.expectLastCall(); - statusListener.onStartup(taskId); - EasyMock.expectLastCall(); - - // We'll wait for some data, then trigger a flush - final CountDownLatch pollLatch = expectEmptyPolls(1, new AtomicInteger()); - expectOffsetFlush(true); - - sourceTask.stop(); - EasyMock.expectLastCall(); - expectOffsetFlush(true); - - statusListener.onShutdown(taskId); - EasyMock.expectLastCall(); - - expectClose(); - - PowerMock.replayAll(); - - workerTask.initialize(TASK_CONFIG); - Future taskFuture = executor.submit(workerTask); - - assertTrue(awaitLatch(pollLatch)); - assertTrue(workerTask.commitOffsets()); - workerTask.stop(); - assertTrue(workerTask.awaitStop(1000)); - - taskFuture.get(); - assertPollMetrics(0); - - PowerMock.verifyAll(); - } - - @Test - public void testCommit() throws Exception { - // Test that the task commits properly when prompted - createWorkerTask(); - - sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class)); - EasyMock.expectLastCall(); - sourceTask.start(TASK_PROPS); - EasyMock.expectLastCall(); - statusListener.onStartup(taskId); - EasyMock.expectLastCall(); - - // We'll wait for some data, then trigger a flush - final CountDownLatch pollLatch = expectPolls(1); - expectOffsetFlush(true); - - expectTopicCreation(TOPIC); - - sourceTask.stop(); - EasyMock.expectLastCall(); - expectOffsetFlush(true); - - statusListener.onShutdown(taskId); - EasyMock.expectLastCall(); - - expectClose(); - - PowerMock.replayAll(); - - workerTask.initialize(TASK_CONFIG); - Future taskFuture = executor.submit(workerTask); - - assertTrue(awaitLatch(pollLatch)); - assertTrue(workerTask.commitOffsets()); - workerTask.stop(); - assertTrue(workerTask.awaitStop(1000)); - - taskFuture.get(); - assertPollMetrics(1); - - PowerMock.verifyAll(); - } - - @Test - public void testCommitFailure() throws Exception { - // Test that the task commits properly when prompted - createWorkerTask(); - - sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class)); - EasyMock.expectLastCall(); - sourceTask.start(TASK_PROPS); - EasyMock.expectLastCall(); - statusListener.onStartup(taskId); - EasyMock.expectLastCall(); - - // We'll wait for some data, then trigger a flush - final CountDownLatch pollLatch = expectPolls(1); - expectOffsetFlush(true); - - expectTopicCreation(TOPIC); - - sourceTask.stop(); - EasyMock.expectLastCall(); - expectOffsetFlush(false); - - statusListener.onShutdown(taskId); - EasyMock.expectLastCall(); - - expectClose(); - - PowerMock.replayAll(); - - workerTask.initialize(TASK_CONFIG); - Future taskFuture = executor.submit(workerTask); - - assertTrue(awaitLatch(pollLatch)); - assertTrue(workerTask.commitOffsets()); - workerTask.stop(); - assertTrue(workerTask.awaitStop(1000)); - - taskFuture.get(); - assertPollMetrics(1); - - PowerMock.verifyAll(); - } - - @Test - public void testSendRecordsConvertsData() throws Exception { - createWorkerTask(); - - List records = new ArrayList<>(); - // Can just use the same record for key and value - records.add(new SourceRecord(PARTITION, OFFSET, TOPIC, null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD)); - - Capture> sent = expectSendRecordAnyTimes(); - - expectTopicCreation(TOPIC); - - PowerMock.replayAll(); - - Whitebox.setInternalState(workerTask, "toSend", records); - Whitebox.invokeMethod(workerTask, "sendRecords"); - assertEquals(SERIALIZED_KEY, sent.getValue().key()); - assertEquals(SERIALIZED_RECORD, sent.getValue().value()); - - PowerMock.verifyAll(); - } - - @Test - public void testSendRecordsPropagatesTimestamp() throws Exception { - final Long timestamp = System.currentTimeMillis(); - - createWorkerTask(); - - List records = Collections.singletonList( - new SourceRecord(PARTITION, OFFSET, TOPIC, null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD, timestamp) - ); - - Capture> sent = expectSendRecordAnyTimes(); - - expectTopicCreation(TOPIC); - - PowerMock.replayAll(); - - Whitebox.setInternalState(workerTask, "toSend", records); - Whitebox.invokeMethod(workerTask, "sendRecords"); - assertEquals(timestamp, sent.getValue().timestamp()); - - PowerMock.verifyAll(); - } - - @Test - public void testSendRecordsCorruptTimestamp() throws Exception { - final Long timestamp = -3L; - createWorkerTask(); - - List records = Collections.singletonList( - new SourceRecord(PARTITION, OFFSET, TOPIC, null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD, timestamp) - ); - - Capture> sent = expectSendRecordAnyTimes(); - - PowerMock.replayAll(); - - Whitebox.setInternalState(workerTask, "toSend", records); - assertThrows(InvalidRecordException.class, () -> Whitebox.invokeMethod(workerTask, "sendRecords")); - assertFalse(sent.hasCaptured()); - - PowerMock.verifyAll(); - } - - @Test - public void testSendRecordsNoTimestamp() throws Exception { - final Long timestamp = -1L; - createWorkerTask(); - - List records = Collections.singletonList( - new SourceRecord(PARTITION, OFFSET, TOPIC, null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD, timestamp) - ); - - Capture> sent = expectSendRecordAnyTimes(); - - expectTopicCreation(TOPIC); - - PowerMock.replayAll(); - - Whitebox.setInternalState(workerTask, "toSend", records); - Whitebox.invokeMethod(workerTask, "sendRecords"); - assertNull(sent.getValue().timestamp()); - - PowerMock.verifyAll(); - } - - @Test - public void testSendRecordsRetries() throws Exception { - createWorkerTask(); - - // Differentiate only by Kafka partition so we can reuse conversion expectations - SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - SourceRecord record3 = new SourceRecord(PARTITION, OFFSET, TOPIC, 3, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - - expectTopicCreation(TOPIC); - - // First round - expectSendRecordOnce(false); - // Any Producer retriable exception should work here - expectSendRecordSyncFailure(new org.apache.kafka.common.errors.TimeoutException("retriable sync failure")); - - // Second round - expectSendRecordOnce(true); - expectSendRecordOnce(false); - - PowerMock.replayAll(); - - // Try to send 3, make first pass, second fail. Should save last two - Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2, record3)); - Whitebox.invokeMethod(workerTask, "sendRecords"); - assertEquals(true, Whitebox.getInternalState(workerTask, "lastSendFailed")); - assertEquals(Arrays.asList(record2, record3), Whitebox.getInternalState(workerTask, "toSend")); - - // Next they all succeed - Whitebox.invokeMethod(workerTask, "sendRecords"); - assertEquals(false, Whitebox.getInternalState(workerTask, "lastSendFailed")); - assertNull(Whitebox.getInternalState(workerTask, "toSend")); - - PowerMock.verifyAll(); - } - - @Test - public void testSendRecordsProducerCallbackFail() throws Exception { - createWorkerTask(); - - SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - - expectTopicCreation(TOPIC); - - expectSendRecordProducerCallbackFail(); - - PowerMock.replayAll(); - - Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); - assertThrows(ConnectException.class, () -> Whitebox.invokeMethod(workerTask, "sendRecords")); - } - - @Test - public void testSendRecordsProducerSendFailsImmediately() { - createWorkerTask(); - - SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - - expectPreliminaryCalls(); - expectTopicCreation(TOPIC); - - EasyMock.expect(producer.send(EasyMock.anyObject(), EasyMock.anyObject())) - .andThrow(new KafkaException("Producer closed while send in progress", new InvalidTopicException(TOPIC))); - - PowerMock.replayAll(); - - Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); - assertThrows(ConnectException.class, () -> Whitebox.invokeMethod(workerTask, "sendRecords")); - } - - @Test - public void testSendRecordsTaskCommitRecordFail() throws Exception { - createWorkerTask(); - - // Differentiate only by Kafka partition so we can reuse conversion expectations - SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - SourceRecord record3 = new SourceRecord(PARTITION, OFFSET, TOPIC, 3, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - - expectTopicCreation(TOPIC); - - // Source task commit record failure will not cause the task to abort - expectSendRecordOnce(false); - expectSendRecordTaskCommitRecordFail(false, false); - expectSendRecordOnce(false); - - PowerMock.replayAll(); - - Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2, record3)); - Whitebox.invokeMethod(workerTask, "sendRecords"); - assertEquals(false, Whitebox.getInternalState(workerTask, "lastSendFailed")); - assertNull(Whitebox.getInternalState(workerTask, "toSend")); - - PowerMock.verifyAll(); - } - - @Test - public void testSlowTaskStart() throws Exception { - final CountDownLatch startupLatch = new CountDownLatch(1); - final CountDownLatch finishStartupLatch = new CountDownLatch(1); - - createWorkerTask(); - - sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class)); - EasyMock.expectLastCall(); - sourceTask.start(TASK_PROPS); - EasyMock.expectLastCall().andAnswer(() -> { - startupLatch.countDown(); - assertTrue(awaitLatch(finishStartupLatch)); - return null; - }); - - statusListener.onStartup(taskId); - EasyMock.expectLastCall(); - - sourceTask.stop(); - EasyMock.expectLastCall(); - expectOffsetFlush(true); - - statusListener.onShutdown(taskId); - EasyMock.expectLastCall(); - - expectClose(); - - PowerMock.replayAll(); - - workerTask.initialize(TASK_CONFIG); - Future workerTaskFuture = executor.submit(workerTask); - - // Stopping immediately while the other thread has work to do should result in no polling, no offset commits, - // exiting the work thread immediately, and the stop() method will be invoked in the background thread since it - // cannot be invoked immediately in the thread trying to stop the task. - assertTrue(awaitLatch(startupLatch)); - workerTask.stop(); - finishStartupLatch.countDown(); - assertTrue(workerTask.awaitStop(1000)); - - workerTaskFuture.get(); - - PowerMock.verifyAll(); - } - - @Test - public void testCancel() { - createWorkerTask(); - - offsetReader.close(); - PowerMock.expectLastCall(); - - producer.close(Duration.ZERO); - PowerMock.expectLastCall(); - - PowerMock.replayAll(); - - workerTask.cancel(); - - PowerMock.verifyAll(); - } - - @Test - public void testMetricsGroup() { - SourceTaskMetricsGroup group = new SourceTaskMetricsGroup(taskId, metrics); - SourceTaskMetricsGroup group1 = new SourceTaskMetricsGroup(taskId1, metrics); - for (int i = 0; i != 10; ++i) { - group.recordPoll(100, 1000 + i * 100); - group.recordWrite(10); - } - for (int i = 0; i != 20; ++i) { - group1.recordPoll(100, 1000 + i * 100); - group1.recordWrite(10); - } - assertEquals(1900.0, metrics.currentMetricValueAsDouble(group.metricGroup(), "poll-batch-max-time-ms"), 0.001d); - assertEquals(1450.0, metrics.currentMetricValueAsDouble(group.metricGroup(), "poll-batch-avg-time-ms"), 0.001d); - assertEquals(33.333, metrics.currentMetricValueAsDouble(group.metricGroup(), "source-record-poll-rate"), 0.001d); - assertEquals(1000, metrics.currentMetricValueAsDouble(group.metricGroup(), "source-record-poll-total"), 0.001d); - assertEquals(3.3333, metrics.currentMetricValueAsDouble(group.metricGroup(), "source-record-write-rate"), 0.001d); - assertEquals(100, metrics.currentMetricValueAsDouble(group.metricGroup(), "source-record-write-total"), 0.001d); - assertEquals(900.0, metrics.currentMetricValueAsDouble(group.metricGroup(), "source-record-active-count"), 0.001d); - - // Close the group - group.close(); - - for (MetricName metricName : group.metricGroup().metrics().metrics().keySet()) { - // Metrics for this group should no longer exist - assertFalse(group.metricGroup().groupId().includes(metricName)); - } - // Sensors for this group should no longer exist - assertNull(group.metricGroup().metrics().getSensor("sink-record-read")); - assertNull(group.metricGroup().metrics().getSensor("sink-record-send")); - assertNull(group.metricGroup().metrics().getSensor("sink-record-active-count")); - assertNull(group.metricGroup().metrics().getSensor("partition-count")); - assertNull(group.metricGroup().metrics().getSensor("offset-seq-number")); - assertNull(group.metricGroup().metrics().getSensor("offset-commit-completion")); - assertNull(group.metricGroup().metrics().getSensor("offset-commit-completion-skip")); - assertNull(group.metricGroup().metrics().getSensor("put-batch-time")); - - assertEquals(2900.0, metrics.currentMetricValueAsDouble(group1.metricGroup(), "poll-batch-max-time-ms"), 0.001d); - assertEquals(1950.0, metrics.currentMetricValueAsDouble(group1.metricGroup(), "poll-batch-avg-time-ms"), 0.001d); - assertEquals(66.667, metrics.currentMetricValueAsDouble(group1.metricGroup(), "source-record-poll-rate"), 0.001d); - assertEquals(2000, metrics.currentMetricValueAsDouble(group1.metricGroup(), "source-record-poll-total"), 0.001d); - assertEquals(6.667, metrics.currentMetricValueAsDouble(group1.metricGroup(), "source-record-write-rate"), 0.001d); - assertEquals(200, metrics.currentMetricValueAsDouble(group1.metricGroup(), "source-record-write-total"), 0.001d); - assertEquals(1800.0, metrics.currentMetricValueAsDouble(group1.metricGroup(), "source-record-active-count"), 0.001d); - } - - @Test - public void testHeaders() throws Exception { - Headers headers = new RecordHeaders(); - headers.add("header_key", "header_value".getBytes()); - - org.apache.kafka.connect.header.Headers connectHeaders = new ConnectHeaders(); - connectHeaders.add("header_key", new SchemaAndValue(Schema.STRING_SCHEMA, "header_value")); - - createWorkerTask(); - - List records = new ArrayList<>(); - records.add(new SourceRecord(PARTITION, OFFSET, TOPIC, null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD, null, connectHeaders)); - - expectTopicCreation(TOPIC); - - Capture> sent = expectSendRecord(TOPIC, true, false, true, true, true, headers); - - PowerMock.replayAll(); - - Whitebox.setInternalState(workerTask, "toSend", records); - Whitebox.invokeMethod(workerTask, "sendRecords"); - assertEquals(SERIALIZED_KEY, sent.getValue().key()); - assertEquals(SERIALIZED_RECORD, sent.getValue().value()); - assertEquals(headers, sent.getValue().headers()); - - PowerMock.verifyAll(); - } - - @Test - public void testHeadersWithCustomConverter() throws Exception { - StringConverter stringConverter = new StringConverter(); - TestConverterWithHeaders testConverter = new TestConverterWithHeaders(); - - createWorkerTask(TargetState.STARTED, stringConverter, testConverter, stringConverter); - - List records = new ArrayList<>(); - - String stringA = "Árvíztűrő tükörfúrógép"; - org.apache.kafka.connect.header.Headers headersA = new ConnectHeaders(); - String encodingA = "latin2"; - headersA.addString("encoding", encodingA); - - records.add(new SourceRecord(PARTITION, OFFSET, TOPIC, null, Schema.STRING_SCHEMA, "a", Schema.STRING_SCHEMA, stringA, null, headersA)); - - String stringB = "Тестовое сообщение"; - org.apache.kafka.connect.header.Headers headersB = new ConnectHeaders(); - String encodingB = "koi8_r"; - headersB.addString("encoding", encodingB); - - records.add(new SourceRecord(PARTITION, OFFSET, TOPIC, null, Schema.STRING_SCHEMA, "b", Schema.STRING_SCHEMA, stringB, null, headersB)); - - expectTopicCreation(TOPIC); - - Capture> sentRecordA = expectSendRecord(TOPIC, false, false, true, true, false, null); - Capture> sentRecordB = expectSendRecord(TOPIC, false, false, true, true, false, null); - - PowerMock.replayAll(); - - Whitebox.setInternalState(workerTask, "toSend", records); - Whitebox.invokeMethod(workerTask, "sendRecords"); - - assertEquals(ByteBuffer.wrap("a".getBytes()), ByteBuffer.wrap(sentRecordA.getValue().key())); - assertEquals( - ByteBuffer.wrap(stringA.getBytes(encodingA)), - ByteBuffer.wrap(sentRecordA.getValue().value()) - ); - assertEquals(encodingA, new String(sentRecordA.getValue().headers().lastHeader("encoding").value())); - - assertEquals(ByteBuffer.wrap("b".getBytes()), ByteBuffer.wrap(sentRecordB.getValue().key())); - assertEquals( - ByteBuffer.wrap(stringB.getBytes(encodingB)), - ByteBuffer.wrap(sentRecordB.getValue().value()) - ); - assertEquals(encodingB, new String(sentRecordB.getValue().headers().lastHeader("encoding").value())); - - PowerMock.verifyAll(); - } - - @Test - public void testTopicCreateWhenTopicExists() throws Exception { - createWorkerTask(); - - SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - - expectPreliminaryCalls(); - TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, null, Collections.emptyList(), Collections.emptyList()); - TopicDescription topicDesc = new TopicDescription(TOPIC, false, Collections.singletonList(topicPartitionInfo)); - EasyMock.expect(admin.describeTopics(TOPIC)).andReturn(Collections.singletonMap(TOPIC, topicDesc)); - - expectSendRecordTaskCommitRecordSucceed(false, false); - expectSendRecordTaskCommitRecordSucceed(false, false); - - PowerMock.replayAll(); - - Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); - Whitebox.invokeMethod(workerTask, "sendRecords"); - } - - @Test - public void testSendRecordsTopicDescribeRetries() throws Exception { - createWorkerTask(); - - SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - - expectPreliminaryCalls(); - // First round - call to describe the topic times out - EasyMock.expect(admin.describeTopics(TOPIC)) - .andThrow(new RetriableException(new TimeoutException("timeout"))); - - // Second round - calls to describe and create succeed - expectTopicCreation(TOPIC); - // Exactly two records are sent - expectSendRecordTaskCommitRecordSucceed(false, false); - expectSendRecordTaskCommitRecordSucceed(false, false); - - PowerMock.replayAll(); - - Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); - Whitebox.invokeMethod(workerTask, "sendRecords"); - assertEquals(true, Whitebox.getInternalState(workerTask, "lastSendFailed")); - assertEquals(Arrays.asList(record1, record2), Whitebox.getInternalState(workerTask, "toSend")); - - // Next they all succeed - Whitebox.invokeMethod(workerTask, "sendRecords"); - assertEquals(false, Whitebox.getInternalState(workerTask, "lastSendFailed")); - assertNull(Whitebox.getInternalState(workerTask, "toSend")); - } - - @Test - public void testSendRecordsTopicCreateRetries() throws Exception { - createWorkerTask(); - - SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - - // First call to describe the topic times out - expectPreliminaryCalls(); - EasyMock.expect(admin.describeTopics(TOPIC)).andReturn(Collections.emptyMap()); - Capture newTopicCapture = EasyMock.newCapture(); - EasyMock.expect(admin.createOrFindTopics(EasyMock.capture(newTopicCapture))) - .andThrow(new RetriableException(new TimeoutException("timeout"))); - - // Second round - expectTopicCreation(TOPIC); - expectSendRecordTaskCommitRecordSucceed(false, false); - expectSendRecordTaskCommitRecordSucceed(false, false); - - PowerMock.replayAll(); - - Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); - Whitebox.invokeMethod(workerTask, "sendRecords"); - assertEquals(true, Whitebox.getInternalState(workerTask, "lastSendFailed")); - assertEquals(Arrays.asList(record1, record2), Whitebox.getInternalState(workerTask, "toSend")); - - // Next they all succeed - Whitebox.invokeMethod(workerTask, "sendRecords"); - assertEquals(false, Whitebox.getInternalState(workerTask, "lastSendFailed")); - assertNull(Whitebox.getInternalState(workerTask, "toSend")); - } - - @Test - public void testSendRecordsTopicDescribeRetriesMidway() throws Exception { - createWorkerTask(); - - // Differentiate only by Kafka partition so we can reuse conversion expectations - SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - SourceRecord record3 = new SourceRecord(PARTITION, OFFSET, OTHER_TOPIC, 3, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - - // First round - expectPreliminaryCalls(OTHER_TOPIC); - expectTopicCreation(TOPIC); - expectSendRecordTaskCommitRecordSucceed(false, false); - expectSendRecordTaskCommitRecordSucceed(false, false); - - // First call to describe the topic times out - EasyMock.expect(admin.describeTopics(OTHER_TOPIC)) - .andThrow(new RetriableException(new TimeoutException("timeout"))); - - // Second round - expectTopicCreation(OTHER_TOPIC); - expectSendRecord(OTHER_TOPIC, false, true, true, true, true, emptyHeaders()); - - PowerMock.replayAll(); - - // Try to send 3, make first pass, second fail. Should save last two - Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2, record3)); - Whitebox.invokeMethod(workerTask, "sendRecords"); - assertEquals(true, Whitebox.getInternalState(workerTask, "lastSendFailed")); - assertEquals(Arrays.asList(record3), Whitebox.getInternalState(workerTask, "toSend")); - - // Next they all succeed - Whitebox.invokeMethod(workerTask, "sendRecords"); - assertEquals(false, Whitebox.getInternalState(workerTask, "lastSendFailed")); - assertNull(Whitebox.getInternalState(workerTask, "toSend")); - - PowerMock.verifyAll(); - } - - @Test - public void testSendRecordsTopicCreateRetriesMidway() throws Exception { - createWorkerTask(); - - // Differentiate only by Kafka partition so we can reuse conversion expectations - SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - SourceRecord record3 = new SourceRecord(PARTITION, OFFSET, OTHER_TOPIC, 3, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - - // First round - expectPreliminaryCalls(OTHER_TOPIC); - expectTopicCreation(TOPIC); - expectSendRecordTaskCommitRecordSucceed(false, false); - expectSendRecordTaskCommitRecordSucceed(false, false); - - EasyMock.expect(admin.describeTopics(OTHER_TOPIC)).andReturn(Collections.emptyMap()); - // First call to create the topic times out - Capture newTopicCapture = EasyMock.newCapture(); - EasyMock.expect(admin.createOrFindTopics(EasyMock.capture(newTopicCapture))) - .andThrow(new RetriableException(new TimeoutException("timeout"))); - - // Second round - expectTopicCreation(OTHER_TOPIC); - expectSendRecord(OTHER_TOPIC, false, true, true, true, true, emptyHeaders()); - - PowerMock.replayAll(); - - // Try to send 3, make first pass, second fail. Should save last two - Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2, record3)); - Whitebox.invokeMethod(workerTask, "sendRecords"); - assertEquals(true, Whitebox.getInternalState(workerTask, "lastSendFailed")); - assertEquals(Arrays.asList(record3), Whitebox.getInternalState(workerTask, "toSend")); - - // Next they all succeed - Whitebox.invokeMethod(workerTask, "sendRecords"); - assertEquals(false, Whitebox.getInternalState(workerTask, "lastSendFailed")); - assertNull(Whitebox.getInternalState(workerTask, "toSend")); - - PowerMock.verifyAll(); - } - - @Test - public void testTopicDescribeFails() { - createWorkerTask(); - - SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - - expectPreliminaryCalls(); - EasyMock.expect(admin.describeTopics(TOPIC)) - .andThrow(new ConnectException(new TopicAuthorizationException("unauthorized"))); - - PowerMock.replayAll(); - - Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); - assertThrows(ConnectException.class, () -> Whitebox.invokeMethod(workerTask, "sendRecords")); - } - - @Test - public void testTopicCreateFails() throws Exception { - createWorkerTask(); - - SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - - expectPreliminaryCalls(); - EasyMock.expect(admin.describeTopics(TOPIC)).andReturn(Collections.emptyMap()); - - Capture newTopicCapture = EasyMock.newCapture(); - EasyMock.expect(admin.createOrFindTopics(EasyMock.capture(newTopicCapture))) - .andThrow(new ConnectException(new TopicAuthorizationException("unauthorized"))); - - PowerMock.replayAll(); - - Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); - assertThrows(ConnectException.class, () -> Whitebox.invokeMethod(workerTask, "sendRecords")); - assertTrue(newTopicCapture.hasCaptured()); - } - - @Test - public void testTopicCreateFailsWithExceptionWhenCreateReturnsTopicNotCreatedOrFound() throws Exception { - createWorkerTask(); - - SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - - expectPreliminaryCalls(); - EasyMock.expect(admin.describeTopics(TOPIC)).andReturn(Collections.emptyMap()); - - Capture newTopicCapture = EasyMock.newCapture(); - EasyMock.expect(admin.createOrFindTopics(EasyMock.capture(newTopicCapture))).andReturn(TopicAdmin.EMPTY_CREATION); - - PowerMock.replayAll(); - - Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); - assertThrows(ConnectException.class, () -> Whitebox.invokeMethod(workerTask, "sendRecords")); - assertTrue(newTopicCapture.hasCaptured()); - } - - @Test - public void testTopicCreateSucceedsWhenCreateReturnsExistingTopicFound() throws Exception { - createWorkerTask(); - - SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - - expectPreliminaryCalls(); - EasyMock.expect(admin.describeTopics(TOPIC)).andReturn(Collections.emptyMap()); - - Capture newTopicCapture = EasyMock.newCapture(); - EasyMock.expect(admin.createOrFindTopics(EasyMock.capture(newTopicCapture))).andReturn(foundTopic(TOPIC)); - - expectSendRecordTaskCommitRecordSucceed(false, false); - expectSendRecordTaskCommitRecordSucceed(false, false); - - PowerMock.replayAll(); - - Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); - Whitebox.invokeMethod(workerTask, "sendRecords"); - } - - @Test - public void testTopicCreateSucceedsWhenCreateReturnsNewTopicFound() throws Exception { - createWorkerTask(); - - SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - - expectPreliminaryCalls(); - EasyMock.expect(admin.describeTopics(TOPIC)).andReturn(Collections.emptyMap()); - - Capture newTopicCapture = EasyMock.newCapture(); - EasyMock.expect(admin.createOrFindTopics(EasyMock.capture(newTopicCapture))).andReturn(createdTopic(TOPIC)); - - expectSendRecordTaskCommitRecordSucceed(false, false); - expectSendRecordTaskCommitRecordSucceed(false, false); - - PowerMock.replayAll(); - - Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); - Whitebox.invokeMethod(workerTask, "sendRecords"); - } - - private TopicAdmin.TopicCreationResponse createdTopic(String topic) { - Set created = Collections.singleton(topic); - Set existing = Collections.emptySet(); - return new TopicAdmin.TopicCreationResponse(created, existing); - } - - private TopicAdmin.TopicCreationResponse foundTopic(String topic) { - Set created = Collections.emptySet(); - Set existing = Collections.singleton(topic); - return new TopicAdmin.TopicCreationResponse(created, existing); - } - - private void expectPreliminaryCalls() { - expectPreliminaryCalls(TOPIC); - } - - private void expectPreliminaryCalls(String topic) { - expectConvertHeadersAndKeyValue(topic, true, emptyHeaders()); - expectApplyTransformationChain(false); - offsetWriter.offset(PARTITION, OFFSET); - PowerMock.expectLastCall(); - } - - private CountDownLatch expectEmptyPolls(int minimum, final AtomicInteger count) throws InterruptedException { - final CountDownLatch latch = new CountDownLatch(minimum); - // Note that we stub these to allow any number of calls because the thread will continue to - // run. The count passed in + latch returned just makes sure we get *at least* that number of - // calls - EasyMock.expect(sourceTask.poll()) - .andStubAnswer(() -> { - count.incrementAndGet(); - latch.countDown(); - Thread.sleep(10); - return Collections.emptyList(); - }); - return latch; - } - - private CountDownLatch expectPolls(int minimum, final AtomicInteger count) throws InterruptedException { - final CountDownLatch latch = new CountDownLatch(minimum); - // Note that we stub these to allow any number of calls because the thread will continue to - // run. The count passed in + latch returned just makes sure we get *at least* that number of - // calls - EasyMock.expect(sourceTask.poll()) - .andStubAnswer(() -> { - count.incrementAndGet(); - latch.countDown(); - Thread.sleep(10); - return RECORDS; - }); - // Fallout of the poll() call - expectSendRecordAnyTimes(); - return latch; - } - - private CountDownLatch expectPolls(int count) throws InterruptedException { - return expectPolls(count, new AtomicInteger()); - } - - @SuppressWarnings("unchecked") - private void expectSendRecordSyncFailure(Throwable error) throws InterruptedException { - expectConvertHeadersAndKeyValue(false); - expectApplyTransformationChain(false); - - offsetWriter.offset(PARTITION, OFFSET); - PowerMock.expectLastCall(); - - EasyMock.expect( - producer.send(EasyMock.anyObject(ProducerRecord.class), - EasyMock.anyObject(org.apache.kafka.clients.producer.Callback.class))) - .andThrow(error); - } - - private Capture> expectSendRecordAnyTimes() throws InterruptedException { - return expectSendRecordTaskCommitRecordSucceed(true, false); - } - - private Capture> expectSendRecordOnce(boolean isRetry) throws InterruptedException { - return expectSendRecordTaskCommitRecordSucceed(false, isRetry); - } - - private Capture> expectSendRecordProducerCallbackFail() throws InterruptedException { - return expectSendRecord(TOPIC, false, false, false, false, true, emptyHeaders()); - } - - private Capture> expectSendRecordTaskCommitRecordSucceed(boolean anyTimes, boolean isRetry) throws InterruptedException { - return expectSendRecord(TOPIC, anyTimes, isRetry, true, true, true, emptyHeaders()); - } - - private Capture> expectSendRecordTaskCommitRecordFail(boolean anyTimes, boolean isRetry) throws InterruptedException { - return expectSendRecord(TOPIC, anyTimes, isRetry, true, false, true, emptyHeaders()); - } - - private Capture> expectSendRecord(boolean anyTimes, boolean isRetry, boolean succeed) throws InterruptedException { - return expectSendRecord(TOPIC, anyTimes, isRetry, succeed, true, true, emptyHeaders()); - } - - private Capture> expectSendRecord( - String topic, - boolean anyTimes, - boolean isRetry, - boolean sendSuccess, - boolean commitSuccess, - boolean isMockedConverters, - Headers headers - ) throws InterruptedException { - if (isMockedConverters) { - expectConvertHeadersAndKeyValue(topic, anyTimes, headers); - } - - expectApplyTransformationChain(anyTimes); - - Capture> sent = EasyMock.newCapture(); - - // 1. Offset data is passed to the offset storage. - if (!isRetry) { - offsetWriter.offset(PARTITION, OFFSET); - if (anyTimes) - PowerMock.expectLastCall().anyTimes(); - else - PowerMock.expectLastCall(); - } - - // 2. Converted data passed to the producer, which will need callbacks invoked for flush to work - IExpectationSetters> expect = EasyMock.expect( - producer.send(EasyMock.capture(sent), - EasyMock.capture(producerCallbacks))); - IAnswer> expectResponse = () -> { - synchronized (producerCallbacks) { - for (org.apache.kafka.clients.producer.Callback cb : producerCallbacks.getValues()) { - if (sendSuccess) { - cb.onCompletion(new RecordMetadata(new TopicPartition("foo", 0), 0, 0, - 0L, 0L, 0, 0), null); - } else { - cb.onCompletion(null, new TopicAuthorizationException("foo")); - } - } - producerCallbacks.reset(); - } - return sendFuture; - }; - if (anyTimes) - expect.andStubAnswer(expectResponse); - else - expect.andAnswer(expectResponse); - - if (sendSuccess) { - // 3. As a result of a successful producer send callback, we'll notify the source task of the record commit - expectTaskCommitRecordWithOffset(anyTimes, commitSuccess); - expectTaskGetTopic(anyTimes); - } - - return sent; - } - - private void expectConvertHeadersAndKeyValue(boolean anyTimes) { - expectConvertHeadersAndKeyValue(TOPIC, anyTimes, emptyHeaders()); - } - - private void expectConvertHeadersAndKeyValue(String topic, boolean anyTimes, Headers headers) { - for (Header header : headers) { - IExpectationSetters convertHeaderExpect = EasyMock.expect(headerConverter.fromConnectHeader(topic, header.key(), Schema.STRING_SCHEMA, new String(header.value()))); - if (anyTimes) - convertHeaderExpect.andStubReturn(header.value()); - else - convertHeaderExpect.andReturn(header.value()); - } - IExpectationSetters convertKeyExpect = EasyMock.expect(keyConverter.fromConnectData(topic, headers, KEY_SCHEMA, KEY)); - if (anyTimes) - convertKeyExpect.andStubReturn(SERIALIZED_KEY); - else - convertKeyExpect.andReturn(SERIALIZED_KEY); - IExpectationSetters convertValueExpect = EasyMock.expect(valueConverter.fromConnectData(topic, headers, RECORD_SCHEMA, RECORD)); - if (anyTimes) - convertValueExpect.andStubReturn(SERIALIZED_RECORD); - else - convertValueExpect.andReturn(SERIALIZED_RECORD); - } - - private void expectApplyTransformationChain(boolean anyTimes) { - final Capture recordCapture = EasyMock.newCapture(); - IExpectationSetters convertKeyExpect = EasyMock.expect(transformationChain.apply(EasyMock.capture(recordCapture))); - if (anyTimes) - convertKeyExpect.andStubAnswer(recordCapture::getValue); - else - convertKeyExpect.andAnswer(recordCapture::getValue); - } - - private void expectTaskCommitRecordWithOffset(boolean anyTimes, boolean succeed) throws InterruptedException { - sourceTask.commitRecord(EasyMock.anyObject(SourceRecord.class), EasyMock.anyObject(RecordMetadata.class)); - IExpectationSetters expect = EasyMock.expectLastCall(); - if (!succeed) { - expect = expect.andThrow(new RuntimeException("Error committing record in source task")); - } - if (anyTimes) { - expect.anyTimes(); - } - } - - private void expectTaskGetTopic(boolean anyTimes) { - final Capture connectorCapture = EasyMock.newCapture(); - final Capture topicCapture = EasyMock.newCapture(); - IExpectationSetters expect = EasyMock.expect(statusBackingStore.getTopic( - EasyMock.capture(connectorCapture), - EasyMock.capture(topicCapture))); - if (anyTimes) { - expect.andStubAnswer(() -> new TopicStatus( - topicCapture.getValue(), - new ConnectorTaskId(connectorCapture.getValue(), 0), - Time.SYSTEM.milliseconds())); - } else { - expect.andAnswer(() -> new TopicStatus( - topicCapture.getValue(), - new ConnectorTaskId(connectorCapture.getValue(), 0), - Time.SYSTEM.milliseconds())); - } - if (connectorCapture.hasCaptured() && topicCapture.hasCaptured()) { - assertEquals("job", connectorCapture.getValue()); - assertEquals(TOPIC, topicCapture.getValue()); - } - } - - private boolean awaitLatch(CountDownLatch latch) { - try { - return latch.await(5000, TimeUnit.MILLISECONDS); - } catch (InterruptedException e) { - // ignore - } - return false; - } - - @SuppressWarnings("unchecked") - private void expectOffsetFlush(boolean succeed) throws Exception { - EasyMock.expect(offsetWriter.beginFlush()).andReturn(true); - Future flushFuture = PowerMock.createMock(Future.class); - EasyMock.expect(offsetWriter.doFlush(EasyMock.anyObject(Callback.class))).andReturn(flushFuture); - // Should throw for failure - IExpectationSetters futureGetExpect = EasyMock.expect( - flushFuture.get(EasyMock.anyLong(), EasyMock.anyObject(TimeUnit.class))); - if (succeed) { - sourceTask.commit(); - EasyMock.expectLastCall(); - futureGetExpect.andReturn(null); - } else { - futureGetExpect.andThrow(new TimeoutException()); - offsetWriter.cancelFlush(); - PowerMock.expectLastCall(); - } - } - - private void assertPollMetrics(int minimumPollCountExpected) { - MetricGroup sourceTaskGroup = workerTask.sourceTaskMetricsGroup().metricGroup(); - MetricGroup taskGroup = workerTask.taskMetricsGroup().metricGroup(); - double pollRate = metrics.currentMetricValueAsDouble(sourceTaskGroup, "source-record-poll-rate"); - double pollTotal = metrics.currentMetricValueAsDouble(sourceTaskGroup, "source-record-poll-total"); - if (minimumPollCountExpected > 0) { - assertEquals(RECORDS.size(), metrics.currentMetricValueAsDouble(taskGroup, "batch-size-max"), 0.000001d); - assertEquals(RECORDS.size(), metrics.currentMetricValueAsDouble(taskGroup, "batch-size-avg"), 0.000001d); - assertTrue(pollRate > 0.0d); - } else { - assertTrue(pollRate == 0.0d); - } - assertTrue(pollTotal >= minimumPollCountExpected); - - double writeRate = metrics.currentMetricValueAsDouble(sourceTaskGroup, "source-record-write-rate"); - double writeTotal = metrics.currentMetricValueAsDouble(sourceTaskGroup, "source-record-write-total"); - if (minimumPollCountExpected > 0) { - assertTrue(writeRate > 0.0d); - } else { - assertTrue(writeRate == 0.0d); - } - assertTrue(writeTotal >= minimumPollCountExpected); - - double pollBatchTimeMax = metrics.currentMetricValueAsDouble(sourceTaskGroup, "poll-batch-max-time-ms"); - double pollBatchTimeAvg = metrics.currentMetricValueAsDouble(sourceTaskGroup, "poll-batch-avg-time-ms"); - if (minimumPollCountExpected > 0) { - assertTrue(pollBatchTimeMax >= 0.0d); - } - assertTrue(Double.isNaN(pollBatchTimeAvg) || pollBatchTimeAvg > 0.0d); - double activeCount = metrics.currentMetricValueAsDouble(sourceTaskGroup, "source-record-active-count"); - double activeCountMax = metrics.currentMetricValueAsDouble(sourceTaskGroup, "source-record-active-count-max"); - assertEquals(0, activeCount, 0.000001d); - if (minimumPollCountExpected > 0) { - assertEquals(RECORDS.size(), activeCountMax, 0.000001d); - } - } - - private RecordHeaders emptyHeaders() { - return new RecordHeaders(); - } - - private abstract static class TestSourceTask extends SourceTask { - } - - private void expectClose() { - producer.close(EasyMock.anyObject(Duration.class)); - EasyMock.expectLastCall(); - - admin.close(EasyMock.anyObject(Duration.class)); - EasyMock.expectLastCall(); - - transformationChain.close(); - EasyMock.expectLastCall(); - } - - private void expectTopicCreation(String topic) { - if (config.topicCreationEnable()) { - EasyMock.expect(admin.describeTopics(topic)).andReturn(Collections.emptyMap()); - Capture newTopicCapture = EasyMock.newCapture(); - EasyMock.expect(admin.createOrFindTopics(EasyMock.capture(newTopicCapture))).andReturn(createdTopic(topic)); - } - } -} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java index b5c9122756a..171f16db996 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.connect.runtime; +import java.util.Collection; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.producer.KafkaProducer; @@ -64,6 +65,7 @@ import org.apache.kafka.connect.storage.StatusBackingStore; import org.apache.kafka.connect.util.ConnectUtils; import org.apache.kafka.connect.util.ConnectorTaskId; import org.apache.kafka.connect.util.FutureCallback; +import org.apache.kafka.connect.util.ParameterizedTest; import org.apache.kafka.connect.util.ThreadedTest; import org.apache.kafka.connect.util.TopicAdmin; import org.apache.kafka.connect.util.TopicCreationGroup; @@ -97,7 +99,11 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; +import org.powermock.modules.junit4.PowerMockRunnerDelegate; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.DEFAULT_TOPIC_CREATION_PREFIX; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.PARTITIONS_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.REPLICATION_FACTOR_CONFIG; import static org.apache.kafka.connect.runtime.WorkerConfig.TOPIC_CREATION_ENABLE_CONFIG; import static org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperatorTest.NOOP_OPERATOR; import static org.easymock.EasyMock.anyObject; @@ -113,6 +119,7 @@ import static org.junit.Assert.assertThrows; import static org.junit.Assert.fail; @RunWith(PowerMockRunner.class) +@PowerMockRunnerDelegate(ParameterizedTest.class) @PrepareForTest({Worker.class, Plugins.class, ConnectUtils.class}) @PowerMockIgnore("javax.management.*") public class WorkerTest extends ThreadedTest { @@ -161,8 +168,16 @@ public class WorkerTest extends ThreadedTest { private String mockFileProviderTestId; private Map connectorProps; - // when this test becomes parameterized, this variable will be a test parameter - public boolean enableTopicCreation = false; + private boolean enableTopicCreation; + + @ParameterizedTest.Parameters + public static Collection parameters() { + return Arrays.asList(false, true); + } + + public WorkerTest(boolean enableTopicCreation) { + this.enableTopicCreation = enableTopicCreation; + } @Before public void setup() { @@ -1453,6 +1468,8 @@ public class WorkerTest extends ThreadedTest { props.put(ConnectorConfig.NAME_CONFIG, CONNECTOR_ID); props.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, WorkerTestConnector.class.getName()); props.put(ConnectorConfig.TASKS_MAX_CONFIG, "1"); + props.put(DEFAULT_TOPIC_CREATION_PREFIX + REPLICATION_FACTOR_CONFIG, String.valueOf(1)); + props.put(DEFAULT_TOPIC_CREATION_PREFIX + PARTITIONS_CONFIG, String.valueOf(1)); return props; } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerWithTopicCreationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerWithTopicCreationTest.java deleted file mode 100644 index a6745ec9974..00000000000 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerWithTopicCreationTest.java +++ /dev/null @@ -1,1510 +0,0 @@ -/* - * 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.runtime; - -import org.apache.kafka.clients.CommonClientConfigs; -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.producer.KafkaProducer; -import org.apache.kafka.clients.producer.ProducerConfig; -import org.apache.kafka.common.Configurable; -import org.apache.kafka.common.config.AbstractConfig; -import org.apache.kafka.common.config.ConfigDef; -import org.apache.kafka.common.config.ConfigException; -import org.apache.kafka.common.config.provider.MockFileConfigProvider; -import org.apache.kafka.common.utils.MockTime; -import org.apache.kafka.common.utils.Time; -import org.apache.kafka.connect.connector.ConnectorContext; -import org.apache.kafka.connect.connector.Task; -import org.apache.kafka.connect.connector.policy.AllConnectorClientConfigOverridePolicy; -import org.apache.kafka.connect.connector.policy.ConnectorClientConfigOverridePolicy; -import org.apache.kafka.connect.connector.policy.NoneConnectorClientConfigOverridePolicy; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.SchemaAndValue; -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.ConnectMetrics.MetricGroup; -import org.apache.kafka.connect.runtime.MockConnectMetrics.MockMetricsReporter; -import org.apache.kafka.connect.runtime.distributed.ClusterConfigState; -import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator; -import org.apache.kafka.connect.runtime.isolation.DelegatingClassLoader; -import org.apache.kafka.connect.runtime.isolation.PluginClassLoader; -import org.apache.kafka.connect.runtime.isolation.Plugins; -import org.apache.kafka.connect.runtime.isolation.Plugins.ClassLoaderUsage; -import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo; -import org.apache.kafka.connect.runtime.standalone.StandaloneConfig; -import org.apache.kafka.connect.sink.SinkTask; -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.Converter; -import org.apache.kafka.connect.storage.HeaderConverter; -import org.apache.kafka.connect.storage.OffsetBackingStore; -import org.apache.kafka.connect.storage.OffsetStorageReader; -import org.apache.kafka.connect.storage.OffsetStorageWriter; -import org.apache.kafka.connect.storage.StatusBackingStore; -import org.apache.kafka.connect.util.ConnectUtils; -import org.apache.kafka.connect.util.ConnectorTaskId; -import org.apache.kafka.connect.util.FutureCallback; -import org.apache.kafka.connect.util.ThreadedTest; -import org.apache.kafka.connect.util.TopicAdmin; -import org.apache.kafka.connect.util.TopicCreationGroup; -import org.easymock.EasyMock; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.powermock.api.easymock.PowerMock; -import org.powermock.api.easymock.annotation.Mock; -import org.powermock.api.easymock.annotation.MockNice; -import org.powermock.api.easymock.annotation.MockStrict; -import org.powermock.core.classloader.annotations.PowerMockIgnore; -import org.powermock.core.classloader.annotations.PrepareForTest; -import org.powermock.modules.junit4.PowerMockRunner; - -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.UUID; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Executor; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.TimeUnit; - -import static org.apache.kafka.connect.runtime.TopicCreationConfig.DEFAULT_TOPIC_CREATION_PREFIX; -import static org.apache.kafka.connect.runtime.TopicCreationConfig.PARTITIONS_CONFIG; -import static org.apache.kafka.connect.runtime.TopicCreationConfig.REPLICATION_FACTOR_CONFIG; -import static org.apache.kafka.connect.runtime.WorkerConfig.TOPIC_CREATION_ENABLE_CONFIG; -import static org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperatorTest.NOOP_OPERATOR; -import static org.easymock.EasyMock.anyObject; -import static org.easymock.EasyMock.eq; -import static org.easymock.EasyMock.expectLastCall; -import static org.hamcrest.CoreMatchers.instanceOf; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertThrows; -import static org.junit.Assert.fail; - -@RunWith(PowerMockRunner.class) -@PrepareForTest({Worker.class, Plugins.class, ConnectUtils.class}) -@PowerMockIgnore("javax.management.*") -public class WorkerWithTopicCreationTest extends ThreadedTest { - - private static final String CONNECTOR_ID = "test-connector"; - private static final ConnectorTaskId TASK_ID = new ConnectorTaskId("job", 0); - private static final String WORKER_ID = "localhost:8083"; - private static final String CLUSTER_ID = "test-cluster"; - private final ConnectorClientConfigOverridePolicy noneConnectorClientConfigOverridePolicy = new NoneConnectorClientConfigOverridePolicy(); - private final ConnectorClientConfigOverridePolicy allConnectorClientConfigOverridePolicy = new AllConnectorClientConfigOverridePolicy(); - - private Map workerProps = new HashMap<>(); - private WorkerConfig config; - private Worker worker; - - private Map defaultProducerConfigs = new HashMap<>(); - private Map defaultConsumerConfigs = new HashMap<>(); - - @Mock - private Plugins plugins; - @Mock - private PluginClassLoader pluginLoader; - @Mock - private DelegatingClassLoader delegatingLoader; - @Mock - private OffsetBackingStore offsetBackingStore; - @MockStrict - private TaskStatus.Listener taskStatusListener; - @MockStrict - private ConnectorStatus.Listener connectorStatusListener; - - @Mock private Herder herder; - @Mock private StatusBackingStore statusBackingStore; - @Mock private SourceConnector connector; - @Mock private CloseableConnectorContext ctx; - @Mock private TestSourceTask task; - @Mock private WorkerSourceTask workerTask; - @Mock private Converter keyConverter; - @Mock private Converter valueConverter; - @Mock private Converter taskKeyConverter; - @Mock private Converter taskValueConverter; - @Mock private HeaderConverter taskHeaderConverter; - @Mock private ExecutorService executorService; - @MockNice private ConnectorConfig connectorConfig; - private String mockFileProviderTestId; - private Map connectorProps; - - // when this test becomes parameterized, this variable will be a test parameter - public boolean enableTopicCreation = true; - - @Before - public void setup() { - super.setup(); - workerProps.put("key.converter", "org.apache.kafka.connect.json.JsonConverter"); - workerProps.put("value.converter", "org.apache.kafka.connect.json.JsonConverter"); - workerProps.put("offset.storage.file.filename", "/tmp/connect.offsets"); - workerProps.put(CommonClientConfigs.METRIC_REPORTER_CLASSES_CONFIG, MockMetricsReporter.class.getName()); - workerProps.put("config.providers", "file"); - workerProps.put("config.providers.file.class", MockFileConfigProvider.class.getName()); - mockFileProviderTestId = UUID.randomUUID().toString(); - workerProps.put("config.providers.file.param.testId", mockFileProviderTestId); - workerProps.put(TOPIC_CREATION_ENABLE_CONFIG, String.valueOf(enableTopicCreation)); - config = new StandaloneConfig(workerProps); - - defaultProducerConfigs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); - defaultProducerConfigs.put( - ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); - defaultProducerConfigs.put( - ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); - defaultProducerConfigs.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, Long.toString(Long.MAX_VALUE)); - defaultProducerConfigs.put(ProducerConfig.ACKS_CONFIG, "all"); - defaultProducerConfigs.put(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, "1"); - defaultProducerConfigs.put(ProducerConfig.DELIVERY_TIMEOUT_MS_CONFIG, Integer.toString(Integer.MAX_VALUE)); - - defaultConsumerConfigs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); - defaultConsumerConfigs.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); - defaultConsumerConfigs.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); - defaultConsumerConfigs - .put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); - defaultConsumerConfigs - .put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); - - // Some common defaults. They might change on individual tests - connectorProps = anyConnectorConfigMap(); - PowerMock.mockStatic(Plugins.class); - } - - @Test - public void testStartAndStopConnector() throws Throwable { - expectConverters(); - expectStartStorage(); - - final String connectorClass = WorkerTest.WorkerTestConnector.class.getName(); - - // Create - EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader).times(2); - EasyMock.expect(plugins.delegatingLoader()).andReturn(delegatingLoader); - EasyMock.expect(delegatingLoader.connectorLoader(connectorClass)).andReturn(pluginLoader); - EasyMock.expect(plugins.newConnector(connectorClass)) - .andReturn(connector); - EasyMock.expect(connector.version()).andReturn("1.0"); - - connectorProps.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, connectorClass); - - EasyMock.expect(connector.version()).andReturn("1.0"); - - expectFileConfigProvider(); - EasyMock.expect(Plugins.compareAndSwapLoaders(pluginLoader)) - .andReturn(delegatingLoader) - .times(3); - connector.initialize(anyObject(ConnectorContext.class)); - EasyMock.expectLastCall(); - connector.start(connectorProps); - EasyMock.expectLastCall(); - - EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)) - .andReturn(pluginLoader).times(3); - - connectorStatusListener.onStartup(CONNECTOR_ID); - EasyMock.expectLastCall(); - - // Remove - connector.stop(); - EasyMock.expectLastCall(); - - connectorStatusListener.onShutdown(CONNECTOR_ID); - EasyMock.expectLastCall(); - - ctx.close(); - expectLastCall(); - - expectStopStorage(); - expectClusterId(); - - PowerMock.replayAll(); - - worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, noneConnectorClientConfigOverridePolicy); - worker.herder = herder; - worker.start(); - - assertEquals(Collections.emptySet(), worker.connectorNames()); - - FutureCallback onFirstStart = new FutureCallback<>(); - worker.startConnector(CONNECTOR_ID, connectorProps, ctx, connectorStatusListener, TargetState.STARTED, onFirstStart); - // Wait for the connector to actually start - assertEquals(TargetState.STARTED, onFirstStart.get(1000, TimeUnit.MILLISECONDS)); - assertEquals(new HashSet<>(Arrays.asList(CONNECTOR_ID)), worker.connectorNames()); - - FutureCallback onSecondStart = new FutureCallback<>(); - worker.startConnector(CONNECTOR_ID, connectorProps, ctx, connectorStatusListener, TargetState.STARTED, onSecondStart); - try { - onSecondStart.get(0, TimeUnit.MILLISECONDS); - fail("Should have failed while trying to start second connector with same name"); - } catch (ExecutionException e) { - assertThat(e.getCause(), instanceOf(ConnectException.class)); - } - - assertStatistics(worker, 1, 0); - assertStartupStatistics(worker, 1, 0, 0, 0); - worker.stopAndAwaitConnector(CONNECTOR_ID); - assertStatistics(worker, 0, 0); - assertStartupStatistics(worker, 1, 0, 0, 0); - assertEquals(Collections.emptySet(), worker.connectorNames()); - // Nothing should be left, so this should effectively be a nop - worker.stop(); - assertStatistics(worker, 0, 0); - - PowerMock.verifyAll(); - MockFileConfigProvider.assertClosed(mockFileProviderTestId); - } - - private void expectFileConfigProvider() { - EasyMock.expect(plugins.newConfigProvider(EasyMock.anyObject(), - EasyMock.eq("config.providers.file"), EasyMock.anyObject())) - .andAnswer(() -> { - MockFileConfigProvider mockFileConfigProvider = new MockFileConfigProvider(); - mockFileConfigProvider.configure(Collections.singletonMap("testId", mockFileProviderTestId)); - return mockFileConfigProvider; - }); - } - - @Test - public void testStartConnectorFailure() throws Exception { - expectConverters(); - expectStartStorage(); - expectFileConfigProvider(); - - final String nonConnectorClass = "java.util.HashMap"; - connectorProps.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, nonConnectorClass); // Bad connector class name - - Exception exception = new ConnectException("Failed to find Connector"); - EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader); - EasyMock.expect(plugins.delegatingLoader()).andReturn(delegatingLoader); - EasyMock.expect(delegatingLoader.connectorLoader(nonConnectorClass)).andReturn(delegatingLoader); - EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)) - .andReturn(delegatingLoader).times(2); - EasyMock.expect(plugins.newConnector(EasyMock.anyString())) - .andThrow(exception); - - connectorStatusListener.onFailure( - EasyMock.eq(CONNECTOR_ID), - EasyMock.anyObject() - ); - EasyMock.expectLastCall(); - - expectClusterId(); - - PowerMock.replayAll(); - - worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, noneConnectorClientConfigOverridePolicy); - worker.herder = herder; - worker.start(); - - assertStatistics(worker, 0, 0); - FutureCallback onStart = new FutureCallback<>(); - worker.startConnector(CONNECTOR_ID, connectorProps, ctx, connectorStatusListener, TargetState.STARTED, onStart); - try { - onStart.get(0, TimeUnit.MILLISECONDS); - fail("Should have failed to start connector"); - } catch (ExecutionException e) { - assertEquals(exception, e.getCause()); - } - - assertStartupStatistics(worker, 1, 1, 0, 0); - assertEquals(Collections.emptySet(), worker.connectorNames()); - - assertStatistics(worker, 0, 0); - assertStartupStatistics(worker, 1, 1, 0, 0); - worker.stopAndAwaitConnector(CONNECTOR_ID); - assertStatistics(worker, 0, 0); - assertStartupStatistics(worker, 1, 1, 0, 0); - - PowerMock.verifyAll(); - } - - @Test - public void testAddConnectorByAlias() throws Throwable { - expectConverters(); - expectStartStorage(); - expectFileConfigProvider(); - - final String connectorAlias = "WorkerTestConnector"; - - EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader).times(2); - EasyMock.expect(plugins.delegatingLoader()).andReturn(delegatingLoader); - EasyMock.expect(delegatingLoader.connectorLoader(connectorAlias)).andReturn(pluginLoader); - EasyMock.expect(plugins.newConnector(connectorAlias)).andReturn(connector); - EasyMock.expect(connector.version()).andReturn("1.0"); - - connectorProps.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, connectorAlias); - - EasyMock.expect(connector.version()).andReturn("1.0"); - EasyMock.expect(Plugins.compareAndSwapLoaders(pluginLoader)) - .andReturn(delegatingLoader) - .times(3); - connector.initialize(anyObject(ConnectorContext.class)); - EasyMock.expectLastCall(); - connector.start(connectorProps); - EasyMock.expectLastCall(); - - EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)) - .andReturn(pluginLoader) - .times(3); - - connectorStatusListener.onStartup(CONNECTOR_ID); - EasyMock.expectLastCall(); - - // Remove - connector.stop(); - EasyMock.expectLastCall(); - - connectorStatusListener.onShutdown(CONNECTOR_ID); - EasyMock.expectLastCall(); - - ctx.close(); - expectLastCall(); - - expectStopStorage(); - expectClusterId(); - - PowerMock.replayAll(); - - worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, noneConnectorClientConfigOverridePolicy); - worker.herder = herder; - worker.start(); - - assertStatistics(worker, 0, 0); - assertEquals(Collections.emptySet(), worker.connectorNames()); - FutureCallback onStart = new FutureCallback<>(); - worker.startConnector(CONNECTOR_ID, connectorProps, ctx, connectorStatusListener, TargetState.STARTED, onStart); - // Wait for the connector to actually start - assertEquals(TargetState.STARTED, onStart.get(1000, TimeUnit.MILLISECONDS)); - assertEquals(new HashSet<>(Arrays.asList(CONNECTOR_ID)), worker.connectorNames()); - assertStatistics(worker, 1, 0); - assertStartupStatistics(worker, 1, 0, 0, 0); - - worker.stopAndAwaitConnector(CONNECTOR_ID); - assertStatistics(worker, 0, 0); - assertStartupStatistics(worker, 1, 0, 0, 0); - assertEquals(Collections.emptySet(), worker.connectorNames()); - // Nothing should be left, so this should effectively be a nop - worker.stop(); - assertStatistics(worker, 0, 0); - assertStartupStatistics(worker, 1, 0, 0, 0); - - PowerMock.verifyAll(); - } - - @Test - public void testAddConnectorByShortAlias() throws Throwable { - expectConverters(); - expectStartStorage(); - expectFileConfigProvider(); - - final String shortConnectorAlias = "WorkerTest"; - - EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader).times(2); - EasyMock.expect(plugins.delegatingLoader()).andReturn(delegatingLoader); - EasyMock.expect(delegatingLoader.connectorLoader(shortConnectorAlias)).andReturn(pluginLoader); - EasyMock.expect(plugins.newConnector(shortConnectorAlias)).andReturn(connector); - EasyMock.expect(connector.version()).andReturn("1.0"); - - connectorProps.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, shortConnectorAlias); - - EasyMock.expect(connector.version()).andReturn("1.0"); - EasyMock.expect(Plugins.compareAndSwapLoaders(pluginLoader)) - .andReturn(delegatingLoader) - .times(3); - connector.initialize(anyObject(ConnectorContext.class)); - EasyMock.expectLastCall(); - connector.start(connectorProps); - EasyMock.expectLastCall(); - - EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)) - .andReturn(pluginLoader) - .times(3); - - connectorStatusListener.onStartup(CONNECTOR_ID); - EasyMock.expectLastCall(); - - // Remove - connector.stop(); - EasyMock.expectLastCall(); - - connectorStatusListener.onShutdown(CONNECTOR_ID); - EasyMock.expectLastCall(); - - ctx.close(); - expectLastCall(); - - expectStopStorage(); - expectClusterId(); - - PowerMock.replayAll(); - - worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, noneConnectorClientConfigOverridePolicy); - worker.herder = herder; - worker.start(); - - assertStatistics(worker, 0, 0); - assertEquals(Collections.emptySet(), worker.connectorNames()); - FutureCallback onStart = new FutureCallback<>(); - worker.startConnector(CONNECTOR_ID, connectorProps, ctx, connectorStatusListener, TargetState.STARTED, onStart); - // Wait for the connector to actually start - assertEquals(TargetState.STARTED, onStart.get(1000, TimeUnit.MILLISECONDS)); - assertEquals(new HashSet<>(Arrays.asList(CONNECTOR_ID)), worker.connectorNames()); - assertStatistics(worker, 1, 0); - - worker.stopAndAwaitConnector(CONNECTOR_ID); - assertStatistics(worker, 0, 0); - assertEquals(Collections.emptySet(), worker.connectorNames()); - // Nothing should be left, so this should effectively be a nop - worker.stop(); - assertStatistics(worker, 0, 0); - - PowerMock.verifyAll(); - } - - @Test - public void testStopInvalidConnector() { - expectConverters(); - expectStartStorage(); - expectFileConfigProvider(); - expectClusterId(); - - PowerMock.replayAll(); - - worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, noneConnectorClientConfigOverridePolicy); - worker.herder = herder; - worker.start(); - - worker.stopAndAwaitConnector(CONNECTOR_ID); - - PowerMock.verifyAll(); - } - - @Test - public void testReconfigureConnectorTasks() throws Throwable { - expectConverters(); - expectStartStorage(); - expectFileConfigProvider(); - - final String connectorClass = WorkerTest.WorkerTestConnector.class.getName(); - - EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader).times(3); - EasyMock.expect(plugins.delegatingLoader()).andReturn(delegatingLoader).times(1); - EasyMock.expect(delegatingLoader.connectorLoader(connectorClass)).andReturn(pluginLoader); - EasyMock.expect(plugins.newConnector(connectorClass)) - .andReturn(connector); - EasyMock.expect(connector.version()).andReturn("1.0"); - - connectorProps.put(SinkConnectorConfig.TOPICS_CONFIG, "foo,bar"); - connectorProps.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, connectorClass); - - EasyMock.expect(connector.version()).andReturn("1.0"); - EasyMock.expect(Plugins.compareAndSwapLoaders(pluginLoader)) - .andReturn(delegatingLoader) - .times(4); - connector.initialize(anyObject(ConnectorContext.class)); - EasyMock.expectLastCall(); - connector.start(connectorProps); - EasyMock.expectLastCall(); - - EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)) - .andReturn(pluginLoader) - .times(4); - - connectorStatusListener.onStartup(CONNECTOR_ID); - EasyMock.expectLastCall(); - - // Reconfigure - EasyMock.>expect(connector.taskClass()).andReturn(TestSourceTask.class); - Map taskProps = new HashMap<>(); - taskProps.put("foo", "bar"); - EasyMock.expect(connector.taskConfigs(2)).andReturn(Arrays.asList(taskProps, taskProps)); - - // Remove - connector.stop(); - EasyMock.expectLastCall(); - - connectorStatusListener.onShutdown(CONNECTOR_ID); - EasyMock.expectLastCall(); - - ctx.close(); - expectLastCall(); - - expectStopStorage(); - expectClusterId(); - - PowerMock.replayAll(); - - worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, noneConnectorClientConfigOverridePolicy); - worker.herder = herder; - worker.start(); - - assertStatistics(worker, 0, 0); - assertEquals(Collections.emptySet(), worker.connectorNames()); - FutureCallback onFirstStart = new FutureCallback<>(); - worker.startConnector(CONNECTOR_ID, connectorProps, ctx, connectorStatusListener, TargetState.STARTED, onFirstStart); - // Wait for the connector to actually start - assertEquals(TargetState.STARTED, onFirstStart.get(1000, TimeUnit.MILLISECONDS)); - assertStatistics(worker, 1, 0); - assertEquals(new HashSet<>(Arrays.asList(CONNECTOR_ID)), worker.connectorNames()); - - FutureCallback onSecondStart = new FutureCallback<>(); - worker.startConnector(CONNECTOR_ID, connectorProps, ctx, connectorStatusListener, TargetState.STARTED, onSecondStart); - try { - onSecondStart.get(0, TimeUnit.MILLISECONDS); - fail("Should have failed while trying to start second connector with same name"); - } catch (ExecutionException e) { - assertThat(e.getCause(), instanceOf(ConnectException.class)); - } - - Map connProps = new HashMap<>(connectorProps); - connProps.put(ConnectorConfig.TASKS_MAX_CONFIG, "2"); - ConnectorConfig connConfig = new SinkConnectorConfig(plugins, connProps); - List> taskConfigs = worker.connectorTaskConfigs(CONNECTOR_ID, connConfig); - Map expectedTaskProps = new HashMap<>(); - expectedTaskProps.put("foo", "bar"); - expectedTaskProps.put(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName()); - expectedTaskProps.put(SinkTask.TOPICS_CONFIG, "foo,bar"); - assertEquals(2, taskConfigs.size()); - assertEquals(expectedTaskProps, taskConfigs.get(0)); - assertEquals(expectedTaskProps, taskConfigs.get(1)); - assertStatistics(worker, 1, 0); - assertStartupStatistics(worker, 1, 0, 0, 0); - worker.stopAndAwaitConnector(CONNECTOR_ID); - assertStatistics(worker, 0, 0); - assertStartupStatistics(worker, 1, 0, 0, 0); - assertEquals(Collections.emptySet(), worker.connectorNames()); - // Nothing should be left, so this should effectively be a nop - worker.stop(); - assertStatistics(worker, 0, 0); - - PowerMock.verifyAll(); - } - - @Test - public void testAddRemoveTask() throws Exception { - expectConverters(); - expectStartStorage(); - expectFileConfigProvider(); - - EasyMock.expect(workerTask.id()).andStubReturn(TASK_ID); - - EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader).times(2); - expectNewWorkerTask(); - Map origProps = new HashMap<>(); - origProps.put(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName()); - - TaskConfig taskConfig = new TaskConfig(origProps); - // We should expect this call, but the pluginLoader being swapped in is only mocked. - // EasyMock.expect(pluginLoader.loadClass(TestSourceTask.class.getName())) - // .andReturn((Class) TestSourceTask.class); - EasyMock.expect(plugins.newTask(TestSourceTask.class)).andReturn(task); - EasyMock.expect(task.version()).andReturn("1.0"); - - workerTask.initialize(taskConfig); - EasyMock.expectLastCall(); - - // Expect that the worker will create converters and will find them using the current classloader ... - assertNotNull(taskKeyConverter); - assertNotNull(taskValueConverter); - assertNotNull(taskHeaderConverter); - expectTaskKeyConverters(ClassLoaderUsage.CURRENT_CLASSLOADER, taskKeyConverter); - expectTaskValueConverters(ClassLoaderUsage.CURRENT_CLASSLOADER, taskValueConverter); - expectTaskHeaderConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, taskHeaderConverter); - - EasyMock.expect(executorService.submit(workerTask)).andReturn(null); - - EasyMock.expect(plugins.delegatingLoader()).andReturn(delegatingLoader); - EasyMock.expect(delegatingLoader.connectorLoader(WorkerTestConnector.class.getName())) - .andReturn(pluginLoader); - EasyMock.expect(Plugins.compareAndSwapLoaders(pluginLoader)).andReturn(delegatingLoader) - .times(2); - - EasyMock.expect(workerTask.loader()).andReturn(pluginLoader); - - EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)).andReturn(pluginLoader) - .times(2); - plugins.connectorClass(WorkerTestConnector.class.getName()); - EasyMock.expectLastCall().andReturn(WorkerTestConnector.class); - // Remove - workerTask.stop(); - EasyMock.expectLastCall(); - EasyMock.expect(workerTask.awaitStop(EasyMock.anyLong())).andStubReturn(true); - EasyMock.expectLastCall(); - - workerTask.removeMetrics(); - EasyMock.expectLastCall(); - - expectStopStorage(); - expectClusterId(); - - PowerMock.replayAll(); - - worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, executorService, - noneConnectorClientConfigOverridePolicy); - worker.herder = herder; - worker.start(); - assertStatistics(worker, 0, 0); - assertStartupStatistics(worker, 0, 0, 0, 0); - assertEquals(Collections.emptySet(), worker.taskIds()); - worker.startTask(TASK_ID, ClusterConfigState.EMPTY, anyConnectorConfigMap(), origProps, taskStatusListener, TargetState.STARTED); - assertStatistics(worker, 0, 1); - assertStartupStatistics(worker, 0, 0, 1, 0); - assertEquals(new HashSet<>(Arrays.asList(TASK_ID)), worker.taskIds()); - worker.stopAndAwaitTask(TASK_ID); - assertStatistics(worker, 0, 0); - assertStartupStatistics(worker, 0, 0, 1, 0); - assertEquals(Collections.emptySet(), worker.taskIds()); - // Nothing should be left, so this should effectively be a nop - worker.stop(); - assertStatistics(worker, 0, 0); - assertStartupStatistics(worker, 0, 0, 1, 0); - - PowerMock.verifyAll(); - } - - @Test - public void testTaskStatusMetricsStatuses() throws Exception { - expectConverters(); - expectStartStorage(); - expectFileConfigProvider(); - - EasyMock.expect(workerTask.id()).andStubReturn(TASK_ID); - - EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader).times(2); - expectNewWorkerTask(); - Map origProps = new HashMap<>(); - origProps.put(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName()); - - TaskConfig taskConfig = new TaskConfig(origProps); - // We should expect this call, but the pluginLoader being swapped in is only mocked. - // EasyMock.expect(pluginLoader.loadClass(TestSourceTask.class.getName())) - // .andReturn((Class) TestSourceTask.class); - EasyMock.expect(plugins.newTask(TestSourceTask.class)).andReturn(task); - EasyMock.expect(task.version()).andReturn("1.0"); - - workerTask.initialize(taskConfig); - EasyMock.expectLastCall(); - - // Expect that the worker will create converters and will find them using the current classloader ... - assertNotNull(taskKeyConverter); - assertNotNull(taskValueConverter); - assertNotNull(taskHeaderConverter); - expectTaskKeyConverters(ClassLoaderUsage.CURRENT_CLASSLOADER, taskKeyConverter); - expectTaskValueConverters(ClassLoaderUsage.CURRENT_CLASSLOADER, taskValueConverter); - expectTaskHeaderConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, taskHeaderConverter); - - EasyMock.expect(executorService.submit(workerTask)).andReturn(null); - - EasyMock.expect(plugins.delegatingLoader()).andReturn(delegatingLoader); - EasyMock.expect(delegatingLoader.connectorLoader(WorkerTestConnector.class.getName())) - .andReturn(pluginLoader); - EasyMock.expect(Plugins.compareAndSwapLoaders(pluginLoader)).andReturn(delegatingLoader) - .times(2); - - EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)).andReturn(pluginLoader) - .times(2); - plugins.connectorClass(WorkerTestConnector.class.getName()); - EasyMock.expectLastCall().andReturn(WorkerTestConnector.class); - - EasyMock.expect(workerTask.awaitStop(EasyMock.anyLong())).andStubReturn(true); - EasyMock.expectLastCall(); - - workerTask.removeMetrics(); - EasyMock.expectLastCall(); - - // Each time we check the task metrics, the worker will call the herder - herder.taskStatus(TASK_ID); - EasyMock.expectLastCall() - .andReturn(new ConnectorStateInfo.TaskState(0, "RUNNING", "worker", "msg")); - - herder.taskStatus(TASK_ID); - EasyMock.expectLastCall() - .andReturn(new ConnectorStateInfo.TaskState(0, "PAUSED", "worker", "msg")); - - herder.taskStatus(TASK_ID); - EasyMock.expectLastCall() - .andReturn(new ConnectorStateInfo.TaskState(0, "FAILED", "worker", "msg")); - - herder.taskStatus(TASK_ID); - EasyMock.expectLastCall() - .andReturn(new ConnectorStateInfo.TaskState(0, "DESTROYED", "worker", "msg")); - - herder.taskStatus(TASK_ID); - EasyMock.expectLastCall() - .andReturn(new ConnectorStateInfo.TaskState(0, "UNASSIGNED", "worker", "msg")); - - // Called when we stop the worker - EasyMock.expect(workerTask.loader()).andReturn(pluginLoader); - workerTask.stop(); - EasyMock.expectLastCall(); - expectClusterId(); - - PowerMock.replayAll(); - - worker = new Worker(WORKER_ID, - new MockTime(), - plugins, - config, - offsetBackingStore, - executorService, - noneConnectorClientConfigOverridePolicy); - - worker.herder = herder; - - worker.start(); - assertStatistics(worker, 0, 0); - assertStartupStatistics(worker, 0, 0, 0, 0); - assertEquals(Collections.emptySet(), worker.taskIds()); - worker.startTask( - TASK_ID, - ClusterConfigState.EMPTY, - anyConnectorConfigMap(), - origProps, - taskStatusListener, - TargetState.STARTED); - - assertStatusMetrics(1L, "connector-running-task-count"); - assertStatusMetrics(1L, "connector-paused-task-count"); - assertStatusMetrics(1L, "connector-failed-task-count"); - assertStatusMetrics(1L, "connector-destroyed-task-count"); - assertStatusMetrics(1L, "connector-unassigned-task-count"); - - worker.stopAndAwaitTask(TASK_ID); - assertStatusMetrics(0L, "connector-running-task-count"); - assertStatusMetrics(0L, "connector-paused-task-count"); - assertStatusMetrics(0L, "connector-failed-task-count"); - assertStatusMetrics(0L, "connector-destroyed-task-count"); - assertStatusMetrics(0L, "connector-unassigned-task-count"); - - PowerMock.verifyAll(); - } - - @Test - public void testConnectorStatusMetricsGroup_taskStatusCounter() { - ConcurrentMap tasks = new ConcurrentHashMap<>(); - tasks.put(new ConnectorTaskId("c1", 0), workerTask); - tasks.put(new ConnectorTaskId("c1", 1), workerTask); - tasks.put(new ConnectorTaskId("c2", 0), workerTask); - - expectConverters(); - expectStartStorage(); - expectFileConfigProvider(); - - EasyMock.expect(Plugins.compareAndSwapLoaders(pluginLoader)).andReturn(delegatingLoader); - EasyMock.expect(Plugins.compareAndSwapLoaders(pluginLoader)).andReturn(delegatingLoader); - - EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)).andReturn(pluginLoader); - - taskStatusListener.onFailure(EasyMock.eq(TASK_ID), EasyMock.anyObject()); - EasyMock.expectLastCall(); - expectClusterId(); - - PowerMock.replayAll(); - - worker = new Worker(WORKER_ID, - new MockTime(), - plugins, - config, - offsetBackingStore, - noneConnectorClientConfigOverridePolicy); - worker.herder = herder; - - Worker.ConnectorStatusMetricsGroup metricGroup = new Worker.ConnectorStatusMetricsGroup( - worker.metrics(), tasks, herder - ); - assertEquals(2L, (long) metricGroup.taskCounter("c1").metricValue(0L)); - assertEquals(1L, (long) metricGroup.taskCounter("c2").metricValue(0L)); - assertEquals(0L, (long) metricGroup.taskCounter("fakeConnector").metricValue(0L)); - } - - @Test - public void testStartTaskFailure() { - expectConverters(); - expectStartStorage(); - expectFileConfigProvider(); - - Map origProps = new HashMap<>(); - origProps.put(TaskConfig.TASK_CLASS_CONFIG, "missing.From.This.Workers.Classpath"); - - EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader); - EasyMock.expect(plugins.delegatingLoader()).andReturn(delegatingLoader); - EasyMock.expect(delegatingLoader.connectorLoader(WorkerTestConnector.class.getName())) - .andReturn(pluginLoader); - - // We would normally expect this since the plugin loader would have been swapped in. However, since we mock out - // all classloader changes, the call actually goes to the normal default classloader. However, this works out - // fine since we just wanted a ClassNotFoundException anyway. - // EasyMock.expect(pluginLoader.loadClass(origProps.get(TaskConfig.TASK_CLASS_CONFIG))) - // .andThrow(new ClassNotFoundException()); - - EasyMock.expect(Plugins.compareAndSwapLoaders(pluginLoader)) - .andReturn(delegatingLoader); - - EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)) - .andReturn(pluginLoader); - - taskStatusListener.onFailure(EasyMock.eq(TASK_ID), EasyMock.anyObject()); - EasyMock.expectLastCall(); - expectClusterId(); - - PowerMock.replayAll(); - - worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, noneConnectorClientConfigOverridePolicy); - worker.herder = herder; - worker.start(); - assertStatistics(worker, 0, 0); - assertStartupStatistics(worker, 0, 0, 0, 0); - - assertFalse(worker.startTask(TASK_ID, ClusterConfigState.EMPTY, anyConnectorConfigMap(), origProps, taskStatusListener, TargetState.STARTED)); - assertStartupStatistics(worker, 0, 0, 1, 1); - - assertStatistics(worker, 0, 0); - assertStartupStatistics(worker, 0, 0, 1, 1); - assertEquals(Collections.emptySet(), worker.taskIds()); - - PowerMock.verifyAll(); - } - - @Test - public void testCleanupTasksOnStop() throws Exception { - expectConverters(); - expectStartStorage(); - expectFileConfigProvider(); - - EasyMock.expect(workerTask.id()).andStubReturn(TASK_ID); - - EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader).times(2); - expectNewWorkerTask(); - Map origProps = new HashMap<>(); - origProps.put(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName()); - - TaskConfig taskConfig = new TaskConfig(origProps); - // We should expect this call, but the pluginLoader being swapped in is only mocked. - // EasyMock.expect(pluginLoader.loadClass(TestSourceTask.class.getName())) - // .andReturn((Class) TestSourceTask.class); - EasyMock.expect(plugins.newTask(TestSourceTask.class)).andReturn(task); - EasyMock.expect(task.version()).andReturn("1.0"); - - workerTask.initialize(taskConfig); - EasyMock.expectLastCall(); - - // Expect that the worker will create converters and will not initially find them using the current classloader ... - assertNotNull(taskKeyConverter); - assertNotNull(taskValueConverter); - assertNotNull(taskHeaderConverter); - expectTaskKeyConverters(ClassLoaderUsage.CURRENT_CLASSLOADER, null); - expectTaskKeyConverters(ClassLoaderUsage.PLUGINS, taskKeyConverter); - expectTaskValueConverters(ClassLoaderUsage.CURRENT_CLASSLOADER, null); - expectTaskValueConverters(ClassLoaderUsage.PLUGINS, taskValueConverter); - expectTaskHeaderConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, null); - expectTaskHeaderConverter(ClassLoaderUsage.PLUGINS, taskHeaderConverter); - - EasyMock.expect(executorService.submit(workerTask)).andReturn(null); - - EasyMock.expect(plugins.delegatingLoader()).andReturn(delegatingLoader); - EasyMock.expect(delegatingLoader.connectorLoader(WorkerTestConnector.class.getName())) - .andReturn(pluginLoader); - - EasyMock.expect(Plugins.compareAndSwapLoaders(pluginLoader)).andReturn(delegatingLoader) - .times(2); - - EasyMock.expect(workerTask.loader()).andReturn(pluginLoader); - - EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)).andReturn(pluginLoader) - .times(2); - plugins.connectorClass(WorkerTestConnector.class.getName()); - EasyMock.expectLastCall().andReturn(WorkerTestConnector.class); - // Remove on Worker.stop() - workerTask.stop(); - EasyMock.expectLastCall(); - - EasyMock.expect(workerTask.awaitStop(EasyMock.anyLong())).andReturn(true); - // Note that in this case we *do not* commit offsets since it's an unclean shutdown - EasyMock.expectLastCall(); - - workerTask.removeMetrics(); - EasyMock.expectLastCall(); - - expectStopStorage(); - expectClusterId(); - - PowerMock.replayAll(); - - worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, executorService, - noneConnectorClientConfigOverridePolicy); - worker.herder = herder; - worker.start(); - assertStatistics(worker, 0, 0); - worker.startTask(TASK_ID, ClusterConfigState.EMPTY, anyConnectorConfigMap(), origProps, taskStatusListener, TargetState.STARTED); - assertStatistics(worker, 0, 1); - worker.stop(); - assertStatistics(worker, 0, 0); - - PowerMock.verifyAll(); - } - - @Test - public void testConverterOverrides() throws Exception { - expectConverters(); - expectStartStorage(); - expectFileConfigProvider(); - - EasyMock.expect(workerTask.id()).andStubReturn(TASK_ID); - - EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader).times(2); - expectNewWorkerTask(); - Map origProps = new HashMap<>(); - origProps.put(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName()); - - TaskConfig taskConfig = new TaskConfig(origProps); - // We should expect this call, but the pluginLoader being swapped in is only mocked. - // EasyMock.expect(pluginLoader.loadClass(TestSourceTask.class.getName())) - // .andReturn((Class) TestSourceTask.class); - EasyMock.expect(plugins.newTask(TestSourceTask.class)).andReturn(task); - EasyMock.expect(task.version()).andReturn("1.0"); - - workerTask.initialize(taskConfig); - EasyMock.expectLastCall(); - - // Expect that the worker will create converters and will not initially find them using the current classloader ... - assertNotNull(taskKeyConverter); - assertNotNull(taskValueConverter); - assertNotNull(taskHeaderConverter); - expectTaskKeyConverters(ClassLoaderUsage.CURRENT_CLASSLOADER, null); - expectTaskKeyConverters(ClassLoaderUsage.PLUGINS, taskKeyConverter); - expectTaskValueConverters(ClassLoaderUsage.CURRENT_CLASSLOADER, null); - expectTaskValueConverters(ClassLoaderUsage.PLUGINS, taskValueConverter); - expectTaskHeaderConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, null); - expectTaskHeaderConverter(ClassLoaderUsage.PLUGINS, taskHeaderConverter); - - EasyMock.expect(executorService.submit(workerTask)).andReturn(null); - - EasyMock.expect(plugins.delegatingLoader()).andReturn(delegatingLoader); - EasyMock.expect(delegatingLoader.connectorLoader(WorkerTestConnector.class.getName())) - .andReturn(pluginLoader); - - EasyMock.expect(Plugins.compareAndSwapLoaders(pluginLoader)).andReturn(delegatingLoader) - .times(2); - - EasyMock.expect(workerTask.loader()).andReturn(pluginLoader); - - EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)).andReturn(pluginLoader) - .times(2); - plugins.connectorClass(WorkerTestConnector.class.getName()); - EasyMock.expectLastCall().andReturn(WorkerTestConnector.class); - - // Remove - workerTask.stop(); - EasyMock.expectLastCall(); - EasyMock.expect(workerTask.awaitStop(EasyMock.anyLong())).andStubReturn(true); - EasyMock.expectLastCall(); - - workerTask.removeMetrics(); - EasyMock.expectLastCall(); - - expectStopStorage(); - expectClusterId(); - - PowerMock.replayAll(); - - worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, executorService, - noneConnectorClientConfigOverridePolicy); - worker.herder = herder; - worker.start(); - assertStatistics(worker, 0, 0); - assertEquals(Collections.emptySet(), worker.taskIds()); - Map connProps = anyConnectorConfigMap(); - connProps.put(ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG, TestConverter.class.getName()); - connProps.put("key.converter.extra.config", "foo"); - connProps.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, TestConfigurableConverter.class.getName()); - connProps.put("value.converter.extra.config", "bar"); - worker.startTask(TASK_ID, ClusterConfigState.EMPTY, connProps, origProps, taskStatusListener, TargetState.STARTED); - assertStatistics(worker, 0, 1); - assertEquals(new HashSet<>(Arrays.asList(TASK_ID)), worker.taskIds()); - worker.stopAndAwaitTask(TASK_ID); - assertStatistics(worker, 0, 0); - assertEquals(Collections.emptySet(), worker.taskIds()); - // Nothing should be left, so this should effectively be a nop - worker.stop(); - assertStatistics(worker, 0, 0); - - // We've mocked the Plugin.newConverter method, so we don't currently configure the converters - - PowerMock.verifyAll(); - } - - @Test - public void testProducerConfigsWithoutOverrides() { - EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX)).andReturn( - new HashMap<>()); - PowerMock.replayAll(); - Map expectedConfigs = new HashMap<>(defaultProducerConfigs); - expectedConfigs.put("client.id", "connector-producer-job-0"); - expectedConfigs.put("metrics.context.connect.kafka.cluster.id", "test-cluster"); - assertEquals(expectedConfigs, - Worker.producerConfigs(TASK_ID, "connector-producer-" + TASK_ID, config, connectorConfig, null, noneConnectorClientConfigOverridePolicy, CLUSTER_ID)); - } - - @Test - public void testProducerConfigsWithOverrides() { - Map props = new HashMap<>(workerProps); - props.put("producer.acks", "-1"); - props.put("producer.linger.ms", "1000"); - props.put("producer.client.id", "producer-test-id"); - WorkerConfig configWithOverrides = new StandaloneConfig(props); - - Map expectedConfigs = new HashMap<>(defaultProducerConfigs); - expectedConfigs.put("acks", "-1"); - expectedConfigs.put("linger.ms", "1000"); - expectedConfigs.put("client.id", "producer-test-id"); - expectedConfigs.put("metrics.context.connect.kafka.cluster.id", "test-cluster"); - EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX)).andReturn( - new HashMap<>()); - PowerMock.replayAll(); - assertEquals(expectedConfigs, - Worker.producerConfigs(TASK_ID, "connector-producer-" + TASK_ID, configWithOverrides, connectorConfig, null, allConnectorClientConfigOverridePolicy, CLUSTER_ID)); - } - - @Test - public void testProducerConfigsWithClientOverrides() { - Map props = new HashMap<>(workerProps); - props.put("producer.acks", "-1"); - props.put("producer.linger.ms", "1000"); - props.put("producer.client.id", "producer-test-id"); - WorkerConfig configWithOverrides = new StandaloneConfig(props); - - Map expectedConfigs = new HashMap<>(defaultProducerConfigs); - expectedConfigs.put("acks", "-1"); - expectedConfigs.put("linger.ms", "5000"); - expectedConfigs.put("batch.size", "1000"); - expectedConfigs.put("client.id", "producer-test-id"); - expectedConfigs.put("metrics.context.connect.kafka.cluster.id", "test-cluster"); - Map connConfig = new HashMap<>(); - connConfig.put("linger.ms", "5000"); - connConfig.put("batch.size", "1000"); - EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX)) - .andReturn(connConfig); - PowerMock.replayAll(); - assertEquals(expectedConfigs, - Worker.producerConfigs(TASK_ID, "connector-producer-" + TASK_ID, configWithOverrides, connectorConfig, null, allConnectorClientConfigOverridePolicy, CLUSTER_ID)); - } - - @Test - public void testConsumerConfigsWithoutOverrides() { - Map expectedConfigs = new HashMap<>(defaultConsumerConfigs); - expectedConfigs.put("group.id", "connect-test"); - expectedConfigs.put("client.id", "connector-consumer-test-1"); - expectedConfigs.put("metrics.context.connect.kafka.cluster.id", "test-cluster"); - EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX)).andReturn(new HashMap<>()); - PowerMock.replayAll(); - assertEquals(expectedConfigs, Worker.consumerConfigs(new ConnectorTaskId("test", 1), config, connectorConfig, - null, noneConnectorClientConfigOverridePolicy, CLUSTER_ID)); - } - - @Test - public void testConsumerConfigsWithOverrides() { - Map props = new HashMap<>(workerProps); - props.put("consumer.auto.offset.reset", "latest"); - props.put("consumer.max.poll.records", "1000"); - props.put("consumer.client.id", "consumer-test-id"); - WorkerConfig configWithOverrides = new StandaloneConfig(props); - - Map expectedConfigs = new HashMap<>(defaultConsumerConfigs); - expectedConfigs.put("group.id", "connect-test"); - expectedConfigs.put("auto.offset.reset", "latest"); - expectedConfigs.put("max.poll.records", "1000"); - expectedConfigs.put("client.id", "consumer-test-id"); - expectedConfigs.put("metrics.context.connect.kafka.cluster.id", "test-cluster"); - EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX)).andReturn(new HashMap<>()); - PowerMock.replayAll(); - assertEquals(expectedConfigs, Worker.consumerConfigs(new ConnectorTaskId("test", 1), configWithOverrides, connectorConfig, - null, noneConnectorClientConfigOverridePolicy, CLUSTER_ID)); - - } - - @Test - public void testConsumerConfigsWithClientOverrides() { - Map props = new HashMap<>(workerProps); - props.put("consumer.auto.offset.reset", "latest"); - props.put("consumer.max.poll.records", "5000"); - WorkerConfig configWithOverrides = new StandaloneConfig(props); - - Map expectedConfigs = new HashMap<>(defaultConsumerConfigs); - expectedConfigs.put("group.id", "connect-test"); - expectedConfigs.put("auto.offset.reset", "latest"); - expectedConfigs.put("max.poll.records", "5000"); - expectedConfigs.put("max.poll.interval.ms", "1000"); - expectedConfigs.put("client.id", "connector-consumer-test-1"); - expectedConfigs.put("metrics.context.connect.kafka.cluster.id", "test-cluster"); - Map connConfig = new HashMap<>(); - connConfig.put("max.poll.records", "5000"); - connConfig.put("max.poll.interval.ms", "1000"); - EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX)) - .andReturn(connConfig); - PowerMock.replayAll(); - assertEquals(expectedConfigs, Worker.consumerConfigs(new ConnectorTaskId("test", 1), configWithOverrides, connectorConfig, - null, allConnectorClientConfigOverridePolicy, CLUSTER_ID)); - } - - @Test - public void testConsumerConfigsClientOverridesWithNonePolicy() { - Map props = new HashMap<>(workerProps); - props.put("consumer.auto.offset.reset", "latest"); - props.put("consumer.max.poll.records", "5000"); - WorkerConfig configWithOverrides = new StandaloneConfig(props); - - Map connConfig = new HashMap<>(); - connConfig.put("max.poll.records", "5000"); - connConfig.put("max.poll.interval.ms", "1000"); - EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX)) - .andReturn(connConfig); - PowerMock.replayAll(); - assertThrows(ConnectException.class, () -> Worker.consumerConfigs(new ConnectorTaskId("test", 1), - configWithOverrides, connectorConfig, null, noneConnectorClientConfigOverridePolicy, CLUSTER_ID)); - } - - @Test - public void testAdminConfigsClientOverridesWithAllPolicy() { - Map props = new HashMap<>(workerProps); - props.put("admin.client.id", "testid"); - props.put("admin.metadata.max.age.ms", "5000"); - props.put("producer.bootstrap.servers", "cbeauho.com"); - props.put("consumer.bootstrap.servers", "localhost:4761"); - WorkerConfig configWithOverrides = new StandaloneConfig(props); - - Map connConfig = new HashMap<>(); - connConfig.put("metadata.max.age.ms", "10000"); - - Map expectedConfigs = new HashMap<>(workerProps); - - expectedConfigs.put("bootstrap.servers", "localhost:9092"); - expectedConfigs.put("client.id", "testid"); - expectedConfigs.put("metadata.max.age.ms", "10000"); - expectedConfigs.put("metrics.context.connect.kafka.cluster.id", "test-cluster"); - - EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_ADMIN_OVERRIDES_PREFIX)) - .andReturn(connConfig); - PowerMock.replayAll(); - assertEquals(expectedConfigs, Worker.adminConfigs(new ConnectorTaskId("test", 1), "", configWithOverrides, connectorConfig, - null, allConnectorClientConfigOverridePolicy, CLUSTER_ID)); - } - - @Test - public void testAdminConfigsClientOverridesWithNonePolicy() { - Map props = new HashMap<>(workerProps); - props.put("admin.client.id", "testid"); - props.put("admin.metadata.max.age.ms", "5000"); - WorkerConfig configWithOverrides = new StandaloneConfig(props); - - Map connConfig = new HashMap<>(); - connConfig.put("metadata.max.age.ms", "10000"); - - EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_ADMIN_OVERRIDES_PREFIX)) - .andReturn(connConfig); - PowerMock.replayAll(); - assertThrows(ConnectException.class, () -> Worker.adminConfigs(new ConnectorTaskId("test", 1), - "", configWithOverrides, connectorConfig, null, noneConnectorClientConfigOverridePolicy, CLUSTER_ID)); - } - - private void assertStatusMetrics(long expected, String metricName) { - MetricGroup statusMetrics = worker.connectorStatusMetricsGroup().metricGroup(TASK_ID.connector()); - if (expected == 0L) { - assertNull(statusMetrics); - return; - } - assertEquals(expected, MockConnectMetrics.currentMetricValue(worker.metrics(), statusMetrics, metricName)); - } - - private void assertStatistics(Worker worker, int connectors, int tasks) { - assertStatusMetrics(tasks, "connector-total-task-count"); - MetricGroup workerMetrics = worker.workerMetricsGroup().metricGroup(); - assertEquals(connectors, MockConnectMetrics.currentMetricValueAsDouble(worker.metrics(), workerMetrics, "connector-count"), 0.0001d); - assertEquals(tasks, MockConnectMetrics.currentMetricValueAsDouble(worker.metrics(), workerMetrics, "task-count"), 0.0001d); - assertEquals(tasks, MockConnectMetrics.currentMetricValueAsDouble(worker.metrics(), workerMetrics, "task-count"), 0.0001d); - } - - private void assertStartupStatistics(Worker worker, int connectorStartupAttempts, int connectorStartupFailures, int taskStartupAttempts, int taskStartupFailures) { - double connectStartupSuccesses = connectorStartupAttempts - connectorStartupFailures; - double taskStartupSuccesses = taskStartupAttempts - taskStartupFailures; - double connectStartupSuccessPct = 0.0d; - double connectStartupFailurePct = 0.0d; - double taskStartupSuccessPct = 0.0d; - double taskStartupFailurePct = 0.0d; - if (connectorStartupAttempts != 0) { - connectStartupSuccessPct = connectStartupSuccesses / connectorStartupAttempts; - connectStartupFailurePct = (double) connectorStartupFailures / connectorStartupAttempts; - } - if (taskStartupAttempts != 0) { - taskStartupSuccessPct = taskStartupSuccesses / taskStartupAttempts; - taskStartupFailurePct = (double) taskStartupFailures / taskStartupAttempts; - } - MetricGroup workerMetrics = worker.workerMetricsGroup().metricGroup(); - assertEquals(connectorStartupAttempts, MockConnectMetrics.currentMetricValueAsDouble(worker.metrics(), workerMetrics, "connector-startup-attempts-total"), 0.0001d); - assertEquals(connectStartupSuccesses, MockConnectMetrics.currentMetricValueAsDouble(worker.metrics(), workerMetrics, "connector-startup-success-total"), 0.0001d); - assertEquals(connectorStartupFailures, MockConnectMetrics.currentMetricValueAsDouble(worker.metrics(), workerMetrics, "connector-startup-failure-total"), 0.0001d); - assertEquals(connectStartupSuccessPct, MockConnectMetrics.currentMetricValueAsDouble(worker.metrics(), workerMetrics, "connector-startup-success-percentage"), 0.0001d); - assertEquals(connectStartupFailurePct, MockConnectMetrics.currentMetricValueAsDouble(worker.metrics(), workerMetrics, "connector-startup-failure-percentage"), 0.0001d); - assertEquals(taskStartupAttempts, MockConnectMetrics.currentMetricValueAsDouble(worker.metrics(), workerMetrics, "task-startup-attempts-total"), 0.0001d); - assertEquals(taskStartupSuccesses, MockConnectMetrics.currentMetricValueAsDouble(worker.metrics(), workerMetrics, "task-startup-success-total"), 0.0001d); - assertEquals(taskStartupFailures, MockConnectMetrics.currentMetricValueAsDouble(worker.metrics(), workerMetrics, "task-startup-failure-total"), 0.0001d); - assertEquals(taskStartupSuccessPct, MockConnectMetrics.currentMetricValueAsDouble(worker.metrics(), workerMetrics, "task-startup-success-percentage"), 0.0001d); - assertEquals(taskStartupFailurePct, MockConnectMetrics.currentMetricValueAsDouble(worker.metrics(), workerMetrics, "task-startup-failure-percentage"), 0.0001d); - } - - private void expectStartStorage() { - offsetBackingStore.configure(anyObject(WorkerConfig.class)); - EasyMock.expectLastCall(); - offsetBackingStore.start(); - EasyMock.expectLastCall(); - EasyMock.expect(herder.statusBackingStore()) - .andReturn(statusBackingStore).anyTimes(); - } - - private void expectStopStorage() { - offsetBackingStore.stop(); - EasyMock.expectLastCall(); - } - - private void expectConverters() { - expectConverters(JsonConverter.class, false); - } - - private void expectConverters(Boolean expectDefaultConverters) { - expectConverters(JsonConverter.class, expectDefaultConverters); - } - - @SuppressWarnings("deprecation") - private void expectConverters(Class converterClass, Boolean expectDefaultConverters) { - // As default converters are instantiated when a task starts, they are expected only if the `startTask` method is called - if (expectDefaultConverters) { - - // Instantiate and configure default - EasyMock.expect(plugins.newConverter(config, WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, ClassLoaderUsage.PLUGINS)) - .andReturn(keyConverter); - EasyMock.expect(plugins.newConverter(config, WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, ClassLoaderUsage.PLUGINS)) - .andReturn(valueConverter); - EasyMock.expectLastCall(); - } - - //internal - Converter internalKeyConverter = PowerMock.createMock(converterClass); - Converter internalValueConverter = PowerMock.createMock(converterClass); - - // Instantiate and configure internal - EasyMock.expect( - plugins.newConverter( - config, - WorkerConfig.INTERNAL_KEY_CONVERTER_CLASS_CONFIG, - ClassLoaderUsage.PLUGINS - ) - ).andReturn(internalKeyConverter); - EasyMock.expect( - plugins.newConverter( - config, - WorkerConfig.INTERNAL_VALUE_CONVERTER_CLASS_CONFIG, - ClassLoaderUsage.PLUGINS - ) - ).andReturn(internalValueConverter); - EasyMock.expectLastCall(); - } - - private void expectTaskKeyConverters(ClassLoaderUsage classLoaderUsage, Converter returning) { - EasyMock.expect( - plugins.newConverter( - anyObject(AbstractConfig.class), - eq(WorkerConfig.KEY_CONVERTER_CLASS_CONFIG), - eq(classLoaderUsage))) - .andReturn(returning); - } - - private void expectTaskValueConverters(ClassLoaderUsage classLoaderUsage, Converter returning) { - EasyMock.expect( - plugins.newConverter( - anyObject(AbstractConfig.class), - eq(WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG), - eq(classLoaderUsage))) - .andReturn(returning); - } - - private void expectTaskHeaderConverter(ClassLoaderUsage classLoaderUsage, HeaderConverter returning) { - EasyMock.expect( - plugins.newHeaderConverter( - anyObject(AbstractConfig.class), - eq(WorkerConfig.HEADER_CONVERTER_CLASS_CONFIG), - eq(classLoaderUsage))) - .andReturn(returning); - } - - private Map anyConnectorConfigMap() { - Map props = new HashMap<>(); - props.put(ConnectorConfig.NAME_CONFIG, CONNECTOR_ID); - props.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, WorkerTestConnector.class.getName()); - props.put(ConnectorConfig.TASKS_MAX_CONFIG, "1"); - props.put(DEFAULT_TOPIC_CREATION_PREFIX + REPLICATION_FACTOR_CONFIG, String.valueOf(1)); - props.put(DEFAULT_TOPIC_CREATION_PREFIX + PARTITIONS_CONFIG, String.valueOf(1)); - return props; - } - - private void expectNewWorkerTask() throws Exception { - PowerMock.expectNew( - WorkerSourceTask.class, EasyMock.eq(TASK_ID), - EasyMock.eq(task), - anyObject(TaskStatus.Listener.class), - EasyMock.eq(TargetState.STARTED), - anyObject(JsonConverter.class), - anyObject(JsonConverter.class), - anyObject(JsonConverter.class), - EasyMock.eq(new TransformationChain<>(Collections.emptyList(), NOOP_OPERATOR)), - anyObject(KafkaProducer.class), - anyObject(TopicAdmin.class), - EasyMock.>anyObject(), - anyObject(OffsetStorageReader.class), - anyObject(OffsetStorageWriter.class), - EasyMock.eq(config), - anyObject(ClusterConfigState.class), - anyObject(ConnectMetrics.class), - EasyMock.eq(pluginLoader), - anyObject(Time.class), - anyObject(RetryWithToleranceOperator.class), - anyObject(StatusBackingStore.class), - anyObject(Executor.class)) - .andReturn(workerTask); - } - - private void expectClusterId() { - PowerMock.mockStaticPartial(ConnectUtils.class, "lookupKafkaClusterId"); - EasyMock.expect(ConnectUtils.lookupKafkaClusterId(EasyMock.anyObject())).andReturn("test-cluster").anyTimes(); - } - - /* Name here needs to be unique as we are testing the aliasing mechanism */ - public static class WorkerTestConnector extends SourceConnector { - - private static final ConfigDef CONFIG_DEF = new ConfigDef() - .define("configName", ConfigDef.Type.STRING, ConfigDef.Importance.HIGH, "Test configName."); - - @Override - public String version() { - return "1.0"; - } - - @Override - public void start(Map props) { - - } - - @Override - public Class taskClass() { - return null; - } - - @Override - public List> taskConfigs(int maxTasks) { - return null; - } - - @Override - public void stop() { - - } - - @Override - public ConfigDef config() { - return CONFIG_DEF; - } - } - - private static class TestSourceTask extends SourceTask { - public TestSourceTask() { - } - - @Override - public String version() { - return "1.0"; - } - - @Override - public void start(Map props) { - } - - @Override - public List poll() throws InterruptedException { - return null; - } - - @Override - public void stop() { - } - } - - public static class TestConverter implements Converter { - public Map configs; - - @Override - public void configure(Map configs, boolean isKey) { - this.configs = configs; - } - - @Override - public byte[] fromConnectData(String topic, Schema schema, Object value) { - return new byte[0]; - } - - @Override - public SchemaAndValue toConnectData(String topic, byte[] value) { - return null; - } - } - - public static class TestConfigurableConverter implements Converter, Configurable { - public Map configs; - - public ConfigDef config() { - return JsonConverterConfig.configDef(); - } - - @Override - public void configure(Map configs) { - this.configs = configs; - new JsonConverterConfig(configs); // requires the `converter.type` config be set - } - - @Override - public void configure(Map configs, boolean isKey) { - this.configs = configs; - } - - @Override - public byte[] fromConnectData(String topic, Schema schema, Object value) { - return new byte[0]; - } - - @Override - public SchemaAndValue toConnectData(String topic, byte[] value) { - return null; - } - } -} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/ParameterizedTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/ParameterizedTest.java new file mode 100644 index 00000000000..5f168913542 --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/ParameterizedTest.java @@ -0,0 +1,83 @@ +/* + * 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.util; + +import java.lang.annotation.Annotation; +import org.junit.runner.Description; +import org.junit.runner.manipulation.Filter; +import org.junit.runner.manipulation.NoTestsRemainException; +import org.junit.runners.Parameterized; + +/** + * Running a single parameterized test causes issue as explained in + * http://youtrack.jetbrains.com/issue/IDEA-65966 and + * https://stackoverflow.com/questions/12798079/initializationerror-with-eclipse-and-junit4-when-executing-a-single-test/18438718#18438718 + * + * As a workaround, the original filter needs to be wrapped and then pass it a deparameterized + * description which removes the parameter part (See deparametrizeName) + */ +public class ParameterizedTest extends Parameterized { + + public ParameterizedTest(Class klass) throws Throwable { + super(klass); + } + + @Override + public void filter(Filter filter) throws NoTestsRemainException { + super.filter(new FilterDecorator(filter)); + } + + private static String deparametrizeName(String name) { + //Each parameter is named as [0], [1] etc + if (name.startsWith("[")) { + return name; + } + + //Convert methodName[index](className) to methodName(className) + int indexOfOpenBracket = name.indexOf('['); + int indexOfCloseBracket = name.indexOf(']') + 1; + return name.substring(0, indexOfOpenBracket).concat(name.substring(indexOfCloseBracket)); + } + + private static Description wrap(Description description) { + String fixedName = deparametrizeName(description.getDisplayName()); + Description clonedDescription = Description.createSuiteDescription( + fixedName, + description.getAnnotations().toArray(new Annotation[0]) + ); + description.getChildren().forEach(child -> clonedDescription.addChild(wrap(child))); + return clonedDescription; + } + + private static class FilterDecorator extends Filter { + private final Filter delegate; + + private FilterDecorator(Filter delegate) { + this.delegate = delegate; + } + + @Override + public boolean shouldRun(Description description) { + return delegate.shouldRun(wrap(description)); + } + + @Override + public String describe() { + return delegate.describe(); + } + } +} \ No newline at end of file