|
@@ -400,7 +400,7 @@ public class RouterRpcClient {
|
|
|
* NN + current user.
|
|
|
* @throws IOException If we cannot get a connection to the NameNode.
|
|
|
*/
|
|
|
- private ConnectionContext getConnection(UserGroupInformation ugi, String nsId,
|
|
|
+ protected ConnectionContext getConnection(UserGroupInformation ugi, String nsId,
|
|
|
String rpcAddress, Class<?> proto) throws IOException {
|
|
|
ConnectionContext connection = null;
|
|
|
try {
|
|
@@ -462,7 +462,7 @@ public class RouterRpcClient {
|
|
|
* @return Retry decision.
|
|
|
* @throws IOException An IO Error occurred.
|
|
|
*/
|
|
|
- private RetryDecision shouldRetry(
|
|
|
+ protected RetryDecision shouldRetry(
|
|
|
final IOException ioe, final int retryCount, final String nsId,
|
|
|
final FederationNamenodeContext namenode,
|
|
|
final boolean listObserverFirst) throws IOException {
|
|
@@ -526,11 +526,12 @@ public class RouterRpcClient {
|
|
|
if (rpcMonitor != null) {
|
|
|
rpcMonitor.proxyOp();
|
|
|
}
|
|
|
- boolean failover = false;
|
|
|
- boolean shouldUseObserver = useObserver;
|
|
|
+
|
|
|
+ ExecutionStatus status = new ExecutionStatus(false, useObserver);
|
|
|
Map<FederationNamenodeContext, IOException> ioes = new LinkedHashMap<>();
|
|
|
for (FederationNamenodeContext namenode : namenodes) {
|
|
|
- if (!shouldUseObserver && (namenode.getState() == FederationNamenodeServiceState.OBSERVER)) {
|
|
|
+ if (!status.isShouldUseObserver()
|
|
|
+ && (namenode.getState() == FederationNamenodeServiceState.OBSERVER)) {
|
|
|
continue;
|
|
|
}
|
|
|
ConnectionContext connection = null;
|
|
@@ -541,83 +542,12 @@ public class RouterRpcClient {
|
|
|
ProxyAndInfo<?> client = connection.getClient();
|
|
|
final Object proxy = client.getProxy();
|
|
|
|
|
|
- ret = invoke(nsId, namenode, useObserver, 0, method, proxy, params);
|
|
|
- if (failover &&
|
|
|
- FederationNamenodeServiceState.OBSERVER != namenode.getState()) {
|
|
|
- // Success on alternate server, update
|
|
|
- InetSocketAddress address = client.getAddress();
|
|
|
- namenodeResolver.updateActiveNamenode(nsId, address);
|
|
|
- }
|
|
|
- if (this.rpcMonitor != null) {
|
|
|
- this.rpcMonitor.proxyOpComplete(true, nsId, namenode.getState());
|
|
|
- }
|
|
|
- if (this.router.getRouterClientMetrics() != null) {
|
|
|
- this.router.getRouterClientMetrics().incInvokedMethod(method);
|
|
|
- }
|
|
|
+ ret = invoke(namenode, useObserver, 0, method, proxy, params);
|
|
|
+ postProcessResult(method, status, namenode, nsId, client);
|
|
|
return ret;
|
|
|
} catch (IOException ioe) {
|
|
|
ioes.put(namenode, ioe);
|
|
|
- if (ioe instanceof ObserverRetryOnActiveException) {
|
|
|
- LOG.info("Encountered ObserverRetryOnActiveException from {}."
|
|
|
- + " Retry active namenode directly.", namenode);
|
|
|
- shouldUseObserver = false;
|
|
|
- } else if (ioe instanceof StandbyException) {
|
|
|
- // Fail over indicated by retry policy and/or NN
|
|
|
- if (this.rpcMonitor != null) {
|
|
|
- this.rpcMonitor.proxyOpFailureStandby(nsId);
|
|
|
- }
|
|
|
- failover = true;
|
|
|
- } else if (isUnavailableException(ioe)) {
|
|
|
- if (this.rpcMonitor != null) {
|
|
|
- this.rpcMonitor.proxyOpFailureCommunicate(nsId);
|
|
|
- }
|
|
|
- if (FederationNamenodeServiceState.OBSERVER == namenode.getState()) {
|
|
|
- namenodeResolver.updateUnavailableNamenode(nsId,
|
|
|
- NetUtils.createSocketAddr(namenode.getRpcAddress()));
|
|
|
- } else {
|
|
|
- failover = true;
|
|
|
- }
|
|
|
- } else if (ioe instanceof RemoteException) {
|
|
|
- if (this.rpcMonitor != null) {
|
|
|
- this.rpcMonitor.proxyOpComplete(true, nsId, namenode.getState());
|
|
|
- }
|
|
|
- RemoteException re = (RemoteException) ioe;
|
|
|
- ioe = re.unwrapRemoteException();
|
|
|
- ioe = getCleanException(ioe);
|
|
|
- // RemoteException returned by NN
|
|
|
- throw ioe;
|
|
|
- } else if (ioe instanceof ConnectionNullException) {
|
|
|
- if (this.rpcMonitor != null) {
|
|
|
- this.rpcMonitor.proxyOpFailureCommunicate(nsId);
|
|
|
- }
|
|
|
- LOG.error("Get connection for {} {} error: {}", nsId, rpcAddress,
|
|
|
- ioe.getMessage());
|
|
|
- // Throw StandbyException so that client can retry
|
|
|
- StandbyException se = new StandbyException(ioe.getMessage());
|
|
|
- se.initCause(ioe);
|
|
|
- throw se;
|
|
|
- } else if (ioe instanceof NoNamenodesAvailableException) {
|
|
|
- IOException cause = (IOException) ioe.getCause();
|
|
|
- if (this.rpcMonitor != null) {
|
|
|
- this.rpcMonitor.proxyOpNoNamenodes(nsId);
|
|
|
- }
|
|
|
- LOG.error("Cannot get available namenode for {} {} error: {}",
|
|
|
- nsId, rpcAddress, ioe.getMessage());
|
|
|
- // Rotate cache so that client can retry the next namenode in the cache
|
|
|
- if (shouldRotateCache(cause)) {
|
|
|
- this.namenodeResolver.rotateCache(nsId, namenode, useObserver);
|
|
|
- }
|
|
|
- // Throw RetriableException so that client can retry
|
|
|
- throw new RetriableException(ioe);
|
|
|
- } else {
|
|
|
- // Other communication error, this is a failure
|
|
|
- // Communication retries are handled by the retry policy
|
|
|
- if (this.rpcMonitor != null) {
|
|
|
- this.rpcMonitor.proxyOpFailureCommunicate(nsId);
|
|
|
- this.rpcMonitor.proxyOpComplete(false, nsId, namenode.getState());
|
|
|
- }
|
|
|
- throw ioe;
|
|
|
- }
|
|
|
+ handleInvokeMethodIOException(namenode, ioe, status, useObserver);
|
|
|
} finally {
|
|
|
if (connection != null) {
|
|
|
connection.release();
|
|
@@ -628,6 +558,24 @@ public class RouterRpcClient {
|
|
|
this.rpcMonitor.proxyOpComplete(false, null, null);
|
|
|
}
|
|
|
|
|
|
+ return handlerAllNamenodeFail(namenodes, method, ioes, params);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * All namenodes cannot successfully process the RPC request,
|
|
|
+ * throw corresponding exceptions according to the exception type of each namenode.
|
|
|
+ *
|
|
|
+ * @param namenodes A prioritized list of namenodes within the same nameservice.
|
|
|
+ * @param method Remote ClientProtocol method to invoke.
|
|
|
+ * @param ioes The exception type of each namenode.
|
|
|
+ * @param params Variable list of parameters matching the method.
|
|
|
+ * @return null
|
|
|
+ * @throws IOException Corresponding IOException according to the
|
|
|
+ * exception type of each namenode.
|
|
|
+ */
|
|
|
+ protected Object handlerAllNamenodeFail(
|
|
|
+ List<? extends FederationNamenodeContext> namenodes, Method method,
|
|
|
+ Map<FederationNamenodeContext, IOException> ioes, Object[] params) throws IOException {
|
|
|
// All namenodes were unavailable or in standby
|
|
|
String msg = "No namenode available to invoke " + method.getName() + " " +
|
|
|
Arrays.deepToString(params) + " in " + namenodes + " from " +
|
|
@@ -658,14 +606,120 @@ public class RouterRpcClient {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * The RPC request is successfully processed by the NameNode, the NameNode status
|
|
|
+ * in the router cache is updated according to the ExecutionStatus.
|
|
|
+ *
|
|
|
+ * @param method Remote method to invoke.
|
|
|
+ * @param status Current execution status.
|
|
|
+ * @param namenode The namenode that successfully processed this RPC request.
|
|
|
+ * @param nsId Nameservice ID.
|
|
|
+ * @param client Connection client.
|
|
|
+ * @throws IOException If the state store cannot be accessed.
|
|
|
+ */
|
|
|
+ protected void postProcessResult(Method method, ExecutionStatus status,
|
|
|
+ FederationNamenodeContext namenode, String nsId, ProxyAndInfo<?> client) throws IOException {
|
|
|
+ if (status.isFailOver() &&
|
|
|
+ FederationNamenodeServiceState.OBSERVER != namenode.getState()) {
|
|
|
+ // Success on alternate server, update
|
|
|
+ InetSocketAddress address = client.getAddress();
|
|
|
+ namenodeResolver.updateActiveNamenode(nsId, address);
|
|
|
+ }
|
|
|
+ if (this.rpcMonitor != null) {
|
|
|
+ this.rpcMonitor.proxyOpComplete(true, nsId, namenode.getState());
|
|
|
+ }
|
|
|
+ if (this.router.getRouterClientMetrics() != null) {
|
|
|
+ this.router.getRouterClientMetrics().incInvokedMethod(method);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * The RPC request to the NameNode throws an exception,
|
|
|
+ * handle it according to the type of exception.
|
|
|
+ *
|
|
|
+ * @param namenode The namenode that processed this RPC request.
|
|
|
+ * @param ioe The exception thrown by this RPC request.
|
|
|
+ * @param status The current execution status.
|
|
|
+ * @param useObserver Whether to use observer namenodes.
|
|
|
+ * @throws IOException If it cannot invoke the method.
|
|
|
+ */
|
|
|
+ protected void handleInvokeMethodIOException(final FederationNamenodeContext namenode,
|
|
|
+ IOException ioe, final ExecutionStatus status, boolean useObserver) throws IOException {
|
|
|
+ String nsId = namenode.getNameserviceId();
|
|
|
+ String rpcAddress = namenode.getRpcAddress();
|
|
|
+ if (ioe instanceof ObserverRetryOnActiveException) {
|
|
|
+ LOG.info("Encountered ObserverRetryOnActiveException from {}."
|
|
|
+ + " Retry active namenode directly.", namenode);
|
|
|
+ status.setShouldUseObserver(false);
|
|
|
+ } else if (ioe instanceof StandbyException) {
|
|
|
+ // Fail over indicated by retry policy and/or NN
|
|
|
+ if (this.rpcMonitor != null) {
|
|
|
+ this.rpcMonitor.proxyOpFailureStandby(nsId);
|
|
|
+ }
|
|
|
+ status.setFailOver(true);
|
|
|
+ } else if (isUnavailableException(ioe)) {
|
|
|
+ if (this.rpcMonitor != null) {
|
|
|
+ this.rpcMonitor.proxyOpFailureCommunicate(nsId);
|
|
|
+ }
|
|
|
+ if (FederationNamenodeServiceState.OBSERVER == namenode.getState()) {
|
|
|
+ namenodeResolver.updateUnavailableNamenode(nsId,
|
|
|
+ NetUtils.createSocketAddr(namenode.getRpcAddress()));
|
|
|
+ } else {
|
|
|
+ status.setFailOver(true);
|
|
|
+ }
|
|
|
+ } else if (ioe instanceof RemoteException) {
|
|
|
+ if (this.rpcMonitor != null) {
|
|
|
+ this.rpcMonitor.proxyOpComplete(true, nsId, namenode.getState());
|
|
|
+ }
|
|
|
+ RemoteException re = (RemoteException) ioe;
|
|
|
+ ioe = re.unwrapRemoteException();
|
|
|
+ ioe = getCleanException(ioe);
|
|
|
+ // RemoteException returned by NN
|
|
|
+ throw ioe;
|
|
|
+ } else if (ioe instanceof ConnectionNullException) {
|
|
|
+ if (this.rpcMonitor != null) {
|
|
|
+ this.rpcMonitor.proxyOpFailureCommunicate(nsId);
|
|
|
+ }
|
|
|
+ LOG.error("Get connection for {} {} error: {}", nsId, rpcAddress,
|
|
|
+ ioe.getMessage());
|
|
|
+ // Throw StandbyException so that client can retry
|
|
|
+ StandbyException se = new StandbyException(ioe.getMessage());
|
|
|
+ se.initCause(ioe);
|
|
|
+ throw se;
|
|
|
+ } else if (ioe instanceof NoNamenodesAvailableException) {
|
|
|
+ IOException cause = (IOException) ioe.getCause();
|
|
|
+ if (this.rpcMonitor != null) {
|
|
|
+ this.rpcMonitor.proxyOpNoNamenodes(nsId);
|
|
|
+ }
|
|
|
+ LOG.error("Cannot get available namenode for {} {} error: {}",
|
|
|
+ nsId, rpcAddress, ioe.getMessage());
|
|
|
+ // Rotate cache so that client can retry the next namenode in the cache
|
|
|
+ if (shouldRotateCache(cause)) {
|
|
|
+ this.namenodeResolver.rotateCache(nsId, namenode, useObserver);
|
|
|
+ }
|
|
|
+ // Throw RetriableException so that client can retry
|
|
|
+ throw new RetriableException(ioe);
|
|
|
+ } else {
|
|
|
+ // Other communication error, this is a failure
|
|
|
+ // Communication retries are handled by the retry policy
|
|
|
+ if (this.rpcMonitor != null) {
|
|
|
+ this.rpcMonitor.proxyOpFailureCommunicate(nsId);
|
|
|
+ this.rpcMonitor.proxyOpComplete(false, nsId, namenode.getState());
|
|
|
+ }
|
|
|
+ throw ioe;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* For tracking some information about the actual client.
|
|
|
* It adds trace info "clientIp:ip", "clientPort:port",
|
|
|
* "clientId:id", "clientCallId:callId" and "realUser:userName"
|
|
|
* in the caller context, removing the old values if they were
|
|
|
* already present.
|
|
|
+ *
|
|
|
+ * @param ugi User group information.
|
|
|
*/
|
|
|
- private void addClientInfoToCallerContext(UserGroupInformation ugi) {
|
|
|
+ protected void addClientInfoToCallerContext(UserGroupInformation ugi) {
|
|
|
CallerContext ctx = CallerContext.getCurrent();
|
|
|
String origContext = ctx == null ? null : ctx.getContext();
|
|
|
byte[] origSignature = ctx == null ? null : ctx.getSignature();
|
|
@@ -706,7 +760,8 @@ public class RouterRpcClient {
|
|
|
* Re-throws exceptions generated by the remote RPC call as either
|
|
|
* RemoteException or IOException.
|
|
|
*
|
|
|
- * @param nsId Identifier for the namespace
|
|
|
+ * @param namenode namenode context.
|
|
|
+ * @param listObserverFirst Observer read case, observer NN will be ranked first.
|
|
|
* @param retryCount Current retry times
|
|
|
* @param method Method to invoke
|
|
|
* @param obj Target object for the method
|
|
@@ -714,8 +769,8 @@ public class RouterRpcClient {
|
|
|
* @return Response from the remote server
|
|
|
* @throws IOException If error occurs.
|
|
|
*/
|
|
|
- private Object invoke(
|
|
|
- String nsId, FederationNamenodeContext namenode, Boolean listObserverFirst,
|
|
|
+ protected Object invoke(
|
|
|
+ FederationNamenodeContext namenode, Boolean listObserverFirst,
|
|
|
int retryCount, final Method method,
|
|
|
final Object obj, final Object... params) throws IOException {
|
|
|
try {
|
|
@@ -725,36 +780,58 @@ public class RouterRpcClient {
|
|
|
return null;
|
|
|
} catch (InvocationTargetException e) {
|
|
|
Throwable cause = e.getCause();
|
|
|
- if (cause instanceof IOException) {
|
|
|
- IOException ioe = (IOException) cause;
|
|
|
-
|
|
|
- // Check if we should retry.
|
|
|
- RetryDecision decision = shouldRetry(ioe, retryCount, nsId, namenode, listObserverFirst);
|
|
|
- if (decision == RetryDecision.RETRY) {
|
|
|
- if (this.rpcMonitor != null) {
|
|
|
- this.rpcMonitor.proxyOpRetries();
|
|
|
- }
|
|
|
+ return handlerInvokeException(namenode, listObserverFirst,
|
|
|
+ retryCount, method, obj, cause, params);
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
- // retry
|
|
|
- return invoke(nsId, namenode, listObserverFirst, ++retryCount, method, obj, params);
|
|
|
- } else if (decision == RetryDecision.FAILOVER_AND_RETRY) {
|
|
|
- // failover, invoker looks for standby exceptions for failover.
|
|
|
- if (ioe instanceof StandbyException) {
|
|
|
- throw ioe;
|
|
|
- } else if (isUnavailableException(ioe)) {
|
|
|
- throw ioe;
|
|
|
- } else {
|
|
|
- throw new StandbyException(ioe.getMessage());
|
|
|
- }
|
|
|
- } else {
|
|
|
+ /**
|
|
|
+ * Handle the exception when an RPC request to the NameNode throws an exception.
|
|
|
+ *
|
|
|
+ * @param namenode namenode context.
|
|
|
+ * @param listObserverFirst Observer read case, observer NN will be ranked first.
|
|
|
+ * @param retryCount Current retry times
|
|
|
+ * @param method Method to invoke
|
|
|
+ * @param obj Target object for the method
|
|
|
+ * @param e The exception thrown by the current invocation.
|
|
|
+ * @param params Variable parameters
|
|
|
+ * @return Response from the remote server
|
|
|
+ * @throws IOException If error occurs.
|
|
|
+ */
|
|
|
+ protected Object handlerInvokeException(FederationNamenodeContext namenode,
|
|
|
+ Boolean listObserverFirst, int retryCount, Method method, Object obj,
|
|
|
+ Throwable e, Object[] params) throws IOException {
|
|
|
+ String nsId = namenode.getNameserviceId();
|
|
|
+ if (e instanceof IOException) {
|
|
|
+ IOException ioe = (IOException) e;
|
|
|
+
|
|
|
+ // Check if we should retry.
|
|
|
+ RetryDecision decision = shouldRetry(ioe, retryCount, nsId, namenode, listObserverFirst);
|
|
|
+ if (decision == RetryDecision.RETRY) {
|
|
|
+ if (this.rpcMonitor != null) {
|
|
|
+ this.rpcMonitor.proxyOpRetries();
|
|
|
+ }
|
|
|
+
|
|
|
+ // retry
|
|
|
+ return invoke(namenode, listObserverFirst, ++retryCount, method, obj, params);
|
|
|
+ } else if (decision == RetryDecision.FAILOVER_AND_RETRY) {
|
|
|
+ // failover, invoker looks for standby exceptions for failover.
|
|
|
+ if (ioe instanceof StandbyException) {
|
|
|
throw ioe;
|
|
|
+ } else if (isUnavailableException(ioe)) {
|
|
|
+ throw ioe;
|
|
|
+ } else {
|
|
|
+ throw new StandbyException(ioe.getMessage());
|
|
|
}
|
|
|
} else {
|
|
|
- throw new IOException(e);
|
|
|
+ throw ioe;
|
|
|
}
|
|
|
+ } else {
|
|
|
+ throw new IOException(e);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+
|
|
|
/**
|
|
|
* Check if the exception comes from an unavailable subcluster.
|
|
|
* @param ioe IOException to check.
|
|
@@ -817,7 +894,7 @@ public class RouterRpcClient {
|
|
|
* @param ioe Exception to clean up.
|
|
|
* @return Copy of the original exception with a clean message.
|
|
|
*/
|
|
|
- private static IOException getCleanException(IOException ioe) {
|
|
|
+ protected static IOException getCleanException(IOException ioe) {
|
|
|
IOException ret = null;
|
|
|
|
|
|
String msg = ioe.getMessage();
|
|
@@ -1185,7 +1262,7 @@ public class RouterRpcClient {
|
|
|
* @param loc Location we are processing.
|
|
|
* @return Exception processed for federation.
|
|
|
*/
|
|
|
- private IOException processException(
|
|
|
+ protected IOException processException(
|
|
|
IOException ioe, RemoteLocationContext loc) {
|
|
|
|
|
|
if (ioe instanceof RemoteException) {
|
|
@@ -1251,7 +1328,7 @@ public class RouterRpcClient {
|
|
|
* @return True if the result is an instance of the required class or if the
|
|
|
* expected class is null.
|
|
|
*/
|
|
|
- private static boolean isExpectedClass(Class<?> expectedClass, Object clazz) {
|
|
|
+ protected static boolean isExpectedClass(Class<?> expectedClass, Object clazz) {
|
|
|
if (expectedClass == null) {
|
|
|
return true;
|
|
|
} else if (clazz == null) {
|
|
@@ -1269,7 +1346,7 @@ public class RouterRpcClient {
|
|
|
* @return True if the result is equals to the expected value or if the
|
|
|
* expected value is null.
|
|
|
*/
|
|
|
- private static boolean isExpectedValue(Object expectedValue, Object value) {
|
|
|
+ protected static boolean isExpectedValue(Object expectedValue, Object value) {
|
|
|
if (expectedValue == null) {
|
|
|
return true;
|
|
|
} else if (value == null) {
|
|
@@ -1414,7 +1491,26 @@ public class RouterRpcClient {
|
|
|
throws IOException {
|
|
|
final List<RemoteResult<T, R>> results = invokeConcurrent(
|
|
|
locations, method, standby, timeOutMs, clazz);
|
|
|
+ return postProcessResult(requireResponse, results);
|
|
|
+ }
|
|
|
|
|
|
+ /**
|
|
|
+ * Post-process the results returned by
|
|
|
+ * {@link RouterRpcClient#invokeConcurrent(Collection, RemoteMethod, boolean, long, Class)}.
|
|
|
+ *
|
|
|
+ * @param requireResponse If true an exception will be thrown if all calls do
|
|
|
+ * not complete. If false exceptions are ignored and all data results
|
|
|
+ * successfully received are returned.
|
|
|
+ * @param results Result of invoking the method per subcluster (list of results),
|
|
|
+ * This includes the exception for each remote location.
|
|
|
+ * @return Result of invoking the method per subcluster: nsId to result.
|
|
|
+ * @param <T> The type of the remote location.
|
|
|
+ * @param <R> The type of the remote method return.
|
|
|
+ * @throws IOException If requiredResponse=true and any of the calls throw an
|
|
|
+ * exception.
|
|
|
+ */
|
|
|
+ protected static <T extends RemoteLocationContext, R> Map<T, R> postProcessResult(
|
|
|
+ boolean requireResponse, List<RemoteResult<T, R>> results) throws IOException {
|
|
|
// Go over the results and exceptions
|
|
|
final Map<T, R> ret = new TreeMap<>();
|
|
|
final List<IOException> thrownExceptions = new ArrayList<>();
|
|
@@ -1480,27 +1576,10 @@ public class RouterRpcClient {
|
|
|
throw new IOException("No remote locations available");
|
|
|
} else if (locations.size() == 1 && timeOutMs <= 0) {
|
|
|
// Shortcut, just one call
|
|
|
- T location = locations.iterator().next();
|
|
|
- String ns = location.getNameserviceId();
|
|
|
- boolean isObserverRead = isObserverReadEligible(ns, m);
|
|
|
- final List<? extends FederationNamenodeContext> namenodes =
|
|
|
- getOrderedNamenodes(ns, isObserverRead);
|
|
|
- RouterRpcFairnessPolicyController controller = getRouterRpcFairnessPolicyController();
|
|
|
- acquirePermit(ns, ugi, method, controller);
|
|
|
- try {
|
|
|
- Class<?> proto = method.getProtocol();
|
|
|
- Object[] paramList = method.getParams(location);
|
|
|
- R result = (R) invokeMethod(
|
|
|
- ugi, namenodes, isObserverRead, proto, m, paramList);
|
|
|
- RemoteResult<T, R> remoteResult = new RemoteResult<>(location, result);
|
|
|
- return Collections.singletonList(remoteResult);
|
|
|
- } catch (IOException ioe) {
|
|
|
- // Localize the exception
|
|
|
- throw processException(ioe, location);
|
|
|
- } finally {
|
|
|
- releasePermit(ns, ugi, method, controller);
|
|
|
- }
|
|
|
+ return invokeSingle(locations.iterator().next(), method);
|
|
|
}
|
|
|
+ RouterRpcFairnessPolicyController controller = getRouterRpcFairnessPolicyController();
|
|
|
+ acquirePermit(CONCURRENT_NS, ugi, method, controller);
|
|
|
|
|
|
List<T> orderedLocations = new ArrayList<>();
|
|
|
List<Callable<Object>> callables = new ArrayList<>();
|
|
@@ -1551,8 +1630,29 @@ public class RouterRpcClient {
|
|
|
this.router.getRouterClientMetrics().incInvokedConcurrent(m);
|
|
|
}
|
|
|
|
|
|
- RouterRpcFairnessPolicyController controller = getRouterRpcFairnessPolicyController();
|
|
|
- acquirePermit(CONCURRENT_NS, ugi, method, controller);
|
|
|
+ return getRemoteResults(method, timeOutMs, controller, orderedLocations, callables);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Invokes multiple concurrent proxy calls to different clients. Returns an
|
|
|
+ * array of results.
|
|
|
+ *
|
|
|
+ * @param <T> The type of the remote location.
|
|
|
+ * @param <R> The type of the remote method return.
|
|
|
+ * @param method The remote method and parameters to invoke.
|
|
|
+ * @param timeOutMs Timeout for each individual call.
|
|
|
+ * @param controller Fairness manager to control handlers assigned per NS.
|
|
|
+ * @param orderedLocations List of remote locations to call concurrently.
|
|
|
+ * @param callables Invoke method for each NameNode.
|
|
|
+ * @return Result of invoking the method per subcluster (list of results),
|
|
|
+ * This includes the exception for each remote location.
|
|
|
+ * @throws IOException If there are errors invoking the method.
|
|
|
+ */
|
|
|
+ protected <T extends RemoteLocationContext, R> List<RemoteResult<T, R>> getRemoteResults(
|
|
|
+ RemoteMethod method, long timeOutMs, RouterRpcFairnessPolicyController controller,
|
|
|
+ List<T> orderedLocations, List<Callable<Object>> callables) throws IOException {
|
|
|
+ final UserGroupInformation ugi = RouterRpcServer.getRemoteUser();
|
|
|
+ final Method m = method.getMethod();
|
|
|
try {
|
|
|
List<Future<Object>> futures = null;
|
|
|
if (timeOutMs > 0) {
|
|
@@ -1561,42 +1661,7 @@ public class RouterRpcClient {
|
|
|
} else {
|
|
|
futures = executorService.invokeAll(callables);
|
|
|
}
|
|
|
- List<RemoteResult<T, R>> results = new ArrayList<>();
|
|
|
- for (int i=0; i<futures.size(); i++) {
|
|
|
- T location = orderedLocations.get(i);
|
|
|
- try {
|
|
|
- Future<Object> future = futures.get(i);
|
|
|
- R result = (R) future.get();
|
|
|
- results.add(new RemoteResult<>(location, result));
|
|
|
- } catch (CancellationException ce) {
|
|
|
- T loc = orderedLocations.get(i);
|
|
|
- String msg = "Invocation to \"" + loc + "\" for \""
|
|
|
- + method.getMethodName() + "\" timed out";
|
|
|
- LOG.error(msg);
|
|
|
- IOException ioe = new SubClusterTimeoutException(msg);
|
|
|
- results.add(new RemoteResult<>(location, ioe));
|
|
|
- } catch (ExecutionException ex) {
|
|
|
- Throwable cause = ex.getCause();
|
|
|
- LOG.debug("Cannot execute {} in {}: {}",
|
|
|
- m.getName(), location, cause.getMessage());
|
|
|
-
|
|
|
- // Convert into IOException if needed
|
|
|
- IOException ioe = null;
|
|
|
- if (cause instanceof IOException) {
|
|
|
- ioe = (IOException) cause;
|
|
|
- } else {
|
|
|
- ioe = new IOException("Unhandled exception while proxying API " +
|
|
|
- m.getName() + ": " + cause.getMessage(), cause);
|
|
|
- }
|
|
|
-
|
|
|
- // Store the exceptions
|
|
|
- results.add(new RemoteResult<>(location, ioe));
|
|
|
- }
|
|
|
- }
|
|
|
- if (rpcMonitor != null) {
|
|
|
- rpcMonitor.proxyOpComplete(true, CONCURRENT, null);
|
|
|
- }
|
|
|
- return results;
|
|
|
+ return processFutures(method, m, orderedLocations, futures);
|
|
|
} catch (RejectedExecutionException e) {
|
|
|
if (rpcMonitor != null) {
|
|
|
rpcMonitor.proxyOpFailureClientOverloaded();
|
|
@@ -1616,6 +1681,99 @@ public class RouterRpcClient {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Handle all futures during the invokeConcurrent call process.
|
|
|
+ *
|
|
|
+ * @param <T> The type of the remote location.
|
|
|
+ * @param <R> The type of the remote method return.
|
|
|
+ * @param method The remote method and parameters to invoke.
|
|
|
+ * @param m The method to invoke.
|
|
|
+ * @param orderedLocations List of remote locations to call concurrently.
|
|
|
+ * @param futures all futures during the invokeConcurrent call process.
|
|
|
+ * @return Result of invoking the method per subcluster (list of results),
|
|
|
+ * This includes the exception for each remote location.
|
|
|
+ * @throws InterruptedException if the current thread was interrupted while waiting.
|
|
|
+ */
|
|
|
+ protected <T extends RemoteLocationContext, R> List<RemoteResult<T, R>> processFutures(
|
|
|
+ RemoteMethod method, Method m, final List<T> orderedLocations,
|
|
|
+ final List<Future<Object>> futures) throws InterruptedException{
|
|
|
+ List<RemoteResult<T, R>> results = new ArrayList<>();
|
|
|
+ for (int i = 0; i< futures.size(); i++) {
|
|
|
+ T location = orderedLocations.get(i);
|
|
|
+ try {
|
|
|
+ Future<Object> future = futures.get(i);
|
|
|
+ R result = (R) future.get();
|
|
|
+ results.add(new RemoteResult<>(location, result));
|
|
|
+ } catch (CancellationException ce) {
|
|
|
+ T loc = orderedLocations.get(i);
|
|
|
+ String msg = "Invocation to \"" + loc + "\" for \""
|
|
|
+ + method.getMethodName() + "\" timed out";
|
|
|
+ LOG.error(msg);
|
|
|
+ IOException ioe = new SubClusterTimeoutException(msg);
|
|
|
+ results.add(new RemoteResult<>(location, ioe));
|
|
|
+ } catch (ExecutionException ex) {
|
|
|
+ Throwable cause = ex.getCause();
|
|
|
+ LOG.debug("Cannot execute {} in {}: {}",
|
|
|
+ m.getName(), location, cause.getMessage());
|
|
|
+
|
|
|
+ // Convert into IOException if needed
|
|
|
+ IOException ioe = null;
|
|
|
+ if (cause instanceof IOException) {
|
|
|
+ ioe = (IOException) cause;
|
|
|
+ } else {
|
|
|
+ ioe = new IOException("Unhandled exception while proxying API " +
|
|
|
+ m.getName() + ": " + cause.getMessage(), cause);
|
|
|
+ }
|
|
|
+
|
|
|
+ // Store the exceptions
|
|
|
+ results.add(new RemoteResult<>(location, ioe));
|
|
|
+ }
|
|
|
+ }
|
|
|
+ if (rpcMonitor != null) {
|
|
|
+ rpcMonitor.proxyOpComplete(true, CONCURRENT, null);
|
|
|
+ }
|
|
|
+ return results;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Invokes a ClientProtocol method against the specified namespace.
|
|
|
+ * <p>
|
|
|
+ * Re-throws exceptions generated by the remote RPC call as either
|
|
|
+ * RemoteException or IOException.
|
|
|
+ *
|
|
|
+ * @param <T> The type of the remote location.
|
|
|
+ * @param <R> The type of the remote method return.
|
|
|
+ * @param location RemoteLocation to invoke.
|
|
|
+ * @param method The remote method and parameters to invoke.
|
|
|
+ * @return Result of invoking the method per subcluster (list of results),
|
|
|
+ * This includes the exception for each remote location.
|
|
|
+ * @throws IOException If there are errors invoking the method.
|
|
|
+ */
|
|
|
+ public <T extends RemoteLocationContext, R> List<RemoteResult<T, R>> invokeSingle(
|
|
|
+ T location, RemoteMethod method) throws IOException {
|
|
|
+ final UserGroupInformation ugi = RouterRpcServer.getRemoteUser();
|
|
|
+ final Method m = method.getMethod();
|
|
|
+ String ns = location.getNameserviceId();
|
|
|
+ boolean isObserverRead = isObserverReadEligible(ns, m);
|
|
|
+ final List<? extends FederationNamenodeContext> namenodes =
|
|
|
+ getOrderedNamenodes(ns, isObserverRead);
|
|
|
+ RouterRpcFairnessPolicyController controller = getRouterRpcFairnessPolicyController();
|
|
|
+ acquirePermit(ns, ugi, method, controller);
|
|
|
+ try {
|
|
|
+ Class<?> proto = method.getProtocol();
|
|
|
+ Object[] paramList = method.getParams(location);
|
|
|
+ R result = (R) invokeMethod(
|
|
|
+ ugi, namenodes, isObserverRead, proto, m, paramList);
|
|
|
+ RemoteResult<T, R> remoteResult = new RemoteResult<>(location, result);
|
|
|
+ return Collections.singletonList(remoteResult);
|
|
|
+ } catch (IOException ioe) {
|
|
|
+ // Localize the exception
|
|
|
+ throw processException(ioe, location);
|
|
|
+ } finally {
|
|
|
+ releasePermit(ns, ugi, method, controller);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Transfer origin thread local context which is necessary to current
|
|
|
* worker thread when invoking method concurrently by executor service.
|
|
@@ -1624,7 +1782,7 @@ public class RouterRpcClient {
|
|
|
* @param originContext origin CallerContext which should be transferred
|
|
|
* to server side.
|
|
|
*/
|
|
|
- private void transferThreadLocalContext(
|
|
|
+ protected void transferThreadLocalContext(
|
|
|
final Call originCall, final CallerContext originContext) {
|
|
|
Server.getCurCall().set(originCall);
|
|
|
CallerContext.setCurrent(originContext);
|
|
@@ -1675,7 +1833,7 @@ public class RouterRpcClient {
|
|
|
* @param controller fairness policy controller to acquire permit from
|
|
|
* @throws IOException If permit could not be acquired for the nsId.
|
|
|
*/
|
|
|
- private void acquirePermit(final String nsId, final UserGroupInformation ugi,
|
|
|
+ protected void acquirePermit(final String nsId, final UserGroupInformation ugi,
|
|
|
final RemoteMethod m, RouterRpcFairnessPolicyController controller)
|
|
|
throws IOException {
|
|
|
if (controller != null) {
|
|
@@ -1708,7 +1866,7 @@ public class RouterRpcClient {
|
|
|
* @param m Remote method that needs to be invoked.
|
|
|
* @param controller fairness policy controller to release permit from
|
|
|
*/
|
|
|
- private void releasePermit(final String nsId, final UserGroupInformation ugi,
|
|
|
+ protected void releasePermit(final String nsId, final UserGroupInformation ugi,
|
|
|
final RemoteMethod m, RouterRpcFairnessPolicyController controller) {
|
|
|
if (controller != null) {
|
|
|
controller.releasePermit(nsId);
|
|
@@ -1782,7 +1940,7 @@ public class RouterRpcClient {
|
|
|
* @return A prioritized list of NNs to use for communication.
|
|
|
* @throws IOException If a NN cannot be located for the nameservice ID.
|
|
|
*/
|
|
|
- private List<? extends FederationNamenodeContext> getOrderedNamenodes(String nsId,
|
|
|
+ protected List<? extends FederationNamenodeContext> getOrderedNamenodes(String nsId,
|
|
|
boolean isObserverRead) throws IOException {
|
|
|
final List<? extends FederationNamenodeContext> namenodes;
|
|
|
|
|
@@ -1802,7 +1960,7 @@ public class RouterRpcClient {
|
|
|
return namenodes;
|
|
|
}
|
|
|
|
|
|
- private boolean isObserverReadEligible(String nsId, Method method) {
|
|
|
+ protected boolean isObserverReadEligible(String nsId, Method method) {
|
|
|
return isReadCall(method) && isNamespaceObserverReadEligible(nsId);
|
|
|
}
|
|
|
|
|
@@ -1857,7 +2015,7 @@ public class RouterRpcClient {
|
|
|
* {@link RouterRpcClient#isUnavailableException(IOException) unavailable exception},
|
|
|
* otherwise false.
|
|
|
*/
|
|
|
- private boolean shouldRotateCache(IOException ioe) {
|
|
|
+ protected boolean shouldRotateCache(IOException ioe) {
|
|
|
if (isUnavailableException(ioe)) {
|
|
|
return true;
|
|
|
}
|
|
@@ -1868,4 +2026,61 @@ public class RouterRpcClient {
|
|
|
}
|
|
|
return isUnavailableException(ioe);
|
|
|
}
|
|
|
+
|
|
|
+
|
|
|
+ /**
|
|
|
+ * The {@link ExecutionStatus} class is a utility class used to track the status of
|
|
|
+ * execution operations performed by the {@link RouterRpcClient}.
|
|
|
+ * It encapsulates the state of an operation, including whether it has completed,
|
|
|
+ * if a failover to a different NameNode should be attempted, and if an observer
|
|
|
+ * NameNode should be used for the operation.
|
|
|
+ *
|
|
|
+ * <p>The status is represented by a flag that indicate the current state of
|
|
|
+ * the execution. The flag can be checked individually to determine how to
|
|
|
+ * proceed with the operation or to handle its results.
|
|
|
+ */
|
|
|
+ protected static class ExecutionStatus {
|
|
|
+
|
|
|
+ /** A byte field used to store the state flags. */
|
|
|
+ private byte flag;
|
|
|
+ private static final byte FAIL_OVER_BIT = 1;
|
|
|
+ private static final byte SHOULD_USE_OBSERVER_BIT = 2;
|
|
|
+ private static final byte COMPLETE_BIT = 4;
|
|
|
+
|
|
|
+ ExecutionStatus() {
|
|
|
+ this(false, false);
|
|
|
+ }
|
|
|
+
|
|
|
+ ExecutionStatus(boolean failOver, boolean shouldUseObserver) {
|
|
|
+ this.flag = 0;
|
|
|
+ setFailOver(failOver);
|
|
|
+ setShouldUseObserver(shouldUseObserver);
|
|
|
+ setComplete(false);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void setFailOver(boolean failOver) {
|
|
|
+ flag = (byte) (failOver ? (flag | FAIL_OVER_BIT) : (flag & ~FAIL_OVER_BIT));
|
|
|
+ }
|
|
|
+
|
|
|
+ private void setShouldUseObserver(boolean shouldUseObserver) {
|
|
|
+ flag = (byte) (shouldUseObserver ?
|
|
|
+ (flag | SHOULD_USE_OBSERVER_BIT) : (flag & ~SHOULD_USE_OBSERVER_BIT));
|
|
|
+ }
|
|
|
+
|
|
|
+ void setComplete(boolean complete) {
|
|
|
+ flag = (byte) (complete ? (flag | COMPLETE_BIT) : (flag & ~COMPLETE_BIT));
|
|
|
+ }
|
|
|
+
|
|
|
+ boolean isFailOver() {
|
|
|
+ return (flag & FAIL_OVER_BIT) != 0;
|
|
|
+ }
|
|
|
+
|
|
|
+ boolean isShouldUseObserver() {
|
|
|
+ return (flag & SHOULD_USE_OBSERVER_BIT) != 0;
|
|
|
+ }
|
|
|
+
|
|
|
+ boolean isComplete() {
|
|
|
+ return (flag & COMPLETE_BIT) != 0;
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|