Skip to content

Commit 695ed25

Browse files
authored
HBASE-24179 Backport fix for "Netty SASL implementation does not wait for challenge response" to branch-2.x (#5472) (#5496)
- Backport HBASE-23881 "Netty SASL implementation does not wait for challenge response" - Backport HBASE-24263 "TestDelegationToken is broken" - Fix assertion to check for InvalidToken.class.getName() to ensure bug is fixed Signed-off-by: Duo Zhang <zhangduo@apache.org> Signed-off-by: Wellington Chevreuil <wchevreuil@apache.org>
1 parent 7639345 commit 695ed25

File tree

5 files changed

+37
-8
lines changed

5 files changed

+37
-8
lines changed

hbase-client/src/main/java/org/apache/hadoop/hbase/security/AbstractHBaseSaslRpcClient.java

Lines changed: 8 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -83,12 +83,18 @@ protected AbstractHBaseSaslRpcClient(Configuration conf,
8383
}
8484
}
8585

86+
/**
87+
* Computes the initial response a client sends to a server to begin the SASL challenge/response
88+
* handshake. If the client's SASL mechanism does not have an initial response, an empty token
89+
* will be returned without querying the evaluateChallenge method, as an authentication processing
90+
* must be started by client.
91+
* @return The client's initial response to send the server (which may be empty).
92+
*/
8693
public byte[] getInitialResponse() throws SaslException {
8794
if (saslClient.hasInitialResponse()) {
8895
return saslClient.evaluateChallenge(EMPTY_TOKEN);
89-
} else {
90-
return EMPTY_TOKEN;
9196
}
97+
return EMPTY_TOKEN;
9298
}
9399

94100
public boolean isComplete() {

hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClientHandler.java

Lines changed: 22 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -53,6 +53,8 @@ public class NettyHBaseSaslRpcClientHandler extends SimpleChannelInboundHandler<
5353

5454
private final Configuration conf;
5555

56+
private final SaslClientAuthenticationProvider provider;
57+
5658
// flag to mark if Crypto AES encryption is enable
5759
private boolean needProcessConnectionHeader = false;
5860

@@ -67,6 +69,7 @@ public NettyHBaseSaslRpcClientHandler(Promise<Boolean> saslPromise, UserGroupInf
6769
this.saslPromise = saslPromise;
6870
this.ugi = ugi;
6971
this.conf = conf;
72+
this.provider = provider;
7073
this.saslRpcClient = new NettyHBaseSaslRpcClient(conf, provider, token, serverAddr,
7174
securityInfo, fallbackAllowed, conf.get("hbase.rpc.protection",
7275
SaslUtil.QualityOfProtection.AUTHENTICATION.name().toLowerCase()));
@@ -83,6 +86,10 @@ private void tryComplete(ChannelHandlerContext ctx) {
8386
return;
8487
}
8588

89+
// HBASE-23881 Clearly log when the client thinks that the SASL negotiation is complete.
90+
if (LOG.isTraceEnabled()) {
91+
LOG.trace("SASL negotiation for {} is complete", provider.getSaslAuthMethod().getName());
92+
}
8693
saslRpcClient.setupSaslHandler(ctx.pipeline());
8794
setCryptoAESOption();
8895

@@ -110,10 +117,19 @@ public byte[] run() throws Exception {
110117
return saslRpcClient.getInitialResponse();
111118
}
112119
});
113-
if (initialResponse != null) {
114-
writeResponse(ctx, initialResponse);
115-
}
116-
tryComplete(ctx);
120+
assert initialResponse != null;
121+
writeResponse(ctx, initialResponse);
122+
// HBASE-23881 We do not want to check if the SaslClient thinks the handshake is
123+
// complete as, at this point, we've not heard a back from the server with it's reply
124+
// to our first challenge response. We should wait for at least one reply
125+
// from the server before calling negotiation complete.
126+
//
127+
// Each SASL mechanism has its own handshake. Some mechanisms calculate a single client buffer
128+
// to be sent to the server while others have multiple exchanges to negotiate authentication.
129+
// GSSAPI(Kerberos) and DIGEST-MD5 both are examples of mechanisms which have multiple steps.
130+
// Mechanisms which have multiple steps will not return true on `SaslClient#isComplete()`
131+
// until the handshake has fully completed. Mechanisms which only send a single buffer may
132+
// return true on `isComplete()` after that initial response is calculated.
117133
} catch (Exception e) {
118134
// the exception thrown by handlerAdded will not be passed to the exceptionCaught below
119135
// because netty will remove a handler if handlerAdded throws an exception.
@@ -145,6 +161,8 @@ public byte[] run() throws Exception {
145161
});
146162
if (response != null) {
147163
writeResponse(ctx, response);
164+
} else {
165+
LOG.trace("SASL challenge response was empty, not sending response to server.");
148166
}
149167
tryComplete(ctx);
150168
}

hbase-examples/src/main/java/org/apache/hadoop/hbase/security/provider/example/ShadeSaslClientAuthenticationProvider.java

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -60,6 +60,12 @@ public UserInformation getUserInfo(User user) {
6060
return userInfoPB.build();
6161
}
6262

63+
@Override
64+
public boolean canRetry() {
65+
// A static username/password either works or it doesn't. No kind of relogin/retry necessary.
66+
return false;
67+
}
68+
6369
static class ShadeSaslClientCallbackHandler implements CallbackHandler {
6470
private final String username;
6571
private final char[] password;

hbase-examples/src/main/java/org/apache/hadoop/hbase/security/provider/example/ShadeSaslServerAuthenticationProvider.java

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -139,7 +139,6 @@ public ShadeSaslServerCallbackHandler(AtomicReference<UserGroupInformation> atte
139139

140140
@Override
141141
public void handle(Callback[] callbacks) throws InvalidToken, UnsupportedCallbackException {
142-
LOG.info("SaslServerCallbackHandler called", new Exception());
143142
NameCallback nc = null;
144143
PasswordCallback pc = null;
145144
AuthorizeCallback ac = null;

hbase-examples/src/test/java/org/apache/hadoop/hbase/security/provider/example/TestShadeSaslAuthenticationProvider.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -305,7 +305,7 @@ void validateRootCause(Throwable rootCause) {
305305
rootCause.printStackTrace(new PrintWriter(writer));
306306
String text = writer.toString();
307307
assertTrue("Message did not contain expected text",
308-
text.contains("Connection reset by peer"));
308+
text.contains(InvalidToken.class.getName()));
309309
}
310310
}
311311
}

0 commit comments

Comments
 (0)