diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
index 96f925f0f2c0f..72d59fbd76837 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
@@ -1679,11 +1679,13 @@ public static class ConnectionId {
private final boolean doPing; //do we need to send ping message
private final int pingInterval; // how often sends ping to the server in msecs
private String saslQop; // here for testing
+ private final AtomicBoolean fallbackToSimpleAuth;
private final Configuration conf; // used to get the expected kerberos principal name
ConnectionId(InetSocketAddress address, Class> protocol,
UserGroupInformation ticket, int rpcTimeout,
- RetryPolicy connectionRetryPolicy, Configuration conf) {
+ RetryPolicy connectionRetryPolicy, Configuration conf,
+ AtomicBoolean fallbackToSimpleAuth) {
this.protocol = protocol;
this.address = address;
this.ticket = ticket;
@@ -1710,6 +1712,7 @@ public static class ConnectionId {
CommonConfigurationKeys.IPC_CLIENT_PING_KEY,
CommonConfigurationKeys.IPC_CLIENT_PING_DEFAULT);
this.pingInterval = (doPing ? Client.getPingInterval(conf) : 0);
+ this.fallbackToSimpleAuth = fallbackToSimpleAuth;
this.conf = conf;
}
@@ -1772,12 +1775,14 @@ String getSaslQop() {
* @param ticket UGI
* @param rpcTimeout timeout
* @param conf Configuration object
+ * @param fallbackToSimpleAuth AtomicBoolean configuring auth fallback behaviour in rpc calls
* @return A ConnectionId instance
* @throws IOException
*/
static ConnectionId getConnectionId(InetSocketAddress addr,
Class> protocol, UserGroupInformation ticket, int rpcTimeout,
- RetryPolicy connectionRetryPolicy, Configuration conf) throws IOException {
+ RetryPolicy connectionRetryPolicy, Configuration conf,
+ AtomicBoolean fallbackToSimpleAuth) throws IOException {
if (connectionRetryPolicy == null) {
final int max = conf.getInt(
@@ -1793,11 +1798,11 @@ static ConnectionId getConnectionId(InetSocketAddress addr,
}
return new ConnectionId(addr, protocol, ticket, rpcTimeout,
- connectionRetryPolicy, conf);
+ connectionRetryPolicy, conf, fallbackToSimpleAuth);
}
static boolean isEqual(Object a, Object b) {
- return a == null ? b == null : a.equals(b);
+ return Objects.equals(a, b);
}
@Override
@@ -1815,7 +1820,8 @@ && isEqual(this.connectionRetryPolicy, that.connectionRetryPolicy)
&& isEqual(this.protocol, that.protocol)
&& this.rpcTimeout == that.rpcTimeout
&& this.tcpNoDelay == that.tcpNoDelay
- && isEqual(this.ticket, that.ticket);
+ && isEqual(this.ticket, that.ticket)
+ && isEqual(this.fallbackToSimpleAuth, that.fallbackToSimpleAuth);
}
return false;
}
@@ -1831,6 +1837,8 @@ public int hashCode() {
result = PRIME * result + rpcTimeout;
result = PRIME * result + (tcpNoDelay ? 1231 : 1237);
result = PRIME * result + ((ticket == null) ? 0 : ticket.hashCode());
+ result = PRIME * result
+ + (fallbackToSimpleAuth == null ? 0 : fallbackToSimpleAuth.hashCode());
return result;
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
index c4457a653e35f..d1a90160a334b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
@@ -136,7 +136,7 @@ protected Invoker(Class> protocol, InetSocketAddress addr,
AtomicBoolean fallbackToSimpleAuth, AlignmentContext alignmentContext)
throws IOException {
this(protocol, Client.ConnectionId.getConnectionId(
- addr, protocol, ticket, rpcTimeout, connectionRetryPolicy, conf),
+ addr, protocol, ticket, rpcTimeout, connectionRetryPolicy, conf, fallbackToSimpleAuth),
conf, factory);
this.fallbackToSimpleAuth = fallbackToSimpleAuth;
this.alignmentContext = alignmentContext;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine2.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine2.java
index e1ee374282897..b14c331bd5e1c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine2.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine2.java
@@ -143,7 +143,7 @@ protected Invoker(Class> protocol, InetSocketAddress addr,
AtomicBoolean fallbackToSimpleAuth, AlignmentContext alignmentContext)
throws IOException {
this(protocol, Client.ConnectionId.getConnectionId(
- addr, protocol, ticket, rpcTimeout, connectionRetryPolicy, conf),
+ addr, protocol, ticket, rpcTimeout, connectionRetryPolicy, conf, fallbackToSimpleAuth),
conf, factory);
this.fallbackToSimpleAuth = fallbackToSimpleAuth;
this.alignmentContext = alignmentContext;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java
index d790e49f5dcf2..9463a7b4de211 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java
@@ -223,7 +223,7 @@ public Invoker(Class> protocol,
AlignmentContext alignmentContext)
throws IOException {
this.remoteId = Client.ConnectionId.getConnectionId(address, protocol,
- ticket, rpcTimeout, null, conf);
+ ticket, rpcTimeout, null, conf, fallbackToSimpleAuth);
this.client = CLIENTS.getClient(conf, factory);
this.fallbackToSimpleAuth = fallbackToSimpleAuth;
this.alignmentContext = alignmentContext;
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java
index 95ff302103d89..d18872bd6629f 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java
@@ -142,7 +142,7 @@ public void setupConf() {
static ConnectionId getConnectionId(InetSocketAddress addr, int rpcTimeout,
Configuration conf) throws IOException {
return ConnectionId.getConnectionId(addr, null,
- UserGroupInformation.getCurrentUser(), rpcTimeout, null, conf);
+ UserGroupInformation.getCurrentUser(), rpcTimeout, null, conf, null);
}
static Writable call(Client client, InetSocketAddress addr,
@@ -1707,7 +1707,7 @@ private Socket checkConnect(String addr, boolean asProxy) throws Exception {
final LongWritable param = new LongWritable(RANDOM.nextLong());
final ConnectionId remoteId = new ConnectionId(
sockAddr, TestBindingProtocol.class, ugi, 0,
- RetryPolicies.TRY_ONCE_THEN_FAIL, conf);
+ RetryPolicies.TRY_ONCE_THEN_FAIL, conf, null);
client.call(RPC.RpcKind.RPC_BUILTIN, param, remoteId, null);
fail("call didn't throw connect exception");
} catch (SocketException se) {
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPCServerResponder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPCServerResponder.java
index 7d7905e6b4674..b7a8729dc1980 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPCServerResponder.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPCServerResponder.java
@@ -71,7 +71,7 @@ public class TestIPCServerResponder {
static Writable call(Client client, Writable param,
InetSocketAddress address) throws IOException {
final ConnectionId remoteId = ConnectionId.getConnectionId(address, null,
- null, 0, null, conf);
+ null, 0, null, conf, null);
return client.call(RpcKind.RPC_BUILTIN, param, remoteId,
RPC.RPC_SERVICE_CLASS_DEFAULT, null);
}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRpcBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRpcBase.java
index 0962b50099c57..2c8508231e5b6 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRpcBase.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRpcBase.java
@@ -22,6 +22,7 @@
import org.apache.hadoop.thirdparty.protobuf.RpcController;
import org.apache.hadoop.thirdparty.protobuf.ServiceException;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import org.apache.hadoop.conf.Configuration;
@@ -124,18 +125,19 @@ protected static RPC.Server setupTestServer(
return server;
}
+ protected static TestRpcService getClient(InetSocketAddress serverAddr, Configuration clientConf)
+ throws ServiceException {
+ return getClient(serverAddr, clientConf, null);
+ }
+
protected static TestRpcService getClient(InetSocketAddress serverAddr,
- Configuration clientConf)
- throws ServiceException {
- try {
- return RPC.getProxy(TestRpcService.class, 0, serverAddr, clientConf);
- } catch (IOException e) {
- throw new ServiceException(e);
- }
+ Configuration clientConf, RetryPolicy connectionRetryPolicy) throws ServiceException {
+ return getClient(serverAddr, clientConf, connectionRetryPolicy, null);
}
protected static TestRpcService getClient(InetSocketAddress serverAddr,
- Configuration clientConf, final RetryPolicy connectionRetryPolicy)
+ Configuration clientConf, final RetryPolicy connectionRetryPolicy,
+ AtomicBoolean fallbackToSimpleAuth)
throws ServiceException {
try {
return RPC.getProtocolProxy(
@@ -146,7 +148,7 @@ protected static TestRpcService getClient(InetSocketAddress serverAddr,
clientConf,
NetUtils.getDefaultSocketFactory(clientConf),
RPC.getRpcTimeout(clientConf),
- connectionRetryPolicy, null).getProxy();
+ connectionRetryPolicy, fallbackToSimpleAuth).getProxy();
} catch (IOException e) {
throw new ServiceException(e);
}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRpcServerHandoff.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRpcServerHandoff.java
index 2e0b3daa220a2..3df523c2d081f 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRpcServerHandoff.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRpcServerHandoff.java
@@ -200,9 +200,8 @@ public Writable call() throws Exception {
Writable param = new BytesWritable(requestBytes);
final Client.ConnectionId remoteId =
Client.ConnectionId.getConnectionId(address, null,
- null, 0, null, conf);
- Writable result = client.call(RPC.RpcKind.RPC_BUILTIN, param, remoteId,
- new AtomicBoolean(false));
+ null, 0, null, conf, null);
+ Writable result = client.call(RPC.RpcKind.RPC_BUILTIN, param, remoteId, null);
return result;
}
}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java
index 72085a19ec711..c36f2ea9fec1f 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java
@@ -72,6 +72,7 @@
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.regex.Pattern;
@@ -321,13 +322,13 @@ public void testPingInterval() throws Exception {
// set doPing to true
newConf.setBoolean(CommonConfigurationKeys.IPC_CLIENT_PING_KEY, true);
ConnectionId remoteId = ConnectionId.getConnectionId(
- new InetSocketAddress(0), TestRpcService.class, null, 0, null, newConf);
+ new InetSocketAddress(0), TestRpcService.class, null, 0, null, newConf, null);
assertEquals(CommonConfigurationKeys.IPC_PING_INTERVAL_DEFAULT,
remoteId.getPingInterval());
// set doPing to false
newConf.setBoolean(CommonConfigurationKeys.IPC_CLIENT_PING_KEY, false);
remoteId = ConnectionId.getConnectionId(new InetSocketAddress(0),
- TestRpcService.class, null, 0, null, newConf);
+ TestRpcService.class, null, 0, null, newConf, null);
assertEquals(0, remoteId.getPingInterval());
}
@@ -569,6 +570,72 @@ public void testSimpleServer() throws Exception {
assertAuthEquals(SIMPLE, getAuthMethod(KERBEROS, SIMPLE, UseToken.OTHER));
}
+ /**
+ * In DfsClient there is a fallback mechanism to simple auth, which passes in an atomic boolean
+ * to the ipc Client, which then sets it during setupIOStreams.
+ * SetupIOStreams were running only once per connection, so if two separate DfsClient was
+ * instantiated, then due to the connection caching inside the ipc client, the second DfsClient
+ * did not have the passed in atomic boolean set properly if the first client was not yet closed,
+ * as setupIOStreams was yielding to set up new streams as it has reused the already existing
+ * connection.
+ * This test mimics this behaviour, and asserts the fallback whether it is set correctly.
+ * @see HADOOP-17975
+ */
+ @Test
+ public void testClientFallbackToSimpleAuthForASecondClient() throws Exception {
+ Configuration serverConf = createConfForAuth(SIMPLE);
+ Server server = startServer(serverConf,
+ setupServerUgi(SIMPLE, serverConf),
+ createServerSecretManager(SIMPLE, new TestTokenSecretManager()));
+ final InetSocketAddress serverAddress = NetUtils.getConnectAddress(server);
+
+ clientFallBackToSimpleAllowed = true;
+ Configuration clientConf = createConfForAuth(KERBEROS);
+ UserGroupInformation clientUgi = setupClientUgi(KERBEROS, clientConf);
+
+ AtomicBoolean fallbackToSimpleAuth1 = new AtomicBoolean();
+ AtomicBoolean fallbackToSimpleAuth2 = new AtomicBoolean();
+ try {
+ LOG.info("trying ugi:"+ clientUgi +" tokens:"+ clientUgi.getTokens());
+ clientUgi.doAs((PrivilegedExceptionAction) () -> {
+ TestRpcService proxy1 = null;
+ TestRpcService proxy2 = null;
+ try {
+ proxy1 = getClient(serverAddress, clientConf, null, fallbackToSimpleAuth1);
+ proxy1.ping(null, newEmptyRequest());
+ // make sure the other side thinks we are who we said we are!!!
+ assertEquals(clientUgi.getUserName(),
+ proxy1.getAuthUser(null, newEmptyRequest()).getUser());
+ AuthMethod authMethod =
+ convert(proxy1.getAuthMethod(null, newEmptyRequest()));
+ assertAuthEquals(SIMPLE, authMethod.toString());
+
+ proxy2 = getClient(serverAddress, clientConf, null, fallbackToSimpleAuth2);
+ proxy2.ping(null, newEmptyRequest());
+ // make sure the other side thinks we are who we said we are!!!
+ assertEquals(clientUgi.getUserName(),
+ proxy2.getAuthUser(null, newEmptyRequest()).getUser());
+ AuthMethod authMethod2 =
+ convert(proxy2.getAuthMethod(null, newEmptyRequest()));
+ assertAuthEquals(SIMPLE, authMethod2.toString());
+ } finally {
+ if (proxy1 != null) {
+ RPC.stopProxy(proxy1);
+ }
+ if (proxy2 != null) {
+ RPC.stopProxy(proxy2);
+ }
+ }
+ return null;
+ });
+ } finally {
+ server.stop();
+ }
+
+ assertTrue("First client does not set to fall back properly.", fallbackToSimpleAuth1.get());
+ assertTrue("Second client does not set to fall back properly.", fallbackToSimpleAuth2.get());
+ }
+
@Test
public void testNoClientFallbackToSimple()
throws Exception {
@@ -815,22 +882,43 @@ private String getAuthMethod(
return e.toString();
}
}
-
+
private String internalGetAuthMethod(
final AuthMethod clientAuth,
final AuthMethod serverAuth,
final UseToken tokenType) throws Exception {
-
- final Configuration serverConf = new Configuration(conf);
- serverConf.set(HADOOP_SECURITY_AUTHENTICATION, serverAuth.toString());
- UserGroupInformation.setConfiguration(serverConf);
-
- final UserGroupInformation serverUgi = (serverAuth == KERBEROS)
- ? UserGroupInformation.createRemoteUser("server/localhost@NONE")
- : UserGroupInformation.createRemoteUser("server");
- serverUgi.setAuthenticationMethod(serverAuth);
final TestTokenSecretManager sm = new TestTokenSecretManager();
+
+ Configuration serverConf = createConfForAuth(serverAuth);
+ Server server = startServer(
+ serverConf,
+ setupServerUgi(serverAuth, serverConf),
+ createServerSecretManager(serverAuth, sm));
+ final InetSocketAddress serverAddress = NetUtils.getConnectAddress(server);
+
+ final Configuration clientConf = createConfForAuth(clientAuth);
+ final UserGroupInformation clientUgi = setupClientUgi(clientAuth, clientConf);
+
+ setupTokenIfNeeded(tokenType, sm, clientUgi, serverAddress);
+
+ try {
+ return createClientAndQueryAuthMethod(serverAddress, clientConf, clientUgi, null);
+ } finally {
+ server.stop();
+ }
+ }
+
+ private Configuration createConfForAuth(AuthMethod clientAuth) {
+ final Configuration clientConf = new Configuration(conf);
+ clientConf.set(HADOOP_SECURITY_AUTHENTICATION, clientAuth.toString());
+ clientConf.setBoolean(
+ CommonConfigurationKeys.IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_KEY,
+ clientFallBackToSimpleAllowed);
+ return clientConf;
+ }
+
+ private SecretManager> createServerSecretManager(AuthMethod serverAuth, TestTokenSecretManager sm) {
boolean useSecretManager = (serverAuth != SIMPLE);
if (enableSecretManager != null) {
useSecretManager &= enableSecretManager;
@@ -839,26 +927,43 @@ private String internalGetAuthMethod(
useSecretManager |= forceSecretManager;
}
final SecretManager> serverSm = useSecretManager ? sm : null;
+ return serverSm;
+ }
+ private Server startServer(Configuration serverConf, UserGroupInformation serverUgi,
+ SecretManager> serverSm) throws IOException, InterruptedException {
Server server = serverUgi.doAs(new PrivilegedExceptionAction() {
@Override
public Server run() throws IOException {
return setupTestServer(serverConf, 5, serverSm);
}
});
+ return server;
+ }
- final Configuration clientConf = new Configuration(conf);
- clientConf.set(HADOOP_SECURITY_AUTHENTICATION, clientAuth.toString());
- clientConf.setBoolean(
- CommonConfigurationKeys.IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_KEY,
- clientFallBackToSimpleAllowed);
+ private UserGroupInformation setupServerUgi(AuthMethod serverAuth,
+ Configuration serverConf) {
+ UserGroupInformation.setConfiguration(serverConf);
+
+ final UserGroupInformation serverUgi = (serverAuth == KERBEROS)
+ ? UserGroupInformation.createRemoteUser("server/localhost@NONE")
+ : UserGroupInformation.createRemoteUser("server");
+ serverUgi.setAuthenticationMethod(serverAuth);
+ return serverUgi;
+ }
+
+ private UserGroupInformation setupClientUgi(AuthMethod clientAuth,
+ Configuration clientConf) {
UserGroupInformation.setConfiguration(clientConf);
-
+
final UserGroupInformation clientUgi =
UserGroupInformation.createRemoteUser("client");
- clientUgi.setAuthenticationMethod(clientAuth);
+ clientUgi.setAuthenticationMethod(clientAuth);
+ return clientUgi;
+ }
- final InetSocketAddress addr = NetUtils.getConnectAddress(server);
+ private void setupTokenIfNeeded(UseToken tokenType, TestTokenSecretManager sm,
+ UserGroupInformation clientUgi, InetSocketAddress addr) {
if (tokenType != UseToken.NONE) {
TestTokenIdentifier tokenId = new TestTokenIdentifier(
new Text(clientUgi.getUserName()));
@@ -881,44 +986,44 @@ public Server run() throws IOException {
}
clientUgi.addToken(token);
}
+ }
- try {
- LOG.info("trying ugi:"+clientUgi+" tokens:"+clientUgi.getTokens());
- return clientUgi.doAs(new PrivilegedExceptionAction() {
- @Override
- public String run() throws IOException {
- TestRpcService proxy = null;
- try {
- proxy = getClient(addr, clientConf);
-
- proxy.ping(null, newEmptyRequest());
- // make sure the other side thinks we are who we said we are!!!
- assertEquals(clientUgi.getUserName(),
- proxy.getAuthUser(null, newEmptyRequest()).getUser());
- AuthMethod authMethod =
- convert(proxy.getAuthMethod(null, newEmptyRequest()));
- // verify sasl completed with correct QOP
- assertEquals((authMethod != SIMPLE) ? expectedQop.saslQop : null,
- RPC.getConnectionIdForProxy(proxy).getSaslQop());
- return authMethod != null ? authMethod.toString() : null;
- } catch (ServiceException se) {
- if (se.getCause() instanceof RemoteException) {
- throw (RemoteException) se.getCause();
- } else if (se.getCause() instanceof IOException) {
- throw (IOException) se.getCause();
- } else {
- throw new RuntimeException(se.getCause());
- }
- } finally {
- if (proxy != null) {
- RPC.stopProxy(proxy);
- }
+ private String createClientAndQueryAuthMethod(InetSocketAddress serverAddress, Configuration clientConf,
+ UserGroupInformation clientUgi, AtomicBoolean fallbackToSimpleAuth)
+ throws IOException, InterruptedException {
+ LOG.info("trying ugi:"+ clientUgi +" tokens:"+ clientUgi.getTokens());
+ return clientUgi.doAs(new PrivilegedExceptionAction() {
+ @Override
+ public String run() throws IOException {
+ TestRpcService proxy = null;
+ try {
+ proxy = getClient(serverAddress, clientConf, null, fallbackToSimpleAuth);
+
+ proxy.ping(null, newEmptyRequest());
+ // make sure the other side thinks we are who we said we are!!!
+ assertEquals(clientUgi.getUserName(),
+ proxy.getAuthUser(null, newEmptyRequest()).getUser());
+ AuthMethod authMethod =
+ convert(proxy.getAuthMethod(null, newEmptyRequest()));
+ // verify sasl completed with correct QOP
+ assertEquals((authMethod != SIMPLE) ? expectedQop.saslQop : null,
+ RPC.getConnectionIdForProxy(proxy).getSaslQop());
+ return authMethod != null ? authMethod.toString() : null;
+ } catch (ServiceException se) {
+ if (se.getCause() instanceof RemoteException) {
+ throw (RemoteException) se.getCause();
+ } else if (se.getCause() instanceof IOException) {
+ throw (IOException) se.getCause();
+ } else {
+ throw new RuntimeException(se.getCause());
+ }
+ } finally {
+ if (proxy != null) {
+ RPC.stopProxy(proxy);
}
}
- });
- } finally {
- server.stop();
- }
+ }
+ });
}
private static void assertAuthEquals(AuthMethod expect,