KAFKA-14132: Replace Easymock & Powermock with Mockito in KafkaBasedLogTest (#14153)

Reviewers: Yash Mayya <yash.mayya@gmail.com>, Divij Vaidya <diviv@amazon.com>
This commit is contained in:
bachmanity1 2023-08-11 17:50:37 +09:00 committed by GitHub
parent cdbc9a8d88
commit f137da04fa
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
3 changed files with 73 additions and 162 deletions

View File

@ -416,7 +416,7 @@ subprojects {
testsToExclude.addAll([ testsToExclude.addAll([
// connect tests // connect tests
"**/KafkaConfigBackingStoreTest.*", "**/KafkaConfigBackingStoreTest.*",
"**/KafkaBasedLogTest.*", "**/StandaloneHerderTest.*", "**/StandaloneHerderTest.*",
"**/WorkerSinkTaskTest.*", "**/WorkerSinkTaskThreadedTest.*" "**/WorkerSinkTaskTest.*", "**/WorkerSinkTaskThreadedTest.*"
]) ])
} }

View File

@ -102,8 +102,8 @@ public class KafkaBasedLog<K, V> {
private Consumer<K, V> consumer; private Consumer<K, V> consumer;
private Optional<Producer<K, V>> producer; private Optional<Producer<K, V>> producer;
private TopicAdmin admin; private TopicAdmin admin;
// Visible for testing
private Thread thread; Thread thread;
private boolean stopRequested; private boolean stopRequested;
private final Queue<Callback<Void>> readLogEndOffsetCallbacks; private final Queue<Callback<Void>> readLogEndOffsetCallbacks;
private final java.util.function.Consumer<TopicAdmin> initializer; private final java.util.function.Consumer<TopicAdmin> initializer;

View File

@ -36,21 +36,15 @@ import org.apache.kafka.common.errors.WakeupException;
import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.header.internals.RecordHeaders;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.record.TimestampType;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.MockTime;
import org.easymock.Capture; import org.apache.kafka.common.utils.Time;
import org.easymock.EasyMock;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import org.junit.runner.RunWith; import org.junit.runner.RunWith;
import org.powermock.api.easymock.PowerMock; import org.mockito.ArgumentCaptor;
import org.powermock.api.easymock.annotation.Mock; import org.mockito.Mock;
import org.powermock.core.classloader.annotations.PowerMockIgnore; import org.mockito.junit.MockitoJUnitRunner;
import org.powermock.core.classloader.annotations.PrepareForTest;
import org.powermock.modules.junit4.PowerMockRunner;
import org.powermock.reflect.Whitebox;
import java.nio.ByteBuffer;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.HashMap; import java.util.HashMap;
@ -64,6 +58,7 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Consumer;
import java.util.function.Supplier; import java.util.function.Supplier;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
@ -72,10 +67,14 @@ import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull; import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.any;
import static org.mockito.Mockito.anyLong;
import static org.mockito.Mockito.eq;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
@RunWith(PowerMockRunner.class) @RunWith(MockitoJUnitRunner.StrictStubs.class)
@PrepareForTest(KafkaBasedLog.class)
@PowerMockIgnore("javax.management.*")
public class KafkaBasedLogTest { public class KafkaBasedLogTest {
private static final String TOPIC = "connect-log"; private static final String TOPIC = "connect-log";
@ -114,29 +113,37 @@ public class KafkaBasedLogTest {
private static final String TP0_VALUE_NEW = "VAL0_NEW"; private static final String TP0_VALUE_NEW = "VAL0_NEW";
private static final String TP1_VALUE_NEW = "VAL1_NEW"; private static final String TP1_VALUE_NEW = "VAL1_NEW";
private Time time = new MockTime(); private final Time time = new MockTime();
private KafkaBasedLog<String, String> store; private KafkaBasedLog<String, String> store;
@Mock @Mock
private Runnable initializer; private Consumer<TopicAdmin> initializer;
@Mock @Mock
private KafkaProducer<String, String> producer; private KafkaProducer<String, String> producer;
private MockConsumer<String, String> consumer;
@Mock
private TopicAdmin admin; private TopicAdmin admin;
private final Supplier<TopicAdmin> topicAdminSupplier = () -> admin;
private MockConsumer<String, String> consumer;
private Map<TopicPartition, List<ConsumerRecord<String, String>>> consumedRecords = new HashMap<>(); private final Map<TopicPartition, List<ConsumerRecord<String, String>>> consumedRecords = new HashMap<>();
private Callback<ConsumerRecord<String, String>> consumedCallback = (error, record) -> { private final Callback<ConsumerRecord<String, String>> consumedCallback = (error, record) -> {
TopicPartition partition = new TopicPartition(record.topic(), record.partition()); TopicPartition partition = new TopicPartition(record.topic(), record.partition());
List<ConsumerRecord<String, String>> records = consumedRecords.computeIfAbsent(partition, k -> new ArrayList<>()); List<ConsumerRecord<String, String>> records = consumedRecords.computeIfAbsent(partition, k -> new ArrayList<>());
records.add(record); records.add(record);
}; };
@SuppressWarnings("unchecked")
@Before @Before
public void setUp() { public void setUp() {
store = PowerMock.createPartialMock(KafkaBasedLog.class, new String[]{"createConsumer", "createProducer"}, store = new KafkaBasedLog<String, String>(TOPIC, PRODUCER_PROPS, CONSUMER_PROPS, topicAdminSupplier, consumedCallback, time, initializer) {
TOPIC, PRODUCER_PROPS, CONSUMER_PROPS, consumedCallback, time, initializer); @Override
protected KafkaProducer<String, String> createProducer() {
return producer;
}
@Override
protected MockConsumer<String, String> createConsumer() {
return consumer;
}
};
consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST);
consumer.updatePartitions(TOPIC, Arrays.asList(TPINFO0, TPINFO1)); consumer.updatePartitions(TOPIC, Arrays.asList(TPINFO0, TPINFO1));
Map<TopicPartition, Long> beginningOffsets = new HashMap<>(); Map<TopicPartition, Long> beginningOffsets = new HashMap<>();
@ -146,12 +153,7 @@ public class KafkaBasedLogTest {
} }
@Test @Test
public void testStartStop() throws Exception { public void testStartStop() {
expectStart();
expectStop();
PowerMock.replayAll();
Map<TopicPartition, Long> endOffsets = new HashMap<>(); Map<TopicPartition, Long> endOffsets = new HashMap<>();
endOffsets.put(TP0, 0L); endOffsets.put(TP0, 0L);
endOffsets.put(TP1, 0L); endOffsets.put(TP1, 0L);
@ -160,19 +162,11 @@ public class KafkaBasedLogTest {
assertEquals(CONSUMER_ASSIGNMENT, consumer.assignment()); assertEquals(CONSUMER_ASSIGNMENT, consumer.assignment());
store.stop(); store.stop();
verifyStartAndStop();
assertFalse(Whitebox.<Thread>getInternalState(store, "thread").isAlive());
assertTrue(consumer.closed());
PowerMock.verifyAll();
} }
@Test @Test
public void testReloadOnStart() throws Exception { public void testReloadOnStart() throws Exception {
expectStart();
expectStop();
PowerMock.replayAll();
Map<TopicPartition, Long> endOffsets = new HashMap<>(); Map<TopicPartition, Long> endOffsets = new HashMap<>();
endOffsets.put(TP0, 1L); endOffsets.put(TP0, 1L);
endOffsets.put(TP1, 1L); endOffsets.put(TP1, 1L);
@ -206,19 +200,11 @@ public class KafkaBasedLogTest {
assertEquals(TP1_VALUE, consumedRecords.get(TP1).get(0).value()); assertEquals(TP1_VALUE, consumedRecords.get(TP1).get(0).value());
store.stop(); store.stop();
verifyStartAndStop();
assertFalse(Whitebox.<Thread>getInternalState(store, "thread").isAlive());
assertTrue(consumer.closed());
PowerMock.verifyAll();
} }
@Test @Test
public void testReloadOnStartWithNoNewRecordsPresent() throws Exception { public void testReloadOnStartWithNoNewRecordsPresent() {
expectStart();
expectStop();
PowerMock.replayAll();
Map<TopicPartition, Long> endOffsets = new HashMap<>(); Map<TopicPartition, Long> endOffsets = new HashMap<>();
endOffsets.put(TP0, 7L); endOffsets.put(TP0, 7L);
endOffsets.put(TP1, 7L); endOffsets.put(TP1, 7L);
@ -241,30 +227,19 @@ public class KafkaBasedLogTest {
store.stop(); store.stop();
assertFalse(Whitebox.<Thread>getInternalState(store, "thread").isAlive()); verifyStartAndStop();
assertTrue(consumer.closed());
PowerMock.verifyAll();
} }
@Test @Test
public void testSendAndReadToEnd() throws Exception { public void testSendAndReadToEnd() throws Exception {
expectStart();
TestFuture<RecordMetadata> tp0Future = new TestFuture<>(); TestFuture<RecordMetadata> tp0Future = new TestFuture<>();
ProducerRecord<String, String> tp0Record = new ProducerRecord<>(TOPIC, TP0_KEY, TP0_VALUE); ProducerRecord<String, String> tp0Record = new ProducerRecord<>(TOPIC, TP0_KEY, TP0_VALUE);
Capture<org.apache.kafka.clients.producer.Callback> callback0 = EasyMock.newCapture(); ArgumentCaptor<org.apache.kafka.clients.producer.Callback> callback0 = ArgumentCaptor.forClass(org.apache.kafka.clients.producer.Callback.class);
EasyMock.expect(producer.send(EasyMock.eq(tp0Record), EasyMock.capture(callback0))).andReturn(tp0Future); when(producer.send(eq(tp0Record), callback0.capture())).thenReturn(tp0Future);
TestFuture<RecordMetadata> tp1Future = new TestFuture<>(); TestFuture<RecordMetadata> tp1Future = new TestFuture<>();
ProducerRecord<String, String> tp1Record = new ProducerRecord<>(TOPIC, TP1_KEY, TP1_VALUE); ProducerRecord<String, String> tp1Record = new ProducerRecord<>(TOPIC, TP1_KEY, TP1_VALUE);
Capture<org.apache.kafka.clients.producer.Callback> callback1 = EasyMock.newCapture(); ArgumentCaptor<org.apache.kafka.clients.producer.Callback> callback1 = ArgumentCaptor.forClass(org.apache.kafka.clients.producer.Callback.class);
EasyMock.expect(producer.send(EasyMock.eq(tp1Record), EasyMock.capture(callback1))).andReturn(tp1Future); when(producer.send(eq(tp1Record), callback1.capture())).thenReturn(tp1Future);
// Producer flushes when read to log end is called
producer.flush();
PowerMock.expectLastCall();
expectStop();
PowerMock.replayAll();
Map<TopicPartition, Long> endOffsets = new HashMap<>(); Map<TopicPartition, Long> endOffsets = new HashMap<>();
endOffsets.put(TP0, 0L); endOffsets.put(TP0, 0L);
@ -335,18 +310,13 @@ public class KafkaBasedLogTest {
// Cleanup // Cleanup
store.stop(); store.stop();
assertFalse(Whitebox.<Thread>getInternalState(store, "thread").isAlive()); // Producer flushes when read to log end is called
assertTrue(consumer.closed()); verify(producer).flush();
PowerMock.verifyAll(); verifyStartAndStop();
} }
@Test @Test
public void testPollConsumerError() throws Exception { public void testPollConsumerError() throws Exception {
expectStart();
expectStop();
PowerMock.replayAll();
final CountDownLatch finishedLatch = new CountDownLatch(1); final CountDownLatch finishedLatch = new CountDownLatch(1);
Map<TopicPartition, Long> endOffsets = new HashMap<>(); Map<TopicPartition, Long> endOffsets = new HashMap<>();
endOffsets.put(TP0, 1L); endOffsets.put(TP0, 1L);
@ -376,22 +346,11 @@ public class KafkaBasedLogTest {
store.stop(); store.stop();
assertFalse(Whitebox.<Thread>getInternalState(store, "thread").isAlive()); verifyStartAndStop();
assertTrue(consumer.closed());
PowerMock.verifyAll();
} }
@Test @Test
public void testGetOffsetsConsumerErrorOnReadToEnd() throws Exception { public void testGetOffsetsConsumerErrorOnReadToEnd() throws Exception {
expectStart();
// Producer flushes when read to log end is called
producer.flush();
PowerMock.expectLastCall();
expectStop();
PowerMock.replayAll();
final CountDownLatch finishedLatch = new CountDownLatch(1); final CountDownLatch finishedLatch = new CountDownLatch(1);
Map<TopicPartition, Long> endOffsets = new HashMap<>(); Map<TopicPartition, Long> endOffsets = new HashMap<>();
endOffsets.put(TP0, 0L); endOffsets.put(TP0, 0L);
@ -433,22 +392,17 @@ public class KafkaBasedLogTest {
store.stop(); store.stop();
assertFalse(Whitebox.<Thread>getInternalState(store, "thread").isAlive()); // Producer flushes when read to log end is called
assertTrue(consumer.closed()); verify(producer).flush();
PowerMock.verifyAll(); verifyStartAndStop();
} }
@Test @Test
public void testProducerError() throws Exception { public void testProducerError() {
expectStart();
TestFuture<RecordMetadata> tp0Future = new TestFuture<>(); TestFuture<RecordMetadata> tp0Future = new TestFuture<>();
ProducerRecord<String, String> tp0Record = new ProducerRecord<>(TOPIC, TP0_KEY, TP0_VALUE); ProducerRecord<String, String> tp0Record = new ProducerRecord<>(TOPIC, TP0_KEY, TP0_VALUE);
Capture<org.apache.kafka.clients.producer.Callback> callback0 = EasyMock.newCapture(); ArgumentCaptor<org.apache.kafka.clients.producer.Callback> callback0 = ArgumentCaptor.forClass(org.apache.kafka.clients.producer.Callback.class);
EasyMock.expect(producer.send(EasyMock.eq(tp0Record), EasyMock.capture(callback0))).andReturn(tp0Future); when(producer.send(eq(tp0Record), callback0.capture())).thenReturn(tp0Future);
expectStop();
PowerMock.replayAll();
Map<TopicPartition, Long> endOffsets = new HashMap<>(); Map<TopicPartition, Long> endOffsets = new HashMap<>();
endOffsets.put(TP0, 0L); endOffsets.put(TP0, 0L);
@ -471,42 +425,31 @@ public class KafkaBasedLogTest {
store.stop(); store.stop();
assertFalse(Whitebox.<Thread>getInternalState(store, "thread").isAlive()); verifyStartAndStop();
assertTrue(consumer.closed());
PowerMock.verifyAll();
} }
@Test @Test
public void testReadEndOffsetsUsingAdmin() throws Exception { public void testReadEndOffsetsUsingAdmin() {
// Create a log that uses the admin supplier
setupWithAdmin();
expectProducerAndConsumerCreate();
Set<TopicPartition> tps = new HashSet<>(Arrays.asList(TP0, TP1)); Set<TopicPartition> tps = new HashSet<>(Arrays.asList(TP0, TP1));
Map<TopicPartition, Long> endOffsets = new HashMap<>(); Map<TopicPartition, Long> endOffsets = new HashMap<>();
endOffsets.put(TP0, 0L); endOffsets.put(TP0, 0L);
endOffsets.put(TP1, 0L); endOffsets.put(TP1, 0L);
admin.retryEndOffsets(EasyMock.eq(tps), EasyMock.anyObject(), EasyMock.anyLong()); admin = mock(TopicAdmin.class);
PowerMock.expectLastCall().andReturn(endOffsets).times(1); when(admin.retryEndOffsets(eq(tps), any(), anyLong())).thenReturn(endOffsets);
admin.endOffsets(EasyMock.eq(tps)); when(admin.endOffsets(eq(tps))).thenReturn(endOffsets);
PowerMock.expectLastCall().andReturn(endOffsets).times(1);
PowerMock.replayAll();
store.start(); store.start();
assertEquals(endOffsets, store.readEndOffsets(tps, false)); assertEquals(endOffsets, store.readEndOffsets(tps, false));
verify(admin).retryEndOffsets(eq(tps), any(), anyLong());
verify(admin).endOffsets(eq(tps));
} }
@Test @Test
public void testReadEndOffsetsUsingAdminThatFailsWithUnsupported() throws Exception { public void testReadEndOffsetsUsingAdminThatFailsWithUnsupported() {
// Create a log that uses the admin supplier
setupWithAdmin();
expectProducerAndConsumerCreate();
Set<TopicPartition> tps = new HashSet<>(Arrays.asList(TP0, TP1)); Set<TopicPartition> tps = new HashSet<>(Arrays.asList(TP0, TP1));
admin = mock(TopicAdmin.class);
// Getting end offsets using the admin client should fail with unsupported version // Getting end offsets using the admin client should fail with unsupported version
admin.retryEndOffsets(EasyMock.eq(tps), EasyMock.anyObject(), EasyMock.anyLong()); when(admin.retryEndOffsets(eq(tps), any(), anyLong())).thenThrow(new UnsupportedVersionException("too old"));
PowerMock.expectLastCall().andThrow(new UnsupportedVersionException("too old"));
// Falls back to the consumer // Falls back to the consumer
Map<TopicPartition, Long> endOffsets = new HashMap<>(); Map<TopicPartition, Long> endOffsets = new HashMap<>();
@ -514,65 +457,33 @@ public class KafkaBasedLogTest {
endOffsets.put(TP1, 0L); endOffsets.put(TP1, 0L);
consumer.updateEndOffsets(endOffsets); consumer.updateEndOffsets(endOffsets);
PowerMock.replayAll();
store.start(); store.start();
assertEquals(endOffsets, store.readEndOffsets(tps, false)); assertEquals(endOffsets, store.readEndOffsets(tps, false));
verify(admin).retryEndOffsets(eq(tps), any(), anyLong());
} }
@Test @Test
public void testReadEndOffsetsUsingAdminThatFailsWithRetriable() throws Exception { public void testReadEndOffsetsUsingAdminThatFailsWithRetriable() {
// Create a log that uses the admin supplier
setupWithAdmin();
expectProducerAndConsumerCreate();
Set<TopicPartition> tps = new HashSet<>(Arrays.asList(TP0, TP1)); Set<TopicPartition> tps = new HashSet<>(Arrays.asList(TP0, TP1));
Map<TopicPartition, Long> endOffsets = new HashMap<>(); Map<TopicPartition, Long> endOffsets = new HashMap<>();
endOffsets.put(TP0, 0L); endOffsets.put(TP0, 0L);
endOffsets.put(TP1, 0L); endOffsets.put(TP1, 0L);
admin = mock(TopicAdmin.class);
// Getting end offsets upon startup should work fine // Getting end offsets upon startup should work fine
admin.retryEndOffsets(EasyMock.eq(tps), EasyMock.anyObject(), EasyMock.anyLong()); when(admin.retryEndOffsets(eq(tps), any(), anyLong())).thenReturn(endOffsets);
PowerMock.expectLastCall().andReturn(endOffsets).times(1);
// Getting end offsets using the admin client should fail with leader not available // Getting end offsets using the admin client should fail with leader not available
admin.endOffsets(EasyMock.eq(tps)); when(admin.endOffsets(eq(tps))).thenThrow(new LeaderNotAvailableException("retry"));
PowerMock.expectLastCall().andThrow(new LeaderNotAvailableException("retry"));
PowerMock.replayAll();
store.start(); store.start();
assertThrows(LeaderNotAvailableException.class, () -> store.readEndOffsets(tps, false)); assertThrows(LeaderNotAvailableException.class, () -> store.readEndOffsets(tps, false));
verify(admin).retryEndOffsets(eq(tps), any(), anyLong());
verify(admin).endOffsets(eq(tps));
} }
@SuppressWarnings("unchecked") private void verifyStartAndStop() {
private void setupWithAdmin() { verify(initializer).accept(admin);
Supplier<TopicAdmin> adminSupplier = () -> admin; verify(producer).close();
java.util.function.Consumer<TopicAdmin> initializer = admin -> { }; assertTrue(consumer.closed());
store = PowerMock.createPartialMock(KafkaBasedLog.class, new String[]{"createConsumer", "createProducer"}, assertFalse(store.thread.isAlive());
TOPIC, PRODUCER_PROPS, CONSUMER_PROPS, adminSupplier, consumedCallback, time, initializer);
} }
private void expectProducerAndConsumerCreate() throws Exception {
PowerMock.expectPrivate(store, "createProducer")
.andReturn(producer);
PowerMock.expectPrivate(store, "createConsumer")
.andReturn(consumer);
}
private void expectStart() throws Exception {
initializer.run();
EasyMock.expectLastCall().times(1);
expectProducerAndConsumerCreate();
}
private void expectStop() {
producer.close();
PowerMock.expectLastCall();
// MockConsumer close is checked after test.
}
private static ByteBuffer buffer(String v) {
return ByteBuffer.wrap(v.getBytes());
}
} }