mirror of https://github.com/apache/druid.git
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:
parent
a080fcdd7b
commit
4c61878f9c
|
@ -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>
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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>
|
||||
|
|
Loading…
Reference in New Issue