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:
shahrs87 2020-05-11 12:03:47 +05:30 committed by Viraj Jasani
parent d1aff27e06
commit be59ddc263
No known key found for this signature in database
GPG Key ID: 3AE697641452FC5D
2 changed files with 17 additions and 2 deletions

View File

@ -30,8 +30,10 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@ -96,6 +98,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";

View File

@ -86,7 +86,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);
}
}
@ -100,7 +100,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());
}