HBASE-17435 Call to preCommitStoreFile() hook encounters SaslException in secure deployment

This commit is contained in:
tedyu 2017-01-09 08:53:17 -08:00
parent bb3fe8a2fb
commit 9b26c9ff37
1 changed files with 18 additions and 1 deletions

View File

@ -25,6 +25,7 @@ import com.google.protobuf.Service;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@ -52,10 +53,12 @@ import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.SecureBul
import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.SecureBulkLoadHFilesResponse;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.Region.BulkLoadListener;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.security.SecureBulkLoadUtil;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.UserProvider;
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.Methods;
@ -123,6 +126,7 @@ public class SecureBulkLoadEndpoint extends SecureBulkLoadService
private Path baseStagingDir;
private RegionCoprocessorEnvironment env;
private Connection conn;
private UserProvider userProvider;
@ -130,6 +134,8 @@ public class SecureBulkLoadEndpoint extends SecureBulkLoadService
public void start(CoprocessorEnvironment env) {
this.env = (RegionCoprocessorEnvironment)env;
random = new SecureRandom();
RegionServerServices svc = ((RegionCoprocessorEnvironment)env).getRegionServerServices();
this.conn = svc.getConnection();
conf = env.getConfiguration();
baseStagingDir = SecureBulkLoadUtil.getBaseStagingDir(conf);
this.userProvider = UserProvider.instantiate(conf);
@ -226,7 +232,18 @@ public class SecureBulkLoadEndpoint extends SecureBulkLoadService
final String bulkToken = request.getBulkToken();
User user = getActiveUser();
final UserGroupInformation ugi = user.getUGI();
if(userToken != null) {
if (userProvider.isHadoopSecurityEnabled()) {
try {
Token tok = TokenUtil.obtainToken(conn);
if (tok != null) {
boolean b = ugi.addToken(tok);
LOG.debug("extra token added " + tok + " ret=" + b);
}
} catch (IOException ioe) {
LOG.warn("unable to add token", ioe);
}
}
if (userToken != null) {
ugi.addToken(userToken);
} else if (userProvider.isHadoopSecurityEnabled()) {
//we allow this to pass through in "simple" security mode