diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java index 001c510..d0443b8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java @@ -302,11 +302,11 @@ public abstract class FSUtils { FsPermission perm, InetSocketAddress[] favoredNodes) throws IOException { if (fs instanceof HFileSystem) { FileSystem backingFs = ((HFileSystem)fs).getBackingFs(); + short replication = Short.parseShort(conf.get(HColumnDescriptor.DFS_REPLICATION, + String.valueOf(HColumnDescriptor.DEFAULT_DFS_REPLICATION))); if (backingFs instanceof DistributedFileSystem) { // Try to use the favoredNodes version via reflection to allow backwards- // compatibility. - short replication = Short.parseShort(conf.get(HColumnDescriptor.DFS_REPLICATION, - String.valueOf(HColumnDescriptor.DEFAULT_DFS_REPLICATION))); try { return (FSDataOutputStream) (DistributedFileSystem.class.getDeclaredMethod("create", Path.class, FsPermission.class, boolean.class, int.class, short.class, long.class, @@ -328,6 +328,8 @@ public abstract class FSUtils { LOG.debug("Ignoring (most likely Reflection related exception) " + e); } } + return fs.create(path, perm, true, getDefaultBufferSize(fs), replication > 0 ? replication + : getDefaultReplication(backingFs, path), getDefaultBlockSize(fs, path), null); } return create(fs, path, perm, true); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java index 936b68f..c5e1bfb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java @@ -1242,8 +1242,8 @@ public class TestAdmin1 { store = r.getStore(Bytes.toBytes(fn1)); for (StoreFile sf : store.getStorefiles()) { assertTrue(sf.toString().contains(fn1)); - assertTrue("Column family " + fn1 + " should have only 1 copy", 1 == sf.getFileInfo() - .getFileStatus().getReplication()); + short rep = sf.getFileInfo().getFileStatus().getReplication(); + assertTrue("Column family " + fn1 + " should have only 1 copy. But: " + rep, 1 == rep); } } } finally {