hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From syuanji...@apache.org
Subject [15/50] [abbrv] hbase git commit: HBASE-15563 'counter' may overflow in BoundedGroupingStrategy (Matt Warhaftig)
Date Wed, 11 May 2016 04:57:47 GMT
HBASE-15563 'counter' may overflow in BoundedGroupingStrategy (Matt Warhaftig)


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

Branch: refs/heads/hbase-12439
Commit: efcf94def7d1b0a93bc9f0693719a0648e0c8f7a
Parents: c0ce47d
Author: tedyu <yuzhihong@gmail.com>
Authored: Thu May 5 10:28:33 2016 -0700
Committer: tedyu <yuzhihong@gmail.com>
Committed: Thu May 5 10:28:33 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/wal/BoundedGroupingStrategy.java     | 14 +++++++++++++-
 1 file changed, 13 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/efcf94de/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedGroupingStrategy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedGroupingStrategy.java
b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedGroupingStrategy.java
index 65c774e..06f8792 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedGroupingStrategy.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedGroupingStrategy.java
@@ -46,7 +46,7 @@ public class BoundedGroupingStrategy implements RegionGroupingStrategy{
     String idStr = Bytes.toString(identifier);
     String groupName = groupNameCache.get(idStr);
     if (null == groupName) {
-      groupName = groupNames[counter.getAndIncrement() % groupNames.length];
+      groupName = groupNames[getAndIncrAtomicInteger(counter, groupNames.length)];
       String extantName = groupNameCache.putIfAbsent(idStr, groupName);
       if (extantName != null) {
         return extantName;
@@ -55,6 +55,18 @@ public class BoundedGroupingStrategy implements RegionGroupingStrategy{
     return groupName;
   }
 
+  // Non-blocking incrementing & resetting of AtomicInteger.
+  private int getAndIncrAtomicInteger(AtomicInteger atomicInt, int reset) {
+    for (;;) {
+      int current = atomicInt.get();
+      int next = (current + 1);
+      if (next == reset) {
+        next = 0;
+      }
+      if (atomicInt.compareAndSet(current, next)) return current;
+    }
+  }
+
   @Override
   public void init(Configuration config, String providerId) {
     int regionGroupNumber = config.getInt(NUM_REGION_GROUPS, DEFAULT_NUM_REGION_GROUPS);


Mime
View raw message