HBASE-19239 Fix findbugs and error-prone issues

Fixes for hbase-metrics
This commit is contained in:
Andrew Purtell 2017-11-15 18:47:45 -08:00
parent 24feb1ab75
commit 357b3dbc3e
3 changed files with 16 additions and 7 deletions

View File

@ -67,6 +67,7 @@ public class HistogramImpl implements Histogram {
histogram.add(value, 1);
}
@Override
public long getCount() {
return counter.getCount();
}
@ -75,6 +76,7 @@ public class HistogramImpl implements Histogram {
return this.histogram.getMax();
}
@Override
public Snapshot snapshot() {
return histogram.snapshotAndReset();
}

View File

@ -60,22 +60,27 @@ public class MetricRegistriesImpl extends MetricRegistries {
});
}
@Override
public boolean remove(MetricRegistryInfo key) {
return registries.remove(key) == null;
}
@Override
public Optional<MetricRegistry> get(MetricRegistryInfo info) {
return Optional.fromNullable(registries.get(info));
}
@Override
public Collection<MetricRegistry> getMetricRegistries() {
return Collections.unmodifiableCollection(registries.values());
}
@Override
public void clear() {
registries.clear();
}
@Override
public Set<MetricRegistryInfo> getMetricRegistryInfos() {
return Collections.unmodifiableSet(registries.keySet());
}

View File

@ -24,7 +24,7 @@ import java.util.ArrayList;
import java.util.Collection;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
@ -42,22 +42,24 @@ class RefCountingMap<K, V> {
private ConcurrentHashMap<K, Payload<V>> map = new ConcurrentHashMap<>();
private static class Payload<V> {
V v;
volatile int refCount;
final AtomicInteger refCount = new AtomicInteger(1); // create with ref count = 1
Payload(V v) {
this.v = v;
this.refCount = 1; // create with ref count = 1
}
}
@edu.umd.cs.findbugs.annotations.SuppressWarnings(
value="AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION",
justification="We use the object monitor to serialize operations on the concurrent map")
V put(K key, Supplier<V> supplier) {
synchronized (map) {
synchronized (this) {
Payload<V> oldValue = map.get(key);
if (oldValue == null) {
oldValue = new Payload<V>(supplier.get());
map.put(key, oldValue);
return oldValue.v;
}
oldValue.refCount++;
oldValue.refCount.incrementAndGet();
return oldValue.v;
}
}
@ -73,10 +75,10 @@ class RefCountingMap<K, V> {
* @return the value associated with the specified key or null if key is removed from map.
*/
V remove(K key) {
synchronized (map) {
synchronized (this) {
Payload<V> oldValue = map.get(key);
if (oldValue != null) {
if (--oldValue.refCount == 0) {
if (oldValue.refCount.decrementAndGet() == 0) {
map.remove(key);
return null;
}