Index: lucene/core/src/java/org/apache/lucene/store/ByteBufferIndexInput.java =================================================================== --- lucene/core/src/java/org/apache/lucene/store/ByteBufferIndexInput.java (revision 0) +++ lucene/core/src/java/org/apache/lucene/store/ByteBufferIndexInput.java (working copy) @@ -0,0 +1,310 @@ +package org.apache.lucene.store; + +/* + * 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.EOFException; +import java.io.IOException; +import java.nio.BufferUnderflowException; +import java.nio.ByteBuffer; +import java.util.Iterator; + +import org.apache.lucene.util.WeakIdentityMap; + +/** + * Base IndexInput implementation that uses an array + * of ByteBuffers to represent a file. + *

+ * Because Java's ByteBuffer uses an int to address the + * values, it's necessary to access a file greater + * Integer.MAX_VALUE in size using multiple byte buffers. + *

+ * For efficiency, this class requires that the buffers + * are a power-of-two (chunkSizePower). + */ +abstract class ByteBufferIndexInput extends IndexInput { + private ByteBuffer[] buffers; + + private final long chunkSizeMask; + private final int chunkSizePower; + + private int offset; + private long length; + private String sliceDescription; + + private int curBufIndex; + + private ByteBuffer curBuf; // redundant for speed: buffers[curBufIndex] + + private boolean isClone = false; + private final WeakIdentityMap clones = WeakIdentityMap.newConcurrentHashMap(); + + ByteBufferIndexInput(String resourceDescription, ByteBuffer[] buffers, long length, int chunkSizePower) throws IOException { + super(resourceDescription); + this.buffers = buffers; + this.length = length; + this.chunkSizePower = chunkSizePower; + this.chunkSizeMask = (1L << chunkSizePower) - 1L; + + assert chunkSizePower >= 0 && chunkSizePower <= 30; + assert (length >>> chunkSizePower) < Integer.MAX_VALUE; + + seek(0L); + } + + @Override + public final byte readByte() throws IOException { + try { + return curBuf.get(); + } catch (BufferUnderflowException e) { + do { + curBufIndex++; + if (curBufIndex >= buffers.length) { + throw new EOFException("read past EOF: " + this); + } + curBuf = buffers[curBufIndex]; + curBuf.position(0); + } while (!curBuf.hasRemaining()); + return curBuf.get(); + } catch (NullPointerException npe) { + throw new AlreadyClosedException("Already closed: " + this); + } + } + + @Override + public final void readBytes(byte[] b, int offset, int len) throws IOException { + try { + curBuf.get(b, offset, len); + } catch (BufferUnderflowException e) { + int curAvail = curBuf.remaining(); + while (len > curAvail) { + curBuf.get(b, offset, curAvail); + len -= curAvail; + offset += curAvail; + curBufIndex++; + if (curBufIndex >= buffers.length) { + throw new EOFException("read past EOF: " + this); + } + curBuf = buffers[curBufIndex]; + curBuf.position(0); + curAvail = curBuf.remaining(); + } + curBuf.get(b, offset, len); + } catch (NullPointerException npe) { + throw new AlreadyClosedException("Already closed: " + this); + } + } + + @Override + public final short readShort() throws IOException { + try { + return curBuf.getShort(); + } catch (BufferUnderflowException e) { + return super.readShort(); + } catch (NullPointerException npe) { + throw new AlreadyClosedException("Already closed: " + this); + } + } + + @Override + public final int readInt() throws IOException { + try { + return curBuf.getInt(); + } catch (BufferUnderflowException e) { + return super.readInt(); + } catch (NullPointerException npe) { + throw new AlreadyClosedException("Already closed: " + this); + } + } + + @Override + public final long readLong() throws IOException { + try { + return curBuf.getLong(); + } catch (BufferUnderflowException e) { + return super.readLong(); + } catch (NullPointerException npe) { + throw new AlreadyClosedException("Already closed: " + this); + } + } + + @Override + public final long getFilePointer() { + try { + return (((long) curBufIndex) << chunkSizePower) + curBuf.position() - offset; + } catch (NullPointerException npe) { + throw new AlreadyClosedException("Already closed: " + this); + } + } + + @Override + public final void seek(long pos) throws IOException { + pos += offset; + // we use >> here to preserve negative, so we will catch AIOOBE: + final int bi = (int) (pos >> chunkSizePower); + try { + final ByteBuffer b = buffers[bi]; + b.position((int) (pos & chunkSizeMask)); + // write values, on exception all is unchanged + this.curBufIndex = bi; + this.curBuf = b; + } catch (ArrayIndexOutOfBoundsException aioobe) { + if (pos < 0L) { + throw new IllegalArgumentException("Seeking to negative position: " + this); + } + throw new EOFException("seek past EOF: " + this); + } catch (IllegalArgumentException iae) { + if (pos < 0L) { + throw new IllegalArgumentException("Seeking to negative position: " + this); + } + throw new EOFException("seek past EOF: " + this); + } catch (NullPointerException npe) { + throw new AlreadyClosedException("Already closed: " + this); + } + } + + @Override + public final long length() { + return length; + } + + @Override + public final ByteBufferIndexInput clone() { + final ByteBufferIndexInput clone = buildSlice(0L, this.length); + try { + clone.seek(getFilePointer()); + } catch(IOException ioe) { + throw new RuntimeException("Should never happen: " + this, ioe); + } + + return clone; + } + + /** + * Creates a slice of this index input, with the given description, offset, and length. The slice is seeked to the beginning. + */ + public final ByteBufferIndexInput slice(String sliceDescription, long offset, long length) { + /* + if (isClone) { // well we could, but this is stupid + throw new IllegalStateException("cannot slice() " + sliceDescription + " from a cloned IndexInput: " + this); + } + */ + final ByteBufferIndexInput clone = buildSlice(offset, length); + clone.sliceDescription = sliceDescription; + try { + clone.seek(0L); + } catch(IOException ioe) { + throw new RuntimeException("Should never happen: " + this, ioe); + } + + return clone; + } + + private ByteBufferIndexInput buildSlice(long offset, long length) { + if (buffers == null) { + throw new AlreadyClosedException("Already closed: " + this); + } + if (offset < 0 || length < 0 || offset+length > this.length) { + throw new IllegalArgumentException("slice() " + sliceDescription + " out of bounds: offset=" + offset + ",length=" + length + ",fileLength=" + this.length + ": " + this); + } + + // include our own offset into the final offset: + offset += this.offset; + + final ByteBufferIndexInput clone = (ByteBufferIndexInput)super.clone(); + clone.isClone = true; + // we keep clone.clones, so it shares the same map with original and we have no additional cost on clones + assert clone.clones == this.clones; + clone.buffers = buildSlice(buffers, offset, length); + clone.offset = (int) (offset & chunkSizeMask); + clone.length = length; + + // register the new clone in our clone list to clean it up on closing: + this.clones.put(clone, Boolean.TRUE); + + return clone; + } + + /** Returns a sliced view from a set of already-existing buffers: + * the last buffer's limit() will be correct, but + * you must deal with offset separately (the first buffer will not be adjusted) */ + private ByteBuffer[] buildSlice(ByteBuffer[] buffers, long offset, long length) { + final long sliceEnd = offset + length; + + final int startIndex = (int) (offset >>> chunkSizePower); + final int endIndex = (int) (sliceEnd >>> chunkSizePower); + + // we always allocate one more slice, the last one may be a 0 byte one + final ByteBuffer slices[] = new ByteBuffer[endIndex - startIndex + 1]; + + for (int i = 0; i < slices.length; i++) { + slices[i] = buffers[startIndex + i].duplicate(); + } + + // set the last buffer's limit for the sliced view. + slices[slices.length - 1].limit((int) (sliceEnd & chunkSizeMask)); + + return slices; + } + + private void unsetBuffers() { + buffers = null; + curBuf = null; + curBufIndex = 0; + } + + @Override + public final void close() throws IOException { + try { + if (buffers == null) return; + + // make local copy, then un-set early + final ByteBuffer[] bufs = buffers; + unsetBuffers(); + + if (isClone) return; + + // for extra safety unset also all clones' buffers: + for (Iterator it = this.clones.keyIterator(); it.hasNext();) { + final ByteBufferIndexInput clone = it.next(); + assert clone.isClone; + clone.unsetBuffers(); + } + this.clones.clear(); + + for (final ByteBuffer b : bufs) { + freeBuffer(b); + } + } finally { + unsetBuffers(); + } + } + + /** + * Called when the contents of a buffer will be no longer needed. + */ + protected abstract void freeBuffer(ByteBuffer b) throws IOException; + + @Override + public final String toString() { + if (sliceDescription != null) { + return super.toString() + " [slice=" + sliceDescription + "]"; + } else { + return super.toString(); + } + } +} Index: lucene/core/src/java/org/apache/lucene/store/ByteBufferIndexInput.java =================================================================== --- lucene/core/src/java/org/apache/lucene/store/ByteBufferIndexInput.java (revision 0) +++ lucene/core/src/java/org/apache/lucene/store/ByteBufferIndexInput.java (working copy) Property changes on: lucene/core/src/java/org/apache/lucene/store/ByteBufferIndexInput.java ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Index: lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java =================================================================== --- lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java (revision 1381525) +++ lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java (working copy) @@ -17,25 +17,20 @@ * limitations under the License. */ -import java.io.EOFException; import java.io.IOException; import java.io.File; import java.io.RandomAccessFile; import java.nio.ByteBuffer; -import java.nio.BufferUnderflowException; import java.nio.channels.ClosedChannelException; // javadoc @link import java.nio.channels.FileChannel; import java.nio.channels.FileChannel.MapMode; -import java.util.Iterator; - import java.security.AccessController; import java.security.PrivilegedExceptionAction; import java.security.PrivilegedActionException; import java.lang.reflect.Method; import org.apache.lucene.util.Constants; -import org.apache.lucene.util.WeakIdentityMap; /** File-based {@link Directory} implementation that uses * mmap for reading, and {@link @@ -47,7 +42,7 @@ * be sure your have plenty of virtual address space, e.g. by * using a 64 bit JRE, or a 32 bit JRE with indexes that are * guaranteed to fit within the address space. - * On 32 bit platforms also consult {@link #setMaxChunkSize} + * On 32 bit platforms also consult {@link #MMapDirectory(File, LockFactory, int)} * if you have problems with mmap failing because of fragmented * address space. If you get an OutOfMemoryException, it is recommended * to reduce the chunk size, until it works. @@ -84,7 +79,7 @@ public class MMapDirectory extends FSDirectory { private boolean useUnmapHack = UNMAP_SUPPORTED; public static final int DEFAULT_MAX_BUFF = Constants.JRE_IS_64BIT ? (1 << 30) : (1 << 28); - private int chunkSizePower; + final int chunkSizePower; /** Create a new MMapDirectory for the named location. * @@ -94,8 +89,7 @@ * @throws IOException */ public MMapDirectory(File path, LockFactory lockFactory) throws IOException { - super(path, lockFactory); - setMaxChunkSize(DEFAULT_MAX_BUFF); + this(path, lockFactory, DEFAULT_MAX_BUFF); } /** Create a new MMapDirectory for the named location and {@link NativeFSLockFactory}. @@ -104,9 +98,37 @@ * @throws IOException */ public MMapDirectory(File path) throws IOException { - super(path, null); - setMaxChunkSize(DEFAULT_MAX_BUFF); + this(path, null); } + + /** + * Create a new MMapDirectory for the named location, specifying the + * maximum chunk size used for memory mapping. + * + * @param path the path of the directory + * @param lockFactory the lock factory to use, or null for the default + * ({@link NativeFSLockFactory}); + * @param maxChunkSize maximum chunk size (default is 1 GiBytes for + * 64 bit JVMs and 256 MiBytes for 32 bit JVMs) used for memory mapping. + *

+ * Especially on 32 bit platform, the address space can be very fragmented, + * so large index files cannot be mapped. Using a lower chunk size makes + * the directory implementation a little bit slower (as the correct chunk + * may be resolved on lots of seeks) but the chance is higher that mmap + * does not fail. On 64 bit Java platforms, this parameter should always + * be {@code 1 << 30}, as the address space is big enough. + *

+ * Please note: The chunk size is always rounded down to a power of 2. + * @throws IOException + */ + public MMapDirectory(File path, LockFactory lockFactory, int maxChunkSize) throws IOException { + super(path, lockFactory); + if (maxChunkSize <= 0) { + throw new IllegalArgumentException("Maximum chunk size for mmap must be >0"); + } + this.chunkSizePower = 31 - Integer.numberOfLeadingZeros(maxChunkSize); + assert this.chunkSizePower >= 0 && this.chunkSizePower <= 30; + } /** * true, if this platform supports unmapping mmapped files. @@ -181,30 +203,8 @@ } /** - * Sets the maximum chunk size (default is 1 GiBytes for - * 64 bit JVMs and 256 MiBytes for 32 bit JVMs) used for memory mapping. - * Especially on 32 bit platform, the address space can be very fragmented, - * so large index files cannot be mapped. - * Using a lower chunk size makes the directory implementation a little - * bit slower (as the correct chunk may be resolved on lots of seeks) - * but the chance is higher that mmap does not fail. On 64 bit - * Java platforms, this parameter should always be {@code 1 << 30}, - * as the address space is big enough. - * Please note: This method always rounds down the chunk size - * to a power of 2. - */ - public final void setMaxChunkSize(final int maxChunkSize) { - if (maxChunkSize <= 0) - throw new IllegalArgumentException("Maximum chunk size for mmap must be >0"); - //System.out.println("Requested chunk size: "+maxChunkSize); - this.chunkSizePower = 31 - Integer.numberOfLeadingZeros(maxChunkSize); - assert this.chunkSizePower >= 0 && this.chunkSizePower <= 30; - //System.out.println("Got chunk size: "+getMaxChunkSize()); - } - - /** * Returns the current mmap chunk size. - * @see #setMaxChunkSize + * @see #MMapDirectory(File, LockFactory, int) */ public final int getMaxChunkSize() { return 1 << chunkSizePower; @@ -217,254 +217,74 @@ File f = new File(getDirectory(), name); RandomAccessFile raf = new RandomAccessFile(f, "r"); try { - return new MMapIndexInput("MMapIndexInput(path=\"" + f + "\")", raf, 0, raf.length(), chunkSizePower); + return new MMapIndexInput("MMapIndexInput(path=\"" + f + "\")", raf); } finally { raf.close(); } } - public IndexInputSlicer createSlicer(final String name, final IOContext context) throws IOException { - ensureOpen(); - final File f = new File(getDirectory(), name); - final RandomAccessFile raf = new RandomAccessFile(f, "r"); + @Override + public IndexInputSlicer createSlicer(String name, IOContext context) throws IOException { + final MMapIndexInput full = (MMapIndexInput) openInput(name, context); return new IndexInputSlicer() { @Override - public void close() throws IOException { - raf.close(); - } - - @Override public IndexInput openSlice(String sliceDescription, long offset, long length) throws IOException { - return new MMapIndexInput("MMapIndexInput(" + sliceDescription + " in path=\"" + f + "\" slice=" + offset + ":" + (offset+length) + ")", raf, offset, length, chunkSizePower); + ensureOpen(); + final IndexInput slice = full.slice(sliceDescription, offset, length); + assert 0L == slice.getFilePointer(); + return slice; } @Override public IndexInput openFullSlice() throws IOException { - return openSlice("full-slice", 0, raf.length()); + ensureOpen(); + final IndexInput slice = full.clone(); + assert 0L == slice.getFilePointer(); + return slice; } + + @Override + public void close() throws IOException { + full.close(); + } }; } - // Because Java's ByteBuffer uses an int to address the - // values, it's necessary to access a file > - // Integer.MAX_VALUE in size using multiple byte buffers. - private final class MMapIndexInput extends IndexInput { - - private ByteBuffer[] buffers; - - private final long length, chunkSizeMask, chunkSize; - private final int chunkSizePower; - - private int curBufIndex; - - private ByteBuffer curBuf; // redundant for speed: buffers[curBufIndex] - - private boolean isClone = false; - private final WeakIdentityMap clones = WeakIdentityMap.newConcurrentHashMap(); - - MMapIndexInput(String resourceDescription, RandomAccessFile raf, long offset, long length, int chunkSizePower) throws IOException { - super(resourceDescription); - this.length = length; - this.chunkSizePower = chunkSizePower; - this.chunkSize = 1L << chunkSizePower; - this.chunkSizeMask = chunkSize - 1L; - - if (chunkSizePower < 0 || chunkSizePower > 30) - throw new IllegalArgumentException("Invalid chunkSizePower used for ByteBuffer size: " + chunkSizePower); - - if ((length >>> chunkSizePower) >= Integer.MAX_VALUE) - throw new IllegalArgumentException("RandomAccessFile too big for chunk size: " + raf.toString()); - - // we always allocate one more buffer, the last one may be a 0 byte one - final int nrBuffers = (int) (length >>> chunkSizePower) + 1; - - //System.out.println("length="+length+", chunkSizePower=" + chunkSizePower + ", chunkSizeMask=" + chunkSizeMask + ", nrBuffers=" + nrBuffers); - - this.buffers = new ByteBuffer[nrBuffers]; - - long bufferStart = 0L; - FileChannel rafc = raf.getChannel(); - for (int bufNr = 0; bufNr < nrBuffers; bufNr++) { - int bufSize = (int) ( (length > (bufferStart + chunkSize)) - ? chunkSize - : (length - bufferStart) - ); - this.buffers[bufNr] = rafc.map(MapMode.READ_ONLY, offset + bufferStart, bufSize); - bufferStart += bufSize; - } - seek(0L); + private final class MMapIndexInput extends ByteBufferIndexInput { + + MMapIndexInput(String resourceDescription, RandomAccessFile raf) throws IOException { + super(resourceDescription, map(raf, 0, raf.length()), raf.length(), chunkSizePower); } - + @Override - public byte readByte() throws IOException { - try { - return curBuf.get(); - } catch (BufferUnderflowException e) { - do { - curBufIndex++; - if (curBufIndex >= buffers.length) { - throw new EOFException("read past EOF: " + this); - } - curBuf = buffers[curBufIndex]; - curBuf.position(0); - } while (!curBuf.hasRemaining()); - return curBuf.get(); - } catch (NullPointerException npe) { - throw new AlreadyClosedException("MMapIndexInput already closed: " + this); - } + protected void freeBuffer(ByteBuffer buffer) throws IOException { + cleanMapping(buffer); } + } - @Override - public void readBytes(byte[] b, int offset, int len) throws IOException { - try { - curBuf.get(b, offset, len); - } catch (BufferUnderflowException e) { - int curAvail = curBuf.remaining(); - while (len > curAvail) { - curBuf.get(b, offset, curAvail); - len -= curAvail; - offset += curAvail; - curBufIndex++; - if (curBufIndex >= buffers.length) { - throw new EOFException("read past EOF: " + this); - } - curBuf = buffers[curBufIndex]; - curBuf.position(0); - curAvail = curBuf.remaining(); - } - curBuf.get(b, offset, len); - } catch (NullPointerException npe) { - throw new AlreadyClosedException("MMapIndexInput already closed: " + this); - } - } - - @Override - public short readShort() throws IOException { - try { - return curBuf.getShort(); - } catch (BufferUnderflowException e) { - return super.readShort(); - } catch (NullPointerException npe) { - throw new AlreadyClosedException("MMapIndexInput already closed: " + this); - } - } - - @Override - public int readInt() throws IOException { - try { - return curBuf.getInt(); - } catch (BufferUnderflowException e) { - return super.readInt(); - } catch (NullPointerException npe) { - throw new AlreadyClosedException("MMapIndexInput already closed: " + this); - } - } - - @Override - public long readLong() throws IOException { - try { - return curBuf.getLong(); - } catch (BufferUnderflowException e) { - return super.readLong(); - } catch (NullPointerException npe) { - throw new AlreadyClosedException("MMapIndexInput already closed: " + this); - } - } + /** Maps a file into a set of buffers */ + ByteBuffer[] map(RandomAccessFile raf, long offset, long length) throws IOException { + if ((length >>> chunkSizePower) >= Integer.MAX_VALUE) + throw new IllegalArgumentException("RandomAccessFile too big for chunk size: " + raf.toString()); - @Override - public long getFilePointer() { - try { - return (((long) curBufIndex) << chunkSizePower) + curBuf.position(); - } catch (NullPointerException npe) { - throw new AlreadyClosedException("MMapIndexInput already closed: " + this); - } + final long chunkSize = 1L << chunkSizePower; + + // we always allocate one more buffer, the last one may be a 0 byte one + final int nrBuffers = (int) (length >>> chunkSizePower) + 1; + + ByteBuffer buffers[] = new ByteBuffer[nrBuffers]; + + long bufferStart = 0L; + FileChannel rafc = raf.getChannel(); + for (int bufNr = 0; bufNr < nrBuffers; bufNr++) { + int bufSize = (int) ( (length > (bufferStart + chunkSize)) + ? chunkSize + : (length - bufferStart) + ); + buffers[bufNr] = rafc.map(MapMode.READ_ONLY, offset + bufferStart, bufSize); + bufferStart += bufSize; } - - @Override - public void seek(long pos) throws IOException { - // we use >> here to preserve negative, so we will catch AIOOBE: - final int bi = (int) (pos >> chunkSizePower); - try { - final ByteBuffer b = buffers[bi]; - b.position((int) (pos & chunkSizeMask)); - // write values, on exception all is unchanged - this.curBufIndex = bi; - this.curBuf = b; - } catch (ArrayIndexOutOfBoundsException aioobe) { - if (pos < 0L) { - throw new IllegalArgumentException("Seeking to negative position: " + this); - } - throw new EOFException("seek past EOF: " + this); - } catch (IllegalArgumentException iae) { - if (pos < 0L) { - throw new IllegalArgumentException("Seeking to negative position: " + this); - } - throw new EOFException("seek past EOF: " + this); - } catch (NullPointerException npe) { - throw new AlreadyClosedException("MMapIndexInput already closed: " + this); - } - } - - @Override - public long length() { - return length; - } - - @Override - public MMapIndexInput clone() { - if (buffers == null) { - throw new AlreadyClosedException("MMapIndexInput already closed: " + this); - } - final MMapIndexInput clone = (MMapIndexInput)super.clone(); - clone.isClone = true; - // we keep clone.clones, so it shares the same map with original and we have no additional cost on clones - assert clone.clones == this.clones; - clone.buffers = new ByteBuffer[buffers.length]; - for (int bufNr = 0; bufNr < buffers.length; bufNr++) { - clone.buffers[bufNr] = buffers[bufNr].duplicate(); - } - try { - clone.seek(getFilePointer()); - } catch(IOException ioe) { - throw new RuntimeException("Should never happen: " + this, ioe); - } - - // register the new clone in our clone list to clean it up on closing: - this.clones.put(clone, Boolean.TRUE); - - return clone; - } - private void unsetBuffers() { - buffers = null; - curBuf = null; - curBufIndex = 0; - } - - @Override - public void close() throws IOException { - try { - if (isClone || buffers == null) return; - - // make local copy, then un-set early - final ByteBuffer[] bufs = buffers; - unsetBuffers(); - - // for extra safety unset also all clones' buffers: - for (Iterator it = this.clones.keyIterator(); it.hasNext();) { - final MMapIndexInput clone = it.next(); - assert clone.isClone; - clone.unsetBuffers(); - } - this.clones.clear(); - - for (final ByteBuffer b : bufs) { - cleanMapping(b); - } - } finally { - unsetBuffers(); - } - } + return buffers; } - } Index: lucene/core/src/test/org/apache/lucene/store/TestMultiMMap.java =================================================================== --- lucene/core/src/test/org/apache/lucene/store/TestMultiMMap.java (revision 1381525) +++ lucene/core/src/test/org/apache/lucene/store/TestMultiMMap.java (working copy) @@ -78,8 +78,7 @@ public void testSeekZero() throws Exception { for (int i = 0; i < 31; i++) { - MMapDirectory mmapDir = new MMapDirectory(_TestUtil.getTempDir("testSeekZero")); - mmapDir.setMaxChunkSize(1<