HDFS-13388. RequestHedgingProxyProvider calls multiple configured NNs all the time. Contributed by Jinglun.

(cherry picked from commit 63803e7051)
This commit is contained in:
Inigo Goiri 2018-04-22 19:49:35 -07:00
parent 6ce89ea2a8
commit c649c76566
2 changed files with 93 additions and 0 deletions

View File

@ -79,6 +79,20 @@ public class RequestHedgingProxyProvider<T> extends
public Object
invoke(Object proxy, final Method method, final Object[] args)
throws Throwable {
if (currentUsedProxy != null) {
try {
Object retVal = method.invoke(currentUsedProxy.proxy, args);
LOG.debug("Invocation successful on [{}]",
currentUsedProxy.proxyInfo);
return retVal;
} catch (InvocationTargetException ex) {
Exception unwrappedException = unwrapInvocationTargetException(ex);
logProxyException(unwrappedException, currentUsedProxy.proxyInfo);
LOG.trace("Unsuccessful invocation on [{}]",
currentUsedProxy.proxyInfo);
throw unwrappedException;
}
}
Map<Future<Object>, ProxyInfo<T>> proxyMap = new HashMap<>();
int numAttempts = 0;

View File

@ -43,10 +43,15 @@ import org.junit.Assert;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
import static org.junit.Assert.assertEquals;
import org.mockito.Matchers;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import static org.mockito.Mockito.when;
import static org.mockito.Mockito.mock;
import com.google.common.collect.Lists;
@ -99,6 +104,80 @@ public class TestRequestHedgingProxyProvider {
Mockito.verify(goodMock).getStats();
}
@Test
public void testRequestNNAfterOneSuccess() throws Exception {
final AtomicInteger goodCount = new AtomicInteger(0);
final AtomicInteger badCount = new AtomicInteger(0);
final ClientProtocol goodMock = mock(ClientProtocol.class);
when(goodMock.getStats()).thenAnswer(new Answer<long[]>() {
@Override
public long[] answer(InvocationOnMock invocation) throws Throwable {
goodCount.incrementAndGet();
Thread.sleep(1000);
return new long[]{1};
}
});
final ClientProtocol badMock = mock(ClientProtocol.class);
when(badMock.getStats()).thenAnswer(new Answer<long[]>() {
@Override
public long[] answer(InvocationOnMock invocation) throws Throwable {
badCount.incrementAndGet();
throw new IOException("Bad mock !!");
}
});
RequestHedgingProxyProvider<ClientProtocol> provider =
new RequestHedgingProxyProvider<>(conf, nnUri, ClientProtocol.class,
createFactory(badMock, goodMock));
ClientProtocol proxy = provider.getProxy().proxy;
proxy.getStats();
assertEquals(1, goodCount.get());
assertEquals(1, badCount.get());
// We will only use the successful proxy after a successful invocation.
proxy.getStats();
assertEquals(2, goodCount.get());
assertEquals(1, badCount.get());
}
@Test
public void testExceptionInfo() throws Exception {
final ClientProtocol goodMock = mock(ClientProtocol.class);
when(goodMock.getStats()).thenAnswer(new Answer<long[]>() {
private boolean first = true;
@Override
public long[] answer(InvocationOnMock invocation)
throws Throwable {
if (first) {
Thread.sleep(1000);
first = false;
return new long[] {1};
} else {
throw new IOException("Expected Exception Info");
}
}
});
final ClientProtocol badMock = mock(ClientProtocol.class);
when(badMock.getStats()).thenAnswer(new Answer<long[]>() {
@Override
public long[] answer(InvocationOnMock invocation)
throws Throwable {
throw new IOException("Bad Mock! This is Standby!");
}
});
RequestHedgingProxyProvider<ClientProtocol> provider =
new RequestHedgingProxyProvider<>(conf, nnUri, ClientProtocol.class,
createFactory(badMock, goodMock));
ClientProtocol proxy = provider.getProxy().proxy;
proxy.getStats();
// Test getting the exception when the successful proxy encounters one.
try {
proxy.getStats();
} catch (IOException e) {
assertExceptionContains("Expected Exception Info", e);
}
}
@Test
public void testHedgingWhenOneIsSlow() throws Exception {
final ClientProtocol goodMock = Mockito.mock(ClientProtocol.class);