You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by us...@apache.org on 2012/08/25 12:06:16 UTC

svn commit: r1377246 [2/7] - in /lucene/dev/branches/lucene3312: ./ dev-tools/ dev-tools/idea/lucene/analysis/morfologik/ dev-tools/idea/lucene/analysis/phonetic/ dev-tools/maven/ dev-tools/scripts/ lucene/ lucene/analysis/ lucene/analysis/common/ luce...

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/MultiLevelSkipListWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/MultiLevelSkipListWriter.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/MultiLevelSkipListWriter.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/MultiLevelSkipListWriter.java Sat Aug 25 10:06:07 2012
@@ -26,6 +26,8 @@ import org.apache.lucene.util.MathUtil;
 /**
  * This abstract class writes skip lists with multiple levels.
  * 
+ * <pre>
+ *
  * Example for skipInterval = 3:
  *                                                     c            (skip level 2)
  *                 c                 c                 c            (skip level 1) 
@@ -45,6 +47,7 @@ import org.apache.lucene.util.MathUtil;
  * 
  * While this class takes care of writing the different skip levels,
  * subclasses must define the actual format of the skip data.
+ * </pre>
  * @lucene.experimental
  */
 
@@ -54,15 +57,23 @@ public abstract class MultiLevelSkipList
   
   // the skip interval in the list with level = 0
   private int skipInterval;
+
+  // skipInterval used for level > 0
+  private int skipMultiplier;
   
   // for every skip level a different buffer is used 
   private RAMOutputStream[] skipBuffer;
 
-  protected MultiLevelSkipListWriter(int skipInterval, int maxSkipLevels, int df) {
+  protected MultiLevelSkipListWriter(int skipInterval, int skipMultiplier, int maxSkipLevels, int df) {
     this.skipInterval = skipInterval;
+    this.skipMultiplier = skipMultiplier;
     
     // calculate the maximum number of skip levels for this document frequency
-    numberOfSkipLevels = MathUtil.log(df, skipInterval);
+    if (df <= skipInterval) {
+      numberOfSkipLevels = 1;
+    } else {
+      numberOfSkipLevels = 1+MathUtil.log(df/skipInterval, skipMultiplier);
+    }
     
     // make sure it does not exceed maxSkipLevels
     if (numberOfSkipLevels > maxSkipLevels) {
@@ -70,6 +81,11 @@ public abstract class MultiLevelSkipList
     }
   }
   
+  // skipMultiplier and skipInterval are the same:
+  protected MultiLevelSkipListWriter(int skipInterval, int maxSkipLevels, int df) {
+    this(skipInterval, skipInterval, maxSkipLevels, df);
+  }
+
   protected void init() {
     skipBuffer = new RAMOutputStream[numberOfSkipLevels];
     for (int i = 0; i < numberOfSkipLevels; i++) {
@@ -95,7 +111,7 @@ public abstract class MultiLevelSkipList
    * @param skipBuffer the skip buffer to write to
    */
   protected abstract void writeSkipData(int level, IndexOutput skipBuffer) throws IOException;
-  
+
   /**
    * Writes the current skip data to the buffers. The current document frequency determines
    * the max level is skip data is to be written to. 
@@ -104,11 +120,15 @@ public abstract class MultiLevelSkipList
    * @throws IOException
    */
   public void bufferSkip(int df) throws IOException {
-    int numLevels;
+
+    assert df % skipInterval == 0;
+    int numLevels = 1;
+    df /= skipInterval;
    
     // determine max level
-    for (numLevels = 0; (df % skipInterval) == 0 && numLevels < numberOfSkipLevels; df /= skipInterval) {
+    while ((df % skipMultiplier) == 0 && numLevels < numberOfSkipLevels) {
       numLevels++;
+      df /= skipMultiplier;
     }
     
     long childPointer = 0;
@@ -150,5 +170,4 @@ public abstract class MultiLevelSkipList
     
     return skipPointer;
   }
-
 }

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsReader.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsReader.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsReader.java Sat Aug 25 10:06:07 2012
@@ -22,8 +22,8 @@ import java.io.IOException;
 import org.apache.lucene.index.StoredFieldVisitor;
 
 /**
- * Codec API for reading stored fields:
- * 
+ * Codec API for reading stored fields.
+ * <p>
  * You need to implement {@link #visitDocument(int, StoredFieldVisitor)} to
  * read the stored fields for a document, implement {@link #clone()} (creating
  * clones of any IndexInputs used, etc), and {@link #close()}

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java Sat Aug 25 10:06:07 2012
@@ -20,6 +20,7 @@ package org.apache.lucene.codecs;
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.Comparator;
+import java.util.Iterator;
 
 import org.apache.lucene.index.AtomicReader;
 import org.apache.lucene.index.DocsAndPositionsEnum;
@@ -187,19 +188,21 @@ public abstract class TermVectorsWriter 
   }
   
   /** Safe (but, slowish) default method to write every
-   *  vector field in the document.  This default
-   *  implementation requires that the vectors implement
-   *  both Fields.size and
-   *  Terms.size. */
+   *  vector field in the document. */
   protected final void addAllDocVectors(Fields vectors, MergeState mergeState) throws IOException {
     if (vectors == null) {
       startDocument(0);
       return;
     }
 
-    final int numFields = vectors.size();
+    int numFields = vectors.size();
     if (numFields == -1) {
-      throw new IllegalStateException("vectors.size() must be implemented (it returned -1)");
+      // count manually! TODO: Maybe enforce that Fields.size() returns something valid?
+      numFields = 0;
+      for (final Iterator<String> it = vectors.iterator(); it.hasNext(); ) {
+        it.next();
+        numFields++;
+      }
     }
     startDocument(numFields);
     
@@ -208,7 +211,9 @@ public abstract class TermVectorsWriter 
     TermsEnum termsEnum = null;
     DocsAndPositionsEnum docsAndPositionsEnum = null;
     
+    int fieldCount = 0;
     for(String fieldName : vectors) {
+      fieldCount++;
       final FieldInfo fieldInfo = mergeState.fieldInfos.fieldInfo(fieldName);
 
       assert lastFieldName == null || fieldName.compareTo(lastFieldName) > 0: "lastFieldName=" + lastFieldName + " fieldName=" + fieldName;
@@ -225,9 +230,14 @@ public abstract class TermVectorsWriter 
       final boolean hasPayloads = terms.hasPayloads();
       assert !hasPayloads || hasPositions;
       
-      final int numTerms = (int) terms.size();
+      int numTerms = (int) terms.size();
       if (numTerms == -1) {
-        throw new IllegalStateException("terms.size() must be implemented (it returned -1)");
+        // count manually. It is stupid, but needed, as Terms.size() is not a mandatory statistics function
+        numTerms = 0;
+        termsEnum = terms.iterator(termsEnum);
+        while(termsEnum.next() != null) {
+          numTerms++;
+        }
       }
       
       startField(fieldInfo, numTerms, hasPositions, hasOffsets, hasPayloads);
@@ -263,6 +273,7 @@ public abstract class TermVectorsWriter 
       }
       assert termCount == numTerms;
     }
+    assert fieldCount == numFields;
   }
   
   /** Return the BytesRef Comparator used to sort terms

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/TermsIndexReaderBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/TermsIndexReaderBase.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/TermsIndexReaderBase.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/TermsIndexReaderBase.java Sat Aug 25 10:06:07 2012
@@ -51,7 +51,7 @@ public abstract class TermsIndexReaderBa
   /** 
    * Similar to TermsEnum, except, the only "metadata" it
    * reports for a given indexed term is the long fileOffset
-   * into the main terms dictionary file:
+   * into the main terms dictionary file.
    */
   public static abstract class FieldIndexEnum {
 

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/VariableGapTermsIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/VariableGapTermsIndexReader.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/VariableGapTermsIndexReader.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/VariableGapTermsIndexReader.java Sat Aug 25 10:06:07 2012
@@ -170,7 +170,7 @@ public class VariableGapTermsIndexReader
 
     private void loadTermsIndex() throws IOException {
       if (fst == null) {
-        IndexInput clone = (IndexInput) in.clone();
+        IndexInput clone = in.clone();
         clone.seek(indexStart);
         fst = new FST<Long>(clone, fstOutputs);
         clone.close();

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/bloom/BloomFilteringPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/bloom/BloomFilteringPostingsFormat.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/bloom/BloomFilteringPostingsFormat.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/bloom/BloomFilteringPostingsFormat.java Sat Aug 25 10:06:07 2012
@@ -207,14 +207,10 @@ public class BloomFilteringPostingsForma
       }
     }
     
-    public int size() throws IOException {
+    public int size() {
       return delegateFieldsProducer.size();
     }
     
-    public long getUniqueTermCount() throws IOException {
-      return delegateFieldsProducer.getUniqueTermCount();
-    }
-    
     class BloomFilteredTerms extends Terms {
       private Terms delegateTerms;
       private FuzzySet filter;

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/intblock/FixedIntBlockIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/intblock/FixedIntBlockIndexInput.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/intblock/FixedIntBlockIndexInput.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/intblock/FixedIntBlockIndexInput.java Sat Aug 25 10:06:07 2012
@@ -26,7 +26,6 @@ import java.io.IOException;
 import org.apache.lucene.codecs.sep.IntIndexInput;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.util.IntsRef;
 
 /** Abstract base class that reads fixed-size blocks of ints
  *  from an IndexInput.  While this is a simple approach, a
@@ -49,7 +48,7 @@ public abstract class FixedIntBlockIndex
   @Override
   public Reader reader() throws IOException {
     final int[] buffer = new int[blockSize];
-    final IndexInput clone = (IndexInput) in.clone();
+    final IndexInput clone = in.clone();
     // TODO: can this be simplified?
     return new Reader(clone, buffer, this.getBlockReader(clone, buffer));
   }
@@ -77,76 +76,48 @@ public abstract class FixedIntBlockIndex
 
   private static class Reader extends IntIndexInput.Reader {
     private final IndexInput in;
+    private final BlockReader blockReader;
+    private final int blockSize;
+    private final int[] pending;
 
-    protected final int[] pending;
-    int upto;
-
+    private int upto;
     private boolean seekPending;
     private long pendingFP;
-    private int pendingUpto;
-    private long lastBlockFP;
-    private final BlockReader blockReader;
-    private final int blockSize;
-    private final IntsRef bulkResult = new IntsRef();
+    private long lastBlockFP = -1;
 
     public Reader(final IndexInput in, final int[] pending, final BlockReader blockReader) {
       this.in = in;
       this.pending = pending;
       this.blockSize = pending.length;
-      bulkResult.ints = pending;
       this.blockReader = blockReader;
       upto = blockSize;
     }
 
     void seek(final long fp, final int upto) {
-      pendingFP = fp;
-      pendingUpto = upto;
-      seekPending = true;
-    }
-
-    private void maybeSeek() throws IOException {
-      if (seekPending) {
-        if (pendingFP != lastBlockFP) {
-          // need new block
-          in.seek(pendingFP);
-          lastBlockFP = pendingFP;
-          blockReader.readBlock();
-        }
-        upto = pendingUpto;
-        seekPending = false;
+      assert upto < blockSize;
+      if (seekPending || fp != lastBlockFP) {
+        pendingFP = fp;
+        seekPending = true;
       }
+      this.upto = upto;
     }
 
     @Override
     public int next() throws IOException {
-      this.maybeSeek();
-      if (upto == blockSize) {
+      if (seekPending) {
+        // Seek & load new block
+        in.seek(pendingFP);
+        lastBlockFP = pendingFP;
+        blockReader.readBlock();
+        seekPending = false;
+      } else if (upto == blockSize) {
+        // Load new block
         lastBlockFP = in.getFilePointer();
         blockReader.readBlock();
         upto = 0;
       }
-
       return pending[upto++];
     }
-
-    @Override
-    public IntsRef read(final int count) throws IOException {
-      this.maybeSeek();
-      if (upto == blockSize) {
-        blockReader.readBlock();
-        upto = 0;
-      }
-      bulkResult.offset = upto;
-      if (upto + count < blockSize) {
-        bulkResult.length = count;
-        upto += count;
-      } else {
-        bulkResult.length = blockSize - upto;
-        upto = blockSize;
-      }
-
-      return bulkResult;
-    }
   }
 
   private class Index extends IntIndexInput.Index {
@@ -178,7 +149,7 @@ public abstract class FixedIntBlockIndex
     }
 
     @Override
-    public void set(final IntIndexInput.Index other) {
+    public void copyFrom(final IntIndexInput.Index other) {
       final Index idx = (Index) other;
       fp = idx.fp;
       upto = idx.upto;

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/intblock/VariableIntBlockIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/intblock/VariableIntBlockIndexInput.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/intblock/VariableIntBlockIndexInput.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/intblock/VariableIntBlockIndexInput.java Sat Aug 25 10:06:07 2012
@@ -26,7 +26,6 @@ import java.io.IOException;
 import org.apache.lucene.codecs.sep.IntIndexInput;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.util.IntsRef;
 
 // TODO: much of this can be shared code w/ the fixed case
 
@@ -51,7 +50,7 @@ public abstract class VariableIntBlockIn
   @Override
   public Reader reader() throws IOException {
     final int[] buffer = new int[maxBlockSize];
-    final IndexInput clone = (IndexInput) in.clone();
+    final IndexInput clone = in.clone();
     // TODO: can this be simplified?
     return new Reader(clone, buffer, this.getBlockReader(clone, buffer));
   }
@@ -90,12 +89,10 @@ public abstract class VariableIntBlockIn
     private long lastBlockFP;
     private int blockSize;
     private final BlockReader blockReader;
-    private final IntsRef bulkResult = new IntsRef();
 
     public Reader(final IndexInput in, final int[] pending, final BlockReader blockReader) {
       this.in = in;
       this.pending = pending;
-      bulkResult.ints = pending;
       this.blockReader = blockReader;
     }
 
@@ -146,26 +143,6 @@ public abstract class VariableIntBlockIn
 
       return pending[upto++];
     }
-
-    @Override
-    public IntsRef read(final int count) throws IOException {
-      this.maybeSeek();
-      if (upto == blockSize) {
-        lastBlockFP = in.getFilePointer();
-        blockSize = blockReader.readBlock();
-        upto = 0;
-      }
-      bulkResult.offset = upto;
-      if (upto + count < blockSize) {
-        bulkResult.length = count;
-        upto += count;
-      } else {
-        bulkResult.length = blockSize - upto;
-        upto = blockSize;
-      }
-
-      return bulkResult;
-    }
   }
 
   private class Index extends IntIndexInput.Index {
@@ -204,7 +181,7 @@ public abstract class VariableIntBlockIn
     }
 
     @Override
-    public void set(final IntIndexInput.Index other) {
+    public void copyFrom(final IntIndexInput.Index other) {
       final Index idx = (Index) other;
       fp = idx.fp;
       upto = idx.upto;

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java Sat Aug 25 10:06:07 2012
@@ -159,7 +159,7 @@ import org.apache.lucene.util.fst.FST; /
  * with the frequency of the term in that document (except when frequencies are
  * omitted: {@link IndexOptions#DOCS_ONLY}).</p>
  * <ul>
- *   <li>FreqFile (.frq) --&gt; Header, &lt;TermFreqs, SkipData&gt; <sup>TermCount</sup></li>
+ *   <li>FreqFile (.frq) --&gt; Header, &lt;TermFreqs, SkipData?&gt; <sup>TermCount</sup></li>
  *   <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
  *   <li>TermFreqs --&gt; &lt;TermFreq&gt; <sup>DocFreq</sup></li>
  *   <li>TermFreq --&gt; DocDelta[, Freq?]</li>

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java Sat Aug 25 10:06:07 2012
@@ -325,7 +325,7 @@ public class Lucene40PostingsReader exte
     
     SegmentDocsEnumBase(IndexInput startFreqIn, Bits liveDocs) {
       this.startFreqIn = startFreqIn;
-      this.freqIn = (IndexInput)startFreqIn.clone();
+      this.freqIn = startFreqIn.clone();
       this.liveDocs = liveDocs;
       
     }
@@ -474,7 +474,7 @@ public class Lucene40PostingsReader exte
 
         if (skipper == null) {
           // This is the first time this enum has ever been used for skipping -- do lazy init
-          skipper = new Lucene40SkipListReader((IndexInput) freqIn.clone(), maxSkipLevels, skipInterval);
+          skipper = new Lucene40SkipListReader(freqIn.clone(), maxSkipLevels, skipInterval);
         }
 
         if (!skipped) {
@@ -705,8 +705,8 @@ public class Lucene40PostingsReader exte
 
     public SegmentDocsAndPositionsEnum(IndexInput freqIn, IndexInput proxIn) {
       startFreqIn = freqIn;
-      this.freqIn = (IndexInput) freqIn.clone();
-      this.proxIn = (IndexInput) proxIn.clone();
+      this.freqIn = freqIn.clone();
+      this.proxIn = proxIn.clone();
     }
 
     public SegmentDocsAndPositionsEnum reset(FieldInfo fieldInfo, StandardTermState termState, Bits liveDocs) throws IOException {
@@ -795,7 +795,7 @@ public class Lucene40PostingsReader exte
 
         if (skipper == null) {
           // This is the first time this enum has ever been used for skipping -- do lazy init
-          skipper = new Lucene40SkipListReader((IndexInput) freqIn.clone(), maxSkipLevels, skipInterval);
+          skipper = new Lucene40SkipListReader(freqIn.clone(), maxSkipLevels, skipInterval);
         }
 
         if (!skipped) {
@@ -913,8 +913,8 @@ public class Lucene40PostingsReader exte
 
     public SegmentFullPositionsEnum(IndexInput freqIn, IndexInput proxIn) {
       startFreqIn = freqIn;
-      this.freqIn = (IndexInput) freqIn.clone();
-      this.proxIn = (IndexInput) proxIn.clone();
+      this.freqIn = freqIn.clone();
+      this.proxIn = proxIn.clone();
     }
 
     public SegmentFullPositionsEnum reset(FieldInfo fieldInfo, StandardTermState termState, Bits liveDocs) throws IOException {
@@ -1009,7 +1009,7 @@ public class Lucene40PostingsReader exte
 
         if (skipper == null) {
           // This is the first time this enum has ever been used for skipping -- do lazy init
-          skipper = new Lucene40SkipListReader((IndexInput) freqIn.clone(), maxSkipLevels, skipInterval);
+          skipper = new Lucene40SkipListReader(freqIn.clone(), maxSkipLevels, skipInterval);
         }
 
         if (!skipped) {

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsWriter.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsWriter.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsWriter.java Sat Aug 25 10:06:07 2012
@@ -185,8 +185,6 @@ public final class Lucene40PostingsWrite
   int lastDocID;
   int df;
   
-  /** Adds a new doc in this term.  If this returns null
-   *  then we just skip consuming positions/payloads. */
   @Override
   public void startDoc(int docID, int termDocFreq) throws IOException {
     // if (DEBUG) System.out.println("SPW:   startDoc seg=" + segment + " docID=" + docID + " tf=" + termDocFreq + " freqOut.fp=" + freqOut.getFilePointer());

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40StoredFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40StoredFieldsReader.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40StoredFieldsReader.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40StoredFieldsReader.java Sat Aug 25 10:06:07 2012
@@ -61,7 +61,7 @@ public final class Lucene40StoredFieldsR
   @Override
   public Lucene40StoredFieldsReader clone() {
     ensureOpen();
-    return new Lucene40StoredFieldsReader(fieldInfos, numTotalDocs, size, (IndexInput)fieldsStream.clone(), (IndexInput)indexStream.clone());
+    return new Lucene40StoredFieldsReader(fieldInfos, numTotalDocs, size, fieldsStream.clone(), indexStream.clone());
   }
   
   // Used only by clone

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java Sat Aug 25 10:06:07 2012
@@ -403,7 +403,7 @@ public class Lucene40TermVectorsReader e
     // NOTE: tvf is pre-positioned by caller
     public TVTermsEnum() {
       this.origTVF = Lucene40TermVectorsReader.this.tvf;
-      tvf = (IndexInput) origTVF.clone();
+      tvf = origTVF.clone();
     }
 
     public boolean canReuse(IndexInput tvf) {
@@ -752,9 +752,9 @@ public class Lucene40TermVectorsReader e
     // These are null when a TermVectorsReader was created
     // on a segment that did not have term vectors saved
     if (tvx != null && tvd != null && tvf != null) {
-      cloneTvx = (IndexInput) tvx.clone();
-      cloneTvd = (IndexInput) tvd.clone();
-      cloneTvf = (IndexInput) tvf.clone();
+      cloneTvx = tvx.clone();
+      cloneTvd = tvd.clone();
+      cloneTvf = tvf.clone();
     }
     
     return new Lucene40TermVectorsReader(fieldInfos, cloneTvx, cloneTvd, cloneTvf, size, numTotalDocs);

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Bytes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Bytes.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Bytes.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Bytes.java Sat Aug 25 10:06:07 2012
@@ -350,7 +350,7 @@ public final class Bytes {
      */
     protected final IndexInput cloneData() {
       assert datIn != null;
-      return (IndexInput) datIn.clone();
+      return datIn.clone();
     }
 
     /**
@@ -358,7 +358,7 @@ public final class Bytes {
      */
     protected final IndexInput cloneIndex() {
       assert idxIn != null;
-      return (IndexInput) idxIn.clone();
+      return idxIn.clone();
     }
 
     @Override

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/PackedIntValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/PackedIntValues.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/PackedIntValues.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/PackedIntValues.java Sat Aug 25 10:06:07 2012
@@ -187,7 +187,7 @@ class PackedIntValues {
       final Source source;
       IndexInput input = null;
       try {
-        input = (IndexInput) datIn.clone();
+        input = datIn.clone();
         
         if (values == null) {
           source = new PackedIntsSource(input, false);
@@ -218,7 +218,7 @@ class PackedIntValues {
 
     @Override
     public Source getDirectSource() throws IOException {
-      return values != null ? new FixedStraightBytesImpl.DirectFixedStraightSource((IndexInput) datIn.clone(), 8, Type.FIXED_INTS_64) : new PackedIntsSource((IndexInput) datIn.clone(), true);
+      return values != null ? new FixedStraightBytesImpl.DirectFixedStraightSource(datIn.clone(), 8, Type.FIXED_INTS_64) : new PackedIntsSource(datIn.clone(), true);
     }
   }
 

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarSortedBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarSortedBytesImpl.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarSortedBytesImpl.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarSortedBytesImpl.java Sat Aug 25 10:06:07 2012
@@ -211,7 +211,7 @@ final class VarSortedBytesImpl {
       valueCount = ordToOffsetIndex.size()-1; // the last value here is just a dummy value to get the length of the last value
       // advance this iterator to the end and clone the stream once it points to the docToOrdIndex header
       ordToOffsetIndex.get(valueCount);
-      docToOrdIndex = PackedInts.getDirectReader((IndexInput) idxIn.clone()); // read the ords in to prevent too many random disk seeks
+      docToOrdIndex = PackedInts.getDirectReader(idxIn.clone()); // read the ords in to prevent too many random disk seeks
       basePointer = datIn.getFilePointer();
       this.datIn = datIn;
     }

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java Sat Aug 25 10:06:07 2012
@@ -18,6 +18,7 @@ package org.apache.lucene.codecs.memory;
  */
 
 import java.io.IOException;
+import java.util.Collections;
 import java.util.Comparator;
 import java.util.Iterator;
 import java.util.Map;
@@ -43,7 +44,6 @@ import org.apache.lucene.store.RAMOutput
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.UnmodifiableIterator;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
 import org.apache.lucene.util.automaton.RunAutomaton;
 import org.apache.lucene.util.automaton.Transition;
@@ -131,7 +131,7 @@ public class DirectPostingsFormat extend
 
     @Override
     public Iterator<String> iterator() {
-      return new UnmodifiableIterator<String>(fields.keySet().iterator());
+      return Collections.unmodifiableSet(fields.keySet()).iterator();
     }
 
     @Override
@@ -145,15 +145,6 @@ public class DirectPostingsFormat extend
     }
 
     @Override
-    public long getUniqueTermCount() {
-      long numTerms = 0;      
-      for(DirectField field : fields.values()) {
-        numTerms += field.terms.length;
-      }
-      return numTerms;
-    }
-
-    @Override
     public void close() {
     }
   }

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java Sat Aug 25 10:06:07 2012
@@ -18,6 +18,7 @@ package org.apache.lucene.codecs.memory;
  */
 
 import java.io.IOException;
+import java.util.Collections;
 import java.util.Comparator;
 import java.util.Iterator;
 import java.util.SortedMap;
@@ -48,7 +49,6 @@ import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IntsRef;
-import org.apache.lucene.util.UnmodifiableIterator;
 import org.apache.lucene.util.fst.Builder;
 import org.apache.lucene.util.fst.ByteSequenceOutputs;
 import org.apache.lucene.util.fst.BytesRefFSTEnum;
@@ -863,7 +863,7 @@ public class MemoryPostingsFormat extend
     return new FieldsProducer() {
       @Override
       public Iterator<String> iterator() {
-        return new UnmodifiableIterator<String>(fields.keySet().iterator());
+        return Collections.unmodifiableSet(fields.keySet()).iterator();
       }
 
       @Override

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java Sat Aug 25 10:06:07 2012
@@ -19,6 +19,7 @@ package org.apache.lucene.codecs.perfiel
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
@@ -34,7 +35,6 @@ import org.apache.lucene.index.SegmentRe
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.UnmodifiableIterator;
 
 /**
  * Enables per field format support.
@@ -199,7 +199,7 @@ public abstract class PerFieldPostingsFo
 
     @Override
     public Iterator<String> iterator() {
-      return new UnmodifiableIterator<String>(fields.keySet().iterator());
+      return Collections.unmodifiableSet(fields.keySet()).iterator();
     }
 
     @Override

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/sep/IntIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/sep/IntIndexInput.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/sep/IntIndexInput.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/sep/IntIndexInput.java Sat Aug 25 10:06:07 2012
@@ -21,7 +21,6 @@ import java.io.Closeable;
 import java.io.IOException;
 
 import org.apache.lucene.store.DataInput;
-import org.apache.lucene.util.IntsRef;
 
 /** Defines basic API for writing ints to an IndexOutput.
  *  IntBlockCodec interacts with this API. @see
@@ -44,7 +43,7 @@ public abstract class IntIndexInput impl
     /** Seeks primary stream to the last read offset */
     public abstract void seek(IntIndexInput.Reader stream) throws IOException;
 
-    public abstract void set(Index other);
+    public abstract void copyFrom(Index other);
     
     @Override
     public abstract Index clone();
@@ -55,23 +54,5 @@ public abstract class IntIndexInput impl
 
     /** Reads next single int */
     public abstract int next() throws IOException;
-
-    /** Reads next chunk of ints */
-    private IntsRef bulkResult;
-
-    /** Read up to count ints. */
-    public IntsRef read(int count) throws IOException {
-      if (bulkResult == null) {
-        bulkResult = new IntsRef();
-        bulkResult.ints = new int[count];
-      } else {
-        bulkResult.grow(count);
-      }
-      for(int i=0;i<count;i++) {
-        bulkResult.ints[i] = next();
-      }
-      bulkResult.length = count;
-      return bulkResult;
-    }
   }
 }

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/sep/SepPostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/sep/SepPostingsReader.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/sep/SepPostingsReader.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/sep/SepPostingsReader.java Sat Aug 25 10:06:07 2012
@@ -160,13 +160,13 @@ public class SepPostingsReader extends P
       if (docIndex == null) {
         docIndex = other.docIndex.clone();
       } else {
-        docIndex.set(other.docIndex);
+        docIndex.copyFrom(other.docIndex);
       }
       if (other.freqIndex != null) {
         if (freqIndex == null) {
           freqIndex = other.freqIndex.clone();
         } else {
-          freqIndex.set(other.freqIndex);
+          freqIndex.copyFrom(other.freqIndex);
         }
       } else {
         freqIndex = null;
@@ -175,7 +175,7 @@ public class SepPostingsReader extends P
         if (posIndex == null) {
           posIndex = other.posIndex.clone();
         } else {
-          posIndex.set(other.posIndex);
+          posIndex.copyFrom(other.posIndex);
         }
       } else {
         posIndex = null;
@@ -352,11 +352,11 @@ public class SepPostingsReader extends P
 
       // TODO: can't we only do this if consumer
       // skipped consuming the previous docs?
-      docIndex.set(termState.docIndex);
+      docIndex.copyFrom(termState.docIndex);
       docIndex.seek(docReader);
 
       if (!omitTF) {
-        freqIndex.set(termState.freqIndex);
+        freqIndex.copyFrom(termState.freqIndex);
         freqIndex.seek(freqReader);
       }
 
@@ -418,7 +418,7 @@ public class SepPostingsReader extends P
 
         if (skipper == null) {
           // This DocsEnum has never done any skipping
-          skipper = new SepSkipListReader((IndexInput) skipIn.clone(),
+          skipper = new SepSkipListReader(skipIn.clone(),
                                           freqIn,
                                           docIn,
                                           posIn,
@@ -506,7 +506,7 @@ public class SepPostingsReader extends P
       freqIndex = freqIn.index();
       posReader = posIn.reader();
       posIndex = posIn.index();
-      payloadIn = (IndexInput) SepPostingsReader.this.payloadIn.clone();
+      payloadIn = SepPostingsReader.this.payloadIn.clone();
     }
 
     SepDocsAndPositionsEnum init(FieldInfo fieldInfo, SepTermState termState, Bits liveDocs) throws IOException {
@@ -516,15 +516,15 @@ public class SepPostingsReader extends P
 
       // TODO: can't we only do this if consumer
       // skipped consuming the previous docs?
-      docIndex.set(termState.docIndex);
+      docIndex.copyFrom(termState.docIndex);
       docIndex.seek(docReader);
       //System.out.println("  docIndex=" + docIndex);
 
-      freqIndex.set(termState.freqIndex);
+      freqIndex.copyFrom(termState.freqIndex);
       freqIndex.seek(freqReader);
       //System.out.println("  freqIndex=" + freqIndex);
 
-      posIndex.set(termState.posIndex);
+      posIndex.copyFrom(termState.posIndex);
       //System.out.println("  posIndex=" + posIndex);
       posSeekPending = true;
       payloadPending = false;
@@ -597,7 +597,7 @@ public class SepPostingsReader extends P
         if (skipper == null) {
           //System.out.println("  create skipper");
           // This DocsEnum has never done any skipping
-          skipper = new SepSkipListReader((IndexInput) skipIn.clone(),
+          skipper = new SepSkipListReader(skipIn.clone(),
                                           freqIn,
                                           docIn,
                                           posIn,
@@ -629,7 +629,7 @@ public class SepPostingsReader extends P
           // NOTE: don't seek pos here; do it lazily
           // instead.  Eg a PhraseQuery may skip to many
           // docs before finally asking for positions...
-          posIndex.set(skipper.getPosIndex());
+          posIndex.copyFrom(skipper.getPosIndex());
           posSeekPending = true;
           count = newCount;
           doc = accum = skipper.getDoc();

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/sep/SepSkipListReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/sep/SepSkipListReader.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/sep/SepSkipListReader.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/sep/SepSkipListReader.java Sat Aug 25 10:06:07 2012
@@ -108,12 +108,12 @@ class SepSkipListReader extends MultiLev
     lastPayloadPointer = payloadBasePointer;
 
     for(int i=0;i<maxNumberOfSkipLevels;i++) {
-      docIndex[i].set(docBaseIndex);
+      docIndex[i].copyFrom(docBaseIndex);
       if (freqIndex != null) {
-        freqIndex[i].set(freqBaseIndex);
+        freqIndex[i].copyFrom(freqBaseIndex);
       }
       if (posBaseIndex != null) {
-        posIndex[i].set(posBaseIndex);
+        posIndex[i].copyFrom(posBaseIndex);
       }
     }
     Arrays.fill(payloadPointer, payloadBasePointer);
@@ -145,20 +145,20 @@ class SepSkipListReader extends MultiLev
     lastPayloadPointer = payloadPointer[level];
     lastPayloadLength = payloadLength[level];
     if (freqIndex != null) {
-      lastFreqIndex.set(freqIndex[level]);
+      lastFreqIndex.copyFrom(freqIndex[level]);
     }
-    lastDocIndex.set(docIndex[level]);
+    lastDocIndex.copyFrom(docIndex[level]);
     if (lastPosIndex != null) {
-      lastPosIndex.set(posIndex[level]);
+      lastPosIndex.copyFrom(posIndex[level]);
     }
 
     if (level > 0) {
       if (freqIndex != null) {
-        freqIndex[level-1].set(freqIndex[level]);
+        freqIndex[level-1].copyFrom(freqIndex[level]);
       }
-      docIndex[level-1].set(docIndex[level]);
+      docIndex[level-1].copyFrom(docIndex[level]);
       if (posIndex != null) {
-        posIndex[level-1].set(posIndex[level]);
+        posIndex[level-1].copyFrom(posIndex[level]);
       }
     }
   }

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java Sat Aug 25 10:06:07 2012
@@ -18,6 +18,7 @@ package org.apache.lucene.codecs.simplet
  */
 
 import java.io.IOException;
+import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -43,7 +44,6 @@ import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.OpenBitSet;
 import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.UnicodeUtil;
-import org.apache.lucene.util.UnmodifiableIterator;
 import org.apache.lucene.util.fst.Builder;
 import org.apache.lucene.util.fst.BytesRefFSTEnum;
 import org.apache.lucene.util.fst.FST;
@@ -70,7 +70,7 @@ class SimpleTextFieldsReader extends Fie
     in = state.dir.openInput(SimpleTextPostingsFormat.getPostingsFileName(state.segmentInfo.name, state.segmentSuffix), state.context);
    
     fieldInfos = state.fieldInfos;
-    fields = readFields((IndexInput)in.clone());
+    fields = readFields(in.clone());
   }
   
   private TreeMap<String,Long> readFields(IndexInput in) throws IOException {
@@ -230,7 +230,7 @@ class SimpleTextFieldsReader extends Fie
     
     public SimpleTextDocsEnum() {
       this.inStart = SimpleTextFieldsReader.this.in;
-      this.in = (IndexInput) this.inStart.clone();
+      this.in = this.inStart.clone();
     }
 
     public boolean canReuse(IndexInput in) {
@@ -330,7 +330,7 @@ class SimpleTextFieldsReader extends Fie
 
     public SimpleTextDocsAndPositionsEnum() {
       this.inStart = SimpleTextFieldsReader.this.in;
-      this.in = (IndexInput) inStart.clone();
+      this.in = inStart.clone();
     }
 
     public boolean canReuse(IndexInput in) {
@@ -500,7 +500,7 @@ class SimpleTextFieldsReader extends Fie
       final PairOutputs<Long,PairOutputs.Pair<Long,Long>> outputs = new PairOutputs<Long,PairOutputs.Pair<Long,Long>>(posIntOutputs,
                                                                                                                       outputsInner);
       b = new Builder<PairOutputs.Pair<Long,PairOutputs.Pair<Long,Long>>>(FST.INPUT_TYPE.BYTE1, outputs);
-      IndexInput in = (IndexInput) SimpleTextFieldsReader.this.in.clone();
+      IndexInput in = SimpleTextFieldsReader.this.in.clone();
       in.seek(termsStart);
       final BytesRef lastTerm = new BytesRef(10);
       long lastDocsStart = -1;
@@ -608,7 +608,7 @@ class SimpleTextFieldsReader extends Fie
 
   @Override
   public Iterator<String> iterator() {
-    return new UnmodifiableIterator<String>(fields.keySet().iterator());
+    return Collections.unmodifiableSet(fields.keySet()).iterator();
   }
 
   private final Map<String,Terms> termsCache = new HashMap<String,Terms>();

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPerDocProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPerDocProducer.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPerDocProducer.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPerDocProducer.java Sat Aug 25 10:06:07 2012
@@ -132,7 +132,7 @@ public class SimpleTextPerDocProducer ex
     @Override
     public Source load() throws IOException {
       boolean success = false;
-      IndexInput in = (IndexInput) input.clone();
+      IndexInput in = input.clone();
       try {
         Source source = null;
         switch (type) {

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsReader.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsReader.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsReader.java Sat Aug 25 10:06:07 2012
@@ -163,7 +163,7 @@ public class SimpleTextStoredFieldsReade
     if (in == null) {
       throw new AlreadyClosedException("this FieldsReader is closed");
     }
-    return new SimpleTextStoredFieldsReader(offsets, (IndexInput) in.clone(), fieldInfos);
+    return new SimpleTextStoredFieldsReader(offsets, in.clone(), fieldInfos);
   }
   
   @Override

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java Sat Aug 25 10:06:07 2012
@@ -19,6 +19,7 @@ package org.apache.lucene.codecs.simplet
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.Comparator;
 import java.util.Iterator;
 import java.util.Map;
@@ -44,8 +45,6 @@ import org.apache.lucene.util.CharsRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.UnicodeUtil;
-import org.apache.lucene.util.UnmodifiableIterator;
-
 import static org.apache.lucene.codecs.simpletext.SimpleTextTermVectorsWriter.*;
 
 /**
@@ -205,7 +204,7 @@ public class SimpleTextTermVectorsReader
     if (in == null) {
       throw new AlreadyClosedException("this TermVectorsReader is closed");
     }
-    return new SimpleTextTermVectorsReader(offsets, (IndexInput) in.clone());
+    return new SimpleTextTermVectorsReader(offsets, in.clone());
   }
   
   @Override
@@ -241,7 +240,7 @@ public class SimpleTextTermVectorsReader
 
     @Override
     public Iterator<String> iterator() {
-      return new UnmodifiableIterator<String>(fields.keySet().iterator());
+      return Collections.unmodifiableSet(fields.keySet()).iterator();
     }
 
     @Override
@@ -250,7 +249,7 @@ public class SimpleTextTermVectorsReader
     }
 
     @Override
-    public int size() throws IOException {
+    public int size() {
       return fields.size();
     }
   }

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/DocumentStoredFieldVisitor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/DocumentStoredFieldVisitor.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/DocumentStoredFieldVisitor.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/DocumentStoredFieldVisitor.java Sat Aug 25 10:06:07 2012
@@ -28,7 +28,8 @@ import org.apache.lucene.index.StoredFie
 
 /** A {@link StoredFieldVisitor} that creates a {@link
  *  Document} containing all stored fields, or only specific
- *  requested fields provided to {@link #DocumentStoredFieldVisitor(Set)}
+ *  requested fields provided to {@link #DocumentStoredFieldVisitor(Set)}.
+ *  <p>
  *  This is used by {@link IndexReader#document(int)} to load a
  *  document.
  *

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/Field.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/Field.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/Field.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/document/Field.java Sat Aug 25 10:06:07 2012
@@ -19,7 +19,6 @@ package org.apache.lucene.document;
 
 import java.io.IOException;
 import java.io.Reader;
-import java.io.StringReader;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.NumericTokenStream;
@@ -74,7 +73,8 @@ public class Field implements IndexableF
   // customize how it's tokenized:
   protected TokenStream tokenStream;
 
-  protected transient NumericTokenStream numericTokenStream;
+  private transient TokenStream internalTokenStream;
+  private transient ReusableStringReader internalReader;
 
   protected float boost = 1.0f;
 
@@ -284,9 +284,6 @@ public class Field implements IndexableF
     if (!(fieldsData instanceof Byte)) {
       throw new IllegalArgumentException("cannot change value type from " + fieldsData.getClass().getSimpleName() + " to Byte");
     }
-    if (numericTokenStream != null) {
-      numericTokenStream.setIntValue(value);
-    }
     fieldsData = Byte.valueOf(value);
   }
 
@@ -294,9 +291,6 @@ public class Field implements IndexableF
     if (!(fieldsData instanceof Short)) {
       throw new IllegalArgumentException("cannot change value type from " + fieldsData.getClass().getSimpleName() + " to Short");
     }
-    if (numericTokenStream != null) {
-      numericTokenStream.setIntValue(value);
-    }
     fieldsData = Short.valueOf(value);
   }
 
@@ -304,9 +298,6 @@ public class Field implements IndexableF
     if (!(fieldsData instanceof Integer)) {
       throw new IllegalArgumentException("cannot change value type from " + fieldsData.getClass().getSimpleName() + " to Integer");
     }
-    if (numericTokenStream != null) {
-      numericTokenStream.setIntValue(value);
-    }
     fieldsData = Integer.valueOf(value);
   }
 
@@ -314,9 +305,6 @@ public class Field implements IndexableF
     if (!(fieldsData instanceof Long)) {
       throw new IllegalArgumentException("cannot change value type from " + fieldsData.getClass().getSimpleName() + " to Long");
     }
-    if (numericTokenStream != null) {
-      numericTokenStream.setLongValue(value);
-    }
     fieldsData = Long.valueOf(value);
   }
 
@@ -324,9 +312,6 @@ public class Field implements IndexableF
     if (!(fieldsData instanceof Float)) {
       throw new IllegalArgumentException("cannot change value type from " + fieldsData.getClass().getSimpleName() + " to Float");
     }
-    if (numericTokenStream != null) {
-      numericTokenStream.setFloatValue(value);
-    }
     fieldsData = Float.valueOf(value);
   }
 
@@ -334,9 +319,6 @@ public class Field implements IndexableF
     if (!(fieldsData instanceof Double)) {
       throw new IllegalArgumentException("cannot change value type from " + fieldsData.getClass().getSimpleName() + " to Double");
     }
-    if (numericTokenStream != null) {
-      numericTokenStream.setDoubleValue(value);
-    }
     fieldsData = Double.valueOf(value);
   }
 
@@ -434,62 +416,44 @@ public class Field implements IndexableF
 
     final NumericType numericType = fieldType().numericType();
     if (numericType != null) {
-      if (numericTokenStream == null) {
+      if (!(internalTokenStream instanceof NumericTokenStream)) {
         // lazy init the TokenStream as it is heavy to instantiate
         // (attributes,...) if not needed (stored field loading)
-        numericTokenStream = new NumericTokenStream(type.numericPrecisionStep());
-        // initialize value in TokenStream
-        final Number val = (Number) fieldsData;
-        switch (numericType) {
-        case INT:
-          numericTokenStream.setIntValue(val.intValue());
-          break;
-        case LONG:
-          numericTokenStream.setLongValue(val.longValue());
-          break;
-        case FLOAT:
-          numericTokenStream.setFloatValue(val.floatValue());
-          break;
-        case DOUBLE:
-          numericTokenStream.setDoubleValue(val.doubleValue());
-          break;
-        default:
-          assert false : "Should never get here";
-        }
-      } else {
-        // OK -- previously cached and we already updated if
-        // setters were called.
+        internalTokenStream = new NumericTokenStream(type.numericPrecisionStep());
       }
-
-      return numericTokenStream;
+      final NumericTokenStream nts = (NumericTokenStream) internalTokenStream;
+      // initialize value in TokenStream
+      final Number val = (Number) fieldsData;
+      switch (numericType) {
+      case INT:
+        nts.setIntValue(val.intValue());
+        break;
+      case LONG:
+        nts.setLongValue(val.longValue());
+        break;
+      case FLOAT:
+        nts.setFloatValue(val.floatValue());
+        break;
+      case DOUBLE:
+        nts.setDoubleValue(val.doubleValue());
+        break;
+      default:
+        assert false : "Should never get here";
+      }
+      return internalTokenStream;
     }
 
     if (!fieldType().tokenized()) {
       if (stringValue() == null) {
         throw new IllegalArgumentException("Non-Tokenized Fields must have a String value");
       }
-
-      return new TokenStream() {
-        CharTermAttribute termAttribute = addAttribute(CharTermAttribute.class);
-        OffsetAttribute offsetAttribute = addAttribute(OffsetAttribute.class);
-        boolean used;
-
-        @Override
-        public boolean incrementToken() {
-          if (used) {
-            return false;
-          }
-          termAttribute.setEmpty().append(stringValue());
-          offsetAttribute.setOffset(0, stringValue().length());
-          used = true;
-          return true;
-        }
-
-        @Override
-        public void reset() {
-          used = false;
-        }
-      };
+      if (!(internalTokenStream instanceof StringTokenStream)) {
+        // lazy init the TokenStream as it is heavy to instantiate
+        // (attributes,...) if not needed (stored field loading)
+        internalTokenStream = new StringTokenStream();
+      }
+      ((StringTokenStream) internalTokenStream).setValue(stringValue());
+      return internalTokenStream;
     }
 
     if (tokenStream != null) {
@@ -497,11 +461,102 @@ public class Field implements IndexableF
     } else if (readerValue() != null) {
       return analyzer.tokenStream(name(), readerValue());
     } else if (stringValue() != null) {
-      return analyzer.tokenStream(name(), new StringReader(stringValue()));
+      if (internalReader == null) {
+        internalReader = new ReusableStringReader();
+      }
+      internalReader.setValue(stringValue());
+      return analyzer.tokenStream(name(), internalReader);
     }
 
     throw new IllegalArgumentException("Field must have either TokenStream, String, Reader or Number value");
   }
+  
+  static final class ReusableStringReader extends Reader {
+    private int pos = 0, size = 0;
+    private String s = null;
+    
+    void setValue(String s) {
+      this.s = s;
+      this.size = s.length();
+      this.pos = 0;
+    }
+    
+    @Override
+    public int read() {
+      if (pos < size) {
+        return s.charAt(pos++);
+      } else {
+        s = null;
+        return -1;
+      }
+    }
+    
+    @Override
+    public int read(char[] c, int off, int len) {
+      if (pos < size) {
+        len = Math.min(len, size-pos);
+        s.getChars(pos, pos+len, c, off);
+        pos += len;
+        return len;
+      } else {
+        s = null;
+        return -1;
+      }
+    }
+    
+    @Override
+    public void close() {
+      pos = size; // this prevents NPE when reading after close!
+      s = null;
+    }
+  }
+  
+  static final class StringTokenStream extends TokenStream {
+    private final CharTermAttribute termAttribute = addAttribute(CharTermAttribute.class);
+    private final OffsetAttribute offsetAttribute = addAttribute(OffsetAttribute.class);
+    private boolean used = false;
+    private String value = null;
+    
+    /** Creates a new TokenStream that returns a String as single token.
+     * <p>Warning: Does not initialize the value, you must call
+     * {@link #setValue()} afterwards!
+     */
+    StringTokenStream() {
+    }
+    
+    /** Sets the string value. */
+    void setValue(String value) {
+      this.value = value;
+    }
+
+    @Override
+    public boolean incrementToken() {
+      if (used) {
+        return false;
+      }
+      clearAttributes();
+      termAttribute.append(value);
+      offsetAttribute.setOffset(0, value.length());
+      used = true;
+      return true;
+    }
+
+    @Override
+    public void end() {
+      final int finalOffset = value.length();
+      offsetAttribute.setOffset(finalOffset, finalOffset);
+    }
+    
+    @Override
+    public void reset() {
+      used = false;
+    }
+
+    @Override
+    public void close() {
+      value = null;
+    }
+  }
 
   /** Specifies whether and how a field should be stored. */
   public static enum Store {

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java Sat Aug 25 10:06:07 2012
@@ -175,17 +175,6 @@ public abstract class AtomicReader exten
     }
     return null;
   }
-
-  /** Returns the number of unique terms (across all fields)
-   *  in this reader.
-   */
-  public final long getUniqueTermCount() throws IOException {
-    final Fields fields = fields();
-    if (fields == null) {
-      return 0;
-    }
-    return fields.getUniqueTermCount();
-  }
   
   /**
    * Returns {@link DocValues} for this field.

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java Sat Aug 25 10:06:07 2012
@@ -1112,21 +1112,6 @@ public class CheckIndex {
         throw new RuntimeException("fieldCount mismatch " + fieldCount + " vs recomputed field count " + computedFieldCount);
       }
     }
-    
-    // for most implementations, this is boring (just the sum across all fields)
-    // but codecs that don't work per-field like preflex actually implement this,
-    // but don't implement it on Terms, so the check isn't redundant.
-    long uniqueTermCountAllFields = fields.getUniqueTermCount();
-    
-    // this means something is seriously screwed, e.g. we are somehow getting enclosed in PFCW!!!!!!
-    
-    if (uniqueTermCountAllFields == -1) {
-      throw new RuntimeException("invalid termCount: -1");
-    }
-    
-    if (status.termCount != uniqueTermCountAllFields) {
-      throw new RuntimeException("termCount mismatch " + uniqueTermCountAllFields + " vs " + (status.termCount));
-    }
 
     if (doPrint) {
       msg("OK [" + status.termCount + " terms; " + status.totFreq + " terms/docs pairs; " + status.totPos + " tokens]");

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/CoalescedDeletes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/CoalescedDeletes.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/CoalescedDeletes.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/CoalescedDeletes.java Sat Aug 25 10:06:07 2012
@@ -47,7 +47,7 @@ class CoalescedDeletes {
 
  public Iterable<Term> termsIterable() {
    return new Iterable<Term>() {
-     @SuppressWarnings("unchecked")
+     @SuppressWarnings({"unchecked","rawtypes"})
      @Override
      public Iterator<Term> iterator() {
        Iterator<Term> subs[] = new Iterator[iterables.size()];

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java Sat Aug 25 10:06:07 2012
@@ -34,6 +34,8 @@ import org.apache.lucene.index.FieldInfo
 public class FieldInfos implements Iterable<FieldInfo> {
   private final boolean hasFreq;
   private final boolean hasProx;
+  private final boolean hasPayloads;
+  private final boolean hasOffsets;
   private final boolean hasVectors;
   private final boolean hasNorms;
   private final boolean hasDocValues;
@@ -45,6 +47,8 @@ public class FieldInfos implements Itera
   public FieldInfos(FieldInfo[] infos) {
     boolean hasVectors = false;
     boolean hasProx = false;
+    boolean hasPayloads = false;
+    boolean hasOffsets = false;
     boolean hasFreq = false;
     boolean hasNorms = false;
     boolean hasDocValues = false;
@@ -58,12 +62,16 @@ public class FieldInfos implements Itera
       hasVectors |= info.hasVectors();
       hasProx |= info.isIndexed() && info.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
       hasFreq |= info.isIndexed() && info.getIndexOptions() != IndexOptions.DOCS_ONLY;
+      hasOffsets |= info.isIndexed() && info.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
       hasNorms |= info.hasNorms();
       hasDocValues |= info.hasDocValues();
+      hasPayloads |= info.hasPayloads();
     }
     
     this.hasVectors = hasVectors;
     this.hasProx = hasProx;
+    this.hasPayloads = hasPayloads;
+    this.hasOffsets = hasOffsets;
     this.hasFreq = hasFreq;
     this.hasNorms = hasNorms;
     this.hasDocValues = hasDocValues;
@@ -79,6 +87,16 @@ public class FieldInfos implements Itera
   public boolean hasProx() {
     return hasProx;
   }
+
+  /** Returns true if any fields have payloads */
+  public boolean hasPayloads() {
+    return hasPayloads;
+  }
+
+  /** Returns true if any fields have offsets */
+  public boolean hasOffsets() {
+    return hasOffsets;
+  }
   
   /**
    * @return true if at least one field has any vectors

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/Fields.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/Fields.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/Fields.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/Fields.java Sat Aug 25 10:06:07 2012
@@ -36,28 +36,7 @@ public abstract class Fields implements 
   /** Returns the number of fields or -1 if the number of
    * distinct field names is unknown. If &gt;= 0,
    * {@link #iterator} will return as many field names. */
-  public abstract int size() throws IOException;
-  
-  /** Returns the number of terms for all fields, or -1 if this 
-   *  measure isn't stored by the codec. Note that, just like 
-   *  other term measures, this measure does not take deleted 
-   *  documents into account. */
-  // TODO: deprecate?
-  public long getUniqueTermCount() throws IOException {
-    long numTerms = 0;
-    for (String field : this) {
-      Terms terms = terms(field);
-      if (terms != null) {
-        final long termCount = terms.size();
-        if (termCount == -1) {
-          return -1;
-        }
-          
-        numTerms += termCount;
-      }
-    }
-    return numTerms;
-  }
+  public abstract int size();
   
   public final static Fields[] EMPTY_ARRAY = new Fields[0];
 }

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java Sat Aug 25 10:06:07 2012
@@ -57,14 +57,9 @@ public class FilterAtomicReader extends 
     }
 
     @Override
-    public int size() throws IOException {
+    public int size() {
       return in.size();
     }
-
-    @Override
-    public long getUniqueTermCount() throws IOException {
-      return in.getUniqueTermCount();
-    }
   }
 
   /** Base class for filtering {@link Terms}

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/LogMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/LogMergePolicy.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/LogMergePolicy.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/LogMergePolicy.java Sat Aug 25 10:06:07 2012
@@ -64,6 +64,13 @@ public abstract class LogMergePolicy ext
    *  @see #setNoCFSRatio */
   public static final double DEFAULT_NO_CFS_RATIO = 0.1;
 
+  /** Default maxCFSSegmentSize value allows compound file
+   * for a segment of any size. The actual file format is
+   * still subject to noCFSRatio.
+   * @see #setMaxCFSSegmentSizeMB(double)
+   */
+  public static final long DEFAULT_MAX_CFS_SEGMENT_SIZE = Long.MAX_VALUE;
+
   protected int mergeFactor = DEFAULT_MERGE_FACTOR;
 
   protected long minMergeSize;
@@ -74,6 +81,7 @@ public abstract class LogMergePolicy ext
   protected int maxMergeDocs = DEFAULT_MAX_MERGE_DOCS;
 
   protected double noCFSRatio = DEFAULT_NO_CFS_RATIO;
+  protected long maxCFSSegmentSize = DEFAULT_MAX_CFS_SEGMENT_SIZE;
 
   protected boolean calibrateSizeByDeletes = true;
   
@@ -136,21 +144,21 @@ public abstract class LogMergePolicy ext
   // Javadoc inherited
   @Override
   public boolean useCompoundFile(SegmentInfos infos, SegmentInfoPerCommit mergedInfo) throws IOException {
-    final boolean doCFS;
-
-    if (!useCompoundFile) {
-      doCFS = false;
-    } else if (noCFSRatio == 1.0) {
-      doCFS = true;
-    } else {
-      long totalSize = 0;
-      for (SegmentInfoPerCommit info : infos) {
-        totalSize += size(info);
-      }
-
-      doCFS = size(mergedInfo) <= noCFSRatio * totalSize;
+    if (!getUseCompoundFile()) {
+      return false;
+    }
+    long mergedInfoSize = size(mergedInfo);
+    if (mergedInfoSize > maxCFSSegmentSize) {
+      return false;
+    }
+    if (getNoCFSRatio() >= 1.0) {
+      return true;
+    }
+    long totalSize = 0;
+    for (SegmentInfoPerCommit info : infos) {
+      totalSize += size(info);
     }
-    return doCFS;
+    return mergedInfoSize <= getNoCFSRatio() * totalSize;
   }
 
   /** Sets whether compound file format should be used for
@@ -674,9 +682,28 @@ public abstract class LogMergePolicy ext
     sb.append("calibrateSizeByDeletes=").append(calibrateSizeByDeletes).append(", ");
     sb.append("maxMergeDocs=").append(maxMergeDocs).append(", ");
     sb.append("useCompoundFile=").append(useCompoundFile).append(", ");
+    sb.append("maxCFSSegmentSizeMB=").append(getMaxCFSSegmentSizeMB()).append(", ");
     sb.append("noCFSRatio=").append(noCFSRatio);
     sb.append("]");
     return sb.toString();
   }
-  
+
+  /** Returns the largest size allowed for a compound file segment */
+  public final double getMaxCFSSegmentSizeMB() {
+    return maxCFSSegmentSize/1024/1024.;
+  }
+
+  /** If a merged segment will be more than this value,
+   *  leave the segment as
+   *  non-compound file even if compound file is enabled.
+   *  Set this to Double.POSITIVE_INFINITY (default) and noCFSRatio to 1.0
+   *  to always use CFS regardless of merge size. */
+  public final void setMaxCFSSegmentSizeMB(double v) {
+    if (v < 0.0) {
+      throw new IllegalArgumentException("maxCFSSegmentSizeMB must be >=0 (got " + v + ")");
+    }
+    v *= 1024 * 1024;
+    this.maxCFSSegmentSize = (v > Long.MAX_VALUE) ? Long.MAX_VALUE : (long) v;
+  }
+
 }

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java Sat Aug 25 10:06:07 2012
@@ -27,7 +27,6 @@ import java.util.SortedMap;
 import java.util.TreeMap;
 
 import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.UnmodifiableIterator;
 
 
 /** An {@link AtomicReader} which reads multiple, parallel indexes.  Each index
@@ -163,7 +162,7 @@ public final class ParallelAtomicReader 
     
     @Override
     public Iterator<String> iterator() {
-      return new UnmodifiableIterator<String>(fields.keySet().iterator());
+      return Collections.unmodifiableSet(fields.keySet()).iterator();
     }
     
     @Override

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java Sat Aug 25 10:06:07 2012
@@ -84,6 +84,7 @@ public class TieredMergePolicy extends M
   private double forceMergeDeletesPctAllowed = 10.0;
   private boolean useCompoundFile = true;
   private double noCFSRatio = 0.1;
+  private long maxCFSSegmentSize = Long.MAX_VALUE;
   private double reclaimDeletesWeight = 2.0;
 
   /** Maximum number of segments to be merged at a time
@@ -127,7 +128,11 @@ public class TieredMergePolicy extends M
    *  sizes of to-be-merged segments (compensating for
    *  percent deleted docs).  Default is 5 GB. */
   public TieredMergePolicy setMaxMergedSegmentMB(double v) {
-    maxMergedSegmentBytes = (long) (v*1024*1024);
+    if (v < 0.0) {
+      throw new IllegalArgumentException("maxMergedSegmentMB must be >=0 (got " + v + ")");
+    }
+    v *= 1024 * 1024;
+    maxMergedSegmentBytes = (v > Long.MAX_VALUE) ? Long.MAX_VALUE : (long) v;
     return this;
   }
 
@@ -162,7 +167,8 @@ public class TieredMergePolicy extends M
     if (v <= 0.0) {
       throw new IllegalArgumentException("floorSegmentMB must be >= 0.0 (got " + v + ")");
     }
-    floorSegmentBytes = (long) (v*1024*1024);
+    v *= 1024 * 1024;
+    floorSegmentBytes = (v > Long.MAX_VALUE) ? Long.MAX_VALUE : (long) v;
     return this;
   }
 
@@ -602,21 +608,21 @@ public class TieredMergePolicy extends M
 
   @Override
   public boolean useCompoundFile(SegmentInfos infos, SegmentInfoPerCommit mergedInfo) throws IOException {
-    final boolean doCFS;
-
-    if (!useCompoundFile) {
-      doCFS = false;
-    } else if (noCFSRatio == 1.0) {
-      doCFS = true;
-    } else {
-      long totalSize = 0;
-      for (SegmentInfoPerCommit info : infos) {
+    if (!getUseCompoundFile()) {
+        return false;
+    }
+    long mergedInfoSize = size(mergedInfo);
+    if (mergedInfoSize > maxCFSSegmentSize) {
+        return false;
+    }
+    if (getNoCFSRatio() >= 1.0) {
+        return true;
+    }
+    long totalSize = 0;
+    for (SegmentInfoPerCommit info : infos) {
         totalSize += size(info);
-      }
-
-      doCFS = size(mergedInfo) <= noCFSRatio * totalSize;
     }
-    return doCFS;
+    return mergedInfoSize <= getNoCFSRatio() * totalSize;
   }
 
   @Override
@@ -629,7 +635,7 @@ public class TieredMergePolicy extends M
     boolean hasDeletions = w.numDeletedDocs(info) > 0;
     return !hasDeletions &&
       info.info.dir == w.getDirectory() &&
-      (info.info.getUseCompoundFile() == useCompoundFile || noCFSRatio < 1.0);
+      (info.info.getUseCompoundFile() == useCompoundFile || noCFSRatio < 1.0 || maxCFSSegmentSize < Long.MAX_VALUE);
   }
 
   // Segment size in bytes, pro-rated by % deleted
@@ -664,7 +670,27 @@ public class TieredMergePolicy extends M
     sb.append("forceMergeDeletesPctAllowed=").append(forceMergeDeletesPctAllowed).append(", ");
     sb.append("segmentsPerTier=").append(segsPerTier).append(", ");
     sb.append("useCompoundFile=").append(useCompoundFile).append(", ");
+    sb.append("maxCFSSegmentSizeMB=").append(getMaxCFSSegmentSizeMB()).append(", ");
     sb.append("noCFSRatio=").append(noCFSRatio);
     return sb.toString();
   }
+
+  /** Returns the largest size allowed for a compound file segment */
+  public final double getMaxCFSSegmentSizeMB() {
+    return maxCFSSegmentSize/1024/1024.;
+  }
+
+  /** If a merged segment will be more than this value,
+   *  leave the segment as
+   *  non-compound file even if compound file is enabled.
+   *  Set this to Double.POSITIVE_INFINITY (default) and noCFSRatio to 1.0
+   *  to always use CFS regardless of merge size. */
+  public final TieredMergePolicy setMaxCFSSegmentSizeMB(double v) {
+    if (v < 0.0) {
+      throw new IllegalArgumentException("maxCFSSegmentSizeMB must be >=0 (got " + v + ")");
+    }
+    v *= 1024 * 1024;
+    this.maxCFSSegmentSize = (v > Long.MAX_VALUE) ? Long.MAX_VALUE : (long) v;
+    return this;
+  }
 }