From 72783ffcf2672519b1842f69948d74c441297ee9 Mon Sep 17 00:00:00 2001 From: Matt Warhaftig Date: Tue, 3 May 2016 21:38:15 -0400 Subject: [PATCH] HBASE-15563 Reset BoundedGroupingStrategy counter to prevent overflow. --- .../apache/hadoop/hbase/wal/BoundedGroupingStrategy.java | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) 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); -- 2.6.4 (Apple Git-63)