mirror of https://github.com/apache/nifi.git
NIFI-2873: Nifi throws UnknownHostException with HA NameNode
Signed-off-by: Matt Burgess <mattyb149@apache.org> NIFI-2873: Changed test hive-site.xml to use local FS, fixed Checkstyle violations This closes #1113
This commit is contained in:
parent
7107616420
commit
e969a5ffe3
|
@ -323,6 +323,8 @@ public class PutHiveStreaming extends AbstractProcessor {
|
||||||
.withMaxOpenConnections(maxConnections)
|
.withMaxOpenConnections(maxConnections)
|
||||||
.withHeartBeatInterval(heartbeatInterval);
|
.withHeartBeatInterval(heartbeatInterval);
|
||||||
|
|
||||||
|
hiveConfigurator.preload(hiveConfig);
|
||||||
|
|
||||||
if (SecurityUtil.isSecurityEnabled(hiveConfig)) {
|
if (SecurityUtil.isSecurityEnabled(hiveConfig)) {
|
||||||
final String principal = context.getProperty(kerberosProperties.getKerberosPrincipal()).getValue();
|
final String principal = context.getProperty(kerberosProperties.getKerberosPrincipal()).getValue();
|
||||||
final String keyTab = context.getProperty(kerberosProperties.getKerberosKeytab()).getValue();
|
final String keyTab = context.getProperty(kerberosProperties.getKerberosKeytab()).getValue();
|
||||||
|
|
|
@ -18,6 +18,7 @@ package org.apache.nifi.util.hive;
|
||||||
|
|
||||||
import org.apache.commons.lang3.StringUtils;
|
import org.apache.commons.lang3.StringUtils;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.hive.conf.HiveConf;
|
import org.apache.hadoop.hive.conf.HiveConf;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
|
@ -71,6 +72,14 @@ public class HiveConfigurator {
|
||||||
return hiveConfig;
|
return hiveConfig;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void preload(Configuration configuration) {
|
||||||
|
try {
|
||||||
|
FileSystem.get(configuration);
|
||||||
|
} catch (IOException ioe) {
|
||||||
|
// Suppress exception as future uses of this configuration will fail
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
public UserGroupInformation authenticate(final Configuration hiveConfig, String principal, String keyTab, long ticketRenewalPeriod, ComponentLog log) throws AuthenticationFailedException {
|
public UserGroupInformation authenticate(final Configuration hiveConfig, String principal, String keyTab, long ticketRenewalPeriod, ComponentLog log) throws AuthenticationFailedException {
|
||||||
|
|
||||||
UserGroupInformation ugi;
|
UserGroupInformation ugi;
|
||||||
|
|
|
@ -17,6 +17,6 @@
|
||||||
<configuration>
|
<configuration>
|
||||||
<property>
|
<property>
|
||||||
<name>fs.default.name</name>
|
<name>fs.default.name</name>
|
||||||
<value>hdfs://hive</value>
|
<value>file:///</value>
|
||||||
</property>
|
</property>
|
||||||
</configuration>
|
</configuration>
|
Loading…
Reference in New Issue