From 0ec0a9403fe8ebf4399b8caa4f5a783792d4fb61 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Stig=20D=C3=B8ssing?= Date: Thu, 19 Sep 2019 21:42:04 +0200 Subject: [PATCH] =?UTF-8?q?HBase-22027:=20Split=20non-MR=20related=20parts?= =?UTF-8?q?=20of=20TokenUtil=20off=20into=20a=20Clien=E2=80=A6=20(#361)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * 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 Signed-off-by: stack --- .../hbase/security/token/ClientTokenUtil.java | 201 ++++++++++++++++++ .../security/token/TestClientTokenUtil.java | 20 +- .../regionserver/SecureBulkLoadManager.java | 4 +- .../hbase/security/token/TokenProvider.java | 2 +- .../hbase/security/token/TokenUtil.java | 190 +++++------------ .../TestDelegationTokenWithEncryption.java | 2 +- .../token/TestGenerateDelegationToken.java | 2 +- 7 files changed, 266 insertions(+), 155 deletions(-) create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/ClientTokenUtil.java rename hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenUtil.java => hbase-client/src/test/java/org/apache/hadoop/hbase/security/token/TestClientTokenUtil.java (80%) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/ClientTokenUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/ClientTokenUtil.java new file mode 100644 index 000000000000..fefe8b907a07 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/ClientTokenUtil.java @@ -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> obtainToken( + AsyncConnection conn) { + CompletableFuture> future = new CompletableFuture<>(); + if (injectedException != null) { + future.completeExceptionally(injectedException); + return future; + } + AsyncTable table = conn.getTable(TableName.META_TABLE_NAME); + table. 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 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 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 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 obtainToken( + final Connection conn, User user) throws IOException, InterruptedException { + return user.runAs(new PrivilegedExceptionAction>() { + @Override + public Token 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 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()); + } + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenUtil.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/security/token/TestClientTokenUtil.java similarity index 80% rename from hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenUtil.java rename to hbase-client/src/test/java/org/apache/hadoop/hbase/security/token/TestClientTokenUtil.java index 585a3ec44cfb..b77008f9d3b3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenUtil.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/security/token/TestClientTokenUtil.java @@ -23,7 +23,6 @@ import java.io.IOException; import java.lang.reflect.Field; -import java.lang.reflect.InvocationTargetException; import java.net.URL; import java.net.URLClassLoader; import java.util.concurrent.CompletableFuture; @@ -43,19 +42,19 @@ import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; @Category(SmallTests.class) -public class TestTokenUtil { +public class TestClientTokenUtil { @ClassRule public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestTokenUtil.class); + HBaseClassTestRule.forClass(TestClientTokenUtil.class); private URLClassLoader cl; @Before public void setUp() { URL urlPU = ProtobufUtil.class.getProtectionDomain().getCodeSource().getLocation(); - URL urlTU = TokenUtil.class.getProtectionDomain().getCodeSource().getLocation(); - cl = new URLClassLoader(new URL[] { urlPU, urlTU }, getClass().getClassLoader()); + URL urlCTU = ClientTokenUtil.class.getProtectionDomain().getCodeSource().getLocation(); + cl = new URLClassLoader(new URL[] { urlPU, urlCTU }, getClass().getClassLoader()); } @After @@ -67,15 +66,15 @@ public void tearDown() throws IOException { public void testObtainToken() throws Exception { Throwable injected = new com.google.protobuf.ServiceException("injected"); - Class tokenUtil = cl.loadClass(TokenUtil.class.getCanonicalName()); - Field shouldInjectFault = tokenUtil.getDeclaredField("injectedException"); + Class clientTokenUtil = cl.loadClass(ClientTokenUtil.class.getCanonicalName()); + Field shouldInjectFault = clientTokenUtil.getDeclaredField("injectedException"); shouldInjectFault.setAccessible(true); shouldInjectFault.set(null, injected); try { - tokenUtil.getMethod("obtainToken", Connection.class).invoke(null, new Object[] { null }); + ClientTokenUtil.obtainToken((Connection)null); fail("Should have injected exception."); - } catch (InvocationTargetException e) { + } catch (IOException e) { Throwable t = e; boolean serviceExceptionFound = false; while ((t = t.getCause()) != null) { @@ -89,8 +88,7 @@ public void testObtainToken() throws Exception { } } - CompletableFuture future = (CompletableFuture) tokenUtil - .getMethod("obtainToken", AsyncConnection.class).invoke(null, new Object[] { null }); + CompletableFuture future = ClientTokenUtil.obtainToken((AsyncConnection)null); try { future.get(); fail("Should have injected exception."); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java index d54be75518be..2dfb43b6b2ed 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java @@ -43,8 +43,8 @@ import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.UserProvider; 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.TokenUtil; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSHDFSUtils; import org.apache.hadoop.hbase.util.FSUtils; @@ -231,7 +231,7 @@ public Map> secureBulkLoadHFiles(final HRegion region, final UserGroupInformation ugi = user.getUGI(); if (userProvider.isHadoopSecurityEnabled()) { try { - Token tok = TokenUtil.obtainToken(conn).get(); + Token tok = ClientTokenUtil.obtainToken(conn).get(); if (tok != null) { boolean b = ugi.addToken(tok); LOG.debug("token added " + tok + " for user " + ugi + " return=" + b); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenProvider.java index b137aaa30eb4..92bd0dbe3096 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenProvider.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenProvider.java @@ -129,7 +129,7 @@ public void getAuthenticationToken(RpcController controller, Token token = secretManager.generateToken(currentUser.getName()); - response.setToken(TokenUtil.toToken(token)).build(); + response.setToken(ClientTokenUtil.toToken(token)).build(); } catch (IOException ioe) { CoprocessorRpcUtils.setControllerException(controller, ioe); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java index 28efb842d067..159f81e8dbf0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java @@ -17,24 +17,13 @@ */ 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.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.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.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.zookeeper.ZKClusterId; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; @@ -47,7 +36,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; /** * Utility methods for obtaining authentication tokens. @@ -57,142 +45,80 @@ public class TokenUtil { // This class is referenced indirectly by User out in common; instances are created by reflection private static final Logger LOG = LoggerFactory.getLogger(TokenUtil.class); - // Set in TestTokenUtil via reflection - private static ServiceException injectedException; - - 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}. - */ + /** + * See {@link ClientTokenUtil#obtainToken(org.apache.hadoop.hbase.client.AsyncConnection)}. + * @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 CompletableFuture> obtainToken( AsyncConnection conn) { - CompletableFuture> future = new CompletableFuture<>(); - if (injectedException != null) { - future.completeExceptionally(injectedException); - return future; - } - AsyncTable table = conn.getTable(TableName.META_TABLE_NAME); - table. 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; + return ClientTokenUtil.obtainToken(conn); } /** - * 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 + * See {@link ClientTokenUtil#obtainToken(org.apache.hadoop.hbase.client.Connection)}. + * @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 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 = - 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(); - } - } + return ClientTokenUtil.obtainToken(conn); } /** - * Converts a Token instance (with embedded identifier) to the protobuf representation. - * - * @param token the Token instance to copy - * @return the protobuf Token message + * 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 AuthenticationProtos.Token toToken(Token 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(); + return ClientTokenUtil.toToken(token); } /** - * Obtain and return an authentication token for the current user. - * @param conn The HBase cluster connection - * @return the authentication token instance + * See {@link ClientTokenUtil#obtainToken(org.apache.hadoop.hbase.client.Connection, + * org.apache.hadoop.hbase.security.User)}. + * @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 obtainToken( final Connection conn, User user) throws IOException, InterruptedException { - return user.runAs(new PrivilegedExceptionAction>() { - @Override - public Token run() throws Exception { - return obtainToken(conn); - } - }); - } - - - private static Text getClusterId(Token token) - throws IOException { - return token.getService() != null - ? token.getService() : new Text("default"); + return ClientTokenUtil.obtainToken(conn, user); } /** - * 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 + * 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 { - try { - Token token = obtainToken(conn, user); + ClientTokenUtil.obtainAndCacheToken(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()); - } + /** + * 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 toToken(AuthenticationProtos.Token proto) { + return ClientTokenUtil.toToken(proto); + } + + private static Text getClusterId(Token token) + throws IOException { + return token.getService() != null + ? token.getService() : new Text("default"); } /** @@ -208,7 +134,7 @@ public static void obtainTokenForJob(final Connection conn, User user, Job job) throws IOException, InterruptedException { try { - Token token = obtainToken(conn, user); + Token token = ClientTokenUtil.obtainToken(conn, user); if (token == null) { throw new IOException("No token returned for user " + user.getName()); @@ -243,7 +169,7 @@ public static void obtainTokenForJob(final Connection conn, public static void obtainTokenForJob(final Connection conn, final JobConf job, User user) throws IOException, InterruptedException { try { - Token token = obtainToken(conn, user); + Token token = ClientTokenUtil.obtainToken(conn, user); if (token == null) { throw new IOException("No token returned for user " + user.getName()); @@ -281,7 +207,7 @@ public static void addTokenForJob(final Connection conn, final JobConf job, User Token token = getAuthToken(conn.getConfiguration(), user); if (token == null) { - token = obtainToken(conn, user); + token = ClientTokenUtil.obtainToken(conn, user); } job.getCredentials().addToken(token.getService(), token); } @@ -300,7 +226,7 @@ public static void addTokenForJob(final Connection conn, User user, Job job) throws IOException, InterruptedException { Token token = getAuthToken(conn.getConfiguration(), user); if (token == null) { - token = obtainToken(conn, user); + token = ClientTokenUtil.obtainToken(conn, user); } job.getCredentials().addToken(token.getService(), token); } @@ -319,7 +245,7 @@ public static boolean addTokenIfMissing(Connection conn, User user) throws IOException, InterruptedException { Token token = getAuthToken(conn.getConfiguration(), user); if (token == null) { - token = obtainToken(conn, user); + token = ClientTokenUtil.obtainToken(conn, user); user.getUGI().addToken(token.getService(), token); return true; } @@ -345,18 +271,4 @@ private static Token getAuthToken(Configuration c 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 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); - } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestDelegationTokenWithEncryption.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestDelegationTokenWithEncryption.java index 6a8b0eb6056b..9353576feb7f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestDelegationTokenWithEncryption.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestDelegationTokenWithEncryption.java @@ -67,7 +67,7 @@ public static void setUp() throws Exception { TEST_UTIL.getConfiguration().set("hbase.rpc.protection", "privacy"); SecureTestCluster.setUp(); try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())) { - Token token = TokenUtil.obtainToken(conn); + Token token = ClientTokenUtil.obtainToken(conn); UserGroupInformation.getCurrentUser().addToken(token); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java index 76a8d44c7784..ee5f18fa7a17 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java @@ -70,7 +70,7 @@ public class TestGenerateDelegationToken extends SecureTestCluster { public static void setUp() throws Exception { SecureTestCluster.setUp(); try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())) { - Token token = TokenUtil.obtainToken(conn); + Token token = ClientTokenUtil.obtainToken(conn); UserGroupInformation.getCurrentUser().addToken(token); } }