Revert "Make kerberos value of hbase.security.authentication property case insensitive (#1687)"

This reverts commit a77829d5b7.
This commit is contained in:
Viraj Jasani 2020-05-14 11:49:23 +05:30
parent 87b8bdf617
commit b44cf90220
No known key found for this signature in database
GPG Key ID: 3AE697641452FC5D
3 changed files with 3 additions and 18 deletions

View File

@ -27,11 +27,9 @@ 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;
@ -103,19 +101,6 @@ 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

@ -173,7 +173,7 @@ public class InfoServer {
return false;
}
String remoteUser = req.getRemoteUser();
if ("kerberos".equalsIgnoreCase(conf.get(HttpServer.HTTP_UI_AUTHENTICATION)) &&
if ("kerberos".equals(conf.get(HttpServer.HTTP_UI_AUTHENTICATION)) &&
conf.getBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, false) &&
remoteUser != null) {
return HttpServer.userHasAdministratorAccess(ctx, remoteUser);

View File

@ -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("kerberos".equalsIgnoreCase(peerConf.get("hbase.security.authentication"))){
if(peerConf.get("hbase.security.authentication").equals("kerberos")){
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 ("kerberos".equalsIgnoreCase(jobConf.get("hadoop.security.authentication"))) {
if (jobConf.get("hadoop.security.authentication").equals("kerberos")) {
TokenCache.obtainTokensForNamenodes(job.getCredentials(), new
Path[] { sourceHashDir }, getConf());
}