You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2019/09/21 00:38:47 UTC

[hbase] branch branch-2 updated: HBase-22027: Split non-MR related parts of TokenUtil off into a ClientTokenUtil, and move ClientTokenUtil to hbase-client (#649)

This is an automated email from the ASF dual-hosted git repository.

stack pushed a commit to branch branch-2
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2 by this push:
     new 047bbfd  HBase-22027: Split non-MR related parts of TokenUtil off into a ClientTokenUtil, and move ClientTokenUtil to hbase-client (#649)
047bbfd is described below

commit 047bbfd4c28c032a07cb3c98bb9e17e5c4348be0
Author: Stig Døssing <sr...@apache.org>
AuthorDate: Sat Sep 21 02:38:41 2019 +0200

    HBase-22027: Split non-MR related parts of TokenUtil off into a ClientTokenUtil, and move ClientTokenUtil to hbase-client (#649)
---
 .../hbase/security/token/ClientTokenUtil.java      | 168 +++++++++++++++++++++
 .../hbase/security/token/TestClientTokenUtil.java  |  41 +++--
 .../hbase/regionserver/SecureBulkLoadManager.java  |   5 +-
 .../hadoop/hbase/security/token/TokenProvider.java |   2 +-
 .../hadoop/hbase/security/token/TokenUtil.java     | 164 ++++++--------------
 .../token/TestDelegationTokenWithEncryption.java   |   2 +-
 .../token/TestGenerateDelegationToken.java         |   2 +-
 7 files changed, 246 insertions(+), 138 deletions(-)

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 0000000..09398ed
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/ClientTokenUtil.java
@@ -0,0 +1,168 @@
+/*
+ * 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 org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+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 HBase cluster connection
+   * @throws IOException if a remote error or serialization problem occurs.
+   * @return the authentication token instance
+   */
+  @InterfaceAudience.Private
+  public 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());
+    }
+  }
+}
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 72%
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 32fcddb..50db3a9 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
@@ -20,45 +20,55 @@ package org.apache.hadoop.hbase.security.token;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.fail;
 
+import java.io.IOException;
 import java.lang.reflect.Field;
-import java.lang.reflect.InvocationTargetException;
 import java.net.URL;
 import java.net.URLClassLoader;
-
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-
+import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
 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);
 
-  @Test
-  public void testObtainToken() throws Exception {
+  private URLClassLoader cl;
+
+  @Before
+  public void setUp() {
     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, urlCTU }, getClass().getClassLoader());
+  }
 
-    ClassLoader cl = new URLClassLoader(new URL[] { urlPU, urlTU }, getClass().getClassLoader());
+  @After
+  public void tearDown() throws IOException {
+    Closeables.close(cl, true);
+  }
 
+  @Test
+  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) {
@@ -73,8 +83,7 @@ public class TestTokenUtil {
     }
 
     Boolean loaded = (Boolean) cl.loadClass(ProtobufUtil.class.getCanonicalName())
-        .getDeclaredMethod("isClassLoaderLoaded")
-        .invoke(null);
+      .getDeclaredMethod("isClassLoaderLoaded").invoke(null);
     assertFalse("Should not have loaded DynamicClassLoader", loaded);
   }
 }
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 6b55744..e0d8a5e 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,9 @@ import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.regionserver.HRegion.BulkLoadListener;
 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;
@@ -229,7 +230,7 @@ public class SecureBulkLoadManager {
     final UserGroupInformation ugi = user.getUGI();
     if (userProvider.isHadoopSecurityEnabled()) {
       try {
-        Token tok = TokenUtil.obtainToken(conn);
+        Token<AuthenticationTokenIdentifier> tok = ClientTokenUtil.obtainToken(conn);
         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 b137aaa..92bd0db 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 class TokenProvider implements AuthenticationProtos.AuthenticationService
 
       Token<AuthenticationTokenIdentifier> 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 c54d905..9e9b685 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
@@ -20,22 +20,12 @@ package org.apache.hadoop.hbase.security.token;
 
 import java.io.IOException;
 import java.lang.reflect.UndeclaredThrowableException;
-import java.security.PrivilegedExceptionAction;
-
-import com.google.protobuf.ByteString;
-import com.google.protobuf.ServiceException;
-
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.TableName;
 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.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapreduce.Job;
@@ -45,6 +35,7 @@ import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+
 /**
  * Utility methods for obtaining authentication tokens.
  */
@@ -53,115 +44,68 @@ 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 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.
    */
-  public 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();
-      }
-    }
+  @Deprecated
+  public static Token<AuthenticationTokenIdentifier> obtainToken(Connection conn)
+      throws IOException {
+    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<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();
+    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<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);
-      }
-    });
-  }
-
-
-  private static Text getClusterId(Token<AuthenticationTokenIdentifier> 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<AuthenticationTokenIdentifier> 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<AuthenticationTokenIdentifier> toToken(AuthenticationProtos.Token proto) {
+    return ClientTokenUtil.toToken(proto);
+  }
+
+  private static Text getClusterId(Token<AuthenticationTokenIdentifier> token)
+      throws IOException {
+    return token.getService() != null
+        ? token.getService() : new Text("default");
   }
 
   /**
@@ -177,7 +121,7 @@ public class TokenUtil {
       User user, Job job)
       throws IOException, InterruptedException {
     try {
-      Token<AuthenticationTokenIdentifier> token = obtainToken(conn, user);
+      Token<AuthenticationTokenIdentifier> token = ClientTokenUtil.obtainToken(conn, user);
 
       if (token == null) {
         throw new IOException("No token returned for user " + user.getName());
@@ -212,7 +156,7 @@ public class TokenUtil {
   public static void obtainTokenForJob(final Connection conn, final JobConf job, User user)
       throws IOException, InterruptedException {
     try {
-      Token<AuthenticationTokenIdentifier> token = obtainToken(conn, user);
+      Token<AuthenticationTokenIdentifier> token = ClientTokenUtil.obtainToken(conn, user);
 
       if (token == null) {
         throw new IOException("No token returned for user " + user.getName());
@@ -250,7 +194,7 @@ public class TokenUtil {
 
     Token<AuthenticationTokenIdentifier> token = getAuthToken(conn.getConfiguration(), user);
     if (token == null) {
-      token = obtainToken(conn, user);
+      token = ClientTokenUtil.obtainToken(conn, user);
     }
     job.getCredentials().addToken(token.getService(), token);
   }
@@ -269,7 +213,7 @@ public class TokenUtil {
       throws IOException, InterruptedException {
     Token<AuthenticationTokenIdentifier> token = getAuthToken(conn.getConfiguration(), user);
     if (token == null) {
-      token = obtainToken(conn, user);
+      token = ClientTokenUtil.obtainToken(conn, user);
     }
     job.getCredentials().addToken(token.getService(), token);
   }
@@ -288,7 +232,7 @@ public class TokenUtil {
       throws IOException, InterruptedException {
     Token<AuthenticationTokenIdentifier> 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;
     }
@@ -314,18 +258,4 @@ public class TokenUtil {
       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);
-  }
 }
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 6a8b0eb..9353576 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 class TestDelegationTokenWithEncryption extends SecureTestCluster {
     TEST_UTIL.getConfiguration().set("hbase.rpc.protection", "privacy");
     SecureTestCluster.setUp();
     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);
     }
   }
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 76a8d44..ee5f18f 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<? extends TokenIdentifier> token = TokenUtil.obtainToken(conn);
+      Token<? extends TokenIdentifier> token = ClientTokenUtil.obtainToken(conn);
       UserGroupInformation.getCurrentUser().addToken(token);
     }
   }