From e1d7fc8c3e1ec0570bfa12dd68d09403fd1671e5 Mon Sep 17 00:00:00 2001 From: Yash Mayya Date: Thu, 10 Aug 2023 21:47:30 +0100 Subject: [PATCH] KAFKA-13187: Replace EasyMock / PowerMock with Mockito in DistributedHerderTest (#14102) Reviewers: Chris Egerton --- build.gradle | 1 - .../distributed/DistributedHerder.java | 6 +- .../distributed/DistributedHerderTest.java | 4010 +++++++---------- 3 files changed, 1557 insertions(+), 2460 deletions(-) diff --git a/build.gradle b/build.gradle index 250aed449117..4dc0c84e9717 100644 --- a/build.gradle +++ b/build.gradle @@ -415,7 +415,6 @@ subprojects { if (JavaVersion.current().isCompatibleWith(JavaVersion.VERSION_16)) { testsToExclude.addAll([ // connect tests - "**/DistributedHerderTest.*", "**/KafkaConfigBackingStoreTest.*", "**/KafkaBasedLogTest.*", "**/StandaloneHerderTest.*", "**/WorkerSinkTaskTest.*", "**/WorkerSinkTaskThreadedTest.*" diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java index 1de487245e08..5f3f40360e02 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java @@ -173,7 +173,8 @@ public class DistributedHerder extends AbstractHerder implements Runnable { // Visible for testing ExecutorService forwardRequestExecutor; - private final ExecutorService herderExecutor; + // Visible for testing + final ExecutorService herderExecutor; // Visible for testing ExecutorService startAndStopExecutor; private final WorkerGroupMember member; @@ -274,7 +275,8 @@ public DistributedHerder(DistributedConfig config, ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy, List restNamespace, ExecutorService forwardRequestExecutor, - AutoCloseable... uponShutdown) { + // https://github.com/mockito/mockito/issues/2601 explains why we can't use varargs here + AutoCloseable[] uponShutdown) { super(worker, workerId, kafkaClusterId, statusBackingStore, configBackingStore, connectorClientConfigOverridePolicy); this.time = time; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java index d603a4d5b6fd..bec8369a3b09 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java @@ -26,6 +26,7 @@ import org.apache.kafka.connect.errors.AlreadyExistsException; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.errors.NotFoundException; +import org.apache.kafka.connect.runtime.AbstractStatus; import org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroup; import org.apache.kafka.connect.runtime.ConnectorConfig; import org.apache.kafka.connect.runtime.ConnectorStatus; @@ -38,6 +39,7 @@ import org.apache.kafka.connect.runtime.SourceConnectorConfig; import org.apache.kafka.connect.runtime.TargetState; import org.apache.kafka.connect.runtime.TaskConfig; +import org.apache.kafka.connect.runtime.TaskStatus; import org.apache.kafka.connect.runtime.TopicStatus; import org.apache.kafka.connect.runtime.Worker; import org.apache.kafka.connect.runtime.WorkerConfig; @@ -67,23 +69,17 @@ import org.apache.kafka.connect.util.Callback; import org.apache.kafka.connect.util.ConnectorTaskId; import org.apache.kafka.connect.util.FutureCallback; -import org.easymock.Capture; -import org.easymock.CaptureType; -import org.easymock.EasyMock; -import org.easymock.IAnswer; 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.powermock.reflect.Whitebox; +import org.mockito.ArgumentCaptor; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; +import org.mockito.stubbing.Answer; +import org.mockito.stubbing.OngoingStubbing; import javax.crypto.SecretKey; -import javax.ws.rs.core.HttpHeaders; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -119,21 +115,32 @@ import static org.apache.kafka.connect.runtime.distributed.IncrementalCooperativeConnectProtocol.CONNECT_PROTOCOL_V1; import static org.apache.kafka.connect.runtime.distributed.IncrementalCooperativeConnectProtocol.CONNECT_PROTOCOL_V2; import static org.apache.kafka.connect.source.SourceTask.TransactionBoundary.CONNECTOR; -import static org.easymock.EasyMock.anyLong; -import static org.easymock.EasyMock.anyObject; -import static org.easymock.EasyMock.capture; -import static org.easymock.EasyMock.isNull; -import static org.easymock.EasyMock.leq; -import static org.easymock.EasyMock.newCapture; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; - -@RunWith(PowerMockRunner.class) -@PrepareForTest({DistributedHerder.class}) -@PowerMockIgnore({"javax.management.*", "javax.crypto.*"}) +import static org.mockito.AdditionalMatchers.leq; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.anyMap; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.ArgumentMatchers.isNull; +import static org.mockito.Mockito.CALLS_REAL_METHODS; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoMoreInteractions; +import static org.mockito.Mockito.when; +import static org.mockito.Mockito.withSettings; + +@RunWith(MockitoJUnitRunner.StrictStubs.class) +@SuppressWarnings("unchecked") public class DistributedHerderTest { private static final Map HERDER_CONFIG = new HashMap<>(); static { @@ -285,27 +292,22 @@ public class DistributedHerderTest { public void setUp() throws Exception { time = new MockTime(); metrics = new MockConnectMetrics(time); - worker = PowerMock.createMock(Worker.class); - EasyMock.expect(worker.isSinkConnector(CONN1)).andStubReturn(Boolean.TRUE); + when(worker.isSinkConnector(CONN1)).thenReturn(Boolean.TRUE); AutoCloseable uponShutdown = () -> shutdownCalled.countDown(); // Default to the old protocol unless specified otherwise connectProtocolVersion = CONNECT_PROTOCOL_V0; - herder = PowerMock.createPartialMock(DistributedHerder.class, - new String[]{"connectorType", "updateDeletedConnectorStatus", "updateDeletedTaskStatus", "validateConnectorConfig", "buildRestartPlan", "recordRestarting"}, - new DistributedConfig(HERDER_CONFIG), worker, WORKER_ID, KAFKA_CLUSTER_ID, - statusBackingStore, configBackingStore, member, MEMBER_URL, restClient, metrics, time, noneConnectorClientConfigOverridePolicy, - Collections.emptyList(), null, new AutoCloseable[]{uponShutdown}); + herder = mock(DistributedHerder.class, withSettings().defaultAnswer(CALLS_REAL_METHODS).useConstructor(new DistributedConfig(HERDER_CONFIG), + worker, WORKER_ID, KAFKA_CLUSTER_ID, statusBackingStore, configBackingStore, member, MEMBER_URL, restClient, metrics, time, + noneConnectorClientConfigOverridePolicy, Collections.emptyList(), null, new AutoCloseable[]{uponShutdown})); configUpdateListener = herder.new ConfigUpdateListener(); rebalanceListener = herder.new RebalanceListener(time); - plugins = PowerMock.createMock(Plugins.class); conn1SinkConfig = new SinkConnectorConfig(plugins, CONN1_CONFIG); conn1SinkConfigUpdated = new SinkConnectorConfig(plugins, CONN1_CONFIG_UPDATED); - EasyMock.expect(herder.connectorType(EasyMock.anyObject())).andReturn(ConnectorType.SOURCE).anyTimes(); - PowerMock.expectPrivate(herder, "updateDeletedConnectorStatus").andVoid().anyTimes(); - PowerMock.expectPrivate(herder, "updateDeletedTaskStatus").andVoid().anyTimes(); + + when(herder.connectorType(anyMap())).thenReturn(ConnectorType.SOURCE); } @After @@ -320,89 +322,69 @@ public void tearDown() { @Test public void testJoinAssignment() throws Exception { // Join group and get assignment - EasyMock.expect(member.memberId()).andStubReturn("member"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); + when(member.memberId()).thenReturn("member"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); expectRebalance(1, Arrays.asList(CONN1), Arrays.asList(TASK1)); expectConfigRefreshAndSnapshot(SNAPSHOT); - Capture> onStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onStart)); - PowerMock.expectLastCall().andAnswer(() -> { + + ArgumentCaptor> onStart = ArgumentCaptor.forClass(Callback.class); + doAnswer(invocation -> { onStart.getValue().onCompletion(null, TargetState.STARTED); return true; - }); - member.wakeup(); - PowerMock.expectLastCall(); - expectExecuteTaskReconfiguration(true, conn1SinkConfig, () -> TASK_CONFIGS); - worker.startSourceTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); - PowerMock.expectLastCall().andReturn(true); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + }).when(worker).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), onStart.capture()); + expectExecuteTaskReconfiguration(true, conn1SinkConfig, invocation -> TASK_CONFIGS); + when(worker.startSourceTask(eq(TASK1), any(), any(), any(), eq(herder), eq(TargetState.STARTED))).thenReturn(true); - PowerMock.replayAll(); + doNothing().when(member).poll(anyLong()); herder.tick(); time.sleep(1000L); assertStatistics(3, 1, 100, 1000L); - - PowerMock.verifyAll(); + verifyNoMoreInteractions(member, configBackingStore, statusBackingStore, worker); } @Test public void testRebalance() throws Exception { // Join group and get assignment - EasyMock.expect(member.memberId()).andStubReturn("member"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); + when(member.memberId()).thenReturn("member"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); expectRebalance(1, Arrays.asList(CONN1), Arrays.asList(TASK1)); expectConfigRefreshAndSnapshot(SNAPSHOT); - Capture> onFirstStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onFirstStart)); - PowerMock.expectLastCall().andAnswer(() -> { - onFirstStart.getValue().onCompletion(null, TargetState.STARTED); - return true; - }); - member.wakeup(); - PowerMock.expectLastCall(); - expectExecuteTaskReconfiguration(true, conn1SinkConfig, () -> TASK_CONFIGS); - worker.startSourceTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); - PowerMock.expectLastCall().andReturn(true); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - expectRebalance(Arrays.asList(CONN1), Arrays.asList(TASK1), ConnectProtocol.Assignment.NO_ERROR, - 1, Arrays.asList(CONN1), Arrays.asList()); - - // and the new assignment started - Capture> onSecondStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onSecondStart)); - PowerMock.expectLastCall().andAnswer(() -> { - onSecondStart.getValue().onCompletion(null, TargetState.STARTED); + ArgumentCaptor> onStart = ArgumentCaptor.forClass(Callback.class); + doAnswer(invocation -> { + onStart.getValue().onCompletion(null, TargetState.STARTED); return true; - }); - member.wakeup(); - PowerMock.expectLastCall(); - expectExecuteTaskReconfiguration(true, conn1SinkConfig, () -> TASK_CONFIGS); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + }).when(worker).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), onStart.capture()); + expectExecuteTaskReconfiguration(true, conn1SinkConfig, invocation -> TASK_CONFIGS); + when(worker.startSourceTask(eq(TASK1), any(), any(), any(), eq(herder), eq(TargetState.STARTED))).thenReturn(true); - PowerMock.replayAll(); + doNothing().when(member).poll(anyLong()); time.sleep(1000L); assertStatistics(0, 0, 0, Double.POSITIVE_INFINITY); - herder.tick(); + herder.tick(); time.sleep(2000L); assertStatistics(3, 1, 100, 2000); - herder.tick(); + verify(worker).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), any()); + verify(worker).connectorTaskConfigs(eq(CONN1), eq(conn1SinkConfig)); + verify(worker).startSourceTask(eq(TASK1), any(), any(), any(), eq(herder), eq(TargetState.STARTED)); + + // Rebalance and get a new assignment + expectRebalance(Arrays.asList(CONN1), Arrays.asList(TASK1), ConnectProtocol.Assignment.NO_ERROR, + 1, Arrays.asList(CONN1), Arrays.asList()); + herder.tick(); time.sleep(3000L); assertStatistics(3, 2, 100, 3000); - PowerMock.verifyAll(); + // Verify that the connector is started twice but the task is only started once (the first mocked rebalance assigns CONN1 and TASK1, + // the second mocked rebalance revokes CONN1 and TASK1 and (re)assigns CONN1) + verify(worker, times(2)).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), any()); + verify(worker, times(2)).connectorTaskConfigs(eq(CONN1), eq(conn1SinkConfig)); + verify(worker).startSourceTask(eq(TASK1), any(), any(), any(), eq(herder), eq(TargetState.STARTED)); + verifyNoMoreInteractions(member, configBackingStore, statusBackingStore, worker); } @Test @@ -410,13 +392,17 @@ public void testIncrementalCooperativeRebalanceForNewMember() throws Exception { connectProtocolVersion = CONNECT_PROTOCOL_V1; // Join group. First rebalance contains revocations from other members. For the new // member the assignment should be empty - EasyMock.expect(member.memberId()).andStubReturn("member"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V1); + when(member.memberId()).thenReturn("member"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V1); expectRebalance(1, Collections.emptyList(), Collections.emptyList()); expectConfigRefreshAndSnapshot(SNAPSHOT); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + doNothing().when(member).poll(anyLong()); + + time.sleep(1000L); + assertStatistics(0, 0, 0, Double.POSITIVE_INFINITY); + + herder.tick(); // The new member got its assignment expectRebalance(Collections.emptyList(), Collections.emptyList(), @@ -424,73 +410,58 @@ public void testIncrementalCooperativeRebalanceForNewMember() throws Exception { 1, Arrays.asList(CONN1), Arrays.asList(TASK1), 0); // and the new assignment started - Capture> onStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onStart)); - PowerMock.expectLastCall().andAnswer(() -> { + ArgumentCaptor> onStart = ArgumentCaptor.forClass(Callback.class); + doAnswer(invocation -> { onStart.getValue().onCompletion(null, TargetState.STARTED); return true; - }); - member.wakeup(); - PowerMock.expectLastCall(); - expectExecuteTaskReconfiguration(true, conn1SinkConfig, () -> TASK_CONFIGS); + }).when(worker).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), onStart.capture()); + expectExecuteTaskReconfiguration(true, conn1SinkConfig, invocation -> TASK_CONFIGS); - worker.startSourceTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); - PowerMock.expectLastCall().andReturn(true); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - PowerMock.replayAll(); - - time.sleep(1000L); - assertStatistics(0, 0, 0, Double.POSITIVE_INFINITY); - herder.tick(); + when(worker.startSourceTask(eq(TASK1), any(), any(), any(), eq(herder), eq(TargetState.STARTED))).thenReturn(true); time.sleep(2000L); assertStatistics(3, 1, 100, 2000); - herder.tick(); + herder.tick(); time.sleep(3000L); assertStatistics(3, 2, 100, 3000); - PowerMock.verifyAll(); + verify(worker).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), any()); + verify(worker).connectorTaskConfigs(eq(CONN1), eq(conn1SinkConfig)); + verify(worker).startSourceTask(eq(TASK1), any(), any(), any(), eq(herder), eq(TargetState.STARTED)); + verifyNoMoreInteractions(member, statusBackingStore, configBackingStore, worker); } @Test public void testIncrementalCooperativeRebalanceForExistingMember() { connectProtocolVersion = CONNECT_PROTOCOL_V1; // Join group. First rebalance contains revocations because a new member joined. - EasyMock.expect(member.memberId()).andStubReturn("member"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V1); + when(member.memberId()).thenReturn("member"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V1); expectRebalance(Arrays.asList(CONN1), Arrays.asList(TASK1), ConnectProtocol.Assignment.NO_ERROR, 1, Collections.emptyList(), Collections.emptyList(), 0); - member.requestRejoin(); - PowerMock.expectLastCall(); - - // In the second rebalance the new member gets its assignment and this member has no - // assignments or revocations - expectRebalance(1, Collections.emptyList(), Collections.emptyList()); - - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - PowerMock.replayAll(); + doNothing().when(member).requestRejoin(); + doNothing().when(member).poll(anyLong()); herder.configState = SNAPSHOT; time.sleep(1000L); assertStatistics(0, 0, 0, Double.POSITIVE_INFINITY); + herder.tick(); + // In the second rebalance the new member gets its assignment and this member has no + // assignments or revocations + expectRebalance(1, Collections.emptyList(), Collections.emptyList()); + time.sleep(2000L); assertStatistics(3, 1, 100, 2000); - herder.tick(); + herder.tick(); time.sleep(3000L); assertStatistics(3, 2, 100, 3000); - PowerMock.verifyAll(); + verifyNoMoreInteractions(member, statusBackingStore, configBackingStore, worker); } @Test @@ -500,26 +471,27 @@ public void testIncrementalCooperativeRebalanceWithDelay() throws Exception { // member was detected missing int rebalanceDelay = 10_000; - EasyMock.expect(member.memberId()).andStubReturn("member"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V1); + when(member.memberId()).thenReturn("member"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V1); expectRebalance(Collections.emptyList(), Collections.emptyList(), ConnectProtocol.Assignment.NO_ERROR, 1, Collections.emptyList(), Arrays.asList(TASK2), rebalanceDelay); expectConfigRefreshAndSnapshot(SNAPSHOT); - worker.startSourceTask(EasyMock.eq(TASK2), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); - PowerMock.expectLastCall().andReturn(true); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall().andAnswer(() -> { + when(worker.startSourceTask(eq(TASK2), any(), any(), any(), eq(herder), eq(TargetState.STARTED))).thenReturn(true); + doAnswer(invocation -> { time.sleep(9900L); return null; - }); + }).when(member).poll(anyLong()); + + // Request to re-join expected because the scheduled rebalance delay has been reached + doNothing().when(member).requestRejoin(); + + time.sleep(1000L); + assertStatistics(0, 0, 0, Double.POSITIVE_INFINITY); - // Request to re-join because the scheduled rebalance delay has been reached - member.requestRejoin(); - PowerMock.expectLastCall(); + herder.tick(); // The member got its assignment and revocation expectRebalance(Collections.emptyList(), Collections.emptyList(), @@ -527,91 +499,65 @@ public void testIncrementalCooperativeRebalanceWithDelay() throws Exception { 1, Arrays.asList(CONN1), Arrays.asList(TASK1), 0); // and the new assignment started - Capture> onStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onStart)); - PowerMock.expectLastCall().andAnswer(() -> { + ArgumentCaptor> onStart = ArgumentCaptor.forClass(Callback.class); + doAnswer(invocation -> { onStart.getValue().onCompletion(null, TargetState.STARTED); return true; - }); - member.wakeup(); - PowerMock.expectLastCall(); - expectExecuteTaskReconfiguration(true, conn1SinkConfig, () -> TASK_CONFIGS); - - worker.startSourceTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); - PowerMock.expectLastCall().andReturn(true); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + }).when(worker).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), onStart.capture()); + expectExecuteTaskReconfiguration(true, conn1SinkConfig, invocation -> TASK_CONFIGS); + when(worker.startSourceTask(eq(TASK1), any(), any(), any(), eq(herder), eq(TargetState.STARTED))).thenReturn(true); - PowerMock.replayAll(); - - time.sleep(1000L); - assertStatistics(0, 0, 0, Double.POSITIVE_INFINITY); - herder.tick(); + doNothing().when(member).poll(anyLong()); herder.tick(); - time.sleep(2000L); assertStatistics(3, 2, 100, 2000); - PowerMock.verifyAll(); + verifyNoMoreInteractions(member, statusBackingStore, configBackingStore, worker); } @Test public void testRebalanceFailedConnector() throws Exception { // Join group and get assignment - EasyMock.expect(member.memberId()).andStubReturn("member"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); + when(member.memberId()).thenReturn("member"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); expectRebalance(1, Arrays.asList(CONN1), Arrays.asList(TASK1)); expectConfigRefreshAndSnapshot(SNAPSHOT); - Capture> onFirstStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onFirstStart)); - PowerMock.expectLastCall().andAnswer(() -> { - onFirstStart.getValue().onCompletion(null, TargetState.STARTED); - return true; - }); - member.wakeup(); - PowerMock.expectLastCall(); - expectExecuteTaskReconfiguration(true, conn1SinkConfig, () -> TASK_CONFIGS); - worker.startSourceTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); - PowerMock.expectLastCall().andReturn(true); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - expectRebalance(Arrays.asList(CONN1), Arrays.asList(TASK1), ConnectProtocol.Assignment.NO_ERROR, - 1, Arrays.asList(CONN1), Arrays.asList()); - - // and the new assignment started - Capture> onSecondStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onSecondStart)); - PowerMock.expectLastCall().andAnswer(() -> { - onSecondStart.getValue().onCompletion(null, TargetState.STARTED); + ArgumentCaptor> onStart = ArgumentCaptor.forClass(Callback.class); + doAnswer(invocation -> { + onStart.getValue().onCompletion(null, TargetState.STARTED); return true; - }); - member.wakeup(); - PowerMock.expectLastCall(); - expectExecuteTaskReconfiguration(false, null, null); + }).when(worker).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), onStart.capture()); + expectExecuteTaskReconfiguration(true, conn1SinkConfig, invocation -> TASK_CONFIGS); + when(worker.startSourceTask(eq(TASK1), any(), any(), any(), eq(herder), eq(TargetState.STARTED))).thenReturn(true); - // worker is not running, so we should see no call to connectorTaskConfigs() - - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - PowerMock.replayAll(); + doNothing().when(member).poll(anyLong()); herder.tick(); time.sleep(1000L); assertStatistics(3, 1, 100, 1000L); + verify(worker).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), any()); + verify(worker).connectorTaskConfigs(eq(CONN1), eq(conn1SinkConfig)); + verify(worker).startSourceTask(eq(TASK1), any(), any(), any(), eq(herder), eq(TargetState.STARTED)); + + // Rebalance and get a new assignment + expectRebalance(Arrays.asList(CONN1), Arrays.asList(TASK1), ConnectProtocol.Assignment.NO_ERROR, + 1, Arrays.asList(CONN1), Arrays.asList()); + + // worker is not running, so we should see no call to connectorTaskConfigs() + expectExecuteTaskReconfiguration(false, null, null); + herder.tick(); time.sleep(2000L); assertStatistics(3, 2, 100, 2000L); - PowerMock.verifyAll(); + verify(worker, times(2)).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), any()); + verify(worker).connectorTaskConfigs(eq(CONN1), eq(conn1SinkConfig)); + verify(worker).startSourceTask(eq(TASK1), any(), any(), any(), eq(herder), eq(TargetState.STARTED)); + + verifyNoMoreInteractions(member, statusBackingStore, configBackingStore, worker); } @Test @@ -629,26 +575,23 @@ public void revokeAndReassign(boolean incompleteRebalance) throws TimeoutExcepti int configOffset = 1; // Join group and get initial assignment - EasyMock.expect(member.memberId()).andStubReturn("member"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(connectProtocolVersion); + when(member.memberId()).thenReturn("member"); + when(member.currentProtocolVersion()).thenReturn(connectProtocolVersion); // The lists need to be mutable because assignments might be removed expectRebalance(configOffset, new ArrayList<>(singletonList(CONN1)), new ArrayList<>(singletonList(TASK1))); expectConfigRefreshAndSnapshot(SNAPSHOT); - Capture> onFirstStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onFirstStart)); - PowerMock.expectLastCall().andAnswer(() -> { - onFirstStart.getValue().onCompletion(null, TargetState.STARTED); + + ArgumentCaptor> onStart = ArgumentCaptor.forClass(Callback.class); + doAnswer(invocation -> { + onStart.getValue().onCompletion(null, TargetState.STARTED); return true; - }); - member.wakeup(); - PowerMock.expectLastCall(); - expectExecuteTaskReconfiguration(true, conn1SinkConfig, () -> TASK_CONFIGS); - worker.startSourceTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); - PowerMock.expectLastCall().andReturn(true); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + }).when(worker).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), onStart.capture()); + when(worker.startSourceTask(eq(TASK1), any(), any(), any(), eq(herder), eq(TargetState.STARTED))).thenReturn(true); + expectExecuteTaskReconfiguration(true, conn1SinkConfig, invocation -> TASK_CONFIGS); + + doNothing().when(member).poll(anyLong()); + + herder.tick(); // worker is stable with an existing set of tasks @@ -659,10 +602,10 @@ public void revokeAndReassign(boolean incompleteRebalance) throws TimeoutExcepti expectRebalance(configOffset, Arrays.asList(), Arrays.asList()); // give it the wrong snapshot, as if we're out of sync/can't reach the broker expectConfigRefreshAndSnapshot(SNAPSHOT); - member.requestRejoin(); - PowerMock.expectLastCall(); + doNothing().when(member).requestRejoin(); // tick exits early because we failed, and doesn't do the poll at the end of the method // the worker did not startWork or reset the rebalanceResolved flag + herder.tick(); } // Revoke the connector in the next rebalance @@ -687,220 +630,164 @@ public void revokeAndReassign(boolean incompleteRebalance) throws TimeoutExcepti ); expectConfigRefreshAndSnapshot(secondSnapshot); } - member.requestRejoin(); - PowerMock.expectLastCall(); + + doNothing().when(member).requestRejoin(); + + herder.tick(); // re-assign the connector back to the same worker to ensure state was cleaned up expectRebalance(configOffset, Arrays.asList(CONN1), Arrays.asList()); - Capture> onSecondStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), - EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onSecondStart)); - PowerMock.expectLastCall().andAnswer(() -> { - onSecondStart.getValue().onCompletion(null, TargetState.STARTED); - return true; - }); - member.wakeup(); - PowerMock.expectLastCall(); - expectExecuteTaskReconfiguration(true, conn1SinkConfig, () -> TASK_CONFIGS); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - PowerMock.replayAll(); herder.tick(); - if (incompleteRebalance) { - herder.tick(); - } - herder.tick(); - herder.tick(); - PowerMock.verifyAll(); + verify(worker, times(2)).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), any()); + verify(worker, times(2)).connectorTaskConfigs(eq(CONN1), eq(conn1SinkConfig)); + verify(worker).startSourceTask(eq(TASK1), any(), any(), any(), eq(herder), eq(TargetState.STARTED)); + verify(worker).stopAndAwaitConnector(CONN1); + + // The tick loop where the revoke happens returns early (because there's a subsequent rebalance) and doesn't result in a poll at + // the end of the method + verify(member, times(2)).poll(anyLong()); + + verifyNoMoreInteractions(member, statusBackingStore, configBackingStore, worker); } @Test public void testHaltCleansUpWorker() { - worker.stopAndAwaitConnectors(); - PowerMock.expectLastCall(); - worker.stopAndAwaitTasks(); - PowerMock.expectLastCall(); - member.stop(); - PowerMock.expectLastCall(); - configBackingStore.stop(); - PowerMock.expectLastCall(); - statusBackingStore.stop(); - PowerMock.expectLastCall(); - worker.stop(); - PowerMock.expectLastCall(); - - PowerMock.replayAll(); - herder.halt(); - PowerMock.verifyAll(); + verify(worker).stopAndAwaitConnectors(); + verify(worker).stopAndAwaitTasks(); + verify(member).stop(); + verify(configBackingStore).stop(); + verify(statusBackingStore).stop(); + verify(worker).stop(); + + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); } @Test public void testCreateConnector() throws Exception { - EasyMock.expect(member.memberId()).andStubReturn("leader"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); + when(member.memberId()).thenReturn("leader"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true); expectConfigRefreshAndSnapshot(SNAPSHOT); - member.wakeup(); - PowerMock.expectLastCall(); + when(statusBackingStore.connectors()).thenReturn(Collections.emptySet()); + doNothing().when(member).poll(anyLong()); + + // Initial rebalance where this member becomes the leader + herder.tick(); // mock the actual validation since its asynchronous nature is difficult to test and should // be covered sufficiently by the unit tests for the AbstractHerder class - Capture> validateCallback = newCapture(); - herder.validateConnectorConfig(EasyMock.eq(CONN2_CONFIG), capture(validateCallback)); - PowerMock.expectLastCall().andAnswer(() -> { + ArgumentCaptor> validateCallback = ArgumentCaptor.forClass(Callback.class); + doAnswer(invocation -> { validateCallback.getValue().onCompletion(null, CONN2_CONFIG_INFOS); return null; - }); + }).when(herder).validateConnectorConfig(eq(CONN2_CONFIG), validateCallback.capture()); // CONN2 is new, should succeed - configBackingStore.putConnectorConfig(CONN2, CONN2_CONFIG); - PowerMock.expectLastCall(); - ConnectorInfo info = new ConnectorInfo(CONN2, CONN2_CONFIG, Collections.emptyList(), - ConnectorType.SOURCE); - putConnectorCallback.onCompletion(null, new Herder.Created<>(true, info)); - PowerMock.expectLastCall(); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - // These will occur just before/during the second tick - member.wakeup(); - PowerMock.expectLastCall(); - member.ensureActive(); - PowerMock.expectLastCall(); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + doNothing().when(configBackingStore).putConnectorConfig(CONN2, CONN2_CONFIG); - // No immediate action besides this -- change will be picked up via the config log + // This will occur just before/during the second tick + doNothing().when(member).ensureActive(); - PowerMock.replayAll(); + // No immediate action besides this -- change will be picked up via the config log herder.putConnectorConfig(CONN2, CONN2_CONFIG, false, putConnectorCallback); - // First tick runs the initial herder request, which issues an asynchronous request for + // This tick runs the initial herder request, which issues an asynchronous request for // connector validation herder.tick(); // Once that validation is complete, another request is added to the herder request queue // for actually performing the config write; this tick is for that request herder.tick(); - time.sleep(1000L); assertStatistics(3, 1, 100, 1000L); - PowerMock.verifyAll(); + ConnectorInfo info = new ConnectorInfo(CONN2, CONN2_CONFIG, Collections.emptyList(), ConnectorType.SOURCE); + verify(putConnectorCallback).onCompletion(isNull(), eq(new Herder.Created<>(true, info))); + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore, putConnectorCallback); } @Test public void testCreateConnectorConfigBackingStoreError() { - EasyMock.expect(member.memberId()).andStubReturn("leader"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); + when(member.memberId()).thenReturn("leader"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true); expectConfigRefreshAndSnapshot(SNAPSHOT); - member.wakeup(); - PowerMock.expectLastCall(); + when(statusBackingStore.connectors()).thenReturn(Collections.emptySet()); + doNothing().when(member).poll(anyLong()); - // mock the actual validation since its asynchronous nature is difficult to test and should + // Initial rebalance where this member becomes the leader + herder.tick(); + + // Mock the actual validation since its asynchronous nature is difficult to test and should // be covered sufficiently by the unit tests for the AbstractHerder class - Capture> validateCallback = newCapture(); - herder.validateConnectorConfig(EasyMock.eq(CONN2_CONFIG), capture(validateCallback)); - PowerMock.expectLastCall().andAnswer(() -> { + ArgumentCaptor> validateCallback = ArgumentCaptor.forClass(Callback.class); + doAnswer(invocation -> { validateCallback.getValue().onCompletion(null, CONN2_CONFIG_INFOS); return null; - }); + }).when(herder).validateConnectorConfig(eq(CONN2_CONFIG), validateCallback.capture()); - configBackingStore.putConnectorConfig(CONN2, CONN2_CONFIG); - PowerMock.expectLastCall().andThrow(new ConnectException("Error writing connector configuration to Kafka")); + doThrow(new ConnectException("Error writing connector configuration to Kafka")) + .when(configBackingStore).putConnectorConfig(CONN2, CONN2_CONFIG); - // verify that the exception from config backing store write is propagated via the callback - putConnectorCallback.onCompletion(EasyMock.anyObject(ConnectException.class), EasyMock.isNull()); - PowerMock.expectLastCall(); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - // These will occur just before/during the second tick - member.wakeup(); - PowerMock.expectLastCall(); - member.ensureActive(); - PowerMock.expectLastCall(); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - PowerMock.replayAll(); + // This will occur just before/during the second tick + doNothing().when(member).ensureActive(); herder.putConnectorConfig(CONN2, CONN2_CONFIG, false, putConnectorCallback); - // First tick runs the initial herder request, which issues an asynchronous request for + // This tick runs the initial herder request, which issues an asynchronous request for // connector validation herder.tick(); // Once that validation is complete, another request is added to the herder request queue // for actually performing the config write; this tick is for that request herder.tick(); - time.sleep(1000L); assertStatistics(3, 1, 100, 1000L); - PowerMock.verifyAll(); + // Verify that the exception thrown during the config backing store write is propagated via the callback + verify(putConnectorCallback).onCompletion(any(ConnectException.class), isNull()); + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore, putConnectorCallback); } @Test public void testCreateConnectorFailedValidation() throws Exception { - EasyMock.expect(member.memberId()).andStubReturn("leader"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); + when(member.memberId()).thenReturn("leader"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true); expectConfigRefreshAndSnapshot(SNAPSHOT); + when(statusBackingStore.connectors()).thenReturn(Collections.emptySet()); + doNothing().when(member).poll(anyLong()); + HashMap config = new HashMap<>(CONN2_CONFIG); config.remove(ConnectorConfig.NAME_CONFIG); - member.wakeup(); - PowerMock.expectLastCall(); - - // mock the actual validation since its asynchronous nature is difficult to test and should + // Mock the actual validation since its asynchronous nature is difficult to test and should // be covered sufficiently by the unit tests for the AbstractHerder class - Capture> validateCallback = newCapture(); - herder.validateConnectorConfig(EasyMock.eq(config), capture(validateCallback)); - PowerMock.expectLastCall().andAnswer(() -> { - // CONN2 creation should fail + ArgumentCaptor> validateCallback = ArgumentCaptor.forClass(Callback.class); + doAnswer(invocation -> { validateCallback.getValue().onCompletion(null, CONN2_INVALID_CONFIG_INFOS); return null; - }); - - Capture error = newCapture(); - putConnectorCallback.onCompletion(capture(error), EasyMock.isNull()); - PowerMock.expectLastCall(); - - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - // These will occur just before/during the second tick - member.wakeup(); - PowerMock.expectLastCall(); - - member.ensureActive(); - PowerMock.expectLastCall(); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - // No immediate action besides this -- change will be picked up via the config log - - PowerMock.replayAll(); + }).when(herder).validateConnectorConfig(eq(config), validateCallback.capture()); herder.putConnectorConfig(CONN2, config, false, putConnectorCallback); herder.tick(); - herder.tick(); - - assertTrue(error.hasCaptured()); - assertTrue(error.getValue() instanceof BadRequestException); + // We don't need another rebalance to occur + doNothing().when(member).ensureActive(); + herder.tick(); time.sleep(1000L); assertStatistics(3, 1, 100, 1000L); - PowerMock.verifyAll(); + ArgumentCaptor error = ArgumentCaptor.forClass(Throwable.class); + verify(putConnectorCallback).onCompletion(error.capture(), isNull()); + assertTrue(error.getValue() instanceof BadRequestException); + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore, putConnectorCallback); } @Test @@ -908,9 +795,7 @@ public void testConnectorNameConflictsWithWorkerGroupId() { Map config = new HashMap<>(CONN2_CONFIG); config.put(ConnectorConfig.NAME_CONFIG, "test-group"); - SinkConnector connectorMock = PowerMock.createMock(SinkConnector.class); - - PowerMock.replayAll(connectorMock); + SinkConnector connectorMock = mock(SinkConnector.class); // CONN2 creation should fail because the worker group id (connect-test-group) conflicts with // the consumer group id we would use for this sink @@ -921,518 +806,209 @@ public void testConnectorNameConflictsWithWorkerGroupId() { assertEquals( Collections.singletonList("Consumer group for sink connector named test-group conflicts with Connect worker group connect-test-group"), nameConfig.errorMessages()); - - PowerMock.verifyAll(); } @Test - public void testExactlyOnceSourceSupportValidation() { - herder = exactlyOnceHerder(); - Map config = new HashMap<>(); - config.put(SourceConnectorConfig.EXACTLY_ONCE_SUPPORT_CONFIG, REQUIRED.toString()); + public void testCreateConnectorAlreadyExists() throws Exception { + when(member.memberId()).thenReturn("leader"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); + expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true); + expectConfigRefreshAndSnapshot(SNAPSHOT); - SourceConnector connectorMock = PowerMock.createMock(SourceConnector.class); - EasyMock.expect(connectorMock.exactlyOnceSupport(EasyMock.eq(config))) - .andReturn(ExactlyOnceSupport.SUPPORTED); + when(statusBackingStore.connectors()).thenReturn(Collections.emptySet()); + doNothing().when(member).poll(anyLong()); - PowerMock.replayAll(connectorMock); + // mock the actual validation since its asynchronous nature is difficult to test and should + // be covered sufficiently by the unit tests for the AbstractHerder class + ArgumentCaptor> validateCallback = ArgumentCaptor.forClass(Callback.class); + doAnswer(invocation -> { + validateCallback.getValue().onCompletion(null, CONN1_CONFIG_INFOS); + return null; + }).when(herder).validateConnectorConfig(eq(CONN1_CONFIG), validateCallback.capture()); - Map validatedConfigs = herder.validateSourceConnectorConfig( - connectorMock, SourceConnectorConfig.configDef(), config); + herder.putConnectorConfig(CONN1, CONN1_CONFIG, false, putConnectorCallback); + herder.tick(); - List errors = validatedConfigs.get(SourceConnectorConfig.EXACTLY_ONCE_SUPPORT_CONFIG).errorMessages(); - assertEquals(Collections.emptyList(), errors); + // We don't need another rebalance to occur + doNothing().when(member).ensureActive(); + herder.tick(); + time.sleep(1000L); + assertStatistics(3, 1, 100, 1000L); - PowerMock.verifyAll(); + // CONN1 already exists + verify(putConnectorCallback).onCompletion(any(AlreadyExistsException.class), isNull()); + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore, putConnectorCallback); } @Test - public void testExactlyOnceSourceSupportValidationOnUnsupportedConnector() { - herder = exactlyOnceHerder(); - Map config = new HashMap<>(); - config.put(SourceConnectorConfig.EXACTLY_ONCE_SUPPORT_CONFIG, REQUIRED.toString()); - - SourceConnector connectorMock = PowerMock.createMock(SourceConnector.class); - EasyMock.expect(connectorMock.exactlyOnceSupport(EasyMock.eq(config))) - .andReturn(ExactlyOnceSupport.UNSUPPORTED); - - PowerMock.replayAll(connectorMock); + public void testDestroyConnector() throws Exception { + when(member.memberId()).thenReturn("leader"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); + // Start with one connector + expectRebalance(1, Arrays.asList(CONN1), Collections.emptyList(), true); + expectConfigRefreshAndSnapshot(SNAPSHOT); - Map validatedConfigs = herder.validateSourceConnectorConfig( - connectorMock, SourceConnectorConfig.configDef(), config); + ArgumentCaptor> onStart = ArgumentCaptor.forClass(Callback.class); + doAnswer(invocation -> { + onStart.getValue().onCompletion(null, TargetState.STARTED); + return true; + }).when(worker).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), onStart.capture()); + expectExecuteTaskReconfiguration(true, conn1SinkConfig, invocation -> TASK_CONFIGS); - List errors = validatedConfigs.get(SourceConnectorConfig.EXACTLY_ONCE_SUPPORT_CONFIG).errorMessages(); - assertEquals( - Collections.singletonList("The connector does not support exactly-once semantics with the provided configuration."), - errors); + when(statusBackingStore.connectors()).thenReturn(Collections.emptySet()); + doNothing().when(member).poll(anyLong()); - PowerMock.verifyAll(); - } + // And delete the connector + doNothing().when(configBackingStore).removeConnectorConfig(CONN1); + doNothing().when(putConnectorCallback).onCompletion(null, new Herder.Created<>(false, null)); - @Test - public void testExactlyOnceSourceSupportValidationOnUnknownConnector() { - herder = exactlyOnceHerder(); - Map config = new HashMap<>(); - config.put(SourceConnectorConfig.EXACTLY_ONCE_SUPPORT_CONFIG, REQUIRED.toString()); + herder.deleteConnectorConfig(CONN1, putConnectorCallback); - SourceConnector connectorMock = PowerMock.createMock(SourceConnector.class); - EasyMock.expect(connectorMock.exactlyOnceSupport(EasyMock.eq(config))) - .andReturn(null); + herder.tick(); + time.sleep(1000L); + assertStatistics("leaderUrl", false, 3, 1, 100, 1000L); - PowerMock.replayAll(connectorMock); + // The change eventually is reflected to the config topic and the deleted connector and + // tasks are revoked + TopicStatus fooStatus = new TopicStatus(FOO_TOPIC, CONN1, 0, time.milliseconds()); + TopicStatus barStatus = new TopicStatus(BAR_TOPIC, CONN1, 0, time.milliseconds()); + when(statusBackingStore.getAllTopics(eq(CONN1))).thenReturn(new HashSet<>(Arrays.asList(fooStatus, barStatus))); + doNothing().when(statusBackingStore).deleteTopic(eq(CONN1), eq(FOO_TOPIC)); + doNothing().when(statusBackingStore).deleteTopic(eq(CONN1), eq(BAR_TOPIC)); - Map validatedConfigs = herder.validateSourceConnectorConfig( - connectorMock, SourceConnectorConfig.configDef(), config); + expectRebalance(Arrays.asList(CONN1), Arrays.asList(TASK1), + ConnectProtocol.Assignment.NO_ERROR, 2, "leader", "leaderUrl", + Collections.emptyList(), Collections.emptyList(), 0, true); + expectConfigRefreshAndSnapshot(ClusterConfigState.EMPTY); + doNothing().when(member).requestRejoin(); - List errors = validatedConfigs.get(SourceConnectorConfig.EXACTLY_ONCE_SUPPORT_CONFIG).errorMessages(); - assertFalse(errors.isEmpty()); - assertTrue( - "Error message did not contain expected text: " + errors.get(0), - errors.get(0).contains("The connector does not implement the API required for preflight validation of exactly-once source support.")); - assertEquals(1, errors.size()); + configUpdateListener.onConnectorConfigRemove(CONN1); // read updated config that removes the connector + herder.configState = ClusterConfigState.EMPTY; + herder.tick(); + time.sleep(1000L); + assertStatistics("leaderUrl", true, 3, 1, 100, 2100L); - PowerMock.verifyAll(); + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore, putConnectorCallback); } @Test - public void testExactlyOnceSourceSupportValidationHandlesConnectorErrorsGracefully() { - herder = exactlyOnceHerder(); - Map config = new HashMap<>(); - config.put(SourceConnectorConfig.EXACTLY_ONCE_SUPPORT_CONFIG, REQUIRED.toString()); + public void testRestartConnector() throws Exception { - SourceConnector connectorMock = PowerMock.createMock(SourceConnector.class); - String errorMessage = "time to add a new unit test :)"; - EasyMock.expect(connectorMock.exactlyOnceSupport(EasyMock.eq(config))) - .andThrow(new NullPointerException(errorMessage)); + // get the initial assignment + when(member.memberId()).thenReturn("leader"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); + expectRebalance(1, singletonList(CONN1), Collections.emptyList(), true); + expectConfigRefreshAndSnapshot(SNAPSHOT); - PowerMock.replayAll(connectorMock); + when(statusBackingStore.connectors()).thenReturn(Collections.emptySet()); + doNothing().when(member).poll(anyLong()); - Map validatedConfigs = herder.validateSourceConnectorConfig( - connectorMock, SourceConnectorConfig.configDef(), config); + ArgumentCaptor> onStart = ArgumentCaptor.forClass(Callback.class); + doAnswer(invocation -> { + onStart.getValue().onCompletion(null, TargetState.STARTED); + return true; + }).when(worker).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), onStart.capture()); + expectExecuteTaskReconfiguration(true, conn1SinkConfig, invocation -> TASK_CONFIGS); - List errors = validatedConfigs.get(SourceConnectorConfig.EXACTLY_ONCE_SUPPORT_CONFIG).errorMessages(); - assertFalse(errors.isEmpty()); - assertTrue( - "Error message did not contain expected text: " + errors.get(0), - errors.get(0).contains(errorMessage)); - assertEquals(1, errors.size()); + // Initial rebalance where this member becomes the leader + herder.tick(); + + doNothing().when(member).ensureActive(); + + doNothing().when(worker).stopAndAwaitConnector(CONN1); + + FutureCallback callback = new FutureCallback<>(); + herder.restartConnector(CONN1, callback); + herder.tick(); + callback.get(1000L, TimeUnit.MILLISECONDS); - PowerMock.verifyAll(); + verify(worker, times(2)).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), any()); + verify(worker, times(2)).connectorTaskConfigs(eq(CONN1), any()); + verify(worker).stopAndAwaitConnector(CONN1); + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); } @Test - public void testExactlyOnceSourceSupportValidationWhenExactlyOnceNotEnabledOnWorker() { - Map config = new HashMap<>(); - config.put(SourceConnectorConfig.EXACTLY_ONCE_SUPPORT_CONFIG, REQUIRED.toString()); - - SourceConnector connectorMock = PowerMock.createMock(SourceConnector.class); - EasyMock.expect(connectorMock.exactlyOnceSupport(EasyMock.eq(config))) - .andReturn(ExactlyOnceSupport.SUPPORTED); + public void testRestartUnknownConnector() throws Exception { + // get the initial assignment + when(member.memberId()).thenReturn("leader"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); + expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true); + expectConfigRefreshAndSnapshot(SNAPSHOT); - PowerMock.replayAll(connectorMock); + when(statusBackingStore.connectors()).thenReturn(Collections.emptySet()); + doNothing().when(member).poll(anyLong()); - Map validatedConfigs = herder.validateSourceConnectorConfig( - connectorMock, SourceConnectorConfig.configDef(), config); + herder.tick(); - List errors = validatedConfigs.get(SourceConnectorConfig.EXACTLY_ONCE_SUPPORT_CONFIG).errorMessages(); - assertEquals( - Collections.singletonList("This worker does not have exactly-once source support enabled."), - errors); + // now handle the connector restart + doNothing().when(member).ensureActive(); + FutureCallback callback = new FutureCallback<>(); + herder.restartConnector(CONN2, callback); + herder.tick(); - PowerMock.verifyAll(); + ExecutionException e = assertThrows(ExecutionException.class, () -> callback.get(1000L, TimeUnit.MILLISECONDS)); + assertEquals(NotFoundException.class, e.getCause().getClass()); + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); } @Test - public void testExactlyOnceSourceSupportValidationHandlesInvalidValuesGracefully() { - herder = exactlyOnceHerder(); - Map config = new HashMap<>(); - config.put(SourceConnectorConfig.EXACTLY_ONCE_SUPPORT_CONFIG, "invalid"); - - SourceConnector connectorMock = PowerMock.createMock(SourceConnector.class); - - PowerMock.replayAll(connectorMock); - - Map validatedConfigs = herder.validateSourceConnectorConfig( - connectorMock, SourceConnectorConfig.configDef(), config); - - List errors = validatedConfigs.get(SourceConnectorConfig.EXACTLY_ONCE_SUPPORT_CONFIG).errorMessages(); - assertFalse(errors.isEmpty()); - assertTrue( - "Error message did not contain expected text: " + errors.get(0), - errors.get(0).contains("String must be one of (case insensitive): ")); - assertEquals(1, errors.size()); - - PowerMock.verifyAll(); - } - - @Test - public void testConnectorTransactionBoundaryValidation() { - herder = exactlyOnceHerder(); - Map config = new HashMap<>(); - config.put(SourceConnectorConfig.TRANSACTION_BOUNDARY_CONFIG, CONNECTOR.toString()); - - SourceConnector connectorMock = PowerMock.createMock(SourceConnector.class); - EasyMock.expect(connectorMock.canDefineTransactionBoundaries(EasyMock.eq(config))) - .andReturn(ConnectorTransactionBoundaries.SUPPORTED); - - PowerMock.replayAll(connectorMock); - - Map validatedConfigs = herder.validateSourceConnectorConfig( - connectorMock, SourceConnectorConfig.configDef(), config); - - List errors = validatedConfigs.get(SourceConnectorConfig.TRANSACTION_BOUNDARY_CONFIG).errorMessages(); - assertEquals(Collections.emptyList(), errors); - - PowerMock.verifyAll(); - } - - @Test - public void testConnectorTransactionBoundaryValidationOnUnsupportedConnector() { - herder = exactlyOnceHerder(); - Map config = new HashMap<>(); - config.put(SourceConnectorConfig.TRANSACTION_BOUNDARY_CONFIG, CONNECTOR.toString()); - - SourceConnector connectorMock = PowerMock.createMock(SourceConnector.class); - EasyMock.expect(connectorMock.canDefineTransactionBoundaries(EasyMock.eq(config))) - .andReturn(ConnectorTransactionBoundaries.UNSUPPORTED); - - PowerMock.replayAll(connectorMock); - - Map validatedConfigs = herder.validateSourceConnectorConfig( - connectorMock, SourceConnectorConfig.configDef(), config); - - List errors = validatedConfigs.get(SourceConnectorConfig.TRANSACTION_BOUNDARY_CONFIG).errorMessages(); - assertFalse(errors.isEmpty()); - assertTrue( - "Error message did not contain expected text: " + errors.get(0), - errors.get(0).contains("The connector does not support connector-defined transaction boundaries with the given configuration.")); - assertEquals(1, errors.size()); - - PowerMock.verifyAll(); - } - - @Test - public void testConnectorTransactionBoundaryValidationHandlesConnectorErrorsGracefully() { - herder = exactlyOnceHerder(); - Map config = new HashMap<>(); - config.put(SourceConnectorConfig.TRANSACTION_BOUNDARY_CONFIG, CONNECTOR.toString()); - - SourceConnector connectorMock = PowerMock.createMock(SourceConnector.class); - String errorMessage = "Wait I thought we tested for this?"; - EasyMock.expect(connectorMock.canDefineTransactionBoundaries(EasyMock.eq(config))) - .andThrow(new ConnectException(errorMessage)); - - PowerMock.replayAll(connectorMock); - - Map validatedConfigs = herder.validateSourceConnectorConfig( - connectorMock, SourceConnectorConfig.configDef(), config); - - List errors = validatedConfigs.get(SourceConnectorConfig.TRANSACTION_BOUNDARY_CONFIG).errorMessages(); - assertFalse(errors.isEmpty()); - assertTrue( - "Error message did not contain expected text: " + errors.get(0), - errors.get(0).contains(errorMessage)); - assertEquals(1, errors.size()); - - PowerMock.verifyAll(); - } - - @Test - public void testConnectorTransactionBoundaryValidationHandlesInvalidValuesGracefully() { - herder = exactlyOnceHerder(); - Map config = new HashMap<>(); - config.put(SourceConnectorConfig.TRANSACTION_BOUNDARY_CONFIG, "CONNECTOR.toString()"); - - SourceConnector connectorMock = PowerMock.createMock(SourceConnector.class); - - PowerMock.replayAll(connectorMock); - - Map validatedConfigs = herder.validateSourceConnectorConfig( - connectorMock, SourceConnectorConfig.configDef(), config); - - List errors = validatedConfigs.get(SourceConnectorConfig.TRANSACTION_BOUNDARY_CONFIG).errorMessages(); - assertFalse(errors.isEmpty()); - assertTrue( - "Error message did not contain expected text: " + errors.get(0), - errors.get(0).contains("String must be one of (case insensitive): ")); - assertEquals(1, errors.size()); - - PowerMock.verifyAll(); - } - - @Test - public void testCreateConnectorAlreadyExists() throws Exception { - EasyMock.expect(member.memberId()).andStubReturn("leader"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - - // mock the actual validation since its asynchronous nature is difficult to test and should - // be covered sufficiently by the unit tests for the AbstractHerder class - Capture> validateCallback = newCapture(); - herder.validateConnectorConfig(EasyMock.eq(CONN1_CONFIG), capture(validateCallback)); - PowerMock.expectLastCall().andAnswer(() -> { - validateCallback.getValue().onCompletion(null, CONN1_CONFIG_INFOS); - return null; - }); - - expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true); - expectConfigRefreshAndSnapshot(SNAPSHOT); - - member.wakeup(); - PowerMock.expectLastCall(); - // CONN1 already exists - putConnectorCallback.onCompletion(EasyMock.anyObject(), EasyMock.isNull()); - PowerMock.expectLastCall(); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - // These will occur just before/during the second tick - member.wakeup(); - PowerMock.expectLastCall(); - member.ensureActive(); - PowerMock.expectLastCall(); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - // No immediate action besides this -- change will be picked up via the config log - - PowerMock.replayAll(); - - herder.putConnectorConfig(CONN1, CONN1_CONFIG, false, putConnectorCallback); - herder.tick(); - herder.tick(); - - time.sleep(1000L); - assertStatistics(3, 1, 100, 1000L); - - PowerMock.verifyAll(); - } - - @Test - public void testDestroyConnector() throws Exception { - EasyMock.expect(member.memberId()).andStubReturn("leader"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - // Start with one connector - expectRebalance(1, Arrays.asList(CONN1), Collections.emptyList(), true); + public void testRestartConnectorRedirectToLeader() throws Exception { + // get the initial assignment + when(member.memberId()).thenReturn("member"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); + expectRebalance(1, Collections.emptyList(), Collections.emptyList()); expectConfigRefreshAndSnapshot(SNAPSHOT); - Capture> onStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onStart)); - PowerMock.expectLastCall().andAnswer(() -> { - onStart.getValue().onCompletion(null, TargetState.STARTED); - return true; - }); - member.wakeup(); - PowerMock.expectLastCall(); - expectExecuteTaskReconfiguration(true, conn1SinkConfig, () -> TASK_CONFIGS); - - // And delete the connector - configBackingStore.removeConnectorConfig(CONN1); - PowerMock.expectLastCall(); - putConnectorCallback.onCompletion(null, new Herder.Created<>(false, null)); - PowerMock.expectLastCall(); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - // The change eventually is reflected to the config topic and the deleted connector and - // tasks are revoked - member.wakeup(); - PowerMock.expectLastCall(); - TopicStatus fooStatus = new TopicStatus(FOO_TOPIC, CONN1, 0, time.milliseconds()); - TopicStatus barStatus = new TopicStatus(BAR_TOPIC, CONN1, 0, time.milliseconds()); - EasyMock.expect(statusBackingStore.getAllTopics(EasyMock.eq(CONN1))).andReturn(new HashSet<>(Arrays.asList(fooStatus, barStatus))).times(2); - statusBackingStore.deleteTopic(EasyMock.eq(CONN1), EasyMock.eq(FOO_TOPIC)); - PowerMock.expectLastCall().times(2); - statusBackingStore.deleteTopic(EasyMock.eq(CONN1), EasyMock.eq(BAR_TOPIC)); - PowerMock.expectLastCall().times(2); - expectRebalance(Arrays.asList(CONN1), Arrays.asList(TASK1), - ConnectProtocol.Assignment.NO_ERROR, 2, "leader", "leaderUrl", - Collections.emptyList(), Collections.emptyList(), 0, true); - expectConfigRefreshAndSnapshot(ClusterConfigState.EMPTY); - member.requestRejoin(); - PowerMock.expectLastCall(); - PowerMock.replayAll(); - - herder.deleteConnectorConfig(CONN1, putConnectorCallback); - herder.tick(); - time.sleep(1000L); - assertStatistics("leaderUrl", false, 3, 1, 100, 1000L); + doNothing().when(member).poll(anyLong()); - configUpdateListener.onConnectorConfigRemove(CONN1); // read updated config that removes the connector - herder.configState = ClusterConfigState.EMPTY; herder.tick(); - time.sleep(1000L); - assertStatistics("leaderUrl", true, 3, 1, 100, 2100L); - - PowerMock.verifyAll(); - } - - @Test - public void testRestartConnector() throws Exception { - - // get the initial assignment - EasyMock.expect(member.memberId()).andStubReturn("leader"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - expectRebalance(1, singletonList(CONN1), Collections.emptyList(), true); - expectConfigRefreshAndSnapshot(SNAPSHOT); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - Capture> onFirstStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onFirstStart)); - PowerMock.expectLastCall().andAnswer(() -> { - onFirstStart.getValue().onCompletion(null, TargetState.STARTED); - return true; - }); - member.wakeup(); - PowerMock.expectLastCall(); - expectExecuteTaskReconfiguration(true, conn1SinkConfig, () -> TASK_CONFIGS); // now handle the connector restart - member.wakeup(); - PowerMock.expectLastCall(); - member.ensureActive(); - PowerMock.expectLastCall(); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - worker.stopAndAwaitConnector(CONN1); - PowerMock.expectLastCall(); - Capture> onSecondStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onSecondStart)); - PowerMock.expectLastCall().andAnswer(() -> { - onSecondStart.getValue().onCompletion(null, TargetState.STARTED); - return true; - }); - member.wakeup(); - PowerMock.expectLastCall(); - expectExecuteTaskReconfiguration(true, conn1SinkConfig, () -> TASK_CONFIGS); - - PowerMock.replayAll(); + doNothing().when(member).ensureActive(); - herder.tick(); FutureCallback callback = new FutureCallback<>(); herder.restartConnector(CONN1, callback); herder.tick(); - callback.get(1000L, TimeUnit.MILLISECONDS); - PowerMock.verifyAll(); + ExecutionException e = assertThrows(ExecutionException.class, () -> callback.get(1000L, TimeUnit.MILLISECONDS)); + assertEquals(NotLeaderException.class, e.getCause().getClass()); + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); } @Test - public void testRestartUnknownConnector() throws Exception { + public void testRestartConnectorRedirectToOwner() throws Exception { // get the initial assignment - EasyMock.expect(member.memberId()).andStubReturn("leader"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); + when(member.memberId()).thenReturn("leader"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true); expectConfigRefreshAndSnapshot(SNAPSHOT); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - // now handle the connector restart - member.wakeup(); - PowerMock.expectLastCall(); - member.ensureActive(); - PowerMock.expectLastCall(); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - PowerMock.replayAll(); + when(statusBackingStore.connectors()).thenReturn(Collections.emptySet()); + doNothing().when(member).poll(anyLong()); herder.tick(); - FutureCallback callback = new FutureCallback<>(); - herder.restartConnector(CONN2, callback); - herder.tick(); - try { - callback.get(1000L, TimeUnit.MILLISECONDS); - fail("Expected NotFoundException to be raised"); - } catch (ExecutionException e) { - assertTrue(e.getCause() instanceof NotFoundException); - } - - PowerMock.verifyAll(); - } - - @Test - public void testRestartConnectorRedirectToLeader() throws Exception { - // get the initial assignment - EasyMock.expect(member.memberId()).andStubReturn("member"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - expectRebalance(1, Collections.emptyList(), Collections.emptyList()); - expectConfigRefreshAndSnapshot(SNAPSHOT); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - // now handle the connector restart - member.wakeup(); - PowerMock.expectLastCall(); - member.ensureActive(); - PowerMock.expectLastCall(); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - PowerMock.replayAll(); - - herder.tick(); - FutureCallback callback = new FutureCallback<>(); - herder.restartConnector(CONN1, callback); - herder.tick(); - - try { - callback.get(1000L, TimeUnit.MILLISECONDS); - fail("Expected NotLeaderException to be raised"); - } catch (ExecutionException e) { - assertTrue(e.getCause() instanceof NotLeaderException); - } - - PowerMock.verifyAll(); - } - - @Test - public void testRestartConnectorRedirectToOwner() throws Exception { - // get the initial assignment - EasyMock.expect(member.memberId()).andStubReturn("leader"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true); - expectConfigRefreshAndSnapshot(SNAPSHOT); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); // now handle the connector restart - member.wakeup(); - PowerMock.expectLastCall(); - member.ensureActive(); - PowerMock.expectLastCall(); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - + doNothing().when(member).ensureActive(); String ownerUrl = "ownerUrl"; - EasyMock.expect(member.ownerUrl(CONN1)).andReturn(ownerUrl); + when(member.ownerUrl(CONN1)).thenReturn(ownerUrl); - PowerMock.replayAll(); - - herder.tick(); time.sleep(1000L); assertStatistics(3, 1, 100, 1000L); FutureCallback callback = new FutureCallback<>(); herder.restartConnector(CONN1, callback); - herder.tick(); + herder.tick(); time.sleep(2000L); assertStatistics(3, 1, 100, 3000L); - try { - callback.get(1000L, TimeUnit.MILLISECONDS); - fail("Expected NotLeaderException to be raised"); - } catch (ExecutionException e) { - assertTrue(e.getCause() instanceof NotAssignedException); - NotAssignedException notAssignedException = (NotAssignedException) e.getCause(); - assertEquals(ownerUrl, notAssignedException.forwardUrl()); - } + ExecutionException e = assertThrows(ExecutionException.class, () -> callback.get(1000L, TimeUnit.MILLISECONDS)); + assertEquals(NotAssignedException.class, e.getCause().getClass()); + assertEquals(ownerUrl, ((NotAssignedException) e.getCause()).forwardUrl()); - PowerMock.verifyAll(); + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); } @Test @@ -1441,23 +1017,18 @@ public void testRestartConnectorAndTasksUnknownConnector() throws Exception { RestartRequest restartRequest = new RestartRequest(connectorName, false, true); // get the initial assignment - EasyMock.expect(member.memberId()).andStubReturn("leader"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); + when(member.memberId()).thenReturn("leader"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true); expectConfigRefreshAndSnapshot(SNAPSHOT); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - // now handle the connector restart - member.wakeup(); - PowerMock.expectLastCall(); - member.ensureActive(); - PowerMock.expectLastCall(); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + when(statusBackingStore.connectors()).thenReturn(Collections.emptySet()); + doNothing().when(member).poll(anyLong()); - PowerMock.replayAll(); + herder.tick(); + // now handle the connector restart + doNothing().when(member).ensureActive(); herder.tick(); FutureCallback callback = new FutureCallback<>(); herder.restartConnectorAndTasks(restartRequest, callback); @@ -1466,560 +1037,431 @@ public void testRestartConnectorAndTasksUnknownConnector() throws Exception { assertTrue(ee.getCause() instanceof NotFoundException); assertTrue(ee.getMessage().contains("Unknown connector:")); - PowerMock.verifyAll(); + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); } @Test public void testRestartConnectorAndTasksNotLeader() throws Exception { - RestartRequest restartRequest = new RestartRequest(CONN1, false, true); - // get the initial assignment - EasyMock.expect(member.memberId()).andStubReturn("member"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); + when(member.memberId()).thenReturn("member"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); expectRebalance(1, Collections.emptyList(), Collections.emptyList()); expectConfigRefreshAndSnapshot(SNAPSHOT); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + doNothing().when(member).poll(anyLong()); - // now handle the connector restart - member.wakeup(); - PowerMock.expectLastCall(); - member.ensureActive(); - PowerMock.expectLastCall(); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + herder.tick(); - PowerMock.replayAll(); + // now handle the connector restart + doNothing().when(member).ensureActive(); herder.tick(); FutureCallback callback = new FutureCallback<>(); + RestartRequest restartRequest = new RestartRequest(CONN1, false, true); herder.restartConnectorAndTasks(restartRequest, callback); herder.tick(); ExecutionException ee = assertThrows(ExecutionException.class, () -> callback.get(1000L, TimeUnit.MILLISECONDS)); assertTrue(ee.getCause() instanceof NotLeaderException); - PowerMock.verifyAll(); + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); } @Test public void testRestartConnectorAndTasksUnknownStatus() throws Exception { - RestartRequest restartRequest = new RestartRequest(CONN1, false, true); - EasyMock.expect(herder.buildRestartPlan(restartRequest)).andReturn(Optional.empty()).anyTimes(); - - configBackingStore.putRestartRequest(restartRequest); - PowerMock.expectLastCall(); - // get the initial assignment - EasyMock.expect(member.memberId()).andStubReturn("leader"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); + when(member.memberId()).thenReturn("leader"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true); expectConfigRefreshAndSnapshot(SNAPSHOT); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - // now handle the connector restart - member.wakeup(); - PowerMock.expectLastCall(); - member.ensureActive(); - PowerMock.expectLastCall(); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - PowerMock.replayAll(); + when(statusBackingStore.connectors()).thenReturn(Collections.emptySet()); + doNothing().when(member).poll(anyLong()); herder.tick(); + + // now handle the connector restart + doNothing().when(member).ensureActive(); + when(statusBackingStore.get(CONN1)).thenReturn(null); + RestartRequest restartRequest = new RestartRequest(CONN1, false, true); + doNothing().when(configBackingStore).putRestartRequest(restartRequest); + FutureCallback callback = new FutureCallback<>(); herder.restartConnectorAndTasks(restartRequest, callback); herder.tick(); ExecutionException ee = assertThrows(ExecutionException.class, () -> callback.get(1000L, TimeUnit.MILLISECONDS)); assertTrue(ee.getCause() instanceof NotFoundException); assertTrue(ee.getMessage().contains("Status for connector")); - PowerMock.verifyAll(); + + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); } @Test public void testRestartConnectorAndTasksSuccess() throws Exception { - RestartPlan restartPlan = PowerMock.createMock(RestartPlan.class); - ConnectorStateInfo connectorStateInfo = PowerMock.createMock(ConnectorStateInfo.class); - EasyMock.expect(restartPlan.restartConnectorStateInfo()).andReturn(connectorStateInfo).anyTimes(); - - RestartRequest restartRequest = new RestartRequest(CONN1, false, true); - EasyMock.expect(herder.buildRestartPlan(restartRequest)).andReturn(Optional.of(restartPlan)).anyTimes(); - - configBackingStore.putRestartRequest(restartRequest); - PowerMock.expectLastCall(); - // get the initial assignment - EasyMock.expect(member.memberId()).andStubReturn("leader"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); + when(member.memberId()).thenReturn("leader"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true); expectConfigRefreshAndSnapshot(SNAPSHOT); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + + when(statusBackingStore.connectors()).thenReturn(Collections.emptySet()); + doNothing().when(member).poll(anyLong()); + + herder.tick(); // now handle the connector restart - member.wakeup(); - PowerMock.expectLastCall(); - member.ensureActive(); - PowerMock.expectLastCall(); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + doNothing().when(member).ensureActive(); - PowerMock.replayAll(); + RestartPlan restartPlan = mock(RestartPlan.class); + ConnectorStateInfo connectorStateInfo = mock(ConnectorStateInfo.class); + when(restartPlan.restartConnectorStateInfo()).thenReturn(connectorStateInfo); + RestartRequest restartRequest = new RestartRequest(CONN1, false, true); + doReturn(Optional.of(restartPlan)).when(herder).buildRestartPlan(restartRequest); + doNothing().when(configBackingStore).putRestartRequest(restartRequest); - herder.tick(); FutureCallback callback = new FutureCallback<>(); herder.restartConnectorAndTasks(restartRequest, callback); herder.tick(); assertEquals(connectorStateInfo, callback.get(1000L, TimeUnit.MILLISECONDS)); - PowerMock.verifyAll(); + + verifyNoMoreInteractions(restartPlan, worker, member, configBackingStore, statusBackingStore); } @Test public void testDoRestartConnectorAndTasksEmptyPlan() { RestartRequest restartRequest = new RestartRequest(CONN1, false, true); - EasyMock.expect(herder.buildRestartPlan(restartRequest)).andReturn(Optional.empty()).anyTimes(); - - PowerMock.replayAll(); - + doReturn(Optional.empty()).when(herder).buildRestartPlan(restartRequest); herder.doRestartConnectorAndTasks(restartRequest); - PowerMock.verifyAll(); + + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); } @Test public void testDoRestartConnectorAndTasksNoAssignments() { ConnectorTaskId taskId = new ConnectorTaskId(CONN1, 0); RestartRequest restartRequest = new RestartRequest(CONN1, false, true); - RestartPlan restartPlan = PowerMock.createMock(RestartPlan.class); - EasyMock.expect(restartPlan.shouldRestartConnector()).andReturn(true).anyTimes(); - EasyMock.expect(restartPlan.shouldRestartTasks()).andReturn(true).anyTimes(); - EasyMock.expect(restartPlan.taskIdsToRestart()).andReturn(Collections.singletonList(taskId)).anyTimes(); + RestartPlan restartPlan = mock(RestartPlan.class); + when(restartPlan.shouldRestartConnector()).thenReturn(true); + when(restartPlan.taskIdsToRestart()).thenReturn(Collections.singletonList(taskId)); - EasyMock.expect(herder.buildRestartPlan(restartRequest)).andReturn(Optional.of(restartPlan)).anyTimes(); + doReturn(Optional.of(restartPlan)).when(herder).buildRestartPlan(restartRequest); - PowerMock.replayAll(); herder.assignment = ExtendedAssignment.empty(); herder.doRestartConnectorAndTasks(restartRequest); - PowerMock.verifyAll(); + + verifyNoMoreInteractions(restartPlan, worker, member, configBackingStore, statusBackingStore); } @Test public void testDoRestartConnectorAndTasksOnlyConnector() { ConnectorTaskId taskId = new ConnectorTaskId(CONN1, 0); RestartRequest restartRequest = new RestartRequest(CONN1, false, true); - RestartPlan restartPlan = PowerMock.createMock(RestartPlan.class); - EasyMock.expect(restartPlan.shouldRestartConnector()).andReturn(true).anyTimes(); - EasyMock.expect(restartPlan.shouldRestartTasks()).andReturn(true).anyTimes(); - EasyMock.expect(restartPlan.taskIdsToRestart()).andReturn(Collections.singletonList(taskId)).anyTimes(); + RestartPlan restartPlan = mock(RestartPlan.class); + when(restartPlan.shouldRestartConnector()).thenReturn(true); + when(restartPlan.taskIdsToRestart()).thenReturn(Collections.singletonList(taskId)); - EasyMock.expect(herder.buildRestartPlan(restartRequest)).andReturn(Optional.of(restartPlan)).anyTimes(); + doReturn(Optional.of(restartPlan)).when(herder).buildRestartPlan(restartRequest); - herder.assignment = PowerMock.createMock(ExtendedAssignment.class); - EasyMock.expect(herder.assignment.connectors()).andReturn(Collections.singletonList(CONN1)).anyTimes(); - EasyMock.expect(herder.assignment.tasks()).andReturn(Collections.emptyList()).anyTimes(); + herder.assignment = mock(ExtendedAssignment.class); + when(herder.assignment.connectors()).thenReturn(Collections.singletonList(CONN1)); + when(herder.assignment.tasks()).thenReturn(Collections.emptyList()); - worker.stopAndAwaitConnector(CONN1); - PowerMock.expectLastCall(); + herder.configState = SNAPSHOT; - Capture> stateCallback = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.anyObject(TargetState.class), capture(stateCallback)); - PowerMock.expectLastCall().andAnswer(() -> { - stateCallback.getValue().onCompletion(null, TargetState.STARTED); - return true; - }); - member.wakeup(); - PowerMock.expectLastCall(); + doNothing().when(worker).stopAndAwaitConnector(CONN1); + ConnectorStatus status = new ConnectorStatus(CONN1, AbstractStatus.State.RESTARTING, WORKER_ID, 0); + doNothing().when(statusBackingStore).put(eq(status)); - herder.onRestart(CONN1); - EasyMock.expectLastCall(); + ArgumentCaptor> stateCallback = ArgumentCaptor.forClass(Callback.class); + doAnswer(invocation -> { + stateCallback.getValue().onCompletion(null, TargetState.STARTED); + return true; + }).when(worker).startConnector(eq(CONN1), any(), any(), eq(herder), any(), stateCallback.capture()); + doNothing().when(member).wakeup(); - PowerMock.replayAll(); herder.doRestartConnectorAndTasks(restartRequest); - PowerMock.verifyAll(); + + verifyNoMoreInteractions(restartPlan, worker, member, configBackingStore, statusBackingStore); } @Test public void testDoRestartConnectorAndTasksOnlyTasks() { RestartRequest restartRequest = new RestartRequest(CONN1, false, true); - RestartPlan restartPlan = PowerMock.createMock(RestartPlan.class); - EasyMock.expect(restartPlan.shouldRestartConnector()).andReturn(true).anyTimes(); - EasyMock.expect(restartPlan.shouldRestartTasks()).andReturn(true).anyTimes(); + RestartPlan restartPlan = mock(RestartPlan.class); + when(restartPlan.shouldRestartConnector()).thenReturn(true); // The connector has three tasks - EasyMock.expect(restartPlan.taskIdsToRestart()).andReturn(Arrays.asList(TASK0, TASK1, TASK2)).anyTimes(); - EasyMock.expect(restartPlan.restartTaskCount()).andReturn(3).anyTimes(); - EasyMock.expect(restartPlan.totalTaskCount()).andReturn(3).anyTimes(); - EasyMock.expect(herder.buildRestartPlan(restartRequest)).andReturn(Optional.of(restartPlan)).anyTimes(); + when(restartPlan.taskIdsToRestart()).thenReturn(Arrays.asList(TASK0, TASK1, TASK2)); + when(restartPlan.totalTaskCount()).thenReturn(3); + doReturn(Optional.of(restartPlan)).when(herder).buildRestartPlan(restartRequest); - herder.assignment = PowerMock.createMock(ExtendedAssignment.class); - EasyMock.expect(herder.assignment.connectors()).andReturn(Collections.emptyList()).anyTimes(); + herder.assignment = mock(ExtendedAssignment.class); + when(herder.assignment.connectors()).thenReturn(Collections.emptyList()); // But only one task is assigned to this worker - EasyMock.expect(herder.assignment.tasks()).andReturn(Collections.singletonList(TASK0)).anyTimes(); + when(herder.assignment.tasks()).thenReturn(Collections.singletonList(TASK0)); herder.configState = SNAPSHOT; - worker.stopAndAwaitTasks(Collections.singletonList(TASK0)); - PowerMock.expectLastCall(); + doNothing().when(worker).stopAndAwaitTasks(Collections.singletonList(TASK0)); - herder.onRestart(TASK0); - EasyMock.expectLastCall(); + TaskStatus status = new TaskStatus(TASK0, AbstractStatus.State.RESTARTING, WORKER_ID, 0); + doNothing().when(statusBackingStore).put(eq(status)); - worker.startSourceTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.anyObject(TargetState.class)); - PowerMock.expectLastCall().andReturn(true); + when(worker.startSourceTask(eq(TASK0), any(), any(), any(), eq(herder), any())).thenReturn(true); - PowerMock.replayAll(); herder.doRestartConnectorAndTasks(restartRequest); - PowerMock.verifyAll(); + + verifyNoMoreInteractions(restartPlan, worker, member, configBackingStore, statusBackingStore); } @Test public void testDoRestartConnectorAndTasksBoth() { ConnectorTaskId taskId = new ConnectorTaskId(CONN1, 0); RestartRequest restartRequest = new RestartRequest(CONN1, false, true); - RestartPlan restartPlan = PowerMock.createMock(RestartPlan.class); - EasyMock.expect(restartPlan.shouldRestartConnector()).andReturn(true).anyTimes(); - EasyMock.expect(restartPlan.shouldRestartTasks()).andReturn(true).anyTimes(); - EasyMock.expect(restartPlan.taskIdsToRestart()).andReturn(Collections.singletonList(taskId)).anyTimes(); - EasyMock.expect(restartPlan.restartTaskCount()).andReturn(1).anyTimes(); - EasyMock.expect(restartPlan.totalTaskCount()).andReturn(1).anyTimes(); - EasyMock.expect(herder.buildRestartPlan(restartRequest)).andReturn(Optional.of(restartPlan)).anyTimes(); - - herder.assignment = PowerMock.createMock(ExtendedAssignment.class); - EasyMock.expect(herder.assignment.connectors()).andReturn(Collections.singletonList(CONN1)).anyTimes(); - EasyMock.expect(herder.assignment.tasks()).andReturn(Collections.singletonList(taskId)).anyTimes(); + RestartPlan restartPlan = mock(RestartPlan.class); + when(restartPlan.shouldRestartConnector()).thenReturn(true); + when(restartPlan.taskIdsToRestart()).thenReturn(Collections.singletonList(taskId)); + when(restartPlan.totalTaskCount()).thenReturn(1); + doReturn(Optional.of(restartPlan)).when(herder).buildRestartPlan(restartRequest); - herder.configState = SNAPSHOT; + herder.assignment = mock(ExtendedAssignment.class); + when(herder.assignment.connectors()).thenReturn(Collections.singletonList(CONN1)); + when(herder.assignment.tasks()).thenReturn(Collections.singletonList(taskId)); - worker.stopAndAwaitConnector(CONN1); - PowerMock.expectLastCall(); + herder.configState = SNAPSHOT; - Capture> stateCallback = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.anyObject(TargetState.class), capture(stateCallback)); + doNothing().when(worker).stopAndAwaitConnector(CONN1); + ConnectorStatus status = new ConnectorStatus(CONN1, AbstractStatus.State.RESTARTING, WORKER_ID, 0); + doNothing().when(statusBackingStore).put(eq(status)); - herder.onRestart(CONN1); - EasyMock.expectLastCall(); + ArgumentCaptor> stateCallback = ArgumentCaptor.forClass(Callback.class); + doAnswer(invocation -> { + stateCallback.getValue().onCompletion(null, TargetState.STARTED); + return true; + }).when(worker).startConnector(eq(CONN1), any(), any(), eq(herder), any(), stateCallback.capture()); + doNothing().when(member).wakeup(); - worker.stopAndAwaitTasks(Collections.singletonList(taskId)); - PowerMock.expectLastCall(); + doNothing().when(worker).stopAndAwaitTasks(Collections.singletonList(taskId)); - herder.onRestart(taskId); - EasyMock.expectLastCall(); + TaskStatus taskStatus = new TaskStatus(TASK0, AbstractStatus.State.RESTARTING, WORKER_ID, 0); + doNothing().when(statusBackingStore).put(eq(taskStatus)); - worker.startSourceTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.anyObject(TargetState.class)); - PowerMock.expectLastCall().andReturn(true); + when(worker.startSourceTask(eq(TASK0), any(), any(), any(), eq(herder), any())).thenReturn(true); - PowerMock.replayAll(); herder.doRestartConnectorAndTasks(restartRequest); - PowerMock.verifyAll(); + + verifyNoMoreInteractions(restartPlan, worker, member, configBackingStore, statusBackingStore); } @Test public void testRestartTask() throws Exception { - EasyMock.expect(worker.connectorTaskConfigs(CONN1, conn1SinkConfig)).andStubReturn(TASK_CONFIGS); - // get the initial assignment - EasyMock.expect(member.memberId()).andStubReturn("leader"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); + when(member.memberId()).thenReturn("leader"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); expectRebalance(1, Collections.emptyList(), singletonList(TASK0), true); expectConfigRefreshAndSnapshot(SNAPSHOT); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - worker.startSourceTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); - PowerMock.expectLastCall().andReturn(true); - // now handle the task restart - member.wakeup(); - PowerMock.expectLastCall(); - member.ensureActive(); - PowerMock.expectLastCall(); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + when(statusBackingStore.connectors()).thenReturn(Collections.emptySet()); + doNothing().when(member).poll(anyLong()); - worker.stopAndAwaitTask(TASK0); - PowerMock.expectLastCall(); - worker.startSourceTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); - PowerMock.expectLastCall().andReturn(true); - - PowerMock.replayAll(); + when(worker.startSourceTask(eq(TASK0), any(), any(), any(), eq(herder), any())).thenReturn(true); herder.tick(); + + // now handle the task restart + doNothing().when(member).ensureActive(); + doNothing().when(worker).stopAndAwaitTask(TASK0); FutureCallback callback = new FutureCallback<>(); herder.restartTask(TASK0, callback); herder.tick(); callback.get(1000L, TimeUnit.MILLISECONDS); - PowerMock.verifyAll(); + verify(worker, times(2)).startSourceTask(eq(TASK0), any(), any(), any(), eq(herder), any()); + verify(worker).stopAndAwaitTask(TASK0); + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); } @Test public void testRestartUnknownTask() throws Exception { // get the initial assignment - EasyMock.expect(member.memberId()).andStubReturn("member"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); + when(member.memberId()).thenReturn("member"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); expectRebalance(1, Collections.emptyList(), Collections.emptyList()); expectConfigRefreshAndSnapshot(SNAPSHOT); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - member.wakeup(); - PowerMock.expectLastCall(); - member.ensureActive(); - PowerMock.expectLastCall(); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + doNothing().when(member).poll(anyLong()); - PowerMock.replayAll(); + herder.tick(); + doNothing().when(member).ensureActive(); FutureCallback callback = new FutureCallback<>(); - herder.tick(); herder.restartTask(new ConnectorTaskId("blah", 0), callback); herder.tick(); - try { - callback.get(1000L, TimeUnit.MILLISECONDS); - fail("Expected NotLeaderException to be raised"); - } catch (ExecutionException e) { - assertTrue(e.getCause() instanceof NotFoundException); - } - - PowerMock.verifyAll(); - } - - @Test - public void testRequestProcessingOrder() { - final DistributedHerder.DistributedHerderRequest req1 = herder.addRequest(100, null, null); - final DistributedHerder.DistributedHerderRequest req2 = herder.addRequest(10, null, null); - final DistributedHerder.DistributedHerderRequest req3 = herder.addRequest(200, null, null); - final DistributedHerder.DistributedHerderRequest req4 = herder.addRequest(200, null, null); + ExecutionException e = assertThrows(ExecutionException.class, () -> callback.get(1000, TimeUnit.MILLISECONDS)); + assertEquals(NotFoundException.class, e.getCause().getClass()); - assertEquals(req2, herder.requests.pollFirst()); // lowest delay - assertEquals(req1, herder.requests.pollFirst()); // next lowest delay - assertEquals(req3, herder.requests.pollFirst()); // same delay as req4, but added first - assertEquals(req4, herder.requests.pollFirst()); + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); } @Test public void testRestartTaskRedirectToLeader() throws Exception { // get the initial assignment - EasyMock.expect(member.memberId()).andStubReturn("member"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); + when(member.memberId()).thenReturn("member"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); expectRebalance(1, Collections.emptyList(), Collections.emptyList()); expectConfigRefreshAndSnapshot(SNAPSHOT); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - // now handle the task restart - member.wakeup(); - PowerMock.expectLastCall(); - member.ensureActive(); - PowerMock.expectLastCall(); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - PowerMock.replayAll(); + doNothing().when(member).poll(anyLong()); herder.tick(); + + // now handle the task restart + doNothing().when(member).ensureActive(); FutureCallback callback = new FutureCallback<>(); herder.restartTask(TASK0, callback); herder.tick(); - try { - callback.get(1000L, TimeUnit.MILLISECONDS); - fail("Expected NotLeaderException to be raised"); - } catch (ExecutionException e) { - assertTrue(e.getCause() instanceof NotLeaderException); - } + ExecutionException e = assertThrows(ExecutionException.class, () -> callback.get(1000, TimeUnit.MILLISECONDS)); + assertEquals(NotLeaderException.class, e.getCause().getClass()); - PowerMock.verifyAll(); + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); } @Test public void testRestartTaskRedirectToOwner() throws Exception { // get the initial assignment - EasyMock.expect(member.memberId()).andStubReturn("leader"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); + when(member.memberId()).thenReturn("leader"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true); expectConfigRefreshAndSnapshot(SNAPSHOT); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + + when(statusBackingStore.connectors()).thenReturn(Collections.emptySet()); + doNothing().when(member).poll(anyLong()); + + herder.tick(); // now handle the task restart String ownerUrl = "ownerUrl"; - EasyMock.expect(member.ownerUrl(TASK0)).andReturn(ownerUrl); - member.wakeup(); - PowerMock.expectLastCall(); - member.ensureActive(); - PowerMock.expectLastCall(); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - PowerMock.replayAll(); + when(member.ownerUrl(TASK0)).thenReturn(ownerUrl); + doNothing().when(member).ensureActive(); - herder.tick(); FutureCallback callback = new FutureCallback<>(); herder.restartTask(TASK0, callback); herder.tick(); - try { - callback.get(1000L, TimeUnit.MILLISECONDS); - fail("Expected NotLeaderException to be raised"); - } catch (ExecutionException e) { - assertTrue(e.getCause() instanceof NotAssignedException); - NotAssignedException notAssignedException = (NotAssignedException) e.getCause(); - assertEquals(ownerUrl, notAssignedException.forwardUrl()); - } + ExecutionException e = assertThrows(ExecutionException.class, () -> callback.get(1000, TimeUnit.MILLISECONDS)); + assertEquals(NotAssignedException.class, e.getCause().getClass()); + assertEquals(ownerUrl, ((NotAssignedException) e.getCause()).forwardUrl()); - PowerMock.verifyAll(); + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); + } + + @Test + public void testRequestProcessingOrder() { + final DistributedHerder.DistributedHerderRequest req1 = herder.addRequest(100, null, null); + final DistributedHerder.DistributedHerderRequest req2 = herder.addRequest(10, null, null); + final DistributedHerder.DistributedHerderRequest req3 = herder.addRequest(200, null, null); + final DistributedHerder.DistributedHerderRequest req4 = herder.addRequest(200, null, null); + + assertEquals(req2, herder.requests.pollFirst()); // lowest delay + assertEquals(req1, herder.requests.pollFirst()); // next lowest delay + assertEquals(req3, herder.requests.pollFirst()); // same delay as req4, but added first + assertEquals(req4, herder.requests.pollFirst()); } @Test - public void testConnectorConfigAdded() { + public void testConnectorConfigAdded() throws Exception { // If a connector was added, we need to rebalance - EasyMock.expect(member.memberId()).andStubReturn("member"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); + when(member.memberId()).thenReturn("member"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); // join, no configs so no need to catch up on config topic expectRebalance(-1, Collections.emptyList(), Collections.emptyList()); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + doNothing().when(member).poll(anyLong()); + + herder.tick(); // join - // apply config - member.wakeup(); - PowerMock.expectLastCall(); - member.ensureActive(); - PowerMock.expectLastCall(); // Checks for config updates and starts rebalance - EasyMock.expect(configBackingStore.snapshot()).andReturn(SNAPSHOT); - member.requestRejoin(); - PowerMock.expectLastCall(); + expectConfigRefreshAndSnapshot(SNAPSHOT); + // Rebalance will be triggered when the new config is detected + doNothing().when(member).requestRejoin(); + + configUpdateListener.onConnectorConfigUpdate(CONN1); // read updated config + herder.tick(); // apply config + // Performs rebalance and gets new assignment expectRebalance(Collections.emptyList(), Collections.emptyList(), ConnectProtocol.Assignment.NO_ERROR, 1, Arrays.asList(CONN1), Collections.emptyList()); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - Capture> onStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onStart)); - PowerMock.expectLastCall().andAnswer(() -> { + + ArgumentCaptor> onStart = ArgumentCaptor.forClass(Callback.class); + doAnswer(invocation -> { onStart.getValue().onCompletion(null, TargetState.STARTED); return true; - }); - member.wakeup(); - PowerMock.expectLastCall(); - expectExecuteTaskReconfiguration(true, conn1SinkConfig, () -> TASK_CONFIGS); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - PowerMock.replayAll(); + }).when(worker).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), onStart.capture()); + expectExecuteTaskReconfiguration(true, conn1SinkConfig, invocation -> TASK_CONFIGS); - herder.tick(); // join - configUpdateListener.onConnectorConfigUpdate(CONN1); // read updated config - herder.tick(); // apply config herder.tick(); // do rebalance - PowerMock.verifyAll(); + verify(worker).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), onStart.capture()); + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); } @Test public void testConnectorConfigUpdate() throws Exception { // Connector config can be applied without any rebalance - EasyMock.expect(member.memberId()).andStubReturn("member"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - EasyMock.expect(worker.connectorNames()).andStubReturn(Collections.singleton(CONN1)); + when(member.memberId()).thenReturn("member"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); // join expectRebalance(1, Arrays.asList(CONN1), Collections.emptyList()); expectConfigRefreshAndSnapshot(SNAPSHOT); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - Capture> onFirstStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onFirstStart)); - PowerMock.expectLastCall().andAnswer(() -> { - onFirstStart.getValue().onCompletion(null, TargetState.STARTED); - return true; - }); - member.wakeup(); - PowerMock.expectLastCall(); - expectExecuteTaskReconfiguration(true, conn1SinkConfig, () -> TASK_CONFIGS); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + doNothing().when(member).poll(anyLong()); - // apply config - member.wakeup(); - PowerMock.expectLastCall(); - member.ensureActive(); - PowerMock.expectLastCall(); - EasyMock.expect(configBackingStore.snapshot()).andReturn(SNAPSHOT); // for this test, it doesn't matter if we use the same config snapshot - worker.stopAndAwaitConnector(CONN1); - PowerMock.expectLastCall(); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - Capture> onSecondStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onSecondStart)); - PowerMock.expectLastCall().andAnswer(() -> { - onSecondStart.getValue().onCompletion(null, TargetState.STARTED); + ArgumentCaptor> onStart = ArgumentCaptor.forClass(Callback.class); + doAnswer(invocation -> { + onStart.getValue().onCompletion(null, TargetState.STARTED); return true; - }); - member.wakeup(); - PowerMock.expectLastCall(); - expectExecuteTaskReconfiguration(true, conn1SinkConfig, () -> TASK_CONFIGS); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + }).when(worker).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), onStart.capture()); + expectExecuteTaskReconfiguration(true, conn1SinkConfig, invocation -> TASK_CONFIGS); - // These will occur just before/during the third tick - member.ensureActive(); - PowerMock.expectLastCall(); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + herder.tick(); - PowerMock.replayAll(); + // apply config + doNothing().when(member).ensureActive(); + when(configBackingStore.snapshot()).thenReturn(SNAPSHOT); // for this test, it doesn't matter if we use the same config snapshot + doNothing().when(worker).stopAndAwaitConnector(CONN1); - herder.tick(); // join configUpdateListener.onConnectorConfigUpdate(CONN1); // read updated config herder.tick(); // apply config herder.tick(); - PowerMock.verifyAll(); + verify(worker, times(2)).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), onStart.capture()); + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); } @Test public void testConnectorConfigUpdateFailedTransformation() throws Exception { // Connector config can be applied without any rebalance - EasyMock.expect(member.memberId()).andStubReturn("member"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - EasyMock.expect(worker.connectorNames()).andStubReturn(Collections.singleton(CONN1)); + when(member.memberId()).thenReturn("member"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); - WorkerConfigTransformer configTransformer = EasyMock.mock(WorkerConfigTransformer.class); + WorkerConfigTransformer configTransformer = mock(WorkerConfigTransformer.class); // join expectRebalance(1, Arrays.asList(CONN1), Collections.emptyList()); expectConfigRefreshAndSnapshot(SNAPSHOT); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - Capture> onStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onStart)); - PowerMock.expectLastCall().andAnswer(() -> { + doNothing().when(member).poll(anyLong()); + + ArgumentCaptor> onStart = ArgumentCaptor.forClass(Callback.class); + doAnswer(invocation -> { onStart.getValue().onCompletion(null, TargetState.STARTED); return true; - }); - member.wakeup(); - PowerMock.expectLastCall(); - expectExecuteTaskReconfiguration(true, conn1SinkConfig, () -> TASK_CONFIGS); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + }).when(worker).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), onStart.capture()); + expectExecuteTaskReconfiguration(true, conn1SinkConfig, invocation -> TASK_CONFIGS); + + herder.tick(); // apply config - member.wakeup(); - PowerMock.expectLastCall(); - member.ensureActive(); - PowerMock.expectLastCall(); + doNothing().when(member).ensureActive(); // During the next tick, throw an error from the transformer ClusterConfigState snapshotWithTransform = new ClusterConfigState( 1, @@ -2034,323 +1476,215 @@ public void testConnectorConfigUpdateFailedTransformation() throws Exception { Collections.emptySet(), configTransformer ); - EasyMock.expect(configBackingStore.snapshot()).andReturn(snapshotWithTransform); - EasyMock.expect(configTransformer.transform(EasyMock.eq(CONN1), EasyMock.anyObject())) - .andThrow(new ConfigException("Simulated exception thrown during config transformation")); - worker.stopAndAwaitConnector(CONN1); - PowerMock.expectLastCall(); - Capture failedStatus = newCapture(); - statusBackingStore.putSafe(capture(failedStatus)); - PowerMock.expectLastCall(); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - // These will occur just before/during the third tick - member.ensureActive(); - PowerMock.expectLastCall(); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - EasyMock.replay(configTransformer); - PowerMock.replayAll(); + when(configBackingStore.snapshot()).thenReturn(snapshotWithTransform); + when(configTransformer.transform(eq(CONN1), any())) + .thenThrow(new ConfigException("Simulated exception thrown during config transformation")); + doNothing().when(worker).stopAndAwaitConnector(CONN1); + + ArgumentCaptor failedStatus = ArgumentCaptor.forClass(ConnectorStatus.class); + doNothing().when(statusBackingStore).putSafe(failedStatus.capture()); - herder.tick(); // join configUpdateListener.onConnectorConfigUpdate(CONN1); // read updated config herder.tick(); // apply config herder.tick(); - PowerMock.verifyAll(); - assertEquals(CONN1, failedStatus.getValue().id()); assertEquals(FAILED, failedStatus.getValue().state()); + + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore, configTransformer); } @Test public void testConnectorPaused() throws Exception { // ensure that target state changes are propagated to the worker - EasyMock.expect(member.memberId()).andStubReturn("member"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - EasyMock.expect(worker.connectorNames()).andStubReturn(Collections.singleton(CONN1)); + when(member.memberId()).thenReturn("member"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); // join expectRebalance(1, Arrays.asList(CONN1), Collections.emptyList()); expectConfigRefreshAndSnapshot(SNAPSHOT); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - Capture> onStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onStart)); - PowerMock.expectLastCall().andAnswer(() -> { - onStart.getValue().onCompletion(null, TargetState.STARTED); - return true; - }); - member.wakeup(); - PowerMock.expectLastCall(); - expectExecuteTaskReconfiguration(true, conn1SinkConfig, () -> TASK_CONFIGS); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - // handle the state change - member.wakeup(); - PowerMock.expectLastCall(); - member.ensureActive(); - PowerMock.expectLastCall(); + doNothing().when(member).poll(anyLong()); - EasyMock.expect(configBackingStore.snapshot()).andReturn(SNAPSHOT_PAUSED_CONN1); - PowerMock.expectLastCall(); - - Capture> onPause = newCapture(); - worker.setTargetState(EasyMock.eq(CONN1), EasyMock.eq(TargetState.PAUSED), capture(onPause)); - PowerMock.expectLastCall().andAnswer(() -> { - onStart.getValue().onCompletion(null, TargetState.PAUSED); - return null; - }); + ArgumentCaptor> onStart = ArgumentCaptor.forClass(Callback.class); + doAnswer(invocation -> { + onStart.getValue().onCompletion(null, TargetState.STARTED); + return true; + }).when(worker).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), onStart.capture()); + expectExecuteTaskReconfiguration(true, conn1SinkConfig, invocation -> TASK_CONFIGS); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + herder.tick(); // join - // These will occur just before/during the third tick - member.ensureActive(); - PowerMock.expectLastCall(); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + // handle the state change + doNothing().when(member).ensureActive(); + when(configBackingStore.snapshot()).thenReturn(SNAPSHOT_PAUSED_CONN1); - PowerMock.replayAll(); + ArgumentCaptor> onPause = ArgumentCaptor.forClass(Callback.class); + doAnswer(invocation -> { + onPause.getValue().onCompletion(null, TargetState.PAUSED); + return null; + }).when(worker).setTargetState(eq(CONN1), eq(TargetState.PAUSED), onPause.capture()); - herder.tick(); // join configUpdateListener.onConnectorTargetStateChange(CONN1); // state changes to paused herder.tick(); // worker should apply the state change herder.tick(); - PowerMock.verifyAll(); + verify(worker).setTargetState(eq(CONN1), eq(TargetState.PAUSED), any(Callback.class)); + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); } @Test public void testConnectorResumed() throws Exception { - EasyMock.expect(member.memberId()).andStubReturn("member"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - EasyMock.expect(worker.connectorNames()).andStubReturn(Collections.singleton(CONN1)); + when(member.memberId()).thenReturn("member"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); // start with the connector paused expectRebalance(1, Arrays.asList(CONN1), Collections.emptyList()); expectConfigRefreshAndSnapshot(SNAPSHOT_PAUSED_CONN1); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - Capture> onStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.PAUSED), capture(onStart)); - PowerMock.expectLastCall().andAnswer(() -> { + doNothing().when(member).poll(anyLong()); + + ArgumentCaptor> onStart = ArgumentCaptor.forClass(Callback.class); + doAnswer(invocation -> { onStart.getValue().onCompletion(null, TargetState.PAUSED); return true; - }); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + }).when(worker).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.PAUSED), onStart.capture()); - // handle the state change - member.wakeup(); - PowerMock.expectLastCall(); - member.ensureActive(); - PowerMock.expectLastCall(); + herder.tick(); // join - EasyMock.expect(configBackingStore.snapshot()).andReturn(SNAPSHOT); - PowerMock.expectLastCall(); + // handle the state change + doNothing().when(member).ensureActive(); + when(configBackingStore.snapshot()).thenReturn(SNAPSHOT); - Capture> onResume = newCapture(); - worker.setTargetState(EasyMock.eq(CONN1), EasyMock.eq(TargetState.STARTED), capture(onResume)); - PowerMock.expectLastCall().andAnswer(() -> { + ArgumentCaptor> onResume = ArgumentCaptor.forClass(Callback.class); + doAnswer(invocation -> { onResume.getValue().onCompletion(null, TargetState.STARTED); return null; - }); - member.wakeup(); - PowerMock.expectLastCall(); - - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - + }).when(worker).setTargetState(eq(CONN1), eq(TargetState.STARTED), onResume.capture()); // we expect reconfiguration after resuming - expectExecuteTaskReconfiguration(true, conn1SinkConfig, () -> TASK_CONFIGS); - - // These will occur just before/during the third tick - member.ensureActive(); - PowerMock.expectLastCall(); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - PowerMock.replayAll(); + expectExecuteTaskReconfiguration(true, conn1SinkConfig, invocation -> TASK_CONFIGS); - herder.tick(); // join configUpdateListener.onConnectorTargetStateChange(CONN1); // state changes to started herder.tick(); // apply state change herder.tick(); - PowerMock.verifyAll(); + verify(worker).setTargetState(eq(CONN1), eq(TargetState.STARTED), any(Callback.class)); + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); } @Test public void testConnectorStopped() throws Exception { // ensure that target state changes are propagated to the worker - EasyMock.expect(member.memberId()).andStubReturn("member"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - EasyMock.expect(worker.connectorNames()).andStubReturn(Collections.singleton(CONN1)); + when(member.memberId()).thenReturn("member"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); // join expectRebalance(1, Arrays.asList(CONN1), Collections.emptyList()); expectConfigRefreshAndSnapshot(SNAPSHOT); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - Capture> onStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onStart)); - PowerMock.expectLastCall().andAnswer(() -> { + doNothing().when(member).poll(anyLong()); + + ArgumentCaptor> onStart = ArgumentCaptor.forClass(Callback.class); + doAnswer(invocation -> { onStart.getValue().onCompletion(null, TargetState.STARTED); return true; - }); - member.wakeup(); - PowerMock.expectLastCall(); - expectExecuteTaskReconfiguration(true, conn1SinkConfig, () -> TASK_CONFIGS); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + }).when(worker).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), onStart.capture()); + expectExecuteTaskReconfiguration(true, conn1SinkConfig, invocation -> TASK_CONFIGS); - // handle the state change - member.wakeup(); - PowerMock.expectLastCall(); - member.ensureActive(); - PowerMock.expectLastCall(); + herder.tick(); // join - EasyMock.expect(configBackingStore.snapshot()).andReturn(SNAPSHOT_STOPPED_CONN1); + // handle the state change + doNothing().when(member).ensureActive(); + when(configBackingStore.snapshot()).thenReturn(SNAPSHOT_STOPPED_CONN1); - Capture> onStop = newCapture(); - worker.setTargetState(EasyMock.eq(CONN1), EasyMock.eq(TargetState.STOPPED), capture(onStop)); - PowerMock.expectLastCall().andAnswer(() -> { - onStart.getValue().onCompletion(null, TargetState.STOPPED); + ArgumentCaptor> onStop = ArgumentCaptor.forClass(Callback.class); + doAnswer(invocation -> { + onStop.getValue().onCompletion(null, TargetState.STOPPED); return null; - }); - - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + }).when(worker).setTargetState(eq(CONN1), eq(TargetState.STOPPED), onStop.capture()); - // These will occur just before/during the third tick - member.ensureActive(); - PowerMock.expectLastCall(); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - PowerMock.replayAll(); - - herder.tick(); // join configUpdateListener.onConnectorTargetStateChange(CONN1); // state changes to stopped herder.tick(); // worker should apply the state change herder.tick(); - PowerMock.verifyAll(); + verify(worker).setTargetState(eq(CONN1), eq(TargetState.STOPPED), any(Callback.class)); + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); } @Test public void testUnknownConnectorPaused() throws Exception { - EasyMock.expect(member.memberId()).andStubReturn("member"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - EasyMock.expect(worker.connectorNames()).andStubReturn(Collections.singleton(CONN1)); + when(member.memberId()).thenReturn("member"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); // join expectRebalance(1, Collections.emptyList(), singletonList(TASK0)); expectConfigRefreshAndSnapshot(SNAPSHOT); - worker.startSourceTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); - PowerMock.expectLastCall().andReturn(true); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - // state change is ignored since we have no target state - member.wakeup(); - PowerMock.expectLastCall(); - member.ensureActive(); - PowerMock.expectLastCall(); + doNothing().when(member).poll(anyLong()); - EasyMock.expect(configBackingStore.snapshot()).andReturn(SNAPSHOT); - PowerMock.expectLastCall(); + when(worker.startSourceTask(eq(TASK0), any(), any(), any(), eq(herder), eq(TargetState.STARTED))).thenReturn(true); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + herder.tick(); // join - PowerMock.replayAll(); + // state change is ignored since we have no target state + doNothing().when(member).ensureActive(); + when(configBackingStore.snapshot()).thenReturn(SNAPSHOT); - herder.tick(); // join configUpdateListener.onConnectorTargetStateChange("unknown-connector"); herder.tick(); // continue - PowerMock.verifyAll(); + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); } @Test public void testStopConnector() throws Exception { - EasyMock.expect(member.memberId()).andStubReturn("leader"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - EasyMock.expect(worker.connectorNames()).andStubReturn(Collections.singleton(CONN1)); + when(member.memberId()).thenReturn("leader"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); // join as leader expectRebalance(1, Collections.emptyList(), singletonList(TASK0), true); expectConfigRefreshAndSnapshot(SNAPSHOT); - worker.startSourceTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); - PowerMock.expectLastCall().andReturn(true); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + + when(statusBackingStore.connectors()).thenReturn(Collections.emptySet()); + doNothing().when(member).poll(anyLong()); + + when(worker.startSourceTask(eq(TASK0), any(), any(), any(), eq(herder), eq(TargetState.STARTED))).thenReturn(true); + + herder.tick(); // join // handle stop request - member.wakeup(); - PowerMock.expectLastCall(); - member.ensureActive(); - PowerMock.expectLastCall(); + doNothing().when(member).ensureActive(); expectConfigRefreshAndSnapshot(SNAPSHOT); - configBackingStore.putTaskConfigs(CONN1, Collections.emptyList()); - PowerMock.expectLastCall(); - configBackingStore.putTargetState(CONN1, TargetState.STOPPED); - PowerMock.expectLastCall(); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - PowerMock.replayAll(); + doNothing().when(configBackingStore).putTaskConfigs(CONN1, Collections.emptyList()); + doNothing().when(configBackingStore).putTargetState(CONN1, TargetState.STOPPED); FutureCallback cb = new FutureCallback<>(); - herder.tick(); // join herder.stopConnector(CONN1, cb); // external request herder.tick(); // continue assertTrue("Callback should already have been invoked by herder", cb.isDone()); cb.get(0, TimeUnit.MILLISECONDS); - PowerMock.verifyAll(); + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); } @Test public void testStopConnectorNotLeader() throws Exception { - EasyMock.expect(member.memberId()).andStubReturn("member"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - EasyMock.expect(worker.connectorNames()).andStubReturn(Collections.singleton(CONN1)); + when(member.memberId()).thenReturn("member"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); - // join as leader + // join as member (non-leader) expectRebalance(1, Collections.emptyList(), singletonList(TASK0)); expectConfigRefreshAndSnapshot(SNAPSHOT); - worker.startSourceTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); - PowerMock.expectLastCall().andReturn(true); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + doNothing().when(member).poll(anyLong()); - // handle stop request - member.wakeup(); - PowerMock.expectLastCall(); - member.ensureActive(); - PowerMock.expectLastCall(); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + when(worker.startSourceTask(eq(TASK0), any(), any(), any(), eq(herder), eq(TargetState.STARTED))).thenReturn(true); - PowerMock.replayAll(); + herder.tick(); + // handle stop request + doNothing().when(member).ensureActive(); FutureCallback cb = new FutureCallback<>(); - herder.tick(); // join herder.stopConnector(CONN1, cb); // external request herder.tick(); // continue @@ -2362,44 +1696,35 @@ public void testStopConnectorNotLeader() throws Exception { ); assertTrue(e.getCause() instanceof NotLeaderException); - PowerMock.verifyAll(); + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); } @Test public void testStopConnectorFailToWriteTaskConfigs() throws Exception { - EasyMock.expect(member.memberId()).andStubReturn("leader"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - EasyMock.expect(worker.connectorNames()).andStubReturn(Collections.singleton(CONN1)); + when(member.memberId()).thenReturn("leader"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); // join as leader expectRebalance(1, Collections.emptyList(), singletonList(TASK0), true); expectConfigRefreshAndSnapshot(SNAPSHOT); - worker.startSourceTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); - PowerMock.expectLastCall().andReturn(true); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - ConnectException taskConfigsWriteException = new ConnectException("Could not write task configs to config topic"); + when(statusBackingStore.connectors()).thenReturn(Collections.emptySet()); + doNothing().when(member).poll(anyLong()); + when(worker.startSourceTask(eq(TASK0), any(), any(), any(), eq(herder), eq(TargetState.STARTED))).thenReturn(true); + + herder.tick(); // join + + ConnectException taskConfigsWriteException = new ConnectException("Could not write task configs to config topic"); // handle stop request - member.wakeup(); - PowerMock.expectLastCall(); - member.ensureActive(); - PowerMock.expectLastCall(); - configBackingStore.putTaskConfigs(CONN1, Collections.emptyList()); + doNothing().when(member).ensureActive(); + doThrow(taskConfigsWriteException).when(configBackingStore).putTaskConfigs(CONN1, Collections.emptyList()); // We do not expect configBackingStore::putTargetState to be invoked, which // is intentional since that call should only take place if we are first able to // successfully write the empty list of task configs - PowerMock.expectLastCall().andThrow(taskConfigsWriteException); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - PowerMock.replayAll(); FutureCallback cb = new FutureCallback<>(); - herder.tick(); // join herder.stopConnector(CONN1, cb); // external request herder.tick(); // continue @@ -2411,7 +1736,7 @@ public void testStopConnectorFailToWriteTaskConfigs() throws Exception { ); assertEquals(e.getCause(), taskConfigsWriteException); - PowerMock.verifyAll(); + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); } @Test @@ -2419,45 +1744,33 @@ public void testConnectorPausedRunningTaskOnly() throws Exception { // even if we don't own the connector, we should still propagate target state // changes to the worker so that tasks will transition correctly - EasyMock.expect(member.memberId()).andStubReturn("member"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - EasyMock.expect(worker.connectorNames()).andStubReturn(Collections.emptySet()); + when(member.memberId()).thenReturn("member"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); // join expectRebalance(1, Collections.emptyList(), singletonList(TASK0)); expectConfigRefreshAndSnapshot(SNAPSHOT); - worker.startSourceTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); - PowerMock.expectLastCall().andReturn(true); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + doNothing().when(member).poll(anyLong()); - // handle the state change - member.wakeup(); - PowerMock.expectLastCall(); - member.ensureActive(); - PowerMock.expectLastCall(); + when(worker.startSourceTask(eq(TASK0), any(), any(), any(), eq(herder), eq(TargetState.STARTED))).thenReturn(true); + + herder.tick(); // join - EasyMock.expect(configBackingStore.snapshot()).andReturn(SNAPSHOT_PAUSED_CONN1); - PowerMock.expectLastCall(); + // handle the state change + doNothing().when(member).ensureActive(); + when(configBackingStore.snapshot()).thenReturn(SNAPSHOT_PAUSED_CONN1); - Capture> onPause = newCapture(); - worker.setTargetState(EasyMock.eq(CONN1), EasyMock.eq(TargetState.PAUSED), capture(onPause)); - PowerMock.expectLastCall().andAnswer(() -> { + ArgumentCaptor> onPause = ArgumentCaptor.forClass(Callback.class); + doAnswer(invocation -> { onPause.getValue().onCompletion(null, TargetState.PAUSED); return null; - }); - - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + }).when(worker).setTargetState(eq(CONN1), eq(TargetState.PAUSED), onPause.capture()); - PowerMock.replayAll(); - - herder.tick(); // join configUpdateListener.onConnectorTargetStateChange(CONN1); // state changes to paused herder.tick(); // apply state change - PowerMock.verifyAll(); + verify(worker).setTargetState(eq(CONN1), eq(TargetState.PAUSED), any(Callback.class)); + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); } @Test @@ -2465,139 +1778,85 @@ public void testConnectorResumedRunningTaskOnly() throws Exception { // even if we don't own the connector, we should still propagate target state // changes to the worker so that tasks will transition correctly - EasyMock.expect(member.memberId()).andStubReturn("member"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - EasyMock.expect(worker.connectorNames()).andStubReturn(Collections.emptySet()); + when(member.memberId()).thenReturn("member"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); // join expectRebalance(1, Collections.emptyList(), singletonList(TASK0)); expectConfigRefreshAndSnapshot(SNAPSHOT_PAUSED_CONN1); - worker.startSourceTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.PAUSED)); - PowerMock.expectLastCall().andReturn(true); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + doNothing().when(member).poll(anyLong()); - // handle the state change - member.wakeup(); - PowerMock.expectLastCall(); - member.ensureActive(); - PowerMock.expectLastCall(); + when(worker.startSourceTask(eq(TASK0), any(), any(), any(), eq(herder), eq(TargetState.PAUSED))).thenReturn(true); + + herder.tick(); // join - EasyMock.expect(configBackingStore.snapshot()).andReturn(SNAPSHOT); - PowerMock.expectLastCall(); + // handle the state change + doNothing().when(member).ensureActive(); + when(configBackingStore.snapshot()).thenReturn(SNAPSHOT); - Capture> onStart = newCapture(); - worker.setTargetState(EasyMock.eq(CONN1), EasyMock.eq(TargetState.STARTED), capture(onStart)); - PowerMock.expectLastCall().andAnswer(() -> { - onStart.getValue().onCompletion(null, TargetState.STARTED); + ArgumentCaptor> onStart = ArgumentCaptor.forClass(Callback.class); + doAnswer(invocation -> { + onStart.getValue().onCompletion(null, TargetState.PAUSED); return null; - }); - member.wakeup(); - PowerMock.expectLastCall(); + }).when(worker).setTargetState(eq(CONN1), eq(TargetState.STARTED), onStart.capture()); expectExecuteTaskReconfiguration(false, null, null); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - // These will occur just before/during the third tick - member.ensureActive(); - PowerMock.expectLastCall(); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - PowerMock.replayAll(); - - herder.tick(); // join configUpdateListener.onConnectorTargetStateChange(CONN1); // state changes to paused herder.tick(); // apply state change herder.tick(); - PowerMock.verifyAll(); + verify(worker).setTargetState(eq(CONN1), eq(TargetState.STARTED), any(Callback.class)); + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); } @Test public void testTaskConfigAdded() { // Task config always requires rebalance - EasyMock.expect(member.memberId()).andStubReturn("member"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); + when(member.memberId()).thenReturn("member"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); // join expectRebalance(-1, Collections.emptyList(), Collections.emptyList()); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + doNothing().when(member).poll(anyLong()); + + herder.tick(); // join - // apply config - member.wakeup(); - PowerMock.expectLastCall(); - member.ensureActive(); - PowerMock.expectLastCall(); // Checks for config updates and starts rebalance - EasyMock.expect(configBackingStore.snapshot()).andReturn(SNAPSHOT); - member.requestRejoin(); - PowerMock.expectLastCall(); + when(configBackingStore.snapshot()).thenReturn(SNAPSHOT); + // Rebalance will be triggered when the new config is detected + doNothing().when(member).requestRejoin(); + + configUpdateListener.onTaskConfigUpdate(Arrays.asList(TASK0, TASK1, TASK2)); // read updated config + herder.tick(); // apply config + // Performs rebalance and gets new assignment expectRebalance(Collections.emptyList(), Collections.emptyList(), ConnectProtocol.Assignment.NO_ERROR, 1, Collections.emptyList(), Arrays.asList(TASK0)); - worker.startSourceTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); - PowerMock.expectLastCall().andReturn(true); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - PowerMock.replayAll(); + expectConfigRefreshAndSnapshot(SNAPSHOT); + when(worker.startSourceTask(eq(TASK0), any(), any(), any(), eq(herder), eq(TargetState.STARTED))).thenReturn(true); - herder.tick(); // join - configUpdateListener.onTaskConfigUpdate(Arrays.asList(TASK0, TASK1, TASK2)); // read updated config - herder.tick(); // apply config herder.tick(); // do rebalance - PowerMock.verifyAll(); + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); } @Test public void testJoinLeaderCatchUpFails() throws Exception { // Join group and as leader fail to do assignment - EasyMock.expect(member.memberId()).andStubReturn("leader"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); + when(member.memberId()).thenReturn("leader"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); + when(configBackingStore.snapshot()).thenReturn(SNAPSHOT); + when(statusBackingStore.connectors()).thenReturn(Collections.emptySet()); + expectRebalance(Collections.emptyList(), Collections.emptyList(), ConnectProtocol.Assignment.CONFIG_MISMATCH, 1, "leader", "leaderUrl", Collections.emptyList(), Collections.emptyList(), 0, true); - // Reading to end of log times out - configBackingStore.refresh(anyLong(), EasyMock.anyObject(TimeUnit.class)); - EasyMock.expectLastCall().andThrow(new TimeoutException()); - member.maybeLeaveGroup(EasyMock.eq("taking too long to read the log")); - EasyMock.expectLastCall(); - member.requestRejoin(); - - // After backoff, restart the process and this time succeed - expectRebalance(1, Arrays.asList(CONN1), Arrays.asList(TASK1), true); - expectConfigRefreshAndSnapshot(SNAPSHOT); - - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - Capture> onStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onStart)); - PowerMock.expectLastCall().andAnswer(() -> { - onStart.getValue().onCompletion(null, TargetState.STARTED); - return true; - }); - member.wakeup(); - PowerMock.expectLastCall(); - expectExecuteTaskReconfiguration(true, conn1SinkConfig, () -> TASK_CONFIGS); - worker.startSourceTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); - PowerMock.expectLastCall().andReturn(true); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - // one more tick, to make sure we don't keep trying to read to the config topic unnecessarily - expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - PowerMock.replayAll(); + // Reading to end of log times out + doThrow(new TimeoutException()).when(configBackingStore).refresh(anyLong(), any(TimeUnit.class)); + doNothing().when(member).maybeLeaveGroup(eq("taking too long to read the log")); + doNothing().when(member).requestRejoin(); long before = time.milliseconds(); int workerUnsyncBackoffMs = DistributedConfig.WORKER_UNSYNC_BACKOFF_MS_DEFAULT; @@ -2609,90 +1868,88 @@ public void testJoinLeaderCatchUpFails() throws Exception { assertStatistics("leaderUrl", true, 3, 0, Double.NEGATIVE_INFINITY, Double.POSITIVE_INFINITY); before = time.milliseconds(); + + // After backoff, restart the process and this time succeed + expectRebalance(1, Arrays.asList(CONN1), Arrays.asList(TASK1), true); + expectConfigRefreshAndSnapshot(SNAPSHOT); + + ArgumentCaptor> onStart = ArgumentCaptor.forClass(Callback.class); + doAnswer(invocation -> { + onStart.getValue().onCompletion(null, TargetState.STARTED); + return true; + }).when(worker).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), onStart.capture()); + expectExecuteTaskReconfiguration(true, conn1SinkConfig, invocation -> TASK_CONFIGS); + + when(worker.startSourceTask(eq(TASK1), any(), any(), any(), eq(herder), eq(TargetState.STARTED))).thenReturn(true); + doNothing().when(member).poll(anyLong()); + herder.tick(); assertEquals(before + coordinatorDiscoveryTimeoutMs, time.milliseconds()); time.sleep(2000L); assertStatistics("leaderUrl", false, 3, 1, 100, 2000L); - // tick once more to ensure that the successful read to the end of the config topic was + // one more tick, to make sure we don't keep trying to read to the config topic unnecessarily + expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true); + + // tick once more to ensure that the successful read to the end of the config topic was // tracked and no further unnecessary attempts were made herder.tick(); - PowerMock.verifyAll(); + verify(configBackingStore, times(2)).refresh(anyLong(), any(TimeUnit.class)); + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); } @Test public void testJoinLeaderCatchUpRetriesForIncrementalCooperative() throws Exception { connectProtocolVersion = CONNECT_PROTOCOL_V1; - // Join group and as leader fail to do assignment - EasyMock.expect(member.memberId()).andStubReturn("leader"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V1); + // Join group as leader + when(member.memberId()).thenReturn("leader"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V1); + when(statusBackingStore.connectors()).thenReturn(Collections.emptySet()); expectRebalance(1, Arrays.asList(CONN1), Arrays.asList(TASK1), true); expectConfigRefreshAndSnapshot(SNAPSHOT); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - // The leader got its assignment - expectRebalance(Collections.emptyList(), Collections.emptyList(), - ConnectProtocol.Assignment.NO_ERROR, - 1, "leader", "leaderUrl", Arrays.asList(CONN1), Arrays.asList(TASK1), 0, true); + doNothing().when(member).poll(anyLong()); - Capture> onStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onStart)); - PowerMock.expectLastCall().andAnswer(() -> { + ArgumentCaptor> onStart = ArgumentCaptor.forClass(Callback.class); + doAnswer(invocation -> { onStart.getValue().onCompletion(null, TargetState.STARTED); return true; - }); - member.wakeup(); - PowerMock.expectLastCall(); - expectExecuteTaskReconfiguration(true, conn1SinkConfig, () -> TASK_CONFIGS); + }).when(worker).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), onStart.capture()); + expectExecuteTaskReconfiguration(true, conn1SinkConfig, invocation -> TASK_CONFIGS); - worker.startSourceTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); - PowerMock.expectLastCall().andReturn(true); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + when(worker.startSourceTask(eq(TASK1), any(), any(), any(), eq(herder), eq(TargetState.STARTED))).thenReturn(true); - // Another rebalance is triggered but this time it fails to read to the max offset and - // triggers a re-sync - expectRebalance(Collections.emptyList(), Collections.emptyList(), - ConnectProtocol.Assignment.CONFIG_MISMATCH, 1, "leader", "leaderUrl", - Collections.emptyList(), Collections.emptyList(), 0, true); + assertStatistics(0, 0, 0, Double.POSITIVE_INFINITY); - // The leader will retry a few times to read to the end of the config log - int retries = 2; - member.requestRejoin(); - for (int i = retries; i >= 0; --i) { - // Reading to end of log times out - configBackingStore.refresh(anyLong(), EasyMock.anyObject(TimeUnit.class)); - EasyMock.expectLastCall().andThrow(new TimeoutException()); - member.maybeLeaveGroup(EasyMock.eq("taking too long to read the log")); - EasyMock.expectLastCall(); - } + herder.tick(); - // After a few retries succeed to read the log to the end + // The leader gets the same assignment after a rebalance is triggered expectRebalance(Collections.emptyList(), Collections.emptyList(), ConnectProtocol.Assignment.NO_ERROR, 1, "leader", "leaderUrl", Arrays.asList(CONN1), Arrays.asList(TASK1), 0, true); - expectConfigRefreshAndSnapshot(SNAPSHOT); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - PowerMock.replayAll(); - - assertStatistics(0, 0, 0, Double.POSITIVE_INFINITY); - herder.tick(); time.sleep(2000L); assertStatistics(3, 1, 100, 2000); + herder.tick(); + // Another rebalance is triggered but this time it fails to read to the max offset and + // triggers a re-sync + expectRebalance(Collections.emptyList(), Collections.emptyList(), + ConnectProtocol.Assignment.CONFIG_MISMATCH, 1, "leader", "leaderUrl", + Collections.emptyList(), Collections.emptyList(), 0, true); + + // The leader will retry a few times to read to the end of the config log + doNothing().when(member).requestRejoin(); + doThrow(TimeoutException.class).when(configBackingStore).refresh(anyLong(), any(TimeUnit.class)); + doNothing().when(member).maybeLeaveGroup(eq("taking too long to read the log")); + long before; int coordinatorDiscoveryTimeoutMs = 100; int maxRetries = 5; + int retries = 3; for (int i = maxRetries; i >= maxRetries - retries; --i) { before = time.milliseconds(); int workerUnsyncBackoffMs = @@ -2702,49 +1959,55 @@ public void testJoinLeaderCatchUpRetriesForIncrementalCooperative() throws Excep coordinatorDiscoveryTimeoutMs = 0; } + // After a few retries succeed to read the log to the end + expectRebalance(Collections.emptyList(), Collections.emptyList(), + ConnectProtocol.Assignment.NO_ERROR, + 1, "leader", "leaderUrl", Arrays.asList(CONN1), Arrays.asList(TASK1), 0, true); + expectConfigRefreshAndSnapshot(SNAPSHOT); + before = time.milliseconds(); coordinatorDiscoveryTimeoutMs = 100; herder.tick(); assertEquals(before + coordinatorDiscoveryTimeoutMs, time.milliseconds()); - PowerMock.verifyAll(); + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); } @Test public void testJoinLeaderCatchUpFailsForIncrementalCooperative() throws Exception { connectProtocolVersion = CONNECT_PROTOCOL_V1; - // Join group and as leader fail to do assignment - EasyMock.expect(member.memberId()).andStubReturn("leader"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V1); + // Join group as leader + when(member.memberId()).thenReturn("leader"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V1); + when(statusBackingStore.connectors()).thenReturn(Collections.emptySet()); expectRebalance(1, Arrays.asList(CONN1), Arrays.asList(TASK1), true); expectConfigRefreshAndSnapshot(SNAPSHOT); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + doNothing().when(member).poll(anyLong()); - // The leader got its assignment + ArgumentCaptor> onStart = ArgumentCaptor.forClass(Callback.class); + doAnswer(invocation -> { + onStart.getValue().onCompletion(null, TargetState.STARTED); + return true; + }).when(worker).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), onStart.capture()); + expectExecuteTaskReconfiguration(true, conn1SinkConfig, invocation -> TASK_CONFIGS); + + when(worker.startSourceTask(eq(TASK1), any(), any(), any(), eq(herder), eq(TargetState.STARTED))).thenReturn(true); + + assertStatistics(0, 0, 0, Double.POSITIVE_INFINITY); + + herder.tick(); + + // The leader gets the same assignment after a rebalance is triggered expectRebalance(Collections.emptyList(), Collections.emptyList(), ConnectProtocol.Assignment.NO_ERROR, 1, "leader", "leaderUrl", Arrays.asList(CONN1), Arrays.asList(TASK1), 0, true); - // and the new assignment started - Capture> onStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onStart)); - PowerMock.expectLastCall().andAnswer(() -> { - onStart.getValue().onCompletion(null, TargetState.STARTED); - return true; - }); - member.wakeup(); - PowerMock.expectLastCall(); - expectExecuteTaskReconfiguration(true, conn1SinkConfig, () -> TASK_CONFIGS); + time.sleep(2000L); + assertStatistics(3, 1, 100, 2000); - worker.startSourceTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); - PowerMock.expectLastCall().andReturn(true); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + herder.tick(); // Another rebalance is triggered but this time it fails to read to the max offset and // triggers a re-sync @@ -2753,41 +2016,13 @@ public void testJoinLeaderCatchUpFailsForIncrementalCooperative() throws Excepti Collections.emptyList(), Collections.emptyList(), 0, true); // The leader will exhaust the retries while trying to read to the end of the config log - int maxRetries = 5; - member.requestRejoin(); - for (int i = maxRetries; i >= 0; --i) { - // Reading to end of log times out - configBackingStore.refresh(anyLong(), EasyMock.anyObject(TimeUnit.class)); - EasyMock.expectLastCall().andThrow(new TimeoutException()); - member.maybeLeaveGroup(EasyMock.eq("taking too long to read the log")); - EasyMock.expectLastCall(); - } - - Capture assignmentCapture = newCapture(); - member.revokeAssignment(capture(assignmentCapture)); - PowerMock.expectLastCall(); - - // After a complete backoff and a revocation of running tasks rejoin and this time succeed - // The worker gets back the assignment that had given up - expectRebalance(Collections.emptyList(), Collections.emptyList(), - ConnectProtocol.Assignment.NO_ERROR, - 1, "leader", "leaderUrl", Arrays.asList(CONN1), Arrays.asList(TASK1), - 0, true); - expectConfigRefreshAndSnapshot(SNAPSHOT); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - PowerMock.replayAll(); - - assertStatistics(0, 0, 0, Double.POSITIVE_INFINITY); - herder.tick(); - - time.sleep(2000L); - assertStatistics(3, 1, 100, 2000); - herder.tick(); + doNothing().when(member).requestRejoin(); + doThrow(TimeoutException.class).when(configBackingStore).refresh(anyLong(), any(TimeUnit.class)); + doNothing().when(member).maybeLeaveGroup(eq("taking too long to read the log")); long before; int coordinatorDiscoveryTimeoutMs = 100; + int maxRetries = 5; for (int i = maxRetries; i > 0; --i) { before = time.milliseconds(); int workerUnsyncBackoffMs = @@ -2797,28 +2032,41 @@ public void testJoinLeaderCatchUpFailsForIncrementalCooperative() throws Excepti coordinatorDiscoveryTimeoutMs = 0; } + ArgumentCaptor assignmentCapture = ArgumentCaptor.forClass(ExtendedAssignment.class); + doNothing().when(member).revokeAssignment(assignmentCapture.capture()); + before = time.milliseconds(); herder.tick(); assertEquals(before, time.milliseconds()); + assertEquals(Collections.singleton(CONN1), assignmentCapture.getValue().connectors()); assertEquals(Collections.singleton(TASK1), assignmentCapture.getValue().tasks()); + + // After a complete backoff and a revocation of running tasks rejoin and this time succeed + // The worker gets back the assignment that had given up + expectRebalance(Collections.emptyList(), Collections.emptyList(), + ConnectProtocol.Assignment.NO_ERROR, + 1, "leader", "leaderUrl", Arrays.asList(CONN1), Arrays.asList(TASK1), + 0, true); + expectConfigRefreshAndSnapshot(SNAPSHOT); + herder.tick(); - PowerMock.verifyAll(); + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); } @Test public void testAccessors() throws Exception { - EasyMock.expect(member.memberId()).andStubReturn("leader"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - EasyMock.expect(worker.getPlugins()).andReturn(plugins).anyTimes(); + when(member.memberId()).thenReturn("leader"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); + when(statusBackingStore.connectors()).thenReturn(Collections.emptySet()); + expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true); - EasyMock.expect(configBackingStore.snapshot()).andReturn(SNAPSHOT).times(2); - WorkerConfigTransformer configTransformer = EasyMock.mock(WorkerConfigTransformer.class); - EasyMock.expect(configTransformer.transform(EasyMock.eq(CONN1), EasyMock.anyObject())) - .andThrow(new AssertionError("Config transformation should not occur when requesting connector or task info")); - EasyMock.replay(configTransformer); + doNothing().when(member).poll(anyLong()); + + WorkerConfigTransformer configTransformer = mock(WorkerConfigTransformer.class); + ClusterConfigState snapshotWithTransform = new ClusterConfigState( 1, null, @@ -2834,17 +2082,7 @@ public void testAccessors() throws Exception { expectConfigRefreshAndSnapshot(snapshotWithTransform); - - member.wakeup(); - PowerMock.expectLastCall().anyTimes(); // list connectors, get connector info, get connector config, get task configs - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - PowerMock.replayAll(); - FutureCallback> listConnectorsCb = new FutureCallback<>(); herder.connectors(listConnectorsCb); FutureCallback connectorInfoCb = new FutureCallback<>(); @@ -2870,93 +2108,61 @@ public void testAccessors() throws Exception { new TaskInfo(TASK2, TASK_CONFIG)), taskConfigsCb.get()); - PowerMock.verifyAll(); + // Config transformation should not occur when requesting connector or task info + verify(configTransformer, never()).transform(eq(CONN1), any()); + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); } @Test public void testPutConnectorConfig() throws Exception { - // connectorConfig uses an async request - member.wakeup(); - PowerMock.expectLastCall(); - - // putConnectorConfig uses an async request - member.wakeup(); - PowerMock.expectLastCall(); - - EasyMock.expect(member.memberId()).andStubReturn("leader"); + when(member.memberId()).thenReturn("leader"); expectRebalance(1, Arrays.asList(CONN1), Collections.emptyList(), true); + when(statusBackingStore.connectors()).thenReturn(Collections.emptySet()); expectConfigRefreshAndSnapshot(SNAPSHOT); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - Capture> onFirstStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onFirstStart)); - PowerMock.expectLastCall().andAnswer(() -> { - onFirstStart.getValue().onCompletion(null, TargetState.STARTED); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); + + ArgumentCaptor> onStart = ArgumentCaptor.forClass(Callback.class); + doAnswer(invocation -> { + onStart.getValue().onCompletion(null, TargetState.STARTED); return true; - }); - member.wakeup(); - PowerMock.expectLastCall(); - expectExecuteTaskReconfiguration(true, conn1SinkConfig, () -> TASK_CONFIGS); + }).when(worker).startConnector(eq(CONN1), eq(CONN1_CONFIG), any(), eq(herder), eq(TargetState.STARTED), onStart.capture()); + expectExecuteTaskReconfiguration(true, conn1SinkConfig, invocation -> TASK_CONFIGS); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + doNothing().when(member).poll(anyLong()); - // Poll loop for second round of calls - member.ensureActive(); - PowerMock.expectLastCall(); + // Should pick up original config + FutureCallback> connectorConfigCb = new FutureCallback<>(); + herder.connectorConfig(CONN1, connectorConfigCb); + herder.tick(); + assertTrue(connectorConfigCb.isDone()); + assertEquals(CONN1_CONFIG, connectorConfigCb.get()); - EasyMock.expect(configBackingStore.snapshot()).andReturn(SNAPSHOT); + // Poll loop for second round of calls + doNothing().when(member).ensureActive(); - Capture> validateCallback = newCapture(); - herder.validateConnectorConfig(EasyMock.eq(CONN1_CONFIG_UPDATED), capture(validateCallback)); - PowerMock.expectLastCall().andAnswer(() -> { + ArgumentCaptor> validateCallback = ArgumentCaptor.forClass(Callback.class); + doAnswer(invocation -> { validateCallback.getValue().onCompletion(null, CONN1_CONFIG_INFOS); return null; - }); - member.wakeup(); - PowerMock.expectLastCall(); + }).when(herder).validateConnectorConfig(eq(CONN1_CONFIG_UPDATED), validateCallback.capture()); - configBackingStore.putConnectorConfig(CONN1, CONN1_CONFIG_UPDATED); - PowerMock.expectLastCall().andAnswer(() -> { + doAnswer(invocation -> { // Simulate response to writing config + waiting until end of log to be read configUpdateListener.onConnectorConfigUpdate(CONN1); return null; - }); + }).when(configBackingStore).putConnectorConfig(eq(CONN1), eq(CONN1_CONFIG_UPDATED)); + // As a result of reconfig, should need to update snapshot. With only connector updates, we'll just restart // connector without rebalance - EasyMock.expect(configBackingStore.snapshot()).andReturn(SNAPSHOT_UPDATED_CONN1_CONFIG).times(2); - worker.stopAndAwaitConnector(CONN1); - PowerMock.expectLastCall(); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - Capture> onSecondStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onSecondStart)); - PowerMock.expectLastCall().andAnswer(() -> { - onSecondStart.getValue().onCompletion(null, TargetState.STARTED); - return true; - }); - member.wakeup(); - PowerMock.expectLastCall(); - - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - expectExecuteTaskReconfiguration(true, conn1SinkConfigUpdated, () -> TASK_CONFIGS); - - // Third tick just to read the config - member.ensureActive(); - PowerMock.expectLastCall(); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + when(configBackingStore.snapshot()).thenReturn(SNAPSHOT_UPDATED_CONN1_CONFIG); + doNothing().when(worker).stopAndAwaitConnector(CONN1); - PowerMock.replayAll(); - - // Should pick up original config - FutureCallback> connectorConfigCb = new FutureCallback<>(); - herder.connectorConfig(CONN1, connectorConfigCb); - herder.tick(); - assertTrue(connectorConfigCb.isDone()); - assertEquals(CONN1_CONFIG, connectorConfigCb.get()); + ArgumentCaptor> onStart2 = ArgumentCaptor.forClass(Callback.class); + doAnswer(invocation -> { + onStart2.getValue().onCompletion(null, TargetState.STARTED); + return true; + }).when(worker).startConnector(eq(CONN1), eq(CONN1_CONFIG_UPDATED), any(), eq(herder), eq(TargetState.STARTED), onStart2.capture()); + expectExecuteTaskReconfiguration(true, conn1SinkConfigUpdated, invocation -> TASK_CONFIGS); // Apply new config. FutureCallback> putConfigCb = new FutureCallback<>(); @@ -2964,33 +2170,39 @@ public void testPutConnectorConfig() throws Exception { herder.tick(); assertTrue(putConfigCb.isDone()); ConnectorInfo updatedInfo = new ConnectorInfo(CONN1, CONN1_CONFIG_UPDATED, Arrays.asList(TASK0, TASK1, TASK2), - ConnectorType.SOURCE); + ConnectorType.SOURCE); assertEquals(new Herder.Created<>(false, updatedInfo), putConfigCb.get()); - // Check config again to validate change + // Third tick just to read the config - check config again to validate change connectorConfigCb = new FutureCallback<>(); herder.connectorConfig(CONN1, connectorConfigCb); herder.tick(); assertTrue(connectorConfigCb.isDone()); assertEquals(CONN1_CONFIG_UPDATED, connectorConfigCb.get()); - PowerMock.verifyAll(); + // Once after initial rebalance and assignment; another after config update + verify(worker, times(2)).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), any()); + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); } @Test public void testKeyRotationWhenWorkerBecomesLeader() throws Exception { long rotationTtlDelay = DistributedConfig.INTER_WORKER_KEY_TTL_MS_MS_DEFAULT; - EasyMock.expect(member.memberId()).andStubReturn("member"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V2); + when(member.memberId()).thenReturn("member"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V2); expectRebalance(1, Collections.emptyList(), Collections.emptyList()); expectConfigRefreshAndSnapshot(SNAPSHOT); + + doNothing().when(member).poll(anyLong()); + + herder.tick(); + // First rebalance: poll indefinitely as no key has been read yet, so expiration doesn't come into play - member.poll(Long.MAX_VALUE); - EasyMock.expectLastCall(); + verify(member).poll(eq(Long.MAX_VALUE)); expectRebalance(2, Collections.emptyList(), Collections.emptyList()); - SessionKey initialKey = new SessionKey(EasyMock.mock(SecretKey.class), 0); + SessionKey initialKey = new SessionKey(mock(SecretKey.class), 0); ClusterConfigState snapshotWithKey = new ClusterConfigState( 2, initialKey, @@ -3003,41 +2215,39 @@ public void testKeyRotationWhenWorkerBecomesLeader() throws Exception { Collections.emptySet(), Collections.emptySet()); expectConfigRefreshAndSnapshot(snapshotWithKey); + + configUpdateListener.onSessionKeyUpdate(initialKey); + herder.tick(); + // Second rebalance: poll indefinitely as worker is follower, so expiration still doesn't come into play - member.poll(Long.MAX_VALUE); - EasyMock.expectLastCall(); + verify(member, times(2)).poll(eq(Long.MAX_VALUE)); expectRebalance(2, Collections.emptyList(), Collections.emptyList(), "member", MEMBER_URL, true); - Capture updatedKey = EasyMock.newCapture(); - configBackingStore.putSessionKey(EasyMock.capture(updatedKey)); - EasyMock.expectLastCall().andAnswer(() -> { + when(statusBackingStore.connectors()).thenReturn(Collections.emptySet()); + ArgumentCaptor updatedKey = ArgumentCaptor.forClass(SessionKey.class); + doAnswer(invocation -> { configUpdateListener.onSessionKeyUpdate(updatedKey.getValue()); return null; - }); - // Third rebalance: poll for a limited time as worker has become leader and must wake up for key expiration - member.poll(leq(rotationTtlDelay)); - EasyMock.expectLastCall(); - - PowerMock.replayAll(); + }).when(configBackingStore).putSessionKey(updatedKey.capture()); - herder.tick(); - configUpdateListener.onSessionKeyUpdate(initialKey); - herder.tick(); herder.tick(); - PowerMock.verifyAll(); + // Third rebalance: poll for a limited time as worker has become leader and must wake up for key expiration + verify(member).poll(eq(rotationTtlDelay)); + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); } @Test public void testKeyRotationDisabledWhenWorkerBecomesFollower() throws Exception { long rotationTtlDelay = DistributedConfig.INTER_WORKER_KEY_TTL_MS_MS_DEFAULT; - EasyMock.expect(member.memberId()).andStubReturn("member"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V2); + when(member.memberId()).thenReturn("member"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V2); expectRebalance(1, Collections.emptyList(), Collections.emptyList(), "member", MEMBER_URL, true); - SecretKey initialSecretKey = EasyMock.mock(SecretKey.class); - EasyMock.expect(initialSecretKey.getAlgorithm()).andReturn(DistributedConfig.INTER_WORKER_KEY_GENERATION_ALGORITHM_DEFAULT).anyTimes(); - EasyMock.expect(initialSecretKey.getEncoded()).andReturn(new byte[32]).anyTimes(); + when(statusBackingStore.connectors()).thenReturn(Collections.emptySet()); + SecretKey initialSecretKey = mock(SecretKey.class); + when(initialSecretKey.getAlgorithm()).thenReturn(DistributedConfig.INTER_WORKER_KEY_GENERATION_ALGORITHM_DEFAULT); + when(initialSecretKey.getEncoded()).thenReturn(new byte[32]); SessionKey initialKey = new SessionKey(initialSecretKey, time.milliseconds()); ClusterConfigState snapshotWithKey = new ClusterConfigState( 1, @@ -3051,202 +2261,178 @@ public void testKeyRotationDisabledWhenWorkerBecomesFollower() throws Exception Collections.emptySet(), Collections.emptySet()); expectConfigRefreshAndSnapshot(snapshotWithKey); - // First rebalance: poll for a limited time as worker is leader and must wake up for key expiration - member.poll(leq(rotationTtlDelay)); - EasyMock.expectLastCall(); - - expectRebalance(1, Collections.emptyList(), Collections.emptyList()); - // Second rebalance: poll indefinitely as worker is no longer leader, so key expiration doesn't come into play - member.poll(Long.MAX_VALUE); - EasyMock.expectLastCall(); - - PowerMock.replayAll(initialSecretKey); + doNothing().when(member).poll(anyLong()); configUpdateListener.onSessionKeyUpdate(initialKey); herder.tick(); + + // First rebalance: poll for a limited time as worker is leader and must wake up for key expiration + verify(member).poll(leq(rotationTtlDelay)); + + expectRebalance(1, Collections.emptyList(), Collections.emptyList()); herder.tick(); - PowerMock.verifyAll(); + // Second rebalance: poll indefinitely as worker is no longer leader, so key expiration doesn't come into play + verify(member).poll(eq(Long.MAX_VALUE)); + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); } @Test public void testPutTaskConfigsSignatureNotRequiredV0() { - Callback taskConfigCb = EasyMock.mock(Callback.class); - - member.wakeup(); - EasyMock.expectLastCall().once(); - EasyMock.expect(member.currentProtocolVersion()).andReturn(CONNECT_PROTOCOL_V0).anyTimes(); - PowerMock.replayAll(taskConfigCb); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); + Callback taskConfigCb = mock(Callback.class); herder.putTaskConfigs(CONN1, TASK_CONFIGS, taskConfigCb, null); - PowerMock.verifyAll(); + // Expect a wakeup call after the request to write task configs is added to the herder's request queue + verify(member).wakeup(); + verifyNoMoreInteractions(member, taskConfigCb); } + @Test public void testPutTaskConfigsSignatureNotRequiredV1() { - Callback taskConfigCb = EasyMock.mock(Callback.class); - - member.wakeup(); - EasyMock.expectLastCall().once(); - EasyMock.expect(member.currentProtocolVersion()).andReturn(CONNECT_PROTOCOL_V1).anyTimes(); - PowerMock.replayAll(taskConfigCb); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V1); + Callback taskConfigCb = mock(Callback.class); herder.putTaskConfigs(CONN1, TASK_CONFIGS, taskConfigCb, null); - PowerMock.verifyAll(); + // Expect a wakeup call after the request to write task configs is added to the herder's request queue + verify(member).wakeup(); + verifyNoMoreInteractions(member, taskConfigCb); } @Test public void testPutTaskConfigsMissingRequiredSignature() { - Callback taskConfigCb = EasyMock.mock(Callback.class); - Capture errorCapture = Capture.newInstance(); - taskConfigCb.onCompletion(capture(errorCapture), EasyMock.eq(null)); - EasyMock.expectLastCall().once(); - - EasyMock.expect(member.currentProtocolVersion()).andReturn(CONNECT_PROTOCOL_V2).anyTimes(); - PowerMock.replayAll(taskConfigCb); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V2); + Callback taskConfigCb = mock(Callback.class); herder.putTaskConfigs(CONN1, TASK_CONFIGS, taskConfigCb, null); - PowerMock.verifyAll(); + ArgumentCaptor errorCapture = ArgumentCaptor.forClass(Throwable.class); + verify(taskConfigCb).onCompletion(errorCapture.capture(), isNull()); assertTrue(errorCapture.getValue() instanceof BadRequestException); + + verifyNoMoreInteractions(member, taskConfigCb); } @Test public void testPutTaskConfigsDisallowedSignatureAlgorithm() { - Callback taskConfigCb = EasyMock.mock(Callback.class); - Capture errorCapture = Capture.newInstance(); - taskConfigCb.onCompletion(capture(errorCapture), EasyMock.eq(null)); - EasyMock.expectLastCall().once(); - - EasyMock.expect(member.currentProtocolVersion()).andReturn(CONNECT_PROTOCOL_V2).anyTimes(); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V2); - InternalRequestSignature signature = EasyMock.mock(InternalRequestSignature.class); - EasyMock.expect(signature.keyAlgorithm()).andReturn("HmacSHA489").anyTimes(); - - PowerMock.replayAll(taskConfigCb, signature); + InternalRequestSignature signature = mock(InternalRequestSignature.class); + when(signature.keyAlgorithm()).thenReturn("HmacSHA489"); + Callback taskConfigCb = mock(Callback.class); herder.putTaskConfigs(CONN1, TASK_CONFIGS, taskConfigCb, signature); - PowerMock.verifyAll(); + ArgumentCaptor errorCapture = ArgumentCaptor.forClass(Throwable.class); + verify(taskConfigCb).onCompletion(errorCapture.capture(), isNull()); assertTrue(errorCapture.getValue() instanceof BadRequestException); + + verifyNoMoreInteractions(member, taskConfigCb); } @Test public void testPutTaskConfigsInvalidSignature() { - Callback taskConfigCb = EasyMock.mock(Callback.class); - Capture errorCapture = Capture.newInstance(); - taskConfigCb.onCompletion(capture(errorCapture), EasyMock.eq(null)); - EasyMock.expectLastCall().once(); - - EasyMock.expect(member.currentProtocolVersion()).andReturn(CONNECT_PROTOCOL_V2).anyTimes(); - - InternalRequestSignature signature = EasyMock.mock(InternalRequestSignature.class); - EasyMock.expect(signature.keyAlgorithm()).andReturn("HmacSHA256").anyTimes(); - EasyMock.expect(signature.isValid(EasyMock.anyObject())).andReturn(false).anyTimes(); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V2); - SessionKey sessionKey = EasyMock.mock(SessionKey.class); - SecretKey secretKey = EasyMock.niceMock(SecretKey.class); - EasyMock.expect(sessionKey.key()).andReturn(secretKey); - EasyMock.expect(sessionKey.creationTimestamp()).andReturn(time.milliseconds()); + InternalRequestSignature signature = mock(InternalRequestSignature.class); + when(signature.keyAlgorithm()).thenReturn("HmacSHA256"); + when(signature.isValid(any())).thenReturn(false); - PowerMock.replayAll(taskConfigCb, signature, sessionKey, secretKey); + SessionKey sessionKey = mock(SessionKey.class); + SecretKey secretKey = mock(SecretKey.class); + when(sessionKey.key()).thenReturn(secretKey); + when(sessionKey.creationTimestamp()).thenReturn(time.milliseconds()); // Read a new session key from the config topic configUpdateListener.onSessionKeyUpdate(sessionKey); + Callback taskConfigCb = mock(Callback.class); herder.putTaskConfigs(CONN1, TASK_CONFIGS, taskConfigCb, signature); - PowerMock.verifyAll(); + ArgumentCaptor errorCapture = ArgumentCaptor.forClass(Throwable.class); + verify(taskConfigCb).onCompletion(errorCapture.capture(), isNull()); assertTrue(errorCapture.getValue() instanceof ConnectRestException); assertEquals(FORBIDDEN.getStatusCode(), ((ConnectRestException) errorCapture.getValue()).statusCode()); + + verifyNoMoreInteractions(member, taskConfigCb); } @Test public void putTaskConfigsWorkerStillStarting() { - Callback taskConfigCb = EasyMock.mock(Callback.class); - Capture errorCapture = Capture.newInstance(); - taskConfigCb.onCompletion(capture(errorCapture), EasyMock.eq(null)); - EasyMock.expectLastCall().once(); - - EasyMock.expect(member.currentProtocolVersion()).andReturn(CONNECT_PROTOCOL_V2).anyTimes(); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V2); - InternalRequestSignature signature = EasyMock.mock(InternalRequestSignature.class); - EasyMock.expect(signature.keyAlgorithm()).andReturn("HmacSHA256").anyTimes(); - EasyMock.expect(signature.isValid(EasyMock.anyObject())).andReturn(true).anyTimes(); - - PowerMock.replayAll(taskConfigCb, signature); + InternalRequestSignature signature = mock(InternalRequestSignature.class); + when(signature.keyAlgorithm()).thenReturn("HmacSHA256"); + Callback taskConfigCb = mock(Callback.class); herder.putTaskConfigs(CONN1, TASK_CONFIGS, taskConfigCb, signature); - PowerMock.verifyAll(); + ArgumentCaptor errorCapture = ArgumentCaptor.forClass(Throwable.class); + verify(taskConfigCb).onCompletion(errorCapture.capture(), isNull()); assertTrue(errorCapture.getValue() instanceof ConnectRestException); assertEquals(SERVICE_UNAVAILABLE.getStatusCode(), ((ConnectRestException) errorCapture.getValue()).statusCode()); + + verifyNoMoreInteractions(member, taskConfigCb); } @Test public void testPutTaskConfigsValidRequiredSignature() { - Callback taskConfigCb = EasyMock.mock(Callback.class); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V2); - member.wakeup(); - EasyMock.expectLastCall().once(); - EasyMock.expect(member.currentProtocolVersion()).andReturn(CONNECT_PROTOCOL_V2).anyTimes(); + InternalRequestSignature signature = mock(InternalRequestSignature.class); + when(signature.keyAlgorithm()).thenReturn("HmacSHA256"); + when(signature.isValid(any())).thenReturn(true); - InternalRequestSignature signature = EasyMock.mock(InternalRequestSignature.class); - EasyMock.expect(signature.keyAlgorithm()).andReturn("HmacSHA256").anyTimes(); - EasyMock.expect(signature.isValid(EasyMock.anyObject())).andReturn(true).anyTimes(); - - SessionKey sessionKey = EasyMock.mock(SessionKey.class); - SecretKey secretKey = EasyMock.niceMock(SecretKey.class); - EasyMock.expect(sessionKey.key()).andReturn(secretKey); - EasyMock.expect(sessionKey.creationTimestamp()).andReturn(time.milliseconds()); - - PowerMock.replayAll(taskConfigCb, signature, sessionKey, secretKey); + SessionKey sessionKey = mock(SessionKey.class); + SecretKey secretKey = mock(SecretKey.class); + when(sessionKey.key()).thenReturn(secretKey); + when(sessionKey.creationTimestamp()).thenReturn(time.milliseconds()); // Read a new session key from the config topic configUpdateListener.onSessionKeyUpdate(sessionKey); + Callback taskConfigCb = mock(Callback.class); herder.putTaskConfigs(CONN1, TASK_CONFIGS, taskConfigCb, signature); - PowerMock.verifyAll(); + // Expect a wakeup call after the request to write task configs is added to the herder's request queue + verify(member).wakeup(); + verifyNoMoreInteractions(member, taskConfigCb); } @Test public void testFailedToWriteSessionKey() throws Exception { // First tick -- after joining the group, we try to write a new // session key to the config topic, and fail - EasyMock.expect(member.memberId()).andStubReturn("leader"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V2); + when(member.memberId()).thenReturn("leader"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V2); expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true); expectConfigRefreshAndSnapshot(SNAPSHOT); - configBackingStore.putSessionKey(anyObject(SessionKey.class)); - EasyMock.expectLastCall().andThrow(new ConnectException("Oh no!")); + + doThrow(new ConnectException("Oh no!")).when(configBackingStore).putSessionKey(any(SessionKey.class)); + herder.tick(); // Second tick -- we read to the end of the config topic first, // then ensure we're still active in the group // then try a second time to write a new session key, // then finally begin polling for group activity expectConfigRefreshAndSnapshot(SNAPSHOT); - member.ensureActive(); - PowerMock.expectLastCall(); - configBackingStore.putSessionKey(anyObject(SessionKey.class)); - EasyMock.expectLastCall(); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - PowerMock.replayAll(); + doNothing().when(configBackingStore).putSessionKey(any(SessionKey.class)); - herder.tick(); herder.tick(); - PowerMock.verifyAll(); + verify(member, times(2)).ensureActive(); + verify(member, times(1)).poll(anyLong()); + verify(configBackingStore, times(2)).putSessionKey(any(SessionKey.class)); } @Test public void testFailedToReadBackNewlyWrittenSessionKey() throws Exception { - SecretKey secretKey = EasyMock.niceMock(SecretKey.class); - EasyMock.expect(secretKey.getAlgorithm()).andReturn(INTER_WORKER_KEY_GENERATION_ALGORITHM_DEFAULT); - EasyMock.expect(secretKey.getEncoded()).andReturn(new byte[32]); + SecretKey secretKey = mock(SecretKey.class); + when(secretKey.getAlgorithm()).thenReturn(INTER_WORKER_KEY_GENERATION_ALGORITHM_DEFAULT); + when(secretKey.getEncoded()).thenReturn(new byte[32]); SessionKey sessionKey = new SessionKey(secretKey, time.milliseconds()); ClusterConfigState snapshotWithSessionKey = new ClusterConfigState( 1, @@ -3265,67 +2451,60 @@ public void testFailedToReadBackNewlyWrittenSessionKey() throws Exception { // actually written the key successfully, but haven't been able to read it back // from the config topic, so to the herder it looks the same as if it'd just failed // to write the key) - EasyMock.expect(member.memberId()).andStubReturn("leader"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V2); + when(member.memberId()).thenReturn("leader"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V2); expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true); expectConfigRefreshAndSnapshot(SNAPSHOT); - configBackingStore.putSessionKey(anyObject(SessionKey.class)); - EasyMock.expectLastCall().andThrow(new ConnectException("Oh no!")); + doThrow(new ConnectException("Oh no!")).when(configBackingStore).putSessionKey(any(SessionKey.class)); + + herder.tick(); // Second tick -- we read to the end of the config topic first, and pick up // the session key that we were able to write the last time, // then ensure we're still active in the group // then finally begin polling for group activity // Importantly, we do not try to write a new session key this time around - configBackingStore.refresh(anyLong(), EasyMock.anyObject(TimeUnit.class)); - EasyMock.expectLastCall().andAnswer(() -> { + doAnswer(invocation -> { configUpdateListener.onSessionKeyUpdate(sessionKey); return null; - }); - EasyMock.expect(configBackingStore.snapshot()).andReturn(snapshotWithSessionKey); - member.ensureActive(); - PowerMock.expectLastCall(); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + }).when(configBackingStore).refresh(anyLong(), any(TimeUnit.class)); + when(configBackingStore.snapshot()).thenReturn(snapshotWithSessionKey); - PowerMock.replayAll(secretKey); - - herder.tick(); herder.tick(); - PowerMock.verifyAll(); + verify(member, times(2)).ensureActive(); + verify(member, times(1)).poll(anyLong()); + verify(configBackingStore, times(1)).putSessionKey(any(SessionKey.class)); } @Test public void testFenceZombiesInvalidSignature() { // Don't have to run the whole gamut of scenarios (invalid signature, missing signature, earlier protocol that doesn't require signatures) // since the task config tests cover that pretty well. One sanity check to ensure that this method is guarded should be sufficient. - Callback taskConfigCb = EasyMock.mock(Callback.class); - Capture errorCapture = Capture.newInstance(); - taskConfigCb.onCompletion(capture(errorCapture), EasyMock.eq(null)); - EasyMock.expectLastCall().once(); - EasyMock.expect(member.currentProtocolVersion()).andReturn(CONNECT_PROTOCOL_V2).anyTimes(); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V2); - InternalRequestSignature signature = EasyMock.mock(InternalRequestSignature.class); - EasyMock.expect(signature.keyAlgorithm()).andReturn("HmacSHA256").anyTimes(); - EasyMock.expect(signature.isValid(EasyMock.anyObject())).andReturn(false).anyTimes(); + InternalRequestSignature signature = mock(InternalRequestSignature.class); + when(signature.keyAlgorithm()).thenReturn("HmacSHA256"); + when(signature.isValid(any())).thenReturn(false); - SessionKey sessionKey = EasyMock.mock(SessionKey.class); - SecretKey secretKey = EasyMock.niceMock(SecretKey.class); - EasyMock.expect(sessionKey.key()).andReturn(secretKey); - EasyMock.expect(sessionKey.creationTimestamp()).andReturn(time.milliseconds()); - - PowerMock.replayAll(taskConfigCb, signature, sessionKey, secretKey); + SessionKey sessionKey = mock(SessionKey.class); + SecretKey secretKey = mock(SecretKey.class); + when(sessionKey.key()).thenReturn(secretKey); + when(sessionKey.creationTimestamp()).thenReturn(time.milliseconds()); // Read a new session key from the config topic configUpdateListener.onSessionKeyUpdate(sessionKey); + Callback taskConfigCb = mock(Callback.class); herder.fenceZombieSourceTasks(CONN1, taskConfigCb, signature); - PowerMock.verifyAll(); + ArgumentCaptor errorCapture = ArgumentCaptor.forClass(Throwable.class); + verify(taskConfigCb).onCompletion(errorCapture.capture(), isNull()); assertTrue(errorCapture.getValue() instanceof ConnectRestException); assertEquals(FORBIDDEN.getStatusCode(), ((ConnectRestException) errorCapture.getValue()).statusCode()); + + verifyNoMoreInteractions(member); } @Test @@ -3340,53 +2519,36 @@ public void testTaskRequestedZombieFencingFailedForwardToLeader() throws Excepti private void testTaskRequestedZombieFencingForwardingToLeader(boolean succeed) throws Exception { expectHerderStartup(); - ExecutorService forwardRequestExecutor = EasyMock.mock(ExecutorService.class); + ExecutorService forwardRequestExecutor = mock(ExecutorService.class); herder.forwardRequestExecutor = forwardRequestExecutor; - EasyMock.expect(member.memberId()).andStubReturn("member"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V2); + when(member.memberId()).thenReturn("member"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V2); expectConfigRefreshAndSnapshot(SNAPSHOT); expectRebalance(1, Collections.emptyList(), Collections.emptyList()); + doNothing().when(member).poll(anyLong()); + + OngoingStubbing> expectRequest = when(restClient.httpRequest( + any(), eq("PUT"), isNull(), isNull(), isNull(), any(), any() + )); - expectAnyTicks(); - - // There's a race condition in DistributedHerder::addRequest that isn't triggered - // often under normal circumstances, but becomes more likely during this test - // because we mock out member::poll and the herder no longer blocks when invoking - // that method - // This can cause the herder to skip a call to member::wakeup during that method, and - // since the number of calls to that method isn't vital to our testing coverage, we permit - // any number of calls - // The race condition itself is benign and should have no negative impact on herder logic; - // see DistributedHerder::addRequest for more detail - member.wakeup(); - EasyMock.expectLastCall().anyTimes(); - - org.easymock.IExpectationSetters> expectRequest = EasyMock.expect( - restClient.httpRequest( - anyObject(), EasyMock.eq("PUT"), EasyMock.isNull(), EasyMock.isNull(), EasyMock.isNull(), anyObject(), anyObject() - )); if (succeed) { - expectRequest.andReturn(null); + expectRequest.thenReturn(null); } else { - expectRequest.andThrow(new ConnectRestException(409, "Rebalance :(")); + expectRequest.thenThrow(new ConnectRestException(409, "Rebalance :(")); } - Capture forwardRequest = EasyMock.newCapture(); - forwardRequestExecutor.execute(EasyMock.capture(forwardRequest)); - EasyMock.expectLastCall().andAnswer(() -> { + ArgumentCaptor forwardRequest = ArgumentCaptor.forClass(Runnable.class); + + doAnswer(invocation -> { forwardRequest.getValue().run(); return null; - }); + }).when(forwardRequestExecutor).execute(forwardRequest.capture()); expectHerderShutdown(); - forwardRequestExecutor.shutdown(); - EasyMock.expectLastCall(); - EasyMock.expect(forwardRequestExecutor.awaitTermination(anyLong(), anyObject())).andReturn(true); - - PowerMock.replayAll(forwardRequestExecutor); - + doNothing().when(forwardRequestExecutor).shutdown(); + when(forwardRequestExecutor.awaitTermination(anyLong(), any())).thenReturn(true); startBackgroundHerder(); @@ -3403,7 +2565,7 @@ private void testTaskRequestedZombieFencingForwardingToLeader(boolean succeed) t stopBackgroundHerder(); - PowerMock.verifyAll(); + verifyNoMoreInteractions(member, worker, forwardRequestExecutor); } @Test @@ -3447,29 +2609,21 @@ private void testExternalZombieFencingRequestThatRequiresNoPhysicalFencing( ) throws Exception { expectHerderStartup(); - EasyMock.expect(member.memberId()).andStubReturn("leader"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V2); - expectConfigRefreshAndSnapshot(SNAPSHOT); + when(member.memberId()).thenReturn("leader"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V2); + expectConfigRefreshAndSnapshot(configState); expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true); - expectAnyTicks(); - - member.wakeup(); - EasyMock.expectLastCall().anyTimes(); - - expectConfigRefreshAndSnapshot(configState); + when(statusBackingStore.connectors()).thenReturn(Collections.emptySet()); + doNothing().when(member).poll(anyLong()); if (expectTaskCountRecord) { - configBackingStore.putTaskCountRecord(CONN1, 1); - EasyMock.expectLastCall(); + doNothing().when(configBackingStore).putTaskCountRecord(CONN1, 1); } expectHerderShutdown(); - PowerMock.replayAll(); - - startBackgroundHerder(); FutureCallback fencing = new FutureCallback<>(); @@ -3479,7 +2633,7 @@ private void testExternalZombieFencingRequestThatRequiresNoPhysicalFencing( stopBackgroundHerder(); - PowerMock.verifyAll(); + verifyNoMoreInteractions(member, worker, configBackingStore, statusBackingStore); } /** @@ -3490,18 +2644,12 @@ private void testExternalZombieFencingRequestThatRequiresNoPhysicalFencing( @Test public void testExternalZombieFencingRequestImmediateCompletion() throws Exception { expectHerderStartup(); - EasyMock.expect(member.memberId()).andStubReturn("leader"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V2); - expectConfigRefreshAndSnapshot(SNAPSHOT); + when(member.memberId()).thenReturn("leader"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V2); expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true); SessionKey sessionKey = expectNewSessionKey(); - expectAnyTicks(); - - member.wakeup(); - EasyMock.expectLastCall().anyTimes(); - ClusterConfigState configState = exactlyOnceSnapshot( sessionKey, TASK_CONFIGS_MAP, @@ -3511,38 +2659,32 @@ public void testExternalZombieFencingRequestImmediateCompletion() throws Excepti ); expectConfigRefreshAndSnapshot(configState); + when(statusBackingStore.connectors()).thenReturn(Collections.emptySet()); + doNothing().when(member).poll(anyLong()); + // The future returned by Worker::fenceZombies - KafkaFuture workerFencingFuture = EasyMock.mock(KafkaFuture.class); - // The future tracked by the herder (which tracks the fencing performed by the worker and the possible followup write to the config topic) - KafkaFuture herderFencingFuture = EasyMock.mock(KafkaFuture.class); + KafkaFuture workerFencingFuture = mock(KafkaFuture.class); + // The future tracked by the herder (which tracks the fencing performed by the worker and the possible followup write to the config topic) + KafkaFuture herderFencingFuture = mock(KafkaFuture.class); // Immediately invoke callbacks that the herder sets up for when the worker fencing and writes to the config topic have completed - for (int i = 0; i < 2; i++) { - Capture> herderFencingCallback = EasyMock.newCapture(); - EasyMock.expect(herderFencingFuture.whenComplete(EasyMock.capture(herderFencingCallback))).andAnswer(() -> { - herderFencingCallback.getValue().accept(null, null); - return null; - }); - } + ArgumentCaptor> herderFencingCallback = ArgumentCaptor.forClass(KafkaFuture.BiConsumer.class); + when(herderFencingFuture.whenComplete(herderFencingCallback.capture())).thenAnswer(invocation -> { + herderFencingCallback.getValue().accept(null, null); + return null; + }); - Capture> fencingFollowup = EasyMock.newCapture(); - EasyMock.expect(workerFencingFuture.thenApply(EasyMock.capture(fencingFollowup))).andAnswer(() -> { + ArgumentCaptor> fencingFollowup = ArgumentCaptor.forClass(KafkaFuture.BaseFunction.class); + when(workerFencingFuture.thenApply(fencingFollowup.capture())).thenAnswer(invocation -> { fencingFollowup.getValue().apply(null); return herderFencingFuture; }); - EasyMock.expect(worker.fenceZombies(EasyMock.eq(CONN1), EasyMock.eq(2), EasyMock.eq(CONN1_CONFIG))) - .andReturn(workerFencingFuture); - - expectConfigRefreshAndSnapshot(configState); + when(worker.fenceZombies(eq(CONN1), eq(2), eq(CONN1_CONFIG))).thenReturn(workerFencingFuture); - configBackingStore.putTaskCountRecord(CONN1, 1); - EasyMock.expectLastCall(); + doNothing().when(configBackingStore).putTaskCountRecord(CONN1, 1); expectHerderShutdown(); - PowerMock.replayAll(workerFencingFuture, herderFencingFuture); - - startBackgroundHerder(); FutureCallback fencing = new FutureCallback<>(); @@ -3552,7 +2694,7 @@ public void testExternalZombieFencingRequestImmediateCompletion() throws Excepti stopBackgroundHerder(); - PowerMock.verifyAll(); + verifyNoMoreInteractions(herderFencingFuture, workerFencingFuture, member, worker, configBackingStore, statusBackingStore); } /** @@ -3561,18 +2703,12 @@ public void testExternalZombieFencingRequestImmediateCompletion() throws Excepti @Test public void testExternalZombieFencingRequestSynchronousFailure() throws Exception { expectHerderStartup(); - EasyMock.expect(member.memberId()).andStubReturn("leader"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V2); - expectConfigRefreshAndSnapshot(SNAPSHOT); + when(member.memberId()).thenReturn("leader"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V2); expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true); SessionKey sessionKey = expectNewSessionKey(); - expectAnyTicks(); - - member.wakeup(); - EasyMock.expectLastCall().anyTimes(); - ClusterConfigState configState = exactlyOnceSnapshot( sessionKey, TASK_CONFIGS_MAP, @@ -3582,15 +2718,14 @@ public void testExternalZombieFencingRequestSynchronousFailure() throws Exceptio ); expectConfigRefreshAndSnapshot(configState); + when(statusBackingStore.connectors()).thenReturn(Collections.emptySet()); + doNothing().when(member).poll(anyLong()); + Exception fencingException = new KafkaException("whoops!"); - EasyMock.expect(worker.fenceZombies(EasyMock.eq(CONN1), EasyMock.eq(2), EasyMock.eq(CONN1_CONFIG))) - .andThrow(fencingException); + when(worker.fenceZombies(eq(CONN1), eq(2), eq(CONN1_CONFIG))).thenThrow(fencingException); expectHerderShutdown(); - PowerMock.replayAll(); - - startBackgroundHerder(); FutureCallback fencing = new FutureCallback<>(); @@ -3601,7 +2736,7 @@ public void testExternalZombieFencingRequestSynchronousFailure() throws Exceptio stopBackgroundHerder(); - PowerMock.verifyAll(); + verifyNoMoreInteractions(member, worker, configBackingStore, statusBackingStore); } /** @@ -3611,18 +2746,12 @@ public void testExternalZombieFencingRequestSynchronousFailure() throws Exceptio @Test public void testExternalZombieFencingRequestAsynchronousFailure() throws Exception { expectHerderStartup(); - EasyMock.expect(member.memberId()).andStubReturn("leader"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V2); - expectConfigRefreshAndSnapshot(SNAPSHOT); + when(member.memberId()).thenReturn("leader"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V2); expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true); SessionKey sessionKey = expectNewSessionKey(); - expectAnyTicks(); - - member.wakeup(); - EasyMock.expectLastCall().anyTimes(); - ClusterConfigState configState = exactlyOnceSnapshot( sessionKey, TASK_CONFIGS_MAP, @@ -3632,32 +2761,28 @@ public void testExternalZombieFencingRequestAsynchronousFailure() throws Excepti ); expectConfigRefreshAndSnapshot(configState); + when(statusBackingStore.connectors()).thenReturn(Collections.emptySet()); + doNothing().when(member).poll(anyLong()); + // The future returned by Worker::fenceZombies - KafkaFuture workerFencingFuture = EasyMock.mock(KafkaFuture.class); - // The future tracked by the herder (which tracks the fencing performed by the worker and the possible followup write to the config topic) - KafkaFuture herderFencingFuture = EasyMock.mock(KafkaFuture.class); + KafkaFuture workerFencingFuture = mock(KafkaFuture.class); + // The future tracked by the herder (which tracks the fencing performed by the worker and the possible followup write to the config topic) + KafkaFuture herderFencingFuture = mock(KafkaFuture.class); // The callbacks that the herder has accrued for outstanding fencing futures - Capture> herderFencingCallbacks = EasyMock.newCapture(CaptureType.ALL); + ArgumentCaptor> herderFencingCallbacks = ArgumentCaptor.forClass(KafkaFuture.BiConsumer.class); - EasyMock.expect(worker.fenceZombies(EasyMock.eq(CONN1), EasyMock.eq(2), EasyMock.eq(CONN1_CONFIG))) - .andReturn(workerFencingFuture); + when(worker.fenceZombies(eq(CONN1), eq(2), eq(CONN1_CONFIG))).thenReturn(workerFencingFuture); - EasyMock.expect(workerFencingFuture.thenApply(EasyMock.>anyObject())) - .andReturn(herderFencingFuture); + when(workerFencingFuture.thenApply(any(KafkaFuture.BaseFunction.class))).thenReturn(herderFencingFuture); CountDownLatch callbacksInstalled = new CountDownLatch(2); - for (int i = 0; i < 2; i++) { - EasyMock.expect(herderFencingFuture.whenComplete(EasyMock.capture(herderFencingCallbacks))).andAnswer(() -> { - callbacksInstalled.countDown(); - return null; - }); - } + when(herderFencingFuture.whenComplete(herderFencingCallbacks.capture())).thenAnswer(invocation -> { + callbacksInstalled.countDown(); + return null; + }); expectHerderShutdown(); - PowerMock.replayAll(workerFencingFuture, herderFencingFuture); - - startBackgroundHerder(); FutureCallback fencing = new FutureCallback<>(); @@ -3666,14 +2791,14 @@ public void testExternalZombieFencingRequestAsynchronousFailure() throws Excepti assertTrue(callbacksInstalled.await(10, TimeUnit.SECONDS)); Exception fencingException = new AuthorizationException("you didn't say the magic word"); - herderFencingCallbacks.getValues().forEach(cb -> cb.accept(null, fencingException)); + herderFencingCallbacks.getAllValues().forEach(cb -> cb.accept(null, fencingException)); ExecutionException exception = assertThrows(ExecutionException.class, () -> fencing.get(10, TimeUnit.SECONDS)); assertTrue(exception.getCause() instanceof ConnectException); stopBackgroundHerder(); - PowerMock.verifyAll(); + verifyNoMoreInteractions(herderFencingFuture, workerFencingFuture, member, worker, configBackingStore, statusBackingStore); } /** @@ -3683,30 +2808,19 @@ public void testExternalZombieFencingRequestAsynchronousFailure() throws Excepti */ @Test public void testExternalZombieFencingRequestDelayedCompletion() throws Exception { - final String conn3 = "SourceC"; + final String conn3 = "sourceC"; final Map tasksPerConnector = new HashMap<>(); tasksPerConnector.put(CONN1, 5); tasksPerConnector.put(CONN2, 3); tasksPerConnector.put(conn3, 12); expectHerderStartup(); - EasyMock.expect(member.memberId()).andStubReturn("leader"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V2); - expectConfigRefreshAndSnapshot(SNAPSHOT); + when(member.memberId()).thenReturn("leader"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V2); expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true); SessionKey sessionKey = expectNewSessionKey(); - expectAnyTicks(); - - // We invoke the herder's fenceZombies method repeatedly, which adds a new request to the queue. - // If the queue is empty, the member is woken up; however, if two or more requests are issued in rapid - // succession, the member won't be woken up. We allow the member to be woken up any number of times - // here since it's not critical to the testing logic and it's difficult to mock things in order to lead to an - // exact number of wakeups. - member.wakeup(); - EasyMock.expectLastCall().anyTimes(); - Map taskCountRecords = new HashMap<>(); taskCountRecords.put(CONN1, 2); taskCountRecords.put(CONN2, 3); @@ -3724,62 +2838,48 @@ public void testExternalZombieFencingRequestDelayedCompletion() throws Exception pendingFencing, tasksPerConnector ); - tasksPerConnector.keySet().forEach(c -> expectConfigRefreshAndSnapshot(configState)); + expectConfigRefreshAndSnapshot(configState); + + when(statusBackingStore.connectors()).thenReturn(Collections.emptySet()); + doNothing().when(member).poll(anyLong()); // The callbacks that the herder has accrued for outstanding fencing futures, which will be completed after // a successful round of fencing and a task record write to the config topic - Map>> herderFencingCallbacks = new HashMap<>(); + Map>> herderFencingCallbacks = new HashMap<>(); // The callbacks that the herder has installed for after a successful round of zombie fencing, but before writing // a task record to the config topic - Map>> workerFencingFollowups = new HashMap<>(); + Map>> workerFencingFollowups = new HashMap<>(); Map callbacksInstalled = new HashMap<>(); - tasksPerConnector.forEach((connector, numStackedRequests) -> { + tasksPerConnector.keySet().forEach(connector -> { // The future returned by Worker::fenceZombies - KafkaFuture workerFencingFuture = EasyMock.mock(KafkaFuture.class); - // The future tracked by the herder (which tracks the fencing performed by the worker and the possible followup write to the config topic) - KafkaFuture herderFencingFuture = EasyMock.mock(KafkaFuture.class); + KafkaFuture workerFencingFuture = mock(KafkaFuture.class); + // The future tracked by the herder (which tracks the fencing performed by the worker and the possible followup write to the config topic) + KafkaFuture herderFencingFuture = mock(KafkaFuture.class); - Capture> herderFencingCallback = EasyMock.newCapture(CaptureType.ALL); + ArgumentCaptor> herderFencingCallback = ArgumentCaptor.forClass(KafkaFuture.BiConsumer.class); herderFencingCallbacks.put(connector, herderFencingCallback); // Don't immediately invoke callbacks that the herder sets up for when the worker fencing and writes to the config topic have completed // Instead, wait for them to be installed, then invoke them explicitly after the fact on a thread separate from the herder's tick thread - EasyMock.expect(herderFencingFuture.whenComplete(EasyMock.capture(herderFencingCallback))) - .andReturn(null) - .times(numStackedRequests + 1); + when(herderFencingFuture.whenComplete(herderFencingCallback.capture())).thenReturn(null); - Capture> fencingFollowup = EasyMock.newCapture(); + ArgumentCaptor> fencingFollowup = ArgumentCaptor.forClass(KafkaFuture.BaseFunction.class); CountDownLatch callbackInstalled = new CountDownLatch(1); workerFencingFollowups.put(connector, fencingFollowup); callbacksInstalled.put(connector, callbackInstalled); - EasyMock.expect(workerFencingFuture.thenApply(EasyMock.capture(fencingFollowup))).andAnswer(() -> { + when(workerFencingFuture.thenApply(fencingFollowup.capture())).thenAnswer(invocation -> { callbackInstalled.countDown(); return herderFencingFuture; }); - // We should only perform a single physical zombie fencing; all the subsequent requests should be stacked onto the first one - EasyMock.expect(worker.fenceZombies( - EasyMock.eq(connector), EasyMock.eq(taskCountRecords.get(connector)), EasyMock.anyObject()) - ).andReturn(workerFencingFuture); - - for (int i = 0; i < numStackedRequests; i++) { - expectConfigRefreshAndSnapshot(configState); - } - - PowerMock.replay(workerFencingFuture, herderFencingFuture); + when(worker.fenceZombies(eq(connector), eq(taskCountRecords.get(connector)), any())).thenReturn(workerFencingFuture); }); - tasksPerConnector.forEach((connector, taskCount) -> { - configBackingStore.putTaskCountRecord(connector, taskCount); - EasyMock.expectLastCall(); - }); + tasksPerConnector.forEach((connector, taskCount) -> doNothing().when(configBackingStore).putTaskCountRecord(eq(connector), eq(taskCount))); expectHerderShutdown(); - PowerMock.replayAll(); - - startBackgroundHerder(); List> stackedFencingRequests = new ArrayList<>(); @@ -3799,7 +2899,7 @@ public void testExternalZombieFencingRequestDelayedCompletion() throws Exception try { assertTrue(latch.await(10, TimeUnit.SECONDS)); workerFencingFollowups.get(connector).getValue().apply(null); - herderFencingCallbacks.get(connector).getValues().forEach(cb -> cb.accept(null, null)); + herderFencingCallbacks.get(connector).getAllValues().forEach(cb -> cb.accept(null, null)); } catch (InterruptedException e) { fail("Unexpectedly interrupted"); } @@ -3811,7 +2911,9 @@ public void testExternalZombieFencingRequestDelayedCompletion() throws Exception stopBackgroundHerder(); - PowerMock.verifyAll(); + // We should only perform a single physical zombie fencing for each connector; all the subsequent requests should be stacked onto the first one + tasksPerConnector.keySet().forEach(connector -> verify(worker).fenceZombies(eq(connector), eq(taskCountRecords.get(connector)), any())); + verifyNoMoreInteractions(member, worker, configBackingStore, statusBackingStore); } @Test @@ -3829,13 +2931,7 @@ public void testVerifyTaskGeneration() { Collections.emptySet(), Collections.emptySet()); - Callback verifyCallback = EasyMock.mock(Callback.class); - for (int i = 0; i < 5; i++) { - verifyCallback.onCompletion(null, null); - EasyMock.expectLastCall(); - } - - PowerMock.replayAll(); + Callback verifyCallback = mock(Callback.class); herder.assignment = new ExtendedAssignment( (short) 2, (short) 0, "leader", "leaderUrl", 0, @@ -3872,7 +2968,7 @@ public void testVerifyTaskGeneration() { assertThrows(ConnectException.class, () -> herder.verifyTaskGenerationAndOwnership(unassignedTask, 1, verifyCallback)); assertThrows(ConnectException.class, () -> herder.verifyTaskGenerationAndOwnership(unassignedTask, 2, verifyCallback)); - PowerMock.verifyAll(); + verify(verifyCallback, times(3)).onCompletion(isNull(), isNull()); } @Test @@ -3901,17 +2997,16 @@ public void testInconsistentConfigs() { // This requires inter-worker communication, so needs the REST API } - @Test public void testThreadNames() { - assertTrue(Whitebox.getInternalState(herder, "herderExecutor"). - getThreadFactory().newThread(EMPTY_RUNNABLE).getName().startsWith(DistributedHerder.class.getSimpleName())); + assertTrue(((ThreadPoolExecutor) herder.herderExecutor).getThreadFactory().newThread(EMPTY_RUNNABLE).getName() + .startsWith(DistributedHerder.class.getSimpleName())); - assertTrue(Whitebox.getInternalState(herder, "forwardRequestExecutor"). - getThreadFactory().newThread(EMPTY_RUNNABLE).getName().startsWith("ForwardRequestExecutor")); + assertTrue(((ThreadPoolExecutor) herder.forwardRequestExecutor).getThreadFactory().newThread(EMPTY_RUNNABLE).getName() + .startsWith("ForwardRequestExecutor")); - assertTrue(Whitebox.getInternalState(herder, "startAndStopExecutor"). - getThreadFactory().newThread(EMPTY_RUNNABLE).getName().startsWith("StartAndStopExecutor")); + assertTrue(((ThreadPoolExecutor) herder.startAndStopExecutor).getThreadFactory().newThread(EMPTY_RUNNABLE).getName() + .startsWith("StartAndStopExecutor")); } @Test @@ -3929,397 +3024,524 @@ public void testPollDurationOnSlowConnectorOperations() { final int rebalanceDelayMs = 20000; final long operationDelayMs = 10000; final long maxPollWaitMs = rebalanceDelayMs - operationDelayMs; - EasyMock.expect(member.memberId()).andStubReturn("member"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(connectProtocolVersion); + when(member.memberId()).thenReturn("member"); + when(member.currentProtocolVersion()).thenReturn(connectProtocolVersion); // Assign the connector to this worker, and have it start expectRebalance(Collections.emptyList(), Collections.emptyList(), ConnectProtocol.Assignment.NO_ERROR, 1, singletonList(CONN1), Collections.emptyList(), rebalanceDelayMs); expectConfigRefreshAndSnapshot(SNAPSHOT); - Capture> onFirstStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onFirstStart)); - PowerMock.expectLastCall().andAnswer(() -> { + ArgumentCaptor> onFirstStart = ArgumentCaptor.forClass(Callback.class); + doAnswer(invocation -> { time.sleep(operationDelayMs); onFirstStart.getValue().onCompletion(null, TargetState.STARTED); return true; - }); - member.wakeup(); - PowerMock.expectLastCall(); - expectExecuteTaskReconfiguration(true, conn1SinkConfig, () -> TASK_CONFIGS); - // We should poll for less than the delay - time to start the connector, meaning that a long connector start - // does not delay the poll timeout - member.poll(leq(maxPollWaitMs)); - PowerMock.expectLastCall(); + }).when(worker).startConnector(eq(CONN1), eq(CONN1_CONFIG), any(), eq(herder), eq(TargetState.STARTED), onFirstStart.capture()); + expectExecuteTaskReconfiguration(true, conn1SinkConfig, invocation -> TASK_CONFIGS); + + herder.tick(); // Rebalance again due to config update - member.wakeup(); - PowerMock.expectLastCall(); expectRebalance(Collections.emptyList(), Collections.emptyList(), ConnectProtocol.Assignment.NO_ERROR, 1, singletonList(CONN1), Collections.emptyList(), rebalanceDelayMs); - EasyMock.expect(configBackingStore.snapshot()).andReturn(SNAPSHOT_UPDATED_CONN1_CONFIG); - - worker.stopAndAwaitConnector(CONN1); - PowerMock.expectLastCall(); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(connectProtocolVersion); - Capture> onSecondStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onSecondStart)); - PowerMock.expectLastCall().andAnswer(() -> { + when(configBackingStore.snapshot()).thenReturn(SNAPSHOT_UPDATED_CONN1_CONFIG); + doNothing().when(worker).stopAndAwaitConnector(CONN1); + + ArgumentCaptor> onSecondStart = ArgumentCaptor.forClass(Callback.class); + doAnswer(invocation -> { time.sleep(operationDelayMs); onSecondStart.getValue().onCompletion(null, TargetState.STARTED); return true; - }); - member.wakeup(); - PowerMock.expectLastCall(); - member.poll(leq(maxPollWaitMs)); - PowerMock.expectLastCall(); + }).when(worker).startConnector(eq(CONN1), eq(CONN1_CONFIG_UPDATED), any(), eq(herder), eq(TargetState.STARTED), onSecondStart.capture()); + expectExecuteTaskReconfiguration(true, conn1SinkConfigUpdated, invocation -> TASK_CONFIGS); + + configUpdateListener.onConnectorConfigUpdate(CONN1); // read updated config + herder.tick(); // Third tick should resolve all outstanding requests expectRebalance(Collections.emptyList(), Collections.emptyList(), ConnectProtocol.Assignment.NO_ERROR, 1, singletonList(CONN1), Collections.emptyList(), rebalanceDelayMs); // which includes querying the connector task configs after the update - expectExecuteTaskReconfiguration(true, conn1SinkConfigUpdated, () -> { + expectExecuteTaskReconfiguration(true, conn1SinkConfigUpdated, invocation -> { time.sleep(operationDelayMs); return TASK_CONFIGS; }); - member.poll(leq(maxPollWaitMs)); - PowerMock.expectLastCall(); - - PowerMock.replayAll(); - herder.tick(); - configUpdateListener.onConnectorConfigUpdate(CONN1); // read updated config - herder.tick(); herder.tick(); - PowerMock.verifyAll(); + + // We should poll for less than the delay - time to start the connector, meaning that a long connector start + // does not delay the poll timeout + verify(member, times(3)).poll(leq(maxPollWaitMs)); + verify(worker, times(2)).startConnector(eq(CONN1), any(), any(), eq(herder), eq(TargetState.STARTED), any()); + verifyNoMoreInteractions(member, worker, configBackingStore); } @Test public void shouldThrowWhenStartAndStopExecutorThrowsRejectedExecutionExceptionAndHerderNotStopping() { - EasyMock.expect(member.memberId()).andStubReturn("leader"); + when(member.memberId()).thenReturn("leader"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); expectRebalance(1, Arrays.asList(CONN1), Collections.emptyList(), true); expectConfigRefreshAndSnapshot(SNAPSHOT); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - - PowerMock.replayAll(); herder.startAndStopExecutor.shutdown(); assertThrows(RejectedExecutionException.class, herder::tick); + } + + @Test + public void testTaskReconfigurationRetriesWithConnectorTaskConfigsException() { + when(member.memberId()).thenReturn("leader"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); + expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true); + expectConfigRefreshAndSnapshot(SNAPSHOT); + + when(worker.isRunning(CONN1)).thenReturn(true); + when(worker.getPlugins()).thenReturn(plugins); + + SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, CONN1_CONFIG); + + when(worker.connectorTaskConfigs(CONN1, sinkConnectorConfig)) + .thenThrow(new ConnectException("Failed to generate task configs")) + .thenThrow(new ConnectException("Failed to generate task configs")) + .thenReturn(TASK_CONFIGS); + + expectAndVerifyTaskReconfigurationRetries(); + } + + @Test + public void testTaskReconfigurationRetriesWithLeaderRequestForwardingException() { + herder = mock(DistributedHerder.class, withSettings().defaultAnswer(CALLS_REAL_METHODS).useConstructor(new DistributedConfig(HERDER_CONFIG), + worker, WORKER_ID, KAFKA_CLUSTER_ID, statusBackingStore, configBackingStore, member, MEMBER_URL, restClient, metrics, time, + noneConnectorClientConfigOverridePolicy, Collections.emptyList(), new MockSynchronousExecutor(), new AutoCloseable[]{})); + + rebalanceListener = herder.new RebalanceListener(time); + + when(member.memberId()).thenReturn("member"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); + expectRebalance(1, Collections.emptyList(), Collections.emptyList(), false); + expectConfigRefreshAndSnapshot(SNAPSHOT); + + when(worker.isRunning(CONN1)).thenReturn(true); + when(worker.getPlugins()).thenReturn(plugins); + + SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, CONN1_CONFIG); + + List> changedTaskConfigs = new ArrayList<>(TASK_CONFIGS); + changedTaskConfigs.add(TASK_CONFIG); + when(worker.connectorTaskConfigs(CONN1, sinkConnectorConfig)).thenReturn(changedTaskConfigs); + + when(restClient.httpRequest(any(), eq("POST"), any(), any(), any(), any(), any())) + .thenThrow(new ConnectException("Request to leader to reconfigure connector tasks failed")) + .thenThrow(new ConnectException("Request to leader to reconfigure connector tasks failed")) + .thenReturn(null); + + expectAndVerifyTaskReconfigurationRetries(); + } + + private void expectAndVerifyTaskReconfigurationRetries() { + // initial tick + herder.tick(); + herder.requestTaskReconfiguration(CONN1); + // process the task reconfiguration request in this tick + herder.tick(); + // advance the time by 250ms so that the task reconfiguration request with initial retry backoff is processed + time.sleep(250); + herder.tick(); + // advance the time by 500ms so that the task reconfiguration request with double the initial retry backoff is processed + time.sleep(500); + herder.tick(); + + // 1. end of initial tick when no request has been added to the herder queue yet + // 2. the third task reconfiguration request is expected to pass; so expect no more retries (a Long.MAX_VALUE poll + // timeout indicates that there is no herder request currently in the queue) + verify(member, times(2)).poll(eq(Long.MAX_VALUE)); + + // task reconfiguration herder request with initial retry backoff + verify(member).poll(eq(250L)); + + // task reconfiguration herder request with double the initial retry backoff + verify(member).poll(eq(500L)); + + verifyNoMoreInteractions(member, worker, restClient); + } + + @Test + public void processRestartRequestsFailureSuppression() { + doNothing().when(member).wakeup(); + + final String connectorName = "foo"; + RestartRequest restartRequest = new RestartRequest(connectorName, false, false); + doThrow(new RuntimeException()).when(herder).buildRestartPlan(restartRequest); + + configUpdateListener.onRestartRequest(restartRequest); + assertEquals(1, herder.pendingRestartRequests.size()); + herder.processRestartRequests(); + assertTrue(herder.pendingRestartRequests.isEmpty()); + + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); + } + + @Test + public void processRestartRequestsDequeue() { + doNothing().when(member).wakeup(); + doReturn(Optional.empty()).when(herder).buildRestartPlan(any(RestartRequest.class)); + + RestartRequest restartRequest = new RestartRequest("foo", false, false); + configUpdateListener.onRestartRequest(restartRequest); + restartRequest = new RestartRequest("bar", false, false); + configUpdateListener.onRestartRequest(restartRequest); + assertEquals(2, herder.pendingRestartRequests.size()); + herder.processRestartRequests(); + assertTrue(herder.pendingRestartRequests.isEmpty()); + } + + @Test + public void preserveHighestImpactRestartRequest() { + doNothing().when(member).wakeup(); + + final String connectorName = "foo"; + RestartRequest restartRequest = new RestartRequest(connectorName, false, false); + configUpdateListener.onRestartRequest(restartRequest); + + // Will overwrite as this is higher impact + restartRequest = new RestartRequest(connectorName, false, true); + configUpdateListener.onRestartRequest(restartRequest); + assertEquals(1, herder.pendingRestartRequests.size()); + assertFalse(herder.pendingRestartRequests.get(connectorName).onlyFailed()); + assertTrue(herder.pendingRestartRequests.get(connectorName).includeTasks()); + + // Will be ignored as the existing request has higher impact + restartRequest = new RestartRequest(connectorName, true, false); + configUpdateListener.onRestartRequest(restartRequest); + assertEquals(1, herder.pendingRestartRequests.size()); + // Compare against existing request + assertFalse(herder.pendingRestartRequests.get(connectorName).onlyFailed()); + assertTrue(herder.pendingRestartRequests.get(connectorName).includeTasks()); + + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); + } + + @Test + public void testExactlyOnceSourceSupportValidation() { + herder = exactlyOnceHerder(); + Map config = new HashMap<>(); + config.put(SourceConnectorConfig.EXACTLY_ONCE_SUPPORT_CONFIG, REQUIRED.toString()); + + SourceConnector connectorMock = mock(SourceConnector.class); + when(connectorMock.exactlyOnceSupport(eq(config))).thenReturn(ExactlyOnceSupport.SUPPORTED); + + Map validatedConfigs = herder.validateSourceConnectorConfig( + connectorMock, SourceConnectorConfig.configDef(), config); + + List errors = validatedConfigs.get(SourceConnectorConfig.EXACTLY_ONCE_SUPPORT_CONFIG).errorMessages(); + assertEquals(Collections.emptyList(), errors); + } + + @Test + public void testExactlyOnceSourceSupportValidationOnUnsupportedConnector() { + herder = exactlyOnceHerder(); + Map config = new HashMap<>(); + config.put(SourceConnectorConfig.EXACTLY_ONCE_SUPPORT_CONFIG, REQUIRED.toString()); + + SourceConnector connectorMock = mock(SourceConnector.class); + when(connectorMock.exactlyOnceSupport(eq(config))).thenReturn(ExactlyOnceSupport.UNSUPPORTED); + + Map validatedConfigs = herder.validateSourceConnectorConfig( + connectorMock, SourceConnectorConfig.configDef(), config); + + List errors = validatedConfigs.get(SourceConnectorConfig.EXACTLY_ONCE_SUPPORT_CONFIG).errorMessages(); + assertEquals( + Collections.singletonList("The connector does not support exactly-once semantics with the provided configuration."), + errors); + } + + @Test + public void testExactlyOnceSourceSupportValidationOnUnknownConnector() { + herder = exactlyOnceHerder(); + Map config = new HashMap<>(); + config.put(SourceConnectorConfig.EXACTLY_ONCE_SUPPORT_CONFIG, REQUIRED.toString()); + + SourceConnector connectorMock = mock(SourceConnector.class); + when(connectorMock.exactlyOnceSupport(eq(config))).thenReturn(null); + + Map validatedConfigs = herder.validateSourceConnectorConfig( + connectorMock, SourceConnectorConfig.configDef(), config); + + List errors = validatedConfigs.get(SourceConnectorConfig.EXACTLY_ONCE_SUPPORT_CONFIG).errorMessages(); + assertFalse(errors.isEmpty()); + assertTrue( + "Error message did not contain expected text: " + errors.get(0), + errors.get(0).contains("The connector does not implement the API required for preflight validation of exactly-once source support.")); + assertEquals(1, errors.size()); + } + + @Test + public void testExactlyOnceSourceSupportValidationHandlesConnectorErrorsGracefully() { + herder = exactlyOnceHerder(); + Map config = new HashMap<>(); + config.put(SourceConnectorConfig.EXACTLY_ONCE_SUPPORT_CONFIG, REQUIRED.toString()); + + SourceConnector connectorMock = mock(SourceConnector.class); + String errorMessage = "time to add a new unit test :)"; + when(connectorMock.exactlyOnceSupport(eq(config))).thenThrow(new NullPointerException(errorMessage)); + + Map validatedConfigs = herder.validateSourceConnectorConfig( + connectorMock, SourceConnectorConfig.configDef(), config); - PowerMock.verifyAll(); + List errors = validatedConfigs.get(SourceConnectorConfig.EXACTLY_ONCE_SUPPORT_CONFIG).errorMessages(); + assertFalse(errors.isEmpty()); + assertTrue( + "Error message did not contain expected text: " + errors.get(0), + errors.get(0).contains(errorMessage)); + assertEquals(1, errors.size()); } @Test - public void testTaskReconfigurationRetriesWithConnectorTaskConfigsException() { - EasyMock.expect(member.memberId()).andStubReturn("leader"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true); - expectConfigRefreshAndSnapshot(SNAPSHOT); + public void testExactlyOnceSourceSupportValidationWhenExactlyOnceNotEnabledOnWorker() { + Map config = new HashMap<>(); + config.put(SourceConnectorConfig.EXACTLY_ONCE_SUPPORT_CONFIG, REQUIRED.toString()); - // end of initial tick - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + SourceConnector connectorMock = mock(SourceConnector.class); + when(connectorMock.exactlyOnceSupport(eq(config))).thenReturn(ExactlyOnceSupport.SUPPORTED); - member.wakeup(); - PowerMock.expectLastCall().anyTimes(); + Map validatedConfigs = herder.validateSourceConnectorConfig( + connectorMock, SourceConnectorConfig.configDef(), config); - member.ensureActive(); - PowerMock.expectLastCall().anyTimes(); + List errors = validatedConfigs.get(SourceConnectorConfig.EXACTLY_ONCE_SUPPORT_CONFIG).errorMessages(); + assertEquals( + Collections.singletonList("This worker does not have exactly-once source support enabled."), + errors); + } - EasyMock.expect(worker.isRunning(CONN1)).andReturn(true).anyTimes(); - EasyMock.expect(worker.getPlugins()).andReturn(plugins).anyTimes(); + @Test + public void testExactlyOnceSourceSupportValidationHandlesInvalidValuesGracefully() { + herder = exactlyOnceHerder(); + Map config = new HashMap<>(); + config.put(SourceConnectorConfig.EXACTLY_ONCE_SUPPORT_CONFIG, "invalid"); - SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, CONN1_CONFIG); - EasyMock.expect(worker.connectorTaskConfigs(CONN1, sinkConnectorConfig)) - .andThrow(new ConnectException("Failed to generate task configs")).times(2); + SourceConnector connectorMock = mock(SourceConnector.class); - EasyMock.expect(worker.connectorTaskConfigs(CONN1, sinkConnectorConfig)).andReturn(TASK_CONFIGS); + Map validatedConfigs = herder.validateSourceConnectorConfig( + connectorMock, SourceConnectorConfig.configDef(), config); - expectAndVerifyTaskReconfigurationRetries(); + List errors = validatedConfigs.get(SourceConnectorConfig.EXACTLY_ONCE_SUPPORT_CONFIG).errorMessages(); + assertFalse(errors.isEmpty()); + assertTrue( + "Error message did not contain expected text: " + errors.get(0), + errors.get(0).contains("String must be one of (case insensitive): ")); + assertEquals(1, errors.size()); } @Test - public void testTaskReconfigurationRetriesWithLeaderRequestForwardingException() { - herder = PowerMock.createPartialMock(DistributedHerder.class, - new String[]{"connectorType", "updateDeletedConnectorStatus", "updateDeletedTaskStatus", "validateConnectorConfig", "buildRestartPlan", "recordRestarting"}, - new DistributedConfig(HERDER_CONFIG), worker, WORKER_ID, KAFKA_CLUSTER_ID, - statusBackingStore, configBackingStore, member, MEMBER_URL, restClient, metrics, time, noneConnectorClientConfigOverridePolicy, - Collections.emptyList(), new MockSynchronousExecutor(), new AutoCloseable[]{}); + public void testConnectorTransactionBoundaryValidation() { + herder = exactlyOnceHerder(); + Map config = new HashMap<>(); + config.put(SourceConnectorConfig.TRANSACTION_BOUNDARY_CONFIG, CONNECTOR.toString()); - rebalanceListener = herder.new RebalanceListener(time); + SourceConnector connectorMock = mock(SourceConnector.class); + when(connectorMock.canDefineTransactionBoundaries(eq(config))) + .thenReturn(ConnectorTransactionBoundaries.SUPPORTED); - EasyMock.expect(member.memberId()).andStubReturn("member"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - expectRebalance(1, Collections.emptyList(), Collections.emptyList(), false); - expectConfigRefreshAndSnapshot(SNAPSHOT); + Map validatedConfigs = herder.validateSourceConnectorConfig( + connectorMock, SourceConnectorConfig.configDef(), config); - // end of initial tick - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + List errors = validatedConfigs.get(SourceConnectorConfig.TRANSACTION_BOUNDARY_CONFIG).errorMessages(); + assertEquals(Collections.emptyList(), errors); + } - member.wakeup(); - PowerMock.expectLastCall().anyTimes(); + @Test + public void testConnectorTransactionBoundaryValidationOnUnsupportedConnector() { + herder = exactlyOnceHerder(); + Map config = new HashMap<>(); + config.put(SourceConnectorConfig.TRANSACTION_BOUNDARY_CONFIG, CONNECTOR.toString()); - member.ensureActive(); - PowerMock.expectLastCall().anyTimes(); + SourceConnector connectorMock = mock(SourceConnector.class); + when(connectorMock.canDefineTransactionBoundaries(eq(config))) + .thenReturn(ConnectorTransactionBoundaries.UNSUPPORTED); - EasyMock.expect(worker.isRunning(CONN1)).andReturn(true).anyTimes(); - EasyMock.expect(worker.getPlugins()).andReturn(plugins).anyTimes(); + Map validatedConfigs = herder.validateSourceConnectorConfig( + connectorMock, SourceConnectorConfig.configDef(), config); - SinkConnectorConfig sinkConnectorConfig = new SinkConnectorConfig(plugins, CONN1_CONFIG); + List errors = validatedConfigs.get(SourceConnectorConfig.TRANSACTION_BOUNDARY_CONFIG).errorMessages(); + assertFalse(errors.isEmpty()); + assertTrue( + "Error message did not contain expected text: " + errors.get(0), + errors.get(0).contains("The connector does not support connector-defined transaction boundaries with the given configuration.")); + assertEquals(1, errors.size()); + } - List> changedTaskConfigs = new ArrayList<>(TASK_CONFIGS); - changedTaskConfigs.add(TASK_CONFIG); - EasyMock.expect(worker.connectorTaskConfigs(CONN1, sinkConnectorConfig)).andReturn(changedTaskConfigs).anyTimes(); + @Test + public void testConnectorTransactionBoundaryValidationHandlesConnectorErrorsGracefully() { + herder = exactlyOnceHerder(); + Map config = new HashMap<>(); + config.put(SourceConnectorConfig.TRANSACTION_BOUNDARY_CONFIG, CONNECTOR.toString()); - EasyMock.expect(restClient.httpRequest( - EasyMock.anyString(), EasyMock.eq("POST"), EasyMock.anyObject(HttpHeaders.class), - EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(SecretKey.class), EasyMock.anyString()) - ).andThrow(new ConnectException("Request to leader to reconfigure connector tasks failed")).times(2); + SourceConnector connectorMock = mock(SourceConnector.class); + String errorMessage = "Wait I thought we tested for this?"; + when(connectorMock.canDefineTransactionBoundaries(eq(config))).thenThrow(new ConnectException(errorMessage)); - EasyMock.expect(restClient.httpRequest( - EasyMock.anyString(), EasyMock.eq("POST"), EasyMock.anyObject(HttpHeaders.class), - EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(SecretKey.class), EasyMock.anyString()) - ).andReturn(null); + Map validatedConfigs = herder.validateSourceConnectorConfig( + connectorMock, SourceConnectorConfig.configDef(), config); - expectAndVerifyTaskReconfigurationRetries(); + List errors = validatedConfigs.get(SourceConnectorConfig.TRANSACTION_BOUNDARY_CONFIG).errorMessages(); + assertFalse(errors.isEmpty()); + assertTrue( + "Error message did not contain expected text: " + errors.get(0), + errors.get(0).contains(errorMessage)); + assertEquals(1, errors.size()); } - private void expectAndVerifyTaskReconfigurationRetries() { - // task reconfiguration herder request with initial retry backoff - member.poll(EasyMock.eq(250L)); - PowerMock.expectLastCall(); - - // task reconfiguration herder request with double the initial retry backoff - member.poll(EasyMock.eq(500L)); - PowerMock.expectLastCall(); - - // the third task reconfiguration request is expected to pass; so expect no more retries (a Long.MAX_VALUE poll - // timeout indicates that there is no herder request currently in the queue) - member.poll(EasyMock.eq(Long.MAX_VALUE)); - PowerMock.expectLastCall(); + @Test + public void testConnectorTransactionBoundaryValidationHandlesInvalidValuesGracefully() { + herder = exactlyOnceHerder(); + Map config = new HashMap<>(); + config.put(SourceConnectorConfig.TRANSACTION_BOUNDARY_CONFIG, "CONNECTOR.toString()"); - PowerMock.replayAll(); + SourceConnector connectorMock = mock(SourceConnector.class); - // initial tick - herder.tick(); - herder.requestTaskReconfiguration(CONN1); - // process the task reconfiguration request in this tick - herder.tick(); - // advance the time by 250ms so that the task reconfiguration request with initial retry backoff is processed - time.sleep(250); - herder.tick(); - // advance the time by 500ms so that the task reconfiguration request with double the initial retry backoff is processed - time.sleep(500); - herder.tick(); + Map validatedConfigs = herder.validateSourceConnectorConfig( + connectorMock, SourceConnectorConfig.configDef(), config); - PowerMock.verifyAll(); + List errors = validatedConfigs.get(SourceConnectorConfig.TRANSACTION_BOUNDARY_CONFIG).errorMessages(); + assertFalse(errors.isEmpty()); + assertTrue( + "Error message did not contain expected text: " + errors.get(0), + errors.get(0).contains("String must be one of (case insensitive): ")); + assertEquals(1, errors.size()); } @Test public void testConnectorOffsets() throws Exception { - EasyMock.expect(member.memberId()).andStubReturn("leader"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); + when(member.memberId()).thenReturn("leader"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true); expectConfigRefreshAndSnapshot(SNAPSHOT); - expectAnyTicks(); - member.wakeup(); - PowerMock.expectLastCall(); + when(statusBackingStore.connectors()).thenReturn(Collections.emptySet()); + doNothing().when(member).poll(anyLong()); + + herder.tick(); - EasyMock.expect(configBackingStore.snapshot()).andReturn(SNAPSHOT); + when(configBackingStore.snapshot()).thenReturn(SNAPSHOT); ConnectorOffsets offsets = new ConnectorOffsets(Collections.singletonList(new ConnectorOffset( Collections.singletonMap("partitionKey", "partitionValue"), Collections.singletonMap("offsetKey", "offsetValue")))); - Capture> callbackCapture = newCapture(); - worker.connectorOffsets(EasyMock.eq(CONN1), EasyMock.eq(CONN1_CONFIG), capture(callbackCapture)); - PowerMock.expectLastCall().andAnswer(() -> { + + ArgumentCaptor> callbackCapture = ArgumentCaptor.forClass(Callback.class); + doAnswer(invocation -> { callbackCapture.getValue().onCompletion(null, offsets); return null; - }); + }).when(worker).connectorOffsets(eq(CONN1), eq(CONN1_CONFIG), callbackCapture.capture()); - PowerMock.replayAll(); - - herder.tick(); FutureCallback cb = new FutureCallback<>(); herder.connectorOffsets(CONN1, cb); herder.tick(); assertEquals(offsets, cb.get(1000, TimeUnit.MILLISECONDS)); - PowerMock.verifyAll(); + verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); } @Test public void testModifyConnectorOffsetsUnknownConnector() throws Exception { // Get the initial assignment - EasyMock.expect(member.memberId()).andStubReturn("leader"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); + when(member.memberId()).thenReturn("leader"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true); expectConfigRefreshAndSnapshot(SNAPSHOT); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - // Now handle the connector offsets modification request - member.wakeup(); - PowerMock.expectLastCall(); - member.ensureActive(); - PowerMock.expectLastCall(); - expectConfigRefreshAndSnapshot(SNAPSHOT); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - PowerMock.replayAll(); herder.tick(); + + // Now handle the connector offsets modification request FutureCallback callback = new FutureCallback<>(); herder.modifyConnectorOffsets("connector-does-not-exist", new HashMap<>(), callback); herder.tick(); ExecutionException e = assertThrows(ExecutionException.class, () -> callback.get(1000L, TimeUnit.MILLISECONDS)); assertTrue(e.getCause() instanceof NotFoundException); - - PowerMock.verifyAll(); } @Test public void testModifyOffsetsConnectorNotInStoppedState() throws Exception { // Get the initial assignment - EasyMock.expect(member.memberId()).andStubReturn("leader"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); + when(member.memberId()).thenReturn("leader"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true); expectConfigRefreshAndSnapshot(SNAPSHOT); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - // Now handle the connector offsets modification request - member.wakeup(); - PowerMock.expectLastCall(); - member.ensureActive(); - PowerMock.expectLastCall(); - expectConfigRefreshAndSnapshot(SNAPSHOT); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - PowerMock.replayAll(); herder.tick(); + + // Now handle the connector offsets modification request FutureCallback callback = new FutureCallback<>(); herder.modifyConnectorOffsets(CONN1, null, callback); herder.tick(); ExecutionException e = assertThrows(ExecutionException.class, () -> callback.get(1000L, TimeUnit.MILLISECONDS)); assertTrue(e.getCause() instanceof BadRequestException); - - PowerMock.verifyAll(); } @Test public void testModifyOffsetsNotLeader() throws Exception { // Get the initial assignment - EasyMock.expect(member.memberId()).andStubReturn("member"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); + when(member.memberId()).thenReturn("member"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); expectRebalance(1, Collections.emptyList(), Collections.emptyList(), false); expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - // Now handle the connector offsets modification request - member.wakeup(); - PowerMock.expectLastCall(); - member.ensureActive(); - PowerMock.expectLastCall(); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - - PowerMock.replayAll(); herder.tick(); + + // Now handle the connector offsets modification request FutureCallback callback = new FutureCallback<>(); herder.modifyConnectorOffsets(CONN1, new HashMap<>(), callback); herder.tick(); ExecutionException e = assertThrows(ExecutionException.class, () -> callback.get(1000L, TimeUnit.MILLISECONDS)); assertTrue(e.getCause() instanceof NotLeaderException); - - PowerMock.verifyAll(); } @Test public void testModifyOffsetsSinkConnector() throws Exception { - EasyMock.reset(herder); - EasyMock.expect(herder.connectorType(EasyMock.anyObject())).andReturn(ConnectorType.SINK).anyTimes(); - PowerMock.expectPrivate(herder, "updateDeletedConnectorStatus").andVoid().anyTimes(); - PowerMock.expectPrivate(herder, "updateDeletedTaskStatus").andVoid().anyTimes(); - + when(herder.connectorType(any())).thenReturn(ConnectorType.SINK); // Get the initial assignment - EasyMock.expect(member.memberId()).andStubReturn("leader"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); + when(member.memberId()).thenReturn("leader"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true); expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + + herder.tick(); // Now handle the alter connector offsets request Map, Map> offsets = Collections.singletonMap( Collections.singletonMap("partitionKey", "partitionValue"), Collections.singletonMap("offsetKey", "offsetValue")); - member.wakeup(); - PowerMock.expectLastCall(); - member.ensureActive(); - PowerMock.expectLastCall(); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1); - Capture> workerCallbackCapture = Capture.newInstance(); - worker.modifyConnectorOffsets(EasyMock.eq(CONN1), EasyMock.eq(CONN1_CONFIG), EasyMock.eq(offsets), capture(workerCallbackCapture)); + + ArgumentCaptor> workerCallbackCapture = ArgumentCaptor.forClass(Callback.class); Message msg = new Message("The offsets for this connector have been altered successfully"); - EasyMock.expectLastCall().andAnswer(() -> { + doAnswer(invocation -> { workerCallbackCapture.getValue().onCompletion(null, msg); return null; - }); + }).when(worker).modifyConnectorOffsets(eq(CONN1), eq(CONN1_CONFIG), eq(offsets), workerCallbackCapture.capture()); - PowerMock.replayAll(); - - herder.tick(); FutureCallback callback = new FutureCallback<>(); herder.alterConnectorOffsets(CONN1, offsets, callback); herder.tick(); assertEquals(msg, callback.get(1000L, TimeUnit.MILLISECONDS)); assertEquals("The offsets for this connector have been altered successfully", msg.message()); - - PowerMock.verifyAll(); } @Test public void testModifyOffsetsSourceConnectorExactlyOnceDisabled() throws Exception { // Get the initial assignment - EasyMock.expect(member.memberId()).andStubReturn("leader"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); + when(member.memberId()).thenReturn("leader"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true); expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); + herder.tick(); // Now handle the reset connector offsets request - member.wakeup(); - PowerMock.expectLastCall(); - member.ensureActive(); - PowerMock.expectLastCall(); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall(); - expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1); - Capture> workerCallbackCapture = Capture.newInstance(); - worker.modifyConnectorOffsets(EasyMock.eq(CONN1), EasyMock.eq(CONN1_CONFIG), isNull(), capture(workerCallbackCapture)); + ArgumentCaptor> workerCallbackCapture = ArgumentCaptor.forClass(Callback.class); Message msg = new Message("The offsets for this connector have been reset successfully"); - EasyMock.expectLastCall().andAnswer(() -> { + doAnswer(invocation -> { workerCallbackCapture.getValue().onCompletion(null, msg); return null; - }); + }).when(worker).modifyConnectorOffsets(eq(CONN1), eq(CONN1_CONFIG), isNull(), workerCallbackCapture.capture()); - PowerMock.replayAll(); - - herder.tick(); FutureCallback callback = new FutureCallback<>(); herder.resetConnectorOffsets(CONN1, callback); herder.tick(); assertEquals(msg, callback.get(1000L, TimeUnit.MILLISECONDS)); assertEquals("The offsets for this connector have been reset successfully", msg.message()); - - PowerMock.verifyAll(); } @Test @@ -4327,86 +3549,64 @@ public void testModifyOffsetsSourceConnectorExactlyOnceEnabled() throws Exceptio // Setup herder with exactly-once support for source connectors enabled herder = exactlyOnceHerder(); rebalanceListener = herder.new RebalanceListener(time); - PowerMock.expectPrivate(herder, "updateDeletedConnectorStatus").andVoid().anyTimes(); - PowerMock.expectPrivate(herder, "updateDeletedTaskStatus").andVoid().anyTimes(); - // Get the initial assignment - EasyMock.expect(member.memberId()).andStubReturn("leader"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); + when(member.memberId()).thenReturn("leader"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true); expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall().anyTimes(); + doNothing().when(member).poll(anyLong()); + + herder.tick(); // Now handle the alter connector offsets request - Map, Map> offsets = Collections.singletonMap( - Collections.singletonMap("partitionKey", "partitionValue"), - Collections.singletonMap("offsetKey", "offsetValue")); - member.wakeup(); - PowerMock.expectLastCall().anyTimes(); - member.ensureActive(); - PowerMock.expectLastCall().anyTimes(); - expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1); - expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1); - EasyMock.expect(herder.connectorType(EasyMock.anyObject())).andReturn(ConnectorType.SOURCE).anyTimes(); + doNothing().when(member).ensureActive(); + when(herder.connectorType(any())).thenReturn(ConnectorType.SOURCE); // Expect a round of zombie fencing to occur - expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1); - KafkaFuture workerFencingFuture = EasyMock.mock(KafkaFuture.class); - KafkaFuture herderFencingFuture = EasyMock.mock(KafkaFuture.class); - EasyMock.expect(worker.fenceZombies(CONN1, SNAPSHOT_STOPPED_CONN1.taskCountRecord(CONN1), CONN1_CONFIG)).andReturn(workerFencingFuture); - EasyMock.expect(workerFencingFuture.thenApply(EasyMock.>anyObject())).andReturn(herderFencingFuture); - - // Two fencing callbacks are added - one is in ZombieFencing::start itself to remove the connector from the active - // fencing list. The other is the callback passed from DistributedHerder::modifyConnectorOffsets in order to - // queue up the actual alter offsets request if the zombie fencing succeeds. - for (int i = 0; i < 2; i++) { - Capture> herderFencingCallback = EasyMock.newCapture(); - EasyMock.expect(herderFencingFuture.whenComplete(EasyMock.capture(herderFencingCallback))).andAnswer(() -> { - herderFencingCallback.getValue().accept(null, null); - return null; - }); - } - - Capture> workerCallbackCapture = Capture.newInstance(); - Message msg = new Message("The offsets for this connector have been altered successfully"); - worker.modifyConnectorOffsets(EasyMock.eq(CONN1), EasyMock.eq(CONN1_CONFIG), EasyMock.eq(offsets), capture(workerCallbackCapture)); - EasyMock.expectLastCall().andAnswer(() -> { - workerCallbackCapture.getValue().onCompletion(null, msg); + KafkaFuture workerFencingFuture = mock(KafkaFuture.class); + KafkaFuture herderFencingFuture = mock(KafkaFuture.class); + when(worker.fenceZombies(CONN1, SNAPSHOT_STOPPED_CONN1.taskCountRecord(CONN1), CONN1_CONFIG)).thenReturn(workerFencingFuture); + when(workerFencingFuture.thenApply(any(KafkaFuture.BaseFunction.class))).thenReturn(herderFencingFuture); + + ArgumentCaptor> herderFencingCallback = ArgumentCaptor.forClass(KafkaFuture.BiConsumer.class); + when(herderFencingFuture.whenComplete(herderFencingCallback.capture())).thenAnswer(invocation -> { + herderFencingCallback.getValue().accept(null, null); return null; }); - // Handle the second alter connector offsets request. No zombie fencing request to the worker is expected now since we - // already did a round of zombie fencing last time and no new tasks came up in the meanwhile. The config snapshot is - // refreshed once at the beginning of the DistributedHerder::modifyConnectorOffsets method, once before checking - // whether zombie fencing is required, and once before actually proceeding to alter connector offsets. - expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1_FENCED); - expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1_FENCED); - expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1_FENCED); - Capture> workerCallbackCapture2 = Capture.newInstance(); - worker.modifyConnectorOffsets(EasyMock.eq(CONN1), EasyMock.eq(CONN1_CONFIG), EasyMock.eq(offsets), capture(workerCallbackCapture2)); - EasyMock.expectLastCall().andAnswer(() -> { - workerCallbackCapture2.getValue().onCompletion(null, msg); - return null; - }); + ArgumentCaptor> workerCallbackCapture = ArgumentCaptor.forClass(Callback.class); + Message msg = new Message("The offsets for this connector have been altered successfully"); - PowerMock.replayAll(workerFencingFuture, herderFencingFuture); + Map, Map> offsets = Collections.singletonMap( + Collections.singletonMap("partitionKey", "partitionValue"), + Collections.singletonMap("offsetKey", "offsetValue")); + doAnswer(invocation -> { + workerCallbackCapture.getValue().onCompletion(null, msg); + return null; + }).when(worker).modifyConnectorOffsets(eq(CONN1), eq(CONN1_CONFIG), eq(offsets), workerCallbackCapture.capture()); - herder.tick(); FutureCallback callback = new FutureCallback<>(); herder.alterConnectorOffsets(CONN1, offsets, callback); - // Process the zombie fencing request - herder.tick(); - // Process the alter offsets request + // Process the zombie fencing request that is queued up first followed by the actual alter offsets request herder.tick(); assertEquals(msg, callback.get(1000L, TimeUnit.MILLISECONDS)); + // Handle the second alter connector offsets request + expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1_FENCED); FutureCallback callback2 = new FutureCallback<>(); herder.alterConnectorOffsets(CONN1, offsets, callback2); herder.tick(); - assertEquals(msg, callback.get(1000L, TimeUnit.MILLISECONDS)); + assertEquals(msg, callback2.get(1000L, TimeUnit.MILLISECONDS)); + + // Two fencing callbacks are added - one is in ZombieFencing::start itself to remove the connector from the active + // fencing list. The other is the callback passed from DistributedHerder::modifyConnectorOffsets in order to + // queue up the actual alter offsets request if the zombie fencing succeeds. + verify(herderFencingFuture, times(2)).whenComplete(any()); - PowerMock.verifyAll(); + // No zombie fencing request to the worker is expected in the second alter connector offsets request since we already + // did a round of zombie fencing the first time and no new tasks came up in the meanwhile. + verify(worker, times(1)).fenceZombies(eq(CONN1), eq(SNAPSHOT_STOPPED_CONN1.taskCountRecord(CONN1)), eq(CONN1_CONFIG)); + verifyNoMoreInteractions(workerFencingFuture, herderFencingFuture, member, worker); } @Test @@ -4414,58 +3614,46 @@ public void testModifyOffsetsSourceConnectorExactlyOnceEnabledZombieFencingFailu // Setup herder with exactly-once support for source connectors enabled herder = exactlyOnceHerder(); rebalanceListener = herder.new RebalanceListener(time); - PowerMock.expectPrivate(herder, "updateDeletedConnectorStatus").andVoid().anyTimes(); - PowerMock.expectPrivate(herder, "updateDeletedTaskStatus").andVoid().anyTimes(); // Get the initial assignment - EasyMock.expect(member.memberId()).andStubReturn("leader"); - EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); + when(member.memberId()).thenReturn("leader"); + when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); expectRebalance(1, Collections.emptyList(), Collections.emptyList(), true); expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall().anyTimes(); + doNothing().when(member).poll(anyLong()); + + herder.tick(); // Now handle the reset connector offsets request - member.wakeup(); - PowerMock.expectLastCall().anyTimes(); - member.ensureActive(); - PowerMock.expectLastCall().anyTimes(); - expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1); - EasyMock.expect(herder.connectorType(EasyMock.anyObject())).andReturn(ConnectorType.SOURCE).anyTimes(); + doNothing().when(member).ensureActive(); + when(herder.connectorType(any())).thenReturn(ConnectorType.SOURCE); // Expect a round of zombie fencing to occur - expectConfigRefreshAndSnapshot(SNAPSHOT_STOPPED_CONN1); - KafkaFuture workerFencingFuture = EasyMock.mock(KafkaFuture.class); - KafkaFuture herderFencingFuture = EasyMock.mock(KafkaFuture.class); - EasyMock.expect(worker.fenceZombies(CONN1, SNAPSHOT_STOPPED_CONN1.taskCountRecord(CONN1), CONN1_CONFIG)).andReturn(workerFencingFuture); - EasyMock.expect(workerFencingFuture.thenApply(EasyMock.>anyObject())).andReturn(herderFencingFuture); - - // Two fencing callbacks are added - one is in ZombieFencing::start itself to remove the connector from the active - // fencing list. The other is the callback passed from DistributedHerder::modifyConnectorOffsets in order to - // queue up the actual reset offsets request if the zombie fencing succeeds. - for (int i = 0; i < 2; i++) { - Capture> herderFencingCallback = EasyMock.newCapture(); - EasyMock.expect(herderFencingFuture.whenComplete(EasyMock.capture(herderFencingCallback))).andAnswer(() -> { - herderFencingCallback.getValue().accept(null, new ConnectException("Failed to perform zombie fencing")); - return null; - }); - } - - PowerMock.replayAll(workerFencingFuture, herderFencingFuture); + KafkaFuture workerFencingFuture = mock(KafkaFuture.class); + KafkaFuture herderFencingFuture = mock(KafkaFuture.class); + when(worker.fenceZombies(CONN1, SNAPSHOT_STOPPED_CONN1.taskCountRecord(CONN1), CONN1_CONFIG)).thenReturn(workerFencingFuture); + when(workerFencingFuture.thenApply(any(KafkaFuture.BaseFunction.class))).thenReturn(herderFencingFuture); + + ArgumentCaptor> herderFencingCallback = ArgumentCaptor.forClass(KafkaFuture.BiConsumer.class); + when(herderFencingFuture.whenComplete(herderFencingCallback.capture())).thenAnswer(invocation -> { + herderFencingCallback.getValue().accept(null, new ConnectException("Failed to perform zombie fencing")); + return null; + }); - herder.tick(); FutureCallback callback = new FutureCallback<>(); herder.resetConnectorOffsets(CONN1, callback); - // Process the zombie fencing request - herder.tick(); - // Process the reset offsets request + // Process the zombie fencing request that is queued up first herder.tick(); ExecutionException e = assertThrows(ExecutionException.class, () -> callback.get(1000L, TimeUnit.MILLISECONDS)); assertEquals(ConnectException.class, e.getCause().getClass()); assertEquals("Failed to perform zombie fencing for source connector prior to modifying offsets", e.getCause().getMessage()); - PowerMock.verifyAll(); + // Two fencing callbacks are added - one is in ZombieFencing::start itself to remove the connector from the active + // fencing list. The other is the callback passed from DistributedHerder::modifyConnectorOffsets in order to + // queue up the actual reset offsets request if the zombie fencing succeeds. + verify(herderFencingFuture, times(2)).whenComplete(any()); + verifyNoMoreInteractions(workerFencingFuture, herderFencingFuture, member, worker); } private void expectRebalance(final long offset, @@ -4521,8 +3709,7 @@ private void expectRebalance(final Collection revokedConnectors, final List assignedTasks, int delay, boolean isLeader) { - member.ensureActive(); - PowerMock.expectLastCall().andAnswer(() -> { + doAnswer(invocation -> { ExtendedAssignment assignment; if (!revokedConnectors.isEmpty() || !revokedTasks.isEmpty()) { rebalanceListener.onRevoked(leader, revokedConnectors, revokedTasks); @@ -4542,32 +3729,27 @@ private void expectRebalance(final Collection revokedConnectors, rebalanceListener.onAssigned(assignment, 3); time.sleep(100L); return null; - }); + }).when(member).ensureActive(); if (isLeader) { - configBackingStore.claimWritePrivileges(); - EasyMock.expectLastCall(); + doNothing().when(configBackingStore).claimWritePrivileges(); } if (!revokedConnectors.isEmpty()) { for (String connector : revokedConnectors) { - worker.stopAndAwaitConnector(connector); - PowerMock.expectLastCall(); + doNothing().when(worker).stopAndAwaitConnector(connector); } } if (!revokedTasks.isEmpty()) { - worker.stopAndAwaitTask(EasyMock.anyObject(ConnectorTaskId.class)); - PowerMock.expectLastCall(); + doNothing().when(worker).stopAndAwaitTask(any(ConnectorTaskId.class)); } if (!revokedConnectors.isEmpty()) { - statusBackingStore.flush(); - PowerMock.expectLastCall(); + doNothing().when(statusBackingStore).flush(); } - member.wakeup(); - PowerMock.expectLastCall(); + doNothing().when(member).wakeup(); } private ClusterConfigState exactlyOnceSnapshot( @@ -4617,40 +3799,25 @@ private ClusterConfigState exactlyOnceSnapshot( Collections.emptySet()); } - private void expectExecuteTaskReconfiguration(boolean running, ConnectorConfig connectorConfig, IAnswer>> answer) { - EasyMock.expect(worker.isRunning(CONN1)).andReturn(running); + private void expectExecuteTaskReconfiguration(boolean running, ConnectorConfig connectorConfig, Answer>> answer) { + when(worker.isRunning(CONN1)).thenReturn(running); if (running) { - EasyMock.expect(worker.getPlugins()).andReturn(plugins); - EasyMock.expect(worker.connectorTaskConfigs(CONN1, connectorConfig)).andAnswer(answer); + when(worker.getPlugins()).thenReturn(plugins); + when(worker.connectorTaskConfigs(CONN1, connectorConfig)).thenAnswer(answer); } } - private void expectAnyTicks() { - member.ensureActive(); - EasyMock.expectLastCall().anyTimes(); - member.poll(EasyMock.anyInt()); - PowerMock.expectLastCall().anyTimes(); - } - private SessionKey expectNewSessionKey() { - SecretKey secretKey = EasyMock.niceMock(SecretKey.class); - EasyMock.expect(secretKey.getAlgorithm()).andReturn(INTER_WORKER_KEY_GENERATION_ALGORITHM_DEFAULT).anyTimes(); - EasyMock.expect(secretKey.getEncoded()).andReturn(new byte[32]).anyTimes(); - SessionKey sessionKey = new SessionKey(secretKey, time.milliseconds() + TimeUnit.DAYS.toMillis(1)); - configBackingStore.putSessionKey(anyObject(SessionKey.class)); - EasyMock.expectLastCall().andAnswer(() -> { - configUpdateListener.onSessionKeyUpdate(sessionKey); - return null; - }); - EasyMock.replay(secretKey); - return sessionKey; + SecretKey secretKey = mock(SecretKey.class); + when(secretKey.getAlgorithm()).thenReturn(INTER_WORKER_KEY_GENERATION_ALGORITHM_DEFAULT); + when(secretKey.getEncoded()).thenReturn(new byte[32]); + return new SessionKey(secretKey, time.milliseconds() + TimeUnit.DAYS.toMillis(1)); } private void expectConfigRefreshAndSnapshot(final ClusterConfigState readToEndSnapshot) { try { - configBackingStore.refresh(anyLong(), EasyMock.anyObject(TimeUnit.class)); - EasyMock.expectLastCall(); - EasyMock.expect(configBackingStore.snapshot()).andReturn(readToEndSnapshot); + doNothing().when(configBackingStore).refresh(anyLong(), any(TimeUnit.class)); + when(configBackingStore.snapshot()).thenReturn(readToEndSnapshot); } catch (TimeoutException e) { fail("Mocked method should not throw checked exception"); } @@ -4670,27 +3837,19 @@ private void stopBackgroundHerder() throws Exception { } private void expectHerderStartup() { - worker.start(); - EasyMock.expectLastCall(); - statusBackingStore.start(); - EasyMock.expectLastCall(); - configBackingStore.start(); - EasyMock.expectLastCall(); + doNothing().when(worker).start(); + doNothing().when(statusBackingStore).start(); + doNothing().when(configBackingStore).start(); } private void expectHerderShutdown() { - worker.stopAndAwaitConnectors(); - EasyMock.expectLastCall(); - worker.stopAndAwaitTasks(); - EasyMock.expectLastCall(); - member.stop(); - EasyMock.expectLastCall(); - statusBackingStore.stop(); - EasyMock.expectLastCall(); - configBackingStore.stop(); - EasyMock.expectLastCall(); - worker.stop(); - EasyMock.expectLastCall(); + doNothing().when(worker).stopAndAwaitConnectors(); + doNothing().when(worker).stopAndAwaitTasks(); + + doNothing().when(member).stop(); + doNothing().when(statusBackingStore).stop(); + doNothing().when(configBackingStore).stop(); + doNothing().when(worker).stop(); } private void assertStatistics(int expectedEpoch, int completedRebalances, double rebalanceTime, double millisSinceLastRebalance) { @@ -4723,67 +3882,6 @@ private void assertStatistics(String expectedLeader, boolean isRebalancing, int } } - @Test - public void processRestartRequestsFailureSuppression() { - member.wakeup(); - PowerMock.expectLastCall().anyTimes(); - - final String connectorName = "foo"; - RestartRequest restartRequest = new RestartRequest(connectorName, false, false); - EasyMock.expect(herder.buildRestartPlan(restartRequest)).andThrow(new RuntimeException()).anyTimes(); - - PowerMock.replayAll(); - - configUpdateListener.onRestartRequest(restartRequest); - assertEquals(1, herder.pendingRestartRequests.size()); - herder.processRestartRequests(); - assertTrue(herder.pendingRestartRequests.isEmpty()); - } - - @Test - public void processRestartRequestsDequeue() { - member.wakeup(); - PowerMock.expectLastCall().anyTimes(); - - EasyMock.expect(herder.buildRestartPlan(EasyMock.anyObject(RestartRequest.class))).andReturn(Optional.empty()).anyTimes(); - - PowerMock.replayAll(); - - RestartRequest restartRequest = new RestartRequest("foo", false, false); - configUpdateListener.onRestartRequest(restartRequest); - restartRequest = new RestartRequest("bar", false, false); - configUpdateListener.onRestartRequest(restartRequest); - assertEquals(2, herder.pendingRestartRequests.size()); - herder.processRestartRequests(); - assertTrue(herder.pendingRestartRequests.isEmpty()); - } - - @Test - public void preserveHighestImpactRestartRequest() { - member.wakeup(); - PowerMock.expectLastCall().anyTimes(); - PowerMock.replayAll(); - - final String connectorName = "foo"; - RestartRequest restartRequest = new RestartRequest(connectorName, false, false); - configUpdateListener.onRestartRequest(restartRequest); - - //will overwrite as this is higher impact - restartRequest = new RestartRequest(connectorName, false, true); - configUpdateListener.onRestartRequest(restartRequest); - assertEquals(1, herder.pendingRestartRequests.size()); - assertFalse(herder.pendingRestartRequests.get(connectorName).onlyFailed()); - assertTrue(herder.pendingRestartRequests.get(connectorName).includeTasks()); - - //will be ignored as the existing request has higher impact - restartRequest = new RestartRequest(connectorName, true, false); - configUpdateListener.onRestartRequest(restartRequest); - assertEquals(1, herder.pendingRestartRequests.size()); - //compare against existing request - assertFalse(herder.pendingRestartRequests.get(connectorName).onlyFailed()); - assertTrue(herder.pendingRestartRequests.get(connectorName).includeTasks()); - } - // We need to use a real class here due to some issue with mocking java.lang.Class private abstract class BogusSourceConnector extends SourceConnector { } @@ -4831,11 +3929,9 @@ public boolean awaitTermination(long timeout, TimeUnit unit) throws InterruptedE private DistributedHerder exactlyOnceHerder() { Map config = new HashMap<>(HERDER_CONFIG); config.put(EXACTLY_ONCE_SOURCE_SUPPORT_CONFIG, "enabled"); - return PowerMock.createPartialMock(DistributedHerder.class, - new String[]{"connectorType", "updateDeletedConnectorStatus", "updateDeletedTaskStatus", "validateConnectorConfig"}, - new DistributedConfig(config), worker, WORKER_ID, KAFKA_CLUSTER_ID, - statusBackingStore, configBackingStore, member, MEMBER_URL, restClient, metrics, time, noneConnectorClientConfigOverridePolicy, - Collections.emptyList(), null, new AutoCloseable[0]); + return mock(DistributedHerder.class, withSettings().defaultAnswer(CALLS_REAL_METHODS).useConstructor(new DistributedConfig(config), + worker, WORKER_ID, KAFKA_CLUSTER_ID, statusBackingStore, configBackingStore, member, MEMBER_URL, restClient, metrics, time, + noneConnectorClientConfigOverridePolicy, Collections.emptyList(), null, new AutoCloseable[0])); } }