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/09/21 19:22:27 UTC

svn commit: r1388574 [15/45] - in /lucene/dev/branches/LUCENE-2878: ./ dev-tools/ dev-tools/eclipse/ dev-tools/eclipse/dot.settings/ dev-tools/idea/ dev-tools/idea/.idea/ dev-tools/idea/.idea/libraries/ dev-tools/idea/lucene/ dev-tools/idea/lucene/anal...

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/SortedBytesMergeUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/SortedBytesMergeUtils.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/SortedBytesMergeUtils.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/SortedBytesMergeUtils.java Fri Sep 21 17:21:34 2012
@@ -32,6 +32,9 @@ import org.apache.lucene.util.PriorityQu
 import org.apache.lucene.util.packed.PackedInts;
 
 /**
+ * Utility class for merging SortedBytes DocValues
+ * instances.
+ *  
  * @lucene.internal
  */
 public final class SortedBytesMergeUtils {
@@ -40,6 +43,8 @@ public final class SortedBytesMergeUtils
     // no instance
   }
 
+  /** Creates the {@link MergeContext} necessary for merging
+   *  the ordinals. */
   public static MergeContext init(Type type, DocValues[] docValues,
       Comparator<BytesRef> comp, int mergeDocCount) {
     int size = -1;
@@ -54,15 +59,32 @@ public final class SortedBytesMergeUtils
     }
     return new MergeContext(comp, mergeDocCount, size, type);
   }
-
+  /**
+   * Encapsulates contextual information about the merge. 
+   * This class holds document id to ordinal mappings, offsets for
+   * variable length values and the comparator to sort the merged
+   * bytes.
+   * 
+   * @lucene.internal
+   */
   public static final class MergeContext {
     private final Comparator<BytesRef> comp;
     private final BytesRef missingValue = new BytesRef();
+
+    /** How many bytes each value occupies, or -1 if it
+     *  varies. */
     public final int sizePerValues; // -1 if var length
+
     final Type type;
+
+    /** Maps each document to the ordinal for its value. */
     public final int[] docToEntry;
+
+    /** File-offset for each document; will be null if it's
+     *  not needed (eg fixed-size values). */
     public long[] offsets; // if non-null #mergeRecords collects byte offsets here
 
+    /** Sole constructor. */
     public MergeContext(Comparator<BytesRef> comp, int mergeDocCount,
         int size, Type type) {
       assert type == Type.BYTES_FIXED_SORTED || type == Type.BYTES_VAR_SORTED;
@@ -75,12 +97,15 @@ public final class SortedBytesMergeUtils
       }
       docToEntry = new int[mergeDocCount];
     }
-    
+
+    /** Returns number of documents merged. */
     public int getMergeDocCount() {
       return docToEntry.length;
     }
   }
 
+  /** Creates the {@link SortedSourceSlice}s for
+   *  merging. */
   public static List<SortedSourceSlice> buildSlices(
       int[] docBases, MergeState.DocMap[] docMaps,
       DocValues[] docValues, MergeContext ctx) throws IOException {
@@ -140,6 +165,8 @@ public final class SortedBytesMergeUtils
     }
   }
 
+  /** Does the "real work" of merging the slices and
+   *  computing the ord mapping. */
   public static int mergeRecords(MergeContext ctx, BytesRefConsumer consumer,
       List<SortedSourceSlice> slices) throws IOException {
     final RecordMerger merger = new RecordMerger(new MergeQueue(slices.size(),
@@ -169,13 +196,40 @@ public final class SortedBytesMergeUtils
     return merger.currentOrd;
   }
   
+  /**
+   * Implementation of this interface consume the merged bytes with their
+   * corresponding ordinal and byte offset. The offset is the byte offset in
+   * target sorted source where the currently merged {@link BytesRef} instance
+   * should be stored at.
+   */
   public static interface BytesRefConsumer {
+    
+    /**
+     * Consumes a single {@link BytesRef}. The provided {@link BytesRef}
+     * instances are strictly increasing with respect to the used
+     * {@link Comparator} used for merging
+     * 
+     * @param ref
+     *          the {@link BytesRef} to consume
+     * @param ord
+     *          the ordinal of the given {@link BytesRef} in the merge target
+     * @param offset
+     *          the byte offset of the given {@link BytesRef} in the merge
+     *          target
+     * @throws IOException
+     *           if an {@link IOException} occurs
+     */
     public void consume(BytesRef ref, int ord, long offset) throws IOException;
   }
   
+  /**
+   * A simple {@link BytesRefConsumer} that writes the merged {@link BytesRef}
+   * instances sequentially to an {@link IndexOutput}.
+   */
   public static final class IndexOutputBytesRefConsumer implements BytesRefConsumer {
     private final IndexOutput datOut;
     
+    /** Sole constructor. */
     public IndexOutputBytesRefConsumer(IndexOutput datOut) {
       this.datOut = datOut;
     }
@@ -186,7 +240,15 @@ public final class SortedBytesMergeUtils
           currentMergedBytes.length);      
     }
   }
-
+  
+  /**
+   * {@link RecordMerger} merges a list of {@link SortedSourceSlice} lazily by
+   * consuming the sorted source records one by one and de-duplicates records
+   * that are shared across slices. The algorithm is based on a lazy priority queue
+   * that prevents reading merge sources into heap memory. 
+   * 
+   * @lucene.internal
+   */
   private static final class RecordMerger {
     private final MergeQueue queue;
     private final SortedSourceSlice[] top;
@@ -231,6 +293,12 @@ public final class SortedBytesMergeUtils
     }
   }
 
+  /**
+   * {@link SortedSourceSlice} represents a single {@link SortedSource} merge candidate.
+   * It encapsulates ordinal and pre-calculated target doc id to ordinal mappings.
+   * This class also holds state private to the merge process.
+   * @lucene.internal
+   */
   public static class SortedSourceSlice {
     final SortedSource source;
     final int readerIdx;
@@ -280,7 +348,10 @@ public final class SortedBytesMergeUtils
       }
       return null;
     }
-    
+
+    /** Fills in the absolute ords for this slice. 
+     * 
+     * @return the provided {@code docToOrd} */
     public int[] toAbsolutOrds(int[] docToOrd) {
       for (int i = docToOrdStart; i < docToOrdEnd; i++) {
         final int mappedOrd = docIDToRelativeOrd[i];
@@ -291,6 +362,7 @@ public final class SortedBytesMergeUtils
       return docToOrd;
     }
 
+    /** Writes ords for this slice. */
     public void writeOrds(PackedInts.Writer writer) throws IOException {
       for (int i = docToOrdStart; i < docToOrdEnd; i++) {
         final int mappedOrd = docIDToRelativeOrd[i];

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java Fri Sep 21 17:21:34 2012
@@ -273,7 +273,7 @@ final class StandardDirectoryReader exte
     return reader;
   }
 
-  private synchronized DirectoryReader doOpenNoWriter(IndexCommit commit) throws IOException {
+  private DirectoryReader doOpenNoWriter(IndexCommit commit) throws IOException {
 
     if (commit == null) {
       if (isCurrent()) {
@@ -298,7 +298,7 @@ final class StandardDirectoryReader exte
     }.run(commit);
   }
 
-  synchronized DirectoryReader doOpenIfChanged(SegmentInfos infos, IndexWriter writer) throws IOException {
+  DirectoryReader doOpenIfChanged(SegmentInfos infos, IndexWriter writer) throws IOException {
     return StandardDirectoryReader.open(directory, writer, infos, getSequentialSubReaders(), termInfosIndexDivisor);
   }
 
@@ -328,7 +328,7 @@ final class StandardDirectoryReader exte
   }
 
   @Override
-  protected synchronized void doClose() throws IOException {
+  protected void doClose() throws IOException {
     IOException ioe = null;
     for (final AtomicReader r : getSequentialSubReaders()) {
       // try to close each reader, even if an exception is thrown

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/StoredFieldVisitor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/StoredFieldVisitor.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/StoredFieldVisitor.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/StoredFieldVisitor.java Fri Sep 21 17:21:34 2012
@@ -35,6 +35,12 @@ import org.apache.lucene.document.Docume
  * @lucene.experimental */
 
 public abstract class StoredFieldVisitor {
+
+  /** Sole constructor. (For invocation by subclass 
+   * constructors, typically implicit.) */
+  protected StoredFieldVisitor() {
+  }
+  
   /** Process a binary field. */
   public void binaryField(FieldInfo fieldInfo, byte[] value, int offset, int length) throws IOException {
   }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/StoredFieldsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/StoredFieldsConsumer.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/StoredFieldsConsumer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/StoredFieldsConsumer.java Fri Sep 21 17:21:34 2012
@@ -44,12 +44,12 @@ final class StoredFieldsConsumer {
   }
 
   private int numStoredFields;
-  private IndexableField[] storedFields;
+  private StorableField[] storedFields;
   private FieldInfo[] fieldInfos;
 
   public void reset() {
     numStoredFields = 0;
-    storedFields = new IndexableField[1];
+    storedFields = new StorableField[1];
     fieldInfos = new FieldInfo[1];
   }
 
@@ -126,10 +126,10 @@ final class StoredFieldsConsumer {
     assert docWriter.writer.testPoint("StoredFieldsWriter.finishDocument end");
   }
 
-  public void addField(IndexableField field, FieldInfo fieldInfo) {
+  public void addField(StorableField field, FieldInfo fieldInfo) {
     if (numStoredFields == storedFields.length) {
       int newSize = ArrayUtil.oversize(numStoredFields + 1, RamUsageEstimator.NUM_BYTES_OBJECT_REF);
-      IndexableField[] newArray = new IndexableField[newSize];
+      StorableField[] newArray = new StorableField[newSize];
       System.arraycopy(storedFields, 0, newArray, 0, numStoredFields);
       storedFields = newArray;
       

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/Term.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/Term.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/Term.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/Term.java Fri Sep 21 17:21:34 2012
@@ -132,8 +132,4 @@ public final class Term implements Compa
 
   @Override
   public final String toString() { return field + ":" + bytes.utf8ToString(); }
-
-  public Term deepCopyOf() {
-    return new Term(field, BytesRef.deepCopyOf(bytes));
-  }
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TermContext.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TermContext.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TermContext.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TermContext.java Fri Sep 21 17:21:34 2012
@@ -32,7 +32,13 @@ import org.apache.lucene.util.BytesRef;
  * @lucene.experimental
  */
 public final class TermContext {
-  public final IndexReaderContext topReaderContext; // for asserting!
+
+  /** Holds the {@link IndexReaderContext} of the top-level
+   *  {@link IndexReader}, used internally only for
+   *  asserting.
+   *
+   *  @lucene.internal */
+  public final IndexReaderContext topReaderContext;
   private final TermState[] states;
   private int docFreq;
   private long totalTermFreq;

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TermState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TermState.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TermState.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TermState.java Fri Sep 21 17:21:34 2012
@@ -27,6 +27,11 @@ package org.apache.lucene.index;
  */
 public abstract class TermState implements Cloneable {
 
+  /** Sole constructor. (For invocation by subclass 
+   *  constructors, typically implicit.) */
+  protected TermState() {
+  }
+
   /**
    * Copies the content of the given {@link TermState} to this instance
    * 

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=1388574&r1=1388573&r2=1388574&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 Fri Sep 21 17:21:34 2012
@@ -30,6 +30,11 @@ import org.apache.lucene.util.automaton.
 
 public abstract class Terms {
 
+  /** Sole constructor. (For invocation by subclass 
+   *  constructors, typically implicit.) */
+  protected Terms() {
+  }
+
   /** Returns an iterator that will step through all
    *  terms. This method will not return null.  If you have
    *  a previous TermsEnum, for example from a different
@@ -114,5 +119,6 @@ public abstract class Terms {
   /** Returns true if documents in this field store payloads. */
   public abstract boolean hasPayloads();
 
+  /** Zero-length array of {@link Terms}. */
   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=1388574&r1=1388573&r2=1388574&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 Fri Sep 21 17:21:34 2012
@@ -45,18 +45,26 @@ public abstract class TermsEnum implemen
 
   private AttributeSource atts = null;
 
+  /** Sole constructor. (For invocation by subclass 
+   *  constructors, typically implicit.) */
+  protected TermsEnum() {
+  }
+
   /** Returns the related attributes. */
   public AttributeSource attributes() {
     if (atts == null) atts = new AttributeSource();
     return atts;
   }
   
-  /** Represents returned result from {@link #seekCeil}.
-   *  If status is FOUND, then the precise term was found.
-   *  If status is NOT_FOUND, then a different term was
-   *  found.  If the status is END, the end of the iteration
-   *  was hit. */
-  public static enum SeekStatus {END, FOUND, NOT_FOUND};
+  /** Represents returned result from {@link #seekCeil}. */
+  public static enum SeekStatus {
+    /** The term was not found, and the end of iteration was hit. */
+    END,
+    /** The precise term was found. */
+    FOUND,
+    /** A different term was found after the requested term */
+    NOT_FOUND
+  };
 
   /** Attempts to seek to the exact term, returning
    *  true if the term is found.  If this returns false, the

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java Fri Sep 21 17:21:34 2012
@@ -84,8 +84,14 @@ 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;
 
+  /** Sole constructor, setting all settings to their
+   *  defaults. */
+  public TieredMergePolicy() {
+  }
+
   /** Maximum number of segments to be merged at a time
    *  during "normal" merging.  For explicit merging (eg,
    *  forceMerge or forceMergeDeletes was called), see {@link
@@ -98,7 +104,9 @@ public class TieredMergePolicy extends M
     return this;
   }
 
-  /** @see #setMaxMergeAtOnce */
+  /** Returns the current maxMergeAtOnce setting.
+   *
+   * @see #setMaxMergeAtOnce */
   public int getMaxMergeAtOnce() {
     return maxMergeAtOnce;
   }
@@ -116,7 +124,9 @@ public class TieredMergePolicy extends M
     return this;
   }
 
-  /** @see #setMaxMergeAtOnceExplicit */
+  /** Returns the current maxMergeAtOnceExplicit setting.
+   *
+   * @see #setMaxMergeAtOnceExplicit */
   public int getMaxMergeAtOnceExplicit() {
     return maxMergeAtOnceExplicit;
   }
@@ -127,11 +137,17 @@ 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;
   }
 
-  /** @see #getMaxMergedSegmentMB */
+  /** Returns the current maxMergedSegmentMB setting.
+   *
+   * @see #getMaxMergedSegmentMB */
   public double getMaxMergedSegmentMB() {
     return maxMergedSegmentBytes/1024/1024.;
   }
@@ -162,11 +178,14 @@ 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;
   }
 
-  /** @see #setFloorSegmentMB */
+  /** Returns the current floorSegmentMB.
+   *
+   *  @see #setFloorSegmentMB */
   public double getFloorSegmentMB() {
     return floorSegmentBytes/(1024*1024.);
   }
@@ -182,7 +201,9 @@ public class TieredMergePolicy extends M
     return this;
   }
 
-  /** @see #setForceMergeDeletesPctAllowed */
+  /** Returns the current forceMergeDeletesPctAllowed setting.
+   *
+   * @see #setForceMergeDeletesPctAllowed */
   public double getForceMergeDeletesPctAllowed() {
     return forceMergeDeletesPctAllowed;
   }
@@ -203,7 +224,9 @@ public class TieredMergePolicy extends M
     return this;
   }
 
-  /** @see #setSegmentsPerTier */
+  /** Returns the current segmentsPerTier setting.
+   *
+   * @see #setSegmentsPerTier */
   public double getSegmentsPerTier() {
     return segsPerTier;
   }
@@ -216,7 +239,9 @@ public class TieredMergePolicy extends M
     return this;
   }
 
-  /** @see  #setUseCompoundFile */
+  /** Returns the current useCompoundFile setting.
+   *
+   * @see  #setUseCompoundFile */
   public boolean getUseCompoundFile() {
     return useCompoundFile;
   }
@@ -234,7 +259,9 @@ public class TieredMergePolicy extends M
     return this;
   }
   
-  /** @see #setNoCFSRatio */
+  /** Returns the current noCFSRatio setting.
+   *
+   * @see #setNoCFSRatio */
   public double getNoCFSRatio() {
     return noCFSRatio;
   }
@@ -260,7 +287,13 @@ public class TieredMergePolicy extends M
   /** Holds score and explanation for a single candidate
    *  merge. */
   protected static abstract class MergeScore {
+    /** Sole constructor. (For invocation by subclass 
+     * constructors, typically implicit.) */
+    protected MergeScore() {
+    }
+    
     abstract double getScore();
+
     abstract String getExplanation();
   }
 
@@ -602,21 +635,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 +662,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 +697,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;
+  }
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TwoPhaseCommitTool.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TwoPhaseCommitTool.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TwoPhaseCommitTool.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TwoPhaseCommitTool.java Fri Sep 21 17:21:34 2012
@@ -27,6 +27,9 @@ import java.util.Map;
  * @lucene.experimental
  */
 public final class TwoPhaseCommitTool {
+  
+  /** No instance */
+  private TwoPhaseCommitTool() {}
 
   /**
    * A wrapper of a {@link TwoPhaseCommit}, which delegates all calls to the
@@ -39,6 +42,7 @@ public final class TwoPhaseCommitTool {
     private final TwoPhaseCommit tpc;
     private  final Map<String, String> commitData;
 
+    /** Sole constructor. */
     public TwoPhaseCommitWrapper(TwoPhaseCommit tpc, Map<String, String> commitData) {
       this.tpc = tpc;
       this.commitData = commitData;
@@ -70,12 +74,12 @@ public final class TwoPhaseCommitTool {
    * object fails to prepareCommit().
    */
   public static class PrepareCommitFailException extends IOException {
-    
+
+    /** Sole constructor. */
     public PrepareCommitFailException(Throwable cause, TwoPhaseCommit obj) {
       super("prepareCommit() failed on " + obj);
       initCause(cause);
     }
-    
   }
 
   /**
@@ -83,7 +87,8 @@ public final class TwoPhaseCommitTool {
    * object fails to commit().
    */
   public static class CommitFailException extends IOException {
-    
+
+    /** Sole constructor. */
     public CommitFailException(Throwable cause, TwoPhaseCommit obj) {
       super("commit() failed on " + obj);
       initCause(cause);

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/UpgradeIndexMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/UpgradeIndexMergePolicy.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/UpgradeIndexMergePolicy.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/UpgradeIndexMergePolicy.java Fri Sep 21 17:21:34 2012
@@ -50,6 +50,7 @@ import java.util.HashMap;
   */
 public class UpgradeIndexMergePolicy extends MergePolicy {
 
+  /** Wrapped {@link MergePolicy}. */
   protected final MergePolicy base;
 
   /** Wrap the given {@link MergePolicy} and intercept forceMerge requests to

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/package.html?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/package.html (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/package.html Fri Sep 21 17:21:34 2012
@@ -21,6 +21,241 @@
 </head>
 <body>
 Code to maintain and access indices.
-<!-- TODO: add a BASIC overview here, including code examples of using postings apis -->
+<!-- TODO: add IndexWriter, IndexWriterConfig, DocValues, etc etc -->
+<h2>Table Of Contents</h2>
+<p>
+    <ol>
+        <li><a href="#postings">Postings APIs</a>
+            <ul>
+                <li><a href="#fields">Fields</a></li>
+                <li><a href="#terms">Terms</a></li>
+                <li><a href="#documents">Documents</a></li>
+                <li><a href="#positions">Positions</a></li>
+            </ul>
+        </li>
+        <li><a href="#stats">Index Statistics</a>
+            <ul>
+                <li><a href="#termstats">Term-level</a></li>
+                <li><a href="#fieldstats">Field-level</a></li>
+                <li><a href="#segmentstats">Segment-level</a></li>
+                <li><a href="#documentstats">Document-level</a></li>
+            </ul>
+        </li>
+    </ol>
+</p>
+<a name="postings"></a>
+<h2>Postings APIs</h2>
+<a name="fields"></a>
+<h4>
+    Fields
+</h4>
+<p>
+{@link org.apache.lucene.index.Fields} is the initial entry point into the 
+postings APIs, this can be obtained in several ways:
+<pre class="prettyprint">
+// access indexed fields for an index segment
+Fields fields = reader.fields();
+// access term vector fields for a specified document
+Fields fields = reader.getTermVectors(docid);
+</pre>
+Fields implements Java's Iterable interface, so its easy to enumerate the
+list of fields:
+<pre class="prettyprint">
+// enumerate list of fields
+for (String field : fields) {
+  // access the terms for this field
+  Terms terms = fields.terms(field);
+}
+</pre>
+</p>
+<a name="terms"></a>
+<h4>
+    Terms
+</h4>
+<p>
+{@link org.apache.lucene.index.Terms} represents the collection of terms
+within a field, exposes some metadata and <a href="#fieldstats">statistics</a>,
+and an API for enumeration.
+<pre class="prettyprint">
+// metadata about the field
+System.out.println("positions? " + terms.hasPositions());
+System.out.println("offsets? " + terms.hasOffsets());
+System.out.println("payloads? " + terms.hasPayloads());
+// iterate through terms
+TermsEnum termsEnum = terms.iterator(null);
+BytesRef term = null;
+while ((term = termsEnum.next()) != null) {
+  doSomethingWith(termsEnum.term());
+}
+</pre>
+{@link org.apache.lucene.index.TermsEnum} provides an iterator over the list
+of terms within a field, some <a href="#termstats">statistics</a> about the term,
+and methods to access the term's <a href="#documents">documents</a> and
+<a href="#positions">positions</a>.
+<pre class="prettyprint">
+// seek to a specific term
+boolean found = termsEnum.seekExact(new BytesRef("foobar"), true);
+if (found) {
+  // get the document frequency
+  System.out.println(termsEnum.docFreq());
+  // enumerate through documents
+  DocsEnum docs = termsEnum.docs(null, null);
+  // enumerate through documents and positions
+  DocsAndPositionsEnum docsAndPositions = termsEnum.docsAndPositions(null, null);
+}
+</pre>
+</p>
+<a name="documents"></a>
+<h4>
+    Documents
+</h4>
+<p>
+{@link org.apache.lucene.index.DocsEnum} is an extension of 
+{@link org.apache.lucene.search.DocIdSetIterator}that iterates over the list of
+documents for a term, along with the term frequency within that document.
+<pre class="prettyprint">
+int docid;
+while ((docid = docsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+  System.out.println(docid);
+  System.out.println(docsEnum.freq());
+}
+</pre>
+</p>
+<a name="positions"></a>
+<h4>
+    Positions
+</h4>
+<p>
+{@link org.apache.lucene.index.DocsAndPositionsEnum} is an extension of 
+{@link org.apache.lucene.index.DocsEnum} that additionally allows iteration
+of the positions a term occurred within the document, and any additional
+per-position information (offsets and payload)
+<pre class="prettyprint">
+int docid;
+while ((docid = docsAndPositionsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+  System.out.println(docid);
+  int freq = docsAndPositionsEnum.freq();
+  for (int i = 0; i < freq; i++) {
+     System.out.println(docsAndPositionsEnum.nextPosition());
+     System.out.println(docsAndPositionsEnum.startOffset());
+     System.out.println(docsAndPositionsEnum.endOffset());
+     System.out.println(docsAndPositionsEnum.getPayload());
+  }
+}
+</pre>
+</p>
+<a name="stats"></a>
+<h2>Index Statistics</h2>
+<a name="termstats"></a>
+<h4>
+    Term statistics
+</h4>
+<p>
+    <ul>
+       <li>{@link org.apache.lucene.index.TermsEnum#docFreq}: Returns the number of 
+           documents that contain at least one occurrence of the term. This statistic 
+           is always available for an indexed term. Note that it will also count 
+           deleted documents, when segments are merged the statistic is updated as 
+           those deleted documents are merged away.
+       <li>{@link org.apache.lucene.index.TermsEnum#totalTermFreq}: Returns the number 
+           of occurrences of this term across all documents. Note that this statistic 
+           is unavailable (returns <code>-1</code>) if term frequencies were omitted 
+           from the index 
+           ({@link org.apache.lucene.index.FieldInfo.IndexOptions#DOCS_ONLY DOCS_ONLY}) 
+           for the field. Like docFreq(), it will also count occurrences that appear in 
+           deleted documents.
+    </ul>
+</p>
+<a name="fieldstats"></a>
+<h4>
+    Field statistics
+</h4>
+<p>
+    <ul>
+       <li>{@link org.apache.lucene.index.Terms#size}: Returns the number of 
+           unique terms in the field. This statistic may be unavailable 
+           (returns <code>-1</code>) for some Terms implementations such as
+           {@link org.apache.lucene.index.MultiTerms}, where it cannot be efficiently
+           computed.  Note that this count also includes terms that appear only
+           in deleted documents: when segments are merged such terms are also merged
+           away and the statistic is then updated.
+       <li>{@link org.apache.lucene.index.Terms#getDocCount}: Returns the number of
+           documents that contain at least one occurrence of any term for this field.
+           This can be thought of as a Field-level docFreq(). Like docFreq() it will
+           also count deleted documents.
+       <li>{@link org.apache.lucene.index.Terms#getSumDocFreq}: Returns the number of
+           postings (term-document mappings in the inverted index) for the field. This
+           can be thought of as the sum of {@link org.apache.lucene.index.TermsEnum#docFreq}
+           across all terms in the field, and like docFreq() it will also count postings
+           that appear in deleted documents.
+       <li>{@link org.apache.lucene.index.Terms#getSumTotalTermFreq}: Returns the number
+           of tokens for the field. This can be thought of as the sum of 
+           {@link org.apache.lucene.index.TermsEnum#totalTermFreq} across all terms in the
+           field, and like totalTermFreq() it will also count occurrences that appear in
+           deleted documents, and will be unavailable (returns <code>-1</code>) if term 
+           frequencies were omitted from the index 
+           ({@link org.apache.lucene.index.FieldInfo.IndexOptions#DOCS_ONLY DOCS_ONLY}) 
+           for the field.
+    </ul>
+</p>
+<a name="segmentstats"></a>
+<h4>
+    Segment statistics
+</h4>
+<p>
+    <ul>
+       <li>{@link org.apache.lucene.index.IndexReader#maxDoc}: Returns the number of 
+           documents (including deleted documents) in the index. 
+       <li>{@link org.apache.lucene.index.IndexReader#numDocs}: Returns the number 
+           of live documents (excluding deleted documents) in the index.
+       <li>{@link org.apache.lucene.index.IndexReader#numDeletedDocs}: Returns the
+           number of deleted documents in the index.
+       <li>{@link org.apache.lucene.index.Fields#size}: Returns the number of indexed
+           fields.
+    </ul>
+</p>
+<a name="documentstats"></a>
+<h4>
+    Document statistics
+</h4>
+<p>
+Document statistics are available during the indexing process for an indexed field: typically
+a {@link org.apache.lucene.search.similarities.Similarity} implementation will store some
+of these values (possibly in a lossy way), into the normalization value for the document in
+its {@link org.apache.lucene.search.similarities.Similarity#computeNorm} method.
+</p>
+<p>
+    <ul>
+       <li>{@link org.apache.lucene.index.FieldInvertState#getLength}: Returns the number of 
+           tokens for this field in the document. Note that this is just the number
+           of times that {@link org.apache.lucene.analysis.TokenStream#incrementToken} returned
+           true, and is unrelated to the values in 
+           {@link org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute}.
+       <li>{@link org.apache.lucene.index.FieldInvertState#getNumOverlap}: Returns the number
+           of tokens for this field in the document that had a position increment of zero. This
+           can be used to compute a document length that discounts artificial tokens
+           such as synonyms.
+       <li>{@link org.apache.lucene.index.FieldInvertState#getPosition}: Returns the accumulated
+           position value for this field in the document: computed from the values of
+           {@link org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute} and including
+           {@link org.apache.lucene.analysis.Analyzer#getPositionIncrementGap}s across multivalued
+           fields.
+       <li>{@link org.apache.lucene.index.FieldInvertState#getOffset}: Returns the total
+           character offset value for this field in the document: computed from the values of
+           {@link org.apache.lucene.analysis.tokenattributes.OffsetAttribute} returned by 
+           {@link org.apache.lucene.analysis.TokenStream#end}, and including
+           {@link org.apache.lucene.analysis.Analyzer#getOffsetGap}s across multivalued
+           fields.
+       <li>{@link org.apache.lucene.index.FieldInvertState#getUniqueTermCount}: Returns the number
+           of unique terms encountered for this field in the document.
+       <li>{@link org.apache.lucene.index.FieldInvertState#getMaxTermFrequency}: Returns the maximum
+           frequency across all unique terms encountered for this field in the document. 
+    </ul>
+</p>
+<p>
+Additional user-supplied statistics can be added to the document as DocValues fields and
+accessed via {@link org.apache.lucene.index.AtomicReader#docValues}.
+</p>
+<p>
 </body>
 </html>

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=1388574&r1=1388573&r2=1388574&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 Fri Sep 21 17:21:34 2012
@@ -155,7 +155,7 @@ public class BooleanQuery extends Query 
 
   /** Returns an iterator on the clauses in this query. It implements the {@link Iterable} interface to
    * make it possible to do:
-   * <pre>for (BooleanClause clause : booleanQuery) {}</pre>
+   * <pre class="prettyprint">for (BooleanClause clause : booleanQuery) {}</pre>
    */
   public final Iterator<BooleanClause> iterator() { return clauses().iterator(); }
 
@@ -331,7 +331,14 @@ public class BooleanQuery extends Query 
           optional.add(subScorer);
         }
       }
-      
+
+      // NOTE: we could also use BooleanScorer, if we knew
+      // this BooleanQuery was embedded in another
+      // BooleanQuery that was also using BooleanScorer (ie,
+      // BooleanScorer can nest).  But this is hard to
+      // detect and we never do so today... (ie, we only
+      // return BooleanScorer for topScorer):
+
       // Check if we can return a BooleanScorer
       if (!scoreDocsInOrder && flags == PostingFeatures.DOCS_AND_FREQS && topScorer && required.size() == 0) {
         return new BooleanScorer(this, disableCoord, minNrShouldMatch, optional, prohibited, maxCoord);
@@ -396,7 +403,7 @@ public class BooleanQuery extends Query 
   public Query rewrite(IndexReader reader) throws IOException {
     if (minNrShouldMatch == 0 && clauses.size() == 1) {                    // optimize 1-clause queries
       BooleanClause c = clauses.get(0);
-      if (!c.isProhibited()) {			  // just return clause
+      if (!c.isProhibited()) {  // just return clause
 
         Query query = c.getQuery().rewrite(reader);    // rewrite first
 
@@ -467,7 +474,7 @@ public class BooleanQuery extends Query 
 
       Query subQuery = c.getQuery();
       if (subQuery != null) {
-        if (subQuery instanceof BooleanQuery) {	  // wrap sub-bools in parens
+        if (subQuery instanceof BooleanQuery) {  // wrap sub-bools in parens
           buffer.append("(");
           buffer.append(subQuery.toString(field));
           buffer.append(")");

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java Fri Sep 21 17:21:34 2012
@@ -18,15 +18,10 @@ package org.apache.lucene.search;
  */
 
 import org.apache.lucene.index.AtomicReaderContext;
-import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.BooleanQuery.BooleanWeight;
-import org.apache.lucene.search.positions.BooleanIntervalIterator;
-import org.apache.lucene.search.positions.ConjunctionIntervalIterator;
-import org.apache.lucene.search.positions.DisjunctionIntervalIterator;
 import org.apache.lucene.search.positions.IntervalIterator;
 
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 
@@ -119,7 +114,7 @@ final class BooleanScorer extends Scorer
   // Therefore the only methods that are implemented are score() and doc().
   private static final class BucketScorer extends Scorer {
 
-    float score;
+    double score;
     int doc = NO_MORE_DOCS;
     int freq;
     
@@ -138,7 +133,7 @@ final class BooleanScorer extends Scorer
     public int nextDoc() { return NO_MORE_DOCS; }
     
     @Override
-    public float score() { return score; }
+    public float score() { return (float)score; }
 
     @Override
     public IntervalIterator positions(boolean collectPositions) throws IOException {
@@ -149,7 +144,7 @@ final class BooleanScorer extends Scorer
 
   static final class Bucket {
     int doc = -1;            // tells if bucket is valid
-    float score;             // incremental score
+    double score;             // incremental score
     // TODO: break out bool anyProhibited, int
     // numRequiredMatched; then we can remove 32 limit on
     // required clauses
@@ -264,7 +259,12 @@ final class BooleanScorer extends Scorer
           // clauses
           //&& (current.bits & requiredMask) == requiredMask) {
           
-          // TODO: can we remove this?  
+          // NOTE: Lucene always passes max =
+          // Integer.MAX_VALUE today, because we never embed
+          // a BooleanScorer inside another (even though
+          // that should work)... but in theory an outside
+          // app could pass a different max so we must check
+          // it:
           if (current.doc >= max){
             tmp = current;
             current = current.next;
@@ -333,7 +333,7 @@ final class BooleanScorer extends Scorer
 
   @Override
   public float freq() throws IOException {
-    return current.coord;
+    throw new UnsupportedOperationException();
   }
 
   @Override
@@ -355,11 +355,6 @@ final class BooleanScorer extends Scorer
   
   @Override
   public Collection<ChildScorer> getChildren() {
-    List<ChildScorer> children = new ArrayList<ChildScorer>();
-    for (SubScorer sub = scorers; sub != null; sub = sub.next) {
-      // TODO: fix this if BQ ever sends us required clauses
-      children.add(new ChildScorer(sub.scorer, sub.prohibited ? "MUST_NOT" : "SHOULD"));
-    }
-    return children;
+    throw new UnsupportedOperationException();
   }
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/CollectionStatistics.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/CollectionStatistics.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/CollectionStatistics.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/CollectionStatistics.java Fri Sep 21 17:21:34 2012
@@ -34,7 +34,7 @@ public class CollectionStatistics {
   public CollectionStatistics(String field, long maxDoc, long docCount, long sumTotalTermFreq, long sumDocFreq) {
     assert maxDoc >= 0;
     assert docCount >= -1 && docCount <= maxDoc; // #docs with field must be <= #docs
-    assert sumDocFreq >= -1;
+    assert sumDocFreq == -1 || sumDocFreq >= docCount; // #postings must be >= #docs with field
     assert sumTotalTermFreq == -1 || sumTotalTermFreq >= sumDocFreq; // #positions must be >= #postings
     this.field = field;
     this.maxDoc = maxDoc;

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=1388574&r1=1388573&r2=1388574&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 Fri Sep 21 17:21:34 2012
@@ -81,8 +81,8 @@ import org.apache.lucene.search.Weight.P
  * a simple example showing how to collect docIDs into a
  * BitSet:</p>
  * 
- * <pre>
- * Searcher searcher = new IndexSearcher(indexReader);
+ * <pre class="prettyprint">
+ * IndexSearcher searcher = new IndexSearcher(indexReader);
  * final BitSet bits = new BitSet(indexReader.maxDoc());
  * searcher.search(query, new Collector() {
  *   private int docBase;

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=1388574&r1=1388573&r2=1388574&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 Fri Sep 21 17:21:34 2012
@@ -136,6 +136,7 @@ class ConjunctionScorer extends Scorer {
   
   @Override
   public float score() throws IOException {
+    // TODO: sum into a double and cast to float if we ever send required clauses to BS1
     float sum = 0.0f;
     for (int i = 0; i < scorers.length; i++) {
       sum += scorers[i].score();

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=1388574&r1=1388573&r2=1388574&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 Fri Sep 21 17:21:34 2012
@@ -97,6 +97,7 @@ class ConjunctionTermScorer extends Scor
 
   @Override
   public float score() throws IOException {
+    // TODO: sum into a double and cast to float if we ever send required clauses to BS1
     float sum = 0.0f;
     for (DocsAndFreqs docs : docsAndFreqs) {
       sum += docs.scorer.score();

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=1388574&r1=1388573&r2=1388574&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 Fri Sep 21 17:21:34 2012
@@ -192,6 +192,7 @@ public class ConstantScoreQuery extends 
 
     @Override
     public float score() throws IOException {
+      assert docIdSetIterator.docID() != NO_MORE_DOCS;
       return theScore;
     }
 

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=1388574&r1=1388573&r2=1388574&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 Fri Sep 21 17:21:34 2012
@@ -61,7 +61,7 @@ public class DisjunctionMaxQuery extends
 
   /**
    * Creates a new DisjunctionMaxQuery
-   * @param disjuncts a Collection<Query> of all the disjuncts to add
+   * @param disjuncts a {@code Collection<Query>} of all the disjuncts to add
    * @param tieBreakerMultiplier   the weight to give to each matching non-maximum disjunct
    */
   public DisjunctionMaxQuery(Collection<Query> disjuncts, float tieBreakerMultiplier) {
@@ -77,14 +77,14 @@ public class DisjunctionMaxQuery extends
   }
 
   /** Add a collection of disjuncts to this disjunction
-   * via Iterable<Query>
+   * via {@code Iterable<Query>}
    * @param disjuncts a collection of queries to add as disjuncts.
    */
   public void add(Collection<Query> disjuncts) {
     this.disjuncts.addAll(disjuncts);
   }
 
-  /** @return An Iterator<Query> over the disjuncts */
+  /** @return An {@code Iterator<Query>} over the disjuncts */
   public Iterator<Query> iterator() {
     return disjuncts.iterator();
   }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java Fri Sep 21 17:21:34 2012
@@ -42,7 +42,6 @@ class DisjunctionSumScorer extends Disju
 
   /** Construct a <code>DisjunctionScorer</code>.
    * @param weight The weight to be used.
-   * @param needsPositions 
    * @param subScorers A collection of at least two subscorers.
    * @param minimumNrMatchers The positive minimum number of subscorers that should
    * match to match this query.
@@ -74,6 +73,7 @@ class DisjunctionSumScorer extends Disju
   @Override
 
   public int nextDoc() throws IOException {
+    assert doc != NO_MORE_DOCS;
     while(true) {
       while (subScorers[0].docID() == doc) {
         if (subScorers[0].nextDoc() != NO_MORE_DOCS) {
@@ -97,10 +97,14 @@ class DisjunctionSumScorer extends Disju
   private void afterNext() throws IOException {
     final Scorer sub = subScorers[0];
     doc = sub.docID();
-    score = sub.score();
-    nrMatchers = 1;
-    countMatches(1);
-    countMatches(2);
+    if (doc == NO_MORE_DOCS) {
+      nrMatchers = Integer.MAX_VALUE; // stop looping
+    } else {
+      score = sub.score();
+      nrMatchers = 1;
+      countMatches(1);
+      countMatches(2);
+    }
   }
   
   // TODO: this currently scores, but so did the previous impl

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DocIdSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DocIdSet.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DocIdSet.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DocIdSet.java Fri Sep 21 17:21:34 2012
@@ -72,7 +72,7 @@ public abstract class DocIdSet {
    * external disk access (as {@link Bits} interface cannot throw
    * {@link IOException}). This is generally true for bit sets
    * like {@link org.apache.lucene.util.FixedBitSet}, which return
-   * itsself if they are used as {@code DocIdSet}.
+   * itself if they are used as {@code DocIdSet}.
    */
   public Bits bits() throws IOException {
     return null;

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DocIdSetIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DocIdSetIterator.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DocIdSetIterator.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DocIdSetIterator.java Fri Sep 21 17:21:34 2012
@@ -60,15 +60,19 @@ public abstract class DocIdSetIterator {
    */
   public abstract int nextDoc() throws IOException;
 
-  /**
-   * Advances to the first beyond (see NOTE below) the current whose document
-   * number is greater than or equal to <i>target</i>. Returns the current
-   * document number or {@link #NO_MORE_DOCS} if there are no more docs in the
-   * set.
+ /**
+   * Advances to the first beyond the current whose document number is greater 
+   * than or equal to <i>target</i>, and returns the document number itself. 
+   * Exhausts the iterator and returns {@link #NO_MORE_DOCS} if <i>target</i> 
+   * is greater than the highest document number in the set.
    * <p>
-   * Behaves as if written:
+   * The behavior of this method is <b>undefined</b> when called with
+   * <code> target &le; current</code>, or after the iterator has exhausted.
+   * Both cases may result in unpredicted behavior.
+   * <p>
+   * When <code> target &gt; current</code> it behaves as if written:
    * 
-   * <pre>
+   * <pre class="prettyprint">
    * int advance(int target) {
    *   int doc;
    *   while ((doc = nextDoc()) &lt; target) {
@@ -79,18 +83,12 @@ public abstract class DocIdSetIterator {
    * 
    * Some implementations are considerably more efficient than that.
    * <p>
-   * <b>NOTE:</b> when <code> target &le; current</code> implementations may opt
-   * not to advance beyond their current {@link #docID()}.
-   * <p>
    * <b>NOTE:</b> this method may be called with {@link #NO_MORE_DOCS} for
    * efficiency by some Scorers. If your implementation cannot efficiently
    * determine that it should exhaust, it is recommended that you check for that
    * value in each call to this method.
    * <p>
-   * <b>NOTE:</b> after the iterator has exhausted you should not call this
-   * method, as it may result in unpredicted behavior.
-   * <p>
-   * 
+   *
    * @since 2.9
    */
   public abstract int advance(int target) throws IOException;

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FieldCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FieldCache.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FieldCache.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FieldCache.java Fri Sep 21 17:21:34 2012
@@ -45,6 +45,9 @@ import org.apache.lucene.util.packed.Pac
  */
 public interface FieldCache {
 
+  /**
+   * Placeholder indicating creation of this cache is currently in-progress.
+   */
   public static final class CreationPlaceholder {
     Object value;
   }
@@ -505,7 +508,7 @@ public interface FieldCache {
       // this special case is the reason that Arrays.binarySearch() isn't useful.
       if (key == null)
         return 0;
-	  
+  
       int low = 1;
       int high = numOrd()-1;
 

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=1388574&r1=1388573&r2=1388574&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 Fri Sep 21 17:21:34 2012
@@ -52,7 +52,7 @@ import org.apache.lucene.util.packed.Pac
  * @since   lucene 1.4
  */
 class FieldCacheImpl implements FieldCache {
-	
+
   private Map<Class<?>,Cache> caches;
   FieldCacheImpl() {
     init();
@@ -173,7 +173,7 @@ class FieldCacheImpl implements FieldCac
         ((AtomicReader)key).addReaderClosedListener(purgeReader); 
       } else {
         // last chance
-        reader.addReaderClosedListener(purgeReader); 				
+        reader.addReaderClosedListener(purgeReader);
       }
     }
   }
@@ -286,7 +286,7 @@ class FieldCacheImpl implements FieldCac
 
   /** Expert: Every composite-key in the internal cache is of this type. */
   static class Entry {
-    final String field;        // which Fieldable
+    final String field;        // which Field
     final Object custom;       // which custom comparator or parser
 
     /** Creates one of these objects for a custom comparator/parser. */
@@ -1110,14 +1110,8 @@ class FieldCacheImpl implements FieldCac
         // Try for coarse estimate for number of bits; this
         // should be an underestimate most of the time, which
         // is fine -- GrowableWriter will reallocate as needed
-        long numUniqueTerms = 0;
-        try {
-          numUniqueTerms = terms.size();
-        } catch (UnsupportedOperationException uoe) {
-          numUniqueTerms = -1;
-        }
-        if (numUniqueTerms != -1) {
-
+        long numUniqueTerms = terms.size();
+        if (numUniqueTerms != -1L) {
           if (numUniqueTerms > termCountHardLimit) {
             // app is misusing the API (there is more than
             // one term per doc); in this case we make best
@@ -1248,13 +1242,8 @@ class FieldCacheImpl implements FieldCac
         // Try for coarse estimate for number of bits; this
         // should be an underestimate most of the time, which
         // is fine -- GrowableWriter will reallocate as needed
-        long numUniqueTerms = 0;
-        try {
-          numUniqueTerms = terms.size();
-        } catch (UnsupportedOperationException uoe) {
-          numUniqueTerms = -1;
-        }
-        if (numUniqueTerms != -1) {
+        long numUniqueTerms = terms.size();
+        if (numUniqueTerms != -1L) {
           if (numUniqueTerms > termCountHardLimit) {
             numUniqueTerms = termCountHardLimit;
           }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java Fri Sep 21 17:21:34 2012
@@ -194,6 +194,9 @@ public abstract class FieldComparator<T>
    *  than the provided value. */
   public abstract int compareDocToValue(int doc, T value) throws IOException;
 
+  /**
+   * Base FieldComparator class for numeric types
+   */
   public static abstract class NumericComparator<T extends Number> extends FieldComparator<T> {
     protected final T missingValue;
     protected final String field;

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FieldValueHitQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FieldValueHitQueue.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FieldValueHitQueue.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FieldValueHitQueue.java Fri Sep 21 17:21:34 2012
@@ -33,6 +33,10 @@ import org.apache.lucene.util.PriorityQu
  */
 public abstract class FieldValueHitQueue<T extends FieldValueHitQueue.Entry> extends PriorityQueue<T> {
 
+  /**
+   * Extension of ScoreDoc to also store the 
+   * {@link FieldComparator} slot.
+   */
   public static class Entry extends ScoreDoc {
     public int slot;
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FilteredDocIdSetIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FilteredDocIdSetIterator.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FilteredDocIdSetIterator.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FilteredDocIdSetIterator.java Fri Sep 21 17:21:34 2012
@@ -28,7 +28,7 @@ import java.io.IOException;
 public abstract class FilteredDocIdSetIterator extends DocIdSetIterator {
   protected DocIdSetIterator _innerIter;
   private int doc;
-	
+
   /**
    * Constructor.
    * @param innerIter Underlying DocIdSetIterator.
@@ -40,7 +40,7 @@ public abstract class FilteredDocIdSetIt
     _innerIter = innerIter;
     doc = -1;
   }
-	
+
   /**
    * Validation method to determine whether a docid should be in the result set.
    * @param doc docid to be tested
@@ -48,7 +48,7 @@ public abstract class FilteredDocIdSetIt
    * @see #FilteredDocIdSetIterator(DocIdSetIterator)
    */
   protected abstract boolean match(int doc);
-	
+
   @Override
   public int docID() {
     return doc;

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=1388574&r1=1388573&r2=1388574&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 Fri Sep 21 17:21:34 2012
@@ -20,6 +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.PostingFeatures;
 import org.apache.lucene.search.positions.IntervalIterator;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.ToStringUtils;
@@ -43,6 +44,7 @@ public class FilteredQuery extends Query
 
   private final Query query;
   private final Filter filter;
+  private final FilterStrategy strategy;
 
   /**
    * Constructs a new query which applies a filter to the results of the original query.
@@ -51,28 +53,28 @@ public class FilteredQuery extends Query
    * @param filter Filter to apply to query results, cannot be <code>null</code>.
    */
   public FilteredQuery (Query query, Filter filter) {
-    if (query == null || filter == null)
-      throw new IllegalArgumentException("Query and filter cannot be null.");
-    this.query = query;
-    this.filter = filter;
+    this(query, filter, RANDOM_ACCESS_FILTER_STRATEGY);
   }
   
   /**
-   * Expert: decides if a filter should be executed as "random-access" or not.
-   * random-access means the filter "filters" in a similar way as deleted docs are filtered
-   * in lucene. This is faster when the filter accepts many documents.
-   * However, when the filter is very sparse, it can be faster to execute the query+filter
-   * as a conjunction in some cases.
-   * 
-   * The default implementation returns true if the first document accepted by the
-   * filter is < 100.
+   * Expert: Constructs a new query which applies a filter to the results of the original query.
+   * {@link Filter#getDocIdSet} will be called every time this query is used in a search.
+   * @param query  Query to be filtered, cannot be <code>null</code>.
+   * @param filter Filter to apply to query results, cannot be <code>null</code>.
+   * @param strategy a filter strategy used to create a filtered scorer. 
    * 
-   * @lucene.internal
+   * @see FilterStrategy
    */
-  protected boolean useRandomAccess(Bits bits, int firstFilterDoc) {
-    return firstFilterDoc < 100;
+  public FilteredQuery (Query query, Filter filter, FilterStrategy strategy) {
+    if (query == null || filter == null)
+      throw new IllegalArgumentException("Query and filter cannot be null.");
+    if (strategy == null)
+      throw new IllegalArgumentException("FilterStrategy can not be null");
+    this.strategy = strategy;
+    this.query = query;
+    this.filter = filter;
   }
-
+  
   /**
    * Returns a Weight that applies the filter to the enclosed query's Weight.
    * This is accomplished by overriding the Scorer returned by the Weight.
@@ -84,9 +86,7 @@ public class FilteredQuery extends Query
       
       @Override
       public boolean scoresDocsOutOfOrder() {
-        // TODO: Support out-of-order scoring!
-        // For now we return false here, as we always get the scorer in order
-        return false;
+        return true;
       }
 
       @Override
@@ -132,117 +132,213 @@ public class FilteredQuery extends Query
           // this means the filter does not accept any documents.
           return null;
         }
+        return strategy.filteredScorer(context, scoreDocsInOrder, topScorer, weight, filterDocIdSet, flags);
         
-        final DocIdSetIterator filterIter = filterDocIdSet.iterator();
-        if (filterIter == null) {
-          // this means the filter does not accept any documents.
-          return null;
+      }
+    };
+  }
+  
+  /**
+   * A scorer that consults the filter iff a document was matched by the
+   * delegate scorer. This is useful if the filter computation is more expensive
+   * than document scoring or if the filter has a linear running time to compute
+   * the next matching doc like exact geo distances.
+   */
+  private static final class QueryFirstScorer extends Scorer {
+    private final Scorer scorer;
+    private int scorerDoc = -1;
+    private Bits filterbits;
+
+    protected QueryFirstScorer(Weight weight, Bits filterBits, Scorer other) {
+      super(weight);
+      this.scorer = other;
+      this.filterbits = filterBits;
+    }
+    
+    // optimization: we are topScorer and collect directly
+    @Override
+    public void score(Collector collector) throws IOException {
+      // the normalization trick already applies the boost of this query,
+      // so we can use the wrapped scorer directly:
+      collector.setScorer(scorer);
+      for (;;) {
+        final int scorerDoc = scorer.nextDoc();
+        if (scorerDoc == DocIdSetIterator.NO_MORE_DOCS) {
+          break;
         }
-
-        final int firstFilterDoc = filterIter.nextDoc();
-        if (firstFilterDoc == DocIdSetIterator.NO_MORE_DOCS) {
-          return null;
+        if (filterbits.get(scorerDoc)) {
+          collector.collect(scorerDoc);
         }
-        
-        final Bits filterAcceptDocs = filterDocIdSet.bits();
-        final boolean useRandomAccess = (filterAcceptDocs != null && FilteredQuery.this.useRandomAccess(filterAcceptDocs, firstFilterDoc));
+      }
+    }
+    
+    @Override
+    public int nextDoc() throws IOException {
+      int doc;
+      for(;;) {
+        doc = scorer.nextDoc();
+        if (doc == Scorer.NO_MORE_DOCS || filterbits.get(doc)) {
+          return scorerDoc = doc;
+        }
+      } 
+    }
+    
+    @Override
+    public int advance(int target) throws IOException {
+      
+      int doc = scorer.advance(target);
+      if (doc != Scorer.NO_MORE_DOCS && !filterbits.get(doc)) {
+        return scorerDoc = nextDoc();
+      } else {
+        return scorerDoc = doc;
+      }
+      
+    }
+    @Override
+    public int docID() {
+      return scorerDoc;
+    }
+    
+    @Override
+    public float score() throws IOException {
+      return scorer.score();
+    }
+    
+    @Override
+    public float freq() throws IOException { return scorer.freq(); }
+    
+    @Override
+    public Collection<ChildScorer> getChildren() {
+      return Collections.singleton(new ChildScorer(scorer, "FILTERED"));
+    }
 
-        if (useRandomAccess) {
-          // if we are using random access, we return the inner scorer, just with other acceptDocs
-          // TODO, replace this by when BooleanWeight is fixed to be consistent with its scorer implementations:
-          // return weight.scorer(context, scoreDocsInOrder, topScorer, filterAcceptDocs);
-          return weight.scorer(context, true, topScorer, flags, filterAcceptDocs);
+    @Override
+    public IntervalIterator positions(boolean collectPositions)
+        throws IOException {
+      return scorer.positions(collectPositions);
+    }
+  }
+  
+  /**
+   * A Scorer that uses a "leap-frog" approach (also called "zig-zag join"). The scorer and the filter
+   * take turns trying to advance to each other's next matching document, often
+   * jumping past the target document. When both land on the same document, it's
+   * collected.
+   */
+  private static class LeapFrogScorer extends Scorer {
+    private final DocIdSetIterator secondary;
+    private final DocIdSetIterator primary;
+    private final Scorer scorer;
+    protected int primaryDoc = -1;
+    protected int secondaryDoc = -1;
+
+    protected LeapFrogScorer(Weight weight, DocIdSetIterator primary, DocIdSetIterator secondary, Scorer scorer) {
+      super(weight);
+      this.primary = primary;
+      this.secondary = secondary;
+      this.scorer = scorer;
+    }
+    
+    // optimization: we are topScorer and collect directly using short-circuited algo
+    @Override
+    public final void score(Collector collector) throws IOException {
+      int primDoc = primaryNext();
+      int secDoc = secondary.advance(primDoc);
+      // the normalization trick already applies the boost of this query,
+      // so we can use the wrapped scorer directly:
+      collector.setScorer(scorer);
+      for (;;) {
+        if (primDoc == secDoc) {
+          // Check if scorer has exhausted, only before collecting.
+          if (primDoc == DocIdSetIterator.NO_MORE_DOCS) {
+            break;
+          }
+          collector.collect(primDoc);
+          primDoc = primary.nextDoc();
+          secDoc = secondary.advance(primDoc);
+        } else if (secDoc > primDoc) {
+          primDoc = primary.advance(secDoc);
         } else {
-          assert firstFilterDoc > -1;
-          // we are gonna advance() this scorer, so we set inorder=true/toplevel=false
-          // we pass null as acceptDocs, as our filter has already respected acceptDocs, no need to do twice
-          final Scorer scorer = weight.scorer(context, true, false, flags, null);
-          return (scorer == null) ? null : new Scorer(this) {
-            private int scorerDoc = -1, filterDoc = firstFilterDoc;
-            
-            // optimization: we are topScorer and collect directly using short-circuited algo
-            @Override
-            public void score(Collector collector) throws IOException {
-              int filterDoc = firstFilterDoc;
-              int scorerDoc = scorer.advance(filterDoc);
-              // the normalization trick already applies the boost of this query,
-              // so we can use the wrapped scorer directly:
-              collector.setScorer(scorer);
-              for (;;) {
-                if (scorerDoc == filterDoc) {
-                  // Check if scorer has exhausted, only before collecting.
-                  if (scorerDoc == DocIdSetIterator.NO_MORE_DOCS) {
-                    break;
-                  }
-                  collector.collect(scorerDoc);
-                  filterDoc = filterIter.nextDoc();
-                  scorerDoc = scorer.advance(filterDoc);
-                } else if (scorerDoc > filterDoc) {
-                  filterDoc = filterIter.advance(scorerDoc);
-                } else {
-                  scorerDoc = scorer.advance(filterDoc);
-                }
-              }
-            }
-            
-            private int advanceToNextCommonDoc() throws IOException {
-              for (;;) {
-                if (scorerDoc < filterDoc) {
-                  scorerDoc = scorer.advance(filterDoc);
-                } else if (scorerDoc == filterDoc) {
-                  return scorerDoc;
-                } else {
-                  filterDoc = filterIter.advance(scorerDoc);
-                }
-              }
-            }
-
-            @Override
-            public int nextDoc() throws IOException {
-              // don't go to next doc on first call
-              // (because filterIter is already on first doc):
-              if (scorerDoc != -1) {
-                filterDoc = filterIter.nextDoc();
-              }
-              return advanceToNextCommonDoc();
-            }
-            
-            @Override
-            public int advance(int target) throws IOException {
-              if (target > filterDoc) {
-                filterDoc = filterIter.advance(target);
-              }
-              return advanceToNextCommonDoc();
-            }
-
-            @Override
-            public int docID() {
-              return scorerDoc;
-            }
-            
-            @Override
-            public float score() throws IOException {
-              return scorer.score();
-            }
-
-            @Override
-            public IntervalIterator positions(boolean collectPositions) throws IOException {
-               return scorer.positions(collectPositions);
-            }
-            
-            @Override
-            public float freq() throws IOException { return scorer.freq(); }
-            
-            @Override
-            public Collection<ChildScorer> getChildren() {
-              return Collections.singleton(new ChildScorer(scorer, "FILTERED"));
-            }
-          };
+          secDoc = secondary.advance(primDoc);
         }
       }
-    };
+    }
+    
+    private final int advanceToNextCommonDoc() throws IOException {
+      for (;;) {
+        if (secondaryDoc < primaryDoc) {
+          secondaryDoc = secondary.advance(primaryDoc);
+        } else if (secondaryDoc == primaryDoc) {
+          return primaryDoc;
+        } else {
+          primaryDoc = primary.advance(secondaryDoc);
+        }
+      }
+    }
+
+    @Override
+    public final int nextDoc() throws IOException {
+      primaryDoc = primaryNext();
+      return advanceToNextCommonDoc();
+    }
+    
+    protected int primaryNext() throws IOException {
+      return primary.nextDoc();
+    }
+    
+    @Override
+    public final int advance(int target) throws IOException {
+      if (target > primaryDoc) {
+        primaryDoc = primary.advance(target);
+      }
+      return advanceToNextCommonDoc();
+    }
+
+    @Override
+    public final int docID() {
+      return secondaryDoc;
+    }
+    
+    @Override
+    public final float score() throws IOException {
+      return scorer.score();
+    }
+    
+    @Override
+    public final float freq() throws IOException { return scorer.freq(); }
+    
+    @Override
+    public final Collection<ChildScorer> getChildren() {
+      return Collections.singleton(new ChildScorer(scorer, "FILTERED"));
+    }
+
+    @Override
+    public IntervalIterator positions(boolean collectPositions)
+        throws IOException {
+      return scorer.positions(collectPositions);
+    }
   }
+  
+  // TODO once we have way to figure out if we use RA or LeapFrog we can remove this scorer
+  private static final class PrimaryAdvancedLeapFrogScorer extends LeapFrogScorer {
+    private final int firstFilteredDoc;
+
+    protected PrimaryAdvancedLeapFrogScorer(Weight weight, int firstFilteredDoc, DocIdSetIterator filterIter, Scorer other) {
+      super(weight, filterIter, other, other);
+      this.firstFilteredDoc = firstFilteredDoc;
+      this.primaryDoc = firstFilteredDoc; // initialize to prevent and advance call to move it further
+    }
 
+    @Override
+    protected int primaryNext() throws IOException {
+      if (secondaryDoc != -1) {
+        return super.primaryNext();
+      } else {
+        return firstFilteredDoc;
+      }
+    }
+  }
+  
   /** Rewrites the query. If the wrapped is an instance of
    * {@link MatchAllDocsQuery} it returns a {@link ConstantScoreQuery}. Otherwise
    * it returns a new {@code FilteredQuery} wrapping the rewritten query. */
@@ -270,10 +366,12 @@ public class FilteredQuery extends Query
     }
   }
 
+  /** Returns this FilteredQuery's (unfiltered) Query */
   public final Query getQuery() {
     return query;
   }
 
+  /** Returns this FilteredQuery's filter */
   public final Filter getFilter() {
     return filter;
   }
@@ -305,15 +403,215 @@ public class FilteredQuery extends Query
       return false;
     assert o instanceof FilteredQuery;
     final FilteredQuery fq = (FilteredQuery) o;
-    return fq.query.equals(this.query) && fq.filter.equals(this.filter);
+    return fq.query.equals(this.query) && fq.filter.equals(this.filter) && fq.strategy.equals(this.strategy);
   }
 
   /** Returns a hash code value for this object. */
   @Override
   public int hashCode() {
     int hash = super.hashCode();
+    hash = hash * 31 + strategy.hashCode();
     hash = hash * 31 + query.hashCode();
     hash = hash * 31 + filter.hashCode();
     return hash;
   }
+  
+  /**
+   * A {@link FilterStrategy} that conditionally uses a random access filter if
+   * the given {@link DocIdSet} supports random access (returns a non-null value
+   * from {@link DocIdSet#bits()}) and
+   * {@link RandomAccessFilterStrategy#useRandomAccess(Bits, int)} returns
+   * <code>true</code>. Otherwise this strategy falls back to a "zig-zag join" (
+   * {@link FilteredQuery#LEAP_FROG_FILTER_FIRST_STRATEGY}) strategy.
+   * 
+   * <p>
+   * Note: this strategy is the default strategy in {@link FilteredQuery}
+   * </p>
+   */
+  public static final FilterStrategy RANDOM_ACCESS_FILTER_STRATEGY = new RandomAccessFilterStrategy();
+  
+  /**
+   * A filter strategy that uses a "leap-frog" approach (also called "zig-zag join"). 
+   * The scorer and the filter
+   * take turns trying to advance to each other's next matching document, often
+   * jumping past the target document. When both land on the same document, it's
+   * collected.
+   * <p>
+   * Note: This strategy uses the filter to lead the iteration.
+   * </p> 
+   */
+  public static final FilterStrategy LEAP_FROG_FILTER_FIRST_STRATEGY = new LeapFrogFilterStrategy(false);
+  
+  /**
+   * A filter strategy that uses a "leap-frog" approach (also called "zig-zag join"). 
+   * The scorer and the filter
+   * take turns trying to advance to each other's next matching document, often
+   * jumping past the target document. When both land on the same document, it's
+   * collected.
+   * <p>
+   * Note: This strategy uses the query to lead the iteration.
+   * </p> 
+   */
+  public static final FilterStrategy LEAP_FROG_QUERY_FIRST_STRATEGY = new LeapFrogFilterStrategy(true);
+  
+  /**
+   * A filter strategy that advances the Query or rather its {@link Scorer} first and consults the
+   * filter {@link DocIdSet} for each matched document.
+   * <p>
+   * Note: this strategy requires a {@link DocIdSet#bits()} to return a non-null value. Otherwise
+   * this strategy falls back to {@link FilteredQuery#LEAP_FROG_QUERY_FIRST_STRATEGY}
+   * </p>
+   * <p>
+   * Use this strategy if the filter computation is more expensive than document
+   * scoring or if the filter has a linear running time to compute the next
+   * matching doc like exact geo distances.
+   * </p>
+   */
+  public static final FilterStrategy QUERY_FIRST_FILTER_STRATEGY = new QueryFirstFilterStrategy();
+  
+  /** Abstract class that defines how the filter ({@link DocIdSet}) applied during document collection. */
+  public static abstract class FilterStrategy {
+    
+    /**
+     * Returns a filtered {@link Scorer} based on this strategy.
+     * 
+     * @param context
+     *          the {@link AtomicReaderContext} for which to return the {@link Scorer}.
+     * @param scoreDocsInOrder
+     *          specifies whether in-order scoring of documents is required. Note
+     *          that if set to false (i.e., out-of-order scoring is required),
+     *          this method can return whatever scoring mode it supports, as every
+     *          in-order scorer is also an out-of-order one. However, an
+     *          out-of-order scorer may not support {@link Scorer#nextDoc()}
+     *          and/or {@link Scorer#advance(int)}, therefore it is recommended to
+     *          request an in-order scorer if use of these methods is required.
+     * @param topScorer
+     *          if true, {@link Scorer#score(Collector)} will be called; if false,
+     *          {@link Scorer#nextDoc()} and/or {@link Scorer#advance(int)} will
+     *          be called.
+     * @param weight the {@link FilteredQuery} {@link Weight} to create the filtered scorer.
+     * @param docIdSet the filter {@link DocIdSet} to apply
+     * //nocommit add javadoc 
+     * @return a filtered scorer
+     * 
+     * @throws IOException if an {@link IOException} occurs
+     */
+    public abstract Scorer filteredScorer(AtomicReaderContext context,
+        boolean scoreDocsInOrder, boolean topScorer, Weight weight,
+        DocIdSet docIdSet, PostingFeatures flags) throws IOException;
+  }
+  
+  /**
+   * A {@link FilterStrategy} that conditionally uses a random access filter if
+   * the given {@link DocIdSet} supports random access (returns a non-null value
+   * from {@link DocIdSet#bits()}) and
+   * {@link RandomAccessFilterStrategy#useRandomAccess(Bits, int)} returns
+   * <code>true</code>. Otherwise this strategy falls back to a "zig-zag join" (
+   * {@link FilteredQuery#LEAP_FROG_FILTER_FIRST_STRATEGY}) strategy .
+   */
+  public static class RandomAccessFilterStrategy extends FilterStrategy {
+
+    @Override
+    public Scorer filteredScorer(AtomicReaderContext context, boolean scoreDocsInOrder, boolean topScorer, Weight weight, DocIdSet docIdSet, PostingFeatures flags) throws IOException {
+      final DocIdSetIterator filterIter = docIdSet.iterator();
+      if (filterIter == null) {
+        // this means the filter does not accept any documents.
+        return null;
+      }  
+
+      final int firstFilterDoc = filterIter.nextDoc();
+      if (firstFilterDoc == DocIdSetIterator.NO_MORE_DOCS) {
+        return null;
+      }
+      
+      final Bits filterAcceptDocs = docIdSet.bits();
+        // force if RA is requested
+      final boolean useRandomAccess = (filterAcceptDocs != null && (useRandomAccess(filterAcceptDocs, firstFilterDoc)));
+      if (useRandomAccess) {
+        // if we are using random access, we return the inner scorer, just with other acceptDocs
+        return weight.scorer(context, scoreDocsInOrder, topScorer, flags, filterAcceptDocs);
+      } else {
+        assert firstFilterDoc > -1;
+        // we are gonna advance() this scorer, so we set inorder=true/toplevel=false
+        // we pass null as acceptDocs, as our filter has already respected acceptDocs, no need to do twice
+        final Scorer scorer = weight.scorer(context, true, false, flags, null);
+        // TODO once we have way to figure out if we use RA or LeapFrog we can remove this scorer
+        return (scorer == null) ? null : new PrimaryAdvancedLeapFrogScorer(weight, firstFilterDoc, filterIter, scorer);
+      }
+    }
+    
+    /**
+     * Expert: decides if a filter should be executed as "random-access" or not.
+     * random-access means the filter "filters" in a similar way as deleted docs are filtered
+     * in Lucene. This is faster when the filter accepts many documents.
+     * However, when the filter is very sparse, it can be faster to execute the query+filter
+     * as a conjunction in some cases.
+     * 
+     * The default implementation returns <code>true</code> if the first document accepted by the
+     * filter is < 100.
+     * 
+     * @lucene.internal
+     */
+    protected boolean useRandomAccess(Bits bits, int firstFilterDoc) {
+      //TODO once we have a cost API on filters and scorers we should rethink this heuristic
+      return firstFilterDoc < 100;
+    }
+  }
+  
+  private static final class LeapFrogFilterStrategy extends FilterStrategy {
+    
+    private final boolean scorerFirst;
+    
+    private LeapFrogFilterStrategy(boolean scorerFirst) {
+      this.scorerFirst = scorerFirst;
+    }
+    @Override
+    public Scorer filteredScorer(AtomicReaderContext context,
+        boolean scoreDocsInOrder, boolean topScorer, Weight weight,
+        DocIdSet docIdSet, PostingFeatures flags) throws IOException {
+      final DocIdSetIterator filterIter = docIdSet.iterator();
+      if (filterIter == null) {
+        // this means the filter does not accept any documents.
+        return null;
+      }
+      // we are gonna advance() this scorer, so we set inorder=true/toplevel=false
+      // we pass null as acceptDocs, as our filter has already respected acceptDocs, no need to do twice
+      final Scorer scorer = weight.scorer(context, true, false, flags, null);
+      if (scorerFirst) {
+        return (scorer == null) ? null : new LeapFrogScorer(weight, scorer, filterIter, scorer);  
+      } else {
+        return (scorer == null) ? null : new LeapFrogScorer(weight, filterIter, scorer, scorer);  
+      }
+    }
+    
+  }
+  
+  /**
+   * A filter strategy that advances the {@link Scorer} first and consults the
+   * {@link DocIdSet} for each matched document.
+   * <p>
+   * Note: this strategy requires a {@link DocIdSet#bits()} to return a non-null value. Otherwise
+   * this strategy falls back to {@link FilteredQuery#LEAP_FROG_QUERY_FIRST_STRATEGY}
+   * </p>
+   * <p>
+   * Use this strategy if the filter computation is more expensive than document
+   * scoring or if the filter has a linear running time to compute the next
+   * matching doc like exact geo distances.
+   * </p>
+   */
+  private static final class QueryFirstFilterStrategy extends FilterStrategy {
+    @Override
+    public Scorer filteredScorer(final AtomicReaderContext context,
+        boolean scoreDocsInOrder, boolean topScorer, Weight weight, 
+        DocIdSet docIdSet, PostingFeatures flags) throws IOException {
+      Bits filterAcceptDocs = docIdSet.bits();
+      if (filterAcceptDocs == null) {
+        return LEAP_FROG_QUERY_FIRST_STRATEGY.filteredScorer(context, scoreDocsInOrder, topScorer, weight, docIdSet, flags);
+      }
+      final Scorer scorer = weight.scorer(context, true, false, flags, null);
+      return scorer == null ? null : new QueryFirstScorer(weight,
+          filterAcceptDocs, scorer);
+    }
+  }
+  
 }