hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From te...@apache.org
Subject hbase git commit: HBASE-14553 [0.98] Per column family DFS Replication doesn't work against hadoop-1
Date Tue, 06 Oct 2015 01:20:08 GMT
Repository: hbase
Updated Branches:
  refs/heads/0.98 fdaecc89f -> 20ebd5b65


HBASE-14553 [0.98] Per column family DFS Replication doesn't work against hadoop-1


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/20ebd5b6
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/20ebd5b6
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/20ebd5b6

Branch: refs/heads/0.98
Commit: 20ebd5b653c8534faf4ddb7d339637c1f0e43f04
Parents: fdaecc8
Author: tedyu <yuzhihong@gmail.com>
Authored: Mon Oct 5 18:20:04 2015 -0700
Committer: tedyu <yuzhihong@gmail.com>
Committed: Mon Oct 5 18:20:04 2015 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hbase/util/FSUtils.java    | 6 ++++--
 .../test/java/org/apache/hadoop/hbase/client/TestAdmin1.java   | 4 ++--
 2 files changed, 6 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/20ebd5b6/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
----------------------------------------------------------------------
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);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/20ebd5b6/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
----------------------------------------------------------------------
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 {


Mime
View raw message