[HBASE-24190] Make kerberos value of hbase.security.authentication property case insensitive (#1687)
Signed-off-by: binlijin <binlijin@gmail.com> Signed-off-by: Viraj Jasani <vjasani@apache.org>
This commit is contained in:
parent
34a74534fe
commit
ff85daf1e2
|
@ -27,9 +27,11 @@ import java.lang.reflect.InvocationTargetException;
|
|||
import java.lang.reflect.Method;
|
||||
import java.util.List;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.ClassRule;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
@ -101,6 +103,19 @@ public class TestHBaseConfiguration {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSecurityConfCaseInsensitive() {
|
||||
Configuration conf = HBaseConfiguration.create();
|
||||
conf.set("hbase.security.authentication", "kerberos");
|
||||
Assert.assertTrue(User.isHBaseSecurityEnabled(conf));
|
||||
|
||||
conf.set("hbase.security.authentication", "KERBEROS");
|
||||
Assert.assertTrue(User.isHBaseSecurityEnabled(conf));
|
||||
|
||||
conf.set("hbase.security.authentication", "KERBeros");
|
||||
Assert.assertTrue(User.isHBaseSecurityEnabled(conf));
|
||||
}
|
||||
|
||||
private static class ReflectiveCredentialProviderClient {
|
||||
public static final String HADOOP_CRED_PROVIDER_FACTORY_CLASS_NAME =
|
||||
"org.apache.hadoop.security.alias.JavaKeyStoreProvider$Factory";
|
||||
|
|
|
@ -173,7 +173,7 @@ public class InfoServer {
|
|||
return false;
|
||||
}
|
||||
String remoteUser = req.getRemoteUser();
|
||||
if ("kerberos".equals(conf.get(HttpServer.HTTP_UI_AUTHENTICATION)) &&
|
||||
if ("kerberos".equalsIgnoreCase(conf.get(HttpServer.HTTP_UI_AUTHENTICATION)) &&
|
||||
conf.getBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, false) &&
|
||||
remoteUser != null) {
|
||||
return HttpServer.userHasAdministratorAccess(ctx, remoteUser);
|
||||
|
|
|
@ -91,7 +91,7 @@ public class SyncTable extends Configured implements Tool {
|
|||
private void initCredentialsForHBase(String zookeeper, Job job) throws IOException {
|
||||
Configuration peerConf = HBaseConfiguration.createClusterConf(job
|
||||
.getConfiguration(), zookeeper);
|
||||
if(peerConf.get("hbase.security.authentication").equals("kerberos")){
|
||||
if("kerberos".equalsIgnoreCase(peerConf.get("hbase.security.authentication"))){
|
||||
TableMapReduceUtil.initCredentialsForCluster(job, peerConf);
|
||||
}
|
||||
}
|
||||
|
@ -105,7 +105,7 @@ public class SyncTable extends Configured implements Tool {
|
|||
Job job = Job.getInstance(getConf(),getConf().get("mapreduce.job.name",
|
||||
"syncTable_" + sourceTableName + "-" + targetTableName));
|
||||
Configuration jobConf = job.getConfiguration();
|
||||
if (jobConf.get("hadoop.security.authentication").equals("kerberos")) {
|
||||
if ("kerberos".equalsIgnoreCase(jobConf.get("hadoop.security.authentication"))) {
|
||||
TokenCache.obtainTokensForNamenodes(job.getCredentials(), new
|
||||
Path[] { sourceHashDir }, getConf());
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue