HADOOP-15571. Multiple FileContexts created with the same configuration object should be allowed to have different umask. Contributed by Vinod Kumar Vavilapalli.
This commit is contained in:
parent
a129e3e74e
commit
498e3bfb6b
|
@ -219,10 +219,12 @@ public class FileContext {
|
||||||
* The FileContext is defined by.
|
* The FileContext is defined by.
|
||||||
* 1) defaultFS (slash)
|
* 1) defaultFS (slash)
|
||||||
* 2) wd
|
* 2) wd
|
||||||
* 3) umask (Obtained by FsPermission.getUMask(conf))
|
* 3) umask (explicitly set via setUMask(),
|
||||||
|
* falling back to FsPermission.getUMask(conf))
|
||||||
*/
|
*/
|
||||||
private final AbstractFileSystem defaultFS; //default FS for this FileContext.
|
private final AbstractFileSystem defaultFS; //default FS for this FileContext.
|
||||||
private Path workingDir; // Fully qualified
|
private Path workingDir; // Fully qualified
|
||||||
|
private FsPermission umask;
|
||||||
private final Configuration conf;
|
private final Configuration conf;
|
||||||
private final UserGroupInformation ugi;
|
private final UserGroupInformation ugi;
|
||||||
final boolean resolveSymlinks;
|
final boolean resolveSymlinks;
|
||||||
|
@ -575,7 +577,7 @@ public class FileContext {
|
||||||
* @return the umask of this FileContext
|
* @return the umask of this FileContext
|
||||||
*/
|
*/
|
||||||
public FsPermission getUMask() {
|
public FsPermission getUMask() {
|
||||||
return FsPermission.getUMask(conf);
|
return (umask != null ? umask : FsPermission.getUMask(conf));
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -583,10 +585,9 @@ public class FileContext {
|
||||||
* @param newUmask the new umask
|
* @param newUmask the new umask
|
||||||
*/
|
*/
|
||||||
public void setUMask(final FsPermission newUmask) {
|
public void setUMask(final FsPermission newUmask) {
|
||||||
FsPermission.setUMask(conf, newUmask);
|
this.umask = newUmask;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Resolve the path following any symlinks or mount points
|
* Resolve the path following any symlinks or mount points
|
||||||
* @param f to be resolved
|
* @param f to be resolved
|
||||||
|
|
|
@ -17,13 +17,17 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.fs;
|
package org.apache.hadoop.fs;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.fail;
|
||||||
|
|
||||||
|
import java.net.URI;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.permission.FsPermission;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import static org.junit.Assert.fail;
|
|
||||||
|
|
||||||
public class TestFileContext {
|
public class TestFileContext {
|
||||||
private static final Logger LOG = LoggerFactory.getLogger(TestFileContext
|
private static final Logger LOG = LoggerFactory.getLogger(TestFileContext
|
||||||
.class);
|
.class);
|
||||||
|
@ -39,4 +43,40 @@ public class TestFileContext {
|
||||||
LOG.info("Expected exception: ", ufse);
|
LOG.info("Expected exception: ", ufse);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testConfBasedAndAPIBasedSetUMask() throws Exception {
|
||||||
|
|
||||||
|
Configuration conf = new Configuration();
|
||||||
|
|
||||||
|
String defaultlUMask =
|
||||||
|
conf.get(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY);
|
||||||
|
assertEquals("Default UMask changed!", "022", defaultlUMask);
|
||||||
|
|
||||||
|
URI uri1 = new URI("file://mydfs:50070/");
|
||||||
|
URI uri2 = new URI("file://tmp");
|
||||||
|
|
||||||
|
FileContext fc1 = FileContext.getFileContext(uri1, conf);
|
||||||
|
FileContext fc2 = FileContext.getFileContext(uri2, conf);
|
||||||
|
assertEquals("Umask for fc1 is incorrect", 022, fc1.getUMask().toShort());
|
||||||
|
assertEquals("Umask for fc2 is incorrect", 022, fc2.getUMask().toShort());
|
||||||
|
|
||||||
|
// Till a user explicitly calls FileContext.setUMask(), the updates through
|
||||||
|
// configuration should be reflected..
|
||||||
|
conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "011");
|
||||||
|
assertEquals("Umask for fc1 is incorrect", 011, fc1.getUMask().toShort());
|
||||||
|
assertEquals("Umask for fc2 is incorrect", 011, fc2.getUMask().toShort());
|
||||||
|
|
||||||
|
// Stop reflecting the conf update for specific FileContexts, once an
|
||||||
|
// explicit setUMask is done.
|
||||||
|
conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "066");
|
||||||
|
fc1.setUMask(FsPermission.createImmutable((short) 00033));
|
||||||
|
assertEquals("Umask for fc1 is incorrect", 033, fc1.getUMask().toShort());
|
||||||
|
assertEquals("Umask for fc2 is incorrect", 066, fc2.getUMask().toShort());
|
||||||
|
|
||||||
|
conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "077");
|
||||||
|
fc2.setUMask(FsPermission.createImmutable((short) 00044));
|
||||||
|
assertEquals("Umask for fc1 is incorrect", 033, fc1.getUMask().toShort());
|
||||||
|
assertEquals("Umask for fc2 is incorrect", 044, fc2.getUMask().toShort());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -96,9 +96,6 @@ public class AggregatedLogFormat {
|
||||||
*/
|
*/
|
||||||
private static final FsPermission APP_LOG_FILE_UMASK = FsPermission
|
private static final FsPermission APP_LOG_FILE_UMASK = FsPermission
|
||||||
.createImmutable((short) (0640 ^ 0777));
|
.createImmutable((short) (0640 ^ 0777));
|
||||||
/** Default permission for the log file. */
|
|
||||||
private static final FsPermission APP_LOG_FILE_PERM =
|
|
||||||
FsPermission.getFileDefault().applyUMask(APP_LOG_FILE_UMASK);
|
|
||||||
|
|
||||||
static {
|
static {
|
||||||
RESERVED_KEYS = new HashMap<String, AggregatedLogFormat.LogKey>();
|
RESERVED_KEYS = new HashMap<String, AggregatedLogFormat.LogKey>();
|
||||||
|
@ -477,10 +474,11 @@ public class AggregatedLogFormat {
|
||||||
@Override
|
@Override
|
||||||
public FSDataOutputStream run() throws Exception {
|
public FSDataOutputStream run() throws Exception {
|
||||||
fc = FileContext.getFileContext(remoteAppLogFile.toUri(), conf);
|
fc = FileContext.getFileContext(remoteAppLogFile.toUri(), conf);
|
||||||
|
fc.setUMask(APP_LOG_FILE_UMASK);
|
||||||
return fc.create(
|
return fc.create(
|
||||||
remoteAppLogFile,
|
remoteAppLogFile,
|
||||||
EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE),
|
EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE),
|
||||||
Options.CreateOpts.perms(APP_LOG_FILE_PERM));
|
new Options.CreateOpts[] {});
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
} catch (InterruptedException e) {
|
} catch (InterruptedException e) {
|
||||||
|
|
Loading…
Reference in New Issue