HBase-22027: Split non-MR related parts of TokenUtil off into a Clien… (#361)

* HBase-22027: Split non-MR related parts of TokenUtil off into a ClientTokenUtil, and move ClientTokenUtil to hbase-client

* Replace uses of deprecated TokenUtil methods with ClientTokenUtil methods. Make methods that don't need to be public package-private

* Don't use reflection where not necessary in TestClientTokenUtil

Signed-off-by: Sean Busbey <busbey@apache.org>
Signed-off-by: stack <stack@apache.org>
This commit is contained in:
Stig Døssing 2019-09-19 21:42:04 +02:00 committed by Michael Stack
parent 1fec3d1dfd
commit 0ec0a9403f
7 changed files with 267 additions and 156 deletions

View File

@ -0,0 +1,201 @@
/*
* 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 com.google.protobuf.ByteString;
import com.google.protobuf.ServiceException;
import java.io.IOException;
import java.lang.reflect.UndeclaredThrowableException;
import java.security.PrivilegedExceptionAction;
import java.util.concurrent.CompletableFuture;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.AsyncConnection;
import org.apache.hadoop.hbase.client.AsyncTable;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.token.Token;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
/**
* Utility methods for obtaining authentication tokens, that do not require hbase-server.
*/
@InterfaceAudience.Public
public final class ClientTokenUtil {
private static final Logger LOG = LoggerFactory.getLogger(ClientTokenUtil.class);
// Set in TestClientTokenUtil via reflection
private static ServiceException injectedException;
private ClientTokenUtil() {}
private static void injectFault() throws ServiceException {
if (injectedException != null) {
throw injectedException;
}
}
/**
* Obtain and return an authentication token for the current user.
* @param conn The async HBase cluster connection
* @return the authentication token instance, wrapped by a {@link CompletableFuture}.
*/
@InterfaceAudience.Private
public static CompletableFuture<Token<AuthenticationTokenIdentifier>> obtainToken(
AsyncConnection conn) {
CompletableFuture<Token<AuthenticationTokenIdentifier>> future = new CompletableFuture<>();
if (injectedException != null) {
future.completeExceptionally(injectedException);
return future;
}
AsyncTable<?> table = conn.getTable(TableName.META_TABLE_NAME);
table.<AuthenticationProtos.AuthenticationService.Interface,
AuthenticationProtos.GetAuthenticationTokenResponse> coprocessorService(
AuthenticationProtos.AuthenticationService::newStub,
(s, c, r) -> s.getAuthenticationToken(c,
AuthenticationProtos.GetAuthenticationTokenRequest.getDefaultInstance(), r),
HConstants.EMPTY_START_ROW).whenComplete((resp, error) -> {
if (error != null) {
future.completeExceptionally(ProtobufUtil.handleRemoteException(error));
} else {
future.complete(toToken(resp.getToken()));
}
});
return future;
}
/**
* Obtain and return an authentication token for the current user.
* @param conn The HBase cluster connection
* @throws IOException if a remote error or serialization problem occurs.
* @return the authentication token instance
*/
@InterfaceAudience.Private
static Token<AuthenticationTokenIdentifier> obtainToken(
Connection conn) throws IOException {
Table meta = null;
try {
injectFault();
meta = conn.getTable(TableName.META_TABLE_NAME);
CoprocessorRpcChannel rpcChannel = meta.coprocessorService(
HConstants.EMPTY_START_ROW);
AuthenticationProtos.AuthenticationService.BlockingInterface service =
AuthenticationProtos.AuthenticationService.newBlockingStub(rpcChannel);
AuthenticationProtos.GetAuthenticationTokenResponse response =
service.getAuthenticationToken(null,
AuthenticationProtos.GetAuthenticationTokenRequest.getDefaultInstance());
return toToken(response.getToken());
} catch (ServiceException se) {
throw ProtobufUtil.handleRemoteException(se);
} finally {
if (meta != null) {
meta.close();
}
}
}
/**
* Converts a Token instance (with embedded identifier) to the protobuf representation.
*
* @param token the Token instance to copy
* @return the protobuf Token message
*/
@InterfaceAudience.Private
static AuthenticationProtos.Token toToken(Token<AuthenticationTokenIdentifier> token) {
AuthenticationProtos.Token.Builder builder = AuthenticationProtos.Token.newBuilder();
builder.setIdentifier(ByteString.copyFrom(token.getIdentifier()));
builder.setPassword(ByteString.copyFrom(token.getPassword()));
if (token.getService() != null) {
builder.setService(ByteString.copyFromUtf8(token.getService().toString()));
}
return builder.build();
}
/**
* Converts a protobuf Token message back into a Token instance.
*
* @param proto the protobuf Token message
* @return the Token instance
*/
@InterfaceAudience.Private
static Token<AuthenticationTokenIdentifier> toToken(AuthenticationProtos.Token proto) {
return new Token<>(
proto.hasIdentifier() ? proto.getIdentifier().toByteArray() : null,
proto.hasPassword() ? proto.getPassword().toByteArray() : null,
AuthenticationTokenIdentifier.AUTH_TOKEN_TYPE,
proto.hasService() ? new Text(proto.getService().toStringUtf8()) : null);
}
/**
* Obtain and return an authentication token for the given user.
* @param conn The HBase cluster connection
* @param user The user to obtain a token for
* @return the authentication token instance
*/
@InterfaceAudience.Private
static Token<AuthenticationTokenIdentifier> obtainToken(
final Connection conn, User user) throws IOException, InterruptedException {
return user.runAs(new PrivilegedExceptionAction<Token<AuthenticationTokenIdentifier>>() {
@Override
public Token<AuthenticationTokenIdentifier> run() throws Exception {
return obtainToken(conn);
}
});
}
/**
* Obtain an authentication token for the given user and add it to the
* user's credentials.
* @param conn The HBase cluster connection
* @param user The user for whom to obtain the token
* @throws IOException If making a remote call to the authentication service fails
* @throws InterruptedException If executing as the given user is interrupted
*/
public static void obtainAndCacheToken(final Connection conn,
User user)
throws IOException, InterruptedException {
try {
Token<AuthenticationTokenIdentifier> token = obtainToken(conn, user);
if (token == null) {
throw new IOException("No token returned for user " + user.getName());
}
if (LOG.isDebugEnabled()) {
LOG.debug("Obtained token " + token.getKind().toString() + " for user " +
user.getName());
}
user.addToken(token);
} catch (IOException | InterruptedException | RuntimeException e) {
throw e;
} catch (Exception e) {
throw new UndeclaredThrowableException(e,
"Unexpected exception obtaining token for user " + user.getName());
}
}
}

View File

@ -23,7 +23,6 @@ import static org.junit.Assert.fail;
import java.io.IOException; import java.io.IOException;
import java.lang.reflect.Field; import java.lang.reflect.Field;
import java.lang.reflect.InvocationTargetException;
import java.net.URL; import java.net.URL;
import java.net.URLClassLoader; import java.net.URLClassLoader;
import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletableFuture;
@ -43,19 +42,19 @@ import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@Category(SmallTests.class) @Category(SmallTests.class)
public class TestTokenUtil { public class TestClientTokenUtil {
@ClassRule @ClassRule
public static final HBaseClassTestRule CLASS_RULE = public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestTokenUtil.class); HBaseClassTestRule.forClass(TestClientTokenUtil.class);
private URLClassLoader cl; private URLClassLoader cl;
@Before @Before
public void setUp() { public void setUp() {
URL urlPU = ProtobufUtil.class.getProtectionDomain().getCodeSource().getLocation(); URL urlPU = ProtobufUtil.class.getProtectionDomain().getCodeSource().getLocation();
URL urlTU = TokenUtil.class.getProtectionDomain().getCodeSource().getLocation(); URL urlCTU = ClientTokenUtil.class.getProtectionDomain().getCodeSource().getLocation();
cl = new URLClassLoader(new URL[] { urlPU, urlTU }, getClass().getClassLoader()); cl = new URLClassLoader(new URL[] { urlPU, urlCTU }, getClass().getClassLoader());
} }
@After @After
@ -67,15 +66,15 @@ public class TestTokenUtil {
public void testObtainToken() throws Exception { public void testObtainToken() throws Exception {
Throwable injected = new com.google.protobuf.ServiceException("injected"); Throwable injected = new com.google.protobuf.ServiceException("injected");
Class<?> tokenUtil = cl.loadClass(TokenUtil.class.getCanonicalName()); Class<?> clientTokenUtil = cl.loadClass(ClientTokenUtil.class.getCanonicalName());
Field shouldInjectFault = tokenUtil.getDeclaredField("injectedException"); Field shouldInjectFault = clientTokenUtil.getDeclaredField("injectedException");
shouldInjectFault.setAccessible(true); shouldInjectFault.setAccessible(true);
shouldInjectFault.set(null, injected); shouldInjectFault.set(null, injected);
try { try {
tokenUtil.getMethod("obtainToken", Connection.class).invoke(null, new Object[] { null }); ClientTokenUtil.obtainToken((Connection)null);
fail("Should have injected exception."); fail("Should have injected exception.");
} catch (InvocationTargetException e) { } catch (IOException e) {
Throwable t = e; Throwable t = e;
boolean serviceExceptionFound = false; boolean serviceExceptionFound = false;
while ((t = t.getCause()) != null) { while ((t = t.getCause()) != null) {
@ -89,8 +88,7 @@ public class TestTokenUtil {
} }
} }
CompletableFuture<?> future = (CompletableFuture<?>) tokenUtil CompletableFuture<?> future = ClientTokenUtil.obtainToken((AsyncConnection)null);
.getMethod("obtainToken", AsyncConnection.class).invoke(null, new Object[] { null });
try { try {
future.get(); future.get();
fail("Should have injected exception."); fail("Should have injected exception.");

View File

@ -43,8 +43,8 @@ import org.apache.hadoop.hbase.regionserver.HRegion.BulkLoadListener;
import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.security.UserProvider;
import org.apache.hadoop.hbase.security.token.AuthenticationTokenIdentifier; import org.apache.hadoop.hbase.security.token.AuthenticationTokenIdentifier;
import org.apache.hadoop.hbase.security.token.ClientTokenUtil;
import org.apache.hadoop.hbase.security.token.FsDelegationToken; import org.apache.hadoop.hbase.security.token.FsDelegationToken;
import org.apache.hadoop.hbase.security.token.TokenUtil;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSHDFSUtils; import org.apache.hadoop.hbase.util.FSHDFSUtils;
import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.FSUtils;
@ -231,7 +231,7 @@ public class SecureBulkLoadManager {
final UserGroupInformation ugi = user.getUGI(); final UserGroupInformation ugi = user.getUGI();
if (userProvider.isHadoopSecurityEnabled()) { if (userProvider.isHadoopSecurityEnabled()) {
try { try {
Token<AuthenticationTokenIdentifier> tok = TokenUtil.obtainToken(conn).get(); Token<AuthenticationTokenIdentifier> tok = ClientTokenUtil.obtainToken(conn).get();
if (tok != null) { if (tok != null) {
boolean b = ugi.addToken(tok); boolean b = ugi.addToken(tok);
LOG.debug("token added " + tok + " for user " + ugi + " return=" + b); LOG.debug("token added " + tok + " for user " + ugi + " return=" + b);

View File

@ -129,7 +129,7 @@ public class TokenProvider implements AuthenticationProtos.AuthenticationService
Token<AuthenticationTokenIdentifier> token = Token<AuthenticationTokenIdentifier> token =
secretManager.generateToken(currentUser.getName()); secretManager.generateToken(currentUser.getName());
response.setToken(TokenUtil.toToken(token)).build(); response.setToken(ClientTokenUtil.toToken(token)).build();
} catch (IOException ioe) { } catch (IOException ioe) {
CoprocessorRpcUtils.setControllerException(controller, ioe); CoprocessorRpcUtils.setControllerException(controller, ioe);
} }

View File

@ -17,24 +17,13 @@
*/ */
package org.apache.hadoop.hbase.security.token; package org.apache.hadoop.hbase.security.token;
import com.google.protobuf.ByteString;
import com.google.protobuf.ServiceException;
import java.io.IOException; import java.io.IOException;
import java.lang.reflect.UndeclaredThrowableException; import java.lang.reflect.UndeclaredThrowableException;
import java.security.PrivilegedExceptionAction;
import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletableFuture;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.AsyncConnection; import org.apache.hadoop.hbase.client.AsyncConnection;
import org.apache.hadoop.hbase.client.AsyncTable;
import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos; import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos.AuthenticationService;
import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos.GetAuthenticationTokenRequest;
import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos.GetAuthenticationTokenResponse;
import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId; import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
@ -47,7 +36,6 @@ import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
/** /**
* Utility methods for obtaining authentication tokens. * Utility methods for obtaining authentication tokens.
@ -57,103 +45,75 @@ public class TokenUtil {
// This class is referenced indirectly by User out in common; instances are created by reflection // This class is referenced indirectly by User out in common; instances are created by reflection
private static final Logger LOG = LoggerFactory.getLogger(TokenUtil.class); private static final Logger LOG = LoggerFactory.getLogger(TokenUtil.class);
// Set in TestTokenUtil via reflection /**
private static ServiceException injectedException; * See {@link ClientTokenUtil#obtainToken(org.apache.hadoop.hbase.client.AsyncConnection)}.
* @deprecated External users should not use this method. Please post on
private static void injectFault() throws ServiceException { * the HBase dev mailing list if you need this method. Internal
if (injectedException != null) { * HBase code should use {@link ClientTokenUtil} instead.
throw injectedException; */
} @Deprecated
}
/**
* Obtain and return an authentication token for the current user.
* @param conn The async HBase cluster connection
* @return the authentication token instance, wrapped by a {@link CompletableFuture}.
*/
public static CompletableFuture<Token<AuthenticationTokenIdentifier>> obtainToken( public static CompletableFuture<Token<AuthenticationTokenIdentifier>> obtainToken(
AsyncConnection conn) { AsyncConnection conn) {
CompletableFuture<Token<AuthenticationTokenIdentifier>> future = new CompletableFuture<>(); return ClientTokenUtil.obtainToken(conn);
if (injectedException != null) {
future.completeExceptionally(injectedException);
return future;
}
AsyncTable<?> table = conn.getTable(TableName.META_TABLE_NAME);
table.<AuthenticationService.Interface, GetAuthenticationTokenResponse> coprocessorService(
AuthenticationProtos.AuthenticationService::newStub,
(s, c, r) -> s.getAuthenticationToken(c,
AuthenticationProtos.GetAuthenticationTokenRequest.getDefaultInstance(), r),
HConstants.EMPTY_START_ROW).whenComplete((resp, error) -> {
if (error != null) {
future.completeExceptionally(ProtobufUtil.handleRemoteException(error));
} else {
future.complete(toToken(resp.getToken()));
}
});
return future;
} }
/** /**
* Obtain and return an authentication token for the current user. * See {@link ClientTokenUtil#obtainToken(org.apache.hadoop.hbase.client.Connection)}.
* @param conn The HBase cluster connection * @deprecated External users should not use this method. Please post on
* @throws IOException if a remote error or serialization problem occurs. * the HBase dev mailing list if you need this method. Internal
* @return the authentication token instance * HBase code should use {@link ClientTokenUtil} instead.
*/ */
@Deprecated
public static Token<AuthenticationTokenIdentifier> obtainToken(Connection conn) public static Token<AuthenticationTokenIdentifier> obtainToken(Connection conn)
throws IOException { throws IOException {
Table meta = null; return ClientTokenUtil.obtainToken(conn);
try {
injectFault();
meta = conn.getTable(TableName.META_TABLE_NAME);
CoprocessorRpcChannel rpcChannel = meta.coprocessorService(HConstants.EMPTY_START_ROW);
AuthenticationProtos.AuthenticationService.BlockingInterface service =
AuthenticationService.newBlockingStub(rpcChannel);
GetAuthenticationTokenResponse response =
service.getAuthenticationToken(null, GetAuthenticationTokenRequest.getDefaultInstance());
return toToken(response.getToken());
} catch (ServiceException se) {
throw ProtobufUtil.handleRemoteException(se);
} finally {
if (meta != null) {
meta.close();
}
}
} }
/** /**
* Converts a Token instance (with embedded identifier) to the protobuf representation. * See {@link ClientTokenUtil#toToken(org.apache.hadoop.security.token.Token)}.
* * @deprecated External users should not use this method. Please post on
* @param token the Token instance to copy * the HBase dev mailing list if you need this method. Internal
* @return the protobuf Token message * HBase code should use {@link ClientTokenUtil} instead.
*/ */
@Deprecated
public static AuthenticationProtos.Token toToken(Token<AuthenticationTokenIdentifier> token) { public static AuthenticationProtos.Token toToken(Token<AuthenticationTokenIdentifier> token) {
AuthenticationProtos.Token.Builder builder = AuthenticationProtos.Token.newBuilder(); return ClientTokenUtil.toToken(token);
builder.setIdentifier(ByteString.copyFrom(token.getIdentifier()));
builder.setPassword(ByteString.copyFrom(token.getPassword()));
if (token.getService() != null) {
builder.setService(ByteString.copyFromUtf8(token.getService().toString()));
}
return builder.build();
} }
/** /**
* Obtain and return an authentication token for the current user. * See {@link ClientTokenUtil#obtainToken(org.apache.hadoop.hbase.client.Connection,
* @param conn The HBase cluster connection * org.apache.hadoop.hbase.security.User)}.
* @return the authentication token instance * @deprecated External users should not use this method. Please post on
* the HBase dev mailing list if you need this method. Internal
* HBase code should use {@link ClientTokenUtil} instead.
*/ */
@Deprecated
public static Token<AuthenticationTokenIdentifier> obtainToken( public static Token<AuthenticationTokenIdentifier> obtainToken(
final Connection conn, User user) throws IOException, InterruptedException { final Connection conn, User user) throws IOException, InterruptedException {
return user.runAs(new PrivilegedExceptionAction<Token<AuthenticationTokenIdentifier>>() { return ClientTokenUtil.obtainToken(conn, user);
@Override
public Token<AuthenticationTokenIdentifier> run() throws Exception {
return obtainToken(conn);
}
});
} }
/**
* See {@link ClientTokenUtil#obtainAndCacheToken(org.apache.hadoop.hbase.client.Connection,
* org.apache.hadoop.hbase.security.User)}.
*/
public static void obtainAndCacheToken(final Connection conn,
User user)
throws IOException, InterruptedException {
ClientTokenUtil.obtainAndCacheToken(conn, user);
}
/**
* See {@link ClientTokenUtil#toToken(org.apache.hadoop.security.token.Token)}.
* @deprecated External users should not use this method. Please post on
* the HBase dev mailing list if you need this method. Internal
* HBase code should use {@link ClientTokenUtil} instead.
*/
@Deprecated
public static Token<AuthenticationTokenIdentifier> toToken(AuthenticationProtos.Token proto) {
return ClientTokenUtil.toToken(proto);
}
private static Text getClusterId(Token<AuthenticationTokenIdentifier> token) private static Text getClusterId(Token<AuthenticationTokenIdentifier> token)
throws IOException { throws IOException {
@ -161,40 +121,6 @@ public class TokenUtil {
? token.getService() : new Text("default"); ? token.getService() : new Text("default");
} }
/**
* Obtain an authentication token for the given user and add it to the
* user's credentials.
* @param conn The HBase cluster connection
* @param user The user for whom to obtain the token
* @throws IOException If making a remote call to the authentication service fails
* @throws InterruptedException If executing as the given user is interrupted
*/
public static void obtainAndCacheToken(final Connection conn,
User user)
throws IOException, InterruptedException {
try {
Token<AuthenticationTokenIdentifier> token = obtainToken(conn, user);
if (token == null) {
throw new IOException("No token returned for user " + user.getName());
}
if (LOG.isDebugEnabled()) {
LOG.debug("Obtained token " + token.getKind().toString() + " for user " +
user.getName());
}
user.addToken(token);
} catch (IOException ioe) {
throw ioe;
} catch (InterruptedException ie) {
throw ie;
} catch (RuntimeException re) {
throw re;
} catch (Exception e) {
throw new UndeclaredThrowableException(e,
"Unexpected exception obtaining token for user " + user.getName());
}
}
/** /**
* Obtain an authentication token on behalf of the given user and add it to * Obtain an authentication token on behalf of the given user and add it to
* the credentials for the given map reduce job. * the credentials for the given map reduce job.
@ -208,7 +134,7 @@ public class TokenUtil {
User user, Job job) User user, Job job)
throws IOException, InterruptedException { throws IOException, InterruptedException {
try { try {
Token<AuthenticationTokenIdentifier> token = obtainToken(conn, user); Token<AuthenticationTokenIdentifier> token = ClientTokenUtil.obtainToken(conn, user);
if (token == null) { if (token == null) {
throw new IOException("No token returned for user " + user.getName()); throw new IOException("No token returned for user " + user.getName());
@ -243,7 +169,7 @@ public class TokenUtil {
public static void obtainTokenForJob(final Connection conn, final JobConf job, User user) public static void obtainTokenForJob(final Connection conn, final JobConf job, User user)
throws IOException, InterruptedException { throws IOException, InterruptedException {
try { try {
Token<AuthenticationTokenIdentifier> token = obtainToken(conn, user); Token<AuthenticationTokenIdentifier> token = ClientTokenUtil.obtainToken(conn, user);
if (token == null) { if (token == null) {
throw new IOException("No token returned for user " + user.getName()); throw new IOException("No token returned for user " + user.getName());
@ -281,7 +207,7 @@ public class TokenUtil {
Token<AuthenticationTokenIdentifier> token = getAuthToken(conn.getConfiguration(), user); Token<AuthenticationTokenIdentifier> token = getAuthToken(conn.getConfiguration(), user);
if (token == null) { if (token == null) {
token = obtainToken(conn, user); token = ClientTokenUtil.obtainToken(conn, user);
} }
job.getCredentials().addToken(token.getService(), token); job.getCredentials().addToken(token.getService(), token);
} }
@ -300,7 +226,7 @@ public class TokenUtil {
throws IOException, InterruptedException { throws IOException, InterruptedException {
Token<AuthenticationTokenIdentifier> token = getAuthToken(conn.getConfiguration(), user); Token<AuthenticationTokenIdentifier> token = getAuthToken(conn.getConfiguration(), user);
if (token == null) { if (token == null) {
token = obtainToken(conn, user); token = ClientTokenUtil.obtainToken(conn, user);
} }
job.getCredentials().addToken(token.getService(), token); job.getCredentials().addToken(token.getService(), token);
} }
@ -319,7 +245,7 @@ public class TokenUtil {
throws IOException, InterruptedException { throws IOException, InterruptedException {
Token<AuthenticationTokenIdentifier> token = getAuthToken(conn.getConfiguration(), user); Token<AuthenticationTokenIdentifier> token = getAuthToken(conn.getConfiguration(), user);
if (token == null) { if (token == null) {
token = obtainToken(conn, user); token = ClientTokenUtil.obtainToken(conn, user);
user.getUGI().addToken(token.getService(), token); user.getUGI().addToken(token.getService(), token);
return true; return true;
} }
@ -345,18 +271,4 @@ public class TokenUtil {
zkw.close(); zkw.close();
} }
} }
/**
* Converts a protobuf Token message back into a Token instance.
*
* @param proto the protobuf Token message
* @return the Token instance
*/
public static Token<AuthenticationTokenIdentifier> toToken(AuthenticationProtos.Token proto) {
return new Token<>(
proto.hasIdentifier() ? proto.getIdentifier().toByteArray() : null,
proto.hasPassword() ? proto.getPassword().toByteArray() : null,
AuthenticationTokenIdentifier.AUTH_TOKEN_TYPE,
proto.hasService() ? new Text(proto.getService().toStringUtf8()) : null);
}
} }

View File

@ -67,7 +67,7 @@ public class TestDelegationTokenWithEncryption extends SecureTestCluster {
TEST_UTIL.getConfiguration().set("hbase.rpc.protection", "privacy"); TEST_UTIL.getConfiguration().set("hbase.rpc.protection", "privacy");
SecureTestCluster.setUp(); SecureTestCluster.setUp();
try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())) { try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())) {
Token<? extends TokenIdentifier> token = TokenUtil.obtainToken(conn); Token<? extends TokenIdentifier> token = ClientTokenUtil.obtainToken(conn);
UserGroupInformation.getCurrentUser().addToken(token); UserGroupInformation.getCurrentUser().addToken(token);
} }
} }

View File

@ -70,7 +70,7 @@ public class TestGenerateDelegationToken extends SecureTestCluster {
public static void setUp() throws Exception { public static void setUp() throws Exception {
SecureTestCluster.setUp(); SecureTestCluster.setUp();
try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())) { try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())) {
Token<? extends TokenIdentifier> token = TokenUtil.obtainToken(conn); Token<? extends TokenIdentifier> token = ClientTokenUtil.obtainToken(conn);
UserGroupInformation.getCurrentUser().addToken(token); UserGroupInformation.getCurrentUser().addToken(token);
} }
} }