From 8942feae585e229cad8c50cce4fc730816584895 Mon Sep 17 00:00:00 2001 From: Istvan Fajth Date: Thu, 4 Nov 2021 23:57:44 +0100 Subject: [PATCH] HADOOP-17975 Fallback to simple auth does not work for a secondary DistributedFileSystem instance --- .../java/org/apache/hadoop/ipc/Client.java | 18 +- .../apache/hadoop/ipc/ProtobufRpcEngine.java | 2 +- .../apache/hadoop/ipc/ProtobufRpcEngine2.java | 2 +- .../apache/hadoop/ipc/WritableRpcEngine.java | 2 +- .../java/org/apache/hadoop/ipc/TestIPC.java | 4 +- .../hadoop/ipc/TestIPCServerResponder.java | 2 +- .../org/apache/hadoop/ipc/TestRpcBase.java | 20 +- .../hadoop/ipc/TestRpcServerHandoff.java | 5 +- .../org/apache/hadoop/ipc/TestSaslRPC.java | 215 +++++++++++++----- 9 files changed, 192 insertions(+), 78 deletions(-) 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,