|
@@ -58,6 +58,8 @@ public class RequestHedgingProxyProvider<T> extends
|
|
|
class RequestHedgingInvocationHandler implements InvocationHandler {
|
|
|
|
|
|
final Map<String, ProxyInfo<T>> targetProxies;
|
|
|
+ // Proxy of the active nn
|
|
|
+ private volatile ProxyInfo<T> currentUsedProxy = null;
|
|
|
|
|
|
public RequestHedgingInvocationHandler(
|
|
|
Map<String, ProxyInfo<T>> targetProxies) {
|
|
@@ -79,104 +81,118 @@ 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;
|
|
|
+ // Need double check locking to guarantee thread-safe since
|
|
|
+ // currentUsedProxy is lazily initialized.
|
|
|
+ if (currentUsedProxy == null) {
|
|
|
+ synchronized (this) {
|
|
|
+ if (currentUsedProxy == null) {
|
|
|
+ Map<Future<Object>, ProxyInfo<T>> proxyMap = new HashMap<>();
|
|
|
+ int numAttempts = 0;
|
|
|
|
|
|
- ExecutorService executor = null;
|
|
|
- CompletionService<Object> completionService;
|
|
|
- try {
|
|
|
- // Optimization : if only 2 proxies are configured and one had failed
|
|
|
- // over, then we dont need to create a threadpool etc.
|
|
|
- targetProxies.remove(toIgnore);
|
|
|
- if (targetProxies.size() == 0) {
|
|
|
- LOG.trace("No valid proxies left");
|
|
|
- throw new RemoteException(IOException.class.getName(),
|
|
|
- "No valid proxies left. All NameNode proxies have failed over.");
|
|
|
- }
|
|
|
- if (targetProxies.size() == 1) {
|
|
|
- ProxyInfo<T> proxyInfo = targetProxies.values().iterator().next();
|
|
|
- try {
|
|
|
- currentUsedProxy = proxyInfo;
|
|
|
- Object retVal = method.invoke(proxyInfo.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;
|
|
|
- }
|
|
|
- }
|
|
|
- executor = Executors.newFixedThreadPool(proxies.size());
|
|
|
- completionService = new ExecutorCompletionService<>(executor);
|
|
|
- for (final Map.Entry<String, ProxyInfo<T>> pEntry :
|
|
|
- targetProxies.entrySet()) {
|
|
|
- Callable<Object> c = new Callable<Object>() {
|
|
|
- @Override
|
|
|
- public Object call() throws Exception {
|
|
|
- LOG.trace("Invoking method {} on proxy {}", method,
|
|
|
- pEntry.getValue().proxyInfo);
|
|
|
- return method.invoke(pEntry.getValue().proxy, args);
|
|
|
- }
|
|
|
- };
|
|
|
- proxyMap.put(completionService.submit(c), pEntry.getValue());
|
|
|
- numAttempts++;
|
|
|
- }
|
|
|
+ ExecutorService executor = null;
|
|
|
+ CompletionService<Object> completionService;
|
|
|
+ try {
|
|
|
+ // Optimization : if only 2 proxies are configured and one had
|
|
|
+ // failed
|
|
|
+ // over, then we dont need to create a threadpool etc.
|
|
|
+ targetProxies.remove(toIgnore);
|
|
|
+ if (targetProxies.size() == 0) {
|
|
|
+ LOG.trace("No valid proxies left");
|
|
|
+ throw new RemoteException(IOException.class.getName(),
|
|
|
+ "No valid proxies left. "
|
|
|
+ + "All NameNode proxies have failed over.");
|
|
|
+ }
|
|
|
+ if (targetProxies.size() == 1) {
|
|
|
+ ProxyInfo<T> proxyInfo =
|
|
|
+ targetProxies.values().iterator().next();
|
|
|
+ try {
|
|
|
+ currentUsedProxy = proxyInfo;
|
|
|
+ Object retVal = method.invoke(proxyInfo.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;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ executor = Executors.newFixedThreadPool(proxies.size());
|
|
|
+ completionService = new ExecutorCompletionService<>(executor);
|
|
|
+ for (final Map.Entry<String, ProxyInfo<T>> pEntry : targetProxies
|
|
|
+ .entrySet()) {
|
|
|
+ Callable<Object> c = new Callable<Object>() {
|
|
|
+ @Override
|
|
|
+ public Object call() throws Exception {
|
|
|
+ LOG.trace("Invoking method {} on proxy {}", method,
|
|
|
+ pEntry.getValue().proxyInfo);
|
|
|
+ return method.invoke(pEntry.getValue().proxy, args);
|
|
|
+ }
|
|
|
+ };
|
|
|
+ proxyMap.put(completionService.submit(c), pEntry.getValue());
|
|
|
+ numAttempts++;
|
|
|
+ }
|
|
|
|
|
|
- Map<String, Exception> badResults = new HashMap<>();
|
|
|
- while (numAttempts > 0) {
|
|
|
- Future<Object> callResultFuture = completionService.take();
|
|
|
- Object retVal;
|
|
|
- try {
|
|
|
- currentUsedProxy = proxyMap.get(callResultFuture);
|
|
|
- retVal = callResultFuture.get();
|
|
|
- LOG.debug("Invocation successful on [{}]",
|
|
|
- currentUsedProxy.proxyInfo);
|
|
|
- return retVal;
|
|
|
- } catch (ExecutionException ex) {
|
|
|
- Exception unwrappedException = unwrapExecutionException(ex);
|
|
|
- ProxyInfo<T> tProxyInfo = proxyMap.get(callResultFuture);
|
|
|
- logProxyException(unwrappedException, tProxyInfo.proxyInfo);
|
|
|
- badResults.put(tProxyInfo.proxyInfo, unwrappedException);
|
|
|
- LOG.trace("Unsuccessful invocation on [{}]", tProxyInfo.proxyInfo);
|
|
|
- numAttempts--;
|
|
|
- }
|
|
|
- }
|
|
|
+ Map<String, Exception> badResults = new HashMap<>();
|
|
|
+ while (numAttempts > 0) {
|
|
|
+ Future<Object> callResultFuture = completionService.take();
|
|
|
+ Object retVal;
|
|
|
+ try {
|
|
|
+ currentUsedProxy = proxyMap.get(callResultFuture);
|
|
|
+ retVal = callResultFuture.get();
|
|
|
+ LOG.debug("Invocation successful on [{}]",
|
|
|
+ currentUsedProxy.proxyInfo);
|
|
|
+ return retVal;
|
|
|
+ } catch (ExecutionException ex) {
|
|
|
+ Exception unwrappedException = unwrapExecutionException(ex);
|
|
|
+ ProxyInfo<T> tProxyInfo = proxyMap.get(callResultFuture);
|
|
|
+ logProxyException(unwrappedException, tProxyInfo.proxyInfo);
|
|
|
+ badResults.put(tProxyInfo.proxyInfo, unwrappedException);
|
|
|
+ LOG.trace("Unsuccessful invocation on [{}]",
|
|
|
+ tProxyInfo.proxyInfo);
|
|
|
+ numAttempts--;
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
- // At this point we should have All bad results (Exceptions)
|
|
|
- // Or should have returned with successful result.
|
|
|
- if (badResults.size() == 1) {
|
|
|
- throw badResults.values().iterator().next();
|
|
|
- } else {
|
|
|
- throw new MultiException(badResults);
|
|
|
- }
|
|
|
- } finally {
|
|
|
- if (executor != null) {
|
|
|
- LOG.trace("Shutting down threadpool executor");
|
|
|
- executor.shutdownNow();
|
|
|
+ // At this point we should have All bad results (Exceptions)
|
|
|
+ // Or should have returned with successful result.
|
|
|
+ if (badResults.size() == 1) {
|
|
|
+ throw badResults.values().iterator().next();
|
|
|
+ } else {
|
|
|
+ throw new MultiException(badResults);
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ if (executor != null) {
|
|
|
+ LOG.trace("Shutting down threadpool executor");
|
|
|
+ executor.shutdownNow();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
+ // Because the above synchronized block will return or throw an exception,
|
|
|
+ // so we don't need to do any check to prevent the first initialized
|
|
|
+ // thread from stepping to following codes.
|
|
|
+ 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;
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
|
|
|
-
|
|
|
- private volatile ProxyInfo<T> currentUsedProxy = null;
|
|
|
+ /** A proxy wrapping {@link RequestHedgingInvocationHandler}. */
|
|
|
+ private ProxyInfo<T> currentUsedHandler = null;
|
|
|
private volatile String toIgnore = null;
|
|
|
|
|
|
public RequestHedgingProxyProvider(Configuration conf, URI uri,
|
|
@@ -187,8 +203,8 @@ public class RequestHedgingProxyProvider<T> extends
|
|
|
@SuppressWarnings("unchecked")
|
|
|
@Override
|
|
|
public synchronized ProxyInfo<T> getProxy() {
|
|
|
- if (currentUsedProxy != null) {
|
|
|
- return currentUsedProxy;
|
|
|
+ if (currentUsedHandler != null) {
|
|
|
+ return currentUsedHandler;
|
|
|
}
|
|
|
Map<String, ProxyInfo<T>> targetProxyInfos = new HashMap<>();
|
|
|
StringBuilder combinedInfo = new StringBuilder("[");
|
|
@@ -203,13 +219,16 @@ public class RequestHedgingProxyProvider<T> extends
|
|
|
RequestHedgingInvocationHandler.class.getClassLoader(),
|
|
|
new Class<?>[]{xface},
|
|
|
new RequestHedgingInvocationHandler(targetProxyInfos));
|
|
|
- return new ProxyInfo<T>(wrappedProxy, combinedInfo.toString());
|
|
|
+ currentUsedHandler =
|
|
|
+ new ProxyInfo<T>(wrappedProxy, combinedInfo.toString());
|
|
|
+ return currentUsedHandler;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public synchronized void performFailover(T currentProxy) {
|
|
|
- toIgnore = this.currentUsedProxy.proxyInfo;
|
|
|
- this.currentUsedProxy = null;
|
|
|
+ toIgnore = ((RequestHedgingInvocationHandler) Proxy.getInvocationHandler(
|
|
|
+ currentUsedHandler.proxy)).currentUsedProxy.proxyInfo;
|
|
|
+ this.currentUsedHandler = null;
|
|
|
}
|
|
|
|
|
|
/**
|