diff --git a/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java index 37bcabab8c5..c4a8a9e5faf 100644 --- a/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java +++ b/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java @@ -194,50 +194,21 @@ public class HFile { new ArrayBlockingQueue(LATENCY_BUFFER_SIZE); private static final BlockingQueue fsPreadLatenciesNanos = new ArrayBlockingQueue(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();