SOLR-6372: HdfsDirectoryFactory should use supplied Configuration for communicating with secure kerberos.

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1617852 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Mark Robert Miller 2014-08-13 23:00:12 +00:00
parent 910d467a93
commit b7ad072dda
2 changed files with 17 additions and 4 deletions

View File

@ -267,6 +267,9 @@ Bug Fixes
* SOLR-6062: Fix undesirable edismax query parser effect (introduced in SOLR-2058) in how phrase queries
generated from pf, pf2, and pf3 are merged into the main query. (Michael Dodsworth via ehatcher)
* SOLR-6372: HdfsDirectoryFactory should use supplied Configuration for communicating with secure kerberos.
(Gregory Chanan via Mark Miller)
Optimizations
---------------------
@ -2021,7 +2024,7 @@ New Features
* SOLR-4921: Admin UI now supports adding documents to Solr (gsingers, steffkes)
* SOLR-4916: Add support to write and read Solr index files and transaction log
files to and from HDFS. (phunt, Mark Miller, Greg Chanan)
files to and from HDFS. (phunt, Mark Miller, Gregory Chanan)
* SOLR-4892: Add FieldMutatingUpdateProcessorFactory subclasses
Parse{Date,Integer,Long,Float,Double,Boolean}UpdateProcessorFactory. These

View File

@ -22,6 +22,7 @@ import java.net.URI;
import java.net.URLEncoder;
import org.apache.hadoop.conf.Configuration;
import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.security.UserGroupInformation;
@ -303,9 +304,18 @@ public class HdfsDirectoryFactory extends CachingDirectoryFactory {
synchronized (HdfsDirectoryFactory.class) {
if (kerberosInit == null) {
kerberosInit = new Boolean(true);
Configuration conf = new Configuration();
conf.set("hadoop.security.authentication", "kerberos");
UserGroupInformation.setConfiguration(conf);
final Configuration conf = getConf();
final String authVal = conf.get(HADOOP_SECURITY_AUTHENTICATION);
final String kerberos = "kerberos";
if (authVal != null && !authVal.equals(kerberos)) {
throw new IllegalArgumentException(HADOOP_SECURITY_AUTHENTICATION
+ " set to: " + authVal + ", not kerberos, but attempting to "
+ " connect to HDFS via kerberos");
}
// let's avoid modifying the supplied configuration, just to be conservative
final Configuration ugiConf = new Configuration(getConf());
ugiConf.set(HADOOP_SECURITY_AUTHENTICATION, kerberos);
UserGroupInformation.setConfiguration(ugiConf);
LOG.info(
"Attempting to acquire kerberos ticket with keytab: {}, principal: {} ",
keytabFile, principal);