You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2012/04/04 17:36:55 UTC
svn commit: r1309459 -
/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
Author: stack
Date: Wed Apr 4 15:36:55 2012
New Revision: 1309459
URL: http://svn.apache.org/viewvc?rev=1309459&view=rev
Log:
HBASE-5706 'Dropping fs latency stats since buffer is full' spam
Modified:
hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java?rev=1309459&r1=1309458&r2=1309459&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java Wed Apr 4 15:36:55 2012
@@ -192,50 +192,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();