Skip to content

Commit

Permalink
HBASE-26655. Implement new OAuth Bearer SASL plugin
Browse files Browse the repository at this point in the history
Signed-off-by: Josh Elser <[email protected]>
  • Loading branch information
anmolnar authored and petersomogyi committed Mar 2, 2022
1 parent e85bf41 commit 783a05d
Show file tree
Hide file tree
Showing 38 changed files with 3,394 additions and 5 deletions.
Original file line number Diff line number Diff line change
@@ -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
* <a href="https://tools.ietf.org/html/rfc6749#section-5.2">RFC 6749: The OAuth
* 2.0 Authorization Framework</a>. Callback handlers should communicate other
* problems by raising an {@code IOException}.
* <p>
* 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
* <a href="https://tools.ietf.org/html/rfc6749#section-5.2">RFC 6749: The OAuth
* 2.0 Authorization Framework</a>.
*
* @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
* <a href="https://tools.ietf.org/html/rfc6749#section-5.2">RFC 6749: The OAuth
* 2.0 Authorization Framework</a>.
*
* @return the (potentially null) error description
*/
public String errorDescription() {
return errorDescription;
}

/**
* Return the (potentially null) error URI as per
* <a href="https://tools.ietf.org/html/rfc6749#section-5.2">RFC 6749: The OAuth
* 2.0 Authorization Framework</a>.
*
* @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
* <a href="https://tools.ietf.org/html/rfc6749#section-5.2">RFC 6749: The OAuth
* 2.0 Authorization Framework</a>. 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;
}
}
Original file line number Diff line number Diff line change
@@ -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 <a href="https://tools.ietf.org/html/rfc6750#section-2.1">RFC 6750 Section 2.1</a>
*
* 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<String, ?> 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<String, ?> props) {
return OAuthBearerUtils.mechanismNamesCompatibleWithPolicy(props);
}
}
}
Original file line number Diff line number Diff line change
@@ -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());
}
}
Original file line number Diff line number Diff line change
@@ -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;
}
}
Loading

0 comments on commit 783a05d

Please sign in to comment.