From 019441ae8414df779b86c86967a2a6c474bf8650 Mon Sep 17 00:00:00 2001 From: Chris Vest Date: Fri, 18 Aug 2017 13:11:00 +0200 Subject: [PATCH 1/2] Reduce contention on RAMFile monitor lock This monitor lock is quite popular in high concurrency, high write work loads, because numBuffers() is called quite often. This changes the numBuffers() method so it no longer takes the RAMFile monitor lock, but instead does a volatile read. --- lucene/core/src/java/org/apache/lucene/store/RAMFile.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/store/RAMFile.java b/lucene/core/src/java/org/apache/lucene/store/RAMFile.java index 1092018e37d6..26c4639b4f97 100644 --- a/lucene/core/src/java/org/apache/lucene/store/RAMFile.java +++ b/lucene/core/src/java/org/apache/lucene/store/RAMFile.java @@ -29,6 +29,7 @@ * @lucene.internal */ public class RAMFile implements Accountable { protected final ArrayList buffers = new ArrayList<>(); + private volatile int numBuffers; long length; RAMDirectory directory; protected long sizeInBytes; @@ -53,6 +54,7 @@ protected final byte[] addBuffer(int size) { byte[] buffer = newBuffer(size); synchronized(this) { buffers.add(buffer); + numBuffers = buffers.size(); sizeInBytes += size; } @@ -66,8 +68,8 @@ protected final synchronized byte[] getBuffer(int index) { return buffers.get(index); } - protected final synchronized int numBuffers() { - return buffers.size(); + protected final int numBuffers() { + return numBuffers; } /** From 20de28c8dda8d9496ccfbd4e9c1bd9c98fca1887 Mon Sep 17 00:00:00 2001 From: Chris Vest Date: Fri, 18 Aug 2017 13:13:28 +0200 Subject: [PATCH 2/2] Make the RAMFile buffer size configurable Larger buffers can improve the write throughput of e.g. NRTCachingDirectories. The buffer size can now be changed via the `org.apache.lucene.store.RAMOutputStream.BUFFER_SIZE` system property. --- .../core/src/java/org/apache/lucene/store/RAMOutputStream.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java b/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java index d75905a03bf9..de0d4ec4f671 100644 --- a/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java +++ b/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java @@ -32,7 +32,7 @@ * @lucene.internal */ public class RAMOutputStream extends IndexOutput implements Accountable { - static final int BUFFER_SIZE = 1024; + static final int BUFFER_SIZE = Integer.getInteger( "org.apache.lucene.store.RAMOutputStream.BUFFER_SIZE", 1024 ); private final RAMFile file;