Skip to content

Commit d94c472

Browse files
authored
HBASE-26667 Integrate user-experience for hbase-client (#4064)
Signed-off-by: Josh Elser <elserj@apache.org>
1 parent c11f377 commit d94c472

6 files changed

Lines changed: 181 additions & 7 deletions

File tree

hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java

Lines changed: 16 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,6 @@
1919
package org.apache.hadoop.hbase.client;
2020

2121
import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
22-
2322
import java.io.IOException;
2423
import java.lang.reflect.Constructor;
2524
import java.security.PrivilegedExceptionAction;
@@ -30,6 +29,7 @@
3029
import org.apache.hadoop.hbase.HBaseConfiguration;
3130
import org.apache.hadoop.hbase.security.User;
3231
import org.apache.hadoop.hbase.security.UserProvider;
32+
import org.apache.hadoop.hbase.security.token.OAuthBearerTokenUtil;
3333
import org.apache.hadoop.hbase.trace.TraceUtil;
3434
import org.apache.hadoop.hbase.util.FutureUtils;
3535
import org.apache.hadoop.hbase.util.ReflectionUtils;
@@ -70,7 +70,11 @@
7070
@InterfaceAudience.Public
7171
public class ConnectionFactory {
7272

73-
public static final String HBASE_CLIENT_ASYNC_CONNECTION_IMPL = "hbase.client.async.connection.impl";
73+
public static final String HBASE_CLIENT_ASYNC_CONNECTION_IMPL =
74+
"hbase.client.async.connection.impl";
75+
76+
/** Environment variable for OAuth Bearer token */
77+
public static final String ENV_OAUTHBEARER_TOKEN = "HBASE_JWT";
7478

7579
/** No public c.tors */
7680
protected ConnectionFactory() {
@@ -214,6 +218,11 @@ public static Connection createConnection(Configuration conf, User user) throws
214218
*/
215219
public static Connection createConnection(Configuration conf, ExecutorService pool,
216220
final User user) throws IOException {
221+
222+
if (System.getenv().containsKey(ENV_OAUTHBEARER_TOKEN)) {
223+
OAuthBearerTokenUtil.addTokenFromEnvironmentVar(user, System.getenv(ENV_OAUTHBEARER_TOKEN));
224+
}
225+
217226
Class<?> clazz = conf.getClass(ConnectionUtils.HBASE_CLIENT_CONNECTION_IMPL,
218227
ConnectionOverAsyncConnection.class, Connection.class);
219228
if (clazz != ConnectionOverAsyncConnection.class) {
@@ -293,6 +302,11 @@ public static CompletableFuture<AsyncConnection> createAsyncConnection(Configura
293302
future.completeExceptionally(new IOException("clusterid came back null"));
294303
return;
295304
}
305+
306+
if (System.getenv().containsKey(ENV_OAUTHBEARER_TOKEN)) {
307+
OAuthBearerTokenUtil.addTokenFromEnvironmentVar(user, System.getenv(ENV_OAUTHBEARER_TOKEN));
308+
}
309+
296310
Class<? extends AsyncConnection> clazz = conf.getClass(HBASE_CLIENT_ASYNC_CONNECTION_IMPL,
297311
AsyncConnectionImpl.class, AsyncConnection.class);
298312
try {

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,7 @@
1717
*/
1818
package org.apache.hadoop.hbase.security.provider;
1919

20-
import static org.apache.hadoop.hbase.security.token.OAuthBearerTokenUtil.TOKEN_KIND;
20+
import static org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerUtils.TOKEN_KIND;
2121
import org.apache.hadoop.security.UserGroupInformation;
2222
import org.apache.yetus.audience.InterfaceAudience;
2323

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

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,7 @@
1717
*/
1818
package org.apache.hadoop.hbase.security.provider;
1919

20-
import static org.apache.hadoop.hbase.security.token.OAuthBearerTokenUtil.TOKEN_KIND;
20+
import static org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerUtils.TOKEN_KIND;
2121
import java.util.Collection;
2222
import java.util.Optional;
2323
import org.apache.hadoop.conf.Configuration;
@@ -35,8 +35,7 @@ public class OAuthBearerSaslProviderSelector extends BuiltInProviderSelector {
3535

3636
private static final Logger LOG = LoggerFactory.getLogger(OAuthBearerSaslProviderSelector.class);
3737

38-
private final Text OAUTHBEARER_TOKEN_KIND_TEXT =
39-
new Text(TOKEN_KIND);
38+
private final Text OAUTHBEARER_TOKEN_KIND_TEXT = new Text(TOKEN_KIND);
4039
private OAuthBearerSaslClientAuthenticationProvider oauthbearer;
4140

4241
@Override public void configure(Configuration conf,

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

Lines changed: 46 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -18,11 +18,19 @@
1818
*/
1919
package org.apache.hadoop.hbase.security.token;
2020

21+
import static org.apache.hadoop.hbase.client.ConnectionFactory.ENV_OAUTHBEARER_TOKEN;
22+
import static org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerUtils.TOKEN_KIND;
2123
import java.security.AccessController;
2224
import java.security.PrivilegedAction;
25+
import java.time.Instant;
26+
import java.time.ZonedDateTime;
27+
import java.time.format.DateTimeParseException;
28+
import java.util.Optional;
2329
import javax.security.auth.Subject;
30+
import org.apache.commons.lang3.StringUtils;
2431
import org.apache.hadoop.hbase.security.User;
2532
import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerToken;
33+
import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerUtils;
2634
import org.apache.hadoop.hbase.security.oauthbearer.internals.OAuthBearerSaslClientProvider;
2735
import org.apache.hadoop.io.Text;
2836
import org.apache.hadoop.security.token.Token;
@@ -36,7 +44,6 @@
3644
@InterfaceAudience.Public
3745
public final class OAuthBearerTokenUtil {
3846
private static final Logger LOG = LoggerFactory.getLogger(OAuthBearerTokenUtil.class);
39-
public static final String TOKEN_KIND = "JWT_AUTH_TOKEN";
4047

4148
static {
4249
OAuthBearerSaslClientProvider.initialize(); // not part of public API
@@ -68,8 +75,46 @@ public static void addTokenForUser(User user, String encodedToken, long lifetime
6875
}
6976
};
7077
subject.getPrivateCredentials().add(jwt);
78+
if (LOG.isDebugEnabled()) {
79+
LOG.debug("JWT token has been added to user credentials with expiry {}",
80+
lifetimeMs == 0 ? "0" : Instant.ofEpochMilli(lifetimeMs).toString());
81+
}
7182
return null;
7283
}
7384
});
7485
}
86+
87+
/**
88+
* Check whether an OAuth Beaerer token is provided in environment variable HBASE_JWT.
89+
* Parse and add it to user private credentials, but only if another token is not already present.
90+
*/
91+
public static void addTokenFromEnvironmentVar(User user, String token) {
92+
Optional<Token<?>> oauthBearerToken = user.getTokens().stream()
93+
.filter((t) -> new Text(OAuthBearerUtils.TOKEN_KIND).equals(t.getKind()))
94+
.findFirst();
95+
96+
if (oauthBearerToken.isPresent()) {
97+
LOG.warn("Ignoring OAuth Bearer token in " + ENV_OAUTHBEARER_TOKEN + " environment "
98+
+ "variable, because another token is already present");
99+
return;
100+
}
101+
102+
String[] tokens = token.split(",");
103+
if (StringUtils.isEmpty(tokens[0])) {
104+
return;
105+
}
106+
long lifetimeMs = 0;
107+
if (tokens.length > 1) {
108+
try {
109+
ZonedDateTime lifetime = ZonedDateTime.parse(tokens[1]);
110+
lifetimeMs = lifetime.toInstant().toEpochMilli();
111+
} catch (DateTimeParseException e) {
112+
throw new RuntimeException("Unable to parse JWT expiry: " + tokens[1], e);
113+
}
114+
} else {
115+
throw new RuntimeException("Expiry information of JWT is missing");
116+
}
117+
118+
addTokenForUser(user, tokens[0], lifetimeMs);
119+
}
75120
}
Lines changed: 115 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,115 @@
1+
/**
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
package org.apache.hadoop.hbase.security.token;
19+
20+
import static org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerUtils.TOKEN_KIND;
21+
import static org.junit.Assert.assertEquals;
22+
import static org.junit.Assert.assertFalse;
23+
import static org.junit.Assert.assertTrue;
24+
import java.security.AccessController;
25+
import java.security.PrivilegedAction;
26+
import java.time.Instant;
27+
import java.util.Optional;
28+
import java.util.Set;
29+
import javax.security.auth.Subject;
30+
import org.apache.hadoop.hbase.HBaseConfiguration;
31+
import org.apache.hadoop.hbase.security.User;
32+
import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerToken;
33+
import org.apache.hadoop.io.Text;
34+
import org.apache.hadoop.security.token.Token;
35+
import org.junit.Test;
36+
37+
public class TestOAuthBearerTokenUtil {
38+
39+
@Test
40+
public void testAddTokenFromEnvVar() {
41+
// Arrange
42+
User user = User.createUserForTesting(HBaseConfiguration.create(), "testuser", new String[] {});
43+
String testToken = "some_base64_encoded_stuff,2022-01-25T16:59:48.614000+00:00";
44+
45+
// Act
46+
OAuthBearerTokenUtil.addTokenFromEnvironmentVar(user, testToken);
47+
48+
// Assert
49+
Optional<Token<?>> oauthBearerToken = user.getTokens().stream()
50+
.filter((t) -> new Text(TOKEN_KIND).equals(t.getKind()))
51+
.findFirst();
52+
assertTrue("Token cannot be found in user tokens", oauthBearerToken.isPresent());
53+
user.runAs(new PrivilegedAction<Object>() {
54+
@Override public Object run() {
55+
Subject subject = Subject.getSubject(AccessController.getContext());
56+
Set<OAuthBearerToken> tokens = subject.getPrivateCredentials(OAuthBearerToken.class);
57+
assertFalse("Token cannot be found in subject's private credentials", tokens.isEmpty());
58+
OAuthBearerToken jwt = tokens.iterator().next();
59+
assertEquals("Invalid encoded JWT value", "some_base64_encoded_stuff", jwt.value());
60+
assertEquals("Invalid JWT expiry", "2022-01-25T16:59:48.614Z",
61+
Instant.ofEpochMilli(jwt.lifetimeMs()).toString());
62+
return null;
63+
}
64+
});
65+
}
66+
67+
@Test(expected = RuntimeException.class)
68+
public void testAddTokenEnvVarWithoutExpiry() {
69+
// Arrange
70+
User user = User.createUserForTesting(new HBaseConfiguration(), "testuser", new String[] {});
71+
String testToken = "some_base64_encoded_stuff";
72+
73+
// Act
74+
OAuthBearerTokenUtil.addTokenFromEnvironmentVar(user, testToken);
75+
76+
// Assert
77+
}
78+
79+
@Test(expected = RuntimeException.class)
80+
public void testAddTokenEnvVarWithInvalidExpiry() {
81+
// Arrange
82+
User user = User.createUserForTesting(new HBaseConfiguration(), "testuser", new String[] {});
83+
String testToken = "some_base64_encoded_stuff,foobarblahblah328742";
84+
85+
// Act
86+
OAuthBearerTokenUtil.addTokenFromEnvironmentVar(user, testToken);
87+
88+
// Assert
89+
}
90+
91+
@Test
92+
public void testAddTokenEnvVarTokenAlreadyPresent() {
93+
// Arrange
94+
User user = User.createUserForTesting(new HBaseConfiguration(), "testuser", new String[] {});
95+
user.addToken(new Token<>(null, null, new Text(TOKEN_KIND), null));
96+
String testToken = "some_base64_encoded_stuff,foobarblahblah328742";
97+
98+
// Act
99+
OAuthBearerTokenUtil.addTokenFromEnvironmentVar(user, testToken);
100+
101+
// Assert
102+
long numberOfTokens = user.getTokens().stream()
103+
.filter((t) -> new Text(TOKEN_KIND).equals(t.getKind()))
104+
.count();
105+
assertEquals("Invalid number of tokens on User", 1, numberOfTokens);
106+
user.runAs(new PrivilegedAction<Object>() {
107+
@Override public Object run() {
108+
Subject subject = Subject.getSubject(AccessController.getContext());
109+
Set<OAuthBearerToken> tokens = subject.getPrivateCredentials(OAuthBearerToken.class);
110+
assertTrue("Token should not have been added to subject's credentials", tokens.isEmpty());
111+
return null;
112+
}
113+
});
114+
}
115+
}

hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerUtils.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@
2525
@InterfaceAudience.Private
2626
public final class OAuthBearerUtils {
2727
public static final String OAUTHBEARER_MECHANISM = "OAUTHBEARER";
28+
public static final String TOKEN_KIND = "HBASE_JWT_TOKEN";
2829

2930
/**
3031
* Verifies configuration for OAuth Bearer authentication mechanism.

0 commit comments

Comments
 (0)