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/10 14:39:49 UTC

svn commit: r1382800 - in /lucene/dev/branches/branch_4x: ./ dev-tools/eclipse/ lucene/ lucene/core/ lucene/core/src/java/org/apache/lucene/store/ lucene/core/src/test/org/apache/lucene/store/ lucene/test-framework/ solr/ solr/core/ solr/core/src/java/...

Author: rmuir
Date: Mon Sep 10 12:39:48 2012
New Revision: 1382800

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

Added:
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/ByteBufferIndexInput.java
      - copied unchanged from r1382224, lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/ByteBufferIndexInput.java
Modified:
    lucene/dev/branches/branch_4x/   (props changed)
    lucene/dev/branches/branch_4x/dev-tools/eclipse/dot.classpath
    lucene/dev/branches/branch_4x/lucene/   (props changed)
    lucene/dev/branches/branch_4x/lucene/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_4x/lucene/core/   (props changed)
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/Directory.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java
    lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/store/TestMultiMMap.java
    lucene/dev/branches/branch_4x/lucene/test-framework/   (props changed)
    lucene/dev/branches/branch_4x/solr/   (props changed)
    lucene/dev/branches/branch_4x/solr/core/   (props changed)
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/MMapDirectoryFactory.java

Modified: lucene/dev/branches/branch_4x/dev-tools/eclipse/dot.classpath
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/dev-tools/eclipse/dot.classpath?rev=1382800&r1=1382799&r2=1382800&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/dev-tools/eclipse/dot.classpath (original)
+++ lucene/dev/branches/branch_4x/dev-tools/eclipse/dot.classpath Mon Sep 10 12:39:48 2012
@@ -63,7 +63,7 @@
 	<classpathentry kind="src" path="lucene/spatial/src/java"/>
 	<classpathentry kind="src" path="lucene/spatial/src/test"/>
 	<classpathentry kind="src" path="lucene/spatial/src/test-files"/>
-	<classpathentry kind="lib" path="lucene/spatial/lib/spatial4j-0.2.jar"/>
+	<classpathentry kind="lib" path="lucene/spatial/lib/spatial4j-0.3.jar"/>
 	<classpathentry kind="src" path="solr/core/src/java"/>
 	<classpathentry kind="src" path="solr/core/src/test"/>
 	<classpathentry kind="src" path="solr/core/src/test-files"/>
@@ -108,21 +108,21 @@
 	<classpathentry kind="lib" path="lucene/benchmark/lib/commons-compress-1.4.1.jar"/>
 	<classpathentry kind="lib" path="lucene/benchmark/lib/xercesImpl-2.9.1.jar"/>
 	<classpathentry kind="lib" path="lucene/benchmark/lib/nekohtml-1.9.15.jar"/>
-	<classpathentry kind="lib" path="solr/lib/commons-fileupload-1.2.1.jar"/>
-	<classpathentry kind="lib" path="solr/lib/commons-cli-1.2.jar"/>
-	<classpathentry kind="lib" path="solr/lib/httpclient-4.1.3.jar"/>
-	<classpathentry kind="lib" path="solr/lib/httpcore-4.1.4.jar"/>
-	<classpathentry kind="lib" path="solr/lib/httpmime-4.1.3.jar"/>
-	<classpathentry kind="lib" path="solr/lib/commons-io-2.1.jar"/>
-	<classpathentry kind="lib" path="solr/lib/commons-lang-2.6.jar"/>
-	<classpathentry kind="lib" path="solr/lib/easymock-2.2.jar"/>
-	<classpathentry kind="lib" path="solr/lib/guava-r05.jar"/>
-	<classpathentry kind="lib" path="solr/lib/jcl-over-slf4j-1.6.4.jar"/>
-	<classpathentry kind="lib" path="solr/lib/log4j-over-slf4j-1.6.4.jar"/>
-	<classpathentry kind="lib" path="solr/lib/slf4j-api-1.6.4.jar"/>
-	<classpathentry kind="lib" path="solr/lib/slf4j-jdk14-1.6.4.jar"/>
-	<classpathentry kind="lib" path="solr/lib/wstx-asl-3.2.7.jar"/>
-	<classpathentry kind="lib" path="solr/lib/zookeeper-3.3.6.jar"/>
+	<classpathentry kind="lib" path="solr/core/lib/commons-fileupload-1.2.1.jar"/>
+	<classpathentry kind="lib" path="solr/core/lib/commons-cli-1.2.jar"/>
+	<classpathentry kind="lib" path="solr/solrj/lib/httpclient-4.1.3.jar"/>
+	<classpathentry kind="lib" path="solr/solrj/lib/httpcore-4.1.4.jar"/>
+	<classpathentry kind="lib" path="solr/solrj/lib/httpmime-4.1.3.jar"/>
+	<classpathentry kind="lib" path="solr/solrj/lib/commons-io-2.1.jar"/>
+	<classpathentry kind="lib" path="solr/core/lib/commons-lang-2.6.jar"/>
+	<classpathentry kind="lib" path="solr/core/lib/easymock-2.2.jar"/>
+	<classpathentry kind="lib" path="solr/core/lib/guava-r05.jar"/>
+	<classpathentry kind="lib" path="solr/solrj/lib/jcl-over-slf4j-1.6.4.jar"/>
+	<classpathentry kind="lib" path="solr/solrj/lib/log4j-over-slf4j-1.6.4.jar"/>
+	<classpathentry kind="lib" path="solr/solrj/lib/slf4j-api-1.6.4.jar"/>
+	<classpathentry kind="lib" path="solr/solrj/lib/slf4j-jdk14-1.6.4.jar"/>
+	<classpathentry kind="lib" path="solr/solrj/lib/wstx-asl-3.2.7.jar"/>
+	<classpathentry kind="lib" path="solr/solrj/lib/zookeeper-3.3.6.jar"/>
   <classpathentry kind="lib" path="solr/example/lib/jetty-continuation-8.1.2.v20120308.jar"/>
   <classpathentry kind="lib" path="solr/example/lib/jetty-deploy-8.1.2.v20120308.jar"/>
   <classpathentry kind="lib" path="solr/example/lib/jetty-http-8.1.2.v20120308.jar"/>

Modified: lucene/dev/branches/branch_4x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/CHANGES.txt?rev=1382800&r1=1382799&r2=1382800&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_4x/lucene/CHANGES.txt Mon Sep 10 12:39:48 2012
@@ -148,6 +148,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 

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/Directory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/Directory.java?rev=1382800&r1=1382799&r2=1382800&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/Directory.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/Directory.java Mon Sep 10 12:39:48 2012
@@ -260,7 +260,10 @@ public abstract class Directory implemen
     /**
      * Returns an {@link IndexInput} slice starting at offset <i>0</i> with a
      * length equal to the length of the underlying file
+     * @deprecated
      */
+    @Deprecated
+    // can we remove this somehow?
     public abstract IndexInput openFullSlice() throws IOException;
   }
   

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java?rev=1382800&r1=1382799&r2=1382800&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java Mon Sep 10 12:39:48 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,70 @@ 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();
+      public IndexInput openSlice(String sliceDescription, long offset, long length) throws IOException {
+        ensureOpen();
+        return full.slice(sliceDescription, offset, length);
       }
-
+      
       @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);
+      public IndexInput openFullSlice() throws IOException {
+        ensureOpen();
+        return full.clone();
       }
 
       @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/branches/branch_4x/lucene/core/src/test/org/apache/lucene/store/TestMultiMMap.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/store/TestMultiMMap.java?rev=1382800&r1=1382799&r2=1382800&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/store/TestMultiMMap.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/store/TestMultiMMap.java Mon Sep 10 12:39:48 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;
@@ -70,16 +71,113 @@ public class TestMultiMMap extends Lucen
     }
     try {
       three.readVInt();
-      fail("Must throw AlreadyClosedExveption");
+      fail("Must throw AlreadyClosedException");
+    } catch (AlreadyClosedException ignore) {
+      // pass
+    }
+    two.close();
+    three.close();
+    // test double close of master:
+    one.close();
+    mmapDir.close();
+  }
+  
+  public void testCloneClose() throws Exception {
+    MMapDirectory mmapDir = new MMapDirectory(_TestUtil.getTempDir("testCloneClose"));
+    IndexOutput io = mmapDir.createOutput("bytes", newIOContext(random()));
+    io.writeVInt(5);
+    io.close();
+    IndexInput one = mmapDir.openInput("bytes", IOContext.DEFAULT);
+    IndexInput two = one.clone();
+    IndexInput three = two.clone(); // clone of clone
+    two.close();
+    assertEquals(5, one.readVInt());
+    try {
+      two.readVInt();
+      fail("Must throw AlreadyClosedException");
+    } catch (AlreadyClosedException ignore) {
+      // pass
+    }
+    assertEquals(5, three.readVInt());
+    one.close();
+    three.close();
+    mmapDir.close();
+  }
+  
+  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 AlreadyClosedException");
+    } catch (AlreadyClosedException ignore) {
+      // pass
+    }
+    try {
+      four.readInt();
+      fail("Must throw AlreadyClosedException");
+    } catch (AlreadyClosedException ignore) {
+      // pass
+    }
+    one.close();
+    two.close();
+    three.close();
+    four.close();
+    // test double-close of slicer:
+    slicer.close();
+    mmapDir.close();
+  }
+
+  public void testCloneSliceClose() throws Exception {
+    MMapDirectory mmapDir = new MMapDirectory(_TestUtil.getTempDir("testCloneSliceClose"));
+    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);
+    one.close();
+    try {
+      one.readInt();
+      fail("Must throw AlreadyClosedException");
     } catch (AlreadyClosedException ignore) {
       // pass
     }
+    assertEquals(2, two.readInt());
+    // reopen a new slice "one":
+    one = slicer.openSlice("first int", 0, 4);
+    assertEquals(1, one.readInt());
+    one.close();
+    two.close();
+    slicer.close();
+    mmapDir.close();
   }
 
   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 +187,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 +219,29 @@ 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();
+      slicer.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 +264,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 +304,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/branches/branch_4x/solr/core/src/java/org/apache/solr/core/MMapDirectoryFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/MMapDirectoryFactory.java?rev=1382800&r1=1382799&r2=1382800&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/MMapDirectoryFactory.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/MMapDirectoryFactory.java Mon Sep 10 12:39:48 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;
   }
 }