Skip to content

Commit

Permalink
HBASE-23330: Fix delegation token fetch with MasterRegistry (apache#1084
Browse files Browse the repository at this point in the history
)

Signed-off-by: Andrew Purtell <apurtell@apache.org>
  • Loading branch information
bharathv authored and thangTang committed Apr 16, 2020
1 parent c3d73f7 commit 8cbece3
Show file tree
Hide file tree
Showing 64 changed files with 3,890 additions and 2,278 deletions.
Expand Up @@ -33,6 +33,7 @@
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
Expand Down Expand Up @@ -278,6 +279,15 @@ CompletableFuture<MasterService.Interface> getMasterStub() {
}, stub -> true, "master stub");
}

String getClusterId() {
try {
return registry.getClusterId().get();
} catch (InterruptedException | ExecutionException e) {
LOG.error("Error fetching cluster ID: ", e);
}
return null;
}

void clearMasterStubCache(MasterService.Interface stub) {
masterStub.compareAndSet(stub, null);
}
Expand Down
Expand Up @@ -204,6 +204,11 @@ default BufferedMutator getBufferedMutator(TableName tableName) throws IOExcepti
*/
AsyncConnection toAsyncConnection();

/**
* @return the cluster ID unique to this HBase cluster.
*/
String getClusterId();

/**
* Retrieve an Hbck implementation to fix an HBase cluster.
* The returned Hbck is not guaranteed to be thread-safe. A new instance should be created by
Expand Down
Expand Up @@ -204,6 +204,11 @@ public AsyncConnection toAsyncConnection() {
return conn;
}

@Override
public String getClusterId() {
return conn.getClusterId();
}

@Override
public Hbck getHbck() throws IOException {
return FutureUtils.get(conn.getHbck());
Expand Down
Expand Up @@ -246,5 +246,11 @@ public void clearRegionLocationCache() {
public AsyncConnection toAsyncConnection() {
return null;
}

@Override
public String getClusterId() {
return null;
}

}
}
Expand Up @@ -296,5 +296,10 @@ public void clearRegionLocationCache() {
public AsyncConnection toAsyncConnection() {
throw new UnsupportedOperationException();
}

@Override
public String getClusterId() {
return null;
}
}
}
Expand Up @@ -105,4 +105,9 @@ public Hbck getHbck(ServerName masterServer) throws IOException {
public AsyncConnection toAsyncConnection() {
return new SharedAsyncConnection(conn.toAsyncConnection());
}

@Override
public String getClusterId() {
return conn.getClusterId();
}
}
Expand Up @@ -26,18 +26,14 @@
import org.apache.hadoop.hbase.client.ConnectionFactory;
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;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.security.token.Token;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;


/**
* Utility methods for obtaining authentication tokens.
*/
Expand Down Expand Up @@ -219,7 +215,7 @@ public static void obtainTokenForJob(final Connection conn, final JobConf job, U
public static void addTokenForJob(final Connection conn, final JobConf job, User user)
throws IOException, InterruptedException {

Token<AuthenticationTokenIdentifier> token = getAuthToken(conn.getConfiguration(), user);
Token<AuthenticationTokenIdentifier> token = getAuthToken(conn, user);
if (token == null) {
token = ClientTokenUtil.obtainToken(conn, user);
}
Expand All @@ -238,7 +234,7 @@ public static void addTokenForJob(final Connection conn, final JobConf job, User
*/
public static void addTokenForJob(final Connection conn, User user, Job job)
throws IOException, InterruptedException {
Token<AuthenticationTokenIdentifier> token = getAuthToken(conn.getConfiguration(), user);
Token<AuthenticationTokenIdentifier> token = getAuthToken(conn, user);
if (token == null) {
token = ClientTokenUtil.obtainToken(conn, user);
}
Expand All @@ -257,7 +253,7 @@ public static void addTokenForJob(final Connection conn, User user, Job job)
*/
public static boolean addTokenIfMissing(Connection conn, User user)
throws IOException, InterruptedException {
Token<AuthenticationTokenIdentifier> token = getAuthToken(conn.getConfiguration(), user);
Token<AuthenticationTokenIdentifier> token = getAuthToken(conn, user);
if (token == null) {
token = ClientTokenUtil.obtainToken(conn, user);
user.getUGI().addToken(token.getService(), token);
Expand All @@ -270,19 +266,12 @@ public static boolean addTokenIfMissing(Connection conn, User user)
* Get the authentication token of the user for the cluster specified in the configuration
* @return null if the user does not have the token, otherwise the auth token for the cluster.
*/
private static Token<AuthenticationTokenIdentifier> getAuthToken(Configuration conf, User user)
throws IOException, InterruptedException {
ZKWatcher zkw = new ZKWatcher(conf, "TokenUtil-getAuthToken", null);
try {
String clusterId = ZKClusterId.readClusterIdZNode(zkw);
if (clusterId == null) {
throw new IOException("Failed to get cluster ID");
}
return new AuthenticationTokenSelector().selectToken(new Text(clusterId), user.getTokens());
} catch (KeeperException e) {
throw new IOException(e);
} finally {
zkw.close();
private static Token<AuthenticationTokenIdentifier> getAuthToken(Connection conn, User user)
throws IOException {
final String clusterId = conn.getClusterId();
if (clusterId == null) {
throw new IOException("Failed to get cluster ID");
}
return new AuthenticationTokenSelector().selectToken(new Text(clusterId), user.getTokens());
}
}
Expand Up @@ -202,6 +202,19 @@ public boolean updateConnectionAccessTime() {
return false;
}

/**
* @return Cluster ID for the HBase cluster or null if there is an err making the connection.
*/
public String getClusterId() {
try {
ConnectionInfo connInfo = getCurrentConnection();
return connInfo.connection.getClusterId();
} catch (IOException e) {
LOG.error("Error getting connection: ", e);
}
return null;
}

class ConnectionInfo {
final Connection connection;
final String userName;
Expand Down
Expand Up @@ -1272,6 +1272,11 @@ public TThriftServerType getThriftServerType() {
return TThriftServerType.ONE;
}

@Override
public String getClusterId() throws TException {
return connectionCache.getClusterId();
}

private static IOError getIOError(Throwable throwable) {
IOError error = new IOErrorWithCause(throwable);
error.setMessage(Throwables.getStackTraceAsString(throwable));
Expand Down

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

0 comments on commit 8cbece3

Please sign in to comment.