Reduce use of mocking and simplify some tests (#12283)

* remove use of mocks for ServiceMetricEvent
* simplify KafkaEmitterTests by moving to Mockito
* speed up KafkaEmitterTest by adjusting reporting frequency in tests
* remove unnecessary easymock and JUnitParams dependencies
This commit is contained in:
Xavier Léauté 2022-02-26 17:23:09 -08:00 committed by GitHub
parent a080fcdd7b
commit 4c61878f9c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
20 changed files with 47 additions and 163 deletions

View File

@ -120,11 +120,6 @@
<artifactId>junit</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.easymock</groupId>
<artifactId>easymock</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>pl.pragmatists</groupId>
<artifactId>JUnitParams</artifactId>

View File

@ -27,10 +27,8 @@ import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
import org.apache.hadoop.metrics2.sink.timeline.TimelineMetric;
import org.easymock.EasyMock;
import org.joda.time.DateTime;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
@ -38,7 +36,6 @@ import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.OutputStream;
import java.util.HashMap;
@RunWith(JUnitParamsRunner.class)
@ -47,24 +44,11 @@ public class WhiteListBasedDruidToTimelineEventConverterTest
private final String prefix = "druid";
private final WhiteListBasedDruidToTimelineEventConverter defaultWhiteListBasedDruidToTimelineEventConverter =
new WhiteListBasedDruidToTimelineEventConverter(prefix, "druid", null, new DefaultObjectMapper());
private ServiceMetricEvent event;
private final DateTime createdTime = DateTimes.nowUtc();
private final String hostname = "testHost:8080";
private final String serviceName = "historical";
private final String defaultNamespace = prefix + "." + serviceName;
@Before
public void setUp()
{
event = EasyMock.createMock(ServiceMetricEvent.class);
EasyMock.expect(event.getHost()).andReturn(hostname).anyTimes();
EasyMock.expect(event.getService()).andReturn(serviceName).anyTimes();
EasyMock.expect(event.getCreatedTime()).andReturn(createdTime).anyTimes();
EasyMock.expect(event.getUserDims()).andReturn(new HashMap<>()).anyTimes();
EasyMock.expect(event.getValue()).andReturn(10).anyTimes();
EasyMock.expect(event.getFeed()).andReturn("metrics").anyTimes();
}
@Test
@Parameters(
{
@ -90,8 +74,12 @@ public class WhiteListBasedDruidToTimelineEventConverterTest
)
public void testDefaultIsInWhiteList(String key, boolean expectedValue)
{
EasyMock.expect(event.getMetric()).andReturn(key).anyTimes();
EasyMock.replay(event);
ServiceMetricEvent event = ServiceMetricEvent
.builder()
.setFeed("metrics")
.build(createdTime, key, 10)
.build(serviceName, hostname);
boolean isIn = defaultWhiteListBasedDruidToTimelineEventConverter.druidEventToTimelineMetric(event) != null;
Assert.assertEquals(expectedValue, isIn);
}

View File

@ -107,11 +107,6 @@
<artifactId>junit</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.easymock</groupId>
<artifactId>easymock</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.hamcrest</groupId>
<artifactId>hamcrest-core</artifactId>

View File

@ -73,11 +73,6 @@
<artifactId>easymock</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>pl.pragmatists</groupId>
<artifactId>JUnitParams</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.druid</groupId>
<artifactId>druid-processing</artifactId>

View File

@ -103,11 +103,6 @@
<artifactId>junit</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.easymock</groupId>
<artifactId>easymock</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>pl.pragmatists</groupId>
<artifactId>JUnitParams</artifactId>

View File

@ -26,10 +26,8 @@ import org.apache.druid.annotations.UsedByJUnitParamsRunner;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
import org.easymock.EasyMock;
import org.joda.time.DateTime;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
@ -37,7 +35,6 @@ import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.OutputStream;
import java.util.HashMap;
@RunWith(JUnitParamsRunner.class)
@ -52,22 +49,10 @@ public class WhiteListBasedConverterTest
null,
new DefaultObjectMapper()
);
private ServiceMetricEvent event;
private DateTime createdTime = DateTimes.nowUtc();
private String hostname = "testHost.yahoo.com:8080";
private String serviceName = "historical";
private String defaultNamespace = prefix + "." + serviceName + "." + GraphiteEmitter.sanitize(hostname);
@Before
public void setUp()
{
event = EasyMock.createMock(ServiceMetricEvent.class);
EasyMock.expect(event.getHost()).andReturn(hostname).anyTimes();
EasyMock.expect(event.getService()).andReturn(serviceName).anyTimes();
EasyMock.expect(event.getCreatedTime()).andReturn(createdTime).anyTimes();
EasyMock.expect(event.getUserDims()).andReturn(new HashMap<>()).anyTimes();
EasyMock.expect(event.getValue()).andReturn(10).anyTimes();
}
private final DateTime createdTime = DateTimes.nowUtc();
private final String hostname = "testHost.yahoo.com:8080";
private final String serviceName = "historical";
private final String defaultNamespace = prefix + "." + serviceName + "." + GraphiteEmitter.sanitize(hostname);
@Test
@Parameters(
@ -94,8 +79,11 @@ public class WhiteListBasedConverterTest
)
public void testDefaultIsInWhiteList(String key, boolean expectedValue)
{
EasyMock.expect(event.getMetric()).andReturn(key).anyTimes();
EasyMock.replay(event);
ServiceMetricEvent event = ServiceMetricEvent
.builder()
.build(createdTime, key, 10)
.build(serviceName, hostname);
boolean isIn = defaultWhiteListBasedConverter.druidEventToGraphite(event) != null;
Assert.assertEquals(expectedValue, isIn);
}

View File

@ -81,16 +81,6 @@
<artifactId>junit</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.easymock</groupId>
<artifactId>easymock</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>pl.pragmatists</groupId>
<artifactId>JUnitParams</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>nl.jqno.equalsverifier</groupId>
<artifactId>equalsverifier</artifactId>

View File

@ -88,13 +88,8 @@
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.easymock</groupId>
<artifactId>easymock</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>pl.pragmatists</groupId>
<artifactId>JUnitParams</artifactId>
<groupId>org.mockito</groupId>
<artifactId>mockito-core</artifactId>
<scope>test</scope>
</dependency>
<dependency>

View File

@ -50,6 +50,8 @@ public class KafkaEmitter implements Emitter
{
private static Logger log = new Logger(KafkaEmitter.class);
private static final int DEFAULT_SEND_INTERVAL_SECONDS = 10;
private static final int DEFAULT_SEND_LOST_INTERVAL_MINUTES = 5;
private static final int DEFAULT_RETRIES = 3;
private final AtomicLong metricLost;
private final AtomicLong alertLost;
@ -64,6 +66,8 @@ public class KafkaEmitter implements Emitter
private final MemoryBoundLinkedBlockingQueue<String> requestQueue;
private final ScheduledExecutorService scheduler;
protected int sendInterval = DEFAULT_SEND_INTERVAL_SECONDS;
public KafkaEmitter(KafkaEmitterConfig config, ObjectMapper jsonMapper)
{
this.config = config;
@ -116,16 +120,16 @@ public class KafkaEmitter implements Emitter
@Override
public void start()
{
scheduler.schedule(this::sendMetricToKafka, 10, TimeUnit.SECONDS);
scheduler.schedule(this::sendAlertToKafka, 10, TimeUnit.SECONDS);
scheduler.schedule(this::sendMetricToKafka, sendInterval, TimeUnit.SECONDS);
scheduler.schedule(this::sendAlertToKafka, sendInterval, TimeUnit.SECONDS);
if (config.getRequestTopic() != null) {
scheduler.schedule(this::sendRequestToKafka, 10, TimeUnit.SECONDS);
scheduler.schedule(this::sendRequestToKafka, sendInterval, TimeUnit.SECONDS);
}
scheduler.scheduleWithFixedDelay(() -> {
log.info("Message lost counter: metricLost=[%d], alertLost=[%d], requestLost=[%d], invalidLost=[%d]",
metricLost.get(), alertLost.get(), requestLost.get(), invalidLost.get()
);
}, 5, 5, TimeUnit.MINUTES);
}, DEFAULT_SEND_LOST_INTERVAL_MINUTES, DEFAULT_SEND_LOST_INTERVAL_MINUTES, TimeUnit.MINUTES);
log.info("Starting Kafka Emitter.");
}
@ -144,8 +148,7 @@ public class KafkaEmitter implements Emitter
sendToKafka(config.getRequestTopic(), requestQueue, setProducerCallback(requestLost));
}
@VisibleForTesting
protected void sendToKafka(final String topic, MemoryBoundLinkedBlockingQueue<String> recordQueue, Callback callback)
private void sendToKafka(final String topic, MemoryBoundLinkedBlockingQueue<String> recordQueue, Callback callback)
{
ObjectContainer<String> objectToSend;
try {

View File

@ -30,10 +30,8 @@ import org.apache.druid.server.QueryStats;
import org.apache.druid.server.RequestLogLine;
import org.apache.druid.server.log.DefaultRequestLogEventBuilderFactory;
import org.apache.druid.server.log.RequestLogEvent;
import org.apache.kafka.clients.producer.Callback;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.Producer;
import org.easymock.EasyMock;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runner.RunWith;
@ -42,6 +40,10 @@ import org.junit.runners.Parameterized;
import java.util.List;
import java.util.concurrent.CountDownLatch;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
@RunWith(Parameterized.class)
public class KafkaEmitterTest
{
@ -57,8 +59,8 @@ public class KafkaEmitterTest
};
}
// there is 10 seconds wait in kafka emitter before it starts sending events to broker, so set a timeout for 15 seconds
@Test(timeout = 15_000)
// there is 1 seconds wait in kafka emitter before it starts sending events to broker, set a timeout for 5 seconds
@Test(timeout = 5_000)
public void testKafkaEmitter() throws InterruptedException
{
final List<ServiceMetricEvent> serviceMetricEvents = ImmutableList.of(
@ -80,7 +82,7 @@ public class KafkaEmitterTest
final CountDownLatch countDownSentEvents = new CountDownLatch(
requestTopic == null ? totalEventsExcludingRequestLogEvents : totalEvents);
final KafkaProducer<String, String> producer = EasyMock.createStrictMock(KafkaProducer.class);
final KafkaProducer<String, String> producer = mock(KafkaProducer.class);
final KafkaEmitter kafkaEmitter = new KafkaEmitter(
new KafkaEmitterConfig("", "metrics", "alerts", requestTopic, "test-cluster", null),
new ObjectMapper()
@ -89,22 +91,17 @@ public class KafkaEmitterTest
@Override
protected Producer<String, String> setKafkaProducer()
{
// override send interval to 1 second
sendInterval = 1;
return producer;
}
@Override
protected void sendToKafka(final String topic, MemoryBoundLinkedBlockingQueue<String> recordQueue,
Callback callback
)
{
countDownSentEvents.countDown();
super.sendToKafka(topic, recordQueue, callback);
}
};
EasyMock.expect(producer.send(EasyMock.anyObject(), EasyMock.anyObject())).andReturn(null)
.times(requestTopic == null ? totalEventsExcludingRequestLogEvents : totalEvents);
EasyMock.replay(producer);
when(producer.send(any(), any())).then((invocation) -> {
countDownSentEvents.countDown();
return null;
});
kafkaEmitter.start();
for (Event event : serviceMetricEvents) {
@ -122,18 +119,5 @@ public class KafkaEmitterTest
Assert.assertEquals(0, kafkaEmitter.getAlertLostCount());
Assert.assertEquals(requestTopic == null ? requestLogEvents.size() : 0, kafkaEmitter.getRequestLostCount());
Assert.assertEquals(0, kafkaEmitter.getInvalidLostCount());
while (true) {
try {
EasyMock.verify(producer);
break;
}
catch (Throwable e) {
// although the latch may have count down, producer.send may not have been called yet in KafkaEmitter
// so wait for sometime before verifying the mock
Thread.sleep(100);
// just continue
}
}
}
}

View File

@ -87,11 +87,6 @@
<artifactId>junit</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.easymock</groupId>
<artifactId>easymock</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>nl.jqno.equalsverifier</groupId>
<artifactId>equalsverifier</artifactId>

View File

@ -146,11 +146,6 @@
<artifactId>junit</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>pl.pragmatists</groupId>
<artifactId>JUnitParams</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
<build>

View File

@ -100,11 +100,6 @@
<artifactId>easymock</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>pl.pragmatists</groupId>
<artifactId>JUnitParams</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.druid</groupId>
<artifactId>druid-server</artifactId>

View File

@ -88,11 +88,6 @@
<artifactId>easymock</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>pl.pragmatists</groupId>
<artifactId>JUnitParams</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.druid</groupId>
<artifactId>druid-server</artifactId>

View File

@ -83,11 +83,6 @@
<artifactId>junit</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.easymock</groupId>
<artifactId>easymock</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.druid</groupId>
<artifactId>druid-core</artifactId>

View File

@ -350,11 +350,6 @@
<artifactId>junit</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.easymock</groupId>
<artifactId>easymock</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
</project>

View File

@ -160,11 +160,6 @@
<artifactId>junit</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.easymock</groupId>
<artifactId>easymock</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.druid</groupId>
<artifactId>druid-server</artifactId>

View File

@ -136,11 +136,6 @@
<artifactId>junit</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>pl.pragmatists</groupId>
<artifactId>JUnitParams</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.mockito</groupId>
<artifactId>mockito-core</artifactId>

View File

@ -131,11 +131,6 @@
<artifactId>junit</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.easymock</groupId>
<artifactId>easymock</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>nl.jqno.equalsverifier</groupId>
<artifactId>equalsverifier</artifactId>
@ -144,6 +139,12 @@
<dependency>
<groupId>org.mockito</groupId>
<artifactId>mockito-core</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.easymock</groupId>
<artifactId>easymock</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.druid</groupId>

View File

@ -420,11 +420,6 @@
<groupId>org.testng</groupId>
<artifactId>testng</artifactId>
</dependency>
<dependency>
<groupId>org.easymock</groupId>
<artifactId>easymock</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>com.github.os72</groupId>
<artifactId>protobuf-dynamic</artifactId>