From 4000a5a6f4aac30b27db4c411bbd3c03572efb00 Mon Sep 17 00:00:00 2001 From: Istvan Fajth Date: Fri, 22 Oct 2021 15:03:52 +0200 Subject: [PATCH 1/7] HADOOP-17975 Fallback to simple auth does not work for a secondary DistributedFileSystem instance. --- .../java/org/apache/hadoop/ipc/Client.java | 64 ++++++++++++------- 1 file changed, 41 insertions(+), 23 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..b1fe6f9b8f7a6 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 @@ -807,17 +807,18 @@ public Object run() throws IOException, InterruptedException { */ private synchronized void setupIOstreams( AtomicBoolean fallbackToSimpleAuth) { - if (socket != null || shouldCloseConnection.get()) { - return; - } - UserGroupInformation ticket = remoteId.getTicket(); - if (ticket != null) { - final UserGroupInformation realUser = ticket.getRealUser(); - if (realUser != null) { - ticket = realUser; - } - } try { + if (socket != null || shouldCloseConnection.get()) { + setFallBackToSimpleAuth(fallbackToSimpleAuth); + return; + } + UserGroupInformation ticket = remoteId.getTicket(); + if (ticket != null) { + final UserGroupInformation realUser = ticket.getRealUser(); + if (realUser != null) { + ticket = realUser; + } + } connectingThread.set(Thread.currentThread()); if (LOG.isDebugEnabled()) { LOG.debug("Connecting to "+server); @@ -863,20 +864,8 @@ public AuthMethod run() remoteId.saslQop = (String)saslRpcClient.getNegotiatedProperty(Sasl.QOP); LOG.debug("Negotiated QOP is :" + remoteId.saslQop); - if (fallbackToSimpleAuth != null) { - fallbackToSimpleAuth.set(false); - } - } else if (UserGroupInformation.isSecurityEnabled()) { - if (!fallbackAllowed) { - throw new AccessControlException( - "Server asks us to fall back to SIMPLE " + - "auth, but this client is configured to only allow secure " + - "connections."); - } - if (fallbackToSimpleAuth != null) { - fallbackToSimpleAuth.set(true); - } } + setFallBackToSimpleAuth(fallbackToSimpleAuth); } if (doPing) { @@ -909,6 +898,35 @@ public AuthMethod run() connectingThread.set(null); } } + + private void setFallBackToSimpleAuth(AtomicBoolean fallbackToSimpleAuth) + throws IOException { + if (fallbackToSimpleAuth == null) { + LOG.trace("Connection {} skips setting fallbackToSimpleAuth as it is null.", remoteId); + return; + } + if (authMethod == null) { + // setupIOStreams() will set up authMethod first, then call this method again. + return; + } + LOG.trace( + "Setting fallbackToSimpleAuth. AuthMethod is {}. Fallback allowed by configuration: {}. " + + "Security is {}.", + authMethod, fallbackAllowed, + UserGroupInformation.isSecurityEnabled() ? "enabled" : "disabled"); + if (authMethod != AuthMethod.SIMPLE) { + LOG.trace("Disabling fallbackToSimpleAuth target does not require SIMPLE authentication."); + fallbackToSimpleAuth.set(false); + } else if (UserGroupInformation.isSecurityEnabled()) { + if (!fallbackAllowed) { + throw new IOException("Server asks us to fall back to SIMPLE auth, but this client is " + + "configured to only allow secure connections."); + } + LOG.trace("Enable fallbackToSimpleAuth for target, as we are allowed to fall back to " + + "SIMPLE authentication."); + fallbackToSimpleAuth.set(true); + } + } private void closeConnection() { if (socket == null) { From 16e923534691a6528b469cbe5d2c658d8b7bb7d7 Mon Sep 17 00:00:00 2001 From: Istvan Fajth Date: Fri, 22 Oct 2021 16:51:59 +0200 Subject: [PATCH 2/7] Fix accidentally changed exception type. --- .../src/main/java/org/apache/hadoop/ipc/Client.java | 6 +++--- 1 file changed, 3 insertions(+), 3 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 b1fe6f9b8f7a6..e6ffa5cae2b4b 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 @@ -900,7 +900,7 @@ public AuthMethod run() } private void setFallBackToSimpleAuth(AtomicBoolean fallbackToSimpleAuth) - throws IOException { + throws AccessControlException { if (fallbackToSimpleAuth == null) { LOG.trace("Connection {} skips setting fallbackToSimpleAuth as it is null.", remoteId); return; @@ -919,8 +919,8 @@ private void setFallBackToSimpleAuth(AtomicBoolean fallbackToSimpleAuth) fallbackToSimpleAuth.set(false); } else if (UserGroupInformation.isSecurityEnabled()) { if (!fallbackAllowed) { - throw new IOException("Server asks us to fall back to SIMPLE auth, but this client is " - + "configured to only allow secure connections."); + throw new AccessControlException("Server asks us to fall back to SIMPLE auth, but this" + + "client is configured to only allow secure connections."); } LOG.trace("Enable fallbackToSimpleAuth for target, as we are allowed to fall back to " + "SIMPLE authentication."); From 6d9def5b680fffe01e1c7d4268ca01cf5ca3d000 Mon Sep 17 00:00:00 2001 From: Istvan Fajth Date: Thu, 4 Nov 2021 19:08:08 +0100 Subject: [PATCH 3/7] Adjust conditions to have more identical behaviour for the setupIOStreams method as it had before the changes in all other cases than the problem we want to solve here. --- .../java/org/apache/hadoop/ipc/Client.java | 38 ++++++++++--------- 1 file changed, 21 insertions(+), 17 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 e6ffa5cae2b4b..a1e47ec0d53fb 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 @@ -901,33 +901,37 @@ public AuthMethod run() private void setFallBackToSimpleAuth(AtomicBoolean fallbackToSimpleAuth) throws AccessControlException { - if (fallbackToSimpleAuth == null) { - LOG.trace("Connection {} skips setting fallbackToSimpleAuth as it is null.", remoteId); + if (authMethod == null || authProtocol != AuthProtocol.SASL) { + if (authProtocol == AuthProtocol.SASL) { + LOG.trace("Auth method is not set, yield from setting auth fallback."); + } return; } - if (authMethod == null) { - // setupIOStreams() will set up authMethod first, then call this method again. - return; + if (fallbackToSimpleAuth == null) { + LOG.trace("Connection {} will skip to set fallbackToSimpleAuth as it is null.", remoteId); + } else { + LOG.trace("Connection {} sets fallbackToSimpleAuth.", remoteId); } - LOG.trace( - "Setting fallbackToSimpleAuth. AuthMethod is {}. Fallback allowed by configuration: {}. " - + "Security is {}.", - authMethod, fallbackAllowed, - UserGroupInformation.isSecurityEnabled() ? "enabled" : "disabled"); + LOG.trace("AuthMethod is {}. Fallback allowed by configuration: {}. Security is {}.", + authMethod, fallbackAllowed, + UserGroupInformation.isSecurityEnabled() ? "enabled" : "disabled"); if (authMethod != AuthMethod.SIMPLE) { - LOG.trace("Disabling fallbackToSimpleAuth target does not require SIMPLE authentication."); - fallbackToSimpleAuth.set(false); + if (fallbackToSimpleAuth != null) { + LOG.trace("Disabling fallbackToSimpleAuth, target does not use SIMPLE authentication."); + fallbackToSimpleAuth.set(false); + } } else if (UserGroupInformation.isSecurityEnabled()) { if (!fallbackAllowed) { - throw new AccessControlException("Server asks us to fall back to SIMPLE auth, but this" + throw new AccessControlException("Server asks us to fall back to SIMPLE auth, but this " + "client is configured to only allow secure connections."); } - LOG.trace("Enable fallbackToSimpleAuth for target, as we are allowed to fall back to " - + "SIMPLE authentication."); - fallbackToSimpleAuth.set(true); + if (fallbackToSimpleAuth != null) { + LOG.trace("Enabling fallbackToSimpleAuth for target, as we are allowed to fall back."); + fallbackToSimpleAuth.set(true); + } } } - + private void closeConnection() { if (socket == null) { return; From 630418abd600df0436e10e77e716e6881c2973e9 Mon Sep 17 00:00:00 2001 From: Istvan Fajth Date: Thu, 4 Nov 2021 23:57:44 +0100 Subject: [PATCH 4/7] Add a test for the problem. Extract parts of the getAuthMethod function in TestRpcBase to make it easier to add. --- .../org/apache/hadoop/ipc/TestRpcBase.java | 20 +- .../org/apache/hadoop/ipc/TestSaslRPC.java | 211 +++++++++++++----- 2 files changed, 169 insertions(+), 62 deletions(-) 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/TestSaslRPC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java index 72085a19ec711..1e128a05e226f 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; @@ -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()));q + 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, From e3ed18f1c4f9d1ba450d113ff45430468e535d61 Mon Sep 17 00:00:00 2001 From: Istvan Fajth Date: Sat, 6 Nov 2021 02:23:57 +0100 Subject: [PATCH 5/7] Remove accidental character added at end of the line. --- .../src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 1e128a05e226f..0a2aa0900df20 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 @@ -616,7 +616,7 @@ public void testClientFallbackToSimpleAuthForASecondClient() throws Exception { assertEquals(clientUgi.getUserName(), proxy2.getAuthUser(null, newEmptyRequest()).getUser()); AuthMethod authMethod2 = - convert(proxy2.getAuthMethod(null, newEmptyRequest()));q + convert(proxy2.getAuthMethod(null, newEmptyRequest())); assertAuthEquals(SIMPLE, authMethod2.toString()); } finally { if (proxy1 != null) { From 80c9afa93a0842b6c83da53c8147a2ac5374c67c Mon Sep 17 00:00:00 2001 From: Istvan Fajth Date: Mon, 22 Nov 2021 13:51:58 +0100 Subject: [PATCH 6/7] Return from setFallbackToSimpleAuth if the AtomicBoolean is already set to true, as in this case we would set it to true again in the same connection. Also removed some logs all information from the removed messaes can be infered from the rest. --- .../src/main/java/org/apache/hadoop/ipc/Client.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 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 a1e47ec0d53fb..49432aff11789 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 @@ -908,13 +908,14 @@ private void setFallBackToSimpleAuth(AtomicBoolean fallbackToSimpleAuth) return; } if (fallbackToSimpleAuth == null) { + // this should happen only during testing. LOG.trace("Connection {} will skip to set fallbackToSimpleAuth as it is null.", remoteId); } else { - LOG.trace("Connection {} sets fallbackToSimpleAuth.", remoteId); + if (fallbackToSimpleAuth.get()) { + // we already set the value to true, we do not need to examine again. + return; + } } - LOG.trace("AuthMethod is {}. Fallback allowed by configuration: {}. Security is {}.", - authMethod, fallbackAllowed, - UserGroupInformation.isSecurityEnabled() ? "enabled" : "disabled"); if (authMethod != AuthMethod.SIMPLE) { if (fallbackToSimpleAuth != null) { LOG.trace("Disabling fallbackToSimpleAuth, target does not use SIMPLE authentication."); From 94cc49efc24cff8e2f0a5c01a7bce17430c4f727 Mon Sep 17 00:00:00 2001 From: Istvan Fajth Date: Tue, 23 Nov 2021 13:20:36 +0100 Subject: [PATCH 7/7] Fix checkstyle issues in test code changes. --- .../src/test/java/org/apache/hadoop/ipc/TestRpcBase.java | 4 ++-- .../src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java | 7 ++++--- 2 files changed, 6 insertions(+), 5 deletions(-) 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 2c8508231e5b6..e9019e3d24eba 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 @@ -126,13 +126,13 @@ protected static RPC.Server setupTestServer( } protected static TestRpcService getClient(InetSocketAddress serverAddr, Configuration clientConf) - throws ServiceException { + throws ServiceException { return getClient(serverAddr, clientConf, null); } protected static TestRpcService getClient(InetSocketAddress serverAddr, Configuration clientConf, RetryPolicy connectionRetryPolicy) throws ServiceException { - return getClient(serverAddr, clientConf, connectionRetryPolicy, null); + return getClient(serverAddr, clientConf, connectionRetryPolicy, null); } protected static TestRpcService getClient(InetSocketAddress serverAddr, 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 0a2aa0900df20..662faea599648 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 @@ -918,7 +918,8 @@ private Configuration createConfForAuth(AuthMethod clientAuth) { return clientConf; } - private SecretManager createServerSecretManager(AuthMethod serverAuth, TestTokenSecretManager sm) { + private SecretManager createServerSecretManager( + AuthMethod serverAuth, TestTokenSecretManager sm) { boolean useSecretManager = (serverAuth != SIMPLE); if (enableSecretManager != null) { useSecretManager &= enableSecretManager; @@ -988,8 +989,8 @@ private void setupTokenIfNeeded(UseToken tokenType, TestTokenSecretManager sm, } } - private String createClientAndQueryAuthMethod(InetSocketAddress serverAddress, Configuration clientConf, - UserGroupInformation clientUgi, AtomicBoolean fallbackToSimpleAuth) + 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() {