HBASE-5706 'Dropping fs latency stats since buffer is full' spam

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1309457 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2012-04-04 15:36:28 +00:00
parent 5e07b71ed6
commit 753afe9de3
1 changed files with 3 additions and 32 deletions

View File

@ -194,50 +194,21 @@ public class HFile {
new ArrayBlockingQueue<Long>(LATENCY_BUFFER_SIZE);
private static final BlockingQueue<Long> fsPreadLatenciesNanos =
new ArrayBlockingQueue<Long>(LATENCY_BUFFER_SIZE);
private static final AtomicLong lastLoggedDataDrop = new AtomicLong(0);
// we don't want to fill up the logs with this message, so only log it
// once every 30 seconds at most
// I also want to avoid locks on the 'critical path' (the common case will be
// uncontended) - hence the CAS
private static void logDroppedLatencyStat() {
final long now = System.currentTimeMillis();
final long earliestAcceptableLog = now - TimeUnit.SECONDS.toMillis(30L);
while (true) {
final long lastLog = lastLoggedDataDrop.get();
if (lastLog < earliestAcceptableLog) {
if (lastLoggedDataDrop.compareAndSet(lastLog, now)) {
LOG.warn("Dropping fs latency stats since buffer is full");
break;
} // otherwise (if the compaseAndSet failed) the while loop retries
} else {
break;
}
}
}
public static final void offerReadLatency(long latencyNanos, boolean pread) {
boolean stored = false;
if (pread) {
stored = fsPreadLatenciesNanos.offer(latencyNanos);
fsPreadLatenciesNanos.offer(latencyNanos); // might be silently dropped, if the queue is full
preadOps.incrementAndGet();
preadTimeNano.addAndGet(latencyNanos);
} else {
stored = fsReadLatenciesNanos.offer(latencyNanos);
fsReadLatenciesNanos.offer(latencyNanos); // might be silently dropped, if the queue is full
readTimeNano.addAndGet(latencyNanos);
readOps.incrementAndGet();
}
if (!stored) {
logDroppedLatencyStat();
}
}
public static final void offerWriteLatency(long latencyNanos) {
final boolean stored = fsWriteLatenciesNanos.offer(latencyNanos);
if (!stored) {
logDroppedLatencyStat();
}
fsWriteLatenciesNanos.offer(latencyNanos); // might be silently dropped, if the queue is full
writeTimeNano.addAndGet(latencyNanos);
writeOps.incrementAndGet();