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 2013/02/02 17:43:08 UTC

svn commit: r1441771 - in /lucene/dev/branches/lucene4547: ./ lucene/ lucene/core/ lucene/core/src/java/org/apache/lucene/codecs/compressing/ lucene/core/src/java/org/apache/lucene/codecs/lucene41/ lucene/core/src/java/org/apache/lucene/codecs/lucene42...

Author: rmuir
Date: Sat Feb  2 16:43:08 2013
New Revision: 1441771

URL: http://svn.apache.org/viewvc?rev=1441771&view=rev
Log:
Merged /lucene/dev/trunk:r1441568-1441570,1441572-1441769

Added:
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42TermVectorsFormat.java
      - copied unchanged from r1441769, lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42TermVectorsFormat.java
Modified:
    lucene/dev/branches/lucene4547/   (props changed)
    lucene/dev/branches/lucene4547/lucene/   (props changed)
    lucene/dev/branches/lucene4547/lucene/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/lucene4547/lucene/core/   (props changed)
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsIndexReader.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsIndexWriter.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsFormat.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsFormat.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42Codec.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/store/ByteBufferIndexInput.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java
    lucene/dev/branches/lucene4547/lucene/test-framework/   (props changed)
    lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/index/BaseTermVectorsFormatTestCase.java

Modified: lucene/dev/branches/lucene4547/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/CHANGES.txt?rev=1441771&r1=1441770&r2=1441771&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/lucene4547/lucene/CHANGES.txt Sat Feb  2 16:43:08 2013
@@ -66,6 +66,13 @@ Optimizations
   facets. Also added OrdinalPolicy.ALL_BUT_DIMENSION. 
   (Shai Erera, Michael McCandless)
 
+* LUCENE-4740: Don't track clones of MMapIndexInput if unmapping
+  is disabled. This reduces GC overhead. (Kristofer Karlsson, Uwe Schindler)
+
+* LUCENE-4733: The default Lucene 4.2 codec now uses a more compact
+  TermVectorsFormat (Lucene42TermVectorsFormat) based on
+  CompressingTermVectorsFormat. (Adrien Grand)
+
 New Features
 
 * LUCENE-4686: New specialized DGapVInt8IntEncoder for facets (now the 

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsIndexReader.java?rev=1441771&r1=1441770&r2=1441771&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsIndexReader.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsIndexReader.java Sat Feb  2 16:43:08 2013
@@ -28,7 +28,11 @@ import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.packed.PackedInts;
 
-class CompressingStoredFieldsIndexReader implements Closeable, Cloneable {
+/**
+ * Random-access reader for {@link CompressingStoredFieldsIndexWriter}.
+ * @lucene.internal
+ */
+public final class CompressingStoredFieldsIndexReader implements Closeable, Cloneable {
 
   final IndexInput fieldsIndexIn;
 

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsIndexWriter.java?rev=1441771&r1=1441770&r2=1441771&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsIndexWriter.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsIndexWriter.java Sat Feb  2 16:43:08 2013
@@ -20,10 +20,54 @@ package org.apache.lucene.codecs.compres
 import java.io.Closeable;
 import java.io.IOException;
 
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.packed.PackedInts;
 
-class CompressingStoredFieldsIndexWriter implements Closeable {
+/**
+ * Efficient index format for block-based {@link Codec}s.
+ * <p> This writer generates a file which can be loaded into memory using
+ * memory-efficient data structures to quickly locate the block that contains
+ * any document.
+ * <p>In order to have a compact in-memory representation, for every block of
+ * 1024 chunks, this index computes the average number of bytes per
+ * chunk and for every chunk, only stores the difference between<ul>
+ * <li>${chunk number} * ${average length of a chunk}</li>
+ * <li>and the actual start offset of the chunk</li></ul></p>
+ * <p>Data is written as follows:</p>
+ * <ul>
+ * <li>PackedIntsVersion, &lt;Block&gt;<sup>BlockCount</sup>, BlocksEndMarker</li>
+ * <li>PackedIntsVersion --&gt; {@link PackedInts#VERSION_CURRENT} as a {@link DataOutput#writeVInt VInt}</li>
+ * <li>BlocksEndMarker --&gt; <tt>0</tt> as a {@link DataOutput#writeVInt VInt}, this marks the end of blocks since blocks are not allowed to start with <tt>0</tt></li>
+ * <li>Block --&gt; BlockChunks, &lt;DocBases&gt;, &lt;StartPointers&gt;</li>
+ * <li>BlockChunks --&gt; a {@link DataOutput#writeVInt VInt} which is the number of chunks encoded in the block</li>
+ * <li>DocBases --&gt; DocBase, AvgChunkDocs, BitsPerDocBaseDelta, DocBaseDeltas</li>
+ * <li>DocBase --&gt; first document ID of the block of chunks, as a {@link DataOutput#writeVInt VInt}</li>
+ * <li>AvgChunkDocs --&gt; average number of documents in a single chunk, as a {@link DataOutput#writeVInt VInt}</li>
+ * <li>BitsPerDocBaseDelta --&gt; number of bits required to represent a delta from the average using <a href="https://developers.google.com/protocol-buffers/docs/encoding#types">ZigZag encoding</a></li>
+ * <li>DocBaseDeltas --&gt; {@link PackedInts packed} array of BlockChunks elements of BitsPerDocBaseDelta bits each, representing the deltas from the average doc base using <a href="https://developers.google.com/protocol-buffers/docs/encoding#types">ZigZag encoding</a>.</li>
+ * <li>StartPointers --&gt; StartPointerBase, AvgChunkSize, BitsPerStartPointerDelta, StartPointerDeltas</li>
+ * <li>StartPointerBase --&gt; the first start pointer of the block, as a {@link DataOutput#writeVLong VLong}</li>
+ * <li>AvgChunkSize --&gt; the average size of a chunk of compressed documents, as a {@link DataOutput#writeVLong VLong}</li>
+ * <li>BitsPerStartPointerDelta --&gt; number of bits required to represent a delta from the average using <a href="https://developers.google.com/protocol-buffers/docs/encoding#types">ZigZag encoding</a></li>
+ * <li>StartPointerDeltas --&gt; {@link PackedInts packed} array of BlockChunks elements of BitsPerStartPointerDelta bits each, representing the deltas from the average start pointer using <a href="https://developers.google.com/protocol-buffers/docs/encoding#types">ZigZag encoding</a></li>
+ * </ul>
+ * <p>Notes</p>
+ * <ul>
+ * <li>For any block, the doc base of the n-th chunk can be restored with
+ * <code>DocBase + AvgChunkDocs * n + DocBaseDeltas[n]</code>.</li>
+ * <li>For any block, the start pointer of the n-th chunk can be restored with
+ * <code>StartPointerBase + AvgChunkSize * n + StartPointerDeltas[n]</code>.</li>
+ * <li>Once data is loaded into memory, you can lookup the start pointer of any
+ * document by performing two binary searches: a first one based on the values
+ * of DocBase in order to find the right block, and then inside the block based
+ * on DocBaseDeltas (by reconstructing the doc bases for every chunk).</li>
+ * </ul>
+ * @lucene.internal
+ */
+public final class CompressingStoredFieldsIndexWriter implements Closeable {
   
   static final int BLOCK_SIZE = 1024; // number of chunks to serialize at once
 

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsFormat.java?rev=1441771&r1=1441770&r2=1441771&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsFormat.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsFormat.java Sat Feb  2 16:43:08 2013
@@ -34,7 +34,7 @@ import org.apache.lucene.store.IOContext
  * order to improve the compression ratio.
  * @lucene.experimental
  */
-public final class CompressingTermVectorsFormat extends TermVectorsFormat {
+public class CompressingTermVectorsFormat extends TermVectorsFormat {
 
   private final String formatName;
   private final String segmentSuffix;
@@ -79,7 +79,7 @@ public final class CompressingTermVector
   }
 
   @Override
-  public TermVectorsReader vectorsReader(Directory directory,
+  public final TermVectorsReader vectorsReader(Directory directory,
       SegmentInfo segmentInfo, FieldInfos fieldInfos, IOContext context)
       throws IOException {
     return new CompressingTermVectorsReader(directory, segmentInfo, segmentSuffix,
@@ -87,7 +87,7 @@ public final class CompressingTermVector
   }
 
   @Override
-  public TermVectorsWriter vectorsWriter(Directory directory,
+  public final TermVectorsWriter vectorsWriter(Directory directory,
       SegmentInfo segmentInfo, IOContext context) throws IOException {
     return new CompressingTermVectorsWriter(directory, segmentInfo, segmentSuffix,
         context, formatName, compressionMode, chunkSize);

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsFormat.java?rev=1441771&r1=1441770&r2=1441771&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsFormat.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsFormat.java Sat Feb  2 16:43:08 2013
@@ -20,6 +20,7 @@ package org.apache.lucene.codecs.lucene4
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.StoredFieldsFormat;
 import org.apache.lucene.codecs.compressing.CompressingStoredFieldsFormat;
+import org.apache.lucene.codecs.compressing.CompressingStoredFieldsIndexWriter;
 import org.apache.lucene.codecs.compressing.CompressionMode;
 import org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat;
 import org.apache.lucene.store.DataOutput;
@@ -50,7 +51,7 @@ import org.apache.lucene.util.packed.Pac
  * <a href="http://fastcompression.blogspot.fr/2011/05/lz4-explained.html">compression format</a>.</p>
  * <p>Here is a more detailed description of the field data file format:</p>
  * <ul>
- * <li>FieldData (.fdt) --&gt; &lt;Header&gt;, PackedIntsVersion, CompressionFormat, &lt;Chunk&gt;<sup>ChunkCount</sup></li>
+ * <li>FieldData (.fdt) --&gt; &lt;Header&gt;, PackedIntsVersion, &lt;Chunk&gt;<sup>ChunkCount</sup></li>
  * <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
  * <li>PackedIntsVersion --&gt; {@link PackedInts#VERSION_CURRENT} as a {@link DataOutput#writeVInt VInt}</li>
  * <li>ChunkCount is not known in advance and is the number of chunks necessary to store all document of the segment</li>
@@ -95,43 +96,11 @@ import org.apache.lucene.util.packed.Pac
  * </ul>
  * </li>
  * <li><a name="field_index" id="field_index"></a>
- * <p>A fields index file (extension <tt>.fdx</tt>). The data stored in this
- * file is read to load an in-memory data-structure that can be used to locate
- * the start offset of a block containing any document in the fields data file.</p>
- * <p>In order to have a compact in-memory representation, for every block of
- * 1024 chunks, this stored fields index computes the average number of bytes per
- * chunk and for every chunk, only stores the difference between<ul>
- * <li>${chunk number} * ${average length of a chunk}</li>
- * <li>and the actual start offset of the chunk</li></ul></p>
- * <p>Data is written as follows:</p>
+ * <p>A fields index file (extension <tt>.fdx</tt>).</p>
  * <ul>
- * <li>FieldsIndex (.fdx) --&gt; &lt;Header&gt;, FieldsIndex, PackedIntsVersion, &lt;Block&gt;<sup>BlockCount</sup>, BlocksEndMarker</li>
+ * <li>FieldsIndex (.fdx) --&gt; &lt;Header&gt;, &lt;ChunkIndex&gt;</li>
  * <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
- * <li>PackedIntsVersion --&gt; {@link PackedInts#VERSION_CURRENT} as a {@link DataOutput#writeVInt VInt}</li>
- * <li>BlocksEndMarker --&gt; <tt>0</tt> as a {@link DataOutput#writeVInt VInt}, this marks the end of blocks since blocks are not allowed to start with <tt>0</tt></li>
- * <li>Block --&gt; BlockChunks, &lt;DocBases&gt;, &lt;StartPointers&gt;</li>
- * <li>BlockChunks --&gt; a {@link DataOutput#writeVInt VInt} which is the number of chunks encoded in the block</li>
- * <li>DocBases --&gt; DocBase, AvgChunkDocs, BitsPerDocBaseDelta, DocBaseDeltas</li>
- * <li>DocBase --&gt; first document ID of the block of chunks, as a {@link DataOutput#writeVInt VInt}</li>
- * <li>AvgChunkDocs --&gt; average number of documents in a single chunk, as a {@link DataOutput#writeVInt VInt}</li>
- * <li>BitsPerDocBaseDelta --&gt; number of bits required to represent a delta from the average using <a href="https://developers.google.com/protocol-buffers/docs/encoding#types">ZigZag encoding</a></li>
- * <li>DocBaseDeltas --&gt; {@link PackedInts packed} array of BlockChunks elements of BitsPerDocBaseDelta bits each, representing the deltas from the average doc base using <a href="https://developers.google.com/protocol-buffers/docs/encoding#types">ZigZag encoding</a>.</li>
- * <li>StartPointers --&gt; StartPointerBase, AvgChunkSize, BitsPerStartPointerDelta, StartPointerDeltas</li>
- * <li>StartPointerBase --&gt; the first start pointer of the block, as a {@link DataOutput#writeVLong VLong}</li>
- * <li>AvgChunkSize --&gt; the average size of a chunk of compressed documents, as a {@link DataOutput#writeVLong VLong}</li>
- * <li>BitsPerStartPointerDelta --&gt; number of bits required to represent a delta from the average using <a href="https://developers.google.com/protocol-buffers/docs/encoding#types">ZigZag encoding</a></li>
- * <li>StartPointerDeltas --&gt; {@link PackedInts packed} array of BlockChunks elements of BitsPerStartPointerDelta bits each, representing the deltas from the average start pointer using <a href="https://developers.google.com/protocol-buffers/docs/encoding#types">ZigZag encoding</a></li>
- * </ul>
- * <p>Notes</p>
- * <ul>
- * <li>For any block, the doc base of the n-th chunk can be restored with
- * <code>DocBase + AvgChunkDocs * n + DocBaseDeltas[n]</code>.</li>
- * <li>For any block, the start pointer of the n-th chunk can be restored with
- * <code>StartPointerBase + AvgChunkSize * n + StartPointerDeltas[n]</code>.</li>
- * <li>Once data is loaded into memory, you can lookup the start pointer of any
- * document by performing two binary searches: a first one based on the values
- * of DocBase in order to find the right block, and then inside the block based
- * on DocBaseDeltas (by reconstructing the doc bases for every chunk).</li>
+ * <li>ChunkIndex: See {@link CompressingStoredFieldsIndexWriter}</li>
  * </ul>
  * </li>
  * </ol>

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42Codec.java?rev=1441771&r1=1441770&r2=1441771&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42Codec.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42Codec.java Sat Feb  2 16:43:08 2013
@@ -18,18 +18,17 @@ package org.apache.lucene.codecs.lucene4
  */
 
 import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.FieldInfosFormat;
 import org.apache.lucene.codecs.FilterCodec;
 import org.apache.lucene.codecs.LiveDocsFormat;
+import org.apache.lucene.codecs.NormsFormat;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.SegmentInfoFormat;
-import org.apache.lucene.codecs.DocValuesFormat;
-import org.apache.lucene.codecs.NormsFormat;
 import org.apache.lucene.codecs.StoredFieldsFormat;
 import org.apache.lucene.codecs.TermVectorsFormat;
 import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
 import org.apache.lucene.codecs.lucene40.Lucene40SegmentInfoFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat;
 import org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat;
 import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
 import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
@@ -49,7 +48,7 @@ import org.apache.lucene.codecs.perfield
 // (it writes a minor version, etc).
 public class Lucene42Codec extends Codec {
   private final StoredFieldsFormat fieldsFormat = new Lucene41StoredFieldsFormat();
-  private final TermVectorsFormat vectorsFormat = new Lucene40TermVectorsFormat();
+  private final TermVectorsFormat vectorsFormat = new Lucene42TermVectorsFormat();
   private final FieldInfosFormat fieldInfosFormat = new Lucene42FieldInfosFormat();
   private final SegmentInfoFormat infosFormat = new Lucene40SegmentInfoFormat();
   private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat();

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/store/ByteBufferIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/store/ByteBufferIndexInput.java?rev=1441771&r1=1441770&r2=1441771&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/store/ByteBufferIndexInput.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/store/ByteBufferIndexInput.java Sat Feb  2 16:43:08 2013
@@ -51,14 +51,15 @@ abstract class ByteBufferIndexInput exte
   private ByteBuffer curBuf; // redundant for speed: buffers[curBufIndex]
 
   private boolean isClone = false;
-  private final WeakIdentityMap<ByteBufferIndexInput,Boolean> clones = WeakIdentityMap.newConcurrentHashMap();
+  private final WeakIdentityMap<ByteBufferIndexInput,Boolean> clones;
   
-  ByteBufferIndexInput(String resourceDescription, ByteBuffer[] buffers, long length, int chunkSizePower) throws IOException {
+  ByteBufferIndexInput(String resourceDescription, ByteBuffer[] buffers, long length, int chunkSizePower, boolean trackClones) throws IOException {
     super(resourceDescription);
     this.buffers = buffers;
     this.length = length;
     this.chunkSizePower = chunkSizePower;
     this.chunkSizeMask = (1L << chunkSizePower) - 1L;
+    this.clones = trackClones ? WeakIdentityMap.<ByteBufferIndexInput,Boolean>newConcurrentHashMap() : null;
     
     assert chunkSizePower >= 0 && chunkSizePower <= 30;   
     assert (length >>> chunkSizePower) < Integer.MAX_VALUE;
@@ -231,7 +232,9 @@ abstract class ByteBufferIndexInput exte
     clone.length = length;
 
     // register the new clone in our clone list to clean it up on closing:
-    this.clones.put(clone, Boolean.TRUE);
+    if (clones != null) {
+      this.clones.put(clone, Boolean.TRUE);
+    }
     
     return clone;
   }
@@ -272,17 +275,21 @@ abstract class ByteBufferIndexInput exte
       // make local copy, then un-set early
       final ByteBuffer[] bufs = buffers;
       unsetBuffers();
-      clones.remove(this);
+      if (clones != null) {
+        clones.remove(this);
+      }
       
       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();
+      if (clones != null) {
+        for (Iterator<ByteBufferIndexInput> it = this.clones.keyIterator(); it.hasNext();) {
+          final ByteBufferIndexInput clone = it.next();
+          assert clone.isClone;
+          clone.unsetBuffers();
+        }
+        this.clones.clear();
       }
-      this.clones.clear();
       
       for (final ByteBuffer b : bufs) {
         freeBuffer(b);

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java?rev=1441771&r1=1441770&r2=1441771&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java Sat Feb  2 16:43:08 2013
@@ -178,36 +178,6 @@ public class MMapDirectory extends FSDir
   }
   
   /**
-   * Try to unmap the buffer, this method silently fails if no support
-   * for that in the JVM. On Windows, this leads to the fact,
-   * that mmapped files cannot be modified or deleted.
-   */
-  final void cleanMapping(final ByteBuffer buffer) throws IOException {
-    if (useUnmapHack) {
-      try {
-        AccessController.doPrivileged(new PrivilegedExceptionAction<Object>() {
-          @Override
-          public Object run() throws Exception {
-            final Method getCleanerMethod = buffer.getClass()
-              .getMethod("cleaner");
-            getCleanerMethod.setAccessible(true);
-            final Object cleaner = getCleanerMethod.invoke(buffer);
-            if (cleaner != null) {
-              cleaner.getClass().getMethod("clean")
-                .invoke(cleaner);
-            }
-            return null;
-          }
-        });
-      } catch (PrivilegedActionException e) {
-        final IOException ioe = new IOException("unable to unmap the mapped buffer");
-        ioe.initCause(e.getCause());
-        throw ioe;
-      }
-    }
-  }
-  
-  /**
    * Returns the current mmap chunk size.
    * @see #MMapDirectory(File, LockFactory, int)
    */
@@ -246,14 +216,42 @@ public class MMapDirectory extends FSDir
   }
 
   private final class MMapIndexInput extends ByteBufferIndexInput {
+    private final boolean useUnmapHack;
     
     MMapIndexInput(String resourceDescription, RandomAccessFile raf) throws IOException {
-      super(resourceDescription, map(raf, 0, raf.length()), raf.length(), chunkSizePower);
+      super(resourceDescription, map(raf, 0, raf.length()), raf.length(), chunkSizePower, getUseUnmap());
+      this.useUnmapHack = getUseUnmap();
     }
     
+    /**
+     * Try to unmap the buffer, this method silently fails if no support
+     * for that in the JVM. On Windows, this leads to the fact,
+     * that mmapped files cannot be modified or deleted.
+     */
     @Override
-    protected void freeBuffer(ByteBuffer buffer) throws IOException {
-      cleanMapping(buffer);
+    protected void freeBuffer(final ByteBuffer buffer) throws IOException {
+      if (useUnmapHack) {
+        try {
+          AccessController.doPrivileged(new PrivilegedExceptionAction<Void>() {
+            @Override
+            public Void run() throws Exception {
+              final Method getCleanerMethod = buffer.getClass()
+                .getMethod("cleaner");
+              getCleanerMethod.setAccessible(true);
+              final Object cleaner = getCleanerMethod.invoke(buffer);
+              if (cleaner != null) {
+                cleaner.getClass().getMethod("clean")
+                  .invoke(cleaner);
+              }
+              return null;
+            }
+          });
+        } catch (PrivilegedActionException e) {
+          final IOException ioe = new IOException("unable to unmap the mapped buffer");
+          ioe.initCause(e.getCause());
+          throw ioe;
+        }
+      }
     }
   }
   

Modified: lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/index/BaseTermVectorsFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/index/BaseTermVectorsFormatTestCase.java?rev=1441771&r1=1441770&r2=1441771&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/index/BaseTermVectorsFormatTestCase.java (original)
+++ lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/index/BaseTermVectorsFormatTestCase.java Sat Feb  2 16:43:08 2013
@@ -25,6 +25,7 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
@@ -388,9 +389,9 @@ public abstract class BaseTermVectorsFor
   }
 
   // to test reuse
-  private TermsEnum termsEnum = null;
-  private DocsEnum docsEnum = null;
-  private DocsAndPositionsEnum docsAndPositionsEnum = null;
+  private final ThreadLocal<TermsEnum> termsEnum = new ThreadLocal<TermsEnum>();
+  private final ThreadLocal<DocsEnum> docsEnum = new ThreadLocal<DocsEnum>();
+  private final ThreadLocal<DocsAndPositionsEnum> docsAndPositionsEnum = new ThreadLocal<DocsAndPositionsEnum>();
 
   protected void assertEquals(RandomTokenStream tk, FieldType ft, Terms terms) throws IOException {
     assertEquals(1, terms.getDocCount());
@@ -406,7 +407,8 @@ public abstract class BaseTermVectorsFor
     }
     final BytesRef[] sortedTerms = uniqueTerms.toArray(new BytesRef[0]);
     Arrays.sort(sortedTerms, terms.getComparator());
-    termsEnum = terms.iterator(random().nextBoolean() ? null : termsEnum);
+    final TermsEnum termsEnum = terms.iterator(random().nextBoolean() ? null : this.termsEnum.get());
+    this.termsEnum.set(termsEnum);
     for (int i = 0; i < sortedTerms.length; ++i) {
       final BytesRef nextTerm = termsEnum.next();
       assertEquals(sortedTerms[i], nextTerm);
@@ -414,7 +416,7 @@ public abstract class BaseTermVectorsFor
       assertEquals(1, termsEnum.docFreq());
 
       final FixedBitSet bits = new FixedBitSet(1);
-      docsEnum = termsEnum.docs(bits, random().nextBoolean() ? null : docsEnum);
+      DocsEnum docsEnum = termsEnum.docs(bits, random().nextBoolean() ? null : this.docsEnum.get());
       assertEquals(DocsEnum.NO_MORE_DOCS, docsEnum.nextDoc());
       bits.set(0);
 
@@ -424,9 +426,10 @@ public abstract class BaseTermVectorsFor
       assertEquals(0, docsEnum.docID());
       assertEquals(tk.freqs.get(termsEnum.term().utf8ToString()), (Integer) docsEnum.freq());
       assertEquals(DocsEnum.NO_MORE_DOCS, docsEnum.nextDoc());
+      this.docsEnum.set(docsEnum);
 
       bits.clear(0);
-      docsAndPositionsEnum = termsEnum.docsAndPositions(bits, random().nextBoolean() ? null : docsAndPositionsEnum);
+      DocsAndPositionsEnum docsAndPositionsEnum = termsEnum.docsAndPositions(bits, random().nextBoolean() ? null : this.docsAndPositionsEnum.get());
       assertEquals(ft.storeTermVectorOffsets() || ft.storeTermVectorPositions(), docsAndPositionsEnum != null);
       if (docsAndPositionsEnum != null) {
         assertEquals(DocsEnum.NO_MORE_DOCS, docsAndPositionsEnum.nextDoc());
@@ -492,6 +495,7 @@ public abstract class BaseTermVectorsFor
         }
         assertEquals(DocsEnum.NO_MORE_DOCS, docsAndPositionsEnum.nextDoc());
       }
+      this.docsAndPositionsEnum.set(docsAndPositionsEnum);
     }
     assertNull(termsEnum.next());
     for (int i = 0; i < 5; ++i) {
@@ -630,4 +634,97 @@ public abstract class BaseTermVectorsFor
     dir.close();
   }
 
+  public void testMerge() throws IOException {
+    final RandomDocumentFactory docFactory = new RandomDocumentFactory(5, 20);
+    final int numDocs = _TestUtil.nextInt(random(), 100, 500);
+    final int numDeletes = random().nextInt(numDocs);
+    final Set<Integer> deletes = new HashSet<Integer>();
+    while (deletes.size() < numDeletes) {
+      deletes.add(random().nextInt(numDocs));
+    }
+    for (Options options : validOptions()) {
+      final RandomDocument[] docs = new RandomDocument[numDocs];
+      for (int i = 0; i < numDocs; ++i) {
+        docs[i] = docFactory.newDocument(_TestUtil.nextInt(random(), 1, 3), _TestUtil.nextInt(random(), 10, 50), options);
+      }
+      final Directory dir = newDirectory();
+      final RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+      for (int i = 0; i < numDocs; ++i) {
+        writer.addDocument(addId(docs[i].toDocument(), ""+i));
+        if (rarely()) {
+          writer.commit();
+        }
+      }
+      for (int delete : deletes) {
+        writer.deleteDocuments(new Term("id", "" + delete));
+      }
+      // merge with deletes
+      writer.forceMerge(1);
+      final IndexReader reader = writer.getReader();
+      for (int i = 0; i < numDocs; ++i) {
+        if (!deletes.contains(i)) {
+          final int docID = docID(reader, ""+i);
+          assertEquals(docs[i], reader.getTermVectors(docID));
+        }
+      }
+      reader.close();
+      writer.close();
+      dir.close();
+    }
+  }
+
+  // run random tests from different threads to make sure the per-thread clones
+  // don't share mutable data
+  public void testClone() throws IOException, InterruptedException {
+    final RandomDocumentFactory docFactory = new RandomDocumentFactory(5, 20);
+    final int numDocs = _TestUtil.nextInt(random(), 100, 1000);
+    for (Options options : validOptions()) {
+      final RandomDocument[] docs = new RandomDocument[numDocs];
+      for (int i = 0; i < numDocs; ++i) {
+        docs[i] = docFactory.newDocument(_TestUtil.nextInt(random(), 1, 3), _TestUtil.nextInt(random(), 10, 50), options);
+      }
+      final Directory dir = newDirectory();
+      final RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+      for (int i = 0; i < numDocs; ++i) {
+        writer.addDocument(addId(docs[i].toDocument(), ""+i));
+      }
+      final IndexReader reader = writer.getReader();
+      for (int i = 0; i < numDocs; ++i) {
+        final int docID = docID(reader, ""+i);
+        assertEquals(docs[i], reader.getTermVectors(docID));
+      }
+
+      final AtomicReference<Throwable> exception = new AtomicReference<Throwable>();
+      final Thread[] threads = new Thread[2];
+      for (int i = 0; i < threads.length; ++i) {
+        threads[i] = new Thread() {
+          @Override
+          public void run() {
+            try {
+              for (int i = 0; i < atLeast(100); ++i) {
+                final int idx = random().nextInt(numDocs);
+                final int docID = docID(reader, ""+idx);
+                assertEquals(docs[idx], reader.getTermVectors(docID));
+              }
+            } catch (Throwable t) {
+              exception.set(t);
+            }
+          }
+        };
+      }
+      for (Thread thread : threads) {
+        thread.start();
+      }
+      for (Thread thread : threads) {
+        thread.join();
+      }
+      reader.close();
+      writer.close();
+      dir.close();
+      if (exception.get() != null) {
+        throw new RuntimeException("One thread threw an exception", exception.get());
+      }
+    }
+  }
+
 }