Revert "HBASE-23194 Remove unused methods from TokenUtil (#737)"

This reverts commit d7b90b3199.
This commit is contained in:
Viraj Jasani 2019-10-28 21:02:57 +05:30 committed by Peter Somogyi
parent 9a80618765
commit 7b4dd9587b
1 changed files with 70 additions and 9 deletions

View File

@ -19,8 +19,11 @@ package org.apache.hadoop.hbase.security.token;
import java.io.IOException;
import java.lang.reflect.UndeclaredThrowableException;
import java.util.concurrent.CompletableFuture;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.client.AsyncConnection;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
@ -39,27 +42,85 @@ import org.slf4j.LoggerFactory;
*/
@InterfaceAudience.Public
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);
private static Text getClusterId(Token<AuthenticationTokenIdentifier> token) {
return token.getService() != null
? token.getService() : new Text("default");
/**
* 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<Token<AuthenticationTokenIdentifier>> obtainToken(
AsyncConnection conn) {
return ClientTokenUtil.obtainToken(conn);
}
/**
* @deprecated As of HBase-3.0. Will be removed in HBase-4.0
* <p>
* See {@link ClientTokenUtil#obtainAndCacheToken(org.apache.hadoop.hbase.client.Connection,
* org.apache.hadoop.hbase.security.User)}.
* 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<AuthenticationTokenIdentifier> obtainToken(Connection conn)
throws IOException {
return ClientTokenUtil.obtainToken(conn);
}
/**
* 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) {
return ClientTokenUtil.toToken(token);
}
/**
* 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 ClientTokenUtil.obtainToken(conn, user);
}
/**
* See {@link ClientTokenUtil#obtainAndCacheToken(org.apache.hadoop.hbase.client.Connection,
* org.apache.hadoop.hbase.security.User)}.
*/
public static void obtainAndCacheToken(final Connection conn,
final User user) throws IOException, InterruptedException {
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)
throws IOException {
return token.getService() != null
? token.getService() : new Text("default");
}
/**
* Obtain an authentication token on behalf of the given user and add it to
* the credentials for the given map reduce job.