Index: hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java (revision 1590760) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java (working copy) @@ -89,7 +89,7 @@ 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"; + public static final String FULL_RWX_PERMISSIONS = "777"; private static final String THREAD_POOLSIZE = "hbase.client.localityCheck.threadPoolSize"; private static final int DEFAULT_THREAD_POOLSIZE = 2; @@ -292,7 +292,7 @@ .getDeclaredMethod("create", Path.class, FsPermission.class, boolean.class, int.class, short.class, long.class, Progressable.class, InetSocketAddress[].class) - .invoke(backingFs, path, FsPermission.getDefault(), true, + .invoke(backingFs, path, perm, true, getDefaultBufferSize(backingFs), getDefaultReplication(backingFs, path), getDefaultBlockSize(backingFs, path), @@ -365,7 +365,7 @@ // make sure that we have a mask, if not, go default. String mask = conf.get(permssionConfKey); if (mask == null) - return FsPermission.getDefault(); + return getFileDefault(); // appy the umask FsPermission umask = new FsPermission(mask); return perm.applyUMask(umask); @@ -374,13 +374,26 @@ "Incorrect umask attempted to be created: " + conf.get(permssionConfKey) + ", using default file permissions.", e); - return FsPermission.getDefault(); + return getFileDefault(); } } - return FsPermission.getDefault(); + return getFileDefault(); } /** + * Get the default permission for file. + * This is the same method as FsPermission.getFileDefault() in Hadoop 2. + * We provide the method here to support compatibility with Hadoop 1. + * See HBASE-11061. Would be better to do this as Interface in hadoop-compat + * w/ hadoop1 and hadoop2 implementations but punting on this since small + * risk this will change in 0.96/0.98 timeframe (only committed to these + * branches). + */ + public static FsPermission getFileDefault() { + return new FsPermission((short)00666); + } + + /** * Checks to see if the specified file system is available * * @param fs filesystem @@ -1911,4 +1924,4 @@ int hbaseSize = conf.getInt("hbase." + dfsKey, defaultSize); conf.setIfUnset(dfsKey, Integer.toString(hbaseSize)); } -} \ No newline at end of file +} Index: hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java =================================================================== --- hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java (revision 1590760) +++ hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java (working copy) @@ -232,12 +232,23 @@ public void testPermMask() throws Exception { Configuration conf = HBaseConfiguration.create(); + FileSystem fs = FileSystem.get(conf); + + // default fs permission + FsPermission defaultFsPerm = FSUtils.getFilePermissions(fs, conf, + HConstants.DATA_FILE_UMASK_KEY); + // 'hbase.data.umask.enable' is false. We will get default fs permission. + assertEquals(FSUtils.getFileDefault(), defaultFsPerm); + 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, + FsPermission defaultStartPerm = FSUtils.getFilePermissions(fs, conf, HConstants.DATA_FILE_UMASK_KEY); - assertEquals(FsPermission.getDefault(), defaultPerms); + // default 'hbase.data.umask'is 000, and this umask will be used when + // 'hbase.data.umask.enable' is true. + // Therefore we will not get the real fs default in this case. + // Instead we will get the starting point FULL_RWX_PERMISSIONS + assertEquals(new FsPermission(FSUtils.FULL_RWX_PERMISSIONS), defaultStartPerm); conf.setStrings(HConstants.DATA_FILE_UMASK_KEY, "077"); // now check that we get the right perms