From 893f392676851e5057982ec94785abf6a86c0e9e Mon Sep 17 00:00:00 2001 From: Andor Molnar Date: Thu, 20 Jan 2022 15:31:30 -0500 Subject: [PATCH 1/5] HBASE-26655. Implement new OAuth Bearer SASL plugin Signed-off-by: Josh Elser --- .../oauthbearer/OAuthBearerTokenCallback.java | 122 +++++++++ .../internals/OAuthBearerSaslClient.java | 193 ++++++++++++++ .../OAuthBearerSaslClientProvider.java | 38 +++ ...OAuthBearerSaslAuthenticationProvider.java | 42 ++++ ...earerSaslClientAuthenticationProvider.java | 155 ++++++++++++ .../OAuthBearerSaslProviderSelector.java | 69 +++++ .../security/token/OAuthBearerTokenUtil.java | 75 ++++++ .../OAuthBearerTokenCallbackTest.java | 76 ++++++ .../internals/OAuthBearerSaslClientTest.java | 76 ++++++ ...thBearerSaslClientCallbackHandlerTest.java | 108 ++++++++ .../exceptions/IllegalSaslStateException.java | 35 +++ .../SaslAuthenticationException.java | 47 ++++ .../auth/AuthenticateCallbackHandler.java | 48 ++++ .../oauthbearer/OAuthBearerToken.java | 75 ++++++ .../oauthbearer/OAuthBearerUtils.java | 66 +++++ .../OAuthBearerClientInitialResponse.java | 146 +++++++++++ .../security/oauthbearer/JwtTestUtils.java | 117 +++++++++ .../OAuthBearerClientInitialResponseTest.java | 110 ++++++++ .../hbase/util/ClassLoaderTestHelper.java | 1 - .../jwt/client/example/JwtClientExample.java | 111 ++++++++ .../src/main/resources/META-INF/LICENSE.vm | 2 +- hbase-server/pom.xml | 4 + .../OAuthBearerValidatorCallback.java | 156 ++++++++++++ .../internals/OAuthBearerSaslServer.java | 238 ++++++++++++++++++ .../OAuthBearerSaslServerProvider.java | 38 +++ .../knox/OAuthBearerConfigException.java | 37 +++ .../OAuthBearerIllegalTokenException.java | 65 +++++ .../internals/knox/OAuthBearerSignedJwt.java | 193 ++++++++++++++ ...arerSignedJwtValidatorCallbackHandler.java | 196 +++++++++++++++ .../knox/OAuthBearerValidationResult.java | 135 ++++++++++ ...earerSaslServerAuthenticationProvider.java | 100 ++++++++ .../oauthbearer/OAuthBearerTokenMock.java | 34 +++ .../OAuthBearerValidatorCallbackTest.java | 75 ++++++ .../internals/OAuthBearerSaslServerTest.java | 119 +++++++++ .../knox/OAuthBearerSignedJwtTest.java | 121 +++++++++ ...SignedJwtValidatorCallbackHandlerTest.java | 166 ++++++++++++ pom.xml | 6 + 37 files changed, 3393 insertions(+), 2 deletions(-) create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerTokenCallback.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslClient.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslClientProvider.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslAuthenticationProvider.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslClientAuthenticationProvider.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslProviderSelector.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/OAuthBearerTokenUtil.java create mode 100644 hbase-client/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerTokenCallbackTest.java create mode 100644 hbase-client/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslClientTest.java create mode 100644 hbase-client/src/test/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslClientCallbackHandlerTest.java create mode 100644 hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/IllegalSaslStateException.java create mode 100644 hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/SaslAuthenticationException.java create mode 100644 hbase-common/src/main/java/org/apache/hadoop/hbase/security/auth/AuthenticateCallbackHandler.java create mode 100644 hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerToken.java create mode 100644 hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerUtils.java create mode 100644 hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerClientInitialResponse.java create mode 100644 hbase-common/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/JwtTestUtils.java create mode 100644 hbase-common/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerClientInitialResponseTest.java create mode 100644 hbase-examples/src/main/java/org/apache/hadoop/hbase/jwt/client/example/JwtClientExample.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerValidatorCallback.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslServer.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslServerProvider.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerConfigException.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerIllegalTokenException.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerSignedJwt.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerSignedJwtValidatorCallbackHandler.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerValidationResult.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslServerAuthenticationProvider.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerTokenMock.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerValidatorCallbackTest.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslServerTest.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerSignedJwtTest.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerSignedJwtValidatorCallbackHandlerTest.java diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerTokenCallback.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerTokenCallback.java new file mode 100644 index 000000000000..a9e28efbc2a8 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerTokenCallback.java @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.security.oauthbearer; + +import javax.security.auth.callback.Callback; +import org.apache.commons.lang3.StringUtils; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * A {@code Callback} for use by the {@code SaslClient} and {@code Login} + * implementations when they require an OAuth 2 bearer token. Callback handlers + * should use the {@link #error(String, String, String)} method to communicate + * errors returned by the authorization server as per + * RFC 6749: The OAuth + * 2.0 Authorization Framework. Callback handlers should communicate other + * problems by raising an {@code IOException}. + *

+ * This class was introduced in 3.0.0 and, while it feels stable, it could + * evolve. We will try to evolve the API in a compatible manner, but we reserve + * the right to make breaking changes in minor releases, if necessary. We will + * update the {@code InterfaceStability} annotation and this notice once the API + * is considered stable. + */ +@InterfaceAudience.Private +public class OAuthBearerTokenCallback implements Callback { + private OAuthBearerToken token = null; + private String errorCode = null; + private String errorDescription = null; + private String errorUri = null; + + /** + * Return the (potentially null) token + * + * @return the (potentially null) token + */ + public OAuthBearerToken token() { + return token; + } + + /** + * Return the optional (but always non-empty if not null) error code as per + * RFC 6749: The OAuth + * 2.0 Authorization Framework. + * + * @return the optional (but always non-empty if not null) error code + */ + public String errorCode() { + return errorCode; + } + + /** + * Return the (potentially null) error description as per + * RFC 6749: The OAuth + * 2.0 Authorization Framework. + * + * @return the (potentially null) error description + */ + public String errorDescription() { + return errorDescription; + } + + /** + * Return the (potentially null) error URI as per + * RFC 6749: The OAuth + * 2.0 Authorization Framework. + * + * @return the (potentially null) error URI + */ + public String errorUri() { + return errorUri; + } + + /** + * Set the token. All error-related values are cleared. + * + * @param token + * the optional token to set + */ + public void token(OAuthBearerToken token) { + this.token = token; + this.errorCode = null; + this.errorDescription = null; + this.errorUri = null; + } + + /** + * Set the error values as per + * RFC 6749: The OAuth + * 2.0 Authorization Framework. Any token is cleared. + * + * @param errorCode + * the mandatory error code to set + * @param errorDescription + * the optional error description to set + * @param errorUri + * the optional error URI to set + */ + public void error(String errorCode, String errorDescription, String errorUri) { + if (StringUtils.isEmpty(errorCode)) { + throw new IllegalArgumentException("error code must not be empty"); + } + this.errorCode = errorCode; + this.errorDescription = errorDescription; + this.errorUri = errorUri; + this.token = null; + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslClient.java new file mode 100644 index 000000000000..2fa07b5a5110 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslClient.java @@ -0,0 +1,193 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.security.oauthbearer.internals; + +import static org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerUtils.OAUTHBEARER_MECHANISM; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.Map; +import java.util.Objects; +import javax.security.auth.callback.Callback; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.callback.UnsupportedCallbackException; +import javax.security.sasl.Sasl; +import javax.security.sasl.SaslClient; +import javax.security.sasl.SaslClientFactory; +import javax.security.sasl.SaslException; +import org.apache.hadoop.hbase.exceptions.IllegalSaslStateException; +import org.apache.hadoop.hbase.security.SaslUtil; +import org.apache.hadoop.hbase.security.auth.AuthenticateCallbackHandler; +import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerToken; +import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerTokenCallback; +import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerUtils; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * {@code SaslClient} implementation for SASL/OAUTHBEARER in Kafka. This + * implementation requires an instance of {@code AuthenticateCallbackHandler} + * that can handle an instance of {@link OAuthBearerTokenCallback} and return + * the {@link OAuthBearerToken} generated by the {@code login()} event on the + * {@code LoginContext}. + * + * See RFC 6750 Section 2.1 + * + * This class has been copy-and-pasted from Kafka codebase. + */ +@InterfaceAudience.Public +public class OAuthBearerSaslClient implements SaslClient { + static final byte BYTE_CONTROL_A = (byte) 0x01; + private static final Logger LOG = LoggerFactory.getLogger(OAuthBearerSaslClient.class); + private final CallbackHandler callbackHandler; + + enum State { + SEND_CLIENT_FIRST_MESSAGE, RECEIVE_SERVER_FIRST_MESSAGE, RECEIVE_SERVER_MESSAGE_AFTER_FAILURE, + COMPLETE, FAILED + } + + private State state; + + public OAuthBearerSaslClient(AuthenticateCallbackHandler callbackHandler) { + this.callbackHandler = Objects.requireNonNull(callbackHandler); + setState(State.SEND_CLIENT_FIRST_MESSAGE); + } + + public CallbackHandler callbackHandler() { + return callbackHandler; + } + + @Override + public String getMechanismName() { + return OAUTHBEARER_MECHANISM; + } + + @Override + public boolean hasInitialResponse() { + return true; + } + + @Override + public byte[] evaluateChallenge(byte[] challenge) throws SaslException { + try { + OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback(); + switch (state) { + case SEND_CLIENT_FIRST_MESSAGE: + if (challenge != null && challenge.length != 0) { + throw new SaslException("Expected empty challenge"); + } + callbackHandler().handle(new Callback[] {callback}); + setState(State.RECEIVE_SERVER_FIRST_MESSAGE); + return new OAuthBearerClientInitialResponse(callback.token().value()).toBytes(); + case RECEIVE_SERVER_FIRST_MESSAGE: + if (challenge != null && challenge.length != 0) { + String jsonErrorResponse = new String(challenge, StandardCharsets.UTF_8); + if (LOG.isDebugEnabled()) { + LOG.debug("Sending %%x01 response to server after receiving an error: {}", + jsonErrorResponse); + } + setState(State.RECEIVE_SERVER_MESSAGE_AFTER_FAILURE); + return new byte[] {BYTE_CONTROL_A}; + } + callbackHandler().handle(new Callback[] {callback}); + if (LOG.isDebugEnabled()) { + LOG.debug("Successfully authenticated as {}", callback.token().principalName()); + } + setState(State.COMPLETE); + return null; + default: + throw new IllegalSaslStateException("Unexpected challenge in Sasl client state " + state); + } + } catch (SaslException e) { + setState(State.FAILED); + throw e; + } catch (IOException | UnsupportedCallbackException e) { + setState(State.FAILED); + throw new SaslException(e.getMessage(), e); + } + } + + @Override + public boolean isComplete() { + return state == State.COMPLETE; + } + + @Override + public byte[] unwrap(byte[] incoming, int offset, int len) { + if (!isComplete()) { + throw new IllegalStateException("Authentication exchange has not completed"); + } + return Arrays.copyOfRange(incoming, offset, offset + len); + } + + @Override + public byte[] wrap(byte[] outgoing, int offset, int len) { + if (!isComplete()) { + throw new IllegalStateException("Authentication exchange has not completed"); + } + return Arrays.copyOfRange(outgoing, offset, offset + len); + } + + @Override + public Object getNegotiatedProperty(String propName) { + if (!isComplete()) { + throw new IllegalStateException("Authentication exchange has not completed"); + } + if (Sasl.QOP.equals(propName)) { + return SaslUtil.QualityOfProtection.AUTHENTICATION.getSaslQop(); + } + return null; + } + + @Override + public void dispose() { + } + + private void setState(State state) { + LOG.debug("Setting SASL/{} client state to {}", OAUTHBEARER_MECHANISM, state); + this.state = state; + } + + public static class OAuthBearerSaslClientFactory implements SaslClientFactory { + @Override + public SaslClient createSaslClient(String[] mechanisms, String authorizationId, String protocol, + String serverName, Map props, CallbackHandler callbackHandler) { + String[] mechanismNamesCompatibleWithPolicy = getMechanismNames(props); + for (String mechanism : mechanisms) { + for (String s : mechanismNamesCompatibleWithPolicy) { + if (s.equals(mechanism)) { + if (!(Objects.requireNonNull(callbackHandler) instanceof AuthenticateCallbackHandler)) { + throw new IllegalArgumentException( + String.format("Callback handler must be castable to %s: %s", + AuthenticateCallbackHandler.class.getName(), + callbackHandler.getClass().getName())); + } + return new OAuthBearerSaslClient((AuthenticateCallbackHandler) callbackHandler); + } + } + } + return null; + } + + @Override + public String[] getMechanismNames(Map props) { + return OAuthBearerUtils.mechanismNamesCompatibleWithPolicy(props); + } + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslClientProvider.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslClientProvider.java new file mode 100644 index 000000000000..3ba721779e99 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslClientProvider.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.security.oauthbearer.internals; + +import static org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerUtils.OAUTHBEARER_MECHANISM; +import java.security.Provider; +import java.security.Security; +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Public +public class OAuthBearerSaslClientProvider extends Provider { + private static final long serialVersionUID = 1L; + + protected OAuthBearerSaslClientProvider() { + super("SASL/OAUTHBEARER Client Provider", 1.0, "SASL/OAUTHBEARER Client Provider for HBase"); + put("SaslClientFactory." + OAUTHBEARER_MECHANISM, + OAuthBearerSaslClient.OAuthBearerSaslClientFactory.class.getName()); + } + + public static void initialize() { + Security.addProvider(new OAuthBearerSaslClientProvider()); + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslAuthenticationProvider.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslAuthenticationProvider.java new file mode 100644 index 000000000000..8b4dcfe5c75b --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslAuthenticationProvider.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.security.provider; + +import static org.apache.hadoop.hbase.security.token.OAuthBearerTokenUtil.TOKEN_KIND; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * Base client for client/server implementations for the OAuth Bearer (JWT) token auth'n method. + */ +@InterfaceAudience.Private +public class OAuthBearerSaslAuthenticationProvider extends BuiltInSaslAuthenticationProvider { + + public static final SaslAuthMethod SASL_AUTH_METHOD = new SaslAuthMethod( + "OAUTHBEARER", (byte)83, "OAUTHBEARER", UserGroupInformation.AuthenticationMethod.TOKEN); + + @Override + public SaslAuthMethod getSaslAuthMethod() { + return SASL_AUTH_METHOD; + } + + @Override + public String getTokenKind() { + return TOKEN_KIND; + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslClientAuthenticationProvider.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslClientAuthenticationProvider.java new file mode 100644 index 000000000000..915706c1027b --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslClientAuthenticationProvider.java @@ -0,0 +1,155 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.security.provider; + +import static org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerUtils.OAUTHBEARER_MECHANISM; +import java.io.IOException; +import java.net.InetAddress; +import java.security.AccessController; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.util.Collections; +import java.util.Comparator; +import java.util.Map; +import java.util.NavigableSet; +import java.util.Set; +import java.util.TreeSet; +import javax.security.auth.Subject; +import javax.security.auth.callback.Callback; +import javax.security.auth.callback.UnsupportedCallbackException; +import javax.security.sasl.Sasl; +import javax.security.sasl.SaslClient; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.security.SaslUtil; +import org.apache.hadoop.hbase.security.SecurityInfo; +import org.apache.hadoop.hbase.security.User; +import org.apache.hadoop.hbase.security.auth.AuthenticateCallbackHandler; +import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerToken; +import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerTokenCallback; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.security.token.TokenIdentifier; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos; + +@InterfaceAudience.Private +public class OAuthBearerSaslClientAuthenticationProvider + extends OAuthBearerSaslAuthenticationProvider + implements SaslClientAuthenticationProvider { + + @Override + public SaslClient createClient(Configuration conf, InetAddress serverAddr, + SecurityInfo securityInfo, Token token, + boolean fallbackAllowed, + Map saslProps) throws IOException { + AuthenticateCallbackHandler callbackHandler = new OAuthBearerSaslClientCallbackHandler(); + callbackHandler.configure(conf, getSaslAuthMethod().getSaslMechanism(), saslProps); + return Sasl.createSaslClient(new String[] { getSaslAuthMethod().getSaslMechanism() }, null, + null, SaslUtil.SASL_DEFAULT_REALM, saslProps, callbackHandler); + } + + public static class OAuthBearerSaslClientCallbackHandler implements AuthenticateCallbackHandler { + private static final Logger LOG = + LoggerFactory.getLogger(OAuthBearerSaslClientCallbackHandler.class); + private boolean configured = false; + + @Override public void configure(Configuration configs, String saslMechanism, + Map saslProps) { + if (!OAUTHBEARER_MECHANISM.equals(saslMechanism)) { + throw new IllegalArgumentException( + String.format("Unexpected SASL mechanism: %s", saslMechanism)); + } + this.configured = true; + } + + @Override + public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException { + if (!configured) { + throw new IllegalStateException( + "OAuthBearerSaslClientCallbackHandler handler must be configured first."); + } + + for (Callback callback : callbacks) { + if (callback instanceof OAuthBearerTokenCallback) { + handleCallback((OAuthBearerTokenCallback) callback); + } else { + throw new UnsupportedCallbackException(callback); + } + } + } + + private void handleCallback(OAuthBearerTokenCallback callback) throws IOException { + if (callback.token() != null) { + throw new IllegalArgumentException("Callback had a token already"); + } + Subject subject = Subject.getSubject(AccessController.getContext()); + Set privateCredentials = subject != null + ? subject.getPrivateCredentials(OAuthBearerToken.class) + : Collections.emptySet(); + callback.token(choosePrivateCredential(privateCredentials)); + } + + private OAuthBearerToken choosePrivateCredential(Set privateCredentials) + throws IOException { + if (privateCredentials.size() == 0) { + throw new IOException("No OAuth Bearer tokens in Subject's private credentials"); + } + if (privateCredentials.size() == 1) { + LOG.debug("Found 1 OAuthBearer token"); + return privateCredentials.iterator().next(); + } else { + /* + * There a very small window of time upon token refresh (on the order of milliseconds) + * where both an old and a new token appear on the Subject's private credentials. + * Rather than implement a lock to eliminate this window, we will deal with it by + * checking for the existence of multiple tokens and choosing the one that has the + * longest lifetime. It is also possible that a bug could cause multiple tokens to + * exist (e.g. KAFKA-7902), so dealing with the unlikely possibility that occurs + * during normal operation also allows us to deal more robustly with potential bugs. + */ + NavigableSet sortedByLifetime = + new TreeSet<>( + new Comparator() { + @Override + public int compare(OAuthBearerToken o1, OAuthBearerToken o2) { + return Long.compare(o1.lifetimeMs(), o2.lifetimeMs()); + } + }); + sortedByLifetime.addAll(privateCredentials); + if (LOG.isWarnEnabled()) { + LOG.warn("Found {} OAuth Bearer tokens in Subject's private credentials; " + + "the oldest expires at {}, will use the newest, which expires at {}", + sortedByLifetime.size(), + LocalDateTime.ofInstant(Instant.ofEpochMilli(sortedByLifetime.first().lifetimeMs()), + ZoneId.systemDefault()), + LocalDateTime.ofInstant(Instant.ofEpochMilli(sortedByLifetime.last().lifetimeMs()), + ZoneId.systemDefault())); + } + return sortedByLifetime.last(); + } + } + } + + @Override + public RPCProtos.UserInformation getUserInfo(User user) { + // Don't send user for token auth. Copied from RpcConnection. + return null; + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslProviderSelector.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslProviderSelector.java new file mode 100644 index 000000000000..88c2eed0c953 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslProviderSelector.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.security.provider; + +import static org.apache.hadoop.hbase.security.token.OAuthBearerTokenUtil.TOKEN_KIND; +import java.util.Collection; +import java.util.Optional; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.security.User; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.security.token.TokenIdentifier; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@InterfaceAudience.Private +public class OAuthBearerSaslProviderSelector extends BuiltInProviderSelector { + + private static final Logger LOG = LoggerFactory.getLogger(OAuthBearerSaslProviderSelector.class); + + private final Text OAUTHBEARER_TOKEN_KIND_TEXT = + new Text(TOKEN_KIND); + private OAuthBearerSaslClientAuthenticationProvider oauthbearer; + + @Override public void configure(Configuration conf, + Collection providers) { + super.configure(conf, providers); + + this.oauthbearer = (OAuthBearerSaslClientAuthenticationProvider) providers.stream() + .filter((p) -> p instanceof OAuthBearerSaslClientAuthenticationProvider) + .findFirst() + .orElseThrow(() -> new RuntimeException( + "OAuthBearerSaslClientAuthenticationProvider not loaded")); + } + + @Override + public Pair> selectProvider( + String clusterId, User user) { + Pair> pair = + super.selectProvider(clusterId, user); + + Optional> optional = user.getTokens().stream() + .filter((t) -> OAUTHBEARER_TOKEN_KIND_TEXT.equals(t.getKind())) + .findFirst(); + if (optional.isPresent()) { + LOG.info("OAuthBearer token found in user tokens"); + return new Pair<>(oauthbearer, optional.get()); + } + + return pair; + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/OAuthBearerTokenUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/OAuthBearerTokenUtil.java new file mode 100644 index 000000000000..d9e42d5973b4 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/OAuthBearerTokenUtil.java @@ -0,0 +1,75 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.security.token; + +import java.security.AccessController; +import java.security.PrivilegedAction; +import javax.security.auth.Subject; +import org.apache.hadoop.hbase.security.User; +import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerToken; +import org.apache.hadoop.hbase.security.oauthbearer.internals.OAuthBearerSaslClientProvider; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.security.token.Token; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Utility methods for obtaining OAuthBearer / JWT authentication tokens. + */ +@InterfaceAudience.Public +public final class OAuthBearerTokenUtil { + private static final Logger LOG = LoggerFactory.getLogger(OAuthBearerTokenUtil.class); + public static final String TOKEN_KIND = "JWT_AUTH_TOKEN"; + + static { + OAuthBearerSaslClientProvider.initialize(); // not part of public API + LOG.info("OAuthBearer SASL client provider has been initialized"); + } + + private OAuthBearerTokenUtil() { } + + /** + * Add token to user's subject private credentials and a hint to provider selector + * to correctly select OAuthBearer SASL provider. + */ + public static void addTokenForUser(User user, String encodedToken, long lifetimeMs) { + user.addToken(new Token<>(null, null, new Text(TOKEN_KIND), null)); + user.runAs(new PrivilegedAction() { + @Override public Object run() { + Subject subject = Subject.getSubject(AccessController.getContext()); + OAuthBearerToken jwt = new OAuthBearerToken() { + @Override public String value() { + return encodedToken; + } + + @Override public long lifetimeMs() { + return lifetimeMs; + } + + @Override public String principalName() { + return user.getName(); + } + }; + subject.getPrivateCredentials().add(jwt); + return null; + } + }); + } +} diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerTokenCallbackTest.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerTokenCallbackTest.java new file mode 100644 index 000000000000..06ce6577ce5d --- /dev/null +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerTokenCallbackTest.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.security.oauthbearer; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertSame; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.MiscTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ MiscTests.class, SmallTests.class}) +public class OAuthBearerTokenCallbackTest { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(OAuthBearerTokenCallbackTest.class); + + private static final OAuthBearerToken TOKEN = new OAuthBearerToken() { + @Override + public String value() { + return "value"; + } + + @Override + public String principalName() { + return "principalName"; + } + + @Override + public long lifetimeMs() { + return 0; + } + }; + + @Test + public void testError() { + String errorCode = "errorCode"; + String errorDescription = "errorDescription"; + String errorUri = "errorUri"; + OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback(); + callback.error(errorCode, errorDescription, errorUri); + assertEquals(errorCode, callback.errorCode()); + assertEquals(errorDescription, callback.errorDescription()); + assertEquals(errorUri, callback.errorUri()); + assertNull(callback.token()); + } + + @Test + public void testToken() { + OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback(); + callback.token(TOKEN); + assertSame(TOKEN, callback.token()); + assertNull(callback.errorCode()); + assertNull(callback.errorDescription()); + assertNull(callback.errorUri()); + } +} diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslClientTest.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslClientTest.java new file mode 100644 index 000000000000..0267084d35e7 --- /dev/null +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslClientTest.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.security.oauthbearer.internals; + +import static org.junit.Assert.assertEquals; +import java.nio.charset.StandardCharsets; +import javax.security.auth.callback.Callback; +import javax.security.auth.callback.UnsupportedCallbackException; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.security.auth.AuthenticateCallbackHandler; +import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerToken; +import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerTokenCallback; +import org.apache.hadoop.hbase.testclassification.MiscTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ MiscTests.class, SmallTests.class}) +public class OAuthBearerSaslClientTest { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(OAuthBearerSaslClientTest.class); + + public static class ExtensionsCallbackHandler implements AuthenticateCallbackHandler { + + @Override + public void handle(Callback[] callbacks) throws UnsupportedCallbackException { + for (Callback callback : callbacks) { + if (callback instanceof OAuthBearerTokenCallback) { + ((OAuthBearerTokenCallback) callback).token(new OAuthBearerToken() { + @Override public String value() { + return ""; + } + + @Override public long lifetimeMs() { + return 100; + } + + @Override public String principalName() { + return "principalName"; + } + }); + } else { + throw new UnsupportedCallbackException(callback); + } + } + } + } + + @Test + public void testAttachesExtensionsToFirstClientMessage() throws Exception { + String expectedToken = new String( + new OAuthBearerClientInitialResponse("").toBytes(), StandardCharsets.UTF_8); + OAuthBearerSaslClient client = new OAuthBearerSaslClient(new ExtensionsCallbackHandler()); + String message = new String(client.evaluateChallenge("".getBytes(StandardCharsets.UTF_8)), + StandardCharsets.UTF_8); + assertEquals(expectedToken, message); + } + +} diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslClientCallbackHandlerTest.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslClientCallbackHandlerTest.java new file mode 100644 index 000000000000..caf85db44faa --- /dev/null +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslClientCallbackHandlerTest.java @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.security.provider; + +import static org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerUtils.OAUTHBEARER_MECHANISM; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; +import java.io.IOException; +import java.security.AccessController; +import java.security.PrivilegedActionException; +import java.security.PrivilegedExceptionAction; +import java.util.Collections; +import java.util.Set; +import javax.security.auth.Subject; +import javax.security.auth.callback.Callback; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerToken; +import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerTokenCallback; +import org.apache.hadoop.hbase.testclassification.MiscTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ MiscTests.class, SmallTests.class}) +public class OAuthBearerSaslClientCallbackHandlerTest { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(OAuthBearerSaslClientCallbackHandlerTest.class); + + private static OAuthBearerToken createTokenWithLifetimeMillis(final long lifetimeMillis) { + return new OAuthBearerToken() { + @Override + public String value() { + return null; + } + + @Override + public String principalName() { + return null; + } + + @Override + public long lifetimeMs() { + return lifetimeMillis; + } + }; + } + + @Test + public void testWithZeroTokens() { + OAuthBearerSaslClientAuthenticationProvider.OAuthBearerSaslClientCallbackHandler handler = + createCallbackHandler(); + PrivilegedActionException e = + assertThrows(PrivilegedActionException.class, () -> Subject.doAs(new Subject(), + (PrivilegedExceptionAction) () -> { + OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback(); + handler.handle(new Callback[] {callback}); + return null; + } + )); + assertEquals(IOException.class, e.getCause().getClass()); + } + + @Test + public void testWithPotentiallyMultipleTokens() throws Exception { + OAuthBearerSaslClientAuthenticationProvider.OAuthBearerSaslClientCallbackHandler handler = + createCallbackHandler(); + Subject.doAs(new Subject(), (PrivilegedExceptionAction) () -> { + final int maxTokens = 4; + final Set privateCredentials = Subject.getSubject(AccessController.getContext()) + .getPrivateCredentials(); + privateCredentials.clear(); + for (int num = 1; num <= maxTokens; ++num) { + privateCredentials.add(createTokenWithLifetimeMillis(num)); + privateCredentials.add(createTokenWithLifetimeMillis(-num)); + OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback(); + handler.handle(new Callback[] {callback}); + assertEquals(num, callback.token().lifetimeMs()); + } + return null; + }); + } + + private static OAuthBearerSaslClientAuthenticationProvider.OAuthBearerSaslClientCallbackHandler + createCallbackHandler() { + OAuthBearerSaslClientAuthenticationProvider.OAuthBearerSaslClientCallbackHandler handler = + new OAuthBearerSaslClientAuthenticationProvider.OAuthBearerSaslClientCallbackHandler(); + handler.configure(new Configuration(), OAUTHBEARER_MECHANISM, Collections.emptyMap()); + return handler; + } +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/IllegalSaslStateException.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/IllegalSaslStateException.java new file mode 100644 index 000000000000..ce7d1f7c3ddb --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/IllegalSaslStateException.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.exceptions; + +import org.apache.yetus.audience.InterfaceAudience; + +/** + * This exception indicates unexpected requests prior to SASL authentication. + * This could be due to misconfigured security. + */ +@InterfaceAudience.Public +public class IllegalSaslStateException extends IllegalStateException { + + private static final long serialVersionUID = 1L; + + public IllegalSaslStateException(String message) { + super(message); + } + +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/SaslAuthenticationException.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/SaslAuthenticationException.java new file mode 100644 index 000000000000..3f4866e0f557 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/SaslAuthenticationException.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.exceptions; + +import javax.security.sasl.SaslServer; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * This exception indicates that SASL authentication has failed. The error message + * in the exception indicates the actual cause of failure. + *

+ * SASL authentication failures typically indicate invalid credentials, but + * could also include other failures specific to the SASL mechanism used + * for authentication. + *

+ *

Note:If {@link SaslServer#evaluateResponse(byte[])} throws this exception during + * authentication, the message from the exception will be sent to clients in the SaslAuthenticate + * response. Custom {@link SaslServer} implementations may throw this exception in order to + * provide custom error messages to clients, but should take care not to include any + * security-critical information in the message that should not be leaked to unauthenticated + * clients. + *

+ */ +@InterfaceAudience.Public +public class SaslAuthenticationException extends RuntimeException { + + private static final long serialVersionUID = 1L; + + public SaslAuthenticationException(String message) { + super(message); + } +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/security/auth/AuthenticateCallbackHandler.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/security/auth/AuthenticateCallbackHandler.java new file mode 100644 index 000000000000..1329e9ac67f7 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/security/auth/AuthenticateCallbackHandler.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.security.auth; + +import java.util.Map; +import javax.security.auth.callback.CallbackHandler; +import org.apache.hadoop.conf.Configuration; +import org.apache.yetus.audience.InterfaceAudience; + +/* + * Callback handler for SASL-based authentication + */ +@InterfaceAudience.Private +public interface AuthenticateCallbackHandler extends CallbackHandler { + + /** + * Configures this callback handler for the specified SASL mechanism. + * + * @param configs Key-value pairs containing the parsed configuration options of + * the client or server. Note that these are the HBase configuration options + * and not the JAAS configuration options. JAAS config options may be obtained + * from `jaasConfigEntries` for callbacks which obtain some configs from the + * JAAS configuration. For configs that may be specified as both HBase config + * as well as JAAS config (e.g. sasl.kerberos.service.name), the configuration + * is treated as invalid if conflicting values are provided. + * @param saslMechanism Negotiated SASL mechanism. For clients, this is the SASL + * mechanism configured for the client. For brokers, this is the mechanism + * negotiated with the client and is one of the mechanisms enabled on the broker. + * @param saslProps SASL properties provided by the SASL library. + */ + default void configure( + Configuration configs, String saslMechanism, Map saslProps) {} +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerToken.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerToken.java new file mode 100644 index 000000000000..769bceea6181 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerToken.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.security.oauthbearer; + +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.yetus.audience.InterfaceStability; + +/** + * The b64token value as defined in + * RFC 6750 Section + * 2.1 along with the token's specific scope and lifetime and principal + * name. + *

+ * A network request would be required to re-hydrate an opaque token, and that + * could result in (for example) an {@code IOException}, but retrievers for + * various attributes ({@link #lifetimeMs()}, etc.) declare no + * exceptions. Therefore, if a network request is required for any of these + * retriever methods, that request could be performed at construction time so + * that the various attributes can be reliably provided thereafter. For example, + * a constructor might declare {@code throws IOException} in such a case. + * Alternatively, the retrievers could throw unchecked exceptions. + * + * @see RFC 6749 + * Section 1.4 and + * RFC 6750 + * Section 2.1 + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public interface OAuthBearerToken { + /** + * The b64token value as defined in + * RFC 6750 Section + * 2.1 + * + * @return b64token value as defined in + * RFC 6750 + * Section 2.1 + */ + String value(); + + /** + * The token's lifetime, expressed as the number of milliseconds since the + * epoch, as per RFC + * 6749 Section 1.4 + * + * @return the token'slifetime, expressed as the number of milliseconds since + * the epoch, as per + * RFC 6749 + * Section 1.4. + */ + long lifetimeMs(); + + /** + * The name of the principal to which this credential applies + * + * @return the always non-null/non-empty principal name + */ + String principalName(); +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerUtils.java new file mode 100644 index 000000000000..19796e855b08 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerUtils.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.security.oauthbearer; + +import java.util.HashMap; +import java.util.Map; +import javax.security.sasl.Sasl; +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +public final class OAuthBearerUtils { + public static final String OAUTHBEARER_MECHANISM = "OAUTHBEARER"; + + /** + * Verifies configuration for OAuth Bearer authentication mechanism. + * Throws RuntimeException if PlainText is not allowed. + */ + public static String[] mechanismNamesCompatibleWithPolicy(Map props) { + if (props != null && "true".equals(String.valueOf(props.get(Sasl.POLICY_NOPLAINTEXT)))) { + throw new RuntimeException("OAuth Bearer authentication mech cannot be used if plaintext is " + + "disallowed."); + } + return new String[] { OAUTHBEARER_MECHANISM }; + } + + /** + * Converts an extensions string into a {@code Map}. + * + * Example: + * {@code parseMap("key=hey,keyTwo=hi,keyThree=hello", "=", ",") => + * { key: "hey", keyTwo: "hi", keyThree: "hello" }} + * + */ + public static Map parseMap(String mapStr, + String keyValueSeparator, String elementSeparator) { + Map map = new HashMap<>(); + + if (!mapStr.isEmpty()) { + String[] attrvals = mapStr.split(elementSeparator); + for (String attrval : attrvals) { + String[] array = attrval.split(keyValueSeparator, 2); + map.put(array[0], array[1]); + } + } + return map; + } + + private OAuthBearerUtils() { + // empty + } +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerClientInitialResponse.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerClientInitialResponse.java new file mode 100644 index 000000000000..2bfd66a7bcaa --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerClientInitialResponse.java @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.security.oauthbearer.internals; + +import java.nio.charset.StandardCharsets; +import java.util.Map; +import java.util.Objects; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import javax.security.sasl.SaslException; +import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerUtils; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * OAuthBearer SASL client's initial message to the server. + * + * This class has been copy-and-pasted from Kafka codebase. + */ +@InterfaceAudience.Public +public class OAuthBearerClientInitialResponse { + private static final Logger LOG = LoggerFactory.getLogger(OAuthBearerClientInitialResponse.class); + static final String SEPARATOR = "\u0001"; + + private static final String SASLNAME = "(?:[\\x01-\\x7F&&[^=,]]|=2C|=3D)+"; + private static final String KEY = "[A-Za-z]+"; + private static final String VALUE = "[\\x21-\\x7E \t\r\n]+"; + + private static final String KVPAIRS = String.format("(%s=%s%s)*", KEY, VALUE, SEPARATOR); + private static final Pattern AUTH_PATTERN = + Pattern.compile("(?[\\w]+)[ ]+(?[-_\\.a-zA-Z0-9]+)"); + private static final Pattern CLIENT_INITIAL_RESPONSE_PATTERN = Pattern.compile( + String.format("n,(a=(?%s))?,%s(?%s)%s", + SASLNAME, SEPARATOR, KVPAIRS, SEPARATOR)); + public static final String AUTH_KEY = "auth"; + + private final String tokenValue; + private final String authorizationId; + + public OAuthBearerClientInitialResponse(byte[] response) throws SaslException { + LOG.trace("Client initial response parsing started"); + String responseMsg = new String(response, StandardCharsets.UTF_8); + Matcher matcher = CLIENT_INITIAL_RESPONSE_PATTERN.matcher(responseMsg); + if (!matcher.matches()) { + throw new SaslException("Invalid OAUTHBEARER client first message"); + } + LOG.trace("Client initial response matches pattern"); + String authzid = matcher.group("authzid"); + this.authorizationId = authzid == null ? "" : authzid; + String kvPairs = matcher.group("kvpairs"); + Map properties = OAuthBearerUtils.parseMap(kvPairs, "=", SEPARATOR); + String auth = properties.get(AUTH_KEY); + if (auth == null) { + throw new SaslException("Invalid OAUTHBEARER client first message: 'auth' not specified"); + } + LOG.trace("Auth key found in client initial response"); + properties.remove(AUTH_KEY); + Matcher authMatcher = AUTH_PATTERN.matcher(auth); + if (!authMatcher.matches()) { + throw new SaslException("Invalid OAUTHBEARER client first message: invalid 'auth' format"); + } + LOG.trace("Client initial response auth matches pattern"); + if (!"bearer".equalsIgnoreCase(authMatcher.group("scheme"))) { + String msg = String.format("Invalid scheme in OAUTHBEARER client first message: %s", + matcher.group("scheme")); + throw new SaslException(msg); + } + this.tokenValue = authMatcher.group("token"); + LOG.trace("Client initial response parsing finished"); + } + + /** + * Constructor + * + * @param tokenValue + * the mandatory token value + * @throws SaslException + * if any extension name or value fails to conform to the required + * regular expression as defined by the specification, or if the + * reserved {@code auth} appears as a key + */ + public OAuthBearerClientInitialResponse(String tokenValue) { + this(tokenValue, ""); + } + + /** + * Constructor + * + * @param tokenValue + * the mandatory token value + * @param authorizationId + * the optional authorization ID + * @throws SaslException + * if any extension name or value fails to conform to the required + * regular expression as defined by the specification, or if the + * reserved {@code auth} appears as a key + */ + public OAuthBearerClientInitialResponse(String tokenValue, String authorizationId) { + this.tokenValue = Objects.requireNonNull(tokenValue, "token value must not be null"); + this.authorizationId = authorizationId == null ? "" : authorizationId; + } + + public byte[] toBytes() { + String authzid = authorizationId.isEmpty() ? "" : "a=" + authorizationId; + + String message = String.format("n,%s,%sauth=Bearer %s%s%s", authzid, + SEPARATOR, tokenValue, SEPARATOR, SEPARATOR); + + return Bytes.toBytes(message); + } + + /** + * Return the always non-null token value + * + * @return the always non-null toklen value + */ + public String tokenValue() { + return tokenValue; + } + + /** + * Return the always non-null authorization ID + * + * @return the always non-null authorization ID + */ + public String authorizationId() { + return authorizationId; + } +} diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/JwtTestUtils.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/JwtTestUtils.java new file mode 100644 index 000000000000..39b4330425d9 --- /dev/null +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/JwtTestUtils.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.security.oauthbearer; + +import com.nimbusds.jose.JOSEException; +import com.nimbusds.jose.JOSEObjectType; +import com.nimbusds.jose.JWSAlgorithm; +import com.nimbusds.jose.JWSHeader; +import com.nimbusds.jose.crypto.RSASSASigner; +import com.nimbusds.jose.jwk.RSAKey; +import com.nimbusds.jose.jwk.gen.RSAKeyGenerator; +import com.nimbusds.jwt.JWTClaimsSet; +import com.nimbusds.jwt.SignedJWT; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.ZoneId; +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Public +public final class JwtTestUtils { + private final static ZoneId ZONE_ID = ZoneId.of("America/Los_Angeles"); + public static final String USER = "user"; + + public static RSAKey generateRSAKey() throws JOSEException { + RSAKeyGenerator rsaKeyGenerator = new RSAKeyGenerator(2048); + return rsaKeyGenerator.keyID("1").generate(); + } + + public static String createSignedJwt(RSAKey rsaKey, String issuer, String subject, + LocalDate expirationTime, LocalDate issueTime, String audience) + throws JOSEException { + JWSHeader jwsHeader = + new JWSHeader.Builder(JWSAlgorithm.RS256) + .type(JOSEObjectType.JWT) + .keyID(rsaKey.getKeyID()) + .build(); + JWTClaimsSet payload = new JWTClaimsSet.Builder() + .issuer(issuer) + .subject(subject) + .issueTime(java.sql.Date.valueOf(issueTime)) + .expirationTime(java.sql.Date.valueOf(expirationTime)) + .audience(audience) + .build(); + SignedJWT signedJwt = new SignedJWT(jwsHeader, payload); + signedJwt.sign(new RSASSASigner(rsaKey)); + return signedJwt.serialize(); + } + + public static String createSignedJwt(RSAKey rsaKey) throws JOSEException { + LocalDateTime now = LocalDateTime.now(ZONE_ID); + JWSHeader jwsHeader = + new JWSHeader.Builder(JWSAlgorithm.RS256) + .type(JOSEObjectType.JWT) + .keyID(rsaKey.getKeyID()) + .build(); + JWTClaimsSet payload = new JWTClaimsSet.Builder() + .subject(USER) + .expirationTime(java.sql.Timestamp.valueOf(now.plusDays(1))) + .build(); + SignedJWT signedJwt = new SignedJWT(jwsHeader, payload); + signedJwt.sign(new RSASSASigner(rsaKey)); + return signedJwt.serialize(); + } + + public static String createSignedJwtWithAudience(RSAKey rsaKey, String aud) throws JOSEException { + LocalDateTime now = LocalDateTime.now(ZONE_ID); + JWSHeader jwsHeader = + new JWSHeader.Builder(JWSAlgorithm.RS256) + .type(JOSEObjectType.JWT) + .keyID(rsaKey.getKeyID()) + .build(); + JWTClaimsSet payload = new JWTClaimsSet.Builder() + .subject(USER) + .expirationTime(java.sql.Timestamp.valueOf(now.plusDays(1))) + .audience(aud) + .build(); + SignedJWT signedJwt = new SignedJWT(jwsHeader, payload); + signedJwt.sign(new RSASSASigner(rsaKey)); + return signedJwt.serialize(); + } + + public static String createSignedJwtWithIssuer(RSAKey rsaKey, String iss) throws JOSEException { + LocalDateTime now = LocalDateTime.now(ZONE_ID); + JWSHeader jwsHeader = + new JWSHeader.Builder(JWSAlgorithm.RS256) + .type(JOSEObjectType.JWT) + .keyID(rsaKey.getKeyID()) + .build(); + JWTClaimsSet payload = new JWTClaimsSet.Builder() + .subject(USER) + .expirationTime(java.sql.Timestamp.valueOf(now.plusDays(1))) + .issuer(iss) + .build(); + SignedJWT signedJwt = new SignedJWT(jwsHeader, payload); + signedJwt.sign(new RSASSASigner(rsaKey)); + return signedJwt.serialize(); + } + + private JwtTestUtils() { + // empty + } +} diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerClientInitialResponseTest.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerClientInitialResponseTest.java new file mode 100644 index 000000000000..86e7d46ea697 --- /dev/null +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerClientInitialResponseTest.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.security.oauthbearer.internals; + +import static org.junit.Assert.assertEquals; +import java.nio.charset.StandardCharsets; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.MiscTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ MiscTests.class, SmallTests.class}) +public class OAuthBearerClientInitialResponseTest { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(OAuthBearerClientInitialResponseTest.class); + + /* + Test how a client would build a response + */ + @Test + public void testBuildClientResponseToBytes() { + String expectedMesssage = "n,,\u0001auth=Bearer 123.345.567\u0001\u0001"; + + OAuthBearerClientInitialResponse response = + new OAuthBearerClientInitialResponse("123.345.567"); + + String message = new String(response.toBytes(), StandardCharsets.UTF_8); + + assertEquals(expectedMesssage, message); + } + + @Test + public void testBuildServerResponseToBytes() throws Exception { + String serverMessage = "n,,\u0001auth=Bearer 123.345.567\u0001\u0001"; + OAuthBearerClientInitialResponse response = + new OAuthBearerClientInitialResponse(serverMessage.getBytes(StandardCharsets.UTF_8)); + + String message = new String(response.toBytes(), StandardCharsets.UTF_8); + + assertEquals(serverMessage, message); + } + + @Test + public void testToken() throws Exception { + String message = "n,,\u0001auth=Bearer 123.345.567\u0001\u0001"; + OAuthBearerClientInitialResponse response = + new OAuthBearerClientInitialResponse(message.getBytes(StandardCharsets.UTF_8)); + assertEquals("123.345.567", response.tokenValue()); + assertEquals("", response.authorizationId()); + } + + @Test + public void testAuthorizationId() throws Exception { + String message = "n,a=myuser,\u0001auth=Bearer 345\u0001\u0001"; + OAuthBearerClientInitialResponse response = + new OAuthBearerClientInitialResponse(message.getBytes(StandardCharsets.UTF_8)); + assertEquals("345", response.tokenValue()); + assertEquals("myuser", response.authorizationId()); + } + + @Test + public void testExtensions() throws Exception { + String message = + "n,,\u0001propA=valueA1, valueA2\u0001auth=Bearer 567\u0001propB=valueB\u0001\u0001"; + OAuthBearerClientInitialResponse response = + new OAuthBearerClientInitialResponse(message.getBytes(StandardCharsets.UTF_8)); + assertEquals("567", response.tokenValue()); + assertEquals("", response.authorizationId()); + } + + // The example in the RFC uses `vF9dft4qmTc2Nvb3RlckBhbHRhdmlzdGEuY29tCg==` as the token + // But since we use Base64Url encoding, padding is omitted. Hence this test verifies without '='. + @Test + public void testRfc7688Example() throws Exception { + String message = "n,a=user@example.com,\u0001host=server.example.com\u0001port=143\u0001" + + "auth=Bearer vF9dft4qmTc2Nvb3RlckBhbHRhdmlzdGEuY29tCg\u0001\u0001"; + OAuthBearerClientInitialResponse response = + new OAuthBearerClientInitialResponse(message.getBytes(StandardCharsets.UTF_8)); + assertEquals("vF9dft4qmTc2Nvb3RlckBhbHRhdmlzdGEuY29tCg", response.tokenValue()); + assertEquals("user@example.com", response.authorizationId()); + } + + @Test + public void testNoExtensionsFromByteArray() throws Exception { + String message = "n,a=user@example.com,\u0001" + + "auth=Bearer vF9dft4qmTc2Nvb3RlckBhbHRhdmlzdGEuY29tCg\u0001\u0001"; + OAuthBearerClientInitialResponse response = + new OAuthBearerClientInitialResponse(message.getBytes(StandardCharsets.UTF_8)); + assertEquals("vF9dft4qmTc2Nvb3RlckBhbHRhdmlzdGEuY29tCg", response.tokenValue()); + assertEquals("user@example.com", response.authorizationId()); + } +} diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/ClassLoaderTestHelper.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/ClassLoaderTestHelper.java index da11879b9b9d..c09887829ec4 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/ClassLoaderTestHelper.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/ClassLoaderTestHelper.java @@ -19,7 +19,6 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; - import java.io.BufferedWriter; import java.io.File; import java.io.FileInputStream; diff --git a/hbase-examples/src/main/java/org/apache/hadoop/hbase/jwt/client/example/JwtClientExample.java b/hbase-examples/src/main/java/org/apache/hadoop/hbase/jwt/client/example/JwtClientExample.java new file mode 100644 index 000000000000..65eafc8edad6 --- /dev/null +++ b/hbase-examples/src/main/java/org/apache/hadoop/hbase/jwt/client/example/JwtClientExample.java @@ -0,0 +1,111 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.jwt.client.example; + +import java.nio.charset.StandardCharsets; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.conf.Configured; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellBuilderFactory; +import org.apache.hadoop.hbase.CellBuilderType; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.security.User; +import org.apache.hadoop.hbase.security.UserProvider; +import org.apache.hadoop.hbase.security.token.OAuthBearerTokenUtil; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.util.Tool; +import org.apache.hadoop.util.ToolRunner; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An example of using OAuthBearer (JWT) authentication with HBase RPC client. + */ +@InterfaceAudience.Private +public class JwtClientExample extends Configured implements Tool { + private static final Logger LOG = LoggerFactory.getLogger(JwtClientExample.class); + private static final String JWT_TOKEN = ""; + + private static final byte[] FAMILY = Bytes.toBytes("d"); + + public JwtClientExample() { + Configuration conf = HBaseConfiguration.create(); + conf.set("hbase.client.sasl.provider.class", + "org.apache.hadoop.hbase.security.provider.OAuthBearerSaslProviderSelector"); + conf.set("hbase.client.sasl.provider.extras", + "org.apache.hadoop.hbase.security.provider.OAuthBearerSaslClientAuthenticationProvider"); + setConf(conf); + } + + @Override public int run(String[] args) throws Exception { + LOG.info("JWT client example has been started"); + + Configuration conf = getConf(); + LOG.info("Config = " + conf.get("hbase.client.sasl.provider.class")); + UserProvider provider = UserProvider.instantiate(conf); + User user = provider.getCurrent(); + + OAuthBearerTokenUtil.addTokenForUser(user, JWT_TOKEN, 0); + LOG.info("JWT token added"); + + try (final Connection conn = ConnectionFactory.createConnection(conf, user)) { + LOG.info("Connected to HBase"); + Admin admin = conn.getAdmin(); + + TableName tn = TableName.valueOf("jwt-test-table"); + if (!admin.isTableAvailable(tn)) { + TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tn) + .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).build()) + .build(); + admin.createTable(tableDescriptor); + } + + Table table = conn.getTable(tn); + byte[] rk = Bytes.toBytes(ThreadLocalRandom.current().nextLong()); + Put p = new Put(rk); + p.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY) + .setRow(rk) + .setFamily(FAMILY) + .setType(Cell.Type.Put) + .setValue("test".getBytes(StandardCharsets.UTF_8)) + .build()); + table.put(p); + + admin.disableTable(tn); + admin.deleteTable(tn); + } + + LOG.info("JWT client example is done"); + return 0; + } + + public static void main(String[] args) throws Exception { + ToolRunner.run(new JwtClientExample(), args); + } +} diff --git a/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm b/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm index afbf0b8842c3..24ae2f39cd79 100644 --- a/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm +++ b/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm @@ -1381,7 +1381,7 @@ You can redistribute it and/or modify it under either the terms of the ## See this FAQ link for justifications: https://www.apache.org/legal/resolved.html ## ## NB: This list is later compared as lower-case. New entries must also be all lower-case -#set($non_aggregate_fine = [ 'public domain', 'new bsd license', 'bsd license', 'bsd', 'bsd 2-clause license', 'mozilla public license version 1.1', 'mozilla public license version 2.0', 'creative commons attribution license, version 2.5' ]) +#set($non_aggregate_fine = [ 'public domain', 'new bsd license', 'bsd license', 'bsd', 'bsd 2-clause license', 'bsd-3-clause', 'mozilla public license version 1.1', 'mozilla public license version 2.0', 'creative commons attribution license, version 2.5', 'apache-2.0' ]) ## include LICENSE sections for anything not under ASL2.0 #foreach( ${dep} in ${projects} ) ## if there are no licenses we'll fail the build later, so diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml index f00fba295e2a..c2d382bf2000 100644 --- a/hbase-server/pom.xml +++ b/hbase-server/pom.xml @@ -349,6 +349,10 @@ bcpkix-jdk15on test + + com.nimbusds + nimbus-jose-jwt + @@ -3958,6 +3959,11 @@ hadoop-hdfs-client ${hadoop-three.version} + + com.nimbusds + nimbus-jose-jwt + ${nimbusds.version} + From 5bb35dd70d0753344486a93251e61ff86d16672e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andor=20Moln=C3=A1r?= Date: Tue, 25 Jan 2022 13:56:11 +0100 Subject: [PATCH 2/5] HBASE-26665 Added e2e test with HBase mini-cluster (#4051) Signed-off-by: Wellington Chevreuil Signed-off-by: Peter Somogyi --- .../token/TestOAuthBearerAuthentication.java | 158 ++++++++++++++++++ 1 file changed, 158 insertions(+) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestOAuthBearerAuthentication.java diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestOAuthBearerAuthentication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestOAuthBearerAuthentication.java new file mode 100644 index 000000000000..ad799d7a20c9 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestOAuthBearerAuthentication.java @@ -0,0 +1,158 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.security.token; + +import static org.junit.Assert.assertArrayEquals; +import com.nimbusds.jose.JOSEException; +import com.nimbusds.jose.JWSAlgorithm; +import com.nimbusds.jose.JWSHeader; +import com.nimbusds.jose.JWSSigner; +import com.nimbusds.jose.crypto.RSASSASigner; +import com.nimbusds.jose.jwk.JWKSet; +import com.nimbusds.jose.jwk.KeyUse; +import com.nimbusds.jose.jwk.RSAKey; +import com.nimbusds.jose.jwk.gen.RSAKeyGenerator; +import com.nimbusds.jwt.JWTClaimsSet; +import com.nimbusds.jwt.SignedJWT; +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.io.OutputStreamWriter; +import java.time.LocalDate; +import java.util.UUID; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.security.User; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.SecurityTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; + +@Category({ SecurityTests.class, MediumTests.class }) +public class TestOAuthBearerAuthentication extends SecureTestCluster { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestOAuthBearerAuthentication.class); + + private static final String AUDIENCE = "valid-hbase-instance"; + private static final String ISSUER = "authorized-issuer"; + + private static RSAKey RSA; + private static File JWKS_FILE; + + @BeforeClass + public static void setUp() throws Exception { + initRSA(); + + TEST_UTIL.getConfiguration().set("hbase.client.sasl.provider.extras", + "org.apache.hadoop.hbase.security.provider.OAuthBearerSaslClientAuthenticationProvider"); + TEST_UTIL.getConfiguration().set("hbase.server.sasl.provider.extras", + "org.apache.hadoop.hbase.security.provider.OAuthBearerSaslServerAuthenticationProvider"); + TEST_UTIL.getConfiguration().set("hbase.client.sasl.provider.class", + "org.apache.hadoop.hbase.security.provider.OAuthBearerSaslProviderSelector"); + TEST_UTIL.getConfiguration().set("hbase.security.oauth.jwt.jwks.file", + JWKS_FILE.getAbsolutePath()); + TEST_UTIL.getConfiguration().set("hbase.security.oauth.jwt.audience", AUDIENCE); + TEST_UTIL.getConfiguration().set("hbase.security.oauth.jwt.issuer", ISSUER); + + SecureTestCluster.setUp(); + } + + @Rule + public TestName testName = new TestName(); + + private static void initRSA() throws JOSEException, IOException { + RSA = new RSAKeyGenerator(2048) + .keyUse(KeyUse.SIGNATURE) // indicate the intended use of the key + .keyID(UUID.randomUUID().toString()) // give the key a unique ID + .generate(); + JWKSet jwkSet = new JWKSet(RSA.toPublicJWK()); + JWKS_FILE = File.createTempFile("oauth_", ".jwks"); + JWKS_FILE.deleteOnExit(); + + try (OutputStream os = new FileOutputStream(JWKS_FILE); + OutputStreamWriter osw = new OutputStreamWriter(os)) { + osw.write(jwkSet.toString(true)); + } + } + + private String generateBase64EncodedToken(String principal) throws JOSEException { + JWSSigner signer = new RSASSASigner(RSA); + LocalDate now = LocalDate.now(); + + JWTClaimsSet claimsSet = new JWTClaimsSet.Builder() + .subject(principal) + .issuer(ISSUER) + .audience(AUDIENCE) + .expirationTime(java.sql.Date.valueOf(now.plusDays(1))) + .build(); + + SignedJWT signedJWT = new SignedJWT( + new JWSHeader.Builder(JWSAlgorithm.RS256).keyID(RSA.getKeyID()).build(), claimsSet); + + signedJWT.sign(signer); + + return signedJWT.serialize(); + } + + private TableName getTestTableName() { + return TableName.valueOf(testName.getMethodName().replaceAll("[^0-9A-Za-z]", "_")); + } + + @Test + public void testOAuthBearerLogin() throws IOException, JOSEException { + TableName tableName = getTestTableName(); + byte[] family = Bytes.toBytes("f"); + byte[] qualifier = Bytes.toBytes("q"); + byte[] row = Bytes.toBytes("row"); + byte[] value = Bytes.toBytes("data"); + + User user = User.createUserForTesting(TEST_UTIL.getConfiguration(), "testuser_jwt", + new String[] {}); + OAuthBearerTokenUtil.addTokenForUser(user, generateBase64EncodedToken(user.getName()), 0); + + try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration(), user)) { + Admin admin = conn.getAdmin(); + TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName) + .setColumnFamily(ColumnFamilyDescriptorBuilder.of(family)).build(); + admin.createTable(tableDescriptor); + try (Table table = conn.getTable(tableName)) { + table.put(new Put(row).addColumn(family, qualifier, value)); + Result result = table.get(new Get(row)); + assertArrayEquals(value, result.getValue(family, qualifier)); + } + } + } +} From 1532e05c8359e213555b6a0963d53adc70943658 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andor=20Moln=C3=A1r?= Date: Fri, 25 Mar 2022 23:21:04 +0100 Subject: [PATCH 3/5] HBASE-26667 Integrate user-experience for hbase-client (#4064) Signed-off-by: Josh Elser --- .../hbase/client/ConnectionFactory.java | 15 ++- ...OAuthBearerSaslAuthenticationProvider.java | 2 +- .../OAuthBearerSaslProviderSelector.java | 5 +- .../security/token/OAuthBearerTokenUtil.java | 47 ++++++- .../token/TestOAuthBearerTokenUtil.java | 115 ++++++++++++++++++ .../oauthbearer/OAuthBearerUtils.java | 1 + 6 files changed, 179 insertions(+), 6 deletions(-) create mode 100644 hbase-client/src/test/java/org/apache/hadoop/hbase/security/token/TestOAuthBearerTokenUtil.java diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java index 4d4559f4b7a9..efa3b0ca0619 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java @@ -18,7 +18,6 @@ package org.apache.hadoop.hbase.client; import static org.apache.hadoop.hbase.util.FutureUtils.addListener; - import java.io.IOException; import java.lang.reflect.Constructor; import java.security.PrivilegedExceptionAction; @@ -29,6 +28,7 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.UserProvider; +import org.apache.hadoop.hbase.security.token.OAuthBearerTokenUtil; import org.apache.hadoop.hbase.trace.TraceUtil; import org.apache.hadoop.hbase.util.FutureUtils; import org.apache.hadoop.hbase.util.ReflectionUtils; @@ -74,6 +74,9 @@ public class ConnectionFactory { public static final String HBASE_CLIENT_ASYNC_CONNECTION_IMPL = "hbase.client.async.connection.impl"; + /** Environment variable for OAuth Bearer token */ + public static final String ENV_OAUTHBEARER_TOKEN = "HBASE_JWT"; + /** No public c.tors */ protected ConnectionFactory() { } @@ -216,6 +219,11 @@ public static Connection createConnection(Configuration conf, User user) throws */ public static Connection createConnection(Configuration conf, ExecutorService pool, final User user) throws IOException { + + if (System.getenv().containsKey(ENV_OAUTHBEARER_TOKEN)) { + OAuthBearerTokenUtil.addTokenFromEnvironmentVar(user, System.getenv(ENV_OAUTHBEARER_TOKEN)); + } + Class clazz = conf.getClass(ConnectionUtils.HBASE_CLIENT_CONNECTION_IMPL, ConnectionOverAsyncConnection.class, Connection.class); if (clazz != ConnectionOverAsyncConnection.class) { @@ -295,6 +303,11 @@ public static CompletableFuture createAsyncConnection(Configura future.completeExceptionally(new IOException("clusterid came back null")); return; } + + if (System.getenv().containsKey(ENV_OAUTHBEARER_TOKEN)) { + OAuthBearerTokenUtil.addTokenFromEnvironmentVar(user, System.getenv(ENV_OAUTHBEARER_TOKEN)); + } + Class clazz = conf.getClass(HBASE_CLIENT_ASYNC_CONNECTION_IMPL, AsyncConnectionImpl.class, AsyncConnection.class); try { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslAuthenticationProvider.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslAuthenticationProvider.java index 8b4dcfe5c75b..315ce9801782 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslAuthenticationProvider.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslAuthenticationProvider.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.hbase.security.provider; -import static org.apache.hadoop.hbase.security.token.OAuthBearerTokenUtil.TOKEN_KIND; +import static org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerUtils.TOKEN_KIND; import org.apache.hadoop.security.UserGroupInformation; import org.apache.yetus.audience.InterfaceAudience; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslProviderSelector.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslProviderSelector.java index 88c2eed0c953..bfd7d8ad67d2 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslProviderSelector.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslProviderSelector.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.hbase.security.provider; -import static org.apache.hadoop.hbase.security.token.OAuthBearerTokenUtil.TOKEN_KIND; +import static org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerUtils.TOKEN_KIND; import java.util.Collection; import java.util.Optional; import org.apache.hadoop.conf.Configuration; @@ -35,8 +35,7 @@ public class OAuthBearerSaslProviderSelector extends BuiltInProviderSelector { private static final Logger LOG = LoggerFactory.getLogger(OAuthBearerSaslProviderSelector.class); - private final Text OAUTHBEARER_TOKEN_KIND_TEXT = - new Text(TOKEN_KIND); + private final Text OAUTHBEARER_TOKEN_KIND_TEXT = new Text(TOKEN_KIND); private OAuthBearerSaslClientAuthenticationProvider oauthbearer; @Override public void configure(Configuration conf, diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/OAuthBearerTokenUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/OAuthBearerTokenUtil.java index d9e42d5973b4..60896378d238 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/OAuthBearerTokenUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/OAuthBearerTokenUtil.java @@ -18,11 +18,19 @@ */ package org.apache.hadoop.hbase.security.token; +import static org.apache.hadoop.hbase.client.ConnectionFactory.ENV_OAUTHBEARER_TOKEN; +import static org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerUtils.TOKEN_KIND; import java.security.AccessController; import java.security.PrivilegedAction; +import java.time.Instant; +import java.time.ZonedDateTime; +import java.time.format.DateTimeParseException; +import java.util.Optional; import javax.security.auth.Subject; +import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerToken; +import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerUtils; import org.apache.hadoop.hbase.security.oauthbearer.internals.OAuthBearerSaslClientProvider; import org.apache.hadoop.io.Text; import org.apache.hadoop.security.token.Token; @@ -36,7 +44,6 @@ @InterfaceAudience.Public public final class OAuthBearerTokenUtil { private static final Logger LOG = LoggerFactory.getLogger(OAuthBearerTokenUtil.class); - public static final String TOKEN_KIND = "JWT_AUTH_TOKEN"; static { OAuthBearerSaslClientProvider.initialize(); // not part of public API @@ -68,8 +75,46 @@ public static void addTokenForUser(User user, String encodedToken, long lifetime } }; subject.getPrivateCredentials().add(jwt); + if (LOG.isDebugEnabled()) { + LOG.debug("JWT token has been added to user credentials with expiry {}", + lifetimeMs == 0 ? "0" : Instant.ofEpochMilli(lifetimeMs).toString()); + } return null; } }); } + + /** + * Check whether an OAuth Beaerer token is provided in environment variable HBASE_JWT. + * Parse and add it to user private credentials, but only if another token is not already present. + */ + public static void addTokenFromEnvironmentVar(User user, String token) { + Optional> oauthBearerToken = user.getTokens().stream() + .filter((t) -> new Text(OAuthBearerUtils.TOKEN_KIND).equals(t.getKind())) + .findFirst(); + + if (oauthBearerToken.isPresent()) { + LOG.warn("Ignoring OAuth Bearer token in " + ENV_OAUTHBEARER_TOKEN + " environment " + + "variable, because another token is already present"); + return; + } + + String[] tokens = token.split(","); + if (StringUtils.isEmpty(tokens[0])) { + return; + } + long lifetimeMs = 0; + if (tokens.length > 1) { + try { + ZonedDateTime lifetime = ZonedDateTime.parse(tokens[1]); + lifetimeMs = lifetime.toInstant().toEpochMilli(); + } catch (DateTimeParseException e) { + throw new RuntimeException("Unable to parse JWT expiry: " + tokens[1], e); + } + } else { + throw new RuntimeException("Expiry information of JWT is missing"); + } + + addTokenForUser(user, tokens[0], lifetimeMs); + } } diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/security/token/TestOAuthBearerTokenUtil.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/security/token/TestOAuthBearerTokenUtil.java new file mode 100644 index 000000000000..f5109d02a1b2 --- /dev/null +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/security/token/TestOAuthBearerTokenUtil.java @@ -0,0 +1,115 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.security.token; + +import static org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerUtils.TOKEN_KIND; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import java.security.AccessController; +import java.security.PrivilegedAction; +import java.time.Instant; +import java.util.Optional; +import java.util.Set; +import javax.security.auth.Subject; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.security.User; +import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerToken; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.security.token.Token; +import org.junit.Test; + +public class TestOAuthBearerTokenUtil { + + @Test + public void testAddTokenFromEnvVar() { + // Arrange + User user = User.createUserForTesting(HBaseConfiguration.create(), "testuser", new String[] {}); + String testToken = "some_base64_encoded_stuff,2022-01-25T16:59:48.614000+00:00"; + + // Act + OAuthBearerTokenUtil.addTokenFromEnvironmentVar(user, testToken); + + // Assert + Optional> oauthBearerToken = user.getTokens().stream() + .filter((t) -> new Text(TOKEN_KIND).equals(t.getKind())) + .findFirst(); + assertTrue("Token cannot be found in user tokens", oauthBearerToken.isPresent()); + user.runAs(new PrivilegedAction() { + @Override public Object run() { + Subject subject = Subject.getSubject(AccessController.getContext()); + Set tokens = subject.getPrivateCredentials(OAuthBearerToken.class); + assertFalse("Token cannot be found in subject's private credentials", tokens.isEmpty()); + OAuthBearerToken jwt = tokens.iterator().next(); + assertEquals("Invalid encoded JWT value", "some_base64_encoded_stuff", jwt.value()); + assertEquals("Invalid JWT expiry", "2022-01-25T16:59:48.614Z", + Instant.ofEpochMilli(jwt.lifetimeMs()).toString()); + return null; + } + }); + } + + @Test(expected = RuntimeException.class) + public void testAddTokenEnvVarWithoutExpiry() { + // Arrange + User user = User.createUserForTesting(new HBaseConfiguration(), "testuser", new String[] {}); + String testToken = "some_base64_encoded_stuff"; + + // Act + OAuthBearerTokenUtil.addTokenFromEnvironmentVar(user, testToken); + + // Assert + } + + @Test(expected = RuntimeException.class) + public void testAddTokenEnvVarWithInvalidExpiry() { + // Arrange + User user = User.createUserForTesting(new HBaseConfiguration(), "testuser", new String[] {}); + String testToken = "some_base64_encoded_stuff,foobarblahblah328742"; + + // Act + OAuthBearerTokenUtil.addTokenFromEnvironmentVar(user, testToken); + + // Assert + } + + @Test + public void testAddTokenEnvVarTokenAlreadyPresent() { + // Arrange + User user = User.createUserForTesting(new HBaseConfiguration(), "testuser", new String[] {}); + user.addToken(new Token<>(null, null, new Text(TOKEN_KIND), null)); + String testToken = "some_base64_encoded_stuff,foobarblahblah328742"; + + // Act + OAuthBearerTokenUtil.addTokenFromEnvironmentVar(user, testToken); + + // Assert + long numberOfTokens = user.getTokens().stream() + .filter((t) -> new Text(TOKEN_KIND).equals(t.getKind())) + .count(); + assertEquals("Invalid number of tokens on User", 1, numberOfTokens); + user.runAs(new PrivilegedAction() { + @Override public Object run() { + Subject subject = Subject.getSubject(AccessController.getContext()); + Set tokens = subject.getPrivateCredentials(OAuthBearerToken.class); + assertTrue("Token should not have been added to subject's credentials", tokens.isEmpty()); + return null; + } + }); + } +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerUtils.java index 19796e855b08..4b798944233b 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerUtils.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerUtils.java @@ -25,6 +25,7 @@ @InterfaceAudience.Private public final class OAuthBearerUtils { public static final String OAUTHBEARER_MECHANISM = "OAUTHBEARER"; + public static final String TOKEN_KIND = "HBASE_JWT_TOKEN"; /** * Verifies configuration for OAuth Bearer authentication mechanism. From 24d0ba0f47b3a666ea8b4d1a8e575f383fbc9a83 Mon Sep 17 00:00:00 2001 From: Andor Molnar Date: Fri, 26 Aug 2022 16:48:16 +0200 Subject: [PATCH 4/5] HBASE-26553. Move JWKS loader to upper level, fix dependencies --- .../token/TestOAuthBearerTokenUtil.java | 10 +++++ .../hbase/util/ClassLoaderTestHelper.java | 1 + hbase-server/pom.xml | 4 ++ ...arerSignedJwtValidatorCallbackHandler.java | 45 +++++-------------- ...earerSaslServerAuthenticationProvider.java | 35 ++++++++++++++- .../internals/OAuthBearerSaslServerTest.java | 4 +- ...SignedJwtValidatorCallbackHandlerTest.java | 4 +- pom.xml | 16 ++++--- 8 files changed, 73 insertions(+), 46 deletions(-) diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/security/token/TestOAuthBearerTokenUtil.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/security/token/TestOAuthBearerTokenUtil.java index f5109d02a1b2..ed804ab99241 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/security/token/TestOAuthBearerTokenUtil.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/security/token/TestOAuthBearerTokenUtil.java @@ -27,15 +27,25 @@ import java.util.Optional; import java.util.Set; import javax.security.auth.Subject; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerToken; +import org.apache.hadoop.hbase.testclassification.SecurityTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.io.Text; import org.apache.hadoop.security.token.Token; +import org.junit.ClassRule; import org.junit.Test; +import org.junit.experimental.categories.Category; +@Category({ SecurityTests.class, SmallTests.class }) public class TestOAuthBearerTokenUtil { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestOAuthBearerTokenUtil.class); + @Test public void testAddTokenFromEnvVar() { // Arrange diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/ClassLoaderTestHelper.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/ClassLoaderTestHelper.java index c09887829ec4..da11879b9b9d 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/ClassLoaderTestHelper.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/ClassLoaderTestHelper.java @@ -19,6 +19,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; + import java.io.BufferedWriter; import java.io.File; import java.io.FileInputStream; diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml index c2d382bf2000..983d33d50e25 100644 --- a/hbase-server/pom.xml +++ b/hbase-server/pom.xml @@ -708,6 +708,10 @@ + + com.nimbusds + nimbus-jose-jwt + org.slf4j @@ -3953,17 +3958,16 @@ hadoop-distcp ${hadoop-three.version} - org.apache.hadoop hadoop-hdfs-client ${hadoop-three.version} - - com.nimbusds - nimbus-jose-jwt - ${nimbusds.version} - + + com.nimbusds + nimbus-jose-jwt + ${nimbusds.version} + From 234675557e7a1d3c6fe1a7a69afb8aceb140c1fd Mon Sep 17 00:00:00 2001 From: Andor Molnar Date: Sun, 28 Aug 2022 07:28:52 +0200 Subject: [PATCH 5/5] HBASE-26553. Spotless fix --- .../hbase/client/ConnectionFactory.java | 4 +- .../oauthbearer/OAuthBearerTokenCallback.java | 59 ++++++-------- .../internals/OAuthBearerSaslClient.java | 36 +++++---- .../OAuthBearerSaslClientProvider.java | 1 + ...OAuthBearerSaslAuthenticationProvider.java | 5 +- ...earerSaslClientAuthenticationProvider.java | 47 +++++------ .../OAuthBearerSaslProviderSelector.java | 18 ++--- .../security/token/OAuthBearerTokenUtil.java | 30 ++++--- .../OAuthBearerTokenCallbackTest.java | 3 +- .../internals/OAuthBearerSaslClientTest.java | 16 ++-- ...thBearerSaslClientCallbackHandlerTest.java | 23 +++--- .../token/TestOAuthBearerTokenUtil.java | 19 ++--- .../exceptions/IllegalSaslStateException.java | 4 +- .../SaslAuthenticationException.java | 19 +++-- .../auth/AuthenticateCallbackHandler.java | 28 +++---- .../oauthbearer/OAuthBearerToken.java | 53 ++++++------- .../oauthbearer/OAuthBearerUtils.java | 24 +++--- .../OAuthBearerClientInitialResponse.java | 40 ++++------ .../security/oauthbearer/JwtTestUtils.java | 59 ++++---------- .../OAuthBearerClientInitialResponseTest.java | 16 ++-- .../jwt/client/example/JwtClientExample.java | 16 ++-- .../OAuthBearerValidatorCallback.java | 79 ++++++++----------- .../internals/OAuthBearerSaslServer.java | 46 +++++------ .../OAuthBearerSaslServerProvider.java | 1 + .../knox/OAuthBearerConfigException.java | 4 +- .../OAuthBearerIllegalTokenException.java | 11 +-- .../internals/knox/OAuthBearerSignedJwt.java | 48 +++++------ ...arerSignedJwtValidatorCallbackHandler.java | 78 ++++++++---------- .../knox/OAuthBearerValidationResult.java | 35 +++----- ...earerSaslServerAuthenticationProvider.java | 38 +++++---- .../oauthbearer/OAuthBearerTokenMock.java | 15 ++-- .../OAuthBearerValidatorCallbackTest.java | 3 +- .../internals/OAuthBearerSaslServerTest.java | 19 ++--- .../knox/OAuthBearerSignedJwtTest.java | 31 +++----- ...SignedJwtValidatorCallbackHandlerTest.java | 35 ++++---- .../token/TestOAuthBearerAuthentication.java | 18 ++--- 36 files changed, 440 insertions(+), 541 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java index efa3b0ca0619..3238ef187a74 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.client; import static org.apache.hadoop.hbase.util.FutureUtils.addListener; + import java.io.IOException; import java.lang.reflect.Constructor; import java.security.PrivilegedExceptionAction; @@ -305,7 +306,8 @@ public static CompletableFuture createAsyncConnection(Configura } if (System.getenv().containsKey(ENV_OAUTHBEARER_TOKEN)) { - OAuthBearerTokenUtil.addTokenFromEnvironmentVar(user, System.getenv(ENV_OAUTHBEARER_TOKEN)); + OAuthBearerTokenUtil.addTokenFromEnvironmentVar(user, + System.getenv(ENV_OAUTHBEARER_TOKEN)); } Class clazz = conf.getClass(HBASE_CLIENT_ASYNC_CONNECTION_IMPL, diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerTokenCallback.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerTokenCallback.java index a9e28efbc2a8..6708bef34898 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerTokenCallback.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerTokenCallback.java @@ -22,19 +22,17 @@ import org.apache.yetus.audience.InterfaceAudience; /** - * A {@code Callback} for use by the {@code SaslClient} and {@code Login} - * implementations when they require an OAuth 2 bearer token. Callback handlers - * should use the {@link #error(String, String, String)} method to communicate - * errors returned by the authorization server as per - * RFC 6749: The OAuth - * 2.0 Authorization Framework. Callback handlers should communicate other - * problems by raising an {@code IOException}. + * A {@code Callback} for use by the {@code SaslClient} and {@code Login} implementations when they + * require an OAuth 2 bearer token. Callback handlers should use the + * {@link #error(String, String, String)} method to communicate errors returned by the authorization + * server as per RFC 6749: The OAuth 2.0 + * Authorization Framework. Callback handlers should communicate other problems by raising an + * {@code IOException}. *

- * This class was introduced in 3.0.0 and, while it feels stable, it could - * evolve. We will try to evolve the API in a compatible manner, but we reserve - * the right to make breaking changes in minor releases, if necessary. We will - * update the {@code InterfaceStability} annotation and this notice once the API - * is considered stable. + * This class was introduced in 3.0.0 and, while it feels stable, it could evolve. We will try to + * evolve the API in a compatible manner, but we reserve the right to make breaking changes in minor + * releases, if necessary. We will update the {@code InterfaceStability} annotation and this notice + * once the API is considered stable. */ @InterfaceAudience.Private public class OAuthBearerTokenCallback implements Callback { @@ -45,7 +43,6 @@ public class OAuthBearerTokenCallback implements Callback { /** * Return the (potentially null) token - * * @return the (potentially null) token */ public OAuthBearerToken token() { @@ -54,9 +51,8 @@ public OAuthBearerToken token() { /** * Return the optional (but always non-empty if not null) error code as per - * RFC 6749: The OAuth - * 2.0 Authorization Framework. - * + * RFC 6749: The OAuth 2.0 Authorization + * Framework. * @return the optional (but always non-empty if not null) error code */ public String errorCode() { @@ -65,9 +61,8 @@ public String errorCode() { /** * Return the (potentially null) error description as per - * RFC 6749: The OAuth - * 2.0 Authorization Framework. - * + * RFC 6749: The OAuth 2.0 Authorization + * Framework. * @return the (potentially null) error description */ public String errorDescription() { @@ -76,9 +71,8 @@ public String errorDescription() { /** * Return the (potentially null) error URI as per - * RFC 6749: The OAuth - * 2.0 Authorization Framework. - * + * RFC 6749: The OAuth 2.0 Authorization + * Framework. * @return the (potentially null) error URI */ public String errorUri() { @@ -86,10 +80,7 @@ public String errorUri() { } /** - * Set the token. All error-related values are cleared. - * - * @param token - * the optional token to set + * Set the token. All error-related values are cleared. n * the optional token to set */ public void token(OAuthBearerToken token) { this.token = token; @@ -99,16 +90,12 @@ public void token(OAuthBearerToken token) { } /** - * Set the error values as per - * RFC 6749: The OAuth - * 2.0 Authorization Framework. Any token is cleared. - * - * @param errorCode - * the mandatory error code to set - * @param errorDescription - * the optional error description to set - * @param errorUri - * the optional error URI to set + * Set the error values as per RFC 6749: + * The OAuth 2.0 Authorization Framework. Any token is cleared. + *

+ * @param errorCode the mandatory error code to set + * @param errorDescription the optional error description to set + * @param errorUri the optional error URI to set */ public void error(String errorCode, String errorDescription, String errorUri) { if (StringUtils.isEmpty(errorCode)) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslClient.java index 2fa07b5a5110..5663e780ac5b 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslClient.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.security.oauthbearer.internals; import static org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerUtils.OAUTHBEARER_MECHANISM; + import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.Arrays; @@ -41,15 +42,14 @@ import org.slf4j.LoggerFactory; /** - * {@code SaslClient} implementation for SASL/OAUTHBEARER in Kafka. This - * implementation requires an instance of {@code AuthenticateCallbackHandler} - * that can handle an instance of {@link OAuthBearerTokenCallback} and return - * the {@link OAuthBearerToken} generated by the {@code login()} event on the - * {@code LoginContext}. - * + * {@code SaslClient} implementation for SASL/OAUTHBEARER in Kafka. This implementation requires an + * instance of {@code AuthenticateCallbackHandler} that can handle an instance of + * {@link OAuthBearerTokenCallback} and return the {@link OAuthBearerToken} generated by the + * {@code login()} event on the {@code LoginContext}. + *

* See RFC 6750 Section 2.1 - * - * This class has been copy-and-pasted from Kafka codebase. + *

+ * This class has been copy-and-pasted from Kafka codebase. */ @InterfaceAudience.Public public class OAuthBearerSaslClient implements SaslClient { @@ -58,8 +58,11 @@ public class OAuthBearerSaslClient implements SaslClient { private final CallbackHandler callbackHandler; enum State { - SEND_CLIENT_FIRST_MESSAGE, RECEIVE_SERVER_FIRST_MESSAGE, RECEIVE_SERVER_MESSAGE_AFTER_FAILURE, - COMPLETE, FAILED + SEND_CLIENT_FIRST_MESSAGE, + RECEIVE_SERVER_FIRST_MESSAGE, + RECEIVE_SERVER_MESSAGE_AFTER_FAILURE, + COMPLETE, + FAILED } private State state; @@ -92,7 +95,7 @@ public byte[] evaluateChallenge(byte[] challenge) throws SaslException { if (challenge != null && challenge.length != 0) { throw new SaslException("Expected empty challenge"); } - callbackHandler().handle(new Callback[] {callback}); + callbackHandler().handle(new Callback[] { callback }); setState(State.RECEIVE_SERVER_FIRST_MESSAGE); return new OAuthBearerClientInitialResponse(callback.token().value()).toBytes(); case RECEIVE_SERVER_FIRST_MESSAGE: @@ -103,9 +106,9 @@ public byte[] evaluateChallenge(byte[] challenge) throws SaslException { jsonErrorResponse); } setState(State.RECEIVE_SERVER_MESSAGE_AFTER_FAILURE); - return new byte[] {BYTE_CONTROL_A}; + return new byte[] { BYTE_CONTROL_A }; } - callbackHandler().handle(new Callback[] {callback}); + callbackHandler().handle(new Callback[] { callback }); if (LOG.isDebugEnabled()) { LOG.debug("Successfully authenticated as {}", callback.token().principalName()); } @@ -173,10 +176,9 @@ public SaslClient createSaslClient(String[] mechanisms, String authorizationId, for (String s : mechanismNamesCompatibleWithPolicy) { if (s.equals(mechanism)) { if (!(Objects.requireNonNull(callbackHandler) instanceof AuthenticateCallbackHandler)) { - throw new IllegalArgumentException( - String.format("Callback handler must be castable to %s: %s", - AuthenticateCallbackHandler.class.getName(), - callbackHandler.getClass().getName())); + throw new IllegalArgumentException(String.format( + "Callback handler must be castable to %s: %s", + AuthenticateCallbackHandler.class.getName(), callbackHandler.getClass().getName())); } return new OAuthBearerSaslClient((AuthenticateCallbackHandler) callbackHandler); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslClientProvider.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslClientProvider.java index 3ba721779e99..161c4c380e78 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslClientProvider.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslClientProvider.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.security.oauthbearer.internals; import static org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerUtils.OAUTHBEARER_MECHANISM; + import java.security.Provider; import java.security.Security; import org.apache.yetus.audience.InterfaceAudience; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslAuthenticationProvider.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslAuthenticationProvider.java index 315ce9801782..c79fc232edbf 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslAuthenticationProvider.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslAuthenticationProvider.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.security.provider; import static org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerUtils.TOKEN_KIND; + import org.apache.hadoop.security.UserGroupInformation; import org.apache.yetus.audience.InterfaceAudience; @@ -27,8 +28,8 @@ @InterfaceAudience.Private public class OAuthBearerSaslAuthenticationProvider extends BuiltInSaslAuthenticationProvider { - public static final SaslAuthMethod SASL_AUTH_METHOD = new SaslAuthMethod( - "OAUTHBEARER", (byte)83, "OAUTHBEARER", UserGroupInformation.AuthenticationMethod.TOKEN); + public static final SaslAuthMethod SASL_AUTH_METHOD = new SaslAuthMethod("OAUTHBEARER", (byte) 83, + "OAUTHBEARER", UserGroupInformation.AuthenticationMethod.TOKEN); @Override public SaslAuthMethod getSaslAuthMethod() { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslClientAuthenticationProvider.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslClientAuthenticationProvider.java index 915706c1027b..b2d5be5c04e1 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslClientAuthenticationProvider.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslClientAuthenticationProvider.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.security.provider; import static org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerUtils.OAUTHBEARER_MECHANISM; + import java.io.IOException; import java.net.InetAddress; import java.security.AccessController; @@ -47,22 +48,21 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos; @InterfaceAudience.Private public class OAuthBearerSaslClientAuthenticationProvider - extends OAuthBearerSaslAuthenticationProvider - implements SaslClientAuthenticationProvider { + extends OAuthBearerSaslAuthenticationProvider implements SaslClientAuthenticationProvider { @Override public SaslClient createClient(Configuration conf, InetAddress serverAddr, - SecurityInfo securityInfo, Token token, - boolean fallbackAllowed, - Map saslProps) throws IOException { + SecurityInfo securityInfo, Token token, boolean fallbackAllowed, + Map saslProps) throws IOException { AuthenticateCallbackHandler callbackHandler = new OAuthBearerSaslClientCallbackHandler(); callbackHandler.configure(conf, getSaslAuthMethod().getSaslMechanism(), saslProps); return Sasl.createSaslClient(new String[] { getSaslAuthMethod().getSaslMechanism() }, null, - null, SaslUtil.SASL_DEFAULT_REALM, saslProps, callbackHandler); + null, SaslUtil.SASL_DEFAULT_REALM, saslProps, callbackHandler); } public static class OAuthBearerSaslClientCallbackHandler implements AuthenticateCallbackHandler { @@ -70,7 +70,8 @@ public static class OAuthBearerSaslClientCallbackHandler implements Authenticate LoggerFactory.getLogger(OAuthBearerSaslClientCallbackHandler.class); private boolean configured = false; - @Override public void configure(Configuration configs, String saslMechanism, + @Override + public void configure(Configuration configs, String saslMechanism, Map saslProps) { if (!OAUTHBEARER_MECHANISM.equals(saslMechanism)) { throw new IllegalArgumentException( @@ -116,26 +117,26 @@ private OAuthBearerToken choosePrivateCredential(Set privateCr return privateCredentials.iterator().next(); } else { /* - * There a very small window of time upon token refresh (on the order of milliseconds) - * where both an old and a new token appear on the Subject's private credentials. - * Rather than implement a lock to eliminate this window, we will deal with it by - * checking for the existence of multiple tokens and choosing the one that has the - * longest lifetime. It is also possible that a bug could cause multiple tokens to - * exist (e.g. KAFKA-7902), so dealing with the unlikely possibility that occurs - * during normal operation also allows us to deal more robustly with potential bugs. + * There a very small window of time upon token refresh (on the order of milliseconds) where + * both an old and a new token appear on the Subject's private credentials. Rather than + * implement a lock to eliminate this window, we will deal with it by checking for the + * existence of multiple tokens and choosing the one that has the longest lifetime. It is + * also possible that a bug could cause multiple tokens to exist (e.g. KAFKA-7902), so + * dealing with the unlikely possibility that occurs during normal operation also allows us + * to deal more robustly with potential bugs. */ NavigableSet sortedByLifetime = - new TreeSet<>( - new Comparator() { - @Override - public int compare(OAuthBearerToken o1, OAuthBearerToken o2) { - return Long.compare(o1.lifetimeMs(), o2.lifetimeMs()); - } - }); + new TreeSet<>(new Comparator() { + @Override + public int compare(OAuthBearerToken o1, OAuthBearerToken o2) { + return Long.compare(o1.lifetimeMs(), o2.lifetimeMs()); + } + }); sortedByLifetime.addAll(privateCredentials); if (LOG.isWarnEnabled()) { - LOG.warn("Found {} OAuth Bearer tokens in Subject's private credentials; " + - "the oldest expires at {}, will use the newest, which expires at {}", + LOG.warn( + "Found {} OAuth Bearer tokens in Subject's private credentials; " + + "the oldest expires at {}, will use the newest, which expires at {}", sortedByLifetime.size(), LocalDateTime.ofInstant(Instant.ofEpochMilli(sortedByLifetime.first().lifetimeMs()), ZoneId.systemDefault()), diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslProviderSelector.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslProviderSelector.java index bfd7d8ad67d2..93901f81ef3f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslProviderSelector.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslProviderSelector.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.security.provider; import static org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerUtils.TOKEN_KIND; + import java.util.Collection; import java.util.Optional; import org.apache.hadoop.conf.Configuration; @@ -38,26 +39,25 @@ public class OAuthBearerSaslProviderSelector extends BuiltInProviderSelector { private final Text OAUTHBEARER_TOKEN_KIND_TEXT = new Text(TOKEN_KIND); private OAuthBearerSaslClientAuthenticationProvider oauthbearer; - @Override public void configure(Configuration conf, + @Override + public void configure(Configuration conf, Collection providers) { super.configure(conf, providers); this.oauthbearer = (OAuthBearerSaslClientAuthenticationProvider) providers.stream() - .filter((p) -> p instanceof OAuthBearerSaslClientAuthenticationProvider) - .findFirst() - .orElseThrow(() -> new RuntimeException( - "OAuthBearerSaslClientAuthenticationProvider not loaded")); + .filter((p) -> p instanceof OAuthBearerSaslClientAuthenticationProvider).findFirst() + .orElseThrow( + () -> new RuntimeException("OAuthBearerSaslClientAuthenticationProvider not loaded")); } @Override - public Pair> selectProvider( - String clusterId, User user) { + public Pair> + selectProvider(String clusterId, User user) { Pair> pair = super.selectProvider(clusterId, user); Optional> optional = user.getTokens().stream() - .filter((t) -> OAUTHBEARER_TOKEN_KIND_TEXT.equals(t.getKind())) - .findFirst(); + .filter((t) -> OAUTHBEARER_TOKEN_KIND_TEXT.equals(t.getKind())).findFirst(); if (optional.isPresent()) { LOG.info("OAuthBearer token found in user tokens"); return new Pair<>(oauthbearer, optional.get()); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/OAuthBearerTokenUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/OAuthBearerTokenUtil.java index 60896378d238..215470f0f319 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/OAuthBearerTokenUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/OAuthBearerTokenUtil.java @@ -1,5 +1,4 @@ -/** - * +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -20,6 +19,7 @@ import static org.apache.hadoop.hbase.client.ConnectionFactory.ENV_OAUTHBEARER_TOKEN; import static org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerUtils.TOKEN_KIND; + import java.security.AccessController; import java.security.PrivilegedAction; import java.time.Instant; @@ -50,27 +50,32 @@ public final class OAuthBearerTokenUtil { LOG.info("OAuthBearer SASL client provider has been initialized"); } - private OAuthBearerTokenUtil() { } + private OAuthBearerTokenUtil() { + } /** - * Add token to user's subject private credentials and a hint to provider selector - * to correctly select OAuthBearer SASL provider. + * Add token to user's subject private credentials and a hint to provider selector to correctly + * select OAuthBearer SASL provider. */ public static void addTokenForUser(User user, String encodedToken, long lifetimeMs) { user.addToken(new Token<>(null, null, new Text(TOKEN_KIND), null)); user.runAs(new PrivilegedAction() { - @Override public Object run() { + @Override + public Object run() { Subject subject = Subject.getSubject(AccessController.getContext()); OAuthBearerToken jwt = new OAuthBearerToken() { - @Override public String value() { + @Override + public String value() { return encodedToken; } - @Override public long lifetimeMs() { + @Override + public long lifetimeMs() { return lifetimeMs; } - @Override public String principalName() { + @Override + public String principalName() { return user.getName(); } }; @@ -85,13 +90,12 @@ public static void addTokenForUser(User user, String encodedToken, long lifetime } /** - * Check whether an OAuth Beaerer token is provided in environment variable HBASE_JWT. - * Parse and add it to user private credentials, but only if another token is not already present. + * Check whether an OAuth Beaerer token is provided in environment variable HBASE_JWT. Parse and + * add it to user private credentials, but only if another token is not already present. */ public static void addTokenFromEnvironmentVar(User user, String token) { Optional> oauthBearerToken = user.getTokens().stream() - .filter((t) -> new Text(OAuthBearerUtils.TOKEN_KIND).equals(t.getKind())) - .findFirst(); + .filter((t) -> new Text(OAuthBearerUtils.TOKEN_KIND).equals(t.getKind())).findFirst(); if (oauthBearerToken.isPresent()) { LOG.warn("Ignoring OAuth Bearer token in " + ENV_OAUTHBEARER_TOKEN + " environment " diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerTokenCallbackTest.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerTokenCallbackTest.java index 06ce6577ce5d..cad364d5b663 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerTokenCallbackTest.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerTokenCallbackTest.java @@ -20,6 +20,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertSame; + import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; @@ -27,7 +28,7 @@ import org.junit.Test; import org.junit.experimental.categories.Category; -@Category({ MiscTests.class, SmallTests.class}) +@Category({ MiscTests.class, SmallTests.class }) public class OAuthBearerTokenCallbackTest { @ClassRule diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslClientTest.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslClientTest.java index 0267084d35e7..1c4dc4b2b079 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslClientTest.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslClientTest.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.security.oauthbearer.internals; import static org.junit.Assert.assertEquals; + import java.nio.charset.StandardCharsets; import javax.security.auth.callback.Callback; import javax.security.auth.callback.UnsupportedCallbackException; @@ -31,7 +32,7 @@ import org.junit.Test; import org.junit.experimental.categories.Category; -@Category({ MiscTests.class, SmallTests.class}) +@Category({ MiscTests.class, SmallTests.class }) public class OAuthBearerSaslClientTest { @ClassRule public static final HBaseClassTestRule CLASS_RULE = @@ -44,15 +45,18 @@ public void handle(Callback[] callbacks) throws UnsupportedCallbackException { for (Callback callback : callbacks) { if (callback instanceof OAuthBearerTokenCallback) { ((OAuthBearerTokenCallback) callback).token(new OAuthBearerToken() { - @Override public String value() { + @Override + public String value() { return ""; } - @Override public long lifetimeMs() { + @Override + public long lifetimeMs() { return 100; } - @Override public String principalName() { + @Override + public String principalName() { return "principalName"; } }); @@ -65,8 +69,8 @@ public void handle(Callback[] callbacks) throws UnsupportedCallbackException { @Test public void testAttachesExtensionsToFirstClientMessage() throws Exception { - String expectedToken = new String( - new OAuthBearerClientInitialResponse("").toBytes(), StandardCharsets.UTF_8); + String expectedToken = + new String(new OAuthBearerClientInitialResponse("").toBytes(), StandardCharsets.UTF_8); OAuthBearerSaslClient client = new OAuthBearerSaslClient(new ExtensionsCallbackHandler()); String message = new String(client.evaluateChallenge("".getBytes(StandardCharsets.UTF_8)), StandardCharsets.UTF_8); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslClientCallbackHandlerTest.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslClientCallbackHandlerTest.java index caf85db44faa..8c6ec840d69a 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslClientCallbackHandlerTest.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslClientCallbackHandlerTest.java @@ -20,6 +20,7 @@ import static org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerUtils.OAUTHBEARER_MECHANISM; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThrows; + import java.io.IOException; import java.security.AccessController; import java.security.PrivilegedActionException; @@ -38,7 +39,7 @@ import org.junit.Test; import org.junit.experimental.categories.Category; -@Category({ MiscTests.class, SmallTests.class}) +@Category({ MiscTests.class, SmallTests.class }) public class OAuthBearerSaslClientCallbackHandlerTest { @ClassRule public static final HBaseClassTestRule CLASS_RULE = @@ -67,14 +68,12 @@ public long lifetimeMs() { public void testWithZeroTokens() { OAuthBearerSaslClientAuthenticationProvider.OAuthBearerSaslClientCallbackHandler handler = createCallbackHandler(); - PrivilegedActionException e = - assertThrows(PrivilegedActionException.class, () -> Subject.doAs(new Subject(), - (PrivilegedExceptionAction) () -> { - OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback(); - handler.handle(new Callback[] {callback}); - return null; - } - )); + PrivilegedActionException e = assertThrows(PrivilegedActionException.class, + () -> Subject.doAs(new Subject(), (PrivilegedExceptionAction) () -> { + OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback(); + handler.handle(new Callback[] { callback }); + return null; + })); assertEquals(IOException.class, e.getCause().getClass()); } @@ -84,14 +83,14 @@ public void testWithPotentiallyMultipleTokens() throws Exception { createCallbackHandler(); Subject.doAs(new Subject(), (PrivilegedExceptionAction) () -> { final int maxTokens = 4; - final Set privateCredentials = Subject.getSubject(AccessController.getContext()) - .getPrivateCredentials(); + final Set privateCredentials = + Subject.getSubject(AccessController.getContext()).getPrivateCredentials(); privateCredentials.clear(); for (int num = 1; num <= maxTokens; ++num) { privateCredentials.add(createTokenWithLifetimeMillis(num)); privateCredentials.add(createTokenWithLifetimeMillis(-num)); OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback(); - handler.handle(new Callback[] {callback}); + handler.handle(new Callback[] { callback }); assertEquals(num, callback.token().lifetimeMs()); } return null; diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/security/token/TestOAuthBearerTokenUtil.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/security/token/TestOAuthBearerTokenUtil.java index ed804ab99241..53281e9281c3 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/security/token/TestOAuthBearerTokenUtil.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/security/token/TestOAuthBearerTokenUtil.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; + import java.security.AccessController; import java.security.PrivilegedAction; import java.time.Instant; @@ -56,12 +57,12 @@ public void testAddTokenFromEnvVar() { OAuthBearerTokenUtil.addTokenFromEnvironmentVar(user, testToken); // Assert - Optional> oauthBearerToken = user.getTokens().stream() - .filter((t) -> new Text(TOKEN_KIND).equals(t.getKind())) - .findFirst(); + Optional> oauthBearerToken = + user.getTokens().stream().filter((t) -> new Text(TOKEN_KIND).equals(t.getKind())).findFirst(); assertTrue("Token cannot be found in user tokens", oauthBearerToken.isPresent()); user.runAs(new PrivilegedAction() { - @Override public Object run() { + @Override + public Object run() { Subject subject = Subject.getSubject(AccessController.getContext()); Set tokens = subject.getPrivateCredentials(OAuthBearerToken.class); assertFalse("Token cannot be found in subject's private credentials", tokens.isEmpty()); @@ -109,12 +110,12 @@ public void testAddTokenEnvVarTokenAlreadyPresent() { OAuthBearerTokenUtil.addTokenFromEnvironmentVar(user, testToken); // Assert - long numberOfTokens = user.getTokens().stream() - .filter((t) -> new Text(TOKEN_KIND).equals(t.getKind())) - .count(); + long numberOfTokens = + user.getTokens().stream().filter((t) -> new Text(TOKEN_KIND).equals(t.getKind())).count(); assertEquals("Invalid number of tokens on User", 1, numberOfTokens); user.runAs(new PrivilegedAction() { - @Override public Object run() { + @Override + public Object run() { Subject subject = Subject.getSubject(AccessController.getContext()); Set tokens = subject.getPrivateCredentials(OAuthBearerToken.class); assertTrue("Token should not have been added to subject's credentials", tokens.isEmpty()); diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/IllegalSaslStateException.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/IllegalSaslStateException.java index ce7d1f7c3ddb..1c20cdbae36c 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/IllegalSaslStateException.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/IllegalSaslStateException.java @@ -20,8 +20,8 @@ import org.apache.yetus.audience.InterfaceAudience; /** - * This exception indicates unexpected requests prior to SASL authentication. - * This could be due to misconfigured security. + * This exception indicates unexpected requests prior to SASL authentication. This could be due to + * misconfigured security. */ @InterfaceAudience.Public public class IllegalSaslStateException extends IllegalStateException { diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/SaslAuthenticationException.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/SaslAuthenticationException.java index 3f4866e0f557..b6f1f86cf3d4 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/SaslAuthenticationException.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/SaslAuthenticationException.java @@ -21,19 +21,18 @@ import org.apache.yetus.audience.InterfaceAudience; /** - * This exception indicates that SASL authentication has failed. The error message - * in the exception indicates the actual cause of failure. + * This exception indicates that SASL authentication has failed. The error message in the exception + * indicates the actual cause of failure. *

- * SASL authentication failures typically indicate invalid credentials, but - * could also include other failures specific to the SASL mechanism used - * for authentication. + * SASL authentication failures typically indicate invalid credentials, but could also include other + * failures specific to the SASL mechanism used for authentication. *

- *

Note:If {@link SaslServer#evaluateResponse(byte[])} throws this exception during + *

+ * Note:If {@link SaslServer#evaluateResponse(byte[])} throws this exception during * authentication, the message from the exception will be sent to clients in the SaslAuthenticate - * response. Custom {@link SaslServer} implementations may throw this exception in order to - * provide custom error messages to clients, but should take care not to include any - * security-critical information in the message that should not be leaked to unauthenticated - * clients. + * response. Custom {@link SaslServer} implementations may throw this exception in order to provide + * custom error messages to clients, but should take care not to include any security-critical + * information in the message that should not be leaked to unauthenticated clients. *

*/ @InterfaceAudience.Public diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/security/auth/AuthenticateCallbackHandler.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/security/auth/AuthenticateCallbackHandler.java index 1329e9ac67f7..f5e38dd8eef9 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/security/auth/AuthenticateCallbackHandler.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/security/auth/AuthenticateCallbackHandler.java @@ -30,19 +30,19 @@ public interface AuthenticateCallbackHandler extends CallbackHandler { /** * Configures this callback handler for the specified SASL mechanism. - * - * @param configs Key-value pairs containing the parsed configuration options of - * the client or server. Note that these are the HBase configuration options - * and not the JAAS configuration options. JAAS config options may be obtained - * from `jaasConfigEntries` for callbacks which obtain some configs from the - * JAAS configuration. For configs that may be specified as both HBase config - * as well as JAAS config (e.g. sasl.kerberos.service.name), the configuration - * is treated as invalid if conflicting values are provided. - * @param saslMechanism Negotiated SASL mechanism. For clients, this is the SASL - * mechanism configured for the client. For brokers, this is the mechanism - * negotiated with the client and is one of the mechanisms enabled on the broker. - * @param saslProps SASL properties provided by the SASL library. + * @param configs Key-value pairs containing the parsed configuration options of the client + * or server. Note that these are the HBase configuration options and not the + * JAAS configuration options. JAAS config options may be obtained from + * `jaasConfigEntries` for callbacks which obtain some configs from the JAAS + * configuration. For configs that may be specified as both HBase config as + * well as JAAS config (e.g. sasl.kerberos.service.name), the configuration + * is treated as invalid if conflicting values are provided. + * @param saslMechanism Negotiated SASL mechanism. For clients, this is the SASL mechanism + * configured for the client. For brokers, this is the mechanism negotiated + * with the client and is one of the mechanisms enabled on the broker. + * @param saslProps SASL properties provided by the SASL library. */ - default void configure( - Configuration configs, String saslMechanism, Map saslProps) {} + default void configure(Configuration configs, String saslMechanism, + Map saslProps) { + } } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerToken.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerToken.java index 769bceea6181..f5b5f441ae6c 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerToken.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerToken.java @@ -22,53 +22,44 @@ /** * The b64token value as defined in - * RFC 6750 Section - * 2.1 along with the token's specific scope and lifetime and principal - * name. - *

- * A network request would be required to re-hydrate an opaque token, and that - * could result in (for example) an {@code IOException}, but retrievers for - * various attributes ({@link #lifetimeMs()}, etc.) declare no - * exceptions. Therefore, if a network request is required for any of these - * retriever methods, that request could be performed at construction time so - * that the various attributes can be reliably provided thereafter. For example, - * a constructor might declare {@code throws IOException} in such a case. - * Alternatively, the retrievers could throw unchecked exceptions. - * - * @see RFC 6749 - * Section 1.4 and - * RFC 6750 - * Section 2.1 + * RFC 6750 Section 2.1 along with the + * token's specific scope and lifetime and principal name. + *

+ * A network request would be required to re-hydrate an opaque token, and that could result in (for + * example) an {@code IOException}, but retrievers for various attributes ({@link #lifetimeMs()}, + * etc.) declare no exceptions. Therefore, if a network request is required for any of these + * retriever methods, that request could be performed at construction time so that the various + * attributes can be reliably provided thereafter. For example, a constructor might declare + * {@code throws IOException} in such a case. Alternatively, the retrievers could throw unchecked + * exceptions. + *

+ * @see RFC 6749 Section 1.4 and + * RFC 6750 Section 2.1 */ @InterfaceAudience.Public @InterfaceStability.Evolving public interface OAuthBearerToken { /** * The b64token value as defined in - * RFC 6750 Section - * 2.1 - * + * RFC 6750 Section 2.1 + *

* @return b64token value as defined in - * RFC 6750 - * Section 2.1 + * RFC 6750 Section 2.1 */ String value(); /** - * The token's lifetime, expressed as the number of milliseconds since the - * epoch, as per RFC - * 6749 Section 1.4 - * - * @return the token'slifetime, expressed as the number of milliseconds since - * the epoch, as per - * RFC 6749 - * Section 1.4. + * The token's lifetime, expressed as the number of milliseconds since the epoch, as per + * RFC 6749 Section 1.4 + *

+ * @return the token'slifetime, expressed as the number of milliseconds since the epoch, as per + * RFC 6749 Section 1.4. */ long lifetimeMs(); /** * The name of the principal to which this credential applies - * + *

* @return the always non-null/non-empty principal name */ String principalName(); diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerUtils.java index 4b798944233b..9ae42188c920 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerUtils.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerUtils.java @@ -28,27 +28,27 @@ public final class OAuthBearerUtils { public static final String TOKEN_KIND = "HBASE_JWT_TOKEN"; /** - * Verifies configuration for OAuth Bearer authentication mechanism. - * Throws RuntimeException if PlainText is not allowed. + * Verifies configuration for OAuth Bearer authentication mechanism. Throws RuntimeException if + * PlainText is not allowed. */ public static String[] mechanismNamesCompatibleWithPolicy(Map props) { if (props != null && "true".equals(String.valueOf(props.get(Sasl.POLICY_NOPLAINTEXT)))) { - throw new RuntimeException("OAuth Bearer authentication mech cannot be used if plaintext is " - + "disallowed."); + throw new RuntimeException( + "OAuth Bearer authentication mech cannot be used if plaintext is " + "disallowed."); } return new String[] { OAUTHBEARER_MECHANISM }; } /** - * Converts an extensions string into a {@code Map}. - * - * Example: - * {@code parseMap("key=hey,keyTwo=hi,keyThree=hello", "=", ",") => - * { key: "hey", keyTwo: "hi", keyThree: "hello" }} - * + * Converts an extensions string into a {@code Map}. + *

+ * Example: + *

+ * {@code parseMap("key=hey,keyTwo=hi,keyThree=hello", "=", ",") => { key: "hey", keyTwo: "hi", + * keyThree: "hello" }} */ - public static Map parseMap(String mapStr, - String keyValueSeparator, String elementSeparator) { + public static Map parseMap(String mapStr, String keyValueSeparator, + String elementSeparator) { Map map = new HashMap<>(); if (!mapStr.isEmpty()) { diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerClientInitialResponse.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerClientInitialResponse.java index 2bfd66a7bcaa..d950335c8711 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerClientInitialResponse.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerClientInitialResponse.java @@ -31,7 +31,7 @@ /** * OAuthBearer SASL client's initial message to the server. - * + *

* This class has been copy-and-pasted from Kafka codebase. */ @InterfaceAudience.Public @@ -46,9 +46,8 @@ public class OAuthBearerClientInitialResponse { private static final String KVPAIRS = String.format("(%s=%s%s)*", KEY, VALUE, SEPARATOR); private static final Pattern AUTH_PATTERN = Pattern.compile("(?[\\w]+)[ ]+(?[-_\\.a-zA-Z0-9]+)"); - private static final Pattern CLIENT_INITIAL_RESPONSE_PATTERN = Pattern.compile( - String.format("n,(a=(?%s))?,%s(?%s)%s", - SASLNAME, SEPARATOR, KVPAIRS, SEPARATOR)); + private static final Pattern CLIENT_INITIAL_RESPONSE_PATTERN = Pattern.compile(String + .format("n,(a=(?%s))?,%s(?%s)%s", SASLNAME, SEPARATOR, KVPAIRS, SEPARATOR)); public static final String AUTH_KEY = "auth"; private final String tokenValue; @@ -88,13 +87,11 @@ public OAuthBearerClientInitialResponse(byte[] response) throws SaslException { /** * Constructor - * - * @param tokenValue - * the mandatory token value - * @throws SaslException - * if any extension name or value fails to conform to the required - * regular expression as defined by the specification, or if the - * reserved {@code auth} appears as a key + *

+ * @param tokenValue the mandatory token value + * @throws SaslException if any extension name or value fails to conform to the required regular + * expression as defined by the specification, or if the reserved + * {@code auth} appears as a key */ public OAuthBearerClientInitialResponse(String tokenValue) { this(tokenValue, ""); @@ -102,15 +99,12 @@ public OAuthBearerClientInitialResponse(String tokenValue) { /** * Constructor - * - * @param tokenValue - * the mandatory token value - * @param authorizationId - * the optional authorization ID - * @throws SaslException - * if any extension name or value fails to conform to the required - * regular expression as defined by the specification, or if the - * reserved {@code auth} appears as a key + *

+ * @param tokenValue the mandatory token value + * @param authorizationId the optional authorization ID + * @throws SaslException if any extension name or value fails to conform to the required regular + * expression as defined by the specification, or if the reserved + * {@code auth} appears as a key */ public OAuthBearerClientInitialResponse(String tokenValue, String authorizationId) { this.tokenValue = Objects.requireNonNull(tokenValue, "token value must not be null"); @@ -120,15 +114,14 @@ public OAuthBearerClientInitialResponse(String tokenValue, String authorizationI public byte[] toBytes() { String authzid = authorizationId.isEmpty() ? "" : "a=" + authorizationId; - String message = String.format("n,%s,%sauth=Bearer %s%s%s", authzid, - SEPARATOR, tokenValue, SEPARATOR, SEPARATOR); + String message = String.format("n,%s,%sauth=Bearer %s%s%s", authzid, SEPARATOR, tokenValue, + SEPARATOR, SEPARATOR); return Bytes.toBytes(message); } /** * Return the always non-null token value - * * @return the always non-null toklen value */ public String tokenValue() { @@ -137,7 +130,6 @@ public String tokenValue() { /** * Return the always non-null authorization ID - * * @return the always non-null authorization ID */ public String authorizationId() { diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/JwtTestUtils.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/JwtTestUtils.java index 39b4330425d9..db14c4a0ad90 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/JwtTestUtils.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/JwtTestUtils.java @@ -42,20 +42,12 @@ public static RSAKey generateRSAKey() throws JOSEException { } public static String createSignedJwt(RSAKey rsaKey, String issuer, String subject, - LocalDate expirationTime, LocalDate issueTime, String audience) - throws JOSEException { - JWSHeader jwsHeader = - new JWSHeader.Builder(JWSAlgorithm.RS256) - .type(JOSEObjectType.JWT) - .keyID(rsaKey.getKeyID()) - .build(); - JWTClaimsSet payload = new JWTClaimsSet.Builder() - .issuer(issuer) - .subject(subject) + LocalDate expirationTime, LocalDate issueTime, String audience) throws JOSEException { + JWSHeader jwsHeader = new JWSHeader.Builder(JWSAlgorithm.RS256).type(JOSEObjectType.JWT) + .keyID(rsaKey.getKeyID()).build(); + JWTClaimsSet payload = new JWTClaimsSet.Builder().issuer(issuer).subject(subject) .issueTime(java.sql.Date.valueOf(issueTime)) - .expirationTime(java.sql.Date.valueOf(expirationTime)) - .audience(audience) - .build(); + .expirationTime(java.sql.Date.valueOf(expirationTime)).audience(audience).build(); SignedJWT signedJwt = new SignedJWT(jwsHeader, payload); signedJwt.sign(new RSASSASigner(rsaKey)); return signedJwt.serialize(); @@ -63,15 +55,10 @@ public static String createSignedJwt(RSAKey rsaKey, String issuer, String subjec public static String createSignedJwt(RSAKey rsaKey) throws JOSEException { LocalDateTime now = LocalDateTime.now(ZONE_ID); - JWSHeader jwsHeader = - new JWSHeader.Builder(JWSAlgorithm.RS256) - .type(JOSEObjectType.JWT) - .keyID(rsaKey.getKeyID()) - .build(); - JWTClaimsSet payload = new JWTClaimsSet.Builder() - .subject(USER) - .expirationTime(java.sql.Timestamp.valueOf(now.plusDays(1))) - .build(); + JWSHeader jwsHeader = new JWSHeader.Builder(JWSAlgorithm.RS256).type(JOSEObjectType.JWT) + .keyID(rsaKey.getKeyID()).build(); + JWTClaimsSet payload = new JWTClaimsSet.Builder().subject(USER) + .expirationTime(java.sql.Timestamp.valueOf(now.plusDays(1))).build(); SignedJWT signedJwt = new SignedJWT(jwsHeader, payload); signedJwt.sign(new RSASSASigner(rsaKey)); return signedJwt.serialize(); @@ -79,16 +66,10 @@ public static String createSignedJwt(RSAKey rsaKey) throws JOSEException { public static String createSignedJwtWithAudience(RSAKey rsaKey, String aud) throws JOSEException { LocalDateTime now = LocalDateTime.now(ZONE_ID); - JWSHeader jwsHeader = - new JWSHeader.Builder(JWSAlgorithm.RS256) - .type(JOSEObjectType.JWT) - .keyID(rsaKey.getKeyID()) - .build(); - JWTClaimsSet payload = new JWTClaimsSet.Builder() - .subject(USER) - .expirationTime(java.sql.Timestamp.valueOf(now.plusDays(1))) - .audience(aud) - .build(); + JWSHeader jwsHeader = new JWSHeader.Builder(JWSAlgorithm.RS256).type(JOSEObjectType.JWT) + .keyID(rsaKey.getKeyID()).build(); + JWTClaimsSet payload = new JWTClaimsSet.Builder().subject(USER) + .expirationTime(java.sql.Timestamp.valueOf(now.plusDays(1))).audience(aud).build(); SignedJWT signedJwt = new SignedJWT(jwsHeader, payload); signedJwt.sign(new RSASSASigner(rsaKey)); return signedJwt.serialize(); @@ -96,16 +77,10 @@ public static String createSignedJwtWithAudience(RSAKey rsaKey, String aud) thro public static String createSignedJwtWithIssuer(RSAKey rsaKey, String iss) throws JOSEException { LocalDateTime now = LocalDateTime.now(ZONE_ID); - JWSHeader jwsHeader = - new JWSHeader.Builder(JWSAlgorithm.RS256) - .type(JOSEObjectType.JWT) - .keyID(rsaKey.getKeyID()) - .build(); - JWTClaimsSet payload = new JWTClaimsSet.Builder() - .subject(USER) - .expirationTime(java.sql.Timestamp.valueOf(now.plusDays(1))) - .issuer(iss) - .build(); + JWSHeader jwsHeader = new JWSHeader.Builder(JWSAlgorithm.RS256).type(JOSEObjectType.JWT) + .keyID(rsaKey.getKeyID()).build(); + JWTClaimsSet payload = new JWTClaimsSet.Builder().subject(USER) + .expirationTime(java.sql.Timestamp.valueOf(now.plusDays(1))).issuer(iss).build(); SignedJWT signedJwt = new SignedJWT(jwsHeader, payload); signedJwt.sign(new RSASSASigner(rsaKey)); return signedJwt.serialize(); diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerClientInitialResponseTest.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerClientInitialResponseTest.java index 86e7d46ea697..7918ff947caf 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerClientInitialResponseTest.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerClientInitialResponseTest.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.security.oauthbearer.internals; import static org.junit.Assert.assertEquals; + import java.nio.charset.StandardCharsets; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; @@ -26,21 +27,20 @@ import org.junit.Test; import org.junit.experimental.categories.Category; -@Category({ MiscTests.class, SmallTests.class}) +@Category({ MiscTests.class, SmallTests.class }) public class OAuthBearerClientInitialResponseTest { @ClassRule public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass(OAuthBearerClientInitialResponseTest.class); /* - Test how a client would build a response + * Test how a client would build a response */ @Test public void testBuildClientResponseToBytes() { String expectedMesssage = "n,,\u0001auth=Bearer 123.345.567\u0001\u0001"; - OAuthBearerClientInitialResponse response = - new OAuthBearerClientInitialResponse("123.345.567"); + OAuthBearerClientInitialResponse response = new OAuthBearerClientInitialResponse("123.345.567"); String message = new String(response.toBytes(), StandardCharsets.UTF_8); @@ -90,8 +90,8 @@ public void testExtensions() throws Exception { // But since we use Base64Url encoding, padding is omitted. Hence this test verifies without '='. @Test public void testRfc7688Example() throws Exception { - String message = "n,a=user@example.com,\u0001host=server.example.com\u0001port=143\u0001" + - "auth=Bearer vF9dft4qmTc2Nvb3RlckBhbHRhdmlzdGEuY29tCg\u0001\u0001"; + String message = "n,a=user@example.com,\u0001host=server.example.com\u0001port=143\u0001" + + "auth=Bearer vF9dft4qmTc2Nvb3RlckBhbHRhdmlzdGEuY29tCg\u0001\u0001"; OAuthBearerClientInitialResponse response = new OAuthBearerClientInitialResponse(message.getBytes(StandardCharsets.UTF_8)); assertEquals("vF9dft4qmTc2Nvb3RlckBhbHRhdmlzdGEuY29tCg", response.tokenValue()); @@ -100,8 +100,8 @@ public void testRfc7688Example() throws Exception { @Test public void testNoExtensionsFromByteArray() throws Exception { - String message = "n,a=user@example.com,\u0001" + - "auth=Bearer vF9dft4qmTc2Nvb3RlckBhbHRhdmlzdGEuY29tCg\u0001\u0001"; + String message = "n,a=user@example.com,\u0001" + + "auth=Bearer vF9dft4qmTc2Nvb3RlckBhbHRhdmlzdGEuY29tCg\u0001\u0001"; OAuthBearerClientInitialResponse response = new OAuthBearerClientInitialResponse(message.getBytes(StandardCharsets.UTF_8)); assertEquals("vF9dft4qmTc2Nvb3RlckBhbHRhdmlzdGEuY29tCg", response.tokenValue()); diff --git a/hbase-examples/src/main/java/org/apache/hadoop/hbase/jwt/client/example/JwtClientExample.java b/hbase-examples/src/main/java/org/apache/hadoop/hbase/jwt/client/example/JwtClientExample.java index 65eafc8edad6..62c3bb30fc94 100644 --- a/hbase-examples/src/main/java/org/apache/hadoop/hbase/jwt/client/example/JwtClientExample.java +++ b/hbase-examples/src/main/java/org/apache/hadoop/hbase/jwt/client/example/JwtClientExample.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -63,7 +63,8 @@ public JwtClientExample() { setConf(conf); } - @Override public int run(String[] args) throws Exception { + @Override + public int run(String[] args) throws Exception { LOG.info("JWT client example has been started"); Configuration conf = getConf(); @@ -81,20 +82,15 @@ public JwtClientExample() { TableName tn = TableName.valueOf("jwt-test-table"); if (!admin.isTableAvailable(tn)) { TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tn) - .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).build()) - .build(); + .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).build()).build(); admin.createTable(tableDescriptor); } Table table = conn.getTable(tn); byte[] rk = Bytes.toBytes(ThreadLocalRandom.current().nextLong()); Put p = new Put(rk); - p.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY) - .setRow(rk) - .setFamily(FAMILY) - .setType(Cell.Type.Put) - .setValue("test".getBytes(StandardCharsets.UTF_8)) - .build()); + p.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(rk).setFamily(FAMILY) + .setType(Cell.Type.Put).setValue("test".getBytes(StandardCharsets.UTF_8)).build()); table.put(p); admin.disableTable(tn); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerValidatorCallback.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerValidatorCallback.java index ec4b3c2a329e..6be3624b6f18 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerValidatorCallback.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerValidatorCallback.java @@ -23,22 +23,19 @@ import org.apache.yetus.audience.InterfaceAudience; /** - * A {@code Callback} for use by the {@code SaslServer} implementation when it - * needs to provide an OAuth 2 bearer token compact serialization for - * validation. Callback handlers should use the - * {@link #error(String, String, String)} method to communicate errors back to - * the SASL Client as per - * RFC 6749: The OAuth - * 2.0 Authorization Framework and the RFC 6749: The OAuth 2.0 + * Authorization Framework and the IANA - * OAuth Extensions Error Registry. Callback handlers should communicate - * other problems by raising an {@code IOException}. + * OAuth Extensions Error Registry. Callback handlers should communicate other problems by + * raising an {@code IOException}. *

- * This class was introduced in 2.0.0 and, while it feels stable, it could - * evolve. We will try to evolve the API in a compatible manner, but we reserve - * the right to make breaking changes in minor releases, if necessary. We will - * update the {@code InterfaceStability} annotation and this notice once the API - * is considered stable. + * This class was introduced in 2.0.0 and, while it feels stable, it could evolve. We will try to + * evolve the API in a compatible manner, but we reserve the right to make breaking changes in minor + * releases, if necessary. We will update the {@code InterfaceStability} annotation and this notice + * once the API is considered stable. */ @InterfaceAudience.Public public class OAuthBearerValidatorCallback implements Callback { @@ -50,9 +47,8 @@ public class OAuthBearerValidatorCallback implements Callback { /** * Constructor - * - * @param tokenValue - * the mandatory/non-blank token value + *

+ * @param tokenValue the mandatory/non-blank token value */ public OAuthBearerValidatorCallback(String tokenValue) { if (StringUtils.isEmpty(tokenValue)) { @@ -63,7 +59,6 @@ public OAuthBearerValidatorCallback(String tokenValue) { /** * Return the (always non-null) token value - * * @return the (always non-null) token value */ public String tokenValue() { @@ -72,7 +67,6 @@ public String tokenValue() { /** * Return the (potentially null) token - * * @return the (potentially null) token */ public OAuthBearerToken token() { @@ -81,12 +75,10 @@ public OAuthBearerToken token() { /** * Return the (potentially null) error status value as per - * RFC 7628: A Set - * of Simple Authentication and Security Layer (SASL) Mechanisms for OAuth - * and the RFC 7628: A Set of Simple + * Authentication and Security Layer (SASL) Mechanisms for OAuth and the IANA * OAuth Extensions Error Registry. - * * @return the (potentially null) error status value */ public String errorStatus() { @@ -95,9 +87,8 @@ public String errorStatus() { /** * Return the (potentially null) error scope value as per - * RFC 7628: A Set - * of Simple Authentication and Security Layer (SASL) Mechanisms for OAuth. - * + * RFC 7628: A Set of Simple + * Authentication and Security Layer (SASL) Mechanisms for OAuth. * @return the (potentially null) error scope value */ public String errorScope() { @@ -106,9 +97,8 @@ public String errorScope() { /** * Return the (potentially null) error openid-configuration value as per - * RFC 7628: A Set - * of Simple Authentication and Security Layer (SASL) Mechanisms for OAuth. - * + * RFC 7628: A Set of Simple + * Authentication and Security Layer (SASL) Mechanisms for OAuth. * @return the (potentially null) error openid-configuration value */ public String errorOpenIDConfiguration() { @@ -116,11 +106,10 @@ public String errorOpenIDConfiguration() { } /** - * Set the token. The token value is unchanged and is expected to match the - * provided token's value. All error values are cleared. - * - * @param token - * the mandatory token to set + * Set the token. The token value is unchanged and is expected to match the provided token's + * value. All error values are cleared. + *

+ * @param token the mandatory token to set */ public void token(OAuthBearerToken token) { this.token = Objects.requireNonNull(token); @@ -130,19 +119,15 @@ public void token(OAuthBearerToken token) { } /** - * Set the error values as per - * RFC 7628: A Set - * of Simple Authentication and Security Layer (SASL) Mechanisms for OAuth. - * Any token is cleared. - * - * @param errorStatus - * the mandatory error status value from the IANA - * OAuth Extensions Error Registry to set - * @param errorScope - * the optional error scope value to set - * @param errorOpenIDConfiguration - * the optional error openid-configuration value to set + * Set the error values as per RFC + * 7628: A Set of Simple Authentication and Security Layer (SASL) Mechanisms for OAuth. Any + * token is cleared. + *

+ * @param errorStatus the mandatory error status value from the IANA + * OAuth Extensions Error Registry to set + * @param errorScope the optional error scope value to set + * @param errorOpenIDConfiguration the optional error openid-configuration value to set */ public void error(String errorStatus, String errorScope, String errorOpenIDConfiguration) { if (StringUtils.isEmpty(errorStatus)) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslServer.java index 3a56d5deb7c7..0206c0be2094 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslServer.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.security.oauthbearer.internals; import static org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerUtils.OAUTHBEARER_MECHANISM; + import com.nimbusds.jose.shaded.json.JSONObject; import java.io.IOException; import java.nio.charset.StandardCharsets; @@ -42,11 +43,10 @@ import org.slf4j.LoggerFactory; /** - * {@code SaslServer} implementation for SASL/OAUTHBEARER in Kafka. An instance - * of {@link OAuthBearerToken} is available upon successful authentication via - * the negotiated property "{@code OAUTHBEARER.token}"; the token could be used - * in a custom authorizer (to authorize based on JWT claims rather than ACLs, - * for example). + * {@code SaslServer} implementation for SASL/OAUTHBEARER in Kafka. An instance of + * {@link OAuthBearerToken} is available upon successful authentication via the negotiated property + * "{@code OAUTHBEARER.token}"; the token could be used in a custom authorizer (to authorize based + * on JWT claims rather than ACLs, for example). */ @InterfaceAudience.Public public class OAuthBearerSaslServer implements SaslServer { @@ -73,24 +73,25 @@ public OAuthBearerSaslServer(CallbackHandler callbackHandler) { } /** - * @throws SaslAuthenticationException - * if access token cannot be validated - *

- * Note: This method may throw - * {@link SaslAuthenticationException} to provide custom error - * messages to clients. But care should be taken to avoid including - * any information in the exception message that should not be - * leaked to unauthenticated clients. It may be safer to throw - * {@link SaslException} in some cases so that a standard error - * message is returned to clients. - *

+ * @throws SaslAuthenticationException if access token cannot be validated + *

+ * Note: This method may throw + * {@link SaslAuthenticationException} to provide custom error + * messages to clients. But care should be taken to avoid + * including any information in the exception message that + * should not be leaked to unauthenticated clients. It may be + * safer to throw {@link SaslException} in some cases so that + * a standard error message is returned to clients. + *

*/ @Override public byte[] evaluateResponse(byte[] response) throws SaslException, SaslAuthenticationException { try { - if (response.length == 1 && response[0] == OAuthBearerSaslClient.BYTE_CONTROL_A && - errorMessage != null) { + if ( + response.length == 1 && response[0] == OAuthBearerSaslClient.BYTE_CONTROL_A + && errorMessage != null + ) { LOG.error("Received %x01 response from client after it received our error"); throw new SaslAuthenticationException(errorMessage); } @@ -166,11 +167,10 @@ public void dispose() { tokenForNegotiatedProperty = null; } - private byte[] process(String tokenValue, String authorizationId) - throws SaslException { + private byte[] process(String tokenValue, String authorizationId) throws SaslException { OAuthBearerValidatorCallback callback = new OAuthBearerValidatorCallback(tokenValue); try { - callbackHandler.handle(new Callback[] {callback}); + callbackHandler.handle(new Callback[] { callback }); } catch (IOException | UnsupportedCallbackException e) { handleCallbackError(e); } @@ -182,8 +182,8 @@ private byte[] process(String tokenValue, String authorizationId) return errorMessage.getBytes(StandardCharsets.UTF_8); } /* - * We support the client specifying an authorization ID as per the SASL - * specification, but it must match the principal name if it is specified. + * We support the client specifying an authorization ID as per the SASL specification, but it + * must match the principal name if it is specified. */ if (!authorizationId.isEmpty() && !authorizationId.equals(token.principalName())) { throw new SaslAuthenticationException(String.format( diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslServerProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslServerProvider.java index 99f7df223b45..eee4c24d6af6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslServerProvider.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslServerProvider.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.security.oauthbearer.internals; import static org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerUtils.OAUTHBEARER_MECHANISM; + import java.security.Provider; import java.security.Security; import org.apache.yetus.audience.InterfaceAudience; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerConfigException.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerConfigException.java index acd5b047e1c8..a0975244d794 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerConfigException.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerConfigException.java @@ -20,8 +20,8 @@ import org.apache.yetus.audience.InterfaceAudience; /** - * Exception thrown when there is a problem with the configuration (an invalid - * option in a JAAS config, for example). + * Exception thrown when there is a problem with the configuration (an invalid option in a JAAS + * config, for example). */ @InterfaceAudience.Public public class OAuthBearerConfigException extends RuntimeException { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerIllegalTokenException.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerIllegalTokenException.java index 09aa28c57ee4..289defeb5f03 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerIllegalTokenException.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerIllegalTokenException.java @@ -21,8 +21,8 @@ import org.apache.yetus.audience.InterfaceAudience; /** - * Exception thrown when token validation fails due to a problem with the token - * itself (as opposed to a missing remote resource or a configuration problem) + * Exception thrown when token validation fails due to a problem with the token itself (as opposed + * to a missing remote resource or a configuration problem) */ @InterfaceAudience.Public public class OAuthBearerIllegalTokenException extends RuntimeException { @@ -31,10 +31,8 @@ public class OAuthBearerIllegalTokenException extends RuntimeException { /** * Constructor - * - * @param reason - * the mandatory reason for the validation failure; it must indicate - * failure + *

+ * @param reason the mandatory reason for the validation failure; it must indicate failure */ public OAuthBearerIllegalTokenException(OAuthBearerValidationResult reason) { super(Objects.requireNonNull(reason, "Reason cannot be null").failureDescription()); @@ -56,7 +54,6 @@ public OAuthBearerIllegalTokenException(OAuthBearerValidationResult reason, Thro /** * Return the (always non-null) reason for the validation failure - * * @return the reason for the validation failure */ public OAuthBearerValidationResult reason() { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerSignedJwt.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerSignedJwt.java index f17457d66edc..7a7d73477cf6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerSignedJwt.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerSignedJwt.java @@ -43,7 +43,7 @@ /** * Signed JWT implementation for OAuth Bearer authentication mech of SASL. - * + *

* This class is based on Kafka's Unsecured JWS token implementation. */ @InterfaceAudience.Public @@ -59,11 +59,9 @@ public class OAuthBearerSignedJwt implements OAuthBearerToken { /** * Constructor base64 encoded JWT token and JWK Set. - * - * @param compactSerialization - * the compact serialization to parse as a signed JWT - * @param jwkSet - * the key set which the signature of this JWT should be verified with + *

+ * @param compactSerialization the compact serialization to parse as a signed JWT + * @param jwkSet the key set which the signature of this JWT should be verified with */ public OAuthBearerSignedJwt(String compactSerialization, JWKSet jwkSet) { this.jwkSet = jwkSet; @@ -87,7 +85,6 @@ public long lifetimeMs() { /** * Return the JWT Claim Set as a {@code Map} - * * @return the (always non-null but possibly empty) claims */ public Map claims() { @@ -96,8 +93,8 @@ public Map claims() { /** * Set required audience, as per - * - * RFC7519 Section 4.1.3 + * RFC7519 Section + * 4.1.3 */ public OAuthBearerSignedJwt audience(String aud) { this.requiredAudience = aud; @@ -106,8 +103,8 @@ public OAuthBearerSignedJwt audience(String aud) { /** * Set required issuer, as per - * - * RFC7519 Section 4.1.1 + * RFC7519 Section + * 4.1.1 */ public OAuthBearerSignedJwt issuer(String iss) { this.requiredIssuer = iss; @@ -124,18 +121,16 @@ public OAuthBearerSignedJwt maxClockSkewSeconds(int value) { } /** - * This method provides a single method for validating the JWT for use in - * request processing. - * - * @throws OAuthBearerIllegalTokenException - * if the compact serialization is not a valid JWT - * (meaning it did not have 3 dot-separated Base64URL sections - * with a digital signature; or the header or claims - * either are not valid Base 64 URL encoded values or are not JSON - * after decoding; or the mandatory '{@code alg}' header value is - * missing) + * This method provides a single method for validating the JWT for use in request processing. + *

+ * @throws OAuthBearerIllegalTokenException if the compact serialization is not a valid JWT + * (meaning it did not have 3 dot-separated Base64URL + * sections with a digital signature; or the header or + * claims either are not valid Base 64 URL encoded values + * or are not JSON after decoding; or the mandatory + * '{@code alg}' header value is missing) */ - public OAuthBearerSignedJwt validate(){ + public OAuthBearerSignedJwt validate() { try { this.claims = validateToken(compactSerialization); Date expirationTimeSeconds = claims.getExpirationTime(); @@ -146,8 +141,8 @@ public OAuthBearerSignedJwt validate(){ lifetime = expirationTimeSeconds.toInstant().toEpochMilli(); String principalName = claims.getSubject(); if (StringUtils.isBlank(principalName)) { - throw new OAuthBearerIllegalTokenException(OAuthBearerValidationResult - .newFailure("No principal name in JWT claim")); + throw new OAuthBearerIllegalTokenException( + OAuthBearerValidationResult.newFailure("No principal name in JWT claim")); } return this; } catch (ParseException | BadJOSEException | JOSEException e) { @@ -184,9 +179,8 @@ private JWTClaimsSet validateToken(String jwtToken) jwtClaimsSetVerifier.setMaxClockSkew(maxClockSkewSeconds); jwtProcessor.setJWTClaimsSetVerifier(jwtClaimsSetVerifier); - JWSKeySelector keySelector = - new JWSVerificationKeySelector<>((JWSAlgorithm)jwt.getHeader().getAlgorithm(), - new ImmutableJWKSet<>(jwkSet)); + JWSKeySelector keySelector = new JWSVerificationKeySelector<>( + (JWSAlgorithm) jwt.getHeader().getAlgorithm(), new ImmutableJWKSet<>(jwkSet)); jwtProcessor.setJWSKeySelector(keySelector); return jwtProcessor.process(jwtToken, null); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerSignedJwtValidatorCallbackHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerSignedJwtValidatorCallbackHandler.java index 35e2a13b880b..148e891000d3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerSignedJwtValidatorCallbackHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerSignedJwtValidatorCallbackHandler.java @@ -18,11 +18,8 @@ package org.apache.hadoop.hbase.security.oauthbearer.internals.knox; import static org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerUtils.OAUTHBEARER_MECHANISM; + import com.nimbusds.jose.jwk.JWKSet; -import java.io.File; -import java.io.IOException; -import java.net.URL; -import java.text.ParseException; import java.util.Map; import java.util.Objects; import javax.security.auth.callback.Callback; @@ -36,36 +33,30 @@ import org.slf4j.LoggerFactory; /** - * A {@code CallbackHandler} that recognizes - * {@link OAuthBearerValidatorCallback} and validates a secure (signed) OAuth 2 - * bearer token (JWT). - * - * It requires a valid JWK Set to be initialized at startup which holds the available - * RSA public keys that JWT signature can be validated with. The Set can be initialized - * via an URL or a local file. - * - * It requires there to be an "exp" (Expiration Time) - * claim of type Number. If "iat" (Issued At) or - * "nbf" (Not Before) claims are present each must be a number that - * precedes the Expiration Time claim, and if both are present the Not Before - * claim must not precede the Issued At claim. It also accepts the following - * options, none of which are required: + * A {@code CallbackHandler} that recognizes {@link OAuthBearerValidatorCallback} and validates a + * secure (signed) OAuth 2 bearer token (JWT). + *

+ * It requires a valid JWK Set to be initialized at startup which holds the available RSA public + * keys that JWT signature can be validated with. The Set can be initialized via an URL or a local + * file. + *

+ * It requires there to be an "exp" (Expiration Time) claim of type Number. If + * "iat" (Issued At) or "nbf" (Not Before) claims are present each must be + * a number that precedes the Expiration Time claim, and if both are present the Not Before claim + * must not precede the Issued At claim. It also accepts the following options, none of which are + * required: *

    - *
  • {@code hbase.security.oauth.jwt.jwks.url} set to a non-empty value if you - * wish to initialize the JWK Set via an URL. HTTPS URLs must have valid certificates. - *
  • - *
  • {@code hbase.security.oauth.jwt.jwks.file} set to a non-empty value if you - * wish to initialize the JWK Set from a local JSON file. - *
  • - *
  • {@code hbase.security.oauth.jwt.audience} set to a String value which - * you want the desired audience ("aud") the JWT to have.
  • - *
  • {@code hbase.security.oauth.jwt.issuer} set to a String value which - * you want the issuer ("iss") of the JWT has to be.
  • - *
  • {@code hbase.security.oauth.jwt.allowableclockskewseconds} set to a positive integer - * value if you wish to allow up to some number of positive seconds of - * clock skew (the default is 0)
  • + *
  • {@code hbase.security.oauth.jwt.jwks.url} set to a non-empty value if you wish to initialize + * the JWK Set via an URL. HTTPS URLs must have valid certificates.
  • + *
  • {@code hbase.security.oauth.jwt.jwks.file} set to a non-empty value if you wish to initialize + * the JWK Set from a local JSON file.
  • + *
  • {@code hbase.security.oauth.jwt.audience} set to a String value which you want the desired + * audience ("aud") the JWT to have.
  • + *
  • {@code hbase.security.oauth.jwt.issuer} set to a String value which you want the issuer + * ("iss") of the JWT has to be.
  • + *
  • {@code hbase.security.oauth.jwt.allowableclockskewseconds} set to a positive integer value if + * you wish to allow up to some number of positive seconds of clock skew (the default is 0)
  • *
- * * This class is based on Kafka's OAuthBearerUnsecuredValidatorCallbackHandler. */ @InterfaceAudience.Public @@ -110,7 +101,8 @@ public void handle(Callback[] callbacks) throws UnsupportedCallbackException { } } - @Override public void configure(Configuration configs, String saslMechanism, + @Override + public void configure(Configuration configs, String saslMechanism, Map saslProps) { if (!OAUTHBEARER_MECHANISM.equals(saslMechanism)) { throw new IllegalArgumentException( @@ -132,11 +124,9 @@ private void handleCallback(OAuthBearerValidatorCallback callback) { if (tokenValue == null) { throw new IllegalArgumentException("Callback missing required token value"); } - OAuthBearerSignedJwt signedJwt = new OAuthBearerSignedJwt(tokenValue, jwkSet) - .audience(requiredAudience()) - .issuer(requiredIssuer()) - .maxClockSkewSeconds(allowableClockSkewSeconds()) - .validate(); + OAuthBearerSignedJwt signedJwt = + new OAuthBearerSignedJwt(tokenValue, jwkSet).audience(requiredAudience()) + .issuer(requiredIssuer()).maxClockSkewSeconds(allowableClockSkewSeconds()).validate(); LOG.info("Successfully validated token with principal {}: {}", signedJwt.principalName(), signedJwt.claims()); @@ -152,19 +142,19 @@ private String requiredIssuer() { } private int allowableClockSkewSeconds() { - String allowableClockSkewSecondsValue = hBaseConfiguration.get( - ALLOWABLE_CLOCK_SKEW_SECONDS_OPTION); + String allowableClockSkewSecondsValue = + hBaseConfiguration.get(ALLOWABLE_CLOCK_SKEW_SECONDS_OPTION); int skewSeconds; try { skewSeconds = StringUtils.isBlank(allowableClockSkewSecondsValue) - ? 0 : Integer.parseInt(allowableClockSkewSecondsValue.trim()); + ? 0 + : Integer.parseInt(allowableClockSkewSecondsValue.trim()); } catch (NumberFormatException e) { throw new OAuthBearerConfigException(e.getMessage(), e); } if (skewSeconds < 0) { - throw new OAuthBearerConfigException( - String.format("Allowable clock skew seconds must not be negative: %s", - allowableClockSkewSecondsValue)); + throw new OAuthBearerConfigException(String.format( + "Allowable clock skew seconds must not be negative: %s", allowableClockSkewSecondsValue)); } return skewSeconds; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerValidationResult.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerValidationResult.java index d41962d5e67b..d0a3ad9ea701 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerValidationResult.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerValidationResult.java @@ -22,7 +22,7 @@ /** * The result of some kind of token validation - * + *

* This class has been copy-and-pasted from Kafka codebase. */ @InterfaceAudience.Public @@ -35,7 +35,6 @@ public final class OAuthBearerValidationResult implements Serializable { /** * Return an instance indicating success - * * @return an instance indicating success */ public static OAuthBearerValidationResult newSuccess() { @@ -44,9 +43,7 @@ public static OAuthBearerValidationResult newSuccess() { /** * Return a new validation failure instance - * - * @param failureDescription - * optional description of the failure + * @param failureDescription optional description of the failure * @return a new validation failure instance */ public static OAuthBearerValidationResult newFailure(String failureDescription) { @@ -55,14 +52,10 @@ public static OAuthBearerValidationResult newFailure(String failureDescription) /** * Return a new validation failure instance - * - * @param failureDescription - * optional description of the failure - * @param failureScope - * optional scope to be reported with the failure - * @param failureOpenIdConfig - * optional OpenID Connect configuration to be reported with the - * failure + * @param failureDescription optional description of the failure + * @param failureScope optional scope to be reported with the failure + * @param failureOpenIdConfig optional OpenID Connect configuration to be reported with the + * failure * @return a new validation failure instance */ public static OAuthBearerValidationResult newFailure(String failureDescription, @@ -85,7 +78,6 @@ private OAuthBearerValidationResult(boolean success, String failureDescription, /** * Return true if this instance indicates success, otherwise false - * * @return true if this instance indicates success, otherwise false */ public boolean success() { @@ -94,7 +86,6 @@ public boolean success() { /** * Return the (potentially null) descriptive message for the failure - * * @return the (potentially null) descriptive message for the failure */ public String failureDescription() { @@ -103,7 +94,6 @@ public String failureDescription() { /** * Return the (potentially null) scope to be reported with the failure - * * @return the (potentially null) scope to be reported with the failure */ public String failureScope() { @@ -111,21 +101,16 @@ public String failureScope() { } /** - * Return the (potentially null) OpenID Connect configuration to be reported - * with the failure - * - * @return the (potentially null) OpenID Connect configuration to be reported - * with the failure + * Return the (potentially null) OpenID Connect configuration to be reported with the failure + * @return the (potentially null) OpenID Connect configuration to be reported with the failure */ public String failureOpenIdConfig() { return failureOpenIdConfig; } /** - * Raise an exception if this instance indicates failure, otherwise do nothing - * - * @throws OAuthBearerIllegalTokenException - * if this instance indicates failure + * Raise an exception if this instance indicates failure, otherwise do nothing n * if this + * instance indicates failure */ public void throwExceptionIfFailed() throws OAuthBearerIllegalTokenException { if (!success()) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslServerAuthenticationProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslServerAuthenticationProvider.java index d6999230800b..9e508bf43803 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslServerAuthenticationProvider.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslServerAuthenticationProvider.java @@ -1,5 +1,5 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one + * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.security.provider; +import com.nimbusds.jose.jwk.JWKSet; import java.io.File; import java.io.IOException; import java.io.InterruptedIOException; @@ -26,7 +27,6 @@ import java.util.Map; import javax.security.sasl.Sasl; import javax.security.sasl.SaslException; -import com.nimbusds.jose.jwk.JWKSet; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.security.auth.AuthenticateCallbackHandler; @@ -41,14 +41,13 @@ @InterfaceAudience.Private public class OAuthBearerSaslServerAuthenticationProvider - extends OAuthBearerSaslAuthenticationProvider - implements SaslServerAuthenticationProvider { + extends OAuthBearerSaslAuthenticationProvider implements SaslServerAuthenticationProvider { private static final String OPTION_PREFIX = "hbase.security.oauth.jwt."; private static final String JWKS_URL = OPTION_PREFIX + "jwks.url"; private static final String JWKS_FILE = OPTION_PREFIX + "jwks.file"; - private static final Logger LOG = LoggerFactory.getLogger( - OAuthBearerSaslServerAuthenticationProvider.class); + private static final Logger LOG = + LoggerFactory.getLogger(OAuthBearerSaslServerAuthenticationProvider.class); private Configuration hbaseConfiguration; private boolean initialized = false; private JWKSet jwkSet = null; @@ -58,7 +57,8 @@ public class OAuthBearerSaslServerAuthenticationProvider LOG.info("OAuthBearer SASL server provider has been initialized"); } - @Override public void init(Configuration conf) throws IOException { + @Override + public void init(Configuration conf) throws IOException { this.hbaseConfiguration = conf; try { loadJwkSet(); @@ -68,9 +68,10 @@ public class OAuthBearerSaslServerAuthenticationProvider this.initialized = true; } - @Override public AttemptingUserProvidingSaslServer createServer( - SecretManager secretManager, Map saslProps) - throws IOException { + @Override + public AttemptingUserProvidingSaslServer + createServer(SecretManager secretManager, Map saslProps) + throws IOException { if (!initialized) { throw new IllegalStateException( @@ -90,9 +91,10 @@ public AttemptingUserProvidingSaslServer run() throws SaslException { new OAuthBearerSignedJwtValidatorCallbackHandler(jwkSet); callbackHandler.configure(hbaseConfiguration, getSaslAuthMethod().getSaslMechanism(), saslProps); - return new AttemptingUserProvidingSaslServer(Sasl.createSaslServer( - getSaslAuthMethod().getSaslMechanism(), null, null, saslProps, - callbackHandler), () -> null); + return new AttemptingUserProvidingSaslServer( + Sasl.createSaslServer(getSaslAuthMethod().getSaslMechanism(), null, null, saslProps, + callbackHandler), + () -> null); } }); } catch (InterruptedException e) { @@ -101,11 +103,13 @@ public AttemptingUserProvidingSaslServer run() throws SaslException { } } - @Override public boolean supportsProtocolAuthentication() { + @Override + public boolean supportsProtocolAuthentication() { return true; } - @Override public UserGroupInformation getAuthorizedUgi(String authzId, + @Override + public UserGroupInformation getAuthorizedUgi(String authzId, SecretManager secretManager) { UserGroupInformation ugi = UserGroupInformation.createRemoteUser(authzId); ugi.setAuthenticationMethod(getSaslAuthMethod().getAuthMethod()); @@ -117,8 +121,8 @@ private void loadJwkSet() throws IOException, ParseException { String jwksUrl = hbaseConfiguration.get(JWKS_URL); if (StringUtils.isBlank(jwksFile) && StringUtils.isBlank(jwksUrl)) { - throw new RuntimeException("Failed to initialize JWKS db. " - + JWKS_FILE + " or " + JWKS_URL + " must be specified in the config."); + throw new RuntimeException("Failed to initialize JWKS db. " + JWKS_FILE + " or " + JWKS_URL + + " must be specified in the config."); } if (!StringUtils.isBlank(jwksFile)) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerTokenMock.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerTokenMock.java index 9e6670b19d30..c82e6162cb86 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerTokenMock.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerTokenMock.java @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerValidatorCallbackTest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerValidatorCallbackTest.java index 8be3cfcaf4a4..6be62f8b8c7c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerValidatorCallbackTest.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerValidatorCallbackTest.java @@ -20,6 +20,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertSame; + import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; @@ -27,7 +28,7 @@ import org.junit.Test; import org.junit.experimental.categories.Category; -@Category({ MiscTests.class, SmallTests.class}) +@Category({ MiscTests.class, SmallTests.class }) public class OAuthBearerValidatorCallbackTest { @ClassRule public static final HBaseClassTestRule CLASS_RULE = diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslServerTest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslServerTest.java index e50ce773bd89..7241793cfa5e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslServerTest.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslServerTest.java @@ -22,6 +22,7 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; + import com.nimbusds.jose.JOSEException; import com.nimbusds.jose.jwk.JWKSet; import com.nimbusds.jose.jwk.RSAKey; @@ -40,7 +41,7 @@ import org.junit.Test; import org.junit.experimental.categories.Category; -@Category({ MiscTests.class, SmallTests.class}) +@Category({ MiscTests.class, SmallTests.class }) public class OAuthBearerSaslServerTest { @ClassRule public static final HBaseClassTestRule CLASS_RULE = @@ -67,11 +68,10 @@ public void setUp() throws JOSEException { @Test public void noAuthorizationIdSpecified() throws Exception { - byte[] nextChallenge = saslServer - .evaluateResponse(clientInitialResponse(null)); + byte[] nextChallenge = saslServer.evaluateResponse(clientInitialResponse(null)); // also asserts that no authentication error is thrown // if OAuthBearerExtensionsValidatorCallback is not supported - assertTrue("Next challenge is not empty",nextChallenge.length == 0); + assertTrue("Next challenge is not empty", nextChallenge.length == 0); } @Test @@ -80,15 +80,13 @@ public void negotiatedProperty() throws Exception { OAuthBearerToken token = (OAuthBearerToken) saslServer.getNegotiatedProperty("OAUTHBEARER.token"); assertNotNull(token); - assertEquals(token.lifetimeMs(), - saslServer.getNegotiatedProperty( - OAuthBearerSaslServer.CREDENTIAL_LIFETIME_MS_SASL_NEGOTIATED_PROPERTY_KEY)); + assertEquals(token.lifetimeMs(), saslServer.getNegotiatedProperty( + OAuthBearerSaslServer.CREDENTIAL_LIFETIME_MS_SASL_NEGOTIATED_PROPERTY_KEY)); } @Test public void authorizatonIdEqualsAuthenticationId() throws Exception { - byte[] nextChallenge = saslServer - .evaluateResponse(clientInitialResponse(USER)); + byte[] nextChallenge = saslServer.evaluateResponse(clientInitialResponse(USER)); assertTrue("Next challenge is not empty", nextChallenge.length == 0); } @@ -105,8 +103,7 @@ public void illegalToken() throws Exception { assertEquals("{\"status\":\"invalid_token\"}", challenge); } - private byte[] clientInitialResponse(String authorizationId) - throws OAuthBearerConfigException { + private byte[] clientInitialResponse(String authorizationId) throws OAuthBearerConfigException { return clientInitialResponse(authorizationId, false); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerSignedJwtTest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerSignedJwtTest.java index 4b72f88bd6c4..e04569601259 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerSignedJwtTest.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerSignedJwtTest.java @@ -19,6 +19,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThrows; + import com.nimbusds.jose.JOSEException; import com.nimbusds.jose.jwk.JWKSet; import com.nimbusds.jose.jwk.RSAKey; @@ -34,7 +35,7 @@ import org.junit.Test; import org.junit.experimental.categories.Category; -@Category({ MiscTests.class, SmallTests.class}) +@Category({ MiscTests.class, SmallTests.class }) public class OAuthBearerSignedJwtTest { @ClassRule public static final HBaseClassTestRule CLASS_RULE = @@ -57,13 +58,12 @@ public void validCompactSerialization() throws JOSEException { LocalDate issuedAt = LocalDate.now(ZONE_ID); LocalDate expirationTime = issuedAt.plusDays(EXP_DAYS); - String validCompactSerialization = - compactSerialization(subject, issuedAt, expirationTime); - OAuthBearerSignedJwt jws = new OAuthBearerSignedJwt(validCompactSerialization, JWK_SET) - .validate(); + String validCompactSerialization = compactSerialization(subject, issuedAt, expirationTime); + OAuthBearerSignedJwt jws = + new OAuthBearerSignedJwt(validCompactSerialization, JWK_SET).validate(); assertEquals(5, jws.claims().size()); assertEquals(subject, jws.claims().get("sub")); - assertEquals(issuedAt, Date.class.cast(jws.claims().get("iat")).toInstant() + assertEquals(issuedAt, Date.class.cast(jws.claims().get("iat")).toInstant() .atZone(ZoneId.systemDefault()).toLocalDate()); assertEquals(expirationTime, Date.class.cast(jws.claims().get("exp")).toInstant() .atZone(ZoneId.systemDefault()).toLocalDate()); @@ -76,8 +76,7 @@ public void missingPrincipal() throws JOSEException { String subject = null; LocalDate issuedAt = LocalDate.now(ZONE_ID); LocalDate expirationTime = issuedAt.plusDays(EXP_DAYS); - String validCompactSerialization = - compactSerialization(subject, issuedAt, expirationTime); + String validCompactSerialization = compactSerialization(subject, issuedAt, expirationTime); assertThrows(OAuthBearerIllegalTokenException.class, () -> new OAuthBearerSignedJwt(validCompactSerialization, JWK_SET).validate()); } @@ -87,19 +86,16 @@ public void blankPrincipalName() throws JOSEException { String subject = " "; LocalDate issuedAt = LocalDate.now(ZONE_ID); LocalDate expirationTime = issuedAt.plusDays(EXP_DAYS); - String validCompactSerialization = - compactSerialization(subject, issuedAt, expirationTime); + String validCompactSerialization = compactSerialization(subject, issuedAt, expirationTime); assertThrows(OAuthBearerIllegalTokenException.class, () -> new OAuthBearerSignedJwt(validCompactSerialization, JWK_SET).validate()); } @Test public void missingIssuer() throws JOSEException { - String validCompactSerialization = - JwtTestUtils.createSignedJwtWithIssuer(RSA_KEY, ""); + String validCompactSerialization = JwtTestUtils.createSignedJwtWithIssuer(RSA_KEY, ""); assertThrows(OAuthBearerIllegalTokenException.class, - () -> new OAuthBearerSignedJwt(validCompactSerialization, JWK_SET) - .issuer("test-issuer") + () -> new OAuthBearerSignedJwt(validCompactSerialization, JWK_SET).issuer("test-issuer") .validate()); } @@ -108,14 +104,13 @@ public void badIssuer() throws JOSEException { String validCompactSerialization = JwtTestUtils.createSignedJwtWithIssuer(RSA_KEY, "bad-issuer"); assertThrows(OAuthBearerIllegalTokenException.class, - () -> new OAuthBearerSignedJwt(validCompactSerialization, JWK_SET) - .issuer("test-issuer") + () -> new OAuthBearerSignedJwt(validCompactSerialization, JWK_SET).issuer("test-issuer") .validate()); } private String compactSerialization(String subject, LocalDate issuedAt, LocalDate expirationTime) throws JOSEException { - return JwtTestUtils.createSignedJwt(RSA_KEY, "me", subject, - expirationTime, issuedAt, "test-audience"); + return JwtTestUtils.createSignedJwt(RSA_KEY, "me", subject, expirationTime, issuedAt, + "test-audience"); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerSignedJwtValidatorCallbackHandlerTest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerSignedJwtValidatorCallbackHandlerTest.java index 2e5d4b795265..5ffba822ee5d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerSignedJwtValidatorCallbackHandlerTest.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerSignedJwtValidatorCallbackHandlerTest.java @@ -22,6 +22,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; + import com.nimbusds.jose.JOSEException; import com.nimbusds.jose.jwk.JWKSet; import com.nimbusds.jose.jwk.RSAKey; @@ -40,7 +41,7 @@ import org.junit.Test; import org.junit.experimental.categories.Category; -@Category({ MiscTests.class, SmallTests.class}) +@Category({ MiscTests.class, SmallTests.class }) public class OAuthBearerSignedJwtValidatorCallbackHandlerTest { @ClassRule public static final HBaseClassTestRule CLASS_RULE = @@ -70,26 +71,23 @@ public void before() throws JOSEException { public void validToken() throws JOSEException, UnsupportedCallbackException { Object validationResult = validationResult(EMPTY_CONFIG, JwtTestUtils.createSignedJwt(RSA_KEY)); assertTrue(validationResult instanceof OAuthBearerValidatorCallback); - assertTrue(((OAuthBearerValidatorCallback) validationResult).token() - instanceof OAuthBearerSignedJwt); + assertTrue( + ((OAuthBearerValidatorCallback) validationResult).token() instanceof OAuthBearerSignedJwt); } @Test - public void missingPrincipal() - throws UnsupportedCallbackException, JOSEException { + public void missingPrincipal() throws UnsupportedCallbackException, JOSEException { LocalDate now = LocalDate.now(ZONE_ID); - String token = JwtTestUtils.createSignedJwt(RSA_KEY, "me", "", - now.plusDays(1), now, "test-aud"); + String token = + JwtTestUtils.createSignedJwt(RSA_KEY, "me", "", now.plusDays(1), now, "test-aud"); confirmFailsValidation(EMPTY_CONFIG, token); } @Test public void tooEarlyExpirationTime() throws JOSEException, UnsupportedCallbackException { LocalDate now = LocalDate.now(ZONE_ID); - String token = JwtTestUtils.createSignedJwt(RSA_KEY, "me", "", - now.minusDays(1), - now.minusDays(1), - "test-aud"); + String token = JwtTestUtils.createSignedJwt(RSA_KEY, "me", "", now.minusDays(1), + now.minusDays(1), "test-aud"); confirmFailsValidation(EMPTY_CONFIG, token); } @@ -98,8 +96,8 @@ public void requiredAudience() throws JOSEException, UnsupportedCallbackExceptio String token = JwtTestUtils.createSignedJwtWithAudience(RSA_KEY, "test-audience"); Object validationResult = validationResult(REQUIRED_AUDIENCE_CONFIG, token); assertTrue(validationResult instanceof OAuthBearerValidatorCallback); - assertTrue(((OAuthBearerValidatorCallback) validationResult).token() - instanceof OAuthBearerSignedJwt); + assertTrue( + ((OAuthBearerValidatorCallback) validationResult).token() instanceof OAuthBearerSignedJwt); } @Test @@ -119,8 +117,8 @@ public void requiredIssuer() throws UnsupportedCallbackException, JOSEException String token = JwtTestUtils.createSignedJwtWithIssuer(RSA_KEY, "test-issuer"); Object validationResult = validationResult(REQUIRED_ISSUER_CONFIG, token); assertTrue(validationResult instanceof OAuthBearerValidatorCallback); - assertTrue(((OAuthBearerValidatorCallback) validationResult).token() - instanceof OAuthBearerSignedJwt); + assertTrue( + ((OAuthBearerValidatorCallback) validationResult).token() instanceof OAuthBearerSignedJwt); } @Test @@ -148,14 +146,13 @@ private void confirmFailsValidation(HBaseConfiguration config, String tokenValue } private OAuthBearerValidatorCallback validationResult(HBaseConfiguration config, - String tokenValue) - throws UnsupportedCallbackException { + String tokenValue) throws UnsupportedCallbackException { OAuthBearerValidatorCallback callback = new OAuthBearerValidatorCallback(tokenValue); - createCallbackHandler(config).handle(new Callback[] {callback}); + createCallbackHandler(config).handle(new Callback[] { callback }); return callback; } - @SuppressWarnings({"unchecked", "rawtypes"}) + @SuppressWarnings({ "unchecked", "rawtypes" }) private OAuthBearerSignedJwtValidatorCallbackHandler createCallbackHandler(HBaseConfiguration config) { OAuthBearerSignedJwtValidatorCallbackHandler callbackHandler = diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestOAuthBearerAuthentication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestOAuthBearerAuthentication.java index ad799d7a20c9..f7774284989b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestOAuthBearerAuthentication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestOAuthBearerAuthentication.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.security.token; import static org.junit.Assert.assertArrayEquals; + import com.nimbusds.jose.JOSEException; import com.nimbusds.jose.JWSAlgorithm; import com.nimbusds.jose.JWSHeader; @@ -94,8 +95,7 @@ public static void setUp() throws Exception { public TestName testName = new TestName(); private static void initRSA() throws JOSEException, IOException { - RSA = new RSAKeyGenerator(2048) - .keyUse(KeyUse.SIGNATURE) // indicate the intended use of the key + RSA = new RSAKeyGenerator(2048).keyUse(KeyUse.SIGNATURE) // indicate the intended use of the key .keyID(UUID.randomUUID().toString()) // give the key a unique ID .generate(); JWKSet jwkSet = new JWKSet(RSA.toPublicJWK()); @@ -112,12 +112,8 @@ private String generateBase64EncodedToken(String principal) throws JOSEException JWSSigner signer = new RSASSASigner(RSA); LocalDate now = LocalDate.now(); - JWTClaimsSet claimsSet = new JWTClaimsSet.Builder() - .subject(principal) - .issuer(ISSUER) - .audience(AUDIENCE) - .expirationTime(java.sql.Date.valueOf(now.plusDays(1))) - .build(); + JWTClaimsSet claimsSet = new JWTClaimsSet.Builder().subject(principal).issuer(ISSUER) + .audience(AUDIENCE).expirationTime(java.sql.Date.valueOf(now.plusDays(1))).build(); SignedJWT signedJWT = new SignedJWT( new JWSHeader.Builder(JWSAlgorithm.RS256).keyID(RSA.getKeyID()).build(), claimsSet); @@ -139,8 +135,8 @@ public void testOAuthBearerLogin() throws IOException, JOSEException { byte[] row = Bytes.toBytes("row"); byte[] value = Bytes.toBytes("data"); - User user = User.createUserForTesting(TEST_UTIL.getConfiguration(), "testuser_jwt", - new String[] {}); + User user = + User.createUserForTesting(TEST_UTIL.getConfiguration(), "testuser_jwt", new String[] {}); OAuthBearerTokenUtil.addTokenForUser(user, generateBase64EncodedToken(user.getName()), 0); try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration(), user)) {