|
@@ -18,9 +18,6 @@
|
|
package org.apache.hadoop.hdfs;
|
|
package org.apache.hadoop.hdfs;
|
|
|
|
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
-import java.lang.reflect.Constructor;
|
|
|
|
-import java.lang.reflect.InvocationHandler;
|
|
|
|
-import java.lang.reflect.Proxy;
|
|
|
|
import java.net.InetSocketAddress;
|
|
import java.net.InetSocketAddress;
|
|
import java.net.URI;
|
|
import java.net.URI;
|
|
import java.util.HashMap;
|
|
import java.util.HashMap;
|
|
@@ -32,31 +29,19 @@ import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.LogFactory;
|
|
import org.apache.commons.logging.LogFactory;
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
-import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
|
|
|
|
-import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
|
|
|
|
|
|
+import org.apache.hadoop.hdfs.NameNodeProxiesClient.ProxyAndInfo;
|
|
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
|
|
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
|
|
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
|
-import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB;
|
|
|
|
-import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB;
|
|
|
|
import org.apache.hadoop.hdfs.protocolPB.JournalProtocolPB;
|
|
import org.apache.hadoop.hdfs.protocolPB.JournalProtocolPB;
|
|
import org.apache.hadoop.hdfs.protocolPB.JournalProtocolTranslatorPB;
|
|
import org.apache.hadoop.hdfs.protocolPB.JournalProtocolTranslatorPB;
|
|
import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolPB;
|
|
import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolPB;
|
|
import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolTranslatorPB;
|
|
import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolTranslatorPB;
|
|
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
|
|
|
-import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
|
|
|
|
import org.apache.hadoop.hdfs.server.namenode.ha.AbstractNNFailoverProxyProvider;
|
|
import org.apache.hadoop.hdfs.server.namenode.ha.AbstractNNFailoverProxyProvider;
|
|
-import org.apache.hadoop.hdfs.server.namenode.ha.WrappedFailoverProxyProvider;
|
|
|
|
import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
|
|
import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
|
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
|
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
|
|
-import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
|
|
|
import org.apache.hadoop.io.Text;
|
|
import org.apache.hadoop.io.Text;
|
|
-import org.apache.hadoop.io.retry.DefaultFailoverProxyProvider;
|
|
|
|
-import org.apache.hadoop.io.retry.FailoverProxyProvider;
|
|
|
|
-import org.apache.hadoop.io.retry.LossyRetryInvocationHandler;
|
|
|
|
import org.apache.hadoop.io.retry.RetryPolicies;
|
|
import org.apache.hadoop.io.retry.RetryPolicies;
|
|
import org.apache.hadoop.io.retry.RetryPolicy;
|
|
import org.apache.hadoop.io.retry.RetryPolicy;
|
|
import org.apache.hadoop.io.retry.RetryProxy;
|
|
import org.apache.hadoop.io.retry.RetryProxy;
|
|
-import org.apache.hadoop.io.retry.RetryUtils;
|
|
|
|
import org.apache.hadoop.ipc.ProtobufRpcEngine;
|
|
import org.apache.hadoop.ipc.ProtobufRpcEngine;
|
|
import org.apache.hadoop.ipc.RPC;
|
|
import org.apache.hadoop.ipc.RPC;
|
|
import org.apache.hadoop.ipc.RefreshCallQueueProtocol;
|
|
import org.apache.hadoop.ipc.RefreshCallQueueProtocol;
|
|
@@ -75,9 +60,6 @@ import org.apache.hadoop.tools.GetUserMappingsProtocol;
|
|
import org.apache.hadoop.tools.protocolPB.GetUserMappingsProtocolClientSideTranslatorPB;
|
|
import org.apache.hadoop.tools.protocolPB.GetUserMappingsProtocolClientSideTranslatorPB;
|
|
import org.apache.hadoop.tools.protocolPB.GetUserMappingsProtocolPB;
|
|
import org.apache.hadoop.tools.protocolPB.GetUserMappingsProtocolPB;
|
|
|
|
|
|
-import com.google.common.annotations.VisibleForTesting;
|
|
|
|
-import com.google.common.base.Preconditions;
|
|
|
|
-
|
|
|
|
/**
|
|
/**
|
|
* Create proxy objects to communicate with a remote NN. All remote access to an
|
|
* Create proxy objects to communicate with a remote NN. All remote access to an
|
|
* NN should be funneled through this class. Most of the time you'll want to use
|
|
* NN should be funneled through this class. Most of the time you'll want to use
|
|
@@ -89,37 +71,6 @@ public class NameNodeProxies {
|
|
|
|
|
|
private static final Log LOG = LogFactory.getLog(NameNodeProxies.class);
|
|
private static final Log LOG = LogFactory.getLog(NameNodeProxies.class);
|
|
|
|
|
|
- /**
|
|
|
|
- * Wrapper for a client proxy as well as its associated service ID.
|
|
|
|
- * This is simply used as a tuple-like return type for
|
|
|
|
- * {@link NameNodeProxies#createProxy} and
|
|
|
|
- * {@link NameNodeProxies#createNonHAProxy}.
|
|
|
|
- */
|
|
|
|
- public static class ProxyAndInfo<PROXYTYPE> {
|
|
|
|
- private final PROXYTYPE proxy;
|
|
|
|
- private final Text dtService;
|
|
|
|
- private final InetSocketAddress address;
|
|
|
|
-
|
|
|
|
- public ProxyAndInfo(PROXYTYPE proxy, Text dtService,
|
|
|
|
- InetSocketAddress address) {
|
|
|
|
- this.proxy = proxy;
|
|
|
|
- this.dtService = dtService;
|
|
|
|
- this.address = address;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- public PROXYTYPE getProxy() {
|
|
|
|
- return proxy;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- public Text getDelegationTokenService() {
|
|
|
|
- return dtService;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- public InetSocketAddress getAddress() {
|
|
|
|
- return address;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
/**
|
|
/**
|
|
* Creates the namenode proxy with the passed protocol. This will handle
|
|
* Creates the namenode proxy with the passed protocol. This will handle
|
|
* creation of either HA- or non-HA-enabled proxy objects, depending upon
|
|
* creation of either HA- or non-HA-enabled proxy objects, depending upon
|
|
@@ -160,103 +111,16 @@ public class NameNodeProxies {
|
|
URI nameNodeUri, Class<T> xface, AtomicBoolean fallbackToSimpleAuth)
|
|
URI nameNodeUri, Class<T> xface, AtomicBoolean fallbackToSimpleAuth)
|
|
throws IOException {
|
|
throws IOException {
|
|
AbstractNNFailoverProxyProvider<T> failoverProxyProvider =
|
|
AbstractNNFailoverProxyProvider<T> failoverProxyProvider =
|
|
- createFailoverProxyProvider(conf, nameNodeUri, xface, true,
|
|
|
|
- fallbackToSimpleAuth);
|
|
|
|
-
|
|
|
|
|
|
+ NameNodeProxiesClient.createFailoverProxyProvider(conf, nameNodeUri,
|
|
|
|
+ xface, true, fallbackToSimpleAuth);
|
|
|
|
+
|
|
if (failoverProxyProvider == null) {
|
|
if (failoverProxyProvider == null) {
|
|
- // Non-HA case
|
|
|
|
return createNonHAProxy(conf, DFSUtilClient.getNNAddress(nameNodeUri),
|
|
return createNonHAProxy(conf, DFSUtilClient.getNNAddress(nameNodeUri),
|
|
xface, UserGroupInformation.getCurrentUser(), true,
|
|
xface, UserGroupInformation.getCurrentUser(), true,
|
|
fallbackToSimpleAuth);
|
|
fallbackToSimpleAuth);
|
|
} else {
|
|
} else {
|
|
- // HA case
|
|
|
|
- DfsClientConf config = new DfsClientConf(conf);
|
|
|
|
- T proxy = (T) RetryProxy.create(xface, failoverProxyProvider,
|
|
|
|
- RetryPolicies.failoverOnNetworkException(
|
|
|
|
- RetryPolicies.TRY_ONCE_THEN_FAIL, config.getMaxFailoverAttempts(),
|
|
|
|
- config.getMaxRetryAttempts(), config.getFailoverSleepBaseMillis(),
|
|
|
|
- config.getFailoverSleepMaxMillis()));
|
|
|
|
-
|
|
|
|
- Text dtService;
|
|
|
|
- if (failoverProxyProvider.useLogicalURI()) {
|
|
|
|
- dtService = HAUtilClient.buildTokenServiceForLogicalUri(nameNodeUri,
|
|
|
|
- HdfsConstants.HDFS_URI_SCHEME);
|
|
|
|
- } else {
|
|
|
|
- dtService = SecurityUtil.buildTokenService(
|
|
|
|
- DFSUtilClient.getNNAddress(nameNodeUri));
|
|
|
|
- }
|
|
|
|
- return new ProxyAndInfo<T>(proxy, dtService,
|
|
|
|
- DFSUtilClient.getNNAddress(nameNodeUri));
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Generate a dummy namenode proxy instance that utilizes our hacked
|
|
|
|
- * {@link LossyRetryInvocationHandler}. Proxy instance generated using this
|
|
|
|
- * method will proactively drop RPC responses. Currently this method only
|
|
|
|
- * support HA setup. null will be returned if the given configuration is not
|
|
|
|
- * for HA.
|
|
|
|
- *
|
|
|
|
- * @param config the configuration containing the required IPC
|
|
|
|
- * properties, client failover configurations, etc.
|
|
|
|
- * @param nameNodeUri the URI pointing either to a specific NameNode
|
|
|
|
- * or to a logical nameservice.
|
|
|
|
- * @param xface the IPC interface which should be created
|
|
|
|
- * @param numResponseToDrop The number of responses to drop for each RPC call
|
|
|
|
- * @param fallbackToSimpleAuth set to true or false during calls to indicate if
|
|
|
|
- * a secure client falls back to simple auth
|
|
|
|
- * @return an object containing both the proxy and the associated
|
|
|
|
- * delegation token service it corresponds to. Will return null of the
|
|
|
|
- * given configuration does not support HA.
|
|
|
|
- * @throws IOException if there is an error creating the proxy
|
|
|
|
- */
|
|
|
|
- @SuppressWarnings("unchecked")
|
|
|
|
- public static <T> ProxyAndInfo<T> createProxyWithLossyRetryHandler(
|
|
|
|
- Configuration config, URI nameNodeUri, Class<T> xface,
|
|
|
|
- int numResponseToDrop, AtomicBoolean fallbackToSimpleAuth)
|
|
|
|
- throws IOException {
|
|
|
|
- Preconditions.checkArgument(numResponseToDrop > 0);
|
|
|
|
- AbstractNNFailoverProxyProvider<T> failoverProxyProvider =
|
|
|
|
- createFailoverProxyProvider(config, nameNodeUri, xface, true,
|
|
|
|
- fallbackToSimpleAuth);
|
|
|
|
-
|
|
|
|
- if (failoverProxyProvider != null) { // HA case
|
|
|
|
- int delay = config.getInt(
|
|
|
|
- HdfsClientConfigKeys.Failover.SLEEPTIME_BASE_KEY,
|
|
|
|
- HdfsClientConfigKeys.Failover.SLEEPTIME_BASE_DEFAULT);
|
|
|
|
- int maxCap = config.getInt(
|
|
|
|
- HdfsClientConfigKeys.Failover.SLEEPTIME_MAX_KEY,
|
|
|
|
- HdfsClientConfigKeys.Failover.SLEEPTIME_MAX_DEFAULT);
|
|
|
|
- int maxFailoverAttempts = config.getInt(
|
|
|
|
- HdfsClientConfigKeys.Failover.MAX_ATTEMPTS_KEY,
|
|
|
|
- HdfsClientConfigKeys.Failover.MAX_ATTEMPTS_DEFAULT);
|
|
|
|
- int maxRetryAttempts = config.getInt(
|
|
|
|
- HdfsClientConfigKeys.Retry.MAX_ATTEMPTS_KEY,
|
|
|
|
- HdfsClientConfigKeys.Retry.MAX_ATTEMPTS_DEFAULT);
|
|
|
|
- InvocationHandler dummyHandler = new LossyRetryInvocationHandler<T>(
|
|
|
|
- numResponseToDrop, failoverProxyProvider,
|
|
|
|
- RetryPolicies.failoverOnNetworkException(
|
|
|
|
- RetryPolicies.TRY_ONCE_THEN_FAIL, maxFailoverAttempts,
|
|
|
|
- Math.max(numResponseToDrop + 1, maxRetryAttempts), delay,
|
|
|
|
- maxCap));
|
|
|
|
-
|
|
|
|
- T proxy = (T) Proxy.newProxyInstance(
|
|
|
|
- failoverProxyProvider.getInterface().getClassLoader(),
|
|
|
|
- new Class[] { xface }, dummyHandler);
|
|
|
|
- Text dtService;
|
|
|
|
- if (failoverProxyProvider.useLogicalURI()) {
|
|
|
|
- dtService = HAUtilClient.buildTokenServiceForLogicalUri(nameNodeUri,
|
|
|
|
- HdfsConstants.HDFS_URI_SCHEME);
|
|
|
|
- } else {
|
|
|
|
- dtService = SecurityUtil.buildTokenService(
|
|
|
|
- DFSUtilClient.getNNAddress(nameNodeUri));
|
|
|
|
- }
|
|
|
|
- return new ProxyAndInfo<T>(proxy, dtService,
|
|
|
|
- DFSUtilClient.getNNAddress(nameNodeUri));
|
|
|
|
- } else {
|
|
|
|
- LOG.warn("Currently creating proxy using " +
|
|
|
|
- "LossyRetryInvocationHandler requires NN HA setup");
|
|
|
|
- return null;
|
|
|
|
|
|
+ return NameNodeProxiesClient.createHAProxy(conf, nameNodeUri, xface,
|
|
|
|
+ failoverProxyProvider);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -303,8 +167,8 @@ public class NameNodeProxies {
|
|
|
|
|
|
T proxy;
|
|
T proxy;
|
|
if (xface == ClientProtocol.class) {
|
|
if (xface == ClientProtocol.class) {
|
|
- proxy = (T) createNNProxyWithClientProtocol(nnAddr, conf, ugi,
|
|
|
|
- withRetries, fallbackToSimpleAuth);
|
|
|
|
|
|
+ proxy = (T) NameNodeProxiesClient.createNonHAProxyWithClientProtocol(
|
|
|
|
+ nnAddr, conf, ugi, withRetries, fallbackToSimpleAuth);
|
|
} else if (xface == JournalProtocol.class) {
|
|
} else if (xface == JournalProtocol.class) {
|
|
proxy = (T) createNNProxyWithJournalProtocol(nnAddr, conf, ugi);
|
|
proxy = (T) createNNProxyWithJournalProtocol(nnAddr, conf, ugi);
|
|
} else if (xface == NamenodeProtocol.class) {
|
|
} else if (xface == NamenodeProtocol.class) {
|
|
@@ -390,45 +254,6 @@ public class NameNodeProxies {
|
|
return new NamenodeProtocolTranslatorPB(proxy);
|
|
return new NamenodeProtocolTranslatorPB(proxy);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
- private static ClientProtocol createNNProxyWithClientProtocol(
|
|
|
|
- InetSocketAddress address, Configuration conf, UserGroupInformation ugi,
|
|
|
|
- boolean withRetries, AtomicBoolean fallbackToSimpleAuth)
|
|
|
|
- throws IOException {
|
|
|
|
- RPC.setProtocolEngine(conf, ClientNamenodeProtocolPB.class, ProtobufRpcEngine.class);
|
|
|
|
-
|
|
|
|
- final RetryPolicy defaultPolicy =
|
|
|
|
- RetryUtils.getDefaultRetryPolicy(
|
|
|
|
- conf,
|
|
|
|
- HdfsClientConfigKeys.Retry.POLICY_ENABLED_KEY,
|
|
|
|
- HdfsClientConfigKeys.Retry.POLICY_ENABLED_DEFAULT,
|
|
|
|
- HdfsClientConfigKeys.Retry.POLICY_SPEC_KEY,
|
|
|
|
- HdfsClientConfigKeys.Retry.POLICY_SPEC_DEFAULT,
|
|
|
|
- SafeModeException.class.getName());
|
|
|
|
-
|
|
|
|
- final long version = RPC.getProtocolVersion(ClientNamenodeProtocolPB.class);
|
|
|
|
- ClientNamenodeProtocolPB proxy = RPC.getProtocolProxy(
|
|
|
|
- ClientNamenodeProtocolPB.class, version, address, ugi, conf,
|
|
|
|
- NetUtils.getDefaultSocketFactory(conf),
|
|
|
|
- org.apache.hadoop.ipc.Client.getTimeout(conf), defaultPolicy,
|
|
|
|
- fallbackToSimpleAuth).getProxy();
|
|
|
|
-
|
|
|
|
- if (withRetries) { // create the proxy with retries
|
|
|
|
-
|
|
|
|
- Map<String, RetryPolicy> methodNameToPolicyMap
|
|
|
|
- = new HashMap<String, RetryPolicy>();
|
|
|
|
- ClientProtocol translatorProxy =
|
|
|
|
- new ClientNamenodeProtocolTranslatorPB(proxy);
|
|
|
|
- return (ClientProtocol) RetryProxy.create(
|
|
|
|
- ClientProtocol.class,
|
|
|
|
- new DefaultFailoverProxyProvider<ClientProtocol>(
|
|
|
|
- ClientProtocol.class, translatorProxy),
|
|
|
|
- methodNameToPolicyMap,
|
|
|
|
- defaultPolicy);
|
|
|
|
- } else {
|
|
|
|
- return new ClientNamenodeProtocolTranslatorPB(proxy);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
|
|
|
|
private static Object createNameNodeProxy(InetSocketAddress address,
|
|
private static Object createNameNodeProxy(InetSocketAddress address,
|
|
Configuration conf, UserGroupInformation ugi, Class<?> xface,
|
|
Configuration conf, UserGroupInformation ugi, Class<?> xface,
|
|
@@ -439,88 +264,4 @@ public class NameNodeProxies {
|
|
return proxy;
|
|
return proxy;
|
|
}
|
|
}
|
|
|
|
|
|
- /** Gets the configured Failover proxy provider's class */
|
|
|
|
- @VisibleForTesting
|
|
|
|
- public static <T> Class<FailoverProxyProvider<T>> getFailoverProxyProviderClass(
|
|
|
|
- Configuration conf, URI nameNodeUri) throws IOException {
|
|
|
|
- if (nameNodeUri == null) {
|
|
|
|
- return null;
|
|
|
|
- }
|
|
|
|
- String host = nameNodeUri.getHost();
|
|
|
|
- String configKey = HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX
|
|
|
|
- + "." + host;
|
|
|
|
- try {
|
|
|
|
- @SuppressWarnings("unchecked")
|
|
|
|
- Class<FailoverProxyProvider<T>> ret = (Class<FailoverProxyProvider<T>>) conf
|
|
|
|
- .getClass(configKey, null, FailoverProxyProvider.class);
|
|
|
|
- return ret;
|
|
|
|
- } catch (RuntimeException e) {
|
|
|
|
- if (e.getCause() instanceof ClassNotFoundException) {
|
|
|
|
- throw new IOException("Could not load failover proxy provider class "
|
|
|
|
- + conf.get(configKey) + " which is configured for authority "
|
|
|
|
- + nameNodeUri, e);
|
|
|
|
- } else {
|
|
|
|
- throw e;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /** Creates the Failover proxy provider instance*/
|
|
|
|
- @VisibleForTesting
|
|
|
|
- public static <T> AbstractNNFailoverProxyProvider<T> createFailoverProxyProvider(
|
|
|
|
- Configuration conf, URI nameNodeUri, Class<T> xface, boolean checkPort,
|
|
|
|
- AtomicBoolean fallbackToSimpleAuth) throws IOException {
|
|
|
|
- Class<FailoverProxyProvider<T>> failoverProxyProviderClass = null;
|
|
|
|
- AbstractNNFailoverProxyProvider<T> providerNN;
|
|
|
|
- Preconditions.checkArgument(
|
|
|
|
- xface.isAssignableFrom(NamenodeProtocols.class),
|
|
|
|
- "Interface %s is not a NameNode protocol", xface);
|
|
|
|
- try {
|
|
|
|
- // Obtain the class of the proxy provider
|
|
|
|
- failoverProxyProviderClass = getFailoverProxyProviderClass(conf,
|
|
|
|
- nameNodeUri);
|
|
|
|
- if (failoverProxyProviderClass == null) {
|
|
|
|
- return null;
|
|
|
|
- }
|
|
|
|
- // Create a proxy provider instance.
|
|
|
|
- Constructor<FailoverProxyProvider<T>> ctor = failoverProxyProviderClass
|
|
|
|
- .getConstructor(Configuration.class, URI.class, Class.class);
|
|
|
|
- FailoverProxyProvider<T> provider = ctor.newInstance(conf, nameNodeUri,
|
|
|
|
- xface);
|
|
|
|
-
|
|
|
|
- // If the proxy provider is of an old implementation, wrap it.
|
|
|
|
- if (!(provider instanceof AbstractNNFailoverProxyProvider)) {
|
|
|
|
- providerNN = new WrappedFailoverProxyProvider<T>(provider);
|
|
|
|
- } else {
|
|
|
|
- providerNN = (AbstractNNFailoverProxyProvider<T>)provider;
|
|
|
|
- }
|
|
|
|
- } catch (Exception e) {
|
|
|
|
- String message = "Couldn't create proxy provider " + failoverProxyProviderClass;
|
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug(message, e);
|
|
|
|
- }
|
|
|
|
- if (e.getCause() instanceof IOException) {
|
|
|
|
- throw (IOException) e.getCause();
|
|
|
|
- } else {
|
|
|
|
- throw new IOException(message, e);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- // Check the port in the URI, if it is logical.
|
|
|
|
- if (checkPort && providerNN.useLogicalURI()) {
|
|
|
|
- int port = nameNodeUri.getPort();
|
|
|
|
- if (port > 0 &&
|
|
|
|
- port != HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT) {
|
|
|
|
- // Throwing here without any cleanup is fine since we have not
|
|
|
|
- // actually created the underlying proxies yet.
|
|
|
|
- throw new IOException("Port " + port + " specified in URI "
|
|
|
|
- + nameNodeUri + " but host '" + nameNodeUri.getHost()
|
|
|
|
- + "' is a logical (HA) namenode"
|
|
|
|
- + " and does not use port information.");
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- providerNN.setFallbackToSimpleAuth(fallbackToSimpleAuth);
|
|
|
|
- return providerNN;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
}
|
|
}
|