Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ public AccessDeniedException() {
}

public AccessDeniedException(Class<?> clazz, String s) {
super( "AccessDenied [" + clazz.getName() + "]: " + s);
super("AccessDenied [" + clazz.getName() + "]: " + s);
}

public AccessDeniedException(String s) {
Expand Down
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;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,161 @@
/*
* 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.OAUTHBEARER_MECHANISM;
import java.io.IOException;
import java.net.InetAddress;
import java.security.AccessController;
import java.util.Collections;
import java.util.Comparator;
import java.util.Date;
import java.util.Map;
import java.util.Set;
import java.util.SortedSet;
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.auth.SaslExtensions;
import org.apache.hadoop.hbase.security.auth.SaslExtensionsCallback;
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<? extends TokenIdentifier> token,
boolean fallbackAllowed,
Map<String, String> 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<String, String> 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 if (callback instanceof SaslExtensionsCallback) {
handleCallback((SaslExtensionsCallback) callback,
Subject.getSubject(AccessController.getContext()));
} 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<OAuthBearerToken> privateCredentials = subject != null
? subject.getPrivateCredentials(OAuthBearerToken.class)
: Collections.emptySet();
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");
callback.token(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.
*/
SortedSet<OAuthBearerToken> sortedByLifetime =
new TreeSet<>(
new Comparator<OAuthBearerToken>() {
@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(), new Date(sortedByLifetime.first().lifetimeMs()),
new Date(sortedByLifetime.last().lifetimeMs()));
}
callback.token(sortedByLifetime.last());
}
}

/**
* Attaches the first {@link SaslExtensions} found in the public credentials of the Subject
*/
private static void handleCallback(SaslExtensionsCallback extensionsCallback, Subject subject) {
if (subject != null && !subject.getPublicCredentials(SaslExtensions.class).isEmpty()) {
SaslExtensions extensions =
subject.getPublicCredentials(SaslExtensions.class).iterator().next();
extensionsCallback.extensions(extensions);
}
}
}

@Override
public RPCProtos.UserInformation getUserInfo(User user) {
// Don't send user for token auth. Copied from RpcConnection.
return null;
}
}
Original file line number Diff line number Diff line change
@@ -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<SaslClientAuthenticationProvider> 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<SaslClientAuthenticationProvider, Token<? extends TokenIdentifier>> selectProvider(
String clusterId, User user) {
Pair<SaslClientAuthenticationProvider, Token<? extends TokenIdentifier>> pair =
super.selectProvider(clusterId, user);

Optional<Token<?>> 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;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,109 @@
/*
* 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.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.security.token.OAuthBearerTokenUtil;
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<Void>) () -> {
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<Void>) () -> {
final int maxTokens = 4;
final Set<Object> 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());
}
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Would be nice to then add a new PrivateCredential with a lesser lifetimeMs, just to make sure that the sorting is working properly (since you test the life times generated in sorted order, 1, 2, 3, 4).

return null;
});
}

private static OAuthBearerSaslClientAuthenticationProvider.OAuthBearerSaslClientCallbackHandler
createCallbackHandler() {
OAuthBearerSaslClientAuthenticationProvider.OAuthBearerSaslClientCallbackHandler handler =
new OAuthBearerSaslClientAuthenticationProvider.OAuthBearerSaslClientCallbackHandler();
handler.configure(new Configuration(), OAuthBearerTokenUtil.OAUTHBEARER_MECHANISM,
Collections.emptyMap());
return handler;
}
}
Loading