mirror of https://github.com/apache/lucene.git
SOLR-6752: Buffer Cache allocate/lost metrics should be exposed.
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1642689 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
56d7da62ba
commit
193c1be0b4
|
@ -461,6 +461,9 @@ Other Changes
|
|||
* SOLR-6370: Allow tests to report/fail on many ZK watches being parallelly
|
||||
requested on the same data (Ramkumar Aiyengar via Timothy Potter)
|
||||
|
||||
* SOLR-6752: Buffer Cache allocate/lost metrics should be exposed.
|
||||
(Mike Drob via Mark Miller)
|
||||
|
||||
================== 4.10.3 ==================
|
||||
|
||||
Bug Fixes
|
||||
|
|
|
@ -209,7 +209,7 @@ public class HdfsDirectoryFactory extends CachingDirectoryFactory {
|
|||
private BlockCache createBlockCache(int numberOfBlocksPerBank, int blockSize,
|
||||
int bankCount, boolean directAllocation, int slabSize, int bufferSize,
|
||||
int bufferCount) {
|
||||
BufferStore.initNewBuffer(bufferSize, bufferCount);
|
||||
BufferStore.initNewBuffer(bufferSize, bufferCount, metrics);
|
||||
long totalMemory = (long) bankCount * (long) numberOfBlocksPerBank
|
||||
* (long) blockSize;
|
||||
|
||||
|
|
|
@ -21,6 +21,7 @@ import java.util.concurrent.ArrayBlockingQueue;
|
|||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
/**
|
||||
* @lucene.experimental
|
||||
|
@ -45,7 +46,14 @@ public class BufferStore implements Store {
|
|||
|
||||
private final int bufferSize;
|
||||
|
||||
private final AtomicLong shardBuffercacheAllocate;
|
||||
private final AtomicLong shardBuffercacheLost;
|
||||
|
||||
public synchronized static void initNewBuffer(int bufferSize, long totalAmount) {
|
||||
initNewBuffer(bufferSize, totalAmount, null);
|
||||
}
|
||||
|
||||
public synchronized static void initNewBuffer(int bufferSize, long totalAmount, Metrics metrics) {
|
||||
if (totalAmount == 0) {
|
||||
return;
|
||||
}
|
||||
|
@ -55,13 +63,21 @@ public class BufferStore implements Store {
|
|||
if (count > Integer.MAX_VALUE) {
|
||||
count = Integer.MAX_VALUE;
|
||||
}
|
||||
BufferStore store = new BufferStore(bufferSize, (int) count);
|
||||
AtomicLong shardBuffercacheLost = new AtomicLong(0);
|
||||
AtomicLong shardBuffercacheAllocate = new AtomicLong(0);
|
||||
if (metrics != null) {
|
||||
shardBuffercacheLost = metrics.shardBuffercacheLost;
|
||||
shardBuffercacheAllocate = metrics.shardBuffercacheAllocate;
|
||||
}
|
||||
BufferStore store = new BufferStore(bufferSize, (int) count, shardBuffercacheAllocate, shardBuffercacheLost);
|
||||
bufferStores.put(bufferSize, store);
|
||||
}
|
||||
}
|
||||
|
||||
private BufferStore(int bufferSize, int count) {
|
||||
private BufferStore(int bufferSize, int count, AtomicLong shardBuffercacheAllocate, AtomicLong shardBuffercacheLost) {
|
||||
this.bufferSize = bufferSize;
|
||||
this.shardBuffercacheAllocate = shardBuffercacheAllocate;
|
||||
this.shardBuffercacheLost = shardBuffercacheLost;
|
||||
buffers = setupBuffers(bufferSize, count);
|
||||
}
|
||||
|
||||
|
@ -102,14 +118,17 @@ public class BufferStore implements Store {
|
|||
checkReturn(buffers.offer(buffer));
|
||||
}
|
||||
|
||||
private void checkReturn(boolean offer) {
|
||||
|
||||
private void checkReturn(boolean accepted) {
|
||||
if (!accepted) {
|
||||
shardBuffercacheLost.incrementAndGet();
|
||||
}
|
||||
}
|
||||
|
||||
private byte[] newBuffer(byte[] buf) {
|
||||
if (buf != null) {
|
||||
return buf;
|
||||
}
|
||||
shardBuffercacheAllocate.incrementAndGet();
|
||||
return new byte[bufferSize];
|
||||
}
|
||||
}
|
||||
|
|
|
@ -49,9 +49,7 @@ public class Metrics implements Updater {
|
|||
public AtomicLong recordWrites = new AtomicLong(0);
|
||||
public AtomicLong queriesExternal = new AtomicLong(0);
|
||||
public AtomicLong queriesInternal = new AtomicLong(0);
|
||||
public AtomicLong shardBuffercacheAllocate1024 = new AtomicLong(0);
|
||||
public AtomicLong shardBuffercacheAllocate8192 = new AtomicLong(0);
|
||||
public AtomicLong shardBuffercacheAllocateOther = new AtomicLong(0);
|
||||
public AtomicLong shardBuffercacheAllocate = new AtomicLong(0);
|
||||
public AtomicLong shardBuffercacheLost = new AtomicLong(0);
|
||||
public Map<String,MethodCall> methodCalls = new ConcurrentHashMap<>();
|
||||
|
||||
|
@ -101,6 +99,8 @@ public class Metrics implements Updater {
|
|||
metricsRecord.setMetric("record.writes", getPerSecond(recordWrites.getAndSet(0), seconds));
|
||||
metricsRecord.setMetric("query.external", getPerSecond(queriesExternal.getAndSet(0), seconds));
|
||||
metricsRecord.setMetric("query.internal", getPerSecond(queriesInternal.getAndSet(0), seconds));
|
||||
metricsRecord.setMetric("buffercache.allocations", getPerSecond(shardBuffercacheAllocate.getAndSet(0), seconds));
|
||||
metricsRecord.setMetric("buffercache.lost", getPerSecond(shardBuffercacheLost.getAndSet(0), seconds));
|
||||
for (Entry<String,MethodCall> entry : methodCalls.entrySet()) {
|
||||
String key = entry.getKey();
|
||||
MethodCall value = entry.getValue();
|
||||
|
|
Loading…
Reference in New Issue