HADOOP-14681. Remove MockitoMaker class. Contributed by Andras Bokor.
This commit is contained in:
parent
218b1b33ff
commit
cca51e916b
|
@ -1,132 +0,0 @@
|
||||||
/**
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
|
||||||
* or more contributor license agreements. See the NOTICE file
|
|
||||||
* distributed with this work for additional information
|
|
||||||
* regarding copyright ownership. The ASF licenses this file
|
|
||||||
* to you under the Apache License, Version 2.0 (the
|
|
||||||
* "License"); you may not use this file except in compliance
|
|
||||||
* with the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
package org.apache.hadoop.test;
|
|
||||||
|
|
||||||
import static org.mockito.Mockito.*;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Helper class to create one-liner stubs, so that instead of: <pre>
|
|
||||||
* SomeType someDescriptiveMock = mock(SomeType.class);
|
|
||||||
* when(someDescriptiveMock.someMethod()).thenReturn(someValue);</pre>
|
|
||||||
* <p>You can now do: <pre>
|
|
||||||
* SomeType someDescriptiveMock = make(stub(SomeType.class)
|
|
||||||
* .returning(someValue).from.someMethod());</pre>
|
|
||||||
*/
|
|
||||||
public class MockitoMaker {
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Create a mock object from a mocked method call.
|
|
||||||
*
|
|
||||||
* @param <T> type of mocked object
|
|
||||||
* @param methodCall for mocked object
|
|
||||||
* @return mocked object
|
|
||||||
*/
|
|
||||||
@SuppressWarnings("unchecked")
|
|
||||||
public static <T> T make(Object methodCall) {
|
|
||||||
StubBuilder<T> sb = StubBuilder.current();
|
|
||||||
when(methodCall).thenReturn(sb.firstReturn, sb.laterReturns);
|
|
||||||
return (T) StubBuilder.current().from;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Create a stub builder of a mocked object.
|
|
||||||
*
|
|
||||||
* @param <T> type of the target object to be mocked
|
|
||||||
* @param target class of the target object to be mocked
|
|
||||||
* @return the stub builder of the mocked object
|
|
||||||
*/
|
|
||||||
public static <T> StubBuilder<T> stub(Class<T> target) {
|
|
||||||
return new StubBuilder<T>(mock(target));
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Builder class for stubs
|
|
||||||
* @param <T> type of the object to be mocked
|
|
||||||
*/
|
|
||||||
public static class StubBuilder<T> {
|
|
||||||
|
|
||||||
/**
|
|
||||||
* The target mock object
|
|
||||||
*/
|
|
||||||
public final T from;
|
|
||||||
|
|
||||||
// We want to be able to use this even when the tests are run in parallel.
|
|
||||||
@SuppressWarnings("rawtypes")
|
|
||||||
private static final ThreadLocal<StubBuilder> tls =
|
|
||||||
new ThreadLocal<StubBuilder>() {
|
|
||||||
@Override protected StubBuilder initialValue() {
|
|
||||||
return new StubBuilder();
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
private Object firstReturn = null;
|
|
||||||
private Object[] laterReturns = {};
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Default constructor for the initial stub builder
|
|
||||||
*/
|
|
||||||
public StubBuilder() {
|
|
||||||
this.from = null;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Construct a stub builder with a mock instance
|
|
||||||
*
|
|
||||||
* @param mockInstance the mock object
|
|
||||||
*/
|
|
||||||
public StubBuilder(T mockInstance) {
|
|
||||||
tls.set(this);
|
|
||||||
this.from = mockInstance;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Get the current stub builder from thread local
|
|
||||||
*
|
|
||||||
* @param <T>
|
|
||||||
* @return the stub builder of the mocked object
|
|
||||||
*/
|
|
||||||
@SuppressWarnings("unchecked")
|
|
||||||
public static <T> StubBuilder<T> current() {
|
|
||||||
return tls.get();
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Set the return value for the current stub builder
|
|
||||||
*
|
|
||||||
* @param value the return value
|
|
||||||
* @return the stub builder
|
|
||||||
*/
|
|
||||||
public StubBuilder<T> returning(Object value) {
|
|
||||||
this.firstReturn = value;
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Set the return values for the current stub builder
|
|
||||||
*
|
|
||||||
* @param value the first return value
|
|
||||||
* @param values the return values for later invocations
|
|
||||||
* @return the stub builder
|
|
||||||
*/
|
|
||||||
public StubBuilder<T> returning(Object value, Object... values) {
|
|
||||||
this.firstReturn = value;
|
|
||||||
this.laterReturns = values;
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -29,7 +29,6 @@ import static org.junit.Assert.*;
|
||||||
|
|
||||||
import static org.mockito.Mockito.*;
|
import static org.mockito.Mockito.*;
|
||||||
|
|
||||||
import static org.apache.hadoop.test.MockitoMaker.*;
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
@ -83,13 +82,14 @@ public class TestDiskChecker {
|
||||||
|
|
||||||
private void _mkdirs(boolean exists, FsPermission before, FsPermission after)
|
private void _mkdirs(boolean exists, FsPermission before, FsPermission after)
|
||||||
throws Throwable {
|
throws Throwable {
|
||||||
File localDir = make(stub(File.class).returning(exists).from.exists());
|
File localDir = mock(File.class);
|
||||||
|
when(localDir.exists()).thenReturn(exists);
|
||||||
when(localDir.mkdir()).thenReturn(true);
|
when(localDir.mkdir()).thenReturn(true);
|
||||||
Path dir = mock(Path.class); // use default stubs
|
Path dir = mock(Path.class); // use default stubs
|
||||||
LocalFileSystem fs = make(stub(LocalFileSystem.class)
|
LocalFileSystem fs = mock(LocalFileSystem.class);
|
||||||
.returning(localDir).from.pathToFile(dir));
|
when(fs.pathToFile(dir)).thenReturn(localDir);
|
||||||
FileStatus stat = make(stub(FileStatus.class)
|
FileStatus stat = mock(FileStatus.class);
|
||||||
.returning(after).from.getPermission());
|
when(stat.getPermission()).thenReturn(after);
|
||||||
when(fs.getFileStatus(dir)).thenReturn(stat);
|
when(fs.getFileStatus(dir)).thenReturn(stat);
|
||||||
|
|
||||||
try {
|
try {
|
||||||
|
|
|
@ -23,7 +23,6 @@ import org.apache.hadoop.mapreduce.v2.app.job.Task;
|
||||||
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
|
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
|
||||||
|
|
||||||
import static org.apache.hadoop.test.MetricsAsserts.*;
|
import static org.apache.hadoop.test.MetricsAsserts.*;
|
||||||
import static org.apache.hadoop.test.MockitoMaker.*;
|
|
||||||
|
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
|
@ -33,10 +32,10 @@ public class TestMRAppMetrics {
|
||||||
|
|
||||||
@Test public void testNames() {
|
@Test public void testNames() {
|
||||||
Job job = mock(Job.class);
|
Job job = mock(Job.class);
|
||||||
Task mapTask = make(stub(Task.class).returning(TaskType.MAP).
|
Task mapTask = mock(Task.class);
|
||||||
from.getType());
|
when(mapTask.getType()).thenReturn(TaskType.MAP);
|
||||||
Task reduceTask = make(stub(Task.class).returning(TaskType.REDUCE).
|
Task reduceTask = mock(Task.class);
|
||||||
from.getType());
|
when(reduceTask.getType()).thenReturn(TaskType.REDUCE);
|
||||||
MRAppMetrics metrics = MRAppMetrics.create();
|
MRAppMetrics metrics = MRAppMetrics.create();
|
||||||
|
|
||||||
metrics.submittedJob(job);
|
metrics.submittedJob(job);
|
||||||
|
|
|
@ -27,7 +27,6 @@ import org.junit.Test;
|
||||||
import static org.junit.Assert.*;
|
import static org.junit.Assert.*;
|
||||||
|
|
||||||
import static org.mockito.Mockito.*;
|
import static org.mockito.Mockito.*;
|
||||||
import static org.apache.hadoop.test.MockitoMaker.*;
|
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.BlockLocation;
|
import org.apache.hadoop.fs.BlockLocation;
|
||||||
|
@ -80,12 +79,14 @@ public class TestMRCJCFileInputFormat {
|
||||||
@Test
|
@Test
|
||||||
public void testNumInputFiles() throws Exception {
|
public void testNumInputFiles() throws Exception {
|
||||||
Configuration conf = spy(new Configuration());
|
Configuration conf = spy(new Configuration());
|
||||||
Job job = make(stub(Job.class).returning(conf).from.getConfiguration());
|
Job mockedJob = mock(Job.class);
|
||||||
FileStatus stat = make(stub(FileStatus.class).returning(0L).from.getLen());
|
when(mockedJob.getConfiguration()).thenReturn(conf);
|
||||||
|
FileStatus stat = mock(FileStatus.class);
|
||||||
|
when(stat.getLen()).thenReturn(0L);
|
||||||
TextInputFormat ispy = spy(new TextInputFormat());
|
TextInputFormat ispy = spy(new TextInputFormat());
|
||||||
doReturn(Arrays.asList(stat)).when(ispy).listStatus(job);
|
doReturn(Arrays.asList(stat)).when(ispy).listStatus(mockedJob);
|
||||||
|
|
||||||
ispy.getSplits(job);
|
ispy.getSplits(mockedJob);
|
||||||
verify(conf).setLong(FileInputFormat.NUM_INPUT_FILES, 1);
|
verify(conf).setLong(FileInputFormat.NUM_INPUT_FILES, 1);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -20,14 +20,14 @@ package org.apache.hadoop.mapred;
|
||||||
import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
|
import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
|
||||||
import static org.apache.hadoop.test.MetricsAsserts.assertGauge;
|
import static org.apache.hadoop.test.MetricsAsserts.assertGauge;
|
||||||
import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
|
import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
|
||||||
import static org.apache.hadoop.test.MockitoMaker.make;
|
|
||||||
import static org.apache.hadoop.test.MockitoMaker.stub;
|
|
||||||
import static org.junit.Assert.assertTrue;
|
import static org.junit.Assert.assertTrue;
|
||||||
import static org.jboss.netty.buffer.ChannelBuffers.wrappedBuffer;
|
import static org.jboss.netty.buffer.ChannelBuffers.wrappedBuffer;
|
||||||
import static org.jboss.netty.handler.codec.http.HttpResponseStatus.OK;
|
import static org.jboss.netty.handler.codec.http.HttpResponseStatus.OK;
|
||||||
import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1;
|
import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1;
|
||||||
import static org.junit.Assert.assertEquals;
|
import static org.junit.Assert.assertEquals;
|
||||||
import static org.junit.Assume.assumeTrue;
|
import static org.junit.Assume.assumeTrue;
|
||||||
|
import static org.mockito.Mockito.mock;
|
||||||
|
import static org.mockito.Mockito.when;
|
||||||
|
|
||||||
import java.io.DataInputStream;
|
import java.io.DataInputStream;
|
||||||
import java.io.EOFException;
|
import java.io.EOFException;
|
||||||
|
@ -188,8 +188,8 @@ public class TestShuffleHandler {
|
||||||
public void testShuffleMetrics() throws Exception {
|
public void testShuffleMetrics() throws Exception {
|
||||||
MetricsSystem ms = new MetricsSystemImpl();
|
MetricsSystem ms = new MetricsSystemImpl();
|
||||||
ShuffleHandler sh = new ShuffleHandler(ms);
|
ShuffleHandler sh = new ShuffleHandler(ms);
|
||||||
ChannelFuture cf = make(stub(ChannelFuture.class).
|
ChannelFuture cf = mock(ChannelFuture.class);
|
||||||
returning(true, false).from.isSuccess());
|
when(cf.isSuccess()).thenReturn(true).thenReturn(false);
|
||||||
|
|
||||||
sh.metrics.shuffleConnections.incr();
|
sh.metrics.shuffleConnections.incr();
|
||||||
sh.metrics.shuffleOutputBytes.incr(1*MiB);
|
sh.metrics.shuffleOutputBytes.incr(1*MiB);
|
||||||
|
@ -1080,10 +1080,10 @@ public class TestShuffleHandler {
|
||||||
new ArrayList<ShuffleHandler.ReduceMapFileCount>();
|
new ArrayList<ShuffleHandler.ReduceMapFileCount>();
|
||||||
|
|
||||||
final ChannelHandlerContext mockCtx =
|
final ChannelHandlerContext mockCtx =
|
||||||
Mockito.mock(ChannelHandlerContext.class);
|
mock(ChannelHandlerContext.class);
|
||||||
final MessageEvent mockEvt = Mockito.mock(MessageEvent.class);
|
final MessageEvent mockEvt = mock(MessageEvent.class);
|
||||||
final Channel mockCh = Mockito.mock(AbstractChannel.class);
|
final Channel mockCh = mock(AbstractChannel.class);
|
||||||
final ChannelPipeline mockPipeline = Mockito.mock(ChannelPipeline.class);
|
final ChannelPipeline mockPipeline = mock(ChannelPipeline.class);
|
||||||
|
|
||||||
// Mock HttpRequest and ChannelFuture
|
// Mock HttpRequest and ChannelFuture
|
||||||
final HttpRequest mockHttpRequest = createMockHttpRequest();
|
final HttpRequest mockHttpRequest = createMockHttpRequest();
|
||||||
|
@ -1094,16 +1094,16 @@ public class TestShuffleHandler {
|
||||||
|
|
||||||
// Mock Netty Channel Context and Channel behavior
|
// Mock Netty Channel Context and Channel behavior
|
||||||
Mockito.doReturn(mockCh).when(mockCtx).getChannel();
|
Mockito.doReturn(mockCh).when(mockCtx).getChannel();
|
||||||
Mockito.when(mockCh.getPipeline()).thenReturn(mockPipeline);
|
when(mockCh.getPipeline()).thenReturn(mockPipeline);
|
||||||
Mockito.when(mockPipeline.get(
|
when(mockPipeline.get(
|
||||||
Mockito.any(String.class))).thenReturn(timerHandler);
|
Mockito.any(String.class))).thenReturn(timerHandler);
|
||||||
Mockito.when(mockCtx.getChannel()).thenReturn(mockCh);
|
when(mockCtx.getChannel()).thenReturn(mockCh);
|
||||||
Mockito.doReturn(mockFuture).when(mockCh).write(Mockito.any(Object.class));
|
Mockito.doReturn(mockFuture).when(mockCh).write(Mockito.any(Object.class));
|
||||||
Mockito.when(mockCh.write(Object.class)).thenReturn(mockFuture);
|
when(mockCh.write(Object.class)).thenReturn(mockFuture);
|
||||||
|
|
||||||
//Mock MessageEvent behavior
|
//Mock MessageEvent behavior
|
||||||
Mockito.doReturn(mockCh).when(mockEvt).getChannel();
|
Mockito.doReturn(mockCh).when(mockEvt).getChannel();
|
||||||
Mockito.when(mockEvt.getChannel()).thenReturn(mockCh);
|
when(mockEvt.getChannel()).thenReturn(mockCh);
|
||||||
Mockito.doReturn(mockHttpRequest).when(mockEvt).getMessage();
|
Mockito.doReturn(mockHttpRequest).when(mockEvt).getMessage();
|
||||||
|
|
||||||
final ShuffleHandler sh = new MockShuffleHandler();
|
final ShuffleHandler sh = new MockShuffleHandler();
|
||||||
|
@ -1127,8 +1127,8 @@ public class TestShuffleHandler {
|
||||||
|
|
||||||
public ChannelFuture createMockChannelFuture(Channel mockCh,
|
public ChannelFuture createMockChannelFuture(Channel mockCh,
|
||||||
final List<ShuffleHandler.ReduceMapFileCount> listenerList) {
|
final List<ShuffleHandler.ReduceMapFileCount> listenerList) {
|
||||||
final ChannelFuture mockFuture = Mockito.mock(ChannelFuture.class);
|
final ChannelFuture mockFuture = mock(ChannelFuture.class);
|
||||||
Mockito.when(mockFuture.getChannel()).thenReturn(mockCh);
|
when(mockFuture.getChannel()).thenReturn(mockCh);
|
||||||
Mockito.doReturn(true).when(mockFuture).isSuccess();
|
Mockito.doReturn(true).when(mockFuture).isSuccess();
|
||||||
Mockito.doAnswer(new Answer() {
|
Mockito.doAnswer(new Answer() {
|
||||||
@Override
|
@Override
|
||||||
|
@ -1146,7 +1146,7 @@ public class TestShuffleHandler {
|
||||||
}
|
}
|
||||||
|
|
||||||
public HttpRequest createMockHttpRequest() {
|
public HttpRequest createMockHttpRequest() {
|
||||||
HttpRequest mockHttpRequest = Mockito.mock(HttpRequest.class);
|
HttpRequest mockHttpRequest = mock(HttpRequest.class);
|
||||||
Mockito.doReturn(HttpMethod.GET).when(mockHttpRequest).getMethod();
|
Mockito.doReturn(HttpMethod.GET).when(mockHttpRequest).getMethod();
|
||||||
Mockito.doAnswer(new Answer() {
|
Mockito.doAnswer(new Answer() {
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -21,8 +21,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
|
||||||
import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
|
import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
|
||||||
import static org.apache.hadoop.test.MetricsAsserts.assertGauge;
|
import static org.apache.hadoop.test.MetricsAsserts.assertGauge;
|
||||||
import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
|
import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
|
||||||
import static org.apache.hadoop.test.MockitoMaker.make;
|
|
||||||
import static org.apache.hadoop.test.MockitoMaker.stub;
|
|
||||||
import static org.junit.Assert.assertNull;
|
import static org.junit.Assert.assertNull;
|
||||||
import static org.mockito.Mockito.mock;
|
import static org.mockito.Mockito.mock;
|
||||||
import static org.mockito.Mockito.when;
|
import static org.mockito.Mockito.when;
|
||||||
|
@ -226,8 +224,8 @@ public class TestQueueMetrics {
|
||||||
|
|
||||||
QueueMetrics parentMetrics =
|
QueueMetrics parentMetrics =
|
||||||
QueueMetrics.forQueue(ms, parentQueueName, null, true, conf);
|
QueueMetrics.forQueue(ms, parentQueueName, null, true, conf);
|
||||||
Queue parentQueue = make(stub(Queue.class).returning(parentMetrics).
|
Queue parentQueue = mock(Queue.class);
|
||||||
from.getMetrics());
|
when(parentQueue.getMetrics()).thenReturn(parentMetrics);
|
||||||
QueueMetrics metrics =
|
QueueMetrics metrics =
|
||||||
QueueMetrics.forQueue(ms, leafQueueName, parentQueue, true, conf);
|
QueueMetrics.forQueue(ms, leafQueueName, parentQueue, true, conf);
|
||||||
MetricsSource parentQueueSource = queueSource(ms, parentQueueName);
|
MetricsSource parentQueueSource = queueSource(ms, parentQueueName);
|
||||||
|
@ -272,8 +270,8 @@ public class TestQueueMetrics {
|
||||||
|
|
||||||
QueueMetrics parentMetrics =
|
QueueMetrics parentMetrics =
|
||||||
QueueMetrics.forQueue(ms, parentQueueName, null, true, conf);
|
QueueMetrics.forQueue(ms, parentQueueName, null, true, conf);
|
||||||
Queue parentQueue = make(stub(Queue.class).returning(parentMetrics).
|
Queue parentQueue = mock(Queue.class);
|
||||||
from.getMetrics());
|
when(parentQueue.getMetrics()).thenReturn(parentMetrics);
|
||||||
QueueMetrics metrics =
|
QueueMetrics metrics =
|
||||||
QueueMetrics.forQueue(ms, leafQueueName, parentQueue, true, conf);
|
QueueMetrics.forQueue(ms, leafQueueName, parentQueue, true, conf);
|
||||||
MetricsSource parentQueueSource = queueSource(ms, parentQueueName);
|
MetricsSource parentQueueSource = queueSource(ms, parentQueueName);
|
||||||
|
@ -359,8 +357,8 @@ public class TestQueueMetrics {
|
||||||
|
|
||||||
QueueMetrics p1Metrics =
|
QueueMetrics p1Metrics =
|
||||||
QueueMetrics.forQueue(ms, p1, null, true, conf);
|
QueueMetrics.forQueue(ms, p1, null, true, conf);
|
||||||
Queue parentQueue1 = make(stub(Queue.class).returning(p1Metrics).
|
Queue parentQueue1 = mock(Queue.class);
|
||||||
from.getMetrics());
|
when(parentQueue1.getMetrics()).thenReturn(p1Metrics);
|
||||||
QueueMetrics metrics =
|
QueueMetrics metrics =
|
||||||
QueueMetrics.forQueue(ms, leafQueueName, parentQueue1, true, conf);
|
QueueMetrics.forQueue(ms, leafQueueName, parentQueue1, true, conf);
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue