You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2012/09/08 04:41:58 UTC

svn commit: r1382224 - in /lucene/dev/trunk: lucene/ lucene/core/src/java/org/apache/lucene/store/ lucene/core/src/test/org/apache/lucene/store/ lucene/test-framework/src/java/org/apache/lucene/store/ solr/core/src/java/org/apache/solr/core/

Author: rmuir
Date: Sat Sep  8 02:41:57 2012
New Revision: 1382224

URL: http://svn.apache.org/viewvc?rev=1382224&view=rev
Log:
LUCENE-4364: MMapDirectory makes too many maps for CFS

Added:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/ByteBufferIndexInput.java   (with props)
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/CompoundFileDirectory.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/Directory.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/NIOFSDirectory.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/SimpleFSDirectory.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestMultiMMap.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/MMapDirectoryFactory.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1382224&r1=1382223&r2=1382224&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Sat Sep  8 02:41:57 2012
@@ -158,6 +158,11 @@ Optimizations
 * LUCENE-4327: Support out-of-order scoring in FilteredQuery for higher
   performance.  (Mike McCandless, Robert Muir)
 
+* LUCENE-4364: Optimize MMapDirectory to not make a mapping per-cfs-slice,
+  instead one map per .cfs file. This reduces the total number of maps.
+  Additionally factor out a (package-private) generic 
+  ByteBufferIndexInput from MMapDirectory.  (Uwe Schindler, Robert Muir)
+
 Build
 
 * LUCENE-4360: Support running the same test suite multiple times in 

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/ByteBufferIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/ByteBufferIndexInput.java?rev=1382224&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/ByteBufferIndexInput.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/ByteBufferIndexInput.java Sat Sep  8 02:41:57 2012
@@ -0,0 +1,307 @@
+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.
+ * <p>
+ * 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.
+ * <p>
+ * For efficiency, this class requires that the buffers
+ * are a power-of-two (<code>chunkSizePower</code>).
+ */
+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<ByteBufferIndexInput,Boolean> 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 {
+    // necessary in case offset != 0 and pos < 0, but pos >= -offset
+    if (pos < 0L) {
+      throw new IllegalArgumentException("Seeking to negative position: " + this);
+    }
+    pos += offset;
+    // we use >> here to preserve negative, so we will catch AIOOBE,
+    // in case pos + offset overflows.
+    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) {
+      throw new EOFException("seek past EOF: " + this);
+    } catch (IllegalArgumentException iae) {
+      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<ByteBufferIndexInput> 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();
+    }
+  }
+}

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/CompoundFileDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/CompoundFileDirectory.java?rev=1382224&r1=1382223&r2=1382224&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/CompoundFileDirectory.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/CompoundFileDirectory.java Sat Sep  8 02:41:57 2012
@@ -22,7 +22,6 @@ import org.apache.lucene.codecs.CodecUti
 import org.apache.lucene.codecs.LiveDocsFormat; // javadocs
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.IndexFormatTooOldException;
 import org.apache.lucene.store.DataOutput; // javadocs
 import org.apache.lucene.util.IOUtils;
 
@@ -102,7 +101,7 @@ public final class CompoundFileDirectory
       boolean success = false;
       handle = directory.createSlicer(fileName, context);
       try {
-        this.entries = readEntries(handle, directory, fileName);
+        this.entries = readEntries(directory, fileName);
         success = true;
       } finally {
         if (!success) {
@@ -121,44 +120,32 @@ public final class CompoundFileDirectory
   }
 
   /** Helper method that reads CFS entries from an input stream */
-  private static final Map<String, FileEntry> readEntries(
-      IndexInputSlicer handle, Directory dir, String name) throws IOException {
+  private static final Map<String, FileEntry> readEntries(Directory dir, String name) throws IOException {
     IOException priorE = null;
-    IndexInput stream = null, entriesStream = null;
+    IndexInput entriesStream = null;
     try {
-      stream = handle.openFullSlice();
-      final int firstInt = stream.readInt();
-      // NOTE: as long as we want to throw indexformattooold (vs corruptindexexception), we need
-      // to read the magic ourselves. See SegmentInfos which also has this.
-      if (firstInt == CodecUtil.CODEC_MAGIC) {
-        CodecUtil.checkHeaderNoMagic(stream, CompoundFileWriter.DATA_CODEC, 
-            CompoundFileWriter.VERSION_START, CompoundFileWriter.VERSION_START);
-        final String entriesFileName = IndexFileNames.segmentFileName(
-                                              IndexFileNames.stripExtension(name), "",
-                                              IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION);
-        entriesStream = dir.openInput(entriesFileName, IOContext.READONCE);
-        CodecUtil.checkHeader(entriesStream, CompoundFileWriter.ENTRY_CODEC, CompoundFileWriter.VERSION_START, CompoundFileWriter.VERSION_START);
-        final int numEntries = entriesStream.readVInt();
-        final Map<String, FileEntry> mapping = new HashMap<String,FileEntry>(numEntries);
-        for (int i = 0; i < numEntries; i++) {
-          final FileEntry fileEntry = new FileEntry();
-          final String id = entriesStream.readString();
-          FileEntry previous = mapping.put(id, fileEntry);
-          if (previous != null) {
-            throw new CorruptIndexException("Duplicate cfs entry id=" + id + " in CFS: " + entriesStream);
-          }
-          fileEntry.offset = entriesStream.readLong();
-          fileEntry.length = entriesStream.readLong();
+      final String entriesFileName = IndexFileNames.segmentFileName(
+                                            IndexFileNames.stripExtension(name), "",
+                                             IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION);
+      entriesStream = dir.openInput(entriesFileName, IOContext.READONCE);
+      CodecUtil.checkHeader(entriesStream, CompoundFileWriter.ENTRY_CODEC, CompoundFileWriter.VERSION_START, CompoundFileWriter.VERSION_START);
+      final int numEntries = entriesStream.readVInt();
+      final Map<String, FileEntry> mapping = new HashMap<String,FileEntry>(numEntries);
+      for (int i = 0; i < numEntries; i++) {
+        final FileEntry fileEntry = new FileEntry();
+        final String id = entriesStream.readString();
+        FileEntry previous = mapping.put(id, fileEntry);
+        if (previous != null) {
+          throw new CorruptIndexException("Duplicate cfs entry id=" + id + " in CFS: " + entriesStream);
         }
-        return mapping;
-      } else {
-        throw new IndexFormatTooOldException(stream, firstInt,
-            CodecUtil.CODEC_MAGIC, CodecUtil.CODEC_MAGIC);
+        fileEntry.offset = entriesStream.readLong();
+        fileEntry.length = entriesStream.readLong();
       }
+      return mapping;
     } catch (IOException ioe) {
       priorE = ioe;
     } finally {
-      IOUtils.closeWhileHandlingException(priorE, stream, entriesStream);
+      IOUtils.closeWhileHandlingException(priorE, entriesStream);
     }
     // this is needed until Java 7's real try-with-resources:
     throw new AssertionError("impossible to get here");
@@ -291,11 +278,6 @@ public final class CompoundFileDirectory
       public IndexInput openSlice(String sliceDescription, long offset, long length) throws IOException {
         return handle.openSlice(sliceDescription, entry.offset + offset, length);
       }
-
-      @Override
-      public IndexInput openFullSlice() throws IOException {
-        return openSlice("full-slice", 0, entry.length);
-      }
     };
   }
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/Directory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/Directory.java?rev=1382224&r1=1382223&r2=1382224&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/Directory.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/Directory.java Sat Sep  8 02:41:57 2012
@@ -229,10 +229,6 @@ public abstract class Directory implemen
       public void close() throws IOException {
         base.close();
       }
-      @Override
-      public IndexInput openFullSlice() {
-        return base.clone();
-      }
     };
   }
 
@@ -256,12 +252,6 @@ public abstract class Directory implemen
      * Returns an {@link IndexInput} slice starting at the given offset with the given length.
      */
     public abstract IndexInput openSlice(String sliceDescription, long offset, long length) throws IOException;
-
-    /**
-     * Returns an {@link IndexInput} slice starting at offset <i>0</i> with a
-     * length equal to the length of the underlying file
-     */
-    public abstract IndexInput openFullSlice() throws IOException;
   }
   
   /** Implementation of an IndexInput that reads from a portion of

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java?rev=1382224&r1=1382223&r2=1382224&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java Sat Sep  8 02:41:57 2012
@@ -17,25 +17,20 @@ package org.apache.lucene.store;
  * 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 @@ import org.apache.lucene.util.WeakIdenti
  * 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 @@ import org.apache.lucene.util.WeakIdenti
 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 @@ public class MMapDirectory extends FSDir
    * @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,8 +98,36 @@ public class MMapDirectory extends FSDir
    * @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.
+   * <p>
+   * 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.
+   * <p>
+   * <b>Please note:</b> 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;
   }
 
   /**
@@ -181,30 +203,8 @@ public class MMapDirectory extends FSDir
   }
   
   /**
-   * 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.
-   * <b>Please note:</b> 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,64 @@ public class MMapDirectory extends FSDir
     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();
+        return full.slice(sliceDescription, offset, length);
       }
-
+      
       @Override
-      public IndexInput openFullSlice() throws IOException {
-        return openSlice("full-slice", 0, raf.length());
+      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<MMapIndexInput,Boolean> 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);
-    }
-  
-    @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);
-      }
-    }
-  
-    @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);
-      }
-    }
+  private final class MMapIndexInput extends ByteBufferIndexInput {
     
-    @Override
-    public long getFilePointer() {
-      try {
-        return (((long) curBufIndex) << chunkSizePower) + curBuf.position();
-      } catch (NullPointerException npe) {
-        throw new AlreadyClosedException("MMapIndexInput already closed: " + this);
-      }
+    MMapIndexInput(String resourceDescription, RandomAccessFile raf) throws IOException {
+      super(resourceDescription, map(raf, 0, raf.length()), raf.length(), chunkSizePower);
     }
-  
-    @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;
+    protected void freeBuffer(ByteBuffer buffer) throws IOException {
+      cleanMapping(buffer);
     }
+  }
   
-    @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;
-    }
+  /** 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());
     
-    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<MMapIndexInput> 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();
-      }
+    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;
     }
+    
+    return buffers;
   }
-
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/NIOFSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/NIOFSDirectory.java?rev=1382224&r1=1382223&r2=1382224&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/NIOFSDirectory.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/NIOFSDirectory.java Sat Sep  8 02:41:57 2012
@@ -98,11 +98,6 @@ public class NIOFSDirectory extends FSDi
         return new NIOFSIndexInput(sliceDescription, path, descriptor, descriptor.getChannel(), offset,
             length, BufferedIndexInput.bufferSize(context), getReadChunkSize());
       }
-
-      @Override
-      public IndexInput openFullSlice() {
-        return openSlice("full-slice", 0, descriptor.length);
-      }
     };
   }
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/SimpleFSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/SimpleFSDirectory.java?rev=1382224&r1=1382223&r2=1382224&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/SimpleFSDirectory.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/SimpleFSDirectory.java Sat Sep  8 02:41:57 2012
@@ -77,11 +77,6 @@ public class SimpleFSDirectory extends F
         return new SimpleFSIndexInput("SimpleFSIndexInput(" + sliceDescription + " in path=\"" + file.getPath() + "\" slice=" + offset + ":" + (offset+length) + ")", descriptor, offset,
             length, BufferedIndexInput.bufferSize(context), getReadChunkSize());
       }
-
-      @Override
-      public IndexInput openFullSlice() {
-        return openSlice("full-slice", 0, descriptor.length);
-      }
     };
   }
 

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestMultiMMap.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestMultiMMap.java?rev=1382224&r1=1382223&r2=1382224&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestMultiMMap.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestMultiMMap.java Sat Sep  8 02:41:57 2012
@@ -25,6 +25,7 @@ import org.apache.lucene.document.Docume
 import org.apache.lucene.document.Field;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.store.Directory.IndexInputSlicer;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util._TestUtil;
@@ -75,11 +76,48 @@ public class TestMultiMMap extends Lucen
       // pass
     }
   }
+  
+  public void testCloneSliceSafety() throws Exception {
+    MMapDirectory mmapDir = new MMapDirectory(_TestUtil.getTempDir("testCloneSliceSafety"));
+    IndexOutput io = mmapDir.createOutput("bytes", newIOContext(random()));
+    io.writeInt(1);
+    io.writeInt(2);
+    io.close();
+    IndexInputSlicer slicer = mmapDir.createSlicer("bytes", newIOContext(random()));
+    IndexInput one = slicer.openSlice("first int", 0, 4);
+    IndexInput two = slicer.openSlice("second int", 4, 4);
+    IndexInput three = one.clone(); // clone of clone
+    IndexInput four = two.clone(); // clone of clone
+    slicer.close();
+    try {
+      one.readInt();
+      fail("Must throw AlreadyClosedException");
+    } catch (AlreadyClosedException ignore) {
+      // pass
+    }
+    try {
+      two.readInt();
+      fail("Must throw AlreadyClosedException");
+    } catch (AlreadyClosedException ignore) {
+      // pass
+    }
+    try {
+      three.readInt();
+      fail("Must throw AlreadyClosedExveption");
+    } catch (AlreadyClosedException ignore) {
+      // pass
+    }
+    try {
+      four.readInt();
+      fail("Must throw AlreadyClosedExveption");
+    } catch (AlreadyClosedException ignore) {
+      // pass
+    }
+  }
 
   public void testSeekZero() throws Exception {
     for (int i = 0; i < 31; i++) {
-      MMapDirectory mmapDir = new MMapDirectory(_TestUtil.getTempDir("testSeekZero"));
-      mmapDir.setMaxChunkSize(1<<i);
+      MMapDirectory mmapDir = new MMapDirectory(_TestUtil.getTempDir("testSeekZero"), null, 1<<i);
       IndexOutput io = mmapDir.createOutput("zeroBytes", newIOContext(random()));
       io.close();
       IndexInput ii = mmapDir.openInput("zeroBytes", newIOContext(random()));
@@ -89,10 +127,23 @@ public class TestMultiMMap extends Lucen
     }
   }
   
+  public void testSeekSliceZero() throws Exception {
+    for (int i = 0; i < 31; i++) {
+      MMapDirectory mmapDir = new MMapDirectory(_TestUtil.getTempDir("testSeekSliceZero"), null, 1<<i);
+      IndexOutput io = mmapDir.createOutput("zeroBytes", newIOContext(random()));
+      io.close();
+      IndexInputSlicer slicer = mmapDir.createSlicer("zeroBytes", newIOContext(random()));
+      IndexInput ii = slicer.openSlice("zero-length slice", 0, 0);
+      ii.seek(0L);
+      ii.close();
+      slicer.close();
+      mmapDir.close();
+    }
+  }
+  
   public void testSeekEnd() throws Exception {
     for (int i = 0; i < 17; i++) {
-      MMapDirectory mmapDir = new MMapDirectory(_TestUtil.getTempDir("testSeekEnd"));
-      mmapDir.setMaxChunkSize(1<<i);
+      MMapDirectory mmapDir = new MMapDirectory(_TestUtil.getTempDir("testSeekEnd"), null, 1<<i);
       IndexOutput io = mmapDir.createOutput("bytes", newIOContext(random()));
       byte bytes[] = new byte[1<<i];
       random().nextBytes(bytes);
@@ -108,10 +159,28 @@ public class TestMultiMMap extends Lucen
     }
   }
   
+  public void testSeekSliceEnd() throws Exception {
+    for (int i = 0; i < 17; i++) {
+      MMapDirectory mmapDir = new MMapDirectory(_TestUtil.getTempDir("testSeekSliceEnd"), null, 1<<i);
+      IndexOutput io = mmapDir.createOutput("bytes", newIOContext(random()));
+      byte bytes[] = new byte[1<<i];
+      random().nextBytes(bytes);
+      io.writeBytes(bytes, bytes.length);
+      io.close();
+      IndexInputSlicer slicer = mmapDir.createSlicer("bytes", newIOContext(random()));
+      IndexInput ii = slicer.openSlice("full slice", 0, bytes.length);
+      byte actual[] = new byte[1<<i];
+      ii.readBytes(actual, 0, actual.length);
+      assertEquals(new BytesRef(bytes), new BytesRef(actual));
+      ii.seek(1<<i);
+      ii.close();
+      mmapDir.close();
+    }
+  }
+  
   public void testSeeking() throws Exception {
     for (int i = 0; i < 10; i++) {
-      MMapDirectory mmapDir = new MMapDirectory(_TestUtil.getTempDir("testSeeking"));
-      mmapDir.setMaxChunkSize(1<<i);
+      MMapDirectory mmapDir = new MMapDirectory(_TestUtil.getTempDir("testSeeking"), null, 1<<i);
       IndexOutput io = mmapDir.createOutput("bytes", newIOContext(random()));
       byte bytes[] = new byte[1<<(i+1)]; // make sure we switch buffers
       random().nextBytes(bytes);
@@ -134,6 +203,36 @@ public class TestMultiMMap extends Lucen
     }
   }
   
+  // note instead of seeking to offset and reading length, this opens slices at the 
+  // the various offset+length and just does readBytes.
+  public void testSlicedSeeking() throws Exception {
+    for (int i = 0; i < 10; i++) {
+      MMapDirectory mmapDir = new MMapDirectory(_TestUtil.getTempDir("testSlicedSeeking"), null, 1<<i);
+      IndexOutput io = mmapDir.createOutput("bytes", newIOContext(random()));
+      byte bytes[] = new byte[1<<(i+1)]; // make sure we switch buffers
+      random().nextBytes(bytes);
+      io.writeBytes(bytes, bytes.length);
+      io.close();
+      IndexInput ii = mmapDir.openInput("bytes", newIOContext(random()));
+      byte actual[] = new byte[1<<(i+1)]; // first read all bytes
+      ii.readBytes(actual, 0, actual.length);
+      ii.close();
+      assertEquals(new BytesRef(bytes), new BytesRef(actual));
+      IndexInputSlicer slicer = mmapDir.createSlicer("bytes", newIOContext(random()));
+      for (int sliceStart = 0; sliceStart < bytes.length; sliceStart++) {
+        for (int sliceLength = 0; sliceLength < bytes.length - sliceStart; sliceLength++) {
+          byte slice[] = new byte[sliceLength];
+          IndexInput input = slicer.openSlice("bytesSlice", sliceStart, slice.length);
+          input.readBytes(slice, 0, slice.length);
+          input.close();
+          assertEquals(new BytesRef(bytes, sliceStart, sliceLength), new BytesRef(slice));
+        }
+      }
+      slicer.close();
+      mmapDir.close();
+    }
+  }
+  
   public void testRandomChunkSizes() throws Exception {
     int num = atLeast(10);
     for (int i = 0; i < num; i++)
@@ -144,8 +243,7 @@ public class TestMultiMMap extends Lucen
     File path = _TestUtil.createTempFile("mmap" + chunkSize, "tmp", workDir);
     path.delete();
     path.mkdirs();
-    MMapDirectory mmapDir = new MMapDirectory(path);
-    mmapDir.setMaxChunkSize(chunkSize);
+    MMapDirectory mmapDir = new MMapDirectory(path, null, chunkSize);
     // we will map a lot, try to turn on the unmap hack
     if (MMapDirectory.UNMAP_SUPPORTED)
       mmapDir.setUseUnmap(true);

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java?rev=1382224&r1=1382223&r2=1382224&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java Sat Sep  8 02:41:57 2012
@@ -790,14 +790,6 @@ public class MockDirectoryWrapper extend
         addFileHandle(ii, name, Handle.Input);
         return ii;
       }
-
-      @Override
-      public IndexInput openFullSlice() throws IOException {
-        maybeYield();
-        IndexInput ii = new MockIndexInputWrapper(MockDirectoryWrapper.this, name, delegateHandle.openFullSlice());
-        addFileHandle(ii, name, Handle.Input);
-        return ii;
-      }
       
     };
     addFileHandle(handle, name, Handle.Slice);

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/MMapDirectoryFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/MMapDirectoryFactory.java?rev=1382224&r1=1382223&r2=1382224&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/MMapDirectoryFactory.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/MMapDirectoryFactory.java Sat Sep  8 02:41:57 2012
@@ -18,6 +18,7 @@ package org.apache.solr.core;
 
 
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.LockFactory; // javadocs
 import org.apache.lucene.store.MMapDirectory;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
@@ -33,8 +34,8 @@ import java.io.IOException;
  *
  * Can set the following parameters:
  * <ul>
- *  <li>unmap -- See {@link org.apache.lucene.store.MMapDirectory#setUseUnmap(boolean)}</li>
- *  <li>maxChunkSize -- The Max chunk size.  See {@link org.apache.lucene.store.MMapDirectory#setMaxChunkSize(int)}</li>
+ *  <li>unmap -- See {@link MMapDirectory#setUseUnmap(boolean)}</li>
+ *  <li>maxChunkSize -- The Max chunk size.  See {@link MMapDirectory#MMapDirectory(File, LockFactory, int)}</li>
  * </ul>
  *
  **/
@@ -55,13 +56,12 @@ public class MMapDirectoryFactory extend
 
   @Override
   protected Directory create(String path) throws IOException {
-    MMapDirectory mapDirectory = new MMapDirectory(new File(path));
+    MMapDirectory mapDirectory = new MMapDirectory(new File(path), null, maxChunk);
     try {
       mapDirectory.setUseUnmap(unmapHack);
     } catch (Exception e) {
       log.warn("Unmap not supported on this JVM, continuing on without setting unmap", e);
     }
-    mapDirectory.setMaxChunkSize(maxChunk);
     return mapDirectory;
   }
 }