HBASE-26667 Integrate user-experience for hbase-client (#4064)
Signed-off-by: Josh Elser <elserj@apache.org>
This commit is contained in:
parent
c11f3779c7
commit
d94c472ef5
|
@ -19,7 +19,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;
|
||||
|
@ -30,6 +29,7 @@ import org.apache.hadoop.hbase.AuthUtil;
|
|||
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;
|
||||
|
@ -70,7 +70,11 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||
@InterfaceAudience.Public
|
||||
public class ConnectionFactory {
|
||||
|
||||
public static final String HBASE_CLIENT_ASYNC_CONNECTION_IMPL = "hbase.client.async.connection.impl";
|
||||
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() {
|
||||
|
@ -214,6 +218,11 @@ public class ConnectionFactory {
|
|||
*/
|
||||
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) {
|
||||
|
@ -293,6 +302,11 @@ public class ConnectionFactory {
|
|||
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<? extends AsyncConnection> clazz = conf.getClass(HBASE_CLIENT_ASYNC_CONNECTION_IMPL,
|
||||
AsyncConnectionImpl.class, AsyncConnection.class);
|
||||
try {
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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 @@ import org.slf4j.LoggerFactory;
|
|||
@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 final class OAuthBearerTokenUtil {
|
|||
}
|
||||
};
|
||||
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<Token<?>> 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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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<Token<?>> 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<Object>() {
|
||||
@Override public Object run() {
|
||||
Subject subject = Subject.getSubject(AccessController.getContext());
|
||||
Set<OAuthBearerToken> 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<Object>() {
|
||||
@Override public Object run() {
|
||||
Subject subject = Subject.getSubject(AccessController.getContext());
|
||||
Set<OAuthBearerToken> tokens = subject.getPrivateCredentials(OAuthBearerToken.class);
|
||||
assertTrue("Token should not have been added to subject's credentials", tokens.isEmpty());
|
||||
return null;
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
|
@ -25,6 +25,7 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||
@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.
|
||||
|
|
Loading…
Reference in New Issue