NIFI-7025: Wrap Hive 3 calls with UGI.doAs

Updated PutHive3Streaming to wrap calls to Hive in UGI.doAs methods
Fixed misleading logging message after the principal has been authenticated with the KDC
When connecting to unsecured Hive 3, a UGI with "simple" auth will be used

Signed-off-by: Matthew Burgess <mattyb149@apache.org>

This closes #4108
This commit is contained in:
jstorck 2020-03-03 20:03:59 -05:00 committed by Matthew Burgess
parent bad254ec5b
commit 4b6de8d164
No known key found for this signature in database
GPG Key ID: 05D3DEB8126DAD24
2 changed files with 189 additions and 150 deletions

View File

@ -67,8 +67,10 @@ import org.apache.nifi.util.hive.HiveOptions;
import org.apache.nifi.util.hive.HiveUtils; import org.apache.nifi.util.hive.HiveUtils;
import org.apache.nifi.util.hive.ValidationResources; import org.apache.nifi.util.hive.ValidationResources;
import javax.security.auth.login.LoginException;
import java.io.IOException; import java.io.IOException;
import java.io.InputStream; import java.io.InputStream;
import java.security.PrivilegedAction;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
@ -321,7 +323,7 @@ public class PutHive3Streaming extends AbstractProcessor {
} }
@OnScheduled @OnScheduled
public void setup(final ProcessContext context) { public void setup(final ProcessContext context) throws IOException {
ComponentLog log = getLogger(); ComponentLog log = getLogger();
rollbackOnFailure = context.getProperty(ROLLBACK_ON_FAILURE).asBoolean(); rollbackOnFailure = context.getProperty(ROLLBACK_ON_FAILURE).asBoolean();
@ -368,9 +370,9 @@ public class PutHive3Streaming extends AbstractProcessor {
throw new ProcessException(ae); throw new ProcessException(ae);
} }
log.info("Successfully logged in as principal {} with keytab {}", new Object[]{resolvedPrincipal, resolvedKeytab}); log.info("Successfully logged in as principal " + resolvedPrincipal);
} else { } else {
ugi = null; ugi = SecurityUtil.loginSimple(hiveConfig);
kerberosUserReference.set(null); kerberosUserReference.set(null);
} }
@ -381,9 +383,10 @@ public class PutHive3Streaming extends AbstractProcessor {
} }
public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
getUgi().doAs((PrivilegedAction<Void>) () -> {
FlowFile flowFile = session.get(); FlowFile flowFile = session.get();
if (flowFile == null) { if (flowFile == null) {
return; return null;
} }
final RecordReaderFactory recordReaderFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class); final RecordReaderFactory recordReaderFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
@ -421,7 +424,13 @@ public class PutHive3Streaming extends AbstractProcessor {
if (SecurityUtil.isSecurityEnabled(hiveConfig)) { if (SecurityUtil.isSecurityEnabled(hiveConfig)) {
final KerberosCredentialsService credentialsService = context.getProperty(KERBEROS_CREDENTIALS_SERVICE).asControllerService(KerberosCredentialsService.class); final KerberosCredentialsService credentialsService = context.getProperty(KERBEROS_CREDENTIALS_SERVICE).asControllerService(KerberosCredentialsService.class);
o = o.withKerberosPrincipal(credentialsService.getPrincipal()).withKerberosKeytab(credentialsService.getKeytab()); final String explicitPrincipal = context.getProperty(KERBEROS_PRINCIPAL).evaluateAttributeExpressions().getValue();
final String resolvedPrincipal;
if (credentialsService != null) {
resolvedPrincipal = credentialsService.getPrincipal();
o = o.withKerberosKeytab(credentialsService.getKeytab());
} else resolvedPrincipal = explicitPrincipal;
o = o.withKerberosPrincipal(resolvedPrincipal);
} }
final HiveOptions options = o; final HiveOptions options = o;
@ -473,7 +482,7 @@ public class PutHive3Streaming extends AbstractProcessor {
rrfe rrfe
); );
session.transfer(flowFile, REL_FAILURE); session.transfer(flowFile, REL_FAILURE);
return; return null;
} }
} }
session.transfer(flowFile, REL_SUCCESS); session.transfer(flowFile, REL_SUCCESS);
@ -547,6 +556,8 @@ public class PutHive3Streaming extends AbstractProcessor {
// Restore original class loader, might not be necessary but is good practice since the processor task changed it // Restore original class loader, might not be necessary but is good practice since the processor task changed it
Thread.currentThread().setContextClassLoader(originalClassloader); Thread.currentThread().setContextClassLoader(originalClassloader);
} }
return null;
});
} }
StreamingConnection makeStreamingConnection(HiveOptions options, RecordReader reader) throws StreamingException { StreamingConnection makeStreamingConnection(HiveOptions options, RecordReader reader) throws StreamingException {
@ -623,5 +634,23 @@ public class PutHive3Streaming extends AbstractProcessor {
super(message, cause); super(message, cause);
} }
} }
UserGroupInformation getUgi() {
getLogger().trace("getting UGI instance");
if (kerberosUserReference.get() != null) {
// if there's a KerberosUser associated with this UGI, check the TGT and relogin if it is close to expiring
KerberosUser kerberosUser = kerberosUserReference.get();
getLogger().debug("kerberosUser is " + kerberosUser);
try {
getLogger().debug("checking TGT on kerberosUser [{}]", new Object[] {kerberosUser});
kerberosUser.checkTGTAndRelogin();
} catch (LoginException e) {
throw new ProcessException("Unable to relogin with kerberos credentials for " + kerberosUser.getPrincipal(), e);
}
} else {
getLogger().debug("kerberosUser was null, will not refresh TGT with KerberosUser");
}
return ugi;
}
} }

View File

@ -155,7 +155,7 @@ public class TestPutHive3Streaming {
System.setProperty("java.security.krb5.kdc", "nifi.kdc"); System.setProperty("java.security.krb5.kdc", "nifi.kdc");
ugi = null; ugi = null;
processor = new MockPutHive3Streaming(); processor = new MockPutHive3Streaming(ugi);
hiveConfigurator = mock(HiveConfigurator.class); hiveConfigurator = mock(HiveConfigurator.class);
hiveConf = new HiveConf(); hiveConf = new HiveConf();
when(hiveConfigurator.getConfigurationFromFiles(anyString())).thenReturn(hiveConf); when(hiveConfigurator.getConfigurationFromFiles(anyString())).thenReturn(hiveConf);
@ -272,6 +272,7 @@ public class TestPutHive3Streaming {
runner.setProperty(PutHive3Streaming.DB_NAME, "default"); runner.setProperty(PutHive3Streaming.DB_NAME, "default");
runner.setProperty(PutHive3Streaming.TABLE_NAME, "users"); runner.setProperty(PutHive3Streaming.TABLE_NAME, "users");
processor.ugi = mock(UserGroupInformation.class); processor.ugi = mock(UserGroupInformation.class);
processor.kerberosUserReference.set(mock(KerberosUser.class));
runner.run(); runner.run();
assertNull(processor.ugi); assertNull(processor.ugi);
assertNull(processor.kerberosUserReference.get()); assertNull(processor.kerberosUserReference.get());
@ -1128,6 +1129,10 @@ public class TestPutHive3Streaming {
new FieldSchema("scale", serdeConstants.DOUBLE_TYPE_NAME, "") new FieldSchema("scale", serdeConstants.DOUBLE_TYPE_NAME, "")
); );
private MockPutHive3Streaming(UserGroupInformation ugi) {
this.ugi = ugi;
}
@Override @Override
StreamingConnection makeStreamingConnection(HiveOptions options, RecordReader reader) throws StreamingException { StreamingConnection makeStreamingConnection(HiveOptions options, RecordReader reader) throws StreamingException {
@ -1175,6 +1180,11 @@ public class TestPutHive3Streaming {
public void setGeneratePermissionsFailure(boolean generatePermissionsFailure) { public void setGeneratePermissionsFailure(boolean generatePermissionsFailure) {
this.generatePermissionsFailure = generatePermissionsFailure; this.generatePermissionsFailure = generatePermissionsFailure;
} }
@Override
UserGroupInformation getUgi() {
return ugi;
}
} }
private class MockHiveStreamingConnection implements StreamingConnection { private class MockHiveStreamingConnection implements StreamingConnection {