+ * Users should create an instance in a try-with-resources block,
+ * {@link #register(AutoCloseable, String) register} resources that they allocate inside that
+ * block, and optionally {@link #clear() clear} those resources before exiting the block if they
+ * will be used outside it.
+ *
closeables;
+ private ClassLoader loader;
+
+ public Closeables() {
+ closeables = new IdentityHashMap<>();
+ }
+
+ /**
+ * Register a resource to be {@link AutoCloseable#close() closed} when this {@link Closeables}
+ * object is {@link #close() closed}.
+ * @param closeable the closeable resource to track; if null, will be silently ignored
+ * @param name a description of the closeable resource to use for logging; may not be null
+ */
+ public void register(AutoCloseable closeable, String name) {
+ Objects.requireNonNull(name, "name may not be null");
+ if (closeable == null) {
+ log.trace("Ignoring null closeable: {}", name);
+ } else {
+ log.trace("Registering closeable {}: {}", name, closeable);
+ this.closeables.put(closeable, name);
+ }
+ }
+
+ /**
+ * Set a {@link ClassLoader} to switch to when {@link AutoCloseable#close() closing}
+ * resources that have been {@link #register(AutoCloseable, String) registered}.
+ *
+ * May not be invoked more than once.
+ * @param loader the loader to use; may not be null
+ * @throws IllegalStateException if invoked more than once
+ */
+ public void useLoader(ClassLoader loader) {
+ if (this.loader != null) {
+ throw new IllegalStateException("May only define classloader once");
+ }
+ Objects.requireNonNull(loader, "class loader may not be null");
+ this.loader = loader;
+ }
+
+ /**
+ * Forget any resources that have been {@link #register(AutoCloseable, String) registered}
+ * before this call. Note that if a {@link ClassLoader} has been set via
+ * {@link #useLoader(ClassLoader)}, it will remain set, and subsequent calls to
+ * {@link #useLoader(ClassLoader)} will still fail.
+ */
+ public void clear() {
+ closeables.clear();
+ }
+
+ /**
+ * {@link AutoCloseable#close() Close} all resources that have been
+ * {@link #register(AutoCloseable, String) registered}, using the {@link ClassLoader} set via
+ * {@link #useLoader(ClassLoader)} (if one has been set), except those that have been forgotten
+ * by a call to {@link #clear()}.
+ *
+ * If any call to {@link AutoCloseable#close()} fails, the exception is caught, logged, and not
+ * propagated to the caller.
+ */
+ @Override
+ public void close() {
+ if (closeables.isEmpty())
+ return;
+
+ try (Utils.UncheckedCloseable loaderSwap = maybeSwapLoaders()) {
+ closeables.forEach(Utils::closeQuietly);
+ closeables.clear();
+ }
+ }
+
+ private Utils.UncheckedCloseable maybeSwapLoaders() {
+ if (loader != null) {
+ return new LoaderSwap(loader)::close;
+ } else {
+ return () -> { };
+ }
+ }
+
+}
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/util/ConnectUtils.java b/connect/runtime/src/main/java/org/apache/kafka/connect/util/ConnectUtils.java
index 0af14cc7f30ec..f34b175ae7e4b 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/util/ConnectUtils.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/util/ConnectUtils.java
@@ -18,8 +18,8 @@
import org.apache.kafka.clients.CommonClientConfigs;
import org.apache.kafka.clients.admin.Admin;
-import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.InvalidRecordException;
+import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.record.RecordBatch;
import org.apache.kafka.connect.connector.Connector;
import org.apache.kafka.connect.errors.ConnectException;
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 d4184bbab2a52..41a6212c35d0d 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
@@ -248,7 +248,7 @@ public void testSinkTasksCloseErrorReporters() throws Exception {
workerSinkTask.initialize(TASK_CONFIG);
workerSinkTask.initializeAndStart();
- workerSinkTask.close();
+ workerSinkTask.doClose();
PowerMock.verifyAll();
}
@@ -271,7 +271,7 @@ public void testSourceTasksCloseErrorReporters() {
PowerMock.replayAll();
workerSourceTask.initialize(TASK_CONFIG);
- workerSourceTask.close();
+ workerSourceTask.doClose();
PowerMock.verifyAll();
}
@@ -299,7 +299,7 @@ public void testCloseErrorReportersExceptionPropagation() {
PowerMock.replayAll();
workerSourceTask.initialize(TASK_CONFIG);
- workerSourceTask.close();
+ workerSourceTask.doClose();
PowerMock.verifyAll();
}
diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/TestCloseables.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/TestCloseables.java
new file mode 100644
index 0000000000000..8532c96be65a7
--- /dev/null
+++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/TestCloseables.java
@@ -0,0 +1,66 @@
+/*
+ * 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.connect.util.Closeables;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class TestCloseables extends Closeables {
+
+ private final Map, Integer> registered;
+
+ private ClassLoader loader;
+ private boolean cleared;
+
+ public TestCloseables() {
+ super();
+ this.registered = new HashMap<>();
+ }
+
+ @Override
+ public void register(AutoCloseable closeable, String description) {
+ super.register(closeable, description);
+ this.registered.compute(closeable.getClass(), (c, numRegistered) -> numRegistered == null ? 1 : numRegistered + 1);
+ }
+
+ @Override
+ public void useLoader(ClassLoader loader) {
+ super.useLoader(loader);
+ this.loader = loader;
+ }
+
+ @Override
+ public void clear() {
+ super.clear();
+ this.cleared = true;
+ }
+
+ public Map, Integer> registered() {
+ return registered;
+ }
+
+ public boolean cleared() {
+ return cleared;
+ }
+
+ public ClassLoader loader() {
+ return loader;
+ }
+
+}
diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java
index 41d4de5e0a86e..336a1f3626cfb 100644
--- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java
+++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java
@@ -356,7 +356,7 @@ public void testShutdown() throws Exception {
sinkTaskContext.getValue().requestCommit(); // Force an offset commit
workerTask.iteration();
workerTask.stop();
- workerTask.close();
+ workerTask.doClose();
PowerMock.verifyAll();
}
diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java
index eef6ca82be63e..a8ac2a0667aaa 100644
--- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java
+++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java
@@ -175,7 +175,7 @@ public void testPollsInBackground() throws Exception {
workerTask.iteration();
}
workerTask.stop();
- workerTask.close();
+ workerTask.doClose();
// Verify contents match expected values, i.e. that they were translated properly. With max
// batch size 1 and poll returns 1 message at a time, we should have a matching # of batches
@@ -224,7 +224,7 @@ public void testCommit() throws Exception {
// Commit finishes synchronously for testing so we can check this immediately
assertEquals(0, workerTask.commitFailures());
workerTask.stop();
- workerTask.close();
+ workerTask.doClose();
assertEquals(2, capturedRecords.getValues().size());
@@ -265,7 +265,7 @@ public void testCommitFailure() throws Exception {
assertEquals(1, workerTask.commitFailures());
assertEquals(false, Whitebox.getInternalState(workerTask, "committing"));
workerTask.stop();
- workerTask.close();
+ workerTask.doClose();
PowerMock.verifyAll();
}
@@ -307,7 +307,7 @@ public void testCommitSuccessFollowedByFailure() throws Exception {
assertEquals(1, workerTask.commitFailures());
assertEquals(false, Whitebox.getInternalState(workerTask, "committing"));
workerTask.stop();
- workerTask.close();
+ workerTask.doClose();
PowerMock.verifyAll();
}
@@ -340,7 +340,7 @@ public void testCommitConsumerFailure() throws Exception {
assertEquals(1, workerTask.commitFailures());
assertEquals(false, Whitebox.getInternalState(workerTask, "committing"));
workerTask.stop();
- workerTask.close();
+ workerTask.doClose();
PowerMock.verifyAll();
}
@@ -377,7 +377,7 @@ public void testCommitTimeout() throws Exception {
assertEquals(1, workerTask.commitFailures());
assertEquals(false, Whitebox.getInternalState(workerTask, "committing"));
workerTask.stop();
- workerTask.close();
+ workerTask.doClose();
PowerMock.verifyAll();
}
@@ -439,7 +439,7 @@ public void testAssignmentPauseResume() throws Exception {
workerTask.iteration();
workerTask.iteration();
workerTask.stop();
- workerTask.close();
+ workerTask.doClose();
PowerMock.verifyAll();
}
@@ -477,7 +477,7 @@ public void testRewind() throws Exception {
workerTask.iteration();
workerTask.iteration();
workerTask.stop();
- workerTask.close();
+ workerTask.doClose();
PowerMock.verifyAll();
}
@@ -503,7 +503,7 @@ public void testRewindOnRebalanceDuringPoll() throws Exception {
workerTask.iteration();
workerTask.iteration();
workerTask.stop();
- workerTask.close();
+ workerTask.doClose();
PowerMock.verifyAll();
}
diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTaskTest.java
index 890c0f7399c5a..f45880ed3cf4a 100644
--- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTaskTest.java
+++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTaskTest.java
@@ -94,7 +94,7 @@ public void standardStartup() {
.addMockedMethod("initialize")
.addMockedMethod("initializeAndStart")
.addMockedMethod("execute")
- .addMockedMethod("close")
+ .addMockedMethod("doClose")
.createStrictMock();
workerTask.initialize(TASK_CONFIG);
@@ -109,7 +109,7 @@ public void standardStartup() {
statusListener.onStartup(taskId);
expectLastCall();
- workerTask.close();
+ workerTask.doClose();
expectLastCall();
statusListener.onShutdown(taskId);
@@ -144,13 +144,13 @@ public void stopBeforeStarting() {
retryWithToleranceOperator, Time.SYSTEM, statusBackingStore)
.addMockedMethod("initialize")
.addMockedMethod("execute")
- .addMockedMethod("close")
+ .addMockedMethod("doClose")
.createStrictMock();
workerTask.initialize(TASK_CONFIG);
EasyMock.expectLastCall();
- workerTask.close();
+ workerTask.doClose();
EasyMock.expectLastCall();
replay(workerTask);
@@ -185,7 +185,7 @@ public void cancelBeforeStopping() throws Exception {
.addMockedMethod("initialize")
.addMockedMethod("initializeAndStart")
.addMockedMethod("execute")
- .addMockedMethod("close")
+ .addMockedMethod("doClose")
.createStrictMock();
final CountDownLatch stopped = new CountDownLatch(1);
@@ -211,7 +211,7 @@ public void cancelBeforeStopping() throws Exception {
statusListener.onStartup(taskId);
expectLastCall();
- workerTask.close();
+ workerTask.doClose();
expectLastCall();
// there should be no call to onShutdown()
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 a064e296b223c..7f7a8a64d0f2f 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
@@ -21,6 +21,8 @@
import org.apache.kafka.clients.admin.FenceProducersResult;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.MetricName;
@@ -42,6 +44,9 @@
import org.apache.kafka.connect.runtime.MockConnectMetrics.MockMetricsReporter;
import org.apache.kafka.connect.storage.ClusterConfigState;
import org.apache.kafka.connect.runtime.distributed.DistributedConfig;
+import org.apache.kafka.connect.runtime.errors.ErrorHandlingMetrics;
+import org.apache.kafka.connect.runtime.errors.LogReporter;
+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;
@@ -58,6 +63,7 @@
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.OffsetStorageReaderImpl;
import org.apache.kafka.connect.storage.StatusBackingStore;
import org.apache.kafka.connect.util.ConnectUtils;
import org.apache.kafka.connect.util.ConnectorTaskId;
@@ -130,6 +136,7 @@
import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.atLeastOnce;
+import static org.mockito.Mockito.doCallRealMethod;
import static org.mockito.Mockito.doNothing;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.mock;
@@ -304,7 +311,7 @@ public void testStartAndStopConnector() throws Throwable {
connectorProps.put(CONNECTOR_CLASS_CONFIG, connectorClass);
// Create
- when(plugins.currentThreadLoader()).thenReturn(delegatingLoader);
+ expectLoaderSwap();
when(plugins.delegatingLoader()).thenReturn(delegatingLoader);
when(delegatingLoader.connectorLoader(connectorClass)).thenReturn(pluginLoader);
when(plugins.newConnector(connectorClass)).thenReturn(sourceConnector);
@@ -350,8 +357,6 @@ public void testStartAndStopConnector() throws Throwable {
worker.stop();
assertStatistics(worker, 0, 0);
-
- verify(plugins, times(2)).currentThreadLoader();
verify(plugins).delegatingLoader();
verify(delegatingLoader).connectorLoader(connectorClass);
verify(plugins).newConnector(connectorClass);
@@ -386,7 +391,7 @@ public void testStartConnectorFailure() throws Exception {
Exception exception = new ConnectException("Failed to find Connector");
- when(plugins.currentThreadLoader()).thenReturn(delegatingLoader);
+ expectLoaderSwap();
when(plugins.delegatingLoader()).thenReturn(delegatingLoader);
when(delegatingLoader.connectorLoader(nonConnectorClass)).thenReturn(delegatingLoader);
pluginsMockedStatic.when(() -> Plugins.compareAndSwapLoaders(delegatingLoader)).thenReturn(delegatingLoader);
@@ -418,7 +423,6 @@ public void testStartConnectorFailure() throws Exception {
assertStatistics(worker, 0, 0);
assertStartupStatistics(worker, 1, 1, 0, 0);
- verify(plugins).currentThreadLoader();
verify(plugins).delegatingLoader();
verify(plugins).delegatingLoader();
verify(delegatingLoader).connectorLoader(nonConnectorClass);
@@ -432,7 +436,7 @@ public void testStartConnectorFailure() throws Exception {
public void testAddConnectorByAlias() throws Throwable {
final String connectorAlias = "SampleSourceConnector";
- when(plugins.currentThreadLoader()).thenReturn(delegatingLoader);
+ expectLoaderSwap();
when(plugins.delegatingLoader()).thenReturn(delegatingLoader);
when(plugins.newConnector(connectorAlias)).thenReturn(sinkConnector);
when(delegatingLoader.connectorLoader(connectorAlias)).thenReturn(pluginLoader);
@@ -469,7 +473,6 @@ public void testAddConnectorByAlias() throws Throwable {
assertStatistics(worker, 0, 0);
assertStartupStatistics(worker, 1, 0, 0, 0);
- verify(plugins, times(2)).currentThreadLoader();
verify(plugins).delegatingLoader();
verify(plugins).newConnector(connectorAlias);
verify(delegatingLoader).connectorLoader(connectorAlias);
@@ -489,7 +492,7 @@ public void testAddConnectorByAlias() throws Throwable {
public void testAddConnectorByShortAlias() throws Throwable {
final String shortConnectorAlias = "WorkerTest";
- when(plugins.currentThreadLoader()).thenReturn(delegatingLoader);
+ expectLoaderSwap();
when(plugins.delegatingLoader()).thenReturn(delegatingLoader);
when(plugins.newConnector(shortConnectorAlias)).thenReturn(sinkConnector);
when(delegatingLoader.connectorLoader(shortConnectorAlias)).thenReturn(pluginLoader);
@@ -520,7 +523,6 @@ public void testAddConnectorByShortAlias() throws Throwable {
worker.stop();
assertStatistics(worker, 0, 0);
- verify(plugins, times(2)).currentThreadLoader();
verify(plugins).delegatingLoader();
verify(plugins).newConnector(shortConnectorAlias);
verify(sinkConnector, times(2)).version();
@@ -550,7 +552,7 @@ public void testStopInvalidConnector() {
public void testReconfigureConnectorTasks() throws Throwable {
final String connectorClass = SampleSourceConnector.class.getName();
- when(plugins.currentThreadLoader()).thenReturn(delegatingLoader);
+ expectLoaderSwap();
when(plugins.delegatingLoader()).thenReturn(delegatingLoader);
when(delegatingLoader.connectorLoader(connectorClass)).thenReturn(pluginLoader);
when(plugins.newConnector(connectorClass)).thenReturn(sinkConnector);
@@ -612,7 +614,6 @@ public void testReconfigureConnectorTasks() throws Throwable {
worker.stop();
assertStatistics(worker, 0, 0);
- verify(plugins, times(3)).currentThreadLoader();
verify(plugins).delegatingLoader();
verify(delegatingLoader).connectorLoader(connectorClass);
verify(plugins).newConnector(connectorClass);
@@ -632,7 +633,7 @@ public void testReconfigureConnectorTasks() throws Throwable {
@Test
public void testAddRemoveSourceTask() {
- when(plugins.currentThreadLoader()).thenReturn(delegatingLoader);
+ expectLoaderSwap();
when(plugins.delegatingLoader()).thenReturn(delegatingLoader);
when(delegatingLoader.connectorLoader(SampleSourceConnector.class.getName())).thenReturn(pluginLoader);
@@ -665,7 +666,6 @@ public void testAddRemoveSourceTask() {
worker.stop();
assertStatistics(worker, 0, 0);
- verify(plugins, times(2)).currentThreadLoader();
verify(plugins).newTask(TestSourceTask.class);
verify(task).version();
verifyTaskConverter(WorkerConfig.KEY_CONVERTER_CLASS_CONFIG);
@@ -685,7 +685,7 @@ public void testAddRemoveSourceTask() {
@Test
public void testAddRemoveSinkTask() {
// Most of the other cases use source tasks; we make sure to get code coverage for sink tasks here as well
- when(plugins.currentThreadLoader()).thenReturn(delegatingLoader);
+ expectLoaderSwap();
when(plugins.delegatingLoader()).thenReturn(delegatingLoader);
when(delegatingLoader.connectorLoader(SampleSinkConnector.class.getName())).thenReturn(pluginLoader);
@@ -723,7 +723,6 @@ public void testAddRemoveSinkTask() {
worker.stop();
assertStatistics(worker, 0, 0);
- verify(plugins, times(2)).currentThreadLoader();
verify(plugins).newTask(TestSinkTask.class);
verify(task).version();
verifyTaskConverter(WorkerConfig.KEY_CONVERTER_CLASS_CONFIG);
@@ -759,9 +758,9 @@ public void testAddRemoveExactlyOnceSourceTask() {
workerProps.put(EXACTLY_ONCE_SOURCE_SUPPORT_CONFIG, "enabled");
config = new DistributedConfig(workerProps);
- when(plugins.currentThreadLoader()).thenReturn(delegatingLoader);
when(plugins.delegatingLoader()).thenReturn(delegatingLoader);
when(delegatingLoader.connectorLoader(SampleSourceConnector.class.getName())).thenReturn(pluginLoader);
+ expectLoaderSwap();
when(plugins.newTask(TestSourceTask.class)).thenReturn(task);
when(task.version()).thenReturn("1.0");
@@ -795,7 +794,6 @@ public void testAddRemoveExactlyOnceSourceTask() {
worker.stop();
assertStatistics(worker, 0, 0);
- verify(plugins, times(2)).currentThreadLoader();
verify(plugins).newTask(TestSourceTask.class);
verify(task).version();
verifyTaskConverter(WorkerConfig.KEY_CONVERTER_CLASS_CONFIG);
@@ -818,8 +816,7 @@ public void testTaskStatusMetricsStatuses() {
mockStorage();
mockFileConfigProvider();
-
- when(plugins.currentThreadLoader()).thenReturn(delegatingLoader);
+ expectLoaderSwap();
Map origProps = Collections.singletonMap(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName());
@@ -908,7 +905,6 @@ public void testTaskStatusMetricsStatuses() {
verifyTaskConverter(WorkerConfig.KEY_CONVERTER_CLASS_CONFIG);
verifyTaskConverter(WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG);
verifyTaskHeaderConverter();
- verify(plugins, times(2)).currentThreadLoader();
}
@Test
@@ -948,7 +944,7 @@ public void testStartTaskFailure() {
Map origProps = Collections.singletonMap(TaskConfig.TASK_CLASS_CONFIG, "missing.From.This.Workers.Classpath");
- when(plugins.currentThreadLoader()).thenReturn(delegatingLoader);
+ expectLoaderSwap();
when(plugins.delegatingLoader()).thenReturn(delegatingLoader);
when(delegatingLoader.connectorLoader(SampleSourceConnector.class.getName())).thenReturn(pluginLoader);
@@ -979,7 +975,7 @@ public void testCleanupTasksOnStop() {
mockStorage();
mockFileConfigProvider();
- when(plugins.currentThreadLoader()).thenReturn(delegatingLoader);
+ expectLoaderSwap();
when(plugins.newTask(TestSourceTask.class)).thenReturn(task);
when(task.version()).thenReturn("1.0");
@@ -1023,7 +1019,6 @@ public void testCleanupTasksOnStop() {
pluginsMockedStatic.verify(() -> Plugins.compareAndSwapLoaders(pluginLoader), times(2));
pluginsMockedStatic.verify(() -> Plugins.compareAndSwapLoaders(delegatingLoader), times(2));
verify(plugins).newTask(TestSourceTask.class);
- verify(plugins, times(2)).currentThreadLoader();
verify(plugins).delegatingLoader();
verify(delegatingLoader).connectorLoader(SampleSourceConnector.class.getName());
@@ -1044,7 +1039,7 @@ public void testConverterOverrides() {
mockStorage();
mockFileConfigProvider();
- when(plugins.currentThreadLoader()).thenReturn(delegatingLoader);
+ expectLoaderSwap();
Map origProps = Collections.singletonMap(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName());
TaskConfig taskConfig = new TaskConfig(origProps);
@@ -1071,7 +1066,6 @@ public void testConverterOverrides() {
pluginsMockedStatic.when(() -> Plugins.compareAndSwapLoaders(pluginLoader)).thenReturn(delegatingLoader);
pluginsMockedStatic.when(() -> Plugins.compareAndSwapLoaders(delegatingLoader)).thenReturn(pluginLoader);
-
worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, executorService,
noneConnectorClientConfigOverridePolicy);
worker.herder = herder;
@@ -1096,7 +1090,6 @@ public void testConverterOverrides() {
WorkerSourceTask instantiatedTask = sourceTaskMockedConstruction.constructed().get(0);
verify(instantiatedTask).initialize(taskConfig);
verify(executorService).submit(any(WorkerSourceTask.class));
- verify(plugins).delegatingLoader();
verify(delegatingLoader).connectorLoader(SampleSourceConnector.class.getName());
pluginsMockedStatic.verify(() -> Plugins.compareAndSwapLoaders(pluginLoader), times(2));
pluginsMockedStatic.verify(() -> Plugins.compareAndSwapLoaders(delegatingLoader), times(2));
@@ -1107,7 +1100,6 @@ public void testConverterOverrides() {
verify(instantiatedTask).awaitStop(anyLong());
verify(instantiatedTask).removeMetrics();
- verify(plugins, times(2)).currentThreadLoader();
verifyStorage();
}
@@ -2018,4 +2010,128 @@ public void stop() {
}
+ @Test
+ public void testCloseableResourcesTrackedWhileInstantiatingSourceTask() {
+ ClusterConfigState configState = mock(ClusterConfigState.class);
+ Map connProps = anyConnectorConfigMap();
+ WorkerMetricsGroup.TaskStatusListener statusListener = mock(WorkerMetricsGroup.TaskStatusListener.class);
+
+ expectLoaderSwap();
+ doReturn(SampleSourceConnector.class).when(plugins).connectorClass(SampleSourceConnector.class.getName());
+ when(plugins.delegatingLoader()).thenReturn(delegatingLoader);
+ when(delegatingLoader.connectorLoader(anyString())).thenReturn(pluginLoader);
+ pluginsMockedStatic.when(() -> Plugins.compareAndSwapLoaders(pluginLoader)).thenReturn(delegatingLoader);
+ pluginsMockedStatic.when(() -> Plugins.compareAndSwapLoaders(delegatingLoader)).thenReturn(pluginLoader);
+ when(plugins.newTask(any())).thenReturn(task);
+ mockTaskConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG, taskKeyConverter);
+ mockTaskConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, taskValueConverter);
+ mockTaskHeaderConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, taskHeaderConverter);
+
+ Map taskProps = new HashMap<>(connProps);
+ taskProps.put(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName());
+
+ worker = new Worker(
+ WORKER_ID,
+ new MockTime(),
+ plugins,
+ config,
+ offsetBackingStore,
+ noneConnectorClientConfigOverridePolicy
+ );
+ worker.herder = herder;
+ worker.start();
+
+ TestCloseables closeables = new TestCloseables();
+ assertTrue("Failed to start task", worker.startTask(
+ TASK_ID, connProps, taskProps, statusListener, closeables,
+ worker.new SourceTaskBuilder(TASK_ID, configState, statusListener, TargetState.STARTED)
+ ));
+
+ Map, Integer> expectedCloseableResources = new HashMap<>();
+ expectedCloseableResources.put(OffsetStorageReaderImpl.class, 1);
+ if (enableTopicCreation) {
+ expectedCloseableResources.put(TopicAdmin.class, 1);
+ }
+ expectedCloseableResources.put(RetryWithToleranceOperator.class, 1);
+ expectedCloseableResources.put(KafkaProducer.class, 1);
+ expectedCloseableResources.put(TransformationChain.class, 1);
+ expectedCloseableResources.put(ErrorHandlingMetrics.class, 1);
+ expectedCloseableResources.put(LogReporter.class, 1);
+ expectedCloseableResources.put(ConnectorOffsetBackingStore.class, 1);
+ expectedCloseableResources.put(taskHeaderConverter.getClass(), 1);
+
+ expectedCloseableResources.forEach((klass, count) ->
+ assertEquals(
+ "class " + klass + " was not registered the expected number of times",
+ count,
+ closeables.registered().remove(klass)
+ )
+ );
+ assertEquals("one anonymous lambda/method reference should also have been registered", 1, closeables.registered().size());
+ }
+
+ @Test
+ public void testCloseableResourcesTrackedWhileInstantiatingSinkTask() {
+ if (enableTopicCreation) {
+ return;
+ }
+
+ ClusterConfigState configState = mock(ClusterConfigState.class);
+ Map connProps = anyConnectorConfigMap();
+ connProps.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, SampleSinkConnector.class.getName());
+ WorkerMetricsGroup.TaskStatusListener statusListener = mock(WorkerMetricsGroup.TaskStatusListener.class);
+
+ expectLoaderSwap();
+ doReturn(SampleSinkConnector.class).when(plugins).connectorClass(SampleSinkConnector.class.getName());
+ when(plugins.delegatingLoader()).thenReturn(delegatingLoader);
+ when(delegatingLoader.connectorLoader(anyString())).thenReturn(pluginLoader);
+ pluginsMockedStatic.when(() -> Plugins.compareAndSwapLoaders(pluginLoader)).thenReturn(delegatingLoader);
+ pluginsMockedStatic.when(() -> Plugins.compareAndSwapLoaders(delegatingLoader)).thenReturn(pluginLoader);
+ when(plugins.newTask(any())).thenReturn(new TestSinkTask());
+ mockTaskConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG, taskKeyConverter);
+ mockTaskConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, taskValueConverter);
+ mockTaskHeaderConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, taskHeaderConverter);
+
+ Map taskProps = new HashMap<>(connProps);
+ taskProps.put(TaskConfig.TASK_CLASS_CONFIG, TestSinkTask.class.getName());
+
+ worker = new Worker(
+ WORKER_ID,
+ new MockTime(),
+ plugins,
+ config,
+ offsetBackingStore,
+ noneConnectorClientConfigOverridePolicy
+ );
+ worker.herder = herder;
+ worker.start();
+
+ TestCloseables closeables = new TestCloseables();
+ assertTrue("Failed to start task", worker.startTask(
+ TASK_ID, connProps, taskProps, statusListener, closeables,
+ worker.new SinkTaskBuilder(TASK_ID, configState, statusListener, TargetState.STARTED)
+ ));
+
+ Map, Integer> expectedCloseableResources = new HashMap<>();
+ expectedCloseableResources.put(TransformationChain.class, 1);
+ expectedCloseableResources.put(RetryWithToleranceOperator.class, 1);
+ expectedCloseableResources.put(KafkaConsumer.class, 1);
+ expectedCloseableResources.put(ErrorHandlingMetrics.class, 1);
+ expectedCloseableResources.put(LogReporter.class, 1);
+ expectedCloseableResources.put(taskHeaderConverter.getClass(), 1);
+
+ expectedCloseableResources.forEach((klass, count) ->
+ assertEquals(
+ "class " + klass + " was not registered the expected number of times",
+ count,
+ closeables.registered().remove(klass)
+ )
+ );
+ assertEquals("one anonymous lambda/method reference should also have been registered", 1, closeables.registered().size());
+ }
+
+ private void expectLoaderSwap() {
+ doCallRealMethod().when(plugins).withClassLoader(any());
+ }
+
}