|
@@ -38,6 +38,7 @@ import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
|
import java.util.Collection;
|
|
|
import java.util.Collections;
|
|
|
+import java.util.HashSet;
|
|
|
import java.util.LinkedHashMap;
|
|
|
import java.util.LinkedList;
|
|
|
import java.util.List;
|
|
@@ -70,16 +71,19 @@ import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
|
|
|
import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeContext;
|
|
|
import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState;
|
|
|
import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.ha.ReadOnly;
|
|
|
import org.apache.hadoop.io.retry.RetryPolicies;
|
|
|
import org.apache.hadoop.io.retry.RetryPolicy;
|
|
|
import org.apache.hadoop.io.retry.RetryPolicy.RetryAction.RetryDecision;
|
|
|
import org.apache.hadoop.ipc.CallerContext;
|
|
|
+import org.apache.hadoop.ipc.ObserverRetryOnActiveException;
|
|
|
import org.apache.hadoop.ipc.RemoteException;
|
|
|
import org.apache.hadoop.ipc.RetriableException;
|
|
|
import org.apache.hadoop.ipc.Server;
|
|
|
import org.apache.hadoop.ipc.Server.Call;
|
|
|
import org.apache.hadoop.ipc.StandbyException;
|
|
|
import org.apache.hadoop.net.ConnectTimeoutException;
|
|
|
+import org.apache.hadoop.net.NetUtils;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
import org.apache.hadoop.util.StringUtils;
|
|
|
import org.eclipse.jetty.util.ajax.JSON;
|
|
@@ -128,6 +132,10 @@ public class RouterRpcClient {
|
|
|
private final RouterRpcMonitor rpcMonitor;
|
|
|
/** Field separator of CallerContext. */
|
|
|
private final String contextFieldSeparator;
|
|
|
+ /** Observer read enabled. Default for all nameservices. */
|
|
|
+ private final boolean observerReadEnabledDefault;
|
|
|
+ /** Nameservice specific overrides of the default setting for enabling observer reads. */
|
|
|
+ private HashSet<String> observerReadEnabledOverrides = new HashSet<>();
|
|
|
|
|
|
/** Pattern to parse a stack trace line. */
|
|
|
private static final Pattern STACK_TRACE_PATTERN =
|
|
@@ -200,6 +208,16 @@ public class RouterRpcClient {
|
|
|
failoverSleepBaseMillis, failoverSleepMaxMillis);
|
|
|
String[] ipProxyUsers = conf.getStrings(DFS_NAMENODE_IP_PROXY_USERS);
|
|
|
this.enableProxyUser = ipProxyUsers != null && ipProxyUsers.length > 0;
|
|
|
+ this.observerReadEnabledDefault = conf.getBoolean(
|
|
|
+ RBFConfigKeys.DFS_ROUTER_OBSERVER_READ_DEFAULT_KEY,
|
|
|
+ RBFConfigKeys.DFS_ROUTER_OBSERVER_READ_DEFAULT_VALUE);
|
|
|
+ String[] observerReadOverrides = conf.getStrings(RBFConfigKeys.DFS_ROUTER_OBSERVER_READ_OVERRIDES);
|
|
|
+ if (observerReadOverrides != null) {
|
|
|
+ observerReadEnabledOverrides.addAll(Arrays.asList(observerReadOverrides));
|
|
|
+ }
|
|
|
+ if (this.observerReadEnabledDefault) {
|
|
|
+ LOG.info("Observer read is enabled for router.");
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -451,6 +469,7 @@ public class RouterRpcClient {
|
|
|
* @param ugi User group information.
|
|
|
* @param namenodes A prioritized list of namenodes within the same
|
|
|
* nameservice.
|
|
|
+ * @param useObserver Whether to use observer namenodes.
|
|
|
* @param method Remote ClientProtocol method to invoke.
|
|
|
* @param params Variable list of parameters matching the method.
|
|
|
* @return The result of invoking the method.
|
|
@@ -462,6 +481,7 @@ public class RouterRpcClient {
|
|
|
public Object invokeMethod(
|
|
|
final UserGroupInformation ugi,
|
|
|
final List<? extends FederationNamenodeContext> namenodes,
|
|
|
+ boolean useObserver,
|
|
|
final Class<?> protocol, final Method method, final Object... params)
|
|
|
throws ConnectException, StandbyException, IOException {
|
|
|
|
|
@@ -478,8 +498,12 @@ public class RouterRpcClient {
|
|
|
rpcMonitor.proxyOp();
|
|
|
}
|
|
|
boolean failover = false;
|
|
|
+ boolean shouldUseObserver = useObserver;
|
|
|
Map<FederationNamenodeContext, IOException> ioes = new LinkedHashMap<>();
|
|
|
for (FederationNamenodeContext namenode : namenodes) {
|
|
|
+ if (!shouldUseObserver && (namenode.getState() == FederationNamenodeServiceState.OBSERVER)) {
|
|
|
+ continue;
|
|
|
+ }
|
|
|
ConnectionContext connection = null;
|
|
|
String nsId = namenode.getNameserviceId();
|
|
|
String rpcAddress = namenode.getRpcAddress();
|
|
@@ -489,13 +513,14 @@ public class RouterRpcClient {
|
|
|
final Object proxy = client.getProxy();
|
|
|
|
|
|
ret = invoke(nsId, 0, method, proxy, params);
|
|
|
- if (failover) {
|
|
|
+ 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);
|
|
|
+ this.rpcMonitor.proxyOpComplete(true, nsId, namenode.getState());
|
|
|
}
|
|
|
if (this.router.getRouterClientMetrics() != null) {
|
|
|
this.router.getRouterClientMetrics().incInvokedMethod(method);
|
|
@@ -503,7 +528,11 @@ public class RouterRpcClient {
|
|
|
return ret;
|
|
|
} catch (IOException ioe) {
|
|
|
ioes.put(namenode, ioe);
|
|
|
- if (ioe instanceof StandbyException) {
|
|
|
+ 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);
|
|
@@ -513,10 +542,15 @@ public class RouterRpcClient {
|
|
|
if (this.rpcMonitor != null) {
|
|
|
this.rpcMonitor.proxyOpFailureCommunicate(nsId);
|
|
|
}
|
|
|
- failover = true;
|
|
|
+ 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);
|
|
|
+ this.rpcMonitor.proxyOpComplete(true, nsId, namenode.getState());
|
|
|
}
|
|
|
RemoteException re = (RemoteException) ioe;
|
|
|
ioe = re.unwrapRemoteException();
|
|
@@ -546,7 +580,7 @@ public class RouterRpcClient {
|
|
|
// Communication retries are handled by the retry policy
|
|
|
if (this.rpcMonitor != null) {
|
|
|
this.rpcMonitor.proxyOpFailureCommunicate(nsId);
|
|
|
- this.rpcMonitor.proxyOpComplete(false, nsId);
|
|
|
+ this.rpcMonitor.proxyOpComplete(false, nsId, namenode.getState());
|
|
|
}
|
|
|
throw ioe;
|
|
|
}
|
|
@@ -557,7 +591,7 @@ public class RouterRpcClient {
|
|
|
}
|
|
|
}
|
|
|
if (this.rpcMonitor != null) {
|
|
|
- this.rpcMonitor.proxyOpComplete(false, null);
|
|
|
+ this.rpcMonitor.proxyOpComplete(false, null, null);
|
|
|
}
|
|
|
|
|
|
// All namenodes were unavailable or in standby
|
|
@@ -640,16 +674,12 @@ public class RouterRpcClient {
|
|
|
* @param params Variable parameters
|
|
|
* @return Response from the remote server
|
|
|
* @throws IOException
|
|
|
- * @throws InterruptedException
|
|
|
*/
|
|
|
private Object invoke(String nsId, int retryCount, final Method method,
|
|
|
final Object obj, final Object... params) throws IOException {
|
|
|
try {
|
|
|
return method.invoke(obj, params);
|
|
|
- } catch (IllegalAccessException e) {
|
|
|
- LOG.error("Unexpected exception while proxying API", e);
|
|
|
- return null;
|
|
|
- } catch (IllegalArgumentException e) {
|
|
|
+ } catch (IllegalAccessException | IllegalArgumentException e) {
|
|
|
LOG.error("Unexpected exception while proxying API", e);
|
|
|
return null;
|
|
|
} catch (InvocationTargetException e) {
|
|
@@ -713,7 +743,7 @@ public class RouterRpcClient {
|
|
|
*/
|
|
|
private boolean isClusterUnAvailable(String nsId) throws IOException {
|
|
|
List<? extends FederationNamenodeContext> nnState = this.namenodeResolver
|
|
|
- .getNamenodesForNameserviceId(nsId);
|
|
|
+ .getNamenodesForNameserviceId(nsId, false);
|
|
|
|
|
|
if (nnState != null) {
|
|
|
for (FederationNamenodeContext nnContext : nnState) {
|
|
@@ -844,13 +874,13 @@ public class RouterRpcClient {
|
|
|
RouterRpcFairnessPolicyController controller = getRouterRpcFairnessPolicyController();
|
|
|
acquirePermit(nsId, ugi, method, controller);
|
|
|
try {
|
|
|
- List<? extends FederationNamenodeContext> nns =
|
|
|
- getNamenodesForNameservice(nsId);
|
|
|
+ boolean isObserverRead = isObserverReadEligible(nsId, method.getMethod());
|
|
|
+ List<? extends FederationNamenodeContext> nns = getOrderedNamenodes(nsId, isObserverRead);
|
|
|
RemoteLocationContext loc = new RemoteLocation(nsId, "/", "/");
|
|
|
Class<?> proto = method.getProtocol();
|
|
|
Method m = method.getMethod();
|
|
|
Object[] params = method.getParams(loc);
|
|
|
- return invokeMethod(ugi, nns, proto, m, params);
|
|
|
+ return invokeMethod(ugi, nns, isObserverRead, proto, m, params);
|
|
|
} finally {
|
|
|
releasePermit(nsId, ugi, method, controller);
|
|
|
}
|
|
@@ -927,7 +957,7 @@ public class RouterRpcClient {
|
|
|
* @throws IOException if the success condition is not met and one of the RPC
|
|
|
* calls generated a remote exception.
|
|
|
*/
|
|
|
- public Object invokeSequential(
|
|
|
+ public <T> T invokeSequential(
|
|
|
final List<? extends RemoteLocationContext> locations,
|
|
|
final RemoteMethod remoteMethod) throws IOException {
|
|
|
return invokeSequential(locations, remoteMethod, null, null);
|
|
@@ -1012,12 +1042,14 @@ public class RouterRpcClient {
|
|
|
for (final RemoteLocationContext loc : locations) {
|
|
|
String ns = loc.getNameserviceId();
|
|
|
acquirePermit(ns, ugi, remoteMethod, controller);
|
|
|
+ boolean isObserverRead = isObserverReadEligible(ns, m);
|
|
|
List<? extends FederationNamenodeContext> namenodes =
|
|
|
- getNamenodesForNameservice(ns);
|
|
|
+ getOrderedNamenodes(ns, isObserverRead);
|
|
|
try {
|
|
|
Class<?> proto = remoteMethod.getProtocol();
|
|
|
Object[] params = remoteMethod.getParams(loc);
|
|
|
- Object result = invokeMethod(ugi, namenodes, proto, m, params);
|
|
|
+ Object result = invokeMethod(
|
|
|
+ ugi, namenodes, isObserverRead, proto, m, params);
|
|
|
// Check if the result is what we expected
|
|
|
if (isExpectedClass(expectedResultClass, result) &&
|
|
|
isExpectedValue(expectedResultValue, result)) {
|
|
@@ -1373,12 +1405,14 @@ public class RouterRpcClient {
|
|
|
String ns = location.getNameserviceId();
|
|
|
RouterRpcFairnessPolicyController controller = getRouterRpcFairnessPolicyController();
|
|
|
acquirePermit(ns, ugi, method, controller);
|
|
|
+ boolean isObserverRead = isObserverReadEligible(ns, m);
|
|
|
final List<? extends FederationNamenodeContext> namenodes =
|
|
|
- getNamenodesForNameservice(ns);
|
|
|
+ getOrderedNamenodes(ns, isObserverRead);
|
|
|
try {
|
|
|
Class<?> proto = method.getProtocol();
|
|
|
Object[] paramList = method.getParams(location);
|
|
|
- R result = (R) invokeMethod(ugi, namenodes, proto, m, paramList);
|
|
|
+ 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) {
|
|
@@ -1396,8 +1430,9 @@ public class RouterRpcClient {
|
|
|
final CallerContext originContext = CallerContext.getCurrent();
|
|
|
for (final T location : locations) {
|
|
|
String nsId = location.getNameserviceId();
|
|
|
+ boolean isObserverRead = isObserverReadEligible(nsId, m);
|
|
|
final List<? extends FederationNamenodeContext> namenodes =
|
|
|
- getNamenodesForNameservice(nsId);
|
|
|
+ getOrderedNamenodes(nsId, isObserverRead);
|
|
|
final Class<?> proto = method.getProtocol();
|
|
|
final Object[] paramList = method.getParams(location);
|
|
|
if (standby) {
|
|
@@ -1414,7 +1449,8 @@ public class RouterRpcClient {
|
|
|
callables.add(
|
|
|
() -> {
|
|
|
transferThreadLocalContext(originCall, originContext);
|
|
|
- return invokeMethod(ugi, nnList, proto, m, paramList);
|
|
|
+ return invokeMethod(
|
|
|
+ ugi, nnList, isObserverRead, proto, m, paramList);
|
|
|
});
|
|
|
}
|
|
|
} else {
|
|
@@ -1423,7 +1459,8 @@ public class RouterRpcClient {
|
|
|
callables.add(
|
|
|
() -> {
|
|
|
transferThreadLocalContext(originCall, originContext);
|
|
|
- return invokeMethod(ugi, namenodes, proto, m, paramList);
|
|
|
+ return invokeMethod(
|
|
|
+ ugi, namenodes, isObserverRead, proto, m, paramList);
|
|
|
});
|
|
|
}
|
|
|
}
|
|
@@ -1512,27 +1549,6 @@ public class RouterRpcClient {
|
|
|
CallerContext.setCurrent(originContext);
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Get a prioritized list of NNs that share the same nameservice ID (in the
|
|
|
- * same namespace). NNs that are reported as ACTIVE will be first in the list.
|
|
|
- *
|
|
|
- * @param nsId The nameservice ID for the namespace.
|
|
|
- * @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> getNamenodesForNameservice(
|
|
|
- final String nsId) throws IOException {
|
|
|
-
|
|
|
- final List<? extends FederationNamenodeContext> namenodes =
|
|
|
- namenodeResolver.getNamenodesForNameserviceId(nsId);
|
|
|
-
|
|
|
- if (namenodes == null || namenodes.isEmpty()) {
|
|
|
- throw new IOException("Cannot locate a registered namenode for " + nsId +
|
|
|
- " from " + router.getRouterId());
|
|
|
- }
|
|
|
- return namenodes;
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Get a prioritized list of NNs that share the same block pool ID (in the
|
|
|
* same namespace). NNs that are reported as ACTIVE will be first in the list.
|
|
@@ -1670,4 +1686,48 @@ public class RouterRpcClient {
|
|
|
}
|
|
|
return null;
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Get a prioritized list of NNs that share the same nameservice ID (in the
|
|
|
+ * same namespace).
|
|
|
+ * In observer read case, OBSERVER NNs will be first in the list.
|
|
|
+ * Otherwise, ACTIVE NNs will be first in the list.
|
|
|
+ *
|
|
|
+ * @param nsId The nameservice ID for the namespace.
|
|
|
+ * @param isObserverRead Read on observer namenode.
|
|
|
+ * @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,
|
|
|
+ boolean isObserverRead) throws IOException {
|
|
|
+ final List<? extends FederationNamenodeContext> namenodes;
|
|
|
+
|
|
|
+ if (RouterStateIdContext.getClientStateIdFromCurrentCall(nsId) > Long.MIN_VALUE) {
|
|
|
+ namenodes = namenodeResolver.getNamenodesForNameserviceId(nsId, isObserverRead);
|
|
|
+ } else {
|
|
|
+ namenodes = namenodeResolver.getNamenodesForNameserviceId(nsId, false);
|
|
|
+ }
|
|
|
+
|
|
|
+ if (namenodes == null || namenodes.isEmpty()) {
|
|
|
+ throw new IOException("Cannot locate a registered namenode for " + nsId +
|
|
|
+ " from " + router.getRouterId());
|
|
|
+ }
|
|
|
+ return namenodes;
|
|
|
+ }
|
|
|
+
|
|
|
+ private boolean isObserverReadEligible(String nsId, Method method) {
|
|
|
+ boolean isReadEnabledForNamespace = observerReadEnabledDefault != observerReadEnabledOverrides.contains(nsId);
|
|
|
+ return isReadEnabledForNamespace && isReadCall(method);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Check if a method is read-only.
|
|
|
+ * @return whether the 'method' is a read-only operation.
|
|
|
+ */
|
|
|
+ private static boolean isReadCall(Method method) {
|
|
|
+ if (!method.isAnnotationPresent(ReadOnly.class)) {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ return !method.getAnnotationsByType(ReadOnly.class)[0].activeOnly();
|
|
|
+ }
|
|
|
}
|