You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2017/01/20 00:21:23 UTC

hbase git commit: HBASE-17494 Guard against cloning family of all cells if no data need be replicated (ChiaPing Tsai)

Repository: hbase
Updated Branches:
  refs/heads/branch-1 0e06ade69 -> abc8c5edd


HBASE-17494 Guard against cloning family of all cells if no data need be replicated (ChiaPing Tsai)


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

Branch: refs/heads/branch-1
Commit: abc8c5edd3035216526b76a9000595fb19d90a6e
Parents: 0e06ade
Author: tedyu <yu...@gmail.com>
Authored: Thu Jan 19 16:21:17 2017 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Thu Jan 19 16:21:17 2017 -0800

----------------------------------------------------------------------
 .../replication/regionserver/Replication.java    | 19 +++++++++++++++----
 1 file changed, 15 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/abc8c5ed/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
index 3465cf3..991eb2f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
@@ -70,6 +70,7 @@ import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
 import org.apache.zookeeper.KeeperException;
 
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import java.util.Collection;
 
 /**
  * Gateway to Replication.  Used by {@link org.apache.hadoop.hbase.regionserver.HRegionServer}.
@@ -266,6 +267,15 @@ public class Replication extends WALActionsListener.Base implements
     scopeWALEdits(htd, logKey, logEdit, this.conf, this.getReplicationManager());
   }
 
+  private static boolean hasReplication(Collection<HColumnDescriptor> families) {
+    for (HColumnDescriptor col : families) {
+      if (col.getScope() != REPLICATION_SCOPE_LOCAL) {
+        return true;
+      }
+    }
+    return false;
+  }
+
   /**
    * Utility method used to set the correct scopes on each log key. Doesn't set a scope on keys from
    * compaction WAL edits and if the scope is local.
@@ -278,8 +288,9 @@ public class Replication extends WALActionsListener.Base implements
   public static void scopeWALEdits(HTableDescriptor htd, WALKey logKey, WALEdit logEdit,
       Configuration conf, ReplicationSourceManager replicationManager) throws IOException {
     NavigableMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
-    byte[] family;
     boolean replicationForBulkLoadEnabled = isReplicationForBulkLoadDataEnabled(conf);
+    Collection<HColumnDescriptor> families = htd.getFamilies();
+    boolean hasReplication = hasReplication(families);
     for (Cell cell : logEdit.getCells()) {
       if (CellUtil.matchingFamily(cell, WALEdit.METAFAMILY)) {
         if (replicationForBulkLoadEnabled && CellUtil.matchingQualifier(cell, WALEdit.BULK_LOAD)) {
@@ -289,7 +300,7 @@ public class Replication extends WALActionsListener.Base implements
           if (maybeEvent != null && (maybeEvent.getEventType() ==
               WALProtos.RegionEventDescriptor.EventType.REGION_CLOSE)) {
             // In serially replication, we use scopes when reading close marker.
-            for (HColumnDescriptor cf :htd.getFamilies()) {
+            for (HColumnDescriptor cf : families) {
               if (cf.getScope() != REPLICATION_SCOPE_LOCAL) {
                 scopes.put(cf.getName(), cf.getScope());
               }
@@ -299,8 +310,8 @@ public class Replication extends WALActionsListener.Base implements
           continue;
 
         }
-      } else {
-        family = CellUtil.cloneFamily(cell);
+      } else if (hasReplication) {
+        byte[] family = CellUtil.cloneFamily(cell);
         // Unexpected, has a tendency to happen in unit tests
         assert htd.getFamily(family) != null;