HBASE-5526 ^Cnfigurable file and directory based umask (Jesse Yates)

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1298657 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
larsh 2012-03-08 23:58:17 +00:00
parent 961455cd9d
commit 7ce77fc5d3
6 changed files with 164 additions and 23 deletions

View File

@ -639,6 +639,11 @@ public final class HConstants {
public static final String CHECKSUM_TYPE_NAME =
"hbase.hstore.checksum.algorithm";
/** Enable file permission modification from standard hbase */
public static final String ENABLE_DATA_FILE_UMASK = "hbase.data.umask.enable";
/** File permission umask to use when creating hbase data files */
public static final String DATA_FILE_UMASK_KEY = "hbase.data.umask";
private HConstants() {
// Can't be instantiated with this ctor.
}

View File

@ -30,10 +30,12 @@ import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue.KeyComparator;
import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
import org.apache.hadoop.hbase.regionserver.metrics.SchemaConfigured;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.io.RawComparator;
import org.apache.hadoop.io.Writable;
@ -266,9 +268,8 @@ public abstract class AbstractHFileWriter extends SchemaConfigured
/** A helper method to create HFile output streams in constructors */
protected static FSDataOutputStream createOutputStream(Configuration conf,
FileSystem fs, Path path) throws IOException {
return fs.create(path, FsPermission.getDefault(), true,
fs.getConf().getInt("io.file.buffer.size", 4096),
fs.getDefaultReplication(), fs.getDefaultBlockSize(),
null);
FsPermission perms = FSUtils.getFilePermissions(fs, conf,
HConstants.DATA_FILE_UMASK_KEY);
return FSUtils.create(fs, path, perms);
}
}

View File

@ -67,6 +67,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.DroppedSnapshotException;
import org.apache.hadoop.hbase.HBaseConfiguration;
@ -769,8 +770,15 @@ public class HRegion implements HeapSize { // , Writable{
// create but before close. If we don't successfully close the file,
// subsequent region reopens will fail the below because create is
// registered in NN.
// first check to get the permissions
FsPermission perms = FSUtils.getFilePermissions(fs, conf,
HConstants.DATA_FILE_UMASK_KEY);
// and then create the file
Path tmpPath = new Path(getTmpDir(), REGIONINFO_FILE);
FSDataOutputStream out = this.fs.create(tmpPath, true);
FSDataOutputStream out = FSUtils.create(fs, tmpPath, perms);
try {
this.regionInfo.write(out);
out.write('\n');

View File

@ -42,6 +42,7 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
import org.apache.hadoop.hbase.HRegionInfo;
@ -61,6 +62,9 @@ import org.apache.hadoop.util.StringUtils;
public abstract class FSUtils {
private static final Log LOG = LogFactory.getLog(FSUtils.class);
/** Full access permissions (starting point for a umask) */
private static final String FULL_RWX_PERMISSIONS = "777";
protected FSUtils() {
super();
}
@ -105,21 +109,91 @@ public abstract class FSUtils {
}
/**
* Create file.
* @param fs filesystem object
* @param p path to create
* @return Path
* @throws IOException e
* Create the specified file on the filesystem. By default, this will:
* <ol>
* <li>overwrite the file if it exists</li>
* <li>apply the umask in the configuration (if it is enabled)</li>
* <li>use the fs configured buffer size (or {@value DEFAULT_BUFFER_SIZE} if
* not set)</li>
* <li>use the default replication</li>
* <li>use the default block size</li>
* <li>not track progress</li>
* </ol>
*
* @param fs {@link FileSystem} on which to write the file
* @param path {@link Path} to the file to write
* @return output stream to the created file
* @throws IOException if the file cannot be created
*/
public static Path create(final FileSystem fs, final Path p)
throws IOException {
if (fs.exists(p)) {
throw new IOException("File already exists " + p.toString());
public static FSDataOutputStream create(FileSystem fs, Path path,
FsPermission perm) throws IOException {
return create(fs, path, perm, true);
}
if (!fs.createNewFile(p)) {
throw new IOException("Failed create of " + p);
/**
* Create the specified file on the filesystem. By default, this will:
* <ol>
* <li>apply the umask in the configuration (if it is enabled)</li>
* <li>use the fs configured buffer size (or {@value DEFAULT_BUFFER_SIZE} if
* not set)</li>
* <li>use the default replication</li>
* <li>use the default block size</li>
* <li>not track progress</li>
* </ol>
*
* @param fs {@link FileSystem} on which to write the file
* @param path {@link Path} to the file to write
* @param perm
* @param overwrite Whether or not the created file should be overwritten.
* @return output stream to the created file
* @throws IOException if the file cannot be created
*/
public static FSDataOutputStream create(FileSystem fs, Path path,
FsPermission perm, boolean overwrite) throws IOException {
LOG.debug("Creating file:" + path + "with permission:" + perm);
return fs.create(path, perm, overwrite,
fs.getConf().getInt("io.file.buffer.size", 4096),
fs.getDefaultReplication(), fs.getDefaultBlockSize(), null);
}
return p;
/**
* Get the file permissions specified in the configuration, if they are
* enabled.
*
* @param fs filesystem that the file will be created on.
* @param conf configuration to read for determining if permissions are
* enabled and which to use
* @param permssionConfKey property key in the configuration to use when
* finding the permission
* @return the permission to use when creating a new file on the fs. If
* special permissions are not specified in the configuration, then
* the default permissions on the the fs will be returned.
*/
public static FsPermission getFilePermissions(final FileSystem fs,
final Configuration conf, final String permssionConfKey) {
boolean enablePermissions = conf.getBoolean(
HConstants.ENABLE_DATA_FILE_UMASK, false);
if (enablePermissions) {
try {
FsPermission perm = new FsPermission(FULL_RWX_PERMISSIONS);
// make sure that we have a mask, if not, go default.
String mask = conf.get(permssionConfKey);
if (mask == null)
return FsPermission.getDefault();
// appy the umask
FsPermission umask = new FsPermission(mask);
return perm.applyUMask(umask);
} catch (IllegalArgumentException e) {
LOG.warn(
"Incorrect umask attempted to be created: "
+ conf.get(permssionConfKey)
+ ", using default file permissions.", e);
return FsPermission.getDefault();
}
}
return FsPermission.getDefault();
}
/**

View File

@ -869,4 +869,18 @@
value to 0.
</description>
</property>
<property>
<name>hbase.data.umask.enable</name>
<value>false</value>
<description>Enable, if true, that file permissions should be assigned
to the files written by the regionserver
</description>
</property>
<property>
<name>hbase.data.umask</name>
<value>000</value>
<description>File permissions that should be used to write data
files when hbase.data.umask.enable is true
</description>
</property>
</configuration>

View File

@ -19,14 +19,24 @@
*/
package org.apache.hadoop.hbase.util;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import org.apache.hadoop.hbase.*;
import java.io.File;
import java.util.UUID;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@ -139,13 +149,42 @@ public class TestFSUtils {
FileStatus status = fs.getFileStatus(testFile);
HDFSBlocksDistribution blocksDistribution =
FSUtils.computeHDFSBlocksDistribution(fs, status, 0, status.getLen());
assertTrue(blocksDistribution.getTopHosts().size() == 3);
assertEquals("Wrong number of hosts distributing blocks.", 3,
blocksDistribution.getTopHosts().size());
} finally {
htu.shutdownMiniDFSCluster();
}
}
@Test
public void testPermMask() throws Exception {
Configuration conf = HBaseConfiguration.create();
conf.setBoolean(HConstants.ENABLE_DATA_FILE_UMASK, true);
FileSystem fs = FileSystem.get(conf);
// first check that we don't crash if we don't have perms set
FsPermission defaultPerms = FSUtils.getFilePermissions(fs, conf,
HConstants.DATA_FILE_UMASK_KEY);
assertEquals(FsPermission.getDefault(), defaultPerms);
conf.setStrings(HConstants.DATA_FILE_UMASK_KEY, "077");
// now check that we get the right perms
FsPermission filePerm = FSUtils.getFilePermissions(fs, conf,
HConstants.DATA_FILE_UMASK_KEY);
assertEquals(new FsPermission("700"), filePerm);
// then that the correct file is created
Path p = new Path("target" + File.separator + UUID.randomUUID().toString());
try {
FSDataOutputStream out = FSUtils.create(fs, p, filePerm);
out.close();
FileStatus stat = fs.getFileStatus(p);
assertEquals(new FsPermission("700"), stat.getPermission());
// and then cleanup
} finally {
fs.delete(p, true);
}
}
@org.junit.Rule
public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =