Repository: hbase Updated Branches: refs/heads/branch-2 41ff7d4c6 -> 0501855bc
http://git-wip-us.apache.org/repos/asf/hbase/blob/0501855b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/IncrementCoalescer.java ---------------------------------------------------------------------- diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/IncrementCoalescer.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/IncrementCoalescer.java index 221786a..edd251d 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/IncrementCoalescer.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/IncrementCoalescer.java @@ -30,7 +30,7 @@ import java.util.concurrent.ThreadFactory; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.LongAdder; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -150,9 +150,9 @@ public class IncrementCoalescer implements IncrementCoalescerMBean { } } - private final AtomicLong failedIncrements = new AtomicLong(); - private final AtomicLong successfulCoalescings = new AtomicLong(); - private final AtomicLong totalIncrements = new AtomicLong(); + private final LongAdder failedIncrements = new LongAdder(); + private final LongAdder successfulCoalescings = new LongAdder(); + private final LongAdder totalIncrements = new LongAdder(); private final ConcurrentMap<FullyQualifiedRow, Long> countersMap = new ConcurrentHashMap<>(100000, 0.75f, 1500); private final ThreadPoolExecutor pool; @@ -176,7 +176,7 @@ public class IncrementCoalescer implements IncrementCoalescerMBean { public boolean queueIncrement(TIncrement inc) throws TException { if (!canQueue()) { - failedIncrements.incrementAndGet(); + failedIncrements.increment(); return false; } return internalQueueTincrement(inc); @@ -184,7 +184,7 @@ public class IncrementCoalescer implements IncrementCoalescerMBean { public boolean queueIncrements(List<TIncrement> incs) throws TException { if (!canQueue()) { - failedIncrements.incrementAndGet(); + failedIncrements.increment(); return false; } @@ -211,7 +211,7 @@ public class IncrementCoalescer implements IncrementCoalescerMBean { //Make sure that the number of threads is scaled. dynamicallySetCoreSize(countersMapSize); - totalIncrements.incrementAndGet(); + totalIncrements.increment(); FullyQualifiedRow key = new FullyQualifiedRow(tableName, rowKey, fam, qual); @@ -224,7 +224,7 @@ public class IncrementCoalescer implements IncrementCoalescerMBean { value = Long.valueOf(currentAmount); } else { value += currentAmount; - successfulCoalescings.incrementAndGet(); + successfulCoalescings.increment(); } // Try to put the value, only if there was none Long oldValue = countersMap.putIfAbsent(key, value); @@ -354,15 +354,15 @@ public class IncrementCoalescer implements IncrementCoalescerMBean { pool.setMaximumPoolSize(newMaxSize); } public long getFailedIncrements() { - return failedIncrements.get(); + return failedIncrements.sum(); } public long getSuccessfulCoalescings() { - return successfulCoalescings.get(); + return successfulCoalescings.sum(); } public long getTotalIncrements() { - return totalIncrements.get(); + return totalIncrements.sum(); } public long getCountersMapSize() {