HDFS-15975. Use LongAdder instead of AtomicLong for branch-3.3 (#2940)
Signed-off-by: Takanobu Asanuma <tasanuma@apache.org>
This commit is contained in:
parent
6649e5888b
commit
f45365f201
@ -23,7 +23,7 @@
|
||||
import org.apache.hadoop.metrics2.MetricsInfo;
|
||||
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
|
||||
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.concurrent.atomic.LongAdder;
|
||||
|
||||
/**
|
||||
* A mutable long counter
|
||||
@ -32,11 +32,11 @@
|
||||
@InterfaceStability.Evolving
|
||||
public class MutableCounterLong extends MutableCounter {
|
||||
|
||||
private AtomicLong value = new AtomicLong();
|
||||
private final LongAdder value = new LongAdder();
|
||||
|
||||
public MutableCounterLong(MetricsInfo info, long initValue) {
|
||||
super(info);
|
||||
this.value.set(initValue);
|
||||
this.value.add(initValue);
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -49,12 +49,12 @@ public void incr() {
|
||||
* @param delta of the increment
|
||||
*/
|
||||
public void incr(long delta) {
|
||||
value.addAndGet(delta);
|
||||
value.add(delta);
|
||||
setChanged();
|
||||
}
|
||||
|
||||
public long value() {
|
||||
return value.get();
|
||||
return value.longValue();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -19,7 +19,7 @@
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.concurrent.atomic.LongAdder;
|
||||
|
||||
/**
|
||||
* The client-side metrics for hedged read feature.
|
||||
@ -28,20 +28,20 @@
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class DFSHedgedReadMetrics {
|
||||
public final AtomicLong hedgedReadOps = new AtomicLong();
|
||||
public final AtomicLong hedgedReadOpsWin = new AtomicLong();
|
||||
public final AtomicLong hedgedReadOpsInCurThread = new AtomicLong();
|
||||
public final LongAdder hedgedReadOps = new LongAdder();
|
||||
public final LongAdder hedgedReadOpsWin = new LongAdder();
|
||||
public final LongAdder hedgedReadOpsInCurThread = new LongAdder();
|
||||
|
||||
public void incHedgedReadOps() {
|
||||
hedgedReadOps.incrementAndGet();
|
||||
hedgedReadOps.increment();
|
||||
}
|
||||
|
||||
public void incHedgedReadOpsInCurThread() {
|
||||
hedgedReadOpsInCurThread.incrementAndGet();
|
||||
hedgedReadOpsInCurThread.increment();
|
||||
}
|
||||
|
||||
public void incHedgedReadWins() {
|
||||
hedgedReadOpsWin.incrementAndGet();
|
||||
hedgedReadOpsWin.increment();
|
||||
}
|
||||
|
||||
public long getHedgedReadOps() {
|
||||
|
@ -26,7 +26,7 @@
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.NoSuchElementException;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.concurrent.atomic.LongAdder;
|
||||
|
||||
/**
|
||||
* This storage statistics tracks how many times each DFS operation was issued.
|
||||
@ -140,21 +140,21 @@ public static OpType fromSymbol(String symbol) {
|
||||
|
||||
public static final String NAME = "DFSOpsCountStatistics";
|
||||
|
||||
private final Map<OpType, AtomicLong> opsCount = new EnumMap<>(OpType.class);
|
||||
private final Map<OpType, LongAdder> opsCount = new EnumMap<>(OpType.class);
|
||||
|
||||
public DFSOpsCountStatistics() {
|
||||
super(NAME);
|
||||
for (OpType opType : OpType.values()) {
|
||||
opsCount.put(opType, new AtomicLong(0));
|
||||
opsCount.put(opType, new LongAdder());
|
||||
}
|
||||
}
|
||||
|
||||
public void incrementOpCounter(OpType op) {
|
||||
opsCount.get(op).addAndGet(1);
|
||||
opsCount.get(op).increment();
|
||||
}
|
||||
|
||||
private class LongIterator implements Iterator<LongStatistic> {
|
||||
private Iterator<Entry<OpType, AtomicLong>> iterator =
|
||||
private final Iterator<Entry<OpType, LongAdder>> iterator =
|
||||
opsCount.entrySet().iterator();
|
||||
|
||||
@Override
|
||||
@ -167,9 +167,9 @@ public LongStatistic next() {
|
||||
if (!iterator.hasNext()) {
|
||||
throw new NoSuchElementException();
|
||||
}
|
||||
final Entry<OpType, AtomicLong> entry = iterator.next();
|
||||
final Entry<OpType, LongAdder> entry = iterator.next();
|
||||
return new LongStatistic(entry.getKey().getSymbol(),
|
||||
entry.getValue().get());
|
||||
entry.getValue().longValue());
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -191,7 +191,7 @@ public Iterator<LongStatistic> getLongStatistics() {
|
||||
@Override
|
||||
public Long getLong(String key) {
|
||||
final OpType type = OpType.fromSymbol(key);
|
||||
return type == null ? null : opsCount.get(type).get();
|
||||
return type == null ? null : opsCount.get(type).longValue();
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -201,8 +201,8 @@ public boolean isTracked(String key) {
|
||||
|
||||
@Override
|
||||
public void reset() {
|
||||
for (AtomicLong count : opsCount.values()) {
|
||||
count.set(0);
|
||||
for (LongAdder count : opsCount.values()) {
|
||||
count.reset();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -42,7 +42,7 @@
|
||||
import java.util.concurrent.ThreadPoolExecutor;
|
||||
import java.util.concurrent.ScheduledThreadPoolExecutor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.concurrent.atomic.LongAdder;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.commons.lang3.time.DurationFormatUtils;
|
||||
@ -120,7 +120,7 @@ public boolean shouldAdvertise() {
|
||||
private final HashMap<ExtendedBlockId, Value> mappableBlockMap =
|
||||
new HashMap<ExtendedBlockId, Value>();
|
||||
|
||||
private final AtomicLong numBlocksCached = new AtomicLong(0);
|
||||
private final LongAdder numBlocksCached = new LongAdder();
|
||||
|
||||
private final FsDatasetImpl dataset;
|
||||
|
||||
@ -143,11 +143,11 @@ public boolean shouldAdvertise() {
|
||||
/**
|
||||
* Number of cache commands that could not be completed successfully
|
||||
*/
|
||||
final AtomicLong numBlocksFailedToCache = new AtomicLong(0);
|
||||
final LongAdder numBlocksFailedToCache = new LongAdder();
|
||||
/**
|
||||
* Number of uncache commands that could not be completed successfully
|
||||
*/
|
||||
final AtomicLong numBlocksFailedToUncache = new AtomicLong(0);
|
||||
final LongAdder numBlocksFailedToUncache = new LongAdder();
|
||||
|
||||
public FsDatasetCache(FsDatasetImpl dataset) throws IOException {
|
||||
this.dataset = dataset;
|
||||
@ -204,7 +204,7 @@ public void initCache(String bpid) throws IOException {
|
||||
for (Map.Entry<ExtendedBlockId, MappableBlock> entry : entrySet) {
|
||||
mappableBlockMap.put(entry.getKey(),
|
||||
new Value(keyToMappableBlock.get(entry.getKey()), State.CACHED));
|
||||
numBlocksCached.addAndGet(1);
|
||||
numBlocksCached.increment();
|
||||
dataset.datanode.getMetrics().incrBlocksCached(1);
|
||||
}
|
||||
}
|
||||
@ -278,7 +278,7 @@ synchronized void cacheBlock(long blockId, String bpid,
|
||||
LOG.debug("Block with id {}, pool {} already exists in the "
|
||||
+ "FsDatasetCache with state {}", blockId, bpid, prevValue.state
|
||||
);
|
||||
numBlocksFailedToCache.incrementAndGet();
|
||||
numBlocksFailedToCache.increment();
|
||||
return;
|
||||
}
|
||||
mappableBlockMap.put(key, new Value(null, State.CACHING));
|
||||
@ -301,7 +301,7 @@ synchronized void uncacheBlock(String bpid, long blockId) {
|
||||
LOG.debug("Block with id {}, pool {} does not need to be uncached, "
|
||||
+ "because it is not currently in the mappableBlockMap.", blockId,
|
||||
bpid);
|
||||
numBlocksFailedToUncache.incrementAndGet();
|
||||
numBlocksFailedToUncache.increment();
|
||||
return;
|
||||
}
|
||||
switch (prevValue.state) {
|
||||
@ -331,7 +331,7 @@ synchronized void uncacheBlock(String bpid, long blockId) {
|
||||
default:
|
||||
LOG.debug("Block with id {}, pool {} does not need to be uncached, "
|
||||
+ "because it is in state {}.", blockId, bpid, prevValue.state);
|
||||
numBlocksFailedToUncache.incrementAndGet();
|
||||
numBlocksFailedToUncache.increment();
|
||||
break;
|
||||
}
|
||||
}
|
||||
@ -469,7 +469,7 @@ public void run() {
|
||||
dataset.datanode.
|
||||
getShortCircuitRegistry().processBlockMlockEvent(key);
|
||||
}
|
||||
numBlocksCached.addAndGet(1);
|
||||
numBlocksCached.increment();
|
||||
dataset.datanode.getMetrics().incrBlocksCached(1);
|
||||
success = true;
|
||||
} finally {
|
||||
@ -482,7 +482,7 @@ public void run() {
|
||||
LOG.debug("Caching of {} was aborted. We are now caching only {} "
|
||||
+ "bytes in total.", key, cacheLoader.getCacheUsed());
|
||||
IOUtils.closeQuietly(mappableBlock);
|
||||
numBlocksFailedToCache.incrementAndGet();
|
||||
numBlocksFailedToCache.increment();
|
||||
|
||||
synchronized (FsDatasetCache.this) {
|
||||
mappableBlockMap.remove(key);
|
||||
@ -561,7 +561,7 @@ public void run() {
|
||||
}
|
||||
long newUsedBytes = cacheLoader.
|
||||
release(key, value.mappableBlock.getLength());
|
||||
numBlocksCached.addAndGet(-1);
|
||||
numBlocksCached.decrement();
|
||||
dataset.datanode.getMetrics().incrBlocksUncached(1);
|
||||
if (revocationTimeMs != 0) {
|
||||
LOG.debug("Uncaching of {} completed. usedBytes = {}",
|
||||
@ -607,15 +607,15 @@ public long getCacheCapacity() {
|
||||
}
|
||||
|
||||
public long getNumBlocksFailedToCache() {
|
||||
return numBlocksFailedToCache.get();
|
||||
return numBlocksFailedToCache.longValue();
|
||||
}
|
||||
|
||||
public long getNumBlocksFailedToUncache() {
|
||||
return numBlocksFailedToUncache.get();
|
||||
return numBlocksFailedToUncache.longValue();
|
||||
}
|
||||
|
||||
public long getNumBlocksCached() {
|
||||
return numBlocksCached.get();
|
||||
return numBlocksCached.longValue();
|
||||
}
|
||||
|
||||
public synchronized boolean isCached(String bpid, long blockId) {
|
||||
|
@ -2264,7 +2264,7 @@ private void cacheBlock(String bpid, long blockId) {
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
cacheManager.numBlocksFailedToCache.incrementAndGet();
|
||||
cacheManager.numBlocksFailedToCache.increment();
|
||||
}
|
||||
}
|
||||
blockFileName = info.getBlockURI().toString();
|
||||
|
@ -27,7 +27,7 @@
|
||||
import java.util.Collection;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.concurrent.atomic.LongAdder;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
@ -182,7 +182,7 @@ private enum State {
|
||||
|
||||
// these are statistics counters.
|
||||
private long numTransactions; // number of transactions
|
||||
private final AtomicLong numTransactionsBatchedInSync = new AtomicLong();
|
||||
private final LongAdder numTransactionsBatchedInSync = new LongAdder();
|
||||
private long totalTimeTransactions; // total time for all transactions
|
||||
private NameNodeMetrics metrics;
|
||||
|
||||
@ -730,7 +730,7 @@ protected void logSync(long mytxid) {
|
||||
if (metrics != null) { // Metrics non-null only when used inside name node
|
||||
metrics.addSync(elapsed);
|
||||
metrics.incrTransactionsBatchedInSync(editsBatchedInSync);
|
||||
numTransactionsBatchedInSync.addAndGet(editsBatchedInSync);
|
||||
numTransactionsBatchedInSync.add(editsBatchedInSync);
|
||||
}
|
||||
|
||||
} finally {
|
||||
@ -770,7 +770,7 @@ private void printStatistics(boolean force) {
|
||||
.append(" Total time for transactions(ms): ")
|
||||
.append(totalTimeTransactions)
|
||||
.append(" Number of transactions batched in Syncs: ")
|
||||
.append(numTransactionsBatchedInSync.get())
|
||||
.append(numTransactionsBatchedInSync.longValue())
|
||||
.append(" Number of syncs: ")
|
||||
.append(editLogStream.getNumSync())
|
||||
.append(" SyncTimes(ms): ")
|
||||
@ -1402,7 +1402,7 @@ private void startLogSegment(final long segmentTxId, int layoutVersion)
|
||||
|
||||
numTransactions = 0;
|
||||
totalTimeTransactions = 0;
|
||||
numTransactionsBatchedInSync.set(0L);
|
||||
numTransactionsBatchedInSync.reset();
|
||||
|
||||
// TODO no need to link this back to storage anymore!
|
||||
// See HDFS-2174.
|
||||
|
@ -22,6 +22,7 @@
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.concurrent.atomic.LongAdder;
|
||||
import java.util.concurrent.locks.Condition;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
|
||||
@ -112,12 +113,12 @@ public Long initialValue() {
|
||||
* The number of time the read lock
|
||||
* has been held longer than the threshold.
|
||||
*/
|
||||
private final AtomicLong numReadLockLongHold = new AtomicLong(0);
|
||||
private final LongAdder numReadLockLongHold = new LongAdder();
|
||||
/**
|
||||
* The number of time the write lock
|
||||
* has been held for longer than the threshold.
|
||||
*/
|
||||
private final AtomicLong numWriteLockLongHold = new AtomicLong(0);
|
||||
private final LongAdder numWriteLockLongHold = new LongAdder();
|
||||
|
||||
@VisibleForTesting
|
||||
static final String OP_NAME_OTHER = "OTHER";
|
||||
@ -186,7 +187,7 @@ public void readUnlock(String opName) {
|
||||
final long readLockIntervalMs =
|
||||
TimeUnit.NANOSECONDS.toMillis(readLockIntervalNanos);
|
||||
if (needReport && readLockIntervalMs >= this.readLockReportingThresholdMs) {
|
||||
numReadLockLongHold.incrementAndGet();
|
||||
numReadLockLongHold.increment();
|
||||
LockHeldInfo localLockHeldInfo;
|
||||
do {
|
||||
localLockHeldInfo = longestReadLockHeldInfo.get();
|
||||
@ -264,7 +265,7 @@ public void writeUnlock(String opName, boolean suppressWriteLockReport) {
|
||||
LogAction logAction = LogThrottlingHelper.DO_NOT_LOG;
|
||||
if (needReport &&
|
||||
writeLockIntervalMs >= this.writeLockReportingThresholdMs) {
|
||||
numWriteLockLongHold.incrementAndGet();
|
||||
numWriteLockLongHold.increment();
|
||||
if (longestWriteLockHeldInfo.getIntervalMs() < writeLockIntervalMs) {
|
||||
longestWriteLockHeldInfo =
|
||||
new LockHeldInfo(currentTimeMs, writeLockIntervalMs,
|
||||
@ -322,7 +323,7 @@ public Condition newWriteLockCondition() {
|
||||
* has been held longer than the threshold
|
||||
*/
|
||||
public long getNumOfReadLockLongHold() {
|
||||
return numReadLockLongHold.get();
|
||||
return numReadLockLongHold.longValue();
|
||||
}
|
||||
|
||||
/**
|
||||
@ -333,7 +334,7 @@ public long getNumOfReadLockLongHold() {
|
||||
* has been held longer than the threshold.
|
||||
*/
|
||||
public long getNumOfWriteLockLongHold() {
|
||||
return numWriteLockLongHold.get();
|
||||
return numWriteLockLongHold.longValue();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -401,9 +401,9 @@ public Void answer(InvocationOnMock invocation) throws Throwable {
|
||||
DFSClient dfsClient = fileSys.getClient();
|
||||
DFSHedgedReadMetrics metrics = dfsClient.getHedgedReadMetrics();
|
||||
// Metrics instance is static, so we need to reset counts from prior tests.
|
||||
metrics.hedgedReadOps.set(0);
|
||||
metrics.hedgedReadOpsWin.set(0);
|
||||
metrics.hedgedReadOpsInCurThread.set(0);
|
||||
metrics.hedgedReadOps.reset();
|
||||
metrics.hedgedReadOpsWin.reset();
|
||||
metrics.hedgedReadOpsInCurThread.reset();
|
||||
|
||||
try {
|
||||
Path file1 = new Path("hedgedReadMaxOut.dat");
|
||||
@ -590,7 +590,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable {
|
||||
String filename = "/hedgedReadMaxOut.dat";
|
||||
DFSHedgedReadMetrics metrics = dfsClient.getHedgedReadMetrics();
|
||||
// Metrics instance is static, so we need to reset counts from prior tests.
|
||||
metrics.hedgedReadOps.set(0);
|
||||
metrics.hedgedReadOps.reset();
|
||||
try {
|
||||
Path file = new Path(filename);
|
||||
output = fileSys.create(file, (short) 2);
|
||||
|
Loading…
Reference in New Issue
Block a user