Skip to content

Commit 047bbfd

Browse files
srdosaintstack
authored andcommitted
HBase-22027: Split non-MR related parts of TokenUtil off into a ClientTokenUtil, and move ClientTokenUtil to hbase-client (#649)
1 parent f1d3b54 commit 047bbfd

File tree

7 files changed

+246
-138
lines changed

7 files changed

+246
-138
lines changed
Lines changed: 168 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,168 @@
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+
19+
package org.apache.hadoop.hbase.security.token;
20+
21+
import com.google.protobuf.ByteString;
22+
import com.google.protobuf.ServiceException;
23+
import java.io.IOException;
24+
import java.lang.reflect.UndeclaredThrowableException;
25+
import java.security.PrivilegedExceptionAction;
26+
import org.apache.hadoop.hbase.HConstants;
27+
import org.apache.hadoop.hbase.TableName;
28+
import org.apache.hadoop.hbase.client.Connection;
29+
import org.apache.hadoop.hbase.client.Table;
30+
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
31+
import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
32+
import org.apache.hadoop.hbase.security.User;
33+
import org.apache.hadoop.io.Text;
34+
import org.apache.hadoop.security.token.Token;
35+
import org.apache.yetus.audience.InterfaceAudience;
36+
import org.slf4j.Logger;
37+
import org.slf4j.LoggerFactory;
38+
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
39+
40+
/**
41+
* Utility methods for obtaining authentication tokens, that do not require hbase-server.
42+
*/
43+
@InterfaceAudience.Public
44+
public final class ClientTokenUtil {
45+
private static final Logger LOG = LoggerFactory.getLogger(ClientTokenUtil.class);
46+
47+
// Set in TestClientTokenUtil via reflection
48+
private static ServiceException injectedException;
49+
50+
private ClientTokenUtil() {}
51+
52+
private static void injectFault() throws ServiceException {
53+
if (injectedException != null) {
54+
throw injectedException;
55+
}
56+
}
57+
58+
/**
59+
* Obtain and return an authentication token for the current user.
60+
* @param conn The HBase cluster connection
61+
* @throws IOException if a remote error or serialization problem occurs.
62+
* @return the authentication token instance
63+
*/
64+
@InterfaceAudience.Private
65+
public static Token<AuthenticationTokenIdentifier> obtainToken(
66+
Connection conn) throws IOException {
67+
Table meta = null;
68+
try {
69+
injectFault();
70+
71+
meta = conn.getTable(TableName.META_TABLE_NAME);
72+
CoprocessorRpcChannel rpcChannel = meta.coprocessorService(
73+
HConstants.EMPTY_START_ROW);
74+
AuthenticationProtos.AuthenticationService.BlockingInterface service =
75+
AuthenticationProtos.AuthenticationService.newBlockingStub(rpcChannel);
76+
AuthenticationProtos.GetAuthenticationTokenResponse response =
77+
service.getAuthenticationToken(null,
78+
AuthenticationProtos.GetAuthenticationTokenRequest.getDefaultInstance());
79+
80+
return toToken(response.getToken());
81+
} catch (ServiceException se) {
82+
throw ProtobufUtil.handleRemoteException(se);
83+
} finally {
84+
if (meta != null) {
85+
meta.close();
86+
}
87+
}
88+
}
89+
90+
/**
91+
* Converts a Token instance (with embedded identifier) to the protobuf representation.
92+
*
93+
* @param token the Token instance to copy
94+
* @return the protobuf Token message
95+
*/
96+
@InterfaceAudience.Private
97+
static AuthenticationProtos.Token toToken(Token<AuthenticationTokenIdentifier> token) {
98+
AuthenticationProtos.Token.Builder builder = AuthenticationProtos.Token.newBuilder();
99+
builder.setIdentifier(ByteString.copyFrom(token.getIdentifier()));
100+
builder.setPassword(ByteString.copyFrom(token.getPassword()));
101+
if (token.getService() != null) {
102+
builder.setService(ByteString.copyFromUtf8(token.getService().toString()));
103+
}
104+
return builder.build();
105+
}
106+
107+
/**
108+
* Converts a protobuf Token message back into a Token instance.
109+
*
110+
* @param proto the protobuf Token message
111+
* @return the Token instance
112+
*/
113+
@InterfaceAudience.Private
114+
static Token<AuthenticationTokenIdentifier> toToken(AuthenticationProtos.Token proto) {
115+
return new Token<>(
116+
proto.hasIdentifier() ? proto.getIdentifier().toByteArray() : null,
117+
proto.hasPassword() ? proto.getPassword().toByteArray() : null,
118+
AuthenticationTokenIdentifier.AUTH_TOKEN_TYPE,
119+
proto.hasService() ? new Text(proto.getService().toStringUtf8()) : null);
120+
}
121+
122+
/**
123+
* Obtain and return an authentication token for the given user.
124+
* @param conn The HBase cluster connection
125+
* @param user The user to obtain a token for
126+
* @return the authentication token instance
127+
*/
128+
@InterfaceAudience.Private
129+
static Token<AuthenticationTokenIdentifier> obtainToken(
130+
final Connection conn, User user) throws IOException, InterruptedException {
131+
return user.runAs(new PrivilegedExceptionAction<Token<AuthenticationTokenIdentifier>>() {
132+
@Override
133+
public Token<AuthenticationTokenIdentifier> run() throws Exception {
134+
return obtainToken(conn);
135+
}
136+
});
137+
}
138+
139+
/**
140+
* Obtain an authentication token for the given user and add it to the
141+
* user's credentials.
142+
* @param conn The HBase cluster connection
143+
* @param user The user for whom to obtain the token
144+
* @throws IOException If making a remote call to the authentication service fails
145+
* @throws InterruptedException If executing as the given user is interrupted
146+
*/
147+
public static void obtainAndCacheToken(final Connection conn,
148+
User user)
149+
throws IOException, InterruptedException {
150+
try {
151+
Token<AuthenticationTokenIdentifier> token = obtainToken(conn, user);
152+
153+
if (token == null) {
154+
throw new IOException("No token returned for user " + user.getName());
155+
}
156+
if (LOG.isDebugEnabled()) {
157+
LOG.debug("Obtained token " + token.getKind().toString() + " for user " +
158+
user.getName());
159+
}
160+
user.addToken(token);
161+
} catch (IOException | InterruptedException | RuntimeException e) {
162+
throw e;
163+
} catch (Exception e) {
164+
throw new UndeclaredThrowableException(e,
165+
"Unexpected exception obtaining token for user " + user.getName());
166+
}
167+
}
168+
}

hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenUtil.java renamed to hbase-client/src/test/java/org/apache/hadoop/hbase/security/token/TestClientTokenUtil.java

Lines changed: 25 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -20,45 +20,55 @@
2020
import static org.junit.Assert.assertFalse;
2121
import static org.junit.Assert.fail;
2222

23+
import java.io.IOException;
2324
import java.lang.reflect.Field;
24-
import java.lang.reflect.InvocationTargetException;
2525
import java.net.URL;
2626
import java.net.URLClassLoader;
27-
2827
import org.apache.hadoop.hbase.HBaseClassTestRule;
2928
import org.apache.hadoop.hbase.client.Connection;
3029
import org.apache.hadoop.hbase.testclassification.SmallTests;
30+
import org.junit.After;
31+
import org.junit.Before;
3132
import org.junit.ClassRule;
3233
import org.junit.Test;
3334
import org.junit.experimental.categories.Category;
34-
35+
import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
3536
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
3637

3738
@Category(SmallTests.class)
38-
public class TestTokenUtil {
39+
public class TestClientTokenUtil {
40+
3941
@ClassRule
4042
public static final HBaseClassTestRule CLASS_RULE =
41-
HBaseClassTestRule.forClass(TestTokenUtil.class);
43+
HBaseClassTestRule.forClass(TestClientTokenUtil.class);
4244

43-
@Test
44-
public void testObtainToken() throws Exception {
45+
private URLClassLoader cl;
46+
47+
@Before
48+
public void setUp() {
4549
URL urlPU = ProtobufUtil.class.getProtectionDomain().getCodeSource().getLocation();
46-
URL urlTU = TokenUtil.class.getProtectionDomain().getCodeSource().getLocation();
50+
URL urlCTU = ClientTokenUtil.class.getProtectionDomain().getCodeSource().getLocation();
51+
cl = new URLClassLoader(new URL[] { urlPU, urlCTU }, getClass().getClassLoader());
52+
}
4753

48-
ClassLoader cl = new URLClassLoader(new URL[] { urlPU, urlTU }, getClass().getClassLoader());
54+
@After
55+
public void tearDown() throws IOException {
56+
Closeables.close(cl, true);
57+
}
4958

59+
@Test
60+
public void testObtainToken() throws Exception {
5061
Throwable injected = new com.google.protobuf.ServiceException("injected");
5162

52-
Class<?> tokenUtil = cl.loadClass(TokenUtil.class.getCanonicalName());
53-
Field shouldInjectFault = tokenUtil.getDeclaredField("injectedException");
63+
Class<?> clientTokenUtil = cl.loadClass(ClientTokenUtil.class.getCanonicalName());
64+
Field shouldInjectFault = clientTokenUtil.getDeclaredField("injectedException");
5465
shouldInjectFault.setAccessible(true);
5566
shouldInjectFault.set(null, injected);
5667

5768
try {
58-
tokenUtil.getMethod("obtainToken", Connection.class)
59-
.invoke(null, new Object[] { null });
69+
ClientTokenUtil.obtainToken((Connection)null);
6070
fail("Should have injected exception.");
61-
} catch (InvocationTargetException e) {
71+
} catch (IOException e) {
6272
Throwable t = e;
6373
boolean serviceExceptionFound = false;
6474
while ((t = t.getCause()) != null) {
@@ -73,8 +83,7 @@ public void testObtainToken() throws Exception {
7383
}
7484

7585
Boolean loaded = (Boolean) cl.loadClass(ProtobufUtil.class.getCanonicalName())
76-
.getDeclaredMethod("isClassLoaderLoaded")
77-
.invoke(null);
86+
.getDeclaredMethod("isClassLoaderLoaded").invoke(null);
7887
assertFalse("Should not have loaded DynamicClassLoader", loaded);
7988
}
8089
}

hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -43,8 +43,9 @@
4343
import org.apache.hadoop.hbase.regionserver.HRegion.BulkLoadListener;
4444
import org.apache.hadoop.hbase.security.User;
4545
import org.apache.hadoop.hbase.security.UserProvider;
46+
import org.apache.hadoop.hbase.security.token.AuthenticationTokenIdentifier;
47+
import org.apache.hadoop.hbase.security.token.ClientTokenUtil;
4648
import org.apache.hadoop.hbase.security.token.FsDelegationToken;
47-
import org.apache.hadoop.hbase.security.token.TokenUtil;
4849
import org.apache.hadoop.hbase.util.Bytes;
4950
import org.apache.hadoop.hbase.util.FSHDFSUtils;
5051
import org.apache.hadoop.hbase.util.FSUtils;
@@ -229,7 +230,7 @@ public Map<byte[], List<Path>> secureBulkLoadHFiles(final HRegion region,
229230
final UserGroupInformation ugi = user.getUGI();
230231
if (userProvider.isHadoopSecurityEnabled()) {
231232
try {
232-
Token tok = TokenUtil.obtainToken(conn);
233+
Token<AuthenticationTokenIdentifier> tok = ClientTokenUtil.obtainToken(conn);
233234
if (tok != null) {
234235
boolean b = ugi.addToken(tok);
235236
LOG.debug("token added " + tok + " for user " + ugi + " return=" + b);

hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenProvider.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -129,7 +129,7 @@ public void getAuthenticationToken(RpcController controller,
129129

130130
Token<AuthenticationTokenIdentifier> token =
131131
secretManager.generateToken(currentUser.getName());
132-
response.setToken(TokenUtil.toToken(token)).build();
132+
response.setToken(ClientTokenUtil.toToken(token)).build();
133133
} catch (IOException ioe) {
134134
CoprocessorRpcUtils.setControllerException(controller, ioe);
135135
}

0 commit comments

Comments
 (0)