Index: lucene/src/java/org/apache/lucene/store/bytebuffer/ByteBufferAllocator.java
===================================================================
--- lucene/src/java/org/apache/lucene/store/bytebuffer/ByteBufferAllocator.java (revision )
+++ lucene/src/java/org/apache/lucene/store/bytebuffer/ByteBufferAllocator.java (revision )
@@ -0,0 +1,97 @@
+package org.apache.lucene.store.bytebuffer;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.lang.reflect.Method;
+import java.nio.ByteBuffer;
+
+/**
+ * A byte buffer allocator simple allocates byte buffers, and handles releasing
+ * them. Implementation can include special direct buffer cleaning when releasing
+ * a buffer, as well as caching of byte buffers.
+ *
+ *
There are two types of buffers that can be allocated, small and big. This
+ * comes in handy when knowing in advance (more or less) the size of the buffers
+ * needed (large files or small), as well as in caching implementations.
+ */
+public interface ByteBufferAllocator {
+
+ /**
+ * Helper class to allocator implementations allowing to clean direct buffers.
+ */
+ public static class Cleaner {
+ public static final boolean CLEAN_SUPPORTED;
+ private static final Method directBufferCleaner;
+ private static final Method directBufferCleanerClean;
+
+ static {
+ Method directBufferCleanerX = null;
+ Method directBufferCleanerCleanX = null;
+ boolean v;
+ try {
+ directBufferCleanerX = Class.forName("java.nio.DirectByteBuffer").getMethod("cleaner");
+ directBufferCleanerX.setAccessible(true);
+ directBufferCleanerCleanX = Class.forName("sun.misc.Cleaner").getMethod("clean");
+ directBufferCleanerCleanX.setAccessible(true);
+ v = true;
+ } catch (Exception e) {
+ v = false;
+ }
+ CLEAN_SUPPORTED = v;
+ directBufferCleaner = directBufferCleanerX;
+ directBufferCleanerClean = directBufferCleanerCleanX;
+ }
+
+ public static void clean(ByteBuffer buffer) {
+ if (CLEAN_SUPPORTED && buffer.isDirect()) {
+ try {
+ Object cleaner = directBufferCleaner.invoke(buffer);
+ directBufferCleanerClean.invoke(cleaner);
+ } catch (Exception e) {
+ // silently ignore exception
+ }
+ }
+ }
+ }
+
+ public static enum Type {
+ SMALL,
+ LARGE
+ }
+
+ /**
+ * The size (in bytes) that is allocated for the provided type.
+ */
+ int sizeInBytes(Type type);
+
+ /**
+ * Allocate a byte buffer for the specific type.
+ */
+ ByteBuffer allocate(Type type) throws IOException;
+
+ /**
+ * Release the buffer.
+ */
+ void release(ByteBuffer buffer);
+
+ /**
+ * Close the allocator, releasing any cached buffers for example.
+ */
+ void close();
+}
Index: lucene/src/java/org/apache/lucene/store/bytebuffer/ByteBufferFile.java
===================================================================
--- lucene/src/java/org/apache/lucene/store/bytebuffer/ByteBufferFile.java (revision )
+++ lucene/src/java/org/apache/lucene/store/bytebuffer/ByteBufferFile.java (revision )
@@ -0,0 +1,79 @@
+package org.apache.lucene.store.bytebuffer;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.CopyOnWriteArrayList;
+
+/**
+ * @author kimchy (shay.banon)
+ */
+public class ByteBufferFile {
+
+ private final CopyOnWriteArrayList buffers = new CopyOnWriteArrayList();
+ private final ByteBufferDirectory dir;
+ final int bufferSize;
+
+ private volatile long length;
+ // This is publicly modifiable via Directory.touchFile(), so direct access not supported
+ private volatile long lastModified = System.currentTimeMillis();
+
+ public ByteBufferFile(ByteBufferDirectory dir, int bufferSize) {
+ this.dir = dir;
+ this.bufferSize = bufferSize;
+ }
+
+ // For non-stream access from thread that might be concurrent with writing
+ public long getLength() {
+ return length;
+ }
+
+ protected void setLength(long length) {
+ this.length = length;
+ }
+
+ // For non-stream access from thread that might be concurrent with writing
+ public long getLastModified() {
+ return lastModified;
+ }
+
+ protected void setLastModified(long lastModified) {
+ this.lastModified = lastModified;
+ }
+
+ protected final void addBuffer(ByteBuffer buffer) {
+ buffers.add(buffer);
+ }
+
+ protected final ByteBuffer getBuffer(int index) {
+ return buffers.get(index);
+ }
+
+ protected final int numBuffers() {
+ return buffers.size();
+ }
+
+ void clean() {
+ if (buffers != null) {
+ for (ByteBuffer buffer : buffers) {
+ dir.releaseBuffer(buffer);
+ }
+ buffers.clear();
+ }
+ }
+}
Index: lucene/src/java/org/apache/lucene/store/bytebuffer/ByteBufferDirectory.java
===================================================================
--- lucene/src/java/org/apache/lucene/store/bytebuffer/ByteBufferDirectory.java (revision )
+++ lucene/src/java/org/apache/lucene/store/bytebuffer/ByteBufferDirectory.java (revision )
@@ -0,0 +1,175 @@
+package org.apache.lucene.store.bytebuffer;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+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 java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * A memory based directory that uses {@link java.nio.ByteBuffer} in order to store the directory content.
+ *
+ * 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.
+ *
+ *
Each "file" is segmented into one or more byte buffers.
+ *
+ *
If constructed with {@link ByteBufferAllocator}, it allows to control the allocation and release of
+ * byte buffer. For example, custom implementations can include caching of byte buffers.
+ *
+ * @author kimchy (shay.banon)
+ */
+public class ByteBufferDirectory extends Directory {
+
+ private final Map files = new ConcurrentHashMap();
+
+ private final ByteBufferAllocator allocator;
+
+ private final boolean internalAllocator;
+
+ /**
+ * Constructs a new directory using {@link PlainByteBufferAllocator}.
+ */
+ public ByteBufferDirectory() {
+ this.allocator = new PlainByteBufferAllocator(false, 1024, 1024 * 10);
+ this.internalAllocator = true;
+ try {
+ setLockFactory(new SingleInstanceLockFactory());
+ } catch (IOException e) {
+ // will not happen
+ }
+ }
+
+ /**
+ * Constructs a new byte buffer directory with a custom allocator.
+ */
+ public ByteBufferDirectory(ByteBufferAllocator allocator) {
+ this.allocator = allocator;
+ this.internalAllocator = false;
+ try {
+ setLockFactory(new SingleInstanceLockFactory());
+ } catch (IOException e) {
+ // will not happen
+ }
+ }
+
+ public void sync(Collection names) throws IOException {
+ // nothing to do here
+ }
+
+ @Override
+ public String[] listAll() throws IOException {
+ return files.keySet().toArray(new String[0]);
+ }
+
+ @Override
+ public boolean fileExists(String name) throws IOException {
+ return files.containsKey(name);
+ }
+
+ @Override
+ public long fileModified(String name) throws IOException {
+ ByteBufferFile file = files.get(name);
+ if (file == null)
+ throw new FileNotFoundException(name);
+ return file.getLastModified();
+ }
+
+ @Override
+ public void touchFile(String name) throws IOException {
+ ByteBufferFile file = files.get(name);
+ if (file == null)
+ throw new FileNotFoundException(name);
+
+ long ts2, ts1 = System.currentTimeMillis();
+ do {
+ try {
+ Thread.sleep(0, 1);
+ } catch (InterruptedException ie) {
+ // In 3.0 we will change this to throw
+ // InterruptedException instead
+ Thread.currentThread().interrupt();
+ throw new RuntimeException(ie);
+ }
+ ts2 = System.currentTimeMillis();
+ } while (ts1 == ts2);
+
+ file.setLastModified(ts2);
+ }
+
+ @Override
+ public void deleteFile(String name) throws IOException {
+ ByteBufferFile file = files.remove(name);
+ if (file == null)
+ throw new FileNotFoundException(name);
+ file.clean();
+ }
+
+ @Override
+ public long fileLength(String name) throws IOException {
+ ByteBufferFile file = files.get(name);
+ if (file == null)
+ throw new FileNotFoundException(name);
+ return file.getLength();
+ }
+
+ @Override
+ public IndexOutput createOutput(String name) throws IOException {
+ ByteBufferAllocator.Type allocatorType = ByteBufferAllocator.Type.LARGE;
+ if (name.contains("segments") || name.endsWith(".del")) {
+ allocatorType = ByteBufferAllocator.Type.SMALL;
+ }
+ ByteBufferFile file = new ByteBufferFile(this, allocator.sizeInBytes(allocatorType));
+ ByteBufferFile existing = files.put(name, file);
+ if (existing != null) {
+ existing.clean();
+ }
+ return new ByteBufferIndexOutput(allocator, allocatorType, file);
+ }
+
+ @Override
+ public IndexInput openInput(String name) throws IOException {
+ ByteBufferFile file = files.get(name);
+ if (file == null)
+ throw new FileNotFoundException(name);
+ return new ByteBufferIndexInput(file);
+ }
+
+ @Override
+ public void close() throws IOException {
+ String[] files = listAll();
+ for (String file : files) {
+ deleteFile(file);
+ }
+ if (internalAllocator) {
+ allocator.close();
+ }
+ }
+
+ void releaseBuffer(ByteBuffer byteBuffer) {
+ allocator.release(byteBuffer);
+ }
+}
Index: lucene/src/java/org/apache/lucene/store/bytebuffer/CachingByteBufferAllocator.java
===================================================================
--- lucene/src/java/org/apache/lucene/store/bytebuffer/CachingByteBufferAllocator.java (revision )
+++ lucene/src/java/org/apache/lucene/store/bytebuffer/CachingByteBufferAllocator.java (revision )
@@ -0,0 +1,81 @@
+package org.apache.lucene.store.bytebuffer;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ArrayBlockingQueue;
+
+/**
+ * The caching byte buffer allocator allows to define a global size for both the small and large buffers
+ * allocated. Those will be reused when possible.
+ */
+public class CachingByteBufferAllocator extends PlainByteBufferAllocator {
+
+ private final ArrayBlockingQueue smallCache;
+ private final ArrayBlockingQueue largeCache;
+
+ /**
+ * @param direct If set to true, will allocate direct buffers (off heap).
+ * @param smallBufferSizeInBytes The size (in bytes) of the small buffer allocation.
+ * @param largeBufferSizeInBytes The size (in bytes) of the large buffer allocation.
+ * @param smallCacheSizeInBytes The size of the small cache buffer in bytes.
+ * @param largeCacheSizeInBytes The size of the large cache buffer in bytes.
+ */
+ public CachingByteBufferAllocator(boolean direct, int smallBufferSizeInBytes, int largeBufferSizeInBytes,
+ int smallCacheSizeInBytes, int largeCacheSizeInBytes) {
+ super(direct, smallBufferSizeInBytes, largeBufferSizeInBytes);
+ this.smallCache = new ArrayBlockingQueue(smallCacheSizeInBytes / smallBufferSizeInBytes);
+ this.largeCache = new ArrayBlockingQueue(largeCacheSizeInBytes / largeBufferSizeInBytes);
+ }
+
+
+ public ByteBuffer allocate(Type type) throws IOException {
+ ByteBuffer buffer = type == Type.SMALL ? smallCache.poll() : largeCache.poll();
+ if (buffer == null) {
+ buffer = super.allocate(type);
+ }
+ return buffer;
+ }
+
+ public void release(ByteBuffer buffer) {
+ if (buffer.capacity() == smallBufferSizeInBytes) {
+ boolean success = smallCache.offer(buffer);
+ if (!success) {
+ super.release(buffer);
+ }
+ } else if (buffer.capacity() == largeBufferSizeInBytes) {
+ boolean success = largeCache.offer(buffer);
+ if (!success) {
+ super.release(buffer);
+ }
+ }
+ // otherwise, just ignore it? not our allocation...
+ }
+
+ public void close() {
+ for (ByteBuffer buffer : smallCache) {
+ super.release(buffer);
+ }
+ smallCache.clear();
+ for (ByteBuffer buffer : largeCache) {
+ super.release(buffer);
+ }
+ largeCache.clear();
+ }
+}
Index: lucene/src/java/org/apache/lucene/store/bytebuffer/ByteBufferIndexOutput.java
===================================================================
--- lucene/src/java/org/apache/lucene/store/bytebuffer/ByteBufferIndexOutput.java (revision )
+++ lucene/src/java/org/apache/lucene/store/bytebuffer/ByteBufferIndexOutput.java (revision )
@@ -0,0 +1,124 @@
+package org.apache.lucene.store.bytebuffer;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+import org.apache.lucene.store.IndexOutput;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ */
+public class ByteBufferIndexOutput extends IndexOutput {
+
+ private final ByteBufferAllocator allocator;
+ private final ByteBufferAllocator.Type allocatorType;
+ private final int BUFFER_SIZE;
+ private final ByteBufferFile file;
+
+ private ByteBuffer currentBuffer;
+ private int currentBufferIndex;
+
+ private long bufferStart;
+
+ public ByteBufferIndexOutput(ByteBufferAllocator allocator, ByteBufferAllocator.Type allocatorType, ByteBufferFile file) throws IOException {
+ this.allocator = allocator;
+ this.allocatorType = allocatorType;
+ this.BUFFER_SIZE = file.bufferSize;
+ this.file = file;
+ // create the first buffer we write to
+ switchCurrentBuffer();
+ }
+
+ @Override
+ public void close() throws IOException {
+ flush();
+ }
+
+ @Override
+ public void seek(long pos) throws IOException {
+ // set the file length in case we seek back
+ // and flush() has not been called yet
+ setFileLength();
+ if (pos < bufferStart || pos >= bufferStart + BUFFER_SIZE) {
+ currentBufferIndex = (int) (pos / BUFFER_SIZE);
+ switchCurrentBuffer();
+ }
+ currentBuffer.position((int) (pos % BUFFER_SIZE));
+ }
+
+ @Override
+ public long length() {
+ return file.getLength();
+ }
+
+ @Override
+ public void writeByte(byte b) throws IOException {
+ if (!currentBuffer.hasRemaining()) {
+ currentBufferIndex++;
+ switchCurrentBuffer();
+ }
+ currentBuffer.put(b);
+ }
+
+ @Override
+ public void writeBytes(byte[] b, int offset, int len) throws IOException {
+ while (len > 0) {
+ if (!currentBuffer.hasRemaining()) {
+ currentBufferIndex++;
+ switchCurrentBuffer();
+ }
+
+ int remainInBuffer = currentBuffer.remaining();
+ int bytesToCopy = len < remainInBuffer ? len : remainInBuffer;
+ currentBuffer.put(b, offset, bytesToCopy);
+ offset += bytesToCopy;
+ len -= bytesToCopy;
+ }
+ }
+
+ private void switchCurrentBuffer() throws IOException {
+ if (currentBufferIndex == file.numBuffers()) {
+ currentBuffer = allocator.allocate(allocatorType);
+ file.addBuffer(currentBuffer);
+ } else {
+ currentBuffer = file.getBuffer(currentBufferIndex);
+ }
+ currentBuffer.position(0);
+ bufferStart = (long) BUFFER_SIZE * (long) currentBufferIndex;
+ }
+
+ private void setFileLength() {
+ long pointer = bufferStart + currentBuffer.position();
+ if (pointer > file.getLength()) {
+ file.setLength(pointer);
+ }
+ }
+
+ @Override
+ public void flush() throws IOException {
+ file.setLastModified(System.currentTimeMillis());
+ setFileLength();
+ }
+
+ @Override
+ public long getFilePointer() {
+ return currentBufferIndex < 0 ? 0 : bufferStart + currentBuffer.position();
+ }
+}
Index: lucene/src/java/org/apache/lucene/store/bytebuffer/ByteBufferIndexInput.java
===================================================================
--- lucene/src/java/org/apache/lucene/store/bytebuffer/ByteBufferIndexInput.java (revision )
+++ lucene/src/java/org/apache/lucene/store/bytebuffer/ByteBufferIndexInput.java (revision )
@@ -0,0 +1,171 @@
+package org.apache.lucene.store.bytebuffer;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.store.IndexInput;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * @author kimchy (shay.banon)
+ */
+public class ByteBufferIndexInput extends IndexInput {
+
+ private final static ByteBuffer EMPTY_BUFFER = ByteBuffer.allocate(0).asReadOnlyBuffer();
+
+ private final ByteBufferFile file;
+ private final long length;
+
+ private ByteBuffer currentBuffer;
+ private int currentBufferIndex;
+
+ private long bufferStart;
+ private final int BUFFER_SIZE;
+
+ public ByteBufferIndexInput(ByteBufferFile file) throws IOException {
+ this.file = file;
+ this.length = file.getLength();
+ this.BUFFER_SIZE = file.bufferSize;
+
+ // make sure that we switch to the
+ // first needed buffer lazily
+ currentBufferIndex = -1;
+ currentBuffer = EMPTY_BUFFER;
+ }
+
+ @Override
+ public void close() {
+ // nothing to do here
+ }
+
+ @Override
+ public long length() {
+ return length;
+ }
+
+// @Override
+// public short readShort() throws IOException {
+// try {
+// return currentBuffer.getShort();
+// } catch (BufferUnderflowException e) {
+// return super.readShort();
+// }
+// }
+//
+// @Override
+// public int readInt() throws IOException {
+// try {
+// return currentBuffer.getInt();
+// } catch (BufferUnderflowException e) {
+// return super.readInt();
+// }
+// }
+//
+// @Override
+// public long readLong() throws IOException {
+// try {
+// return currentBuffer.getLong();
+// } catch (BufferUnderflowException e) {
+// return super.readLong();
+// }
+// }
+
+ @Override
+ public byte readByte() throws IOException {
+ if (!currentBuffer.hasRemaining()) {
+ currentBufferIndex++;
+ switchCurrentBuffer(true);
+ }
+ return currentBuffer.get();
+ }
+
+ @Override
+ public void readBytes(byte[] b, int offset, int len) throws IOException {
+ while (len > 0) {
+ if (!currentBuffer.hasRemaining()) {
+ currentBufferIndex++;
+ switchCurrentBuffer(true);
+ }
+
+ int remainInBuffer = currentBuffer.remaining();
+ int bytesToCopy = len < remainInBuffer ? len : remainInBuffer;
+ currentBuffer.get(b, offset, bytesToCopy);
+ offset += bytesToCopy;
+ len -= bytesToCopy;
+ }
+ }
+
+ @Override
+ public long getFilePointer() {
+ return currentBufferIndex < 0 ? 0 : bufferStart + currentBuffer.position();
+ }
+
+ @Override
+ public void seek(long pos) throws IOException {
+ if (currentBuffer == EMPTY_BUFFER || pos < bufferStart || pos >= bufferStart + BUFFER_SIZE) {
+ currentBufferIndex = (int) (pos / BUFFER_SIZE);
+ if (currentBufferIndex >= file.numBuffers()) {
+ // if we are past EOF, don't throw one here, instead, move it to the last position in the last buffer
+ currentBufferIndex = file.numBuffers() - 1;
+ currentBuffer = currentBufferIndex == -1 ? EMPTY_BUFFER : file.getBuffer(currentBufferIndex);
+ currentBuffer.position(currentBuffer.limit());
+ return;
+ } else {
+ switchCurrentBuffer(false);
+ }
+ }
+ try {
+ currentBuffer.position((int) (pos % BUFFER_SIZE));
+ } catch (IllegalArgumentException e) {
+ currentBuffer.position(currentBuffer.limit());
+ }
+ }
+
+ private void switchCurrentBuffer(boolean enforceEOF) throws IOException {
+ if (currentBufferIndex >= file.numBuffers()) {
+ if (enforceEOF) {
+ throw new IOException("Read past EOF");
+ }
+ } else {
+ ByteBuffer buffer = file.getBuffer(currentBufferIndex);
+ // we must duplicate (and make it read only while we are at it) since we need position and such to be independent
+ currentBuffer = buffer.asReadOnlyBuffer();
+ currentBuffer.position(0);
+ bufferStart = (long) BUFFER_SIZE * (long) currentBufferIndex;
+ // if we are at the tip, limit the current buffer to only whats available to read
+ long buflen = length - bufferStart;
+ if (buflen < BUFFER_SIZE) {
+ currentBuffer.limit((int) buflen);
+ if (enforceEOF && buflen == 0) {
+ throw new IOException("Read past EOF");
+ }
+ }
+ }
+ }
+
+ @Override
+ public Object clone() {
+ ByteBufferIndexInput cloned = (ByteBufferIndexInput) super.clone();
+ if (currentBuffer != EMPTY_BUFFER) {
+ cloned.currentBuffer = currentBuffer.asReadOnlyBuffer();
+ cloned.currentBuffer.position(currentBuffer.position());
+ }
+ return cloned;
+ }
+}
Index: lucene/src/java/org/apache/lucene/store/bytebuffer/PlainByteBufferAllocator.java
===================================================================
--- lucene/src/java/org/apache/lucene/store/bytebuffer/PlainByteBufferAllocator.java (revision )
+++ lucene/src/java/org/apache/lucene/store/bytebuffer/PlainByteBufferAllocator.java (revision )
@@ -0,0 +1,67 @@
+package org.apache.lucene.store.bytebuffer;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * A simple byte buffer allocator that does not caching. The direct flag
+ * allows to control if the byte buffer will be allocated off heap or not.
+ */
+public class PlainByteBufferAllocator implements ByteBufferAllocator {
+
+ protected final boolean direct;
+
+ protected final int smallBufferSizeInBytes;
+
+ protected final int largeBufferSizeInBytes;
+
+ /**
+ * Constructs a new plain byte buffer allocator that does no caching.
+ *
+ * @param direct If set to true, will allocate direct buffers (off heap).
+ * @param smallBufferSizeInBytes The size (in bytes) of the small buffer allocation.
+ * @param largeBufferSizeInBytes The size (in bytes) of the large buffer allocation.
+ */
+ public PlainByteBufferAllocator(boolean direct, int smallBufferSizeInBytes, int largeBufferSizeInBytes) {
+ this.direct = direct;
+ this.smallBufferSizeInBytes = smallBufferSizeInBytes;
+ this.largeBufferSizeInBytes = largeBufferSizeInBytes;
+ }
+
+ public int sizeInBytes(Type type) {
+ return type == Type.SMALL ? smallBufferSizeInBytes : largeBufferSizeInBytes;
+ }
+
+ public ByteBuffer allocate(Type type) throws IOException {
+ int sizeToAllocate = type == Type.SMALL ? smallBufferSizeInBytes : largeBufferSizeInBytes;
+ if (direct) {
+ return ByteBuffer.allocateDirect(sizeToAllocate);
+ }
+ return ByteBuffer.allocate(sizeToAllocate);
+ }
+
+ public void release(ByteBuffer buffer) {
+ Cleaner.clean(buffer);
+ }
+
+ public void close() {
+ // nothing to do here...
+ }
+}