add size in bytes to bb dir

This commit is contained in:
Shay Banon 2011-12-31 20:33:17 +02:00
parent 8cf8b478af
commit 53b1b6e4bb
2 changed files with 17 additions and 0 deletions

View File

@ -28,6 +28,7 @@ import java.nio.ByteBuffer;
import java.util.Collection;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicLong;
/**
* A memory based directory that uses {@link java.nio.ByteBuffer} in order to store the directory content.
@ -48,6 +49,8 @@ public class ByteBufferDirectory extends Directory {
private final boolean internalAllocator;
final AtomicLong sizeInBytes = new AtomicLong();
/**
* Constructs a new directory using {@link PlainByteBufferAllocator}.
*/
@ -74,6 +77,13 @@ public class ByteBufferDirectory extends Directory {
}
}
/**
* Returns the size in bytes of the directory, chunk by buffer size.
*/
public long sizeInBytes() {
return sizeInBytes.get();
}
public void sync(Collection<String> names) throws IOException {
// nothing to do here
}
@ -123,6 +133,7 @@ public class ByteBufferDirectory extends Directory {
ByteBufferFile file = files.remove(name);
if (file == null)
throw new FileNotFoundException(name);
sizeInBytes.addAndGet(-file.sizeInBytes.get());
file.delete();
}
@ -143,6 +154,7 @@ public class ByteBufferDirectory extends Directory {
ByteBufferFile file = new ByteBufferFile(this, allocator.sizeInBytes(allocatorType));
ByteBufferFile existing = files.put(name, file);
if (existing != null) {
sizeInBytes.addAndGet(-existing.sizeInBytes.get());
existing.delete();
}
return new ByteBufferIndexOutput(name, allocator, allocatorType, file);

View File

@ -20,6 +20,7 @@ package org.apache.lucene.store.bytebuffer;
import java.nio.ByteBuffer;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
/**
*/
@ -35,6 +36,8 @@ public class ByteBufferFile {
private final AtomicInteger refCount = new AtomicInteger(1);
final AtomicLong sizeInBytes = new AtomicLong();
public ByteBufferFile(ByteBufferDirectory dir, int bufferSize) {
this.dir = dir;
this.bufferSize = bufferSize;
@ -60,6 +63,8 @@ public class ByteBufferFile {
protected final void addBuffer(ByteBuffer buffer) {
buffers.add(buffer);
sizeInBytes.addAndGet(buffer.remaining());
dir.sizeInBytes.addAndGet(buffer.remaining());
}
protected final ByteBuffer getBuffer(int index) {