NIFI-4311 Allowing umask to get set properly before initializing the FileSystem

Signed-off-by: Pierre Villard <pierre.villard.fr@gmail.com>

This closes #2106.
This commit is contained in:
Bryan Bende 2017-08-22 14:19:14 -04:00 committed by Pierre Villard
parent 05700a2008
commit cf57639396
3 changed files with 30 additions and 17 deletions

View File

@ -252,6 +252,9 @@ public abstract class AbstractHadoopProcessor extends AbstractProcessor {
getConfigurationFromResources(config, configResources);
// give sub-classes a chance to process configuration
preProcessConfiguration(config, context);
// first check for timeout on HDFS connection, because FileSystem has a hard coded 15 minute timeout
checkHdfsUriForTimeout(config);
@ -287,6 +290,17 @@ public abstract class AbstractHadoopProcessor extends AbstractProcessor {
return new HdfsResources(config, fs, ugi);
}
/**
* This method will be called after the Configuration has been created, but before the FileSystem is created,
* allowing sub-classes to take further action on the Configuration before creating the FileSystem.
*
* @param config the Configuration that will be used to create the FileSystem
* @param context the context that can be used to retrieve additional values
*/
protected void preProcessConfiguration(final Configuration config, final ProcessContext context) {
}
/**
* This exists in order to allow unit tests to override it so that they don't take several minutes waiting for UDP packets to be received
*

View File

@ -210,14 +210,8 @@ public abstract class AbstractPutHDFSRecord extends AbstractHadoopProcessor {
return putHdfsRecordProperties;
}
@OnScheduled
public final void onScheduled(final ProcessContext context) throws IOException {
super.abstractOnScheduled(context);
this.remoteOwner = context.getProperty(REMOTE_OWNER).getValue();
this.remoteGroup = context.getProperty(REMOTE_GROUP).getValue();
@Override
protected void preProcessConfiguration(Configuration config, ProcessContext context) {
// Set umask once, to avoid thread safety issues doing it in onTrigger
final PropertyValue umaskProp = context.getProperty(UMASK);
final short dfsUmask;
@ -226,8 +220,16 @@ public abstract class AbstractPutHDFSRecord extends AbstractHadoopProcessor {
} else {
dfsUmask = FsPermission.DEFAULT_UMASK;
}
final Configuration conf = getConfiguration();
FsPermission.setUMask(conf, new FsPermission(dfsUmask));
FsPermission.setUMask(config, new FsPermission(dfsUmask));
}
@OnScheduled
public final void onScheduled(final ProcessContext context) throws IOException {
super.abstractOnScheduled(context);
this.remoteOwner = context.getProperty(REMOTE_OWNER).getValue();
this.remoteGroup = context.getProperty(REMOTE_GROUP).getValue();
}
/**

View File

@ -32,7 +32,6 @@ import org.apache.nifi.annotation.behavior.WritesAttributes;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.SeeAlso;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.components.AllowableValue;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.PropertyValue;
@ -188,10 +187,8 @@ public class PutHDFS extends AbstractHadoopProcessor {
return props;
}
@OnScheduled
public void onScheduled(ProcessContext context) throws Exception {
super.abstractOnScheduled(context);
@Override
protected void preProcessConfiguration(final Configuration config, final ProcessContext context) {
// Set umask once, to avoid thread safety issues doing it in onTrigger
final PropertyValue umaskProp = context.getProperty(UMASK);
final short dfsUmask;
@ -200,8 +197,8 @@ public class PutHDFS extends AbstractHadoopProcessor {
} else {
dfsUmask = FsPermission.DEFAULT_UMASK;
}
final Configuration conf = getConfiguration();
FsPermission.setUMask(conf, new FsPermission(dfsUmask));
FsPermission.setUMask(config, new FsPermission(dfsUmask));
}
@Override