update docs

This commit is contained in:
kimchy 2010-03-03 00:50:53 +02:00
parent 12e2ba822f
commit 03ee0a942d
6 changed files with 49 additions and 42 deletions
modules/elasticsearch/src

@ -23,9 +23,6 @@ import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.SingleInstanceLockFactory;
import org.elasticsearch.util.SizeUnit;
import org.elasticsearch.util.SizeValue;
import org.elasticsearch.util.concurrent.highscalelib.NonBlockingHashMap;
import sun.nio.ch.DirectBuffer;
import java.io.FileNotFoundException;
@ -34,40 +31,52 @@ import java.nio.ByteBuffer;
import java.util.Map;
import java.util.Queue;
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.ConcurrentHashMap;
/**
* @author kimchy (Shay Banon)
* A memory based directory that uses {@link java.nio.ByteBuffer} in order to store the directory content.
*
* <p>The benefit of using {@link java.nio.ByteBuffer} is the fact that it can be stored in "native" memory
* outside of the JVM heap, thus not incurring the GC overhead of large in memory index.
*
* <p>Each "file" is segmented into one or more byte buffers.
*
* <p>Since its good practice to cache byte buffers, it also provide a simple mechanism to define a cache
* of byte buffers that are reused when possible.
*
* @author kimchy (shay.banon)
*/
public class ByteBufferDirectory extends Directory {
private final Map<String, ByteBufferFile> files = new NonBlockingHashMap<String, ByteBufferFile>();
private final Map<String, ByteBufferFile> files = new ConcurrentHashMap<String, ByteBufferFile>();
private final Queue<ByteBuffer> cache;
private final int bufferSizeInBytes;
private final SizeValue bufferSize;
private final SizeValue cacheSize;
private final int cacheSizeInBytes;
private final boolean disableCache;
private final boolean direct;
public ByteBufferDirectory() {
this(new SizeValue(1, SizeUnit.KB), new SizeValue(20, SizeUnit.MB), false, false);
}
public ByteBufferDirectory(SizeValue bufferSize, SizeValue cacheSize, boolean direct, boolean warmCache) {
disableCache = cacheSize.bytes() == 0;
if (!disableCache && cacheSize.bytes() < bufferSize.bytes()) {
throw new IllegalArgumentException("Cache size [" + cacheSize + "] is smaller than buffer size [" + bufferSize + "]");
/**
* Constructs a new byte buffer directory.
*
* @param bufferSizeInBytes The size of a byte buffer
* @param cacheSizeInBytes The size of the cache, set to <code>0</code> to disable caching
* @param direct Should the byte buffers be stored outside the heap (<code>true</code) or in head (<code>false</code>)
* @param warmCache Should the cache be warmed
*/
public ByteBufferDirectory(int bufferSizeInBytes, int cacheSizeInBytes, boolean direct, boolean warmCache) {
disableCache = cacheSizeInBytes == 0;
if (!disableCache && cacheSizeInBytes < bufferSizeInBytes) {
throw new IllegalArgumentException("Cache size [" + cacheSizeInBytes + "] is smaller than buffer size [" + bufferSizeInBytes + "]");
}
this.bufferSize = bufferSize;
this.bufferSizeInBytes = (int) bufferSize.bytes();
int numberOfCacheEntries = (int) (cacheSize.bytes() / bufferSize.bytes());
this.bufferSizeInBytes = bufferSizeInBytes;
int numberOfCacheEntries = cacheSizeInBytes / bufferSizeInBytes;
this.cache = disableCache ? null : new ArrayBlockingQueue<ByteBuffer>(numberOfCacheEntries);
this.cacheSize = disableCache ? new SizeValue(0, SizeUnit.BYTES) : new SizeValue(numberOfCacheEntries * bufferSize.bytes(), SizeUnit.BYTES);
this.cacheSizeInBytes = disableCache ? 0 : numberOfCacheEntries * bufferSizeInBytes;
this.direct = direct;
setLockFactory(new SingleInstanceLockFactory());
if (!disableCache && warmCache) {
@ -77,15 +86,11 @@ public class ByteBufferDirectory extends Directory {
}
}
public SizeValue bufferSize() {
return this.bufferSize;
public int cacheSizeInBytes() {
return this.cacheSizeInBytes;
}
public SizeValue cacheSize() {
return this.cacheSize;
}
int bufferSizeInBytes() {
public int bufferSizeInBytes() {
return bufferSizeInBytes;
}
@ -93,6 +98,10 @@ public class ByteBufferDirectory extends Directory {
return direct;
}
public boolean isCacheEnabled() {
return !disableCache;
}
@Override public String[] listAll() throws IOException {
return files.keySet().toArray(new String[0]);
}

@ -22,7 +22,7 @@ package org.elasticsearch.index.store.memory;
import java.nio.ByteBuffer;
/**
* @author kimchy (Shay Banon)
* @author kimchy (shay.banon)
*/
public class ByteBufferFile {

@ -25,7 +25,7 @@ import java.io.IOException;
import java.nio.ByteBuffer;
/**
* @author kimchy (Shay Banon)
* @author kimchy (shay.banon)
*/
public class ByteBufferIndexInput extends IndexInput {

@ -26,7 +26,7 @@ import java.nio.ByteBuffer;
import java.util.ArrayList;
/**
* @author kimchy (Shay Banon)
* @author kimchy (shay.banon)
*/
public class ByteBufferIndexOutput extends IndexOutput {

@ -28,7 +28,7 @@ import org.elasticsearch.util.SizeValue;
import org.elasticsearch.util.settings.Settings;
/**
* @author kimchy (Shay Banon)
* @author kimchy (shay.banon)
*/
public class ByteBufferStore extends AbstractStore<ByteBufferDirectory> {
@ -49,9 +49,9 @@ public class ByteBufferStore extends AbstractStore<ByteBufferDirectory> {
this.cacheSize = componentSettings.getAsSize("cacheSize", new SizeValue(20, SizeUnit.MB));
this.direct = componentSettings.getAsBoolean("direct", true);
this.warmCache = componentSettings.getAsBoolean("warmCache", true);
this.directory = new ByteBufferDirectory(bufferSize, cacheSize, direct, warmCache);
this.directory = new ByteBufferDirectory((int) bufferSize.bytes(), (int) cacheSize.bytes(), direct, warmCache);
logger.debug("Using [ByteBuffer] Store with bufferSize[{}], cacheSize[{}], direct[{}], warmCache[{}]",
new Object[]{directory.bufferSize(), directory.cacheSize(), directory.isDirect(), warmCache});
new Object[]{bufferSize, cacheSize, directory.isDirect(), warmCache});
}
@Override public ByteBufferDirectory directory() {

@ -23,8 +23,6 @@ import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.Lock;
import org.apache.lucene.store.LockObtainFailedException;
import org.elasticsearch.util.SizeUnit;
import org.elasticsearch.util.SizeValue;
import org.testng.annotations.Test;
import java.io.IOException;
@ -38,49 +36,49 @@ import static org.hamcrest.Matchers.*;
public class SimpleByteBufferStoreTests {
@Test public void test1BufferNoCache() throws Exception {
ByteBufferDirectory dir = new ByteBufferDirectory(new SizeValue(1, SizeUnit.BYTES), new SizeValue(0, SizeUnit.BYTES), true, false);
ByteBufferDirectory dir = new ByteBufferDirectory(1, 0, true, false);
insertData(dir);
verifyData(dir);
dir.close();
}
@Test public void test1Buffer() throws Exception {
ByteBufferDirectory dir = new ByteBufferDirectory(new SizeValue(1, SizeUnit.BYTES), new SizeValue(10, SizeUnit.BYTES), true, false);
ByteBufferDirectory dir = new ByteBufferDirectory(1, 10, true, false);
insertData(dir);
verifyData(dir);
dir.close();
}
@Test public void test3Buffer() throws Exception {
ByteBufferDirectory dir = new ByteBufferDirectory(new SizeValue(3, SizeUnit.BYTES), new SizeValue(10, SizeUnit.BYTES), true, false);
ByteBufferDirectory dir = new ByteBufferDirectory(3, 10, true, false);
insertData(dir);
verifyData(dir);
dir.close();
}
@Test public void test10Buffer() throws Exception {
ByteBufferDirectory dir = new ByteBufferDirectory(new SizeValue(10, SizeUnit.BYTES), new SizeValue(20, SizeUnit.BYTES), true, false);
ByteBufferDirectory dir = new ByteBufferDirectory(10, 20, true, false);
insertData(dir);
verifyData(dir);
dir.close();
}
@Test public void test15Buffer() throws Exception {
ByteBufferDirectory dir = new ByteBufferDirectory(new SizeValue(15, SizeUnit.BYTES), new SizeValue(30, SizeUnit.BYTES), true, false);
ByteBufferDirectory dir = new ByteBufferDirectory(15, 30, true, false);
insertData(dir);
verifyData(dir);
dir.close();
}
@Test public void test40Buffer() throws Exception {
ByteBufferDirectory dir = new ByteBufferDirectory(new SizeValue(40, SizeUnit.BYTES), new SizeValue(80, SizeUnit.BYTES), true, false);
ByteBufferDirectory dir = new ByteBufferDirectory(40, 80, true, false);
insertData(dir);
verifyData(dir);
dir.close();
}
@Test public void testSimpleLocking() throws Exception {
ByteBufferDirectory dir = new ByteBufferDirectory(new SizeValue(40, SizeUnit.BYTES), new SizeValue(80, SizeUnit.BYTES), true, false);
ByteBufferDirectory dir = new ByteBufferDirectory(40, 80, true, false);
Lock lock = dir.makeLock("testlock");