You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2012/08/13 15:53:27 UTC

svn commit: r1372423 [31/45] - in /lucene/dev/branches/LUCENE-2878: ./ dev-tools/ dev-tools/eclipse/ dev-tools/idea/.idea/libraries/ dev-tools/maven/ dev-tools/maven/lucene/ dev-tools/maven/lucene/analysis/common/ dev-tools/maven/lucene/analysis/icu/ d...

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java Mon Aug 13 13:52:46 2012
@@ -842,7 +842,7 @@ public class IndexWriter implements Clos
         if (hitOOM) {
           rollbackInternal();
         } else {
-          closeInternal(waitForMerges, !hitOOM);
+          closeInternal(waitForMerges, true);
         }
       }
     }
@@ -870,7 +870,7 @@ public class IndexWriter implements Clos
   }
 
   private void closeInternal(boolean waitForMerges, boolean doFlush) throws IOException {
-
+    boolean interrupted = false;
     try {
 
       if (pendingCommit != null) {
@@ -883,26 +883,57 @@ public class IndexWriter implements Clos
 
       docWriter.close();
 
-      // Only allow a new merge to be triggered if we are
-      // going to wait for merges:
-      if (doFlush) {
-        flush(waitForMerges, true);
-      } else {
-        docWriter.abort(); // already closed
-      }
-
-      if (waitForMerges)
-        // Give merge scheduler last chance to run, in case
-        // any pending merges are waiting:
-        mergeScheduler.merge(this);
-
-      mergePolicy.close();
-
-      synchronized(this) {
-        finishMerges(waitForMerges);
-        stopMerges = true;
+      try {
+        // Only allow a new merge to be triggered if we are
+        // going to wait for merges:
+        if (doFlush) {
+          flush(waitForMerges, true);
+        } else {
+          docWriter.abort(); // already closed
+        }
+        
+      } finally {
+        try {
+          // clean up merge scheduler in all cases, although flushing may have failed:
+          interrupted = Thread.interrupted();
+        
+          if (waitForMerges) {
+            try {
+              // Give merge scheduler last chance to run, in case
+              // any pending merges are waiting:
+              mergeScheduler.merge(this);
+            } catch (ThreadInterruptedException tie) {
+              // ignore any interruption, does not matter
+              interrupted = true;
+              if (infoStream.isEnabled("IW")) {
+                infoStream.message("IW", "interrupted while waiting for final merges");
+              }
+            }
+          }
+          
+          synchronized(this) {
+            for (;;) {
+              try {
+                finishMerges(waitForMerges && !interrupted);
+                break;
+              } catch (ThreadInterruptedException tie) {
+                // by setting the interrupted status, the
+                // next call to finishMerges will pass false,
+                // so it will not wait
+                interrupted = true;
+                if (infoStream.isEnabled("IW")) {
+                  infoStream.message("IW", "interrupted while waiting for merges to finish");
+                }
+              }
+            }
+            stopMerges = true;
+          }
+          
+        } finally {
+          // shutdown policy, scheduler and all threads (this call is not interruptible):
+          IOUtils.closeWhileHandlingException(mergePolicy, mergeScheduler);
+        }
       }
-      mergeScheduler.close();
 
       if (infoStream.isEnabled("IW")) {
         infoStream.message("IW", "now call final commit()");
@@ -943,6 +974,8 @@ public class IndexWriter implements Clos
           }
         }
       }
+      // finally, restore interrupt status:
+      if (interrupted) Thread.currentThread().interrupt();
     }
   }
 
@@ -1208,6 +1241,78 @@ public class IndexWriter implements Clos
     }
   }
 
+  /** Expert: attempts to delete by document ID, as long as
+   *  the provided reader is a near-real-time reader (from {@link
+   *  DirectoryReader#open(IndexWriter,boolean)}).  If the
+   *  provided reader is an NRT reader obtained from this
+   *  writer, and its segment has not been merged away, then
+   *  the delete succeeds and this method returns true; else, it
+   *  returns false the caller must then separately delete by
+   *  Term or Query.
+   *
+   *  <b>NOTE</b>: this method can only delete documents
+   *  visible to the currently open NRT reader.  If you need
+   *  to delete documents indexed after opening the NRT
+   *  reader you must use the other deleteDocument methods
+   *  (e.g., {@link #deleteDocuments(Term)}). */
+  public synchronized boolean tryDeleteDocument(IndexReader readerIn, int docID) throws IOException {
+
+    final AtomicReader reader;
+    if (readerIn instanceof AtomicReader) {
+      // Reader is already atomic: use the incoming docID:
+      reader = (AtomicReader) readerIn;
+    } else {
+      // Composite reader: lookup sub-reader and re-base docID:
+      List<AtomicReaderContext> leaves = readerIn.getTopReaderContext().leaves();
+      int subIndex = ReaderUtil.subIndex(docID, leaves);
+      reader = leaves.get(subIndex).reader();
+      docID -= leaves.get(subIndex).docBase;
+      assert docID >= 0;
+      assert docID < reader.maxDoc();
+    }
+
+    if (!(reader instanceof SegmentReader)) {
+      throw new IllegalArgumentException("the reader must be a SegmentReader or composite reader containing only SegmentReaders");
+    }
+      
+    final SegmentInfoPerCommit info = ((SegmentReader) reader).getSegmentInfo();
+
+    // TODO: this is a slow linear search, but, number of
+    // segments should be contained unless something is
+    // seriously wrong w/ the index, so it should be a minor
+    // cost:
+
+    if (segmentInfos.indexOf(info) != -1) {
+      ReadersAndLiveDocs rld = readerPool.get(info, false);
+      if (rld != null) {
+        synchronized(bufferedDeletesStream) {
+          rld.initWritableLiveDocs();
+          if (rld.delete(docID)) {
+            final int fullDelCount = rld.info.getDelCount() + rld.getPendingDeleteCount();
+            if (fullDelCount == rld.info.info.getDocCount()) {
+              // If a merge has already registered for this
+              // segment, we leave it in the readerPool; the
+              // merge will skip merging it and will then drop
+              // it once it's done:
+              if (!mergingSegments.contains(rld.info)) {
+                segmentInfos.remove(rld.info);
+                readerPool.drop(rld.info);
+                checkpoint();
+              }
+            }
+          }
+          //System.out.println("  yes " + info.info.name + " " + docID);
+          return true;
+        }
+      } else {
+        //System.out.println("  no rld " + info.info.name + " " + docID);
+      }
+    } else {
+      //System.out.println("  no seg " + info.info.name + " " + docID);
+    }
+    return false;
+  }
+
   /**
    * Deletes the document(s) containing any of the
    * terms. All given deletes are applied and flushed atomically
@@ -2207,9 +2312,7 @@ public class IndexWriter implements Clos
         }
         SegmentInfos sis = new SegmentInfos(); // read infos from dir
         sis.read(dir);
-        final Set<String> dsFilesCopied = new HashSet<String>();
-        final Map<String, String> dsNames = new HashMap<String, String>();
-        final Set<String> copiedFiles = new HashSet<String>();
+
         for (SegmentInfoPerCommit info : sis) {
           assert !infos.contains(info): "dup info dir=" + info.info.dir + " name=" + info.info.name;
 
@@ -2222,7 +2325,7 @@ public class IndexWriter implements Clos
 
           IOContext context = new IOContext(new MergeInfo(info.info.getDocCount(), info.info.sizeInBytes(), true, -1));
           
-          infos.add(copySegmentAsIs(info, newSegName, dsNames, dsFilesCopied, context, copiedFiles));
+          infos.add(copySegmentAsIs(info, newSegName, context));
         }
       }
 
@@ -2358,25 +2461,9 @@ public class IndexWriter implements Clos
   }
 
   /** Copies the segment files as-is into the IndexWriter's directory. */
-  // TODO: this can be substantially simplified now that 3.x support/shared docstores is removed!
-  private SegmentInfoPerCommit copySegmentAsIs(SegmentInfoPerCommit info, String segName,
-                                               Map<String, String> dsNames, Set<String> dsFilesCopied, IOContext context,
-                                               Set<String> copiedFiles)
+  private SegmentInfoPerCommit copySegmentAsIs(SegmentInfoPerCommit info, String segName, IOContext context)
       throws IOException {
-    // Determine if the doc store of this segment needs to be copied. It's
-    // only relevant for segments that share doc store with others,
-    // because the DS might have been copied already, in which case we
-    // just want to update the DS name of this SegmentInfo.
-    final String dsName = info.info.name;
-    assert dsName != null;
-    final String newDsName;
-    if (dsNames.containsKey(dsName)) {
-      newDsName = dsNames.get(dsName);
-    } else {
-      dsNames.put(dsName, segName);
-      newDsName = segName;
-    }
-
+    
     // note: we don't really need this fis (its copied), but we load it up
     // so we don't pass a null value to the si writer
     FieldInfos fis = getFieldInfos(info.info);
@@ -2391,7 +2478,7 @@ public class IndexWriter implements Clos
     }
 
     //System.out.println("copy seg=" + info.info.name + " version=" + info.info.getVersion());
-    // Same SI as before but we change directory, name and docStoreSegment:
+    // Same SI as before but we change directory and name
     SegmentInfo newInfo = new SegmentInfo(directory, info.info.getVersion(), segName, info.info.getDocCount(),
                                           info.info.getUseCompoundFile(),
                                           info.info.getCodec(), info.info.getDiagnostics(), attributes);
@@ -2408,16 +2495,10 @@ public class IndexWriter implements Clos
     }
     newInfo.setFiles(segFiles);
 
-    // We must rewrite the SI file because it references
-    // segment name (its own name, if its 3.x, and doc
-    // store segment name):
+    // We must rewrite the SI file because it references segment name in its list of files, etc
     TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(directory);
-    try {
-      newInfo.getCodec().segmentInfoFormat().getSegmentInfoWriter().write(trackingDir, newInfo, fis, context);
-    } catch (UnsupportedOperationException uoe) {
-      // OK: 3x codec cannot write a new SI file;
-      // SegmentInfos will write this on commit
-    }
+
+    newInfo.getCodec().segmentInfoFormat().getSegmentInfoWriter().write(trackingDir, newInfo, fis, context);
 
     final Collection<String> siFiles = trackingDir.getCreatedFiles();
 
@@ -2432,8 +2513,7 @@ public class IndexWriter implements Clos
       }
 
       assert !directory.fileExists(newFileName): "file \"" + newFileName + "\" already exists; siFiles=" + siFiles;
-      assert !copiedFiles.contains(file): "file \"" + file + "\" is being copied more than once";
-      copiedFiles.add(file);
+
       info.info.dir.copy(directory, file, newFileName, context);
     }
     
@@ -3052,7 +3132,7 @@ public class IndexWriter implements Clos
     checkpoint();
 
     if (infoStream.isEnabled("IW")) {
-      infoStream.message("IW", "after commit: " + segString());
+      infoStream.message("IW", "after commitMerge: " + segString());
     }
 
     if (merge.maxNumSegments != -1 && !dropSegment) {

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java Mon Aug 13 13:52:46 2012
@@ -392,7 +392,7 @@ public final class IndexWriterConfig ext
    * @see #setMaxBufferedDocs(int)
    * @see #setRAMBufferSizeMB(double)
    */
-  public IndexWriterConfig setFlushPolicy(FlushPolicy flushPolicy) {
+  IndexWriterConfig setFlushPolicy(FlushPolicy flushPolicy) {
     this.flushPolicy = flushPolicy;
     return this;
   }
@@ -422,7 +422,7 @@ public final class IndexWriterConfig ext
   }
   
   @Override
-  public FlushPolicy getFlushPolicy() {
+  FlushPolicy getFlushPolicy() {
     return flushPolicy;
   }
   

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/IndexableFieldType.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/IndexableFieldType.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/IndexableFieldType.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/IndexableFieldType.java Mon Aug 13 13:52:46 2012
@@ -42,6 +42,9 @@ public interface IndexableFieldType {
 
   /** True if term vector positions should be indexed */
   public boolean storeTermVectorPositions();
+  
+  /** True if term vector payloads should be indexed */
+  public boolean storeTermVectorPayloads();
 
   /** True if norms should not be indexed */
   public boolean omitNorms();

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java Mon Aug 13 13:52:46 2012
@@ -19,6 +19,7 @@ package org.apache.lucene.index;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat; // javadocs
 import org.apache.lucene.index.DocumentsWriterPerThread.IndexingChain;
 import org.apache.lucene.index.IndexWriter.IndexReaderWarmer;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
@@ -146,6 +147,29 @@ public class LiveIndexWriterConfig {
    * Takes effect immediately, but only applies to newly flushed/merged
    * segments.
    * 
+   * <p>
+   * <b>NOTE:</b> This parameter does not apply to all PostingsFormat implementations,
+   * including the default one in this release. It only makes sense for term indexes
+   * that are implemented as a fixed gap between terms. For example, 
+   * {@link Lucene40PostingsFormat} implements the term index instead based upon how
+   * terms share prefixes. To configure its parameters (the minimum and maximum size
+   * for a block), you would instead use  {@link Lucene40PostingsFormat#Lucene40PostingsFormat(int, int)}.
+   * which can also be configured on a per-field basis:
+   * <pre class="prettyprint">
+   * //customize Lucene40PostingsFormat, passing minBlockSize=50, maxBlockSize=100
+   * final PostingsFormat tweakedPostings = new Lucene40PostingsFormat(50, 100);
+   * iwc.setCodec(new Lucene40Codec() {
+   *   &#64;Override
+   *   public PostingsFormat getPostingsFormatForField(String field) {
+   *     if (field.equals("fieldWithTonsOfTerms"))
+   *       return tweakedPostings;
+   *     else
+   *       return super.getPostingsFormatForField(field);
+   *   }
+   * });
+   * </pre>
+   * Note that other implementations may have their own parameters, or no parameters at all.
+   * 
    * @see IndexWriterConfig#DEFAULT_TERM_INDEX_INTERVAL
    */
   public LiveIndexWriterConfig setTermIndexInterval(int interval) { // TODO: this should be private to the codec, not settable here
@@ -335,6 +359,10 @@ public class LiveIndexWriterConfig {
    * <p>
    * Takes effect immediately, but only applies to readers opened after this
    * call
+   * <p>
+   * <b>NOTE:</b> divisor settings &gt; 1 do not apply to all PostingsFormat
+   * implementations, including the default one in this release. It only makes
+   * sense for terms indexes that can efficiently re-sample terms at load time.
    */
   public LiveIndexWriterConfig setReaderTermsIndexDivisor(int divisor) {
     if (divisor <= 0 && divisor != -1) {
@@ -462,7 +490,7 @@ public class LiveIndexWriterConfig {
   /**
    * @see IndexWriterConfig#setFlushPolicy(FlushPolicy)
    */
-  public FlushPolicy getFlushPolicy() {
+  FlushPolicy getFlushPolicy() {
     return flushPolicy;
   }
   
@@ -497,7 +525,6 @@ public class LiveIndexWriterConfig {
     sb.append("mergePolicy=").append(getMergePolicy()).append("\n");
     sb.append("indexerThreadPool=").append(getIndexerThreadPool()).append("\n");
     sb.append("readerPooling=").append(getReaderPooling()).append("\n");
-    sb.append("flushPolicy=").append(getFlushPolicy()).append("\n");
     sb.append("perThreadHardLimitMB=").append(getRAMPerThreadHardLimitMB()).append("\n");
     return sb.toString();
   }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MergeScheduler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MergeScheduler.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MergeScheduler.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MergeScheduler.java Mon Aug 13 13:52:46 2012
@@ -17,6 +17,7 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
+import java.io.Closeable;
 import java.io.IOException;
 
 /** <p>Expert: {@link IndexWriter} uses an instance
@@ -26,7 +27,7 @@ import java.io.IOException;
  *
  * @lucene.experimental
 */
-public abstract class MergeScheduler {
+public abstract class MergeScheduler implements Closeable {
 
   /** Run the merges provided by {@link IndexWriter#getNextMerge()}. */
   public abstract void merge(IndexWriter writer) throws IOException;

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MergeState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MergeState.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MergeState.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MergeState.java Mon Aug 13 13:52:46 2012
@@ -199,6 +199,7 @@ public class MergeState {
   // and we could make a codec(wrapper) to do all of this privately so IW is uninvolved
   public PayloadProcessorProvider payloadProcessorProvider;
   public ReaderPayloadProcessor[] readerPayloadProcessor;
+  public ReaderPayloadProcessor currentReaderPayloadProcessor;
   public PayloadProcessor[] currentPayloadProcessor;
 
   // TODO: get rid of this? it tells you which segments are 'aligned' (e.g. for bulk merging)

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MultiDocsAndPositionsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MultiDocsAndPositionsEnum.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MultiDocsAndPositionsEnum.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MultiDocsAndPositionsEnum.java Mon Aug 13 13:52:46 2012
@@ -20,6 +20,7 @@ package org.apache.lucene.index;
 import org.apache.lucene.util.BytesRef;
 
 import java.io.IOException;
+import java.util.Arrays;
 
 /**
  * Exposes flex API, merged from flex API of sub-segments.
@@ -137,11 +138,6 @@ public final class MultiDocsAndPositions
   }
 
   @Override
-  public boolean hasPayload() {
-    return current.hasPayload();
-  }
-
-  @Override
   public BytesRef getPayload() throws IOException {
     return current.getPayload();
   }
@@ -150,6 +146,16 @@ public final class MultiDocsAndPositions
   public final static class EnumWithSlice {
     public DocsAndPositionsEnum docsAndPositionsEnum;
     public ReaderSlice slice;
+    
+    @Override
+    public String toString() {
+      return slice.toString()+":"+docsAndPositionsEnum;
+    }
+  }
+  
+  @Override
+  public String toString() {
+    return "MultiDocsAndPositionsEnum(" + Arrays.toString(getSubs()) + ")";
   }
 }
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MultiFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MultiFields.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MultiFields.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MultiFields.java Mon Aug 13 13:52:46 2012
@@ -19,6 +19,7 @@ package org.apache.lucene.index;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Collection;
@@ -27,6 +28,7 @@ import java.util.concurrent.ConcurrentHa
 
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.MergedIterator;
 
 /**
  * Exposes flex API, merged from flex API of sub-segments.
@@ -122,32 +124,54 @@ public final class MultiFields extends F
   }
   
   /** Returns {@link DocsEnum} for the specified field &
-   *  term.  This may return null if the term does not
-   *  exist. */
-  public static DocsEnum getTermDocsEnum(IndexReader r, Bits liveDocs, String field, BytesRef term, boolean needsFreqs) throws IOException {
+   *  term.  This will return null if the field or term does
+   *  not exist. */
+  public static DocsEnum getTermDocsEnum(IndexReader r, Bits liveDocs, String field, BytesRef term) throws IOException {
+    return getTermDocsEnum(r, liveDocs, field, term, DocsEnum.FLAG_FREQS);
+  }
+  
+  /** Returns {@link DocsEnum} for the specified field &
+   *  term, with control over whether freqs are required.
+   *  Some codecs may be able to optimize their
+   *  implementation when freqs are not required.  This will
+   *  return null if the field or term does not exist.  See {@link
+   *  TermsEnum#docs(Bits,DocsEnum,int)}.*/
+  public static DocsEnum getTermDocsEnum(IndexReader r, Bits liveDocs, String field, BytesRef term, int flags) throws IOException {
     assert field != null;
     assert term != null;
     final Terms terms = getTerms(r, field);
     if (terms != null) {
       final TermsEnum termsEnum = terms.iterator(null);
       if (termsEnum.seekExact(term, true)) {
-        return termsEnum.docs(liveDocs, null, needsFreqs);
+        return termsEnum.docs(liveDocs, null, flags);
       }
     }
     return null;
   }
 
   /** Returns {@link DocsAndPositionsEnum} for the specified
-   *  field & term.  This may return null if the term does
-   *  not exist or positions were not indexed. */
-  public static DocsAndPositionsEnum getTermPositionsEnum(IndexReader r, Bits liveDocs, String field, BytesRef term, boolean needsOffsets) throws IOException {
+   *  field & term.  This will return null if the field or
+   *  term does not exist or positions were not indexed. 
+   *  @see #getTermPositionsEnum(IndexReader, Bits, String, BytesRef, int) */
+  public static DocsAndPositionsEnum getTermPositionsEnum(IndexReader r, Bits liveDocs, String field, BytesRef term) throws IOException {
+    return getTermPositionsEnum(r, liveDocs, field, term, DocsAndPositionsEnum.FLAG_OFFSETS | DocsAndPositionsEnum.FLAG_PAYLOADS);
+  }
+
+  /** Returns {@link DocsAndPositionsEnum} for the specified
+   *  field & term, with control over whether offsets and payloads are
+   *  required.  Some codecs may be able to optimize
+   *  their implementation when offsets and/or payloads are not
+   *  required. This will return null if the field or term does not
+   *  exist or positions were not indexed. See {@link
+   *  TermsEnum#docsAndPositions(Bits,DocsAndPositionsEnum,int)}. */
+  public static DocsAndPositionsEnum getTermPositionsEnum(IndexReader r, Bits liveDocs, String field, BytesRef term, int flags) throws IOException {
     assert field != null;
     assert term != null;
     final Terms terms = getTerms(r, field);
     if (terms != null) {
       final TermsEnum termsEnum = terms.iterator(null);
       if (termsEnum.seekExact(term, true)) {
-        return termsEnum.docsAndPositions(liveDocs, null, needsOffsets);
+        return termsEnum.docsAndPositions(liveDocs, null, flags);
       }
     }
     return null;
@@ -158,22 +182,14 @@ public final class MultiFields extends F
     this.subSlices = subSlices;
   }
 
+  @SuppressWarnings("unchecked")
   @Override
-  public FieldsEnum iterator() throws IOException {
-
-    final List<FieldsEnum> fieldsEnums = new ArrayList<FieldsEnum>();
-    final List<ReaderSlice> fieldsSlices = new ArrayList<ReaderSlice>();
+  public Iterator<String> iterator() {
+    Iterator<String> subIterators[] = new Iterator[subs.length];
     for(int i=0;i<subs.length;i++) {
-      fieldsEnums.add(subs[i].iterator());
-      fieldsSlices.add(subSlices[i]);
-    }
-    if (fieldsEnums.size() == 0) {
-      return FieldsEnum.EMPTY;
-    } else {
-      return new MultiFieldsEnum(this,
-                                 fieldsEnums.toArray(FieldsEnum.EMPTY_ARRAY),
-                                 fieldsSlices.toArray(ReaderSlice.EMPTY_ARRAY));
+      subIterators[i] = subs[i].iterator();
     }
+    return new MergedIterator<String>(subIterators);
   }
 
   @Override

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MultiTerms.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MultiTerms.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MultiTerms.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MultiTerms.java Mon Aug 13 13:52:46 2012
@@ -37,12 +37,19 @@ public final class MultiTerms extends Te
   private final Terms[] subs;
   private final ReaderSlice[] subSlices;
   private final Comparator<BytesRef> termComp;
+  private final boolean hasOffsets;
+  private final boolean hasPositions;
+  private final boolean hasPayloads;
 
   public MultiTerms(Terms[] subs, ReaderSlice[] subSlices) throws IOException {
     this.subs = subs;
     this.subSlices = subSlices;
     
     Comparator<BytesRef> _termComp = null;
+    assert subs.length > 0 : "inefficient: don't use MultiTerms over one sub";
+    boolean _hasOffsets = true;
+    boolean _hasPositions = true;
+    boolean _hasPayloads = false;
     for(int i=0;i<subs.length;i++) {
       if (_termComp == null) {
         _termComp = subs[i].getComparator();
@@ -54,9 +61,15 @@ public final class MultiTerms extends Te
           throw new IllegalStateException("sub-readers have different BytesRef.Comparators; cannot merge");
         }
       }
+      _hasOffsets &= subs[i].hasOffsets();
+      _hasPositions &= subs[i].hasPositions();
+      _hasPayloads |= subs[i].hasPayloads();
     }
 
     termComp = _termComp;
+    hasOffsets = _hasOffsets;
+    hasPositions = _hasPositions;
+    hasPayloads = hasPositions && _hasPayloads; // if all subs have pos, and at least one has payloads.
   }
 
   @Override
@@ -142,5 +155,20 @@ public final class MultiTerms extends Te
   public Comparator<BytesRef> getComparator() {
     return termComp;
   }
+
+  @Override
+  public boolean hasOffsets() {
+    return hasOffsets;
+  }
+
+  @Override
+  public boolean hasPositions() {
+    return hasPositions;
+  }
+  
+  @Override
+  public boolean hasPayloads() {
+    return hasPayloads;
+  }
 }
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java Mon Aug 13 13:52:46 2012
@@ -344,7 +344,7 @@ public final class MultiTermsEnum extend
   }
 
   @Override
-  public DocsEnum docs(Bits liveDocs, DocsEnum reuse, boolean needsFreqs) throws IOException {
+  public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
     MultiDocsEnum docsEnum;
     // Can only reuse if incoming enum is also a MultiDocsEnum
     if (reuse != null && reuse instanceof MultiDocsEnum) {
@@ -394,16 +394,15 @@ public final class MultiTermsEnum extend
       }
 
       assert entry.index < docsEnum.subDocsEnum.length: entry.index + " vs " + docsEnum.subDocsEnum.length + "; " + subs.length;
-      final DocsEnum subDocsEnum = entry.terms.docs(b, docsEnum.subDocsEnum[entry.index], needsFreqs);
+      final DocsEnum subDocsEnum = entry.terms.docs(b, docsEnum.subDocsEnum[entry.index], flags);
       if (subDocsEnum != null) {
         docsEnum.subDocsEnum[entry.index] = subDocsEnum;
         subDocs[upto].docsEnum = subDocsEnum;
         subDocs[upto].slice = entry.subSlice;
         upto++;
       } else {
-        // One of our subs cannot provide freqs:
-        assert needsFreqs;
-        return null;
+        // One of our subs cannot provide a docsenum:
+        assert false;
       }
     }
 
@@ -415,7 +414,7 @@ public final class MultiTermsEnum extend
   }
 
   @Override
-  public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException {
+  public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
     MultiDocsAndPositionsEnum docsAndPositionsEnum;
     // Can only reuse if incoming enum is also a MultiDocsAndPositionsEnum
     if (reuse != null && reuse instanceof MultiDocsAndPositionsEnum) {
@@ -466,7 +465,7 @@ public final class MultiTermsEnum extend
       }
 
       assert entry.index < docsAndPositionsEnum.subDocsAndPositionsEnum.length: entry.index + " vs " + docsAndPositionsEnum.subDocsAndPositionsEnum.length + "; " + subs.length;
-      final DocsAndPositionsEnum subPostings = entry.terms.docsAndPositions(b, docsAndPositionsEnum.subDocsAndPositionsEnum[entry.index], needsOffsets);
+      final DocsAndPositionsEnum subPostings = entry.terms.docsAndPositions(b, docsAndPositionsEnum.subDocsAndPositionsEnum[entry.index], flags);
 
       if (subPostings != null) {
         docsAndPositionsEnum.subDocsAndPositionsEnum[entry.index] = subPostings;
@@ -474,7 +473,7 @@ public final class MultiTermsEnum extend
         subDocsAndPositions[upto].slice = entry.subSlice;
         upto++;
       } else {
-        if (entry.terms.docs(b, null, false) != null) {
+        if (entry.terms.docs(b, null, 0) != null) {
           // At least one of our subs does not store
           // offsets or positions -- we can't correctly
           // produce a MultiDocsAndPositions enum

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java Mon Aug 13 13:52:46 2012
@@ -27,6 +27,7 @@ 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
@@ -121,12 +122,10 @@ public final class ParallelAtomicReader 
     for (final AtomicReader reader : this.parallelReaders) {
       final Fields readerFields = reader.fields();
       if (readerFields != null) {
-        final FieldsEnum it = readerFields.iterator();
-        String name;
-        while ((name = it.next()) != null) {
+        for (String field : readerFields) {
           // only add if the reader responsible for that field name is the current:
-          if (fieldToReader.get(name) == reader) {
-            this.fields.addField(name, it.terms());
+          if (fieldToReader.get(field) == reader) {
+            this.fields.addField(field, readerFields.terms(field));
           }
         }
       }
@@ -151,33 +150,6 @@ public final class ParallelAtomicReader 
     return buffer.append(')').toString();
   }
   
-  private final class ParallelFieldsEnum extends FieldsEnum {
-    private String currentField;
-    private final Iterator<String> keys;
-    private final ParallelFields fields;
-    
-    ParallelFieldsEnum(ParallelFields fields) {
-      this.fields = fields;
-      keys = fields.fields.keySet().iterator();
-    }
-    
-    @Override
-    public String next() {
-      if (keys.hasNext()) {
-        currentField = keys.next();
-      } else {
-        currentField = null;
-      }
-      return currentField;
-    }
-    
-    @Override
-    public Terms terms() {
-      return fields.terms(currentField);
-    }
-    
-  }
-  
   // Single instance of this, per ParallelReader instance
   private final class ParallelFields extends Fields {
     final Map<String,Terms> fields = new TreeMap<String,Terms>();
@@ -190,8 +162,8 @@ public final class ParallelAtomicReader 
     }
     
     @Override
-    public FieldsEnum iterator() {
-      return new ParallelFieldsEnum(this);
+    public Iterator<String> iterator() {
+      return new UnmodifiableIterator<String>(fields.keySet().iterator());
     }
     
     @Override

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java Mon Aug 13 13:52:46 2012
@@ -24,6 +24,7 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
+import java.util.regex.Matcher;
 
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.store.Directory;
@@ -242,16 +243,31 @@ public final class SegmentInfo {
   private Set<String> setFiles;
 
   public void setFiles(Set<String> files) {
+    checkFileNames(files);
     setFiles = files;
     sizeInBytes = -1;
   }
 
   public void addFiles(Collection<String> files) {
+    checkFileNames(files);
     setFiles.addAll(files);
+    sizeInBytes = -1;
   }
 
   public void addFile(String file) {
+    checkFileNames(Collections.singleton(file));
     setFiles.add(file);
+    sizeInBytes = -1;
+  }
+  
+  private void checkFileNames(Collection<String> files) {
+    Matcher m = IndexFileNames.CODEC_FILE_PATTERN.matcher("");
+    for (String file : files) {
+      m.reset(file);
+      if (!m.matches()) {
+        throw new IllegalArgumentException("invalid codec filename '" + file + "', must match: " + IndexFileNames.CODEC_FILE_PATTERN.pattern());
+      }
+    }
   }
     
   /**

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java Mon Aug 13 13:52:46 2012
@@ -20,6 +20,7 @@ package org.apache.lucene.index;
 import java.io.IOException;
 
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
+import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
 import org.apache.lucene.codecs.TermVectorsWriter;
 import org.apache.lucene.util.ByteBlockPool;
 import org.apache.lucene.util.BytesRef;
@@ -36,9 +37,12 @@ final class TermVectorsConsumerPerField 
   boolean doVectors;
   boolean doVectorPositions;
   boolean doVectorOffsets;
+  boolean doVectorPayloads;
 
   int maxNumPostings;
   OffsetAttribute offsetAttribute;
+  PayloadAttribute payloadAttribute;
+  boolean hasPayloads; // if enabled, and we actually saw any for this field
 
   public TermVectorsConsumerPerField(TermsHashPerField termsHashPerField, TermVectorsConsumer termsWriter, FieldInfo fieldInfo) {
     this.termsHashPerField = termsHashPerField;
@@ -58,13 +62,46 @@ final class TermVectorsConsumerPerField 
     doVectors = false;
     doVectorPositions = false;
     doVectorOffsets = false;
+    doVectorPayloads = false;
+    hasPayloads = false;
 
     for(int i=0;i<count;i++) {
       IndexableField field = fields[i];
-      if (field.fieldType().indexed() && field.fieldType().storeTermVectors()) {
-        doVectors = true;
-        doVectorPositions |= field.fieldType().storeTermVectorPositions();
-        doVectorOffsets |= field.fieldType().storeTermVectorOffsets();
+      if (field.fieldType().indexed()) {
+        if (field.fieldType().storeTermVectors()) {
+          doVectors = true;
+          doVectorPositions |= field.fieldType().storeTermVectorPositions();
+          doVectorOffsets |= field.fieldType().storeTermVectorOffsets();
+          if (doVectorPositions) {
+            doVectorPayloads |= field.fieldType().storeTermVectorPayloads();
+          } else if (field.fieldType().storeTermVectorPayloads()) {
+            // TODO: move this check somewhere else, and impl the other missing ones
+            throw new IllegalArgumentException("cannot index term vector payloads for field: " + field + " without term vector positions");
+          }
+        } else {
+          if (field.fieldType().storeTermVectorOffsets()) {
+            throw new IllegalArgumentException("cannot index term vector offsets when term vectors are not indexed (field=\"" + field.name());
+          }
+          if (field.fieldType().storeTermVectorPositions()) {
+            throw new IllegalArgumentException("cannot index term vector positions when term vectors are not indexed (field=\"" + field.name());
+          }
+          if (field.fieldType().storeTermVectorPayloads()) {
+            throw new IllegalArgumentException("cannot index term vector payloads when term vectors are not indexed (field=\"" + field.name());
+          }
+        }
+      } else {
+        if (field.fieldType().storeTermVectors()) {
+          throw new IllegalArgumentException("cannot index term vectors when field is not indexed (field=\"" + field.name());
+        }
+        if (field.fieldType().storeTermVectorOffsets()) {
+          throw new IllegalArgumentException("cannot index term vector offsets when field is not indexed (field=\"" + field.name());
+        }
+        if (field.fieldType().storeTermVectorPositions()) {
+          throw new IllegalArgumentException("cannot index term vector positions when field is not indexed (field=\"" + field.name());
+        }
+        if (field.fieldType().storeTermVectorPayloads()) {
+          throw new IllegalArgumentException("cannot index term vector payloads when field is not indexed (field=\"" + field.name());
+        }
       }
     }
 
@@ -121,7 +158,7 @@ final class TermVectorsConsumerPerField 
 
     final int[] termIDs = termsHashPerField.sortPostings(tv.getComparator());
 
-    tv.startField(fieldInfo, numPostings, doVectorPositions, doVectorOffsets);
+    tv.startField(fieldInfo, numPostings, doVectorPositions, doVectorOffsets, hasPayloads);
     
     final ByteSliceReader posReader = doVectorPositions ? termsWriter.vectorSliceReaderPos : null;
     final ByteSliceReader offReader = doVectorOffsets ? termsWriter.vectorSliceReaderOff : null;
@@ -165,52 +202,64 @@ final class TermVectorsConsumerPerField 
     } else {
       offsetAttribute = null;
     }
+    if (doVectorPayloads && fieldState.attributeSource.hasAttribute(PayloadAttribute.class)) {
+      payloadAttribute = fieldState.attributeSource.getAttribute(PayloadAttribute.class);
+    } else {
+      payloadAttribute = null;
+    }
   }
-
-  @Override
-  void newTerm(final int termID) {
-    assert docState.testPoint("TermVectorsTermsWriterPerField.newTerm start");
-    TermVectorsPostingsArray postings = (TermVectorsPostingsArray) termsHashPerField.postingsArray;
-
-    postings.freqs[termID] = 1;
-
+  
+  void writeProx(TermVectorsPostingsArray postings, int termID) {    
     if (doVectorOffsets) {
       int startOffset = fieldState.offset + offsetAttribute.startOffset();
       int endOffset = fieldState.offset + offsetAttribute.endOffset();
 
-      termsHashPerField.writeVInt(1, startOffset);
+      termsHashPerField.writeVInt(1, startOffset - postings.lastOffsets[termID]);
       termsHashPerField.writeVInt(1, endOffset - startOffset);
       postings.lastOffsets[termID] = endOffset;
     }
 
     if (doVectorPositions) {
-      termsHashPerField.writeVInt(0, fieldState.position);
+      final BytesRef payload;
+      if (payloadAttribute == null) {
+        payload = null;
+      } else {
+        payload = payloadAttribute.getPayload();
+      }
+      
+      final int pos = fieldState.position - postings.lastPositions[termID];
+      if (payload != null && payload.length > 0) {
+        termsHashPerField.writeVInt(0, (pos<<1)|1);
+        termsHashPerField.writeVInt(0, payload.length);
+        termsHashPerField.writeBytes(0, payload.bytes, payload.offset, payload.length);
+        hasPayloads = true;
+      } else {
+        termsHashPerField.writeVInt(0, pos<<1);
+      }
       postings.lastPositions[termID] = fieldState.position;
     }
   }
 
   @Override
-  void addTerm(final int termID) {
+  void newTerm(final int termID) {
+    assert docState.testPoint("TermVectorsTermsWriterPerField.newTerm start");
+    TermVectorsPostingsArray postings = (TermVectorsPostingsArray) termsHashPerField.postingsArray;
 
-    assert docState.testPoint("TermVectorsTermsWriterPerField.addTerm start");
+    postings.freqs[termID] = 1;
+    postings.lastOffsets[termID] = 0;
+    postings.lastPositions[termID] = 0;
+    
+    writeProx(postings, termID);
+  }
 
+  @Override
+  void addTerm(final int termID) {
+    assert docState.testPoint("TermVectorsTermsWriterPerField.addTerm start");
     TermVectorsPostingsArray postings = (TermVectorsPostingsArray) termsHashPerField.postingsArray;
 
     postings.freqs[termID]++;
 
-    if (doVectorOffsets) {
-      int startOffset = fieldState.offset + offsetAttribute.startOffset();
-      int endOffset = fieldState.offset + offsetAttribute.endOffset();
-
-      termsHashPerField.writeVInt(1, startOffset - postings.lastOffsets[termID]);
-      termsHashPerField.writeVInt(1, endOffset - startOffset);
-      postings.lastOffsets[termID] = endOffset;
-    }
-
-    if (doVectorPositions) {
-      termsHashPerField.writeVInt(0, fieldState.position - postings.lastPositions[termID]);
-      postings.lastPositions[termID] = fieldState.position;
-    }
+    writeProx(postings, termID);
   }
 
   @Override

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/Terms.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/Terms.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/Terms.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/Terms.java Mon Aug 13 13:52:46 2012
@@ -104,6 +104,15 @@ public abstract class Terms {
    *  measures, this measure does not take deleted documents
    *  into account. */
   public abstract int getDocCount() throws IOException;
+  
+  /** Returns true if documents in this field store offsets. */
+  public abstract boolean hasOffsets();
+  
+  /** Returns true if documents in this field store positions. */
+  public abstract boolean hasPositions();
+  
+  /** Returns true if documents in this field store payloads. */
+  public abstract boolean hasPayloads();
 
   public final static Terms[] EMPTY_ARRAY = new Terms[0];
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TermsEnum.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TermsEnum.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TermsEnum.java Mon Aug 13 13:52:46 2012
@@ -140,26 +140,55 @@ public abstract class TermsEnum implemen
 
   /** Get {@link DocsEnum} for the current term.  Do not
    *  call this when the enum is unpositioned.  This method
-   *  may return null (if needsFreqs is true but freqs were
-   *  not indexed for this field).
+   *  will not return null.
+   *  
+   * @param liveDocs unset bits are documents that should not
+   * be returned
+   * @param reuse pass a prior DocsEnum for possible reuse */
+  public final DocsEnum docs(Bits liveDocs, DocsEnum reuse) throws IOException {
+    return docs(liveDocs, reuse, DocsEnum.FLAG_FREQS);
+  }
+
+  /** Get {@link DocsEnum} for the current term, with
+   *  control over whether freqs are required.  Do not
+   *  call this when the enum is unpositioned.  This method
+   *  will not return null.
    *  
    * @param liveDocs unset bits are documents that should not
    * be returned
    * @param reuse pass a prior DocsEnum for possible reuse
-   * @param needsFreqs true if the caller intends to call
-   * {@link DocsEnum#freq}.  If you pass false you must not
-   * call {@link DocsEnum#freq} in the returned DocsEnum. */
-  public abstract DocsEnum docs(Bits liveDocs, DocsEnum reuse, boolean needsFreqs) throws IOException;
+   * @param flags specifies which optional per-document values
+   *        you require; see {@link DocsEnum#FLAG_FREQS} 
+   * @see #docs(Bits, DocsEnum, int) */
+  public abstract DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException;
 
   /** Get {@link DocsAndPositionsEnum} for the current term.
-   *  Do not call this when the enum is unpositioned.
-   *  This method will only return null if needsOffsets is
-   *  true but offsets were not indexed.
+   *  Do not call this when the enum is unpositioned.  This
+   *  method will return null if positions were not
+   *  indexed.
+   *  
+   *  @param liveDocs unset bits are documents that should not
+   *  be returned
+   *  @param reuse pass a prior DocsAndPositionsEnum for possible reuse
+   *  @see #docsAndPositions(Bits, DocsAndPositionsEnum, int) */
+  public final DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse) throws IOException {
+    return docsAndPositions(liveDocs, reuse, DocsAndPositionsEnum.FLAG_OFFSETS | DocsAndPositionsEnum.FLAG_PAYLOADS);
+  }
+
+  /** Get {@link DocsAndPositionsEnum} for the current term,
+   *  with control over whether offsets and payloads are
+   *  required.  Some codecs may be able to optimize their
+   *  implementation when offsets and/or payloads are not required.
+   *  Do not call this when the enum is unpositioned.  This
+   *  will return null if positions were not indexed.
+
    *  @param liveDocs unset bits are documents that should not
    *  be returned
    *  @param reuse pass a prior DocsAndPositionsEnum for possible reuse
-   *  @param needsOffsets true if offsets are required */
-  public abstract DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException;
+   *  @param flags specifies which optional per-position values you
+   *         require; see {@link DocsAndPositionsEnum#FLAG_OFFSETS} and 
+   *         {@link DocsAndPositionsEnum#FLAG_PAYLOADS}. */
+  public abstract DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException;
 
   /**
    * Expert: Returns the TermsEnums internal state to position the TermsEnum
@@ -220,12 +249,12 @@ public abstract class TermsEnum implemen
     }
 
     @Override
-    public DocsEnum docs(Bits liveDocs, DocsEnum reuse, boolean needsFreqs) {
+    public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) {
       throw new IllegalStateException("this method should never be called");
     }
       
     @Override
-    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) {
+    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) {
       throw new IllegalStateException("this method should never be called");
     }
       

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java Mon Aug 13 13:52:46 2012
@@ -212,7 +212,11 @@ public class BooleanQuery extends Query 
     }
 
     public float coord(int overlap, int maxOverlap) {
-      return similarity.coord(overlap, maxOverlap);
+      // LUCENE-4300: in most cases of maxOverlap=1, BQ rewrites itself away,
+      // so coord() is not applied. But when BQ cannot optimize itself away
+      // for a single clause (minNrShouldMatch, prohibited clauses, etc), its
+      // important not to apply coord(1,1) for consistency, it might not be 1.0F
+      return maxOverlap == 1 ? 1F : similarity.coord(overlap, maxOverlap);
     }
 
     @Override
@@ -239,7 +243,7 @@ public class BooleanQuery extends Query 
       for (Iterator<Weight> wIter = weights.iterator(); wIter.hasNext();) {
         Weight w = wIter.next();
         BooleanClause c = cIter.next();
-        if (w.scorer(context, true, true, FeatureFlags.DOCS, context.reader().getLiveDocs()) == null) {
+        if (w.scorer(context, true, true, PostingFeatures.DOCS_AND_FREQS, context.reader().getLiveDocs()) == null) {
           if (c.isRequired()) {
             fail = true;
             Explanation r = new Explanation(0.0f, "no match on required clause (" + c.getQuery().toString() + ")");
@@ -302,7 +306,7 @@ public class BooleanQuery extends Query 
 
     @Override
     public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
-        boolean topScorer, FeatureFlags flags, Bits acceptDocs)
+        boolean topScorer, PostingFeatures flags, Bits acceptDocs)
         throws IOException {
       if (termConjunction) {
         // specialized scorer for term conjunctions
@@ -329,7 +333,7 @@ public class BooleanQuery extends Query 
       }
       
       // Check if we can return a BooleanScorer
-      if (!scoreDocsInOrder && flags == FeatureFlags.DOCS && topScorer && required.size() == 0) {
+      if (!scoreDocsInOrder && flags == PostingFeatures.DOCS_AND_FREQS && topScorer && required.size() == 0) {
         return new BooleanScorer(this, disableCoord, minNrShouldMatch, optional, prohibited, maxCoord);
       }
       
@@ -347,7 +351,7 @@ public class BooleanQuery extends Query 
       return new BooleanScorer2(this, disableCoord, minNrShouldMatch, required, prohibited, optional, maxCoord);
     }
 
-    private Scorer createConjunctionTermScorer(AtomicReaderContext context, Bits acceptDocs, FeatureFlags flags)
+    private Scorer createConjunctionTermScorer(AtomicReaderContext context, Bits acceptDocs, PostingFeatures flags)
         throws IOException {
 
       // TODO: fix scorer API to specify "needsScores" up
@@ -360,11 +364,9 @@ public class BooleanQuery extends Query 
         final Scorer scorer = weight.scorer(context, true, false, flags, acceptDocs);
         if (scorer == null) {
           return null;
-        }
-        if (scorer instanceof TermScorer) {
-          docsAndFreqs[i] = new DocsAndFreqs((TermScorer) scorer);
         } else {
-          docsAndFreqs[i] = new DocsAndFreqs((MatchOnlyTermScorer) scorer);
+          assert scorer instanceof TermScorer;
+          docsAndFreqs[i] = new DocsAndFreqs((TermScorer) scorer);
         }
       }
       return new ConjunctionTermScorer(this, disableCoord ? 1.0f : coord(

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BooleanScorer2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BooleanScorer2.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BooleanScorer2.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BooleanScorer2.java Mon Aug 13 13:52:46 2012
@@ -42,16 +42,16 @@ class BooleanScorer2 extends Scorer {
   private final List<Scorer> prohibitedScorers;
 
   private class Coordinator {
-    float[] coordFactors = null;
-    int maxCoord = 0; // to be increased for each non prohibited scorer
-    int nrMatchers; // to be increased by score() of match counting scorers.
-    
-    void init(boolean disableCoord) { // use after all scorers have been added.
+    final float coordFactors[];
+
+    Coordinator(int maxCoord, boolean disableCoord) {
       coordFactors = new float[optionalScorers.size() + requiredScorers.size() + 1];
       for (int i = 0; i < coordFactors.length; i++) {
         coordFactors[i] = disableCoord ? 1.0f : ((BooleanWeight)weight).coord(i, maxCoord);
       }
     }
+    
+    int nrMatchers; // to be increased by score() of match counting scorers.
   }
 
   private final Coordinator coordinator;
@@ -93,15 +93,13 @@ class BooleanScorer2 extends Scorer {
     if (minNrShouldMatch < 0) {
       throw new IllegalArgumentException("Minimum number of optional scorers should not be negative");
     }
-    coordinator = new Coordinator();
     this.minNrShouldMatch = minNrShouldMatch;
-    coordinator.maxCoord = maxCoord;
 
     optionalScorers = optional;
     requiredScorers = required;    
     prohibitedScorers = prohibited;
+    coordinator = new Coordinator(maxCoord, disableCoord);
     
-    coordinator.init(disableCoord);
     countingSumScorer = makeCountingSumScorer(disableCoord);
   }
   
@@ -183,7 +181,7 @@ class BooleanScorer2 extends Scorer {
                                               List<Scorer> requiredScorers) throws IOException {
     // each scorer from the list counted as a single matcher
     final int requiredNrMatchers = requiredScorers.size();
-    return new ConjunctionScorer(weight, disableCoord ? 1.0f : ((BooleanWeight)weight).coord(requiredScorers.size(), requiredScorers.size()), requiredScorers) {
+    return new ConjunctionScorer(weight, requiredScorers) {
       private int lastScoredDoc = -1;
       // Save the score of lastScoredDoc, so that we don't compute it more than
       // once in score().
@@ -207,8 +205,8 @@ class BooleanScorer2 extends Scorer {
   }
 
   private Scorer dualConjunctionSumScorer(boolean disableCoord,
-                                          Scorer req1, Scorer req2) throws IOException { // non counting.
-    return new ConjunctionScorer(weight, disableCoord ? 1.0f : ((BooleanWeight)weight).coord(2, 2), req1, req2);
+                                                Scorer req1, Scorer req2) throws IOException { // non counting.
+    return new ConjunctionScorer(weight, req1, req2);
     // All scorers match, so defaultSimilarity always has 1 as
     // the coordination factor.
     // Therefore the sum of the scores of two scorers

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/Collector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/Collector.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/Collector.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/Collector.java Mon Aug 13 13:52:46 2012
@@ -21,7 +21,7 @@ import java.io.IOException;
 
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.IndexReaderContext;
-import org.apache.lucene.search.Weight.FeatureFlags;
+import org.apache.lucene.search.Weight.PostingFeatures;
 
 /**
  * <p>Expert: Collectors are primarily meant to be used to
@@ -174,9 +174,9 @@ public abstract class Collector {
    */
   public abstract boolean acceptsDocsOutOfOrder();
   
-  
-  public FeatureFlags scorerFlags() {
-    return FeatureFlags.DOCS;
+  //nocommit add javadocs
+  public PostingFeatures postingFeatures() {
+    return PostingFeatures.DOCS_AND_FREQS;
   }
   
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java Mon Aug 13 13:52:46 2012
@@ -32,19 +32,17 @@ class ConjunctionScorer extends Scorer {
   
   private final Scorer[] scorersOrdered;
   private final Scorer[] scorers;
-  private final float coord;
   private int lastDoc = -1;
 
-  public ConjunctionScorer(Weight weight, float coord, Collection<Scorer> scorers) throws IOException {
-    this(weight, coord, scorers.toArray(new Scorer[scorers.size()]));
+  public ConjunctionScorer(Weight weight, Collection<Scorer> scorers) throws IOException {
+    this(weight, scorers.toArray(new Scorer[scorers.size()]));
   }
   
-  public ConjunctionScorer(Weight weight, float coord, Scorer... scorers) throws IOException {
+  public ConjunctionScorer(Weight weight, Scorer... scorers) throws IOException {
     super(weight);
     scorersOrdered = new Scorer[scorers.length];
     System.arraycopy(scorers, 0, scorersOrdered, 0, scorers.length);
     this.scorers = scorers;
-    this.coord = coord;
     
     for (int i = 0; i < scorers.length; i++) {
       if (scorers[i].nextDoc() == NO_MORE_DOCS) {
@@ -142,7 +140,7 @@ class ConjunctionScorer extends Scorer {
     for (int i = 0; i < scorers.length; i++) {
       sum += scorers[i].score();
     }
-    return sum * coord;
+    return sum;
   }
   
   @Override

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionTermScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionTermScorer.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionTermScorer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionTermScorer.java Mon Aug 13 13:52:46 2012
@@ -128,10 +128,6 @@ class ConjunctionTermScorer extends Scor
       this(termScorer, termScorer.getDocsEnum(), termScorer.getDocFreq());
     }
     
-    DocsAndFreqs(MatchOnlyTermScorer termScorer) {
-      this(termScorer, termScorer.getDocsEnum(), termScorer.getDocFreq());
-    }
-    
     DocsAndFreqs(Scorer scorer, DocsEnum docs, int docFreq) {
       this.docs = docs;
       this.docFreq = docFreq;

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java Mon Aug 13 13:52:46 2012
@@ -20,7 +20,7 @@ package org.apache.lucene.search;
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.search.Weight.FeatureFlags;
+import org.apache.lucene.search.Weight.PostingFeatures;
 import org.apache.lucene.search.positions.IntervalIterator;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.ToStringUtils;
@@ -124,7 +124,7 @@ public class ConstantScoreQuery extends 
 
     @Override
     public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
-        boolean topScorer, FeatureFlags flags, final Bits acceptDocs) throws IOException {
+        boolean topScorer, PostingFeatures flags, final Bits acceptDocs) throws IOException {
       final DocIdSetIterator disi;
       if (filter != null) {
         assert query == null;
@@ -151,7 +151,7 @@ public class ConstantScoreQuery extends 
 
     @Override
     public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
-      final Scorer cs = scorer(context, true, false, FeatureFlags.DOCS, context.reader().getLiveDocs());
+      final Scorer cs = scorer(context, true, false, PostingFeatures.DOCS_AND_FREQS, context.reader().getLiveDocs());
       final boolean exists = (cs != null && cs.advance(doc) == doc);
 
       final ComplexExplanation result = new ComplexExplanation();
@@ -225,8 +225,8 @@ public class ConstantScoreQuery extends 
         }
         
         @Override
-        public FeatureFlags scorerFlags() {
-          return collector.scorerFlags();
+        public PostingFeatures postingFeatures() {
+          return collector.postingFeatures();
         }
 
         @Override

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java Mon Aug 13 13:52:46 2012
@@ -152,7 +152,7 @@ public class DisjunctionMaxQuery extends
     /** Create the scorer used to score our associated DisjunctionMaxQuery */
     @Override
     public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
-        boolean topScorer, FeatureFlags flags, Bits acceptDocs) throws IOException {
+        boolean topScorer, PostingFeatures flags, Bits acceptDocs) throws IOException {
       Scorer[] scorers = new Scorer[weights.size()];
       int idx = 0;
       for (Weight w : weights) {

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java Mon Aug 13 13:52:46 2012
@@ -64,13 +64,10 @@ final class ExactPhraseScorer extends Sc
   
   private final Similarity.ExactSimScorer docScorer;
 
-  private final boolean needsOffsets;
-  
   ExactPhraseScorer(Weight weight, PhraseQuery.PostingsAndFreq[] postings,
-      Similarity.ExactSimScorer docScorer, boolean needsOffsets) throws IOException {
+      Similarity.ExactSimScorer docScorer) throws IOException {
     super(weight);
     this.docScorer = docScorer;
-    this.needsOffsets = needsOffsets;
     
     chunkStates = new ChunkState[postings.length];
     
@@ -334,7 +331,7 @@ final class ExactPhraseScorer extends Sc
     TermIntervalIterator[] posIters = new TermIntervalIterator[chunkStates.length];
     DocsAndPositionsEnum[] enums = new DocsAndPositionsEnum[chunkStates.length];
     for (int i = 0; i < chunkStates.length; i++) {
-      posIters[i] = new TermIntervalIterator(this, enums[i] = chunkStates[i].factory.docsAndPositionsEnum(needsOffsets), false, collectPositions);
+      posIters[i] = new TermIntervalIterator(this, enums[i] = chunkStates[i].factory.docsAndPositionsEnum(), false, collectPositions);
     }
     return new PhraseScorer.AdvancingIntervalIterator(this, collectPositions, enums, new BlockIntervalIterator(this, collectPositions, posIters));
   }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java Mon Aug 13 13:52:46 2012
@@ -364,7 +364,7 @@ class FieldCacheImpl implements FieldCac
               break;
             }
             final byte termval = parser.parseByte(term);
-            docs = termsEnum.docs(null, docs, false);
+            docs = termsEnum.docs(null, docs, 0);
             while (true) {
               final int docID = docs.nextDoc();
               if (docID == DocIdSetIterator.NO_MORE_DOCS) {
@@ -437,7 +437,7 @@ class FieldCacheImpl implements FieldCac
               break;
             }
             final short termval = parser.parseShort(term);
-            docs = termsEnum.docs(null, docs, false);
+            docs = termsEnum.docs(null, docs, 0);
             while (true) {
               final int docID = docs.nextDoc();
               if (docID == DocIdSetIterator.NO_MORE_DOCS) {
@@ -541,7 +541,7 @@ class FieldCacheImpl implements FieldCac
               retArray = new int[maxDoc];
             }
 
-            docs = termsEnum.docs(null, docs, false);
+            docs = termsEnum.docs(null, docs, 0);
             while (true) {
               final int docID = docs.nextDoc();
               if (docID == DocIdSetIterator.NO_MORE_DOCS) {
@@ -608,7 +608,7 @@ class FieldCacheImpl implements FieldCac
             res = new FixedBitSet(maxDoc);
           }
 
-          docs = termsEnum.docs(null, docs, false);
+          docs = termsEnum.docs(null, docs, 0);
           // TODO: use bulk API
           while (true) {
             final int docID = docs.nextDoc();
@@ -691,7 +691,7 @@ class FieldCacheImpl implements FieldCac
               retArray = new float[maxDoc];
             }
             
-            docs = termsEnum.docs(null, docs, false);
+            docs = termsEnum.docs(null, docs, 0);
             while (true) {
               final int docID = docs.nextDoc();
               if (docID == DocIdSetIterator.NO_MORE_DOCS) {
@@ -779,7 +779,7 @@ class FieldCacheImpl implements FieldCac
               retArray = new long[maxDoc];
             }
 
-            docs = termsEnum.docs(null, docs, false);
+            docs = termsEnum.docs(null, docs, 0);
             while (true) {
               final int docID = docs.nextDoc();
               if (docID == DocIdSetIterator.NO_MORE_DOCS) {
@@ -868,7 +868,7 @@ class FieldCacheImpl implements FieldCac
               retArray = new double[maxDoc];
             }
 
-            docs = termsEnum.docs(null, docs, false);
+            docs = termsEnum.docs(null, docs, 0);
             while (true) {
               final int docID = docs.nextDoc();
               if (docID == DocIdSetIterator.NO_MORE_DOCS) {
@@ -1042,12 +1042,12 @@ class FieldCacheImpl implements FieldCac
       }
 
       @Override
-      public DocsEnum docs(Bits liveDocs, DocsEnum reuse, boolean needsFreqs) throws IOException {
+      public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
         throw new UnsupportedOperationException();
       }
 
       @Override
-      public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException {
+      public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
         throw new UnsupportedOperationException();
       }
 
@@ -1167,7 +1167,7 @@ class FieldCacheImpl implements FieldCac
             termOrdToBytesOffset = termOrdToBytesOffset.resize(ArrayUtil.oversize(1+termOrd, 1));
           }
           termOrdToBytesOffset.set(termOrd, bytes.copyUsingLengthPrefix(term));
-          docs = termsEnum.docs(null, docs, false);
+          docs = termsEnum.docs(null, docs, 0);
           while (true) {
             final int docID = docs.nextDoc();
             if (docID == DocIdSetIterator.NO_MORE_DOCS) {
@@ -1288,7 +1288,7 @@ class FieldCacheImpl implements FieldCac
             break;
           }
           final long pointer = bytes.copyUsingLengthPrefix(term);
-          docs = termsEnum.docs(null, docs, false);
+          docs = termsEnum.docs(null, docs, 0);
           while (true) {
             final int docID = docs.nextDoc();
             if (docID == DocIdSetIterator.NO_MORE_DOCS) {

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java Mon Aug 13 13:52:46 2012
@@ -124,7 +124,7 @@ public class FilteredQuery extends Query
 
       // return a filtering scorer
       @Override
-      public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder, boolean topScorer, FeatureFlags flags, Bits acceptDocs) throws IOException {
+      public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder, boolean topScorer, PostingFeatures flags, Bits acceptDocs) throws IOException {
         assert filter != null;
 
         final DocIdSet filterDocIdSet = filter.getDocIdSet(context, acceptDocs);

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java Mon Aug 13 13:52:46 2012
@@ -122,7 +122,7 @@ public class FuzzyTermsEnum extends Term
     this.realPrefixLength = prefixLength > termLength ? termLength : prefixLength;
     // if minSimilarity >= 1, we treat it as number of edits
     if (minSimilarity >= 1f) {
-      this.minSimilarity = 1 - (minSimilarity+1) / this.termLength;
+      this.minSimilarity = 0; // just driven by number of edits
       maxEdits = (int) minSimilarity;
       raw = true;
     } else {
@@ -272,14 +272,14 @@ public class FuzzyTermsEnum extends Term
   }
   
   @Override
-  public DocsEnum docs(Bits liveDocs, DocsEnum reuse, boolean needsFreqs) throws IOException {
-    return actualEnum.docs(liveDocs, reuse, needsFreqs);
+  public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
+    return actualEnum.docs(liveDocs, reuse, flags);
   }
   
   @Override
   public DocsAndPositionsEnum docsAndPositions(Bits liveDocs,
-                                               DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException {
-    return actualEnum.docsAndPositions(liveDocs, reuse, needsOffsets);
+                                               DocsAndPositionsEnum reuse, int flags) throws IOException {
+    return actualEnum.docsAndPositions(liveDocs, reuse, flags);
   }
   
   @Override

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java Mon Aug 13 13:52:46 2012
@@ -569,7 +569,7 @@ public class IndexSearcher {
     // always use single thread:
     for (AtomicReaderContext ctx : leaves) { // search each subreader
       collector.setNextReader(ctx);
-      Scorer scorer = weight.scorer(ctx, !collector.acceptsDocsOutOfOrder(), true, collector.scorerFlags(), ctx.reader().getLiveDocs());
+      Scorer scorer = weight.scorer(ctx, !collector.acceptsDocsOutOfOrder(), true, collector.postingFeatures(), ctx.reader().getLiveDocs());
       if (scorer != null) {
         scorer.score(collector);
       }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java Mon Aug 13 13:52:46 2012
@@ -117,7 +117,7 @@ public class MatchAllDocsQuery extends Q
 
     @Override
     public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
-        boolean topScorer, FeatureFlags flags, Bits acceptDocs) throws IOException {
+        boolean topScorer, PostingFeatures flags, Bits acceptDocs) throws IOException {
       return new MatchAllScorer(context.reader(), acceptDocs, this, queryWeight);
     }
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java Mon Aug 13 13:52:46 2012
@@ -19,6 +19,7 @@ package org.apache.lucene.search;
 
 import org.apache.lucene.index.*;
 import org.apache.lucene.search.PhraseQuery.TermDocsEnumFactory;
+import org.apache.lucene.search.Weight.PostingFeatures;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.search.similarities.Similarity.SloppySimScorer;
 import org.apache.lucene.util.*;
@@ -165,7 +166,7 @@ public class MultiPhraseQuery extends Qu
 
     @Override
     public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
-        boolean topScorer, FeatureFlags flags, Bits acceptDocs) throws IOException {
+        boolean topScorer, PostingFeatures flags, Bits acceptDocs) throws IOException {
       assert !termArrays.isEmpty();
       final AtomicReader reader = context.reader();
       final Bits liveDocs = acceptDocs;
@@ -207,7 +208,7 @@ public class MultiPhraseQuery extends Qu
             // None of the terms are in this reader
             return null;
           }
-          factory = new MultiTermDocsEnumFactory(liveDocs, context, terms, termContexts, termsEnum);
+          factory = new MultiTermDocsEnumFactory(liveDocs, context, terms, termContexts, termsEnum, flags);
         } else {
           final Term term = terms[0];
           TermState termState = termContexts.get(term).get(context.ord);
@@ -216,15 +217,15 @@ public class MultiPhraseQuery extends Qu
             return null;
           }
           termsEnum.seekExact(term.bytes(), termState);
-          postingsEnum = termsEnum.docsAndPositions(liveDocs, null, false);
+          postingsEnum = termsEnum.docsAndPositions(liveDocs, null, 0);
 
           if (postingsEnum == null) {
             // term does exist, but has no positions
-            assert termsEnum.docs(liveDocs, null, false) != null: "termstate found but no term exists in reader";
+            assert termsEnum.docs(liveDocs, null, 0) != null: "termstate found but no term exists in reader";
             throw new IllegalStateException("field \"" + term.field() + "\" was indexed without position data; cannot run PhraseQuery (term=" + term.text() + ")");
           }
 
-          factory = new TermDocsEnumFactory(BytesRef.deepCopyOf(term.bytes()), termsEnum, acceptDocs);
+          factory = new TermDocsEnumFactory(BytesRef.deepCopyOf(term.bytes()), termsEnum, flags, acceptDocs);
         }
         
         postingsFreqs[pos] = new PhraseQuery.PostingsAndFreq(postingsEnum, factory, termsEnum.docFreq() , positions.get(pos).intValue(), terms);
@@ -236,20 +237,20 @@ public class MultiPhraseQuery extends Qu
       }
 
       if (slop == 0) {
-        ExactPhraseScorer s = new ExactPhraseScorer(this, postingsFreqs, similarity.exactSimScorer(stats, context), flags == FeatureFlags.OFFSETS);
+        ExactPhraseScorer s = new ExactPhraseScorer(this, postingsFreqs, similarity.exactSimScorer(stats, context));
         if (s.noDocs) {
           return null;
         } else {
           return s;
         }
       } else {
-        return new SloppyPhraseScorer(this, postingsFreqs, slop, similarity.sloppySimScorer(stats, context), flags == FeatureFlags.OFFSETS);
+        return new SloppyPhraseScorer(this, postingsFreqs, slop, similarity.sloppySimScorer(stats, context));
       }
     }
 
     @Override
     public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
-      Scorer scorer = scorer(context, true, false, FeatureFlags.POSITIONS, context.reader().getLiveDocs());
+      Scorer scorer = scorer(context, true, false, PostingFeatures.POSITIONS, context.reader().getLiveDocs());
       if (scorer != null) {
         int newDoc = scorer.advance(doc);
         if (newDoc == doc) {
@@ -401,16 +402,16 @@ public class MultiPhraseQuery extends Qu
     Map<Term, TermContext> termContexts;
 
     MultiTermDocsEnumFactory(Bits liveDocs, AtomicReaderContext context, Term[] terms,
-                             Map<Term,TermContext> termContexts, TermsEnum termsEnum) throws IOException {
-      super(termsEnum, liveDocs);
+                             Map<Term,TermContext> termContexts, TermsEnum termsEnum, PostingFeatures flags) throws IOException {
+      super(termsEnum, flags, liveDocs);
       this.context = context;
       this.terms = terms;
       this.termContexts = termContexts;
     }
 
     @Override
-    public DocsAndPositionsEnum docsAndPositionsEnum(boolean offsets) throws IOException {
-      return new UnionDocsAndPositionsEnum(liveDocs, context, terms, termContexts, termsEnum, offsets);
+    public DocsAndPositionsEnum docsAndPositionsEnum() throws IOException {
+      return new UnionDocsAndPositionsEnum(liveDocs, context, terms, termContexts, termsEnum, flags);
     }
 
   }
@@ -535,11 +536,11 @@ class UnionDocsAndPositionsEnum extends 
 
   public UnionDocsAndPositionsEnum(Bits liveDocs, AtomicReaderContext context, Term[] terms,
                                    Map<Term,TermContext> termContexts, TermsEnum termsEnum) throws IOException {
-    this(liveDocs, context, terms, termContexts, termsEnum, false);
+    this(liveDocs, context, terms, termContexts, termsEnum, PostingFeatures.POSITIONS);
   }
 
   public UnionDocsAndPositionsEnum(Bits liveDocs, AtomicReaderContext context, Term[] terms,
-                                     Map<Term,TermContext> termContexts, TermsEnum termsEnum, boolean needsOffsets) throws IOException {
+                                     Map<Term,TermContext> termContexts, TermsEnum termsEnum, PostingFeatures flags) throws IOException {
     List<DocsAndPositionsEnum> docsEnums = new LinkedList<DocsAndPositionsEnum>();
     for (int i = 0; i < terms.length; i++) {
       final Term term = terms[i];
@@ -549,7 +550,7 @@ class UnionDocsAndPositionsEnum extends 
         continue;
       }
       termsEnum.seekExact(term.bytes(), termState);
-      DocsAndPositionsEnum postings = termsEnum.docsAndPositions(liveDocs, null, needsOffsets);
+      DocsAndPositionsEnum postings = termsEnum.docsAndPositions(liveDocs, null, flags.docsAndPositionsFlags());
       if (postings == null) {
         // term does exist, but has no positions
         throw new IllegalStateException("field \"" + term.field() + "\" was indexed without position data; cannot run PhraseQuery (term=" + term.text() + ")");
@@ -613,12 +614,7 @@ class UnionDocsAndPositionsEnum extends 
 
   @Override
   public BytesRef getPayload() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public boolean hasPayload() {
-    throw new UnsupportedOperationException();
+    return null;
   }
 
   @Override

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java Mon Aug 13 13:52:46 2012
@@ -106,7 +106,7 @@ public class MultiTermQueryWrapperFilter
       do {
         // System.out.println("  iter termCount=" + termCount + " term=" +
         // enumerator.term().toBytesString());
-        docsEnum = termsEnum.docs(acceptDocs, docsEnum, false);
+        docsEnum = termsEnum.docs(acceptDocs, docsEnum, 0);
         int docid;
         while ((docid = docsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
           bitSet.set(docid);

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/NRTManager.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/NRTManager.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/NRTManager.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/NRTManager.java Mon Aug 13 13:52:46 2012
@@ -27,6 +27,7 @@ import java.util.concurrent.locks.Reentr
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.SegmentInfoPerCommit;
 import org.apache.lucene.index.IndexReader; // javadocs
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexableField;
@@ -254,6 +255,14 @@ public class NRTManager extends Referenc
     long getAndIncrementGeneration() {
       return indexingGen.getAndIncrement();
     }
+
+    public long tryDeleteDocument(IndexReader reader, int docID) throws IOException {
+      if (writer.tryDeleteDocument(reader, docID)) {
+        return indexingGen.get();
+      } else {
+        return -1;
+      }
+    }
   }
 
   /**