HDFS-15032. Properly handle InvocationTargetExceptions in the proxy created by ProxyCombiner. This fixes a bug encountered by the HDFS balancer when used with Observer Nodes. Contributed by Erik Krogen.
(cherry picked from c174d50b30
)
This commit is contained in:
parent
0f25cbbb25
commit
3402c87353
|
@ -17,9 +17,11 @@
|
|||
*/
|
||||
package org.apache.hadoop.ipc;
|
||||
|
||||
import com.google.common.base.Joiner;
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.InvocationHandler;
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.lang.reflect.Method;
|
||||
import java.lang.reflect.Proxy;
|
||||
|
||||
|
@ -74,7 +76,8 @@ public final class ProxyCombiner {
|
|||
+ combinedProxyInterface + " do not cover method " + m);
|
||||
}
|
||||
|
||||
InvocationHandler handler = new CombinedProxyInvocationHandler(proxies);
|
||||
InvocationHandler handler =
|
||||
new CombinedProxyInvocationHandler(combinedProxyInterface, proxies);
|
||||
return (T) Proxy.newProxyInstance(combinedProxyInterface.getClassLoader(),
|
||||
new Class[] {combinedProxyInterface}, handler);
|
||||
}
|
||||
|
@ -82,9 +85,12 @@ public final class ProxyCombiner {
|
|||
private static final class CombinedProxyInvocationHandler
|
||||
implements RpcInvocationHandler {
|
||||
|
||||
private final Class<?> proxyInterface;
|
||||
private final Object[] proxies;
|
||||
|
||||
private CombinedProxyInvocationHandler(Object[] proxies) {
|
||||
private CombinedProxyInvocationHandler(Class<?> proxyInterface,
|
||||
Object[] proxies) {
|
||||
this.proxyInterface = proxyInterface;
|
||||
this.proxies = proxies;
|
||||
}
|
||||
|
||||
|
@ -97,6 +103,8 @@ public final class ProxyCombiner {
|
|||
return method.invoke(underlyingProxy, args);
|
||||
} catch (IllegalAccessException|IllegalArgumentException e) {
|
||||
lastException = e;
|
||||
} catch (InvocationTargetException ite) {
|
||||
throw ite.getCause();
|
||||
}
|
||||
}
|
||||
// This shouldn't happen since the method coverage was verified in build()
|
||||
|
@ -116,6 +124,12 @@ public final class ProxyCombiner {
|
|||
return RPC.getConnectionIdForProxy(proxies[0]);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "CombinedProxy[" + proxyInterface.getSimpleName() + "]["
|
||||
+ Joiner.on(",").join(proxies) + "]";
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
MultipleIOException.Builder exceptionBuilder =
|
||||
|
|
|
@ -125,10 +125,25 @@ public class TestBalancerWithHANameNodes {
|
|||
/**
|
||||
* Test Balancer with ObserverNodes.
|
||||
*/
|
||||
@Test(timeout = 60000)
|
||||
@Test(timeout = 120000)
|
||||
public void testBalancerWithObserver() throws Exception {
|
||||
testBalancerWithObserver(false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test Balancer with ObserverNodes when one has failed.
|
||||
*/
|
||||
@Test(timeout = 180000)
|
||||
public void testBalancerWithObserverWithFailedNode() throws Exception {
|
||||
testBalancerWithObserver(true);
|
||||
}
|
||||
|
||||
private void testBalancerWithObserver(boolean withObserverFailure)
|
||||
throws Exception {
|
||||
final Configuration conf = new HdfsConfiguration();
|
||||
TestBalancer.initConf(conf);
|
||||
// Avoid the same FS being reused between tests
|
||||
conf.setBoolean("fs.hdfs.impl.disable.cache", true);
|
||||
|
||||
MiniQJMHACluster qjmhaCluster = null;
|
||||
try {
|
||||
|
@ -142,6 +157,10 @@ public class TestBalancerWithHANameNodes {
|
|||
namesystemSpies.add(
|
||||
NameNodeAdapter.spyOnNamesystem(cluster.getNameNode(i)));
|
||||
}
|
||||
if (withObserverFailure) {
|
||||
// First observer NN is at index 2
|
||||
cluster.shutdownNameNode(2);
|
||||
}
|
||||
|
||||
DistributedFileSystem dfs = HATestUtil.configureObserverReadFs(
|
||||
cluster, conf, ObserverReadProxyProvider.class, true);
|
||||
|
@ -149,9 +168,10 @@ public class TestBalancerWithHANameNodes {
|
|||
|
||||
doTest(conf);
|
||||
for (int i = 0; i < cluster.getNumNameNodes(); i++) {
|
||||
// First observer node is at idx 2 so it should get both getBlocks calls
|
||||
// all other NameNodes should see 0 getBlocks calls
|
||||
int expectedCount = (i == 2) ? 2 : 0;
|
||||
// First observer node is at idx 2, or 3 if 2 has been shut down
|
||||
// It should get both getBlocks calls, all other NNs should see 0 calls
|
||||
int expectedObserverIdx = withObserverFailure ? 3 : 2;
|
||||
int expectedCount = (i == expectedObserverIdx) ? 2 : 0;
|
||||
verify(namesystemSpies.get(i), times(expectedCount))
|
||||
.getBlocks(any(), anyLong(), anyLong());
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue