|
@@ -17,31 +17,31 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.hdfs.server.namenode.ha;
|
|
|
|
|
|
+import java.io.Closeable;
|
|
|
import java.io.IOException;
|
|
|
import java.lang.reflect.InvocationHandler;
|
|
|
import java.lang.reflect.Method;
|
|
|
import java.lang.reflect.Proxy;
|
|
|
import java.net.URI;
|
|
|
import java.util.ArrayList;
|
|
|
-import java.util.Collections;
|
|
|
import java.util.List;
|
|
|
|
|
|
import java.util.function.Supplier;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.permission.FsAction;
|
|
|
import org.apache.hadoop.hdfs.ClientGSIContext;
|
|
|
-import org.apache.hadoop.hdfs.NameNodeProxiesClient;
|
|
|
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
|
|
|
-
|
|
|
-import com.google.common.annotations.VisibleForTesting;
|
|
|
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
import org.apache.hadoop.ipc.AlignmentContext;
|
|
|
+import org.apache.hadoop.ipc.RPC;
|
|
|
import org.apache.hadoop.ipc.RemoteException;
|
|
|
import org.apache.hadoop.ipc.StandbyException;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
+import com.google.common.annotations.VisibleForTesting;
|
|
|
+
|
|
|
/**
|
|
|
* A {@link org.apache.hadoop.io.retry.FailoverProxyProvider} implementation
|
|
|
* that supports reading from observer namenode(s).
|
|
@@ -56,16 +56,20 @@ import org.slf4j.LoggerFactory;
|
|
|
* observer is turned off.
|
|
|
*/
|
|
|
public class ObserverReadProxyProvider<T extends ClientProtocol>
|
|
|
- extends ConfiguredFailoverProxyProvider<T> {
|
|
|
+ extends AbstractNNFailoverProxyProvider<T> {
|
|
|
private static final Logger LOG = LoggerFactory.getLogger(
|
|
|
ObserverReadProxyProvider.class);
|
|
|
|
|
|
/** Client-side context for syncing with the NameNode server side */
|
|
|
private AlignmentContext alignmentContext;
|
|
|
|
|
|
+ private AbstractNNFailoverProxyProvider<T> failoverProxy;
|
|
|
+ /** All NameNdoe proxies */
|
|
|
+ private List<NNProxyInfo<T>> nameNodeProxies =
|
|
|
+ new ArrayList<NNProxyInfo<T>>();
|
|
|
/** Proxies for the observer namenodes */
|
|
|
- private final List<AddressRpcProxyPair<T>> observerProxies =
|
|
|
- new ArrayList<>();
|
|
|
+ private final List<NNProxyInfo<T>> observerProxies =
|
|
|
+ new ArrayList<NNProxyInfo<T>>();
|
|
|
|
|
|
/**
|
|
|
* Whether reading from observer is enabled. If this is false, all read
|
|
@@ -87,36 +91,43 @@ public class ObserverReadProxyProvider<T extends ClientProtocol>
|
|
|
/** The last proxy that has been used. Only used for testing */
|
|
|
private volatile ProxyInfo<T> lastProxy = null;
|
|
|
|
|
|
- @SuppressWarnings("unchecked")
|
|
|
+ /**
|
|
|
+ * By default ObserverReadProxyProvider uses
|
|
|
+ * {@link ConfiguredFailoverProxyProvider} for failover.
|
|
|
+ */
|
|
|
public ObserverReadProxyProvider(
|
|
|
Configuration conf, URI uri, Class<T> xface, HAProxyFactory<T> factory)
|
|
|
throws IOException {
|
|
|
+ this(conf, uri, xface, factory,
|
|
|
+ new ConfiguredFailoverProxyProvider<T>(conf, uri, xface,factory));
|
|
|
+ }
|
|
|
+
|
|
|
+ public ObserverReadProxyProvider(
|
|
|
+ Configuration conf, URI uri, Class<T> xface, HAProxyFactory<T> factory,
|
|
|
+ AbstractNNFailoverProxyProvider<T> failoverProxy)
|
|
|
+ throws IOException {
|
|
|
super(conf, uri, xface, factory);
|
|
|
- alignmentContext = new ClientGSIContext();
|
|
|
+ this.failoverProxy = failoverProxy;
|
|
|
+ this.alignmentContext = new ClientGSIContext();
|
|
|
((ClientHAProxyFactory<T>) factory).setAlignmentContext(alignmentContext);
|
|
|
|
|
|
+ // Get all NameNode proxies
|
|
|
+ nameNodeProxies = getProxyAddresses(uri,
|
|
|
+ HdfsClientConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY);
|
|
|
// Find out all the observer proxies
|
|
|
- for (AddressRpcProxyPair<T> ap : this.proxies) {
|
|
|
- ap.namenode = (T) NameNodeProxiesClient.createProxyWithAlignmentContext(
|
|
|
- ap.address, conf, ugi, false, getFallbackToSimpleAuth(),
|
|
|
- alignmentContext);
|
|
|
- if (isObserverState(ap)) {
|
|
|
- observerProxies.add(ap);
|
|
|
+ for (NNProxyInfo<T> pi : nameNodeProxies) {
|
|
|
+ createProxyIfNeeded(pi);
|
|
|
+ if (isObserverState(pi)) {
|
|
|
+ observerProxies.add(pi);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ // TODO: No observers is not an error
|
|
|
+ // Just direct all reads go to the active NameNode
|
|
|
if (observerProxies.isEmpty()) {
|
|
|
throw new RuntimeException("Couldn't find any namenode proxy in " +
|
|
|
"OBSERVER state");
|
|
|
}
|
|
|
-
|
|
|
- // Randomize the list to prevent all clients pointing to the same one
|
|
|
- boolean randomized = conf.getBoolean(
|
|
|
- HdfsClientConfigKeys.Failover.RANDOM_ORDER,
|
|
|
- HdfsClientConfigKeys.Failover.RANDOM_ORDER_DEFAULT);
|
|
|
- if (randomized) {
|
|
|
- Collections.shuffle(observerProxies);
|
|
|
- }
|
|
|
}
|
|
|
|
|
|
public synchronized AlignmentContext getAlignmentContext() {
|
|
@@ -127,17 +138,13 @@ public class ObserverReadProxyProvider<T extends ClientProtocol>
|
|
|
@Override
|
|
|
public synchronized ProxyInfo<T> getProxy() {
|
|
|
// We just create a wrapped proxy containing all the proxies
|
|
|
- List<ProxyInfo<T>> observerProxies = new ArrayList<>();
|
|
|
StringBuilder combinedInfo = new StringBuilder("[");
|
|
|
|
|
|
for (int i = 0; i < this.observerProxies.size(); i++) {
|
|
|
if (i > 0) {
|
|
|
combinedInfo.append(",");
|
|
|
}
|
|
|
- AddressRpcProxyPair<T> p = this.observerProxies.get(i);
|
|
|
- ProxyInfo<T> pInfo = getProxy(p);
|
|
|
- observerProxies.add(pInfo);
|
|
|
- combinedInfo.append(pInfo.proxyInfo);
|
|
|
+ combinedInfo.append(observerProxies.get(i).proxyInfo);
|
|
|
}
|
|
|
|
|
|
combinedInfo.append(']');
|
|
@@ -148,6 +155,11 @@ public class ObserverReadProxyProvider<T extends ClientProtocol>
|
|
|
return new ProxyInfo<>(wrappedProxy, combinedInfo.toString());
|
|
|
}
|
|
|
|
|
|
+ @Override
|
|
|
+ public void performFailover(T currentProxy) {
|
|
|
+ failoverProxy.performFailover(currentProxy);
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Check if a method is read-only.
|
|
|
*
|
|
@@ -176,14 +188,14 @@ public class ObserverReadProxyProvider<T extends ClientProtocol>
|
|
|
return lastProxy;
|
|
|
}
|
|
|
|
|
|
- boolean isObserverState(AddressRpcProxyPair<T> ap) {
|
|
|
+ boolean isObserverState(NNProxyInfo<T> pi) {
|
|
|
// TODO: should introduce new ClientProtocol method to verify the
|
|
|
// underlying service state, which does not require superuser access
|
|
|
// The is a workaround
|
|
|
IOException ioe = null;
|
|
|
try {
|
|
|
// Verify write access first
|
|
|
- ap.namenode.reportBadBlocks(new LocatedBlock[0]);
|
|
|
+ pi.proxy.reportBadBlocks(new LocatedBlock[0]);
|
|
|
return false; // Only active NameNode allows write
|
|
|
} catch (RemoteException re) {
|
|
|
IOException sbe = re.unwrapRemoteException(StandbyException.class);
|
|
@@ -194,14 +206,14 @@ public class ObserverReadProxyProvider<T extends ClientProtocol>
|
|
|
ioe = e;
|
|
|
}
|
|
|
if (ioe != null) {
|
|
|
- LOG.error("Failed to connect to {}", ap.address, ioe);
|
|
|
+ LOG.error("Failed to connect to {}", pi.getAddress(), ioe);
|
|
|
return false;
|
|
|
}
|
|
|
// Verify read access
|
|
|
// For now we assume only Observer nodes allow reads
|
|
|
// Stale reads on StandbyNode should be turned off
|
|
|
try {
|
|
|
- ap.namenode.checkAccess("/", FsAction.READ);
|
|
|
+ pi.proxy.checkAccess("/", FsAction.READ);
|
|
|
return true;
|
|
|
} catch (RemoteException re) {
|
|
|
IOException sbe = re.unwrapRemoteException(StandbyException.class);
|
|
@@ -212,19 +224,19 @@ public class ObserverReadProxyProvider<T extends ClientProtocol>
|
|
|
ioe = e;
|
|
|
}
|
|
|
if (ioe != null) {
|
|
|
- LOG.error("Failed to connect to {}", ap.address, ioe);
|
|
|
+ LOG.error("Failed to connect to {}", pi.getAddress(), ioe);
|
|
|
}
|
|
|
return false;
|
|
|
}
|
|
|
|
|
|
|
|
|
class ObserverReadInvocationHandler implements InvocationHandler {
|
|
|
- final List<ProxyInfo<T>> observerProxies;
|
|
|
+ final List<NNProxyInfo<T>> observerProxies;
|
|
|
final ProxyInfo<T> activeProxy;
|
|
|
|
|
|
- ObserverReadInvocationHandler(List<ProxyInfo<T>> observerProxies) {
|
|
|
+ ObserverReadInvocationHandler(List<NNProxyInfo<T>> observerProxies) {
|
|
|
this.observerProxies = observerProxies;
|
|
|
- this.activeProxy = ObserverReadProxyProvider.super.getProxy();
|
|
|
+ this.activeProxy = failoverProxy.getProxy();
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -244,7 +256,7 @@ public class ObserverReadProxyProvider<T extends ClientProtocol>
|
|
|
if (observerReadEnabled && isRead(method)) {
|
|
|
// Loop through all the proxies, starting from the current index.
|
|
|
for (int i = 0; i < observerProxies.size(); i++) {
|
|
|
- ProxyInfo<T> current = observerProxies.get(currentIndex.get());
|
|
|
+ NNProxyInfo<T> current = observerProxies.get(currentIndex.get());
|
|
|
try {
|
|
|
retVal = method.invoke(current.proxy, args);
|
|
|
lastProxy = current;
|
|
@@ -275,4 +287,23 @@ public class ObserverReadProxyProvider<T extends ClientProtocol>
|
|
|
return retVal;
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public synchronized void close() throws IOException {
|
|
|
+ failoverProxy.close();
|
|
|
+ for (ProxyInfo<T> pi : nameNodeProxies) {
|
|
|
+ if (pi.proxy != null) {
|
|
|
+ if (pi.proxy instanceof Closeable) {
|
|
|
+ ((Closeable)pi.proxy).close();
|
|
|
+ } else {
|
|
|
+ RPC.stopProxy(pi.proxy);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public boolean useLogicalURI() {
|
|
|
+ return failoverProxy.useLogicalURI();
|
|
|
+ }
|
|
|
}
|