mirror of
https://github.com/apache/druid.git
synced 2025-02-17 07:25:02 +00:00
Update mocking libraries for Java 11 support (#7596)
* update easymock / powermock for to 4.0.2 / 2.0.2 for JDK11 support * update tests to use new easymock interfaces * fix tests failing due to easymock fixes * remove dependency on jmockit * fix race condition in ResourcePoolTest
This commit is contained in:
parent
a00d3e1931
commit
f7bfe8f269
@ -209,13 +209,19 @@ public class ResourcePoolTest
|
||||
blockedThread.waitForValueToBeGotten(1, TimeUnit.SECONDS);
|
||||
pool.close();
|
||||
|
||||
|
||||
EasyMock.verify(resourceFactory);
|
||||
EasyMock.reset(resourceFactory);
|
||||
// billy0Thread calling ResourceContainer.returnResource() will result
|
||||
// in a call to resourceFactory.close() when latch2 is triggered
|
||||
resourceFactory.close("billy0");
|
||||
EasyMock.expectLastCall().once();
|
||||
EasyMock.replay(resourceFactory);
|
||||
|
||||
latch2.countDown();
|
||||
blockedThread.waitForValueToBeGotten(1, TimeUnit.SECONDS);
|
||||
// wait for billy0Thread to have called resourceFactory.close() to avoid race
|
||||
// between billy0Thread calling it and verify() checking for the call
|
||||
billy0Thread.join();
|
||||
|
||||
EasyMock.verify(resourceFactory);
|
||||
EasyMock.reset(resourceFactory);
|
||||
|
@ -38,27 +38,15 @@
|
||||
</properties>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.jmockit</groupId>
|
||||
<artifactId>jmockit</artifactId>
|
||||
<version>1.25</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>junit</groupId>
|
||||
<artifactId>junit</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.hamcrest</groupId>
|
||||
<artifactId>hamcrest-library</artifactId>
|
||||
<version>1.3</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.dataformat</groupId>
|
||||
<artifactId>jackson-dataformat-yaml</artifactId>
|
||||
<version>2.8.3</version>
|
||||
<version>${jackson.version}</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
|
@ -30,8 +30,6 @@ import com.google.inject.Injector;
|
||||
import com.google.inject.Module;
|
||||
import com.google.inject.name.Names;
|
||||
import com.google.inject.util.Providers;
|
||||
import mockit.Mock;
|
||||
import mockit.MockUp;
|
||||
import org.apache.druid.client.CachingClusteredClient;
|
||||
import org.apache.druid.client.DruidServer;
|
||||
import org.apache.druid.client.ImmutableDruidServer;
|
||||
@ -62,6 +60,7 @@ import org.apache.druid.query.QuerySegmentWalker;
|
||||
import org.apache.druid.query.QueryToolChestWarehouse;
|
||||
import org.apache.druid.query.Result;
|
||||
import org.apache.druid.query.RetryQueryRunnerConfig;
|
||||
import org.apache.druid.query.SegmentDescriptor;
|
||||
import org.apache.druid.query.groupby.GroupByQuery;
|
||||
import org.apache.druid.query.movingaverage.test.TestConfig;
|
||||
import org.apache.druid.query.timeseries.TimeseriesQuery;
|
||||
@ -70,6 +69,7 @@ import org.apache.druid.server.ClientQuerySegmentWalker;
|
||||
import org.apache.druid.server.initialization.ServerConfig;
|
||||
import org.apache.druid.timeline.TimelineLookup;
|
||||
import org.hamcrest.core.IsInstanceOf;
|
||||
import org.joda.time.Interval;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
@ -132,7 +132,33 @@ public class MovingAverageQueryTest
|
||||
binder.bindConstant().annotatedWith(Names.named("serviceName")).to("queryTest");
|
||||
binder.bindConstant().annotatedWith(Names.named("servicePort")).to(0);
|
||||
binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(1);
|
||||
binder.bind(QuerySegmentWalker.class).toProvider(Providers.of(null));
|
||||
binder.bind(QuerySegmentWalker.class).toProvider(Providers.of(new QuerySegmentWalker()
|
||||
{
|
||||
@Override
|
||||
public <T> QueryRunner<T> getQueryRunnerForIntervals(Query<T> query, Iterable<Interval> intervals)
|
||||
{
|
||||
return new QueryRunner<T>()
|
||||
{
|
||||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
public Sequence<T> run(QueryPlus queryPlus, Map responseContext)
|
||||
{
|
||||
if (query instanceof GroupByQuery) {
|
||||
return (Sequence<T>) Sequences.simple(groupByResults);
|
||||
} else if (query instanceof TimeseriesQuery) {
|
||||
return (Sequence<T>) Sequences.simple(timeseriesResults);
|
||||
}
|
||||
throw new UnsupportedOperationException("unexpected query type " + query.getType());
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> QueryRunner<T> getQueryRunnerForSegments(Query<T> query, Iterable<SegmentDescriptor> specs)
|
||||
{
|
||||
return getQueryRunnerForIntervals(query, null);
|
||||
}
|
||||
}));
|
||||
}
|
||||
);
|
||||
|
||||
@ -293,46 +319,6 @@ public class MovingAverageQueryTest
|
||||
@Test
|
||||
public void testQuery() throws IOException
|
||||
{
|
||||
|
||||
|
||||
// create mocks for nested queries
|
||||
@SuppressWarnings("unused")
|
||||
|
||||
MockUp<GroupByQuery> groupByQuery = new MockUp<GroupByQuery>()
|
||||
{
|
||||
@Mock
|
||||
public QueryRunner getRunner(QuerySegmentWalker walker)
|
||||
{
|
||||
return new QueryRunner()
|
||||
{
|
||||
@Override
|
||||
public Sequence run(QueryPlus queryPlus, Map responseContext)
|
||||
{
|
||||
return Sequences.simple(groupByResults);
|
||||
}
|
||||
};
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
@SuppressWarnings("unused")
|
||||
MockUp<TimeseriesQuery> timeseriesQuery = new MockUp<TimeseriesQuery>()
|
||||
{
|
||||
@Mock
|
||||
public QueryRunner getRunner(QuerySegmentWalker walker)
|
||||
{
|
||||
return new QueryRunner()
|
||||
{
|
||||
@Override
|
||||
public Sequence run(QueryPlus queryPlus, Map responseContext)
|
||||
{
|
||||
return Sequences.simple(timeseriesResults);
|
||||
}
|
||||
};
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
Query<?> query = jsonMapper.readValue(getQueryString(), Query.class);
|
||||
assertThat(query, IsInstanceOf.instanceOf(getExpectedQueryType()));
|
||||
|
||||
|
@ -1740,8 +1740,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||
).andReturn(Futures.immediateFailedFuture(new RuntimeException())).times(2);
|
||||
taskQueue.shutdown(
|
||||
EasyMock.contains("sequenceName-0"),
|
||||
EasyMock.eq("All tasks in group [%s] failed to transition to publishing state"),
|
||||
EasyMock.eq(0)
|
||||
EasyMock.eq("Task [%s] failed to respond to [set end offsets] in a timely manner, killing task"),
|
||||
EasyMock.contains("sequenceName-0")
|
||||
);
|
||||
expectLastCall().times(2);
|
||||
expect(taskQueue.add(capture(captured))).andReturn(true).times(2);
|
||||
|
@ -1204,6 +1204,9 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
||||
|
||||
expect(recordSupplier.poll(anyLong())).andReturn(records.subList(2, 13)).once();
|
||||
|
||||
recordSupplier.close();
|
||||
expectLastCall().once();
|
||||
|
||||
replayAll();
|
||||
|
||||
final KinesisIndexTask task = createTask(
|
||||
@ -1360,6 +1363,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
||||
|
||||
expect(recordSupplier.poll(anyLong())).andReturn(records.subList(2, 13)).once();
|
||||
|
||||
recordSupplier.close();
|
||||
expectLastCall().once();
|
||||
|
||||
replayAll();
|
||||
|
||||
@ -1533,7 +1538,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
||||
.once();
|
||||
|
||||
recordSupplier.close();
|
||||
expectLastCall().once();
|
||||
expectLastCall().atLeastOnce();
|
||||
|
||||
replayAll();
|
||||
|
||||
@ -1896,6 +1901,9 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
||||
.andReturn(Collections.emptyList())
|
||||
.anyTimes();
|
||||
|
||||
recordSupplier.close();
|
||||
expectLastCall().once();
|
||||
|
||||
replayAll();
|
||||
|
||||
final KinesisIndexTask task1 = createTask(
|
||||
@ -2015,6 +2023,9 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
||||
recordSupplier.seek(anyObject(), anyString());
|
||||
expectLastCall().anyTimes();
|
||||
|
||||
recordSupplier.close();
|
||||
expectLastCall().once();
|
||||
|
||||
// simulate 1 record at a time
|
||||
expect(recordSupplier.poll(anyLong())).andReturn(Collections.singletonList(records.get(0)))
|
||||
.once()
|
||||
|
@ -2026,8 +2026,8 @@ public class KinesisSupervisorTest extends EasyMockSupport
|
||||
).andReturn(Futures.immediateFailedFuture(new RuntimeException())).times(2);
|
||||
taskQueue.shutdown(
|
||||
EasyMock.contains("sequenceName-0"),
|
||||
EasyMock.eq("All tasks in group [%s] failed to transition to publishing state"),
|
||||
EasyMock.eq(0)
|
||||
EasyMock.eq("Task [%s] failed to respond to [set end offsets] in a timely manner, killing task"),
|
||||
EasyMock.contains("sequenceName-0")
|
||||
);
|
||||
EasyMock.expectLastCall().times(2);
|
||||
EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(2);
|
||||
@ -3062,7 +3062,7 @@ public class KinesisSupervisorTest extends EasyMockSupport
|
||||
.anyTimes();
|
||||
expect(taskClient.setEndOffsetsAsync(
|
||||
EasyMock.anyString(),
|
||||
EasyMock.eq(ImmutableMap.of("0", "10")),
|
||||
EasyMock.eq(ImmutableMap.of(shardId1, "10")),
|
||||
EasyMock.anyBoolean()
|
||||
))
|
||||
.andReturn(Futures.immediateFuture(true))
|
||||
|
4
pom.xml
4
pom.xml
@ -94,7 +94,7 @@
|
||||
<slf4j.version>1.7.12</slf4j.version>
|
||||
<!-- If compiling with different hadoop version also modify default hadoop coordinates in TaskConfig.java -->
|
||||
<hadoop.compile.version>2.8.3</hadoop.compile.version>
|
||||
<powermock.version>1.6.6</powermock.version>
|
||||
<powermock.version>2.0.2</powermock.version>
|
||||
<aws.sdk.version>1.11.199</aws.sdk.version>
|
||||
<caffeine.version>2.5.5</caffeine.version>
|
||||
<!-- When upgrading ZK, edit docs and integration tests as well (integration-tests/docker-base/setup.sh) -->
|
||||
@ -790,7 +790,7 @@
|
||||
<dependency>
|
||||
<groupId>org.easymock</groupId>
|
||||
<artifactId>easymock</artifactId>
|
||||
<version>3.4</version>
|
||||
<version>4.0.2</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
|
@ -207,7 +207,7 @@ public class ChainedExecutionQueryRunnerTest
|
||||
final CountDownLatch queriesInterrupted = new CountDownLatch(2);
|
||||
final CountDownLatch queryIsRegistered = new CountDownLatch(1);
|
||||
|
||||
Capture<ListenableFuture> capturedFuture = new Capture<>();
|
||||
Capture<ListenableFuture> capturedFuture = Capture.newInstance();
|
||||
QueryWatcher watcher = EasyMock.createStrictMock(QueryWatcher.class);
|
||||
watcher.registerQuery(
|
||||
EasyMock.anyObject(),
|
||||
|
@ -1874,8 +1874,8 @@ public class CachingClusteredClientTest
|
||||
.andReturn(expectations.getQueryRunner())
|
||||
.times(0, 1);
|
||||
|
||||
final Capture<? extends QueryPlus> capture = new Capture();
|
||||
final Capture<? extends Map> context = new Capture();
|
||||
final Capture<? extends QueryPlus> capture = Capture.newInstance();
|
||||
final Capture<? extends Map> context = Capture.newInstance();
|
||||
QueryRunner queryable = expectations.getQueryRunner();
|
||||
|
||||
if (query instanceof TimeseriesQuery) {
|
||||
@ -2023,8 +2023,8 @@ public class CachingClusteredClientTest
|
||||
.andReturn(expectations.getQueryRunner())
|
||||
.once();
|
||||
|
||||
final Capture<? extends QueryPlus> capture = new Capture();
|
||||
final Capture<? extends Map> context = new Capture();
|
||||
final Capture<? extends QueryPlus> capture = Capture.newInstance();
|
||||
final Capture<? extends Map> context = Capture.newInstance();
|
||||
queryCaptures.add(capture);
|
||||
QueryRunner queryable = expectations.getQueryRunner();
|
||||
|
||||
|
Loading…
x
Reference in New Issue
Block a user