You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ja...@apache.org on 2013/05/30 09:53:46 UTC

svn commit: r1487777 [18/50] - in /lucene/dev/branches/security: ./ dev-tools/ dev-tools/eclipse/dot.settings/ dev-tools/idea/.idea/ dev-tools/idea/.idea/libraries/ dev-tools/idea/lucene/replicator/ dev-tools/maven/ dev-tools/maven/lucene/ dev-tools/ma...

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/CachingWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/CachingWrapperFilter.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/CachingWrapperFilter.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/CachingWrapperFilter.java Thu May 30 07:53:18 2013
@@ -18,15 +18,17 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collections;
+import java.util.List;
 import java.util.Map;
 import java.util.WeakHashMap;
 
 import org.apache.lucene.index.AtomicReader;
 import org.apache.lucene.index.AtomicReaderContext;
-import org.apache.lucene.index.DirectoryReader; // javadocs
-import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.RamUsageEstimator;
 
 /**
  * Wraps another {@link Filter}'s result and caches it.  The purpose is to allow
@@ -47,25 +49,28 @@ public class CachingWrapperFilter extend
     this.filter = filter;
   }
 
-  /** Provide the DocIdSet to be cached, using the DocIdSet provided
-   *  by the wrapped Filter.
-   *  <p>This implementation returns the given {@link DocIdSet}, if {@link DocIdSet#isCacheable}
-   *  returns <code>true</code>, else it copies the {@link DocIdSetIterator} into
-   *  a {@link FixedBitSet}.
+  /** 
+   *  Provide the DocIdSet to be cached, using the DocIdSet provided
+   *  by the wrapped Filter. <p>This implementation returns the given {@link DocIdSet},
+   *  if {@link DocIdSet#isCacheable} returns <code>true</code>, else it copies the 
+   *  {@link DocIdSetIterator} into a {@link FixedBitSet}.
+   *  <p>Note: This method returns {@linkplain #EMPTY_DOCIDSET} if the given docIdSet
+   *  is <code>null</code> or if {@link DocIdSet#iterator()} return <code>null</code>. The empty
+   *  instance is use as a placeholder in the cache instead of the <code>null</code> value.
    */
   protected DocIdSet docIdSetToCache(DocIdSet docIdSet, AtomicReader reader) throws IOException {
     if (docIdSet == null) {
       // this is better than returning null, as the nonnull result can be cached
-      return DocIdSet.EMPTY_DOCIDSET;
+      return EMPTY_DOCIDSET;
     } else if (docIdSet.isCacheable()) {
       return docIdSet;
     } else {
       final DocIdSetIterator it = docIdSet.iterator();
       // null is allowed to be returned by iterator(),
-      // in this case we wrap with the empty set,
+      // in this case we wrap with the sentinel set,
       // which is cacheable.
       if (it == null) {
-        return DocIdSet.EMPTY_DOCIDSET;
+        return EMPTY_DOCIDSET;
       } else {
         final FixedBitSet bits = new FixedBitSet(reader.maxDoc());
         bits.or(it);
@@ -91,9 +96,9 @@ public class CachingWrapperFilter extend
       cache.put(key, docIdSet);
     }
 
-    return BitsFilteredDocIdSet.wrap(docIdSet, acceptDocs);
+    return docIdSet == EMPTY_DOCIDSET ? null : BitsFilteredDocIdSet.wrap(docIdSet, acceptDocs);
   }
-
+  
   @Override
   public String toString() {
     return "CachingWrapperFilter("+filter+")";
@@ -110,4 +115,41 @@ public class CachingWrapperFilter extend
   public int hashCode() {
     return (filter.hashCode() ^ 0x1117BF25);
   }
+  
+  /** An empty {@code DocIdSet} instance */
+  protected static final DocIdSet EMPTY_DOCIDSET = new DocIdSet() {
+    
+    @Override
+    public DocIdSetIterator iterator() {
+      return DocIdSetIterator.empty();
+    }
+    
+    @Override
+    public boolean isCacheable() {
+      return true;
+    }
+    
+    // we explicitly provide no random access, as this filter is 100% sparse and iterator exits faster
+    @Override
+    public Bits bits() {
+      return null;
+    }
+  };
+
+  /** Returns total byte size used by cached filters. */
+  public long sizeInBytes() {
+
+    // Sync only to pull the current set of values:
+    List<DocIdSet> docIdSets;
+    synchronized(cache) {
+      docIdSets = new ArrayList<DocIdSet>(cache.values());
+    }
+
+    long total = 0;
+    for(DocIdSet dis : docIdSets) {
+      total += RamUsageEstimator.sizeOf(dis);
+    }
+
+    return total;
+  }
 }

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/Collector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/Collector.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/Collector.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/Collector.java Thu May 30 07:53:18 2013
@@ -134,7 +134,10 @@ public abstract class Collector {
   /**
    * Called once for every document matching a query, with the unbased document
    * number.
-   * 
+   * <p>Note: The collection of the current segment can be terminated by throwing
+   * a {@link CollectionTerminatedException}. In this case, the last docs of the
+   * current {@link AtomicReaderContext} will be skipped and {@link IndexSearcher}
+   * will swallow the exception and continue collection with the next leaf.
    * <p>
    * Note: This is called in an inner search loop. For good search performance,
    * implementations of this method should not call {@link IndexSearcher#doc(int)} or

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java Thu May 30 07:53:18 2013
@@ -206,6 +206,11 @@ public class ConstantScoreQuery extends 
       return docIdSetIterator.advance(target);
     }
     
+    @Override
+    public long cost() {
+      return docIdSetIterator.cost();
+    }
+
     private Collector wrapCollector(final Collector collector) {
       return new Collector() {
         @Override

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java Thu May 30 07:53:18 2013
@@ -28,6 +28,7 @@ class DisjunctionMaxScorer extends Disju
   /* Multiplier applied to non-maximum-scoring subqueries for a document as they are summed into the result. */
   private final float tieBreakerMultiplier;
   private int doc = -1;
+  private int freq = -1;
 
   /* Used when scoring currently matching doc. */
   private float scoreSum;
@@ -55,8 +56,8 @@ class DisjunctionMaxScorer extends Disju
 
   @Override
   public int nextDoc() throws IOException {
-    if (numScorers == 0) return doc = NO_MORE_DOCS;
-    while (subScorers[0].docID() == doc) {
+    assert doc != NO_MORE_DOCS;
+    while(true) {
       if (subScorers[0].nextDoc() != NO_MORE_DOCS) {
         heapAdjust(0);
       } else {
@@ -65,9 +66,11 @@ class DisjunctionMaxScorer extends Disju
           return doc = NO_MORE_DOCS;
         }
       }
+      if (subScorers[0].docID() != doc) {
+        afterNext();
+        return doc;
+      }
     }
-    
-    return doc = subScorers[0].docID();
   }
 
   @Override
@@ -80,47 +83,40 @@ class DisjunctionMaxScorer extends Disju
    */
   @Override
   public float score() throws IOException {
-    int doc = subScorers[0].docID();
-    scoreSum = scoreMax = subScorers[0].score();
-    int size = numScorers;
-    scoreAll(1, size, doc);
-    scoreAll(2, size, doc);
     return scoreMax + (scoreSum - scoreMax) * tieBreakerMultiplier;
   }
+  
+  private void afterNext() throws IOException {
+    doc = subScorers[0].docID();
+    if (doc != NO_MORE_DOCS) {
+      scoreSum = scoreMax = subScorers[0].score();
+      freq = 1;
+      scoreAll(1);
+      scoreAll(2);
+    }
+  }
 
   // Recursively iterate all subScorers that generated last doc computing sum and max
-  private void scoreAll(int root, int size, int doc) throws IOException {
-    if (root < size && subScorers[root].docID() == doc) {
+  private void scoreAll(int root) throws IOException {
+    if (root < numScorers && subScorers[root].docID() == doc) {
       float sub = subScorers[root].score();
+      freq++;
       scoreSum += sub;
       scoreMax = Math.max(scoreMax, sub);
-      scoreAll((root<<1)+1, size, doc);
-      scoreAll((root<<1)+2, size, doc);
+      scoreAll((root<<1)+1);
+      scoreAll((root<<1)+2);
     }
   }
 
   @Override
   public int freq() throws IOException {
-    int doc = subScorers[0].docID();
-    int size = numScorers;
-    return 1 + freq(1, size, doc) + freq(2, size, doc);
-  }
-  
-  // Recursively iterate all subScorers that generated last doc computing sum and max
-  private int freq(int root, int size, int doc) throws IOException {
-    int freq = 0;
-    if (root < size && subScorers[root].docID() == doc) {
-      freq++;
-      freq += freq((root<<1)+1, size, doc);
-      freq += freq((root<<1)+2, size, doc);
-    }
     return freq;
   }
 
   @Override
   public int advance(int target) throws IOException {
-    if (numScorers == 0) return doc = NO_MORE_DOCS;
-    while (subScorers[0].docID() < target) {
+    assert doc != NO_MORE_DOCS;
+    while(true) {
       if (subScorers[0].advance(target) != NO_MORE_DOCS) {
         heapAdjust(0);
       } else {
@@ -129,7 +125,10 @@ class DisjunctionMaxScorer extends Disju
           return doc = NO_MORE_DOCS;
         }
       }
+      if (subScorers[0].docID() >= target) {
+        afterNext();
+        return doc;
+      }
     }
-    return doc = subScorers[0].docID();
   }
 }

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java Thu May 30 07:53:18 2013
@@ -105,4 +105,13 @@ abstract class DisjunctionScorer extends
     }
     return children;
   }
+
+  @Override
+  public long cost() {
+    long sum = 0;
+    for (int i = 0; i < numScorers; i++) {
+      sum += subScorers[i].cost();
+    }
+    return sum;
+  } 
 }

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java Thu May 30 07:53:18 2013
@@ -17,84 +17,58 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
-import java.util.List;
 import java.io.IOException;
 
 /** A Scorer for OR like queries, counterpart of <code>ConjunctionScorer</code>.
  * This Scorer implements {@link Scorer#advance(int)} and uses advance() on the given Scorers. 
  */
 class DisjunctionSumScorer extends DisjunctionScorer { 
-  /** The minimum number of scorers that should match. */
-  private final int minimumNrMatchers;
-  
   /** The document number of the current match. */
   private int doc = -1;
 
   /** The number of subscorers that provide the current match. */
   protected int nrMatchers = -1;
 
-  private double score = Float.NaN;
+  protected double score = Float.NaN;
+  private final float[] coord;
   
   /** Construct a <code>DisjunctionScorer</code>.
    * @param weight The weight to be used.
-   * @param subScorers A collection of at least two subscorers.
-   * @param minimumNrMatchers The positive minimum number of subscorers that should
-   * match to match this query.
-   * <br>When <code>minimumNrMatchers</code> is bigger than
-   * the number of <code>subScorers</code>,
-   * no matches will be produced.
-   * <br>When minimumNrMatchers equals the number of subScorers,
-   * it more efficient to use <code>ConjunctionScorer</code>.
+   * @param subScorers Array of at least two subscorers.
+   * @param coord Table of coordination factors
    */
-  public DisjunctionSumScorer(Weight weight, List<Scorer> subScorers, int minimumNrMatchers) throws IOException {
-    super(weight, subScorers.toArray(new Scorer[subScorers.size()]), subScorers.size());
+  DisjunctionSumScorer(Weight weight, Scorer[] subScorers, float[] coord) throws IOException {
+    super(weight, subScorers, subScorers.length);
 
-    if (minimumNrMatchers <= 0) {
-      throw new IllegalArgumentException("Minimum nr of matchers must be positive");
-    }
     if (numScorers <= 1) {
       throw new IllegalArgumentException("There must be at least 2 subScorers");
     }
-
-    this.minimumNrMatchers = minimumNrMatchers;
-  }
-  
-  /** Construct a <code>DisjunctionScorer</code>, using one as the minimum number
-   * of matching subscorers.
-   */
-  public DisjunctionSumScorer(Weight weight, List<Scorer> subScorers) throws IOException {
-    this(weight, subScorers, 1);
+    this.coord = coord;
   }
 
   @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) {
-          heapAdjust(0);
-        } else {
-          heapRemoveRoot();
-          if (numScorers < minimumNrMatchers) {
-            return doc = NO_MORE_DOCS;
-          }
+      if (subScorers[0].nextDoc() != NO_MORE_DOCS) {
+        heapAdjust(0);
+      } else {
+        heapRemoveRoot();
+        if (numScorers == 0) {
+          return doc = NO_MORE_DOCS;
         }
       }
-      afterNext();
-      if (nrMatchers >= minimumNrMatchers) {
-        break;
+      if (subScorers[0].docID() != doc) {
+        afterNext();
+        return doc;
       }
     }
-    
-    return doc;
   }
   
   private void afterNext() throws IOException {
     final Scorer sub = subScorers[0];
     doc = sub.docID();
-    if (doc == NO_MORE_DOCS) {
-      nrMatchers = Integer.MAX_VALUE; // stop looping
-    } else {
+    if (doc != NO_MORE_DOCS) {
       score = sub.score();
       nrMatchers = 1;
       countMatches(1);
@@ -104,9 +78,8 @@ class DisjunctionSumScorer extends Disju
   
   // TODO: this currently scores, but so did the previous impl
   // TODO: remove recursion.
-  // TODO: if we separate scoring, out of here, modify this
-  // and afterNext() to terminate when nrMatchers == minimumNrMatchers
-  // then also change freq() to just always compute it from scratch
+  // TODO: if we separate scoring, out of here, 
+  // then change freq() to just always compute it from scratch
   private void countMatches(int root) throws IOException {
     if (root < numScorers && subScorers[root].docID() == doc) {
       nrMatchers++;
@@ -121,7 +94,7 @@ class DisjunctionSumScorer extends Disju
    */
   @Override
   public float score() throws IOException { 
-    return (float)score; 
+    return (float)score * coord[nrMatchers]; 
   }
    
   @Override
@@ -146,8 +119,8 @@ class DisjunctionSumScorer extends Disju
    */
   @Override
   public int advance(int target) throws IOException {
-    if (numScorers == 0) return doc = NO_MORE_DOCS;
-    while (subScorers[0].docID() < target) {
+    assert doc != NO_MORE_DOCS;
+    while(true) {
       if (subScorers[0].advance(target) != NO_MORE_DOCS) {
         heapAdjust(0);
       } else {
@@ -156,14 +129,10 @@ class DisjunctionSumScorer extends Disju
           return doc = NO_MORE_DOCS;
         }
       }
-    }
-    
-    afterNext();
-
-    if (nrMatchers >= minimumNrMatchers) {
-      return doc;
-    } else {
-      return nextDoc();
+      if (subScorers[0].docID() >= target) {
+        afterNext();
+        return doc;
+      }
     }
   }
 }

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/DocIdSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/DocIdSet.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/DocIdSet.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/DocIdSet.java Thu May 30 07:53:18 2013
@@ -26,38 +26,8 @@ import org.apache.lucene.util.Bits;
  */
 public abstract class DocIdSet {
 
-  /** An empty {@code DocIdSet} instance for easy use, e.g. in Filters that hit no documents. */
-  public static final DocIdSet EMPTY_DOCIDSET = new DocIdSet() {
-    
-    private final DocIdSetIterator iterator = new DocIdSetIterator() {
-      @Override
-      public int advance(int target) { return NO_MORE_DOCS; }
-      @Override
-      public int docID() { return NO_MORE_DOCS; }
-      @Override
-      public int nextDoc() { return NO_MORE_DOCS; }
-    };
-    
-    @Override
-    public DocIdSetIterator iterator() {
-      return iterator;
-    }
-    
-    @Override
-    public boolean isCacheable() {
-      return true;
-    }
-    
-    // we explicitely provide no random access, as this filter is 100% sparse and iterator exits faster
-    @Override
-    public Bits bits() {
-      return null;
-    }
-  };
-    
   /** Provides a {@link DocIdSetIterator} to access the set.
-   * This implementation can return <code>null</code> or
-   * <code>{@linkplain #EMPTY_DOCIDSET}.iterator()</code> if there
+   * This implementation can return <code>null</code> if there
    * are no docs that match. */
   public abstract DocIdSetIterator iterator() throws IOException;
 

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/DocIdSetIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/DocIdSetIterator.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/DocIdSetIterator.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/DocIdSetIterator.java Thu May 30 07:53:18 2013
@@ -28,6 +28,37 @@ import java.io.IOException;
  */
 public abstract class DocIdSetIterator {
   
+  /** An empty {@code DocIdSetIterator} instance */
+  public static final DocIdSetIterator empty() {
+    return new DocIdSetIterator() {
+      boolean exhausted = false;
+      
+      @Override
+      public int advance(int target) {
+        assert !exhausted;
+        assert target >= 0;
+        exhausted = true;
+        return NO_MORE_DOCS;
+      }
+      
+      @Override
+      public int docID() {
+        return exhausted ? NO_MORE_DOCS : -1;
+      }
+      @Override
+      public int nextDoc() {
+        assert !exhausted;
+        exhausted = true;
+        return NO_MORE_DOCS;
+      }
+      
+      @Override
+      public long cost() {
+        return 0;
+      }
+    };
+  }
+  
   /**
    * When returned by {@link #nextDoc()}, {@link #advance(int)} and
    * {@link #docID()} it means there are no more docs in the iterator.
@@ -37,7 +68,7 @@ public abstract class DocIdSetIterator {
   /**
    * Returns the following:
    * <ul>
-   * <li>-1 or {@link #NO_MORE_DOCS} if {@link #nextDoc()} or
+   * <li><code>-1</code> if {@link #nextDoc()} or
    * {@link #advance(int)} were not called yet.
    * <li>{@link #NO_MORE_DOCS} if the iterator has exhausted.
    * <li>Otherwise it should return the doc ID it is currently on.
@@ -93,4 +124,23 @@ public abstract class DocIdSetIterator {
    */
   public abstract int advance(int target) throws IOException;
 
+  /** Slow (linear) implementation of {@link #advance} relying on
+   *  {@link #nextDoc()} to advance beyond the target position. */
+  protected final int slowAdvance(int target) throws IOException {
+    assert docID() < target;
+    int doc;
+    do {
+      doc = nextDoc();
+    } while (doc < target);
+    return doc;
+  }
+
+  /**
+   * Returns the estimated cost of this {@link DocIdSetIterator}.
+   * <p>
+   * This is generally an upper bound of the number of documents this iterator
+   * might match, but may be a rough heuristic, hardcoded value, or otherwise
+   * completely inaccurate.
+   */
+  public abstract long cost();
 }

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/DocTermOrdsRangeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/DocTermOrdsRangeFilter.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/DocTermOrdsRangeFilter.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/DocTermOrdsRangeFilter.java Thu May 30 07:53:18 2013
@@ -90,7 +90,7 @@ public abstract class DocTermOrdsRangeFi
         }      
 
         if (inclusiveUpperPoint < 0 || inclusiveLowerPoint > inclusiveUpperPoint) {
-          return DocIdSet.EMPTY_DOCIDSET;
+          return null;
         }
         
         assert inclusiveLowerPoint >= 0 && inclusiveUpperPoint >= 0;

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/DocTermOrdsRewriteMethod.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/DocTermOrdsRewriteMethod.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/DocTermOrdsRewriteMethod.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/DocTermOrdsRewriteMethod.java Thu May 30 07:53:18 2013
@@ -23,7 +23,6 @@ import java.util.Comparator;
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.SortedSetDocValues;
-import org.apache.lucene.index.SortedSetDocValuesTermsEnum;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.Bits;
@@ -98,7 +97,7 @@ public final class DocTermOrdsRewriteMet
         
         @Override
         public TermsEnum iterator(TermsEnum reuse) {
-          return new SortedSetDocValuesTermsEnum(docTermOrds);
+          return docTermOrds.termsEnum();
         }
 
         @Override
@@ -144,7 +143,7 @@ public final class DocTermOrdsRewriteMet
           termSet.set(termsEnum.ord());
         } while (termsEnum.next() != null);
       } else {
-        return DocIdSet.EMPTY_DOCIDSET;
+        return null;
       }
       
       return new FieldCacheDocIdSet(context.reader().maxDoc(), acceptDocs) {

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java Thu May 30 07:53:18 2013
@@ -33,6 +33,7 @@ final class ExactPhraseScorer extends Sc
   private final int[] gens = new int[CHUNK];
 
   boolean noDocs;
+  private final long cost;
 
   private final static class ChunkState {
     final DocsAndPositionsEnum posEnum;
@@ -65,6 +66,9 @@ final class ExactPhraseScorer extends Sc
     chunkStates = new ChunkState[postings.length];
 
     endMinus1 = postings.length-1;
+    
+    // min(cost)
+    cost = postings[0].postings.cost();
 
     for(int i=0;i<postings.length;i++) {
 
@@ -315,4 +319,9 @@ final class ExactPhraseScorer extends Sc
 
     return freq;
   }
+
+  @Override
+  public long cost() {
+    return cost;
+  }
 }

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FieldCacheDocIdSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FieldCacheDocIdSet.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FieldCacheDocIdSet.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FieldCacheDocIdSet.java Thu May 30 07:53:18 2013
@@ -111,6 +111,11 @@ public abstract class FieldCacheDocIdSet
           }
           return doc = NO_MORE_DOCS;
         }
+
+        @Override
+        public long cost() {
+          return maxDoc;
+        }
       };
     } else if (acceptDocs instanceof FixedBitSet || acceptDocs instanceof OpenBitSet) {
       // special case for FixedBitSet / OpenBitSet: use the iterator and filter it
@@ -151,6 +156,11 @@ public abstract class FieldCacheDocIdSet
           }
           return doc = NO_MORE_DOCS;
         }
+
+        @Override
+        public long cost() {
+          return maxDoc;
+        }
       };
     }
   }

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java Thu May 30 07:53:18 2013
@@ -1388,7 +1388,7 @@ class FieldCacheImpl implements FieldCac
     }
     
     DocTermOrds dto = (DocTermOrds) caches.get(DocTermOrds.class).get(reader, new CacheKey(field, null), false);
-    return dto.iterator(dto.getOrdTermsEnum(reader));
+    return dto.iterator(reader);
   }
 
   static final class DocTermOrdsCache extends Cache {

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java Thu May 30 07:53:18 2013
@@ -121,7 +121,7 @@ public abstract class FieldCacheRangeFil
         }      
 
         if (inclusiveUpperPoint < 0 || inclusiveLowerPoint > inclusiveUpperPoint) {
-          return DocIdSet.EMPTY_DOCIDSET;
+          return null;
         }
         
         assert inclusiveLowerPoint >= 0 && inclusiveUpperPoint >= 0;
@@ -178,7 +178,7 @@ public abstract class FieldCacheRangeFil
         }      
 
         if (inclusiveUpperPoint < 0 || inclusiveLowerPoint > inclusiveUpperPoint) {
-          return DocIdSet.EMPTY_DOCIDSET;
+          return null;
         }
         
         assert inclusiveLowerPoint >= 0 && inclusiveUpperPoint >= 0;
@@ -216,7 +216,7 @@ public abstract class FieldCacheRangeFil
         if (lowerVal != null) {
           final byte i = lowerVal.byteValue();
           if (!includeLower && i == Byte.MAX_VALUE)
-            return DocIdSet.EMPTY_DOCIDSET;
+            return null;
           inclusiveLowerPoint = (byte) (includeLower ?  i : (i + 1));
         } else {
           inclusiveLowerPoint = Byte.MIN_VALUE;
@@ -224,14 +224,14 @@ public abstract class FieldCacheRangeFil
         if (upperVal != null) {
           final byte i = upperVal.byteValue();
           if (!includeUpper && i == Byte.MIN_VALUE)
-            return DocIdSet.EMPTY_DOCIDSET;
+            return null;
           inclusiveUpperPoint = (byte) (includeUpper ? i : (i - 1));
         } else {
           inclusiveUpperPoint = Byte.MAX_VALUE;
         }
         
         if (inclusiveLowerPoint > inclusiveUpperPoint)
-          return DocIdSet.EMPTY_DOCIDSET;
+          return null;
         
         final FieldCache.Bytes values = FieldCache.DEFAULT.getBytes(context.reader(), field, (FieldCache.ByteParser) parser, false);
         return new FieldCacheDocIdSet(context.reader().maxDoc(), acceptDocs) {
@@ -267,7 +267,7 @@ public abstract class FieldCacheRangeFil
         if (lowerVal != null) {
           short i = lowerVal.shortValue();
           if (!includeLower && i == Short.MAX_VALUE)
-            return DocIdSet.EMPTY_DOCIDSET;
+            return null;
           inclusiveLowerPoint = (short) (includeLower ? i : (i + 1));
         } else {
           inclusiveLowerPoint = Short.MIN_VALUE;
@@ -275,14 +275,14 @@ public abstract class FieldCacheRangeFil
         if (upperVal != null) {
           short i = upperVal.shortValue();
           if (!includeUpper && i == Short.MIN_VALUE)
-            return DocIdSet.EMPTY_DOCIDSET;
+            return null;
           inclusiveUpperPoint = (short) (includeUpper ? i : (i - 1));
         } else {
           inclusiveUpperPoint = Short.MAX_VALUE;
         }
         
         if (inclusiveLowerPoint > inclusiveUpperPoint)
-          return DocIdSet.EMPTY_DOCIDSET;
+          return null;
         
         final FieldCache.Shorts values = FieldCache.DEFAULT.getShorts(context.reader(), field, (FieldCache.ShortParser) parser, false);
         return new FieldCacheDocIdSet(context.reader().maxDoc(), acceptDocs) {
@@ -318,7 +318,7 @@ public abstract class FieldCacheRangeFil
         if (lowerVal != null) {
           int i = lowerVal.intValue();
           if (!includeLower && i == Integer.MAX_VALUE)
-            return DocIdSet.EMPTY_DOCIDSET;
+            return null;
           inclusiveLowerPoint = includeLower ? i : (i + 1);
         } else {
           inclusiveLowerPoint = Integer.MIN_VALUE;
@@ -326,14 +326,14 @@ public abstract class FieldCacheRangeFil
         if (upperVal != null) {
           int i = upperVal.intValue();
           if (!includeUpper && i == Integer.MIN_VALUE)
-            return DocIdSet.EMPTY_DOCIDSET;
+            return null;
           inclusiveUpperPoint = includeUpper ? i : (i - 1);
         } else {
           inclusiveUpperPoint = Integer.MAX_VALUE;
         }
         
         if (inclusiveLowerPoint > inclusiveUpperPoint)
-          return DocIdSet.EMPTY_DOCIDSET;
+          return null;
         
         final FieldCache.Ints values = FieldCache.DEFAULT.getInts(context.reader(), field, (FieldCache.IntParser) parser, false);
         return new FieldCacheDocIdSet(context.reader().maxDoc(), acceptDocs) {
@@ -369,7 +369,7 @@ public abstract class FieldCacheRangeFil
         if (lowerVal != null) {
           long i = lowerVal.longValue();
           if (!includeLower && i == Long.MAX_VALUE)
-            return DocIdSet.EMPTY_DOCIDSET;
+            return null;
           inclusiveLowerPoint = includeLower ? i : (i + 1L);
         } else {
           inclusiveLowerPoint = Long.MIN_VALUE;
@@ -377,14 +377,14 @@ public abstract class FieldCacheRangeFil
         if (upperVal != null) {
           long i = upperVal.longValue();
           if (!includeUpper && i == Long.MIN_VALUE)
-            return DocIdSet.EMPTY_DOCIDSET;
+            return null;
           inclusiveUpperPoint = includeUpper ? i : (i - 1L);
         } else {
           inclusiveUpperPoint = Long.MAX_VALUE;
         }
         
         if (inclusiveLowerPoint > inclusiveUpperPoint)
-          return DocIdSet.EMPTY_DOCIDSET;
+          return null;
         
         final FieldCache.Longs values = FieldCache.DEFAULT.getLongs(context.reader(), field, (FieldCache.LongParser) parser, false);
         return new FieldCacheDocIdSet(context.reader().maxDoc(), acceptDocs) {
@@ -422,7 +422,7 @@ public abstract class FieldCacheRangeFil
         if (lowerVal != null) {
           float f = lowerVal.floatValue();
           if (!includeUpper && f > 0.0f && Float.isInfinite(f))
-            return DocIdSet.EMPTY_DOCIDSET;
+            return null;
           int i = NumericUtils.floatToSortableInt(f);
           inclusiveLowerPoint = NumericUtils.sortableIntToFloat( includeLower ?  i : (i + 1) );
         } else {
@@ -431,7 +431,7 @@ public abstract class FieldCacheRangeFil
         if (upperVal != null) {
           float f = upperVal.floatValue();
           if (!includeUpper && f < 0.0f && Float.isInfinite(f))
-            return DocIdSet.EMPTY_DOCIDSET;
+            return null;
           int i = NumericUtils.floatToSortableInt(f);
           inclusiveUpperPoint = NumericUtils.sortableIntToFloat( includeUpper ? i : (i - 1) );
         } else {
@@ -439,7 +439,7 @@ public abstract class FieldCacheRangeFil
         }
         
         if (inclusiveLowerPoint > inclusiveUpperPoint)
-          return DocIdSet.EMPTY_DOCIDSET;
+          return null;
         
         final FieldCache.Floats values = FieldCache.DEFAULT.getFloats(context.reader(), field, (FieldCache.FloatParser) parser, false);
         return new FieldCacheDocIdSet(context.reader().maxDoc(), acceptDocs) {
@@ -477,7 +477,7 @@ public abstract class FieldCacheRangeFil
         if (lowerVal != null) {
           double f = lowerVal.doubleValue();
           if (!includeUpper && f > 0.0 && Double.isInfinite(f))
-            return DocIdSet.EMPTY_DOCIDSET;
+            return null;
           long i = NumericUtils.doubleToSortableLong(f);
           inclusiveLowerPoint = NumericUtils.sortableLongToDouble( includeLower ?  i : (i + 1L) );
         } else {
@@ -486,7 +486,7 @@ public abstract class FieldCacheRangeFil
         if (upperVal != null) {
           double f = upperVal.doubleValue();
           if (!includeUpper && f < 0.0 && Double.isInfinite(f))
-            return DocIdSet.EMPTY_DOCIDSET;
+            return null;
           long i = NumericUtils.doubleToSortableLong(f);
           inclusiveUpperPoint = NumericUtils.sortableLongToDouble( includeUpper ? i : (i - 1L) );
         } else {
@@ -494,7 +494,7 @@ public abstract class FieldCacheRangeFil
         }
         
         if (inclusiveLowerPoint > inclusiveUpperPoint)
-          return DocIdSet.EMPTY_DOCIDSET;
+          return null;
         
         final FieldCache.Doubles values = FieldCache.DEFAULT.getDoubles(context.reader(), field, (FieldCache.DoubleParser) parser, false);
         // ignore deleted docs if range doesn't contain 0

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FieldCacheRewriteMethod.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FieldCacheRewriteMethod.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FieldCacheRewriteMethod.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FieldCacheRewriteMethod.java Thu May 30 07:53:18 2013
@@ -23,7 +23,6 @@ import java.util.Comparator;
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.index.SortedDocValuesTermsEnum;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.Bits;
@@ -98,7 +97,7 @@ public final class FieldCacheRewriteMeth
         
         @Override
         public TermsEnum iterator(TermsEnum reuse) {
-          return new SortedDocValuesTermsEnum(fcsi);
+          return fcsi.termsEnum();
         }
 
         @Override
@@ -147,7 +146,7 @@ public final class FieldCacheRewriteMeth
           }
         } while (termsEnum.next() != null);
       } else {
-        return DocIdSet.EMPTY_DOCIDSET;
+        return null;
       }
       
       return new FieldCacheDocIdSet(context.reader().maxDoc(), acceptDocs) {

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java Thu May 30 07:53:18 2013
@@ -19,7 +19,6 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 
-import org.apache.lucene.index.AtomicReader; // javadocs
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.SortedDocValues;
@@ -235,7 +234,7 @@ public abstract class FieldComparator<T>
 
     @Override
     public int compare(int slot1, int slot2) {
-      return values[slot1] - values[slot2];
+      return Byte.compare(values[slot1], values[slot2]);
     }
 
     @Override
@@ -247,7 +246,7 @@ public abstract class FieldComparator<T>
         v2 = missingValue;
       }
 
-      return bottom - v2;
+      return Byte.compare(bottom, v2);
     }
 
     @Override
@@ -287,7 +286,7 @@ public abstract class FieldComparator<T>
       if (docsWithField != null && docValue == 0 && !docsWithField.get(doc)) {
         docValue = missingValue;
       }
-      return docValue - value.byteValue();
+      return Byte.compare(docValue, value.byteValue());
     }
   }
 
@@ -307,15 +306,7 @@ public abstract class FieldComparator<T>
 
     @Override
     public int compare(int slot1, int slot2) {
-      final double v1 = values[slot1];
-      final double v2 = values[slot2];
-      if (v1 > v2) {
-        return 1;
-      } else if (v1 < v2) {
-        return -1;
-      } else {
-        return 0;
-      }
+      return Double.compare(values[slot1], values[slot2]);
     }
 
     @Override
@@ -327,13 +318,7 @@ public abstract class FieldComparator<T>
         v2 = missingValue;
       }
 
-      if (bottom > v2) {
-        return 1;
-      } else if (bottom < v2) {
-        return -1;
-      } else {
-        return 0;
-      }
+      return Double.compare(bottom, v2);
     }
 
     @Override
@@ -375,13 +360,7 @@ public abstract class FieldComparator<T>
       if (docsWithField != null && docValue == 0 && !docsWithField.get(doc)) {
         docValue = missingValue;
       }
-      if (docValue < value) {
-        return -1;
-      } else if (docValue > value) {
-        return 1;
-      } else {
-        return 0;
-      }
+      return Double.compare(docValue, value);
     }
   }
 
@@ -401,17 +380,7 @@ public abstract class FieldComparator<T>
     
     @Override
     public int compare(int slot1, int slot2) {
-      // TODO: are there sneaky non-branch ways to compute
-      // sign of float?
-      final float v1 = values[slot1];
-      final float v2 = values[slot2];
-      if (v1 > v2) {
-        return 1;
-      } else if (v1 < v2) {
-        return -1;
-      } else {
-        return 0;
-      }
+      return Float.compare(values[slot1], values[slot2]);
     }
 
     @Override
@@ -423,14 +392,8 @@ public abstract class FieldComparator<T>
       if (docsWithField != null && v2 == 0 && !docsWithField.get(doc)) {
         v2 = missingValue;
       }
-      
-      if (bottom > v2) {
-        return 1;
-      } else if (bottom < v2) {
-        return -1;
-      } else {
-        return 0;
-      }
+
+      return Float.compare(bottom, v2);
     }
 
     @Override
@@ -472,13 +435,7 @@ public abstract class FieldComparator<T>
       if (docsWithField != null && docValue == 0 && !docsWithField.get(doc)) {
         docValue = missingValue;
       }
-      if (docValue < value) {
-        return -1;
-      } else if (docValue > value) {
-        return 1;
-      } else {
-        return 0;
-      }
+      return Float.compare(docValue, value);
     }
   }
 
@@ -498,7 +455,7 @@ public abstract class FieldComparator<T>
 
     @Override
     public int compare(int slot1, int slot2) {
-      return values[slot1] - values[slot2];
+      return Short.compare(values[slot1], values[slot2]);
     }
 
     @Override
@@ -510,7 +467,7 @@ public abstract class FieldComparator<T>
         v2 = missingValue;
       }
 
-      return bottom - v2;
+      return Short.compare(bottom, v2);
     }
 
     @Override
@@ -552,7 +509,7 @@ public abstract class FieldComparator<T>
       if (docsWithField != null && docValue == 0 && !docsWithField.get(doc)) {
         docValue = missingValue;
       }
-      return docValue - value;
+      return Short.compare(docValue, value);
     }
   }
 
@@ -572,27 +529,11 @@ public abstract class FieldComparator<T>
         
     @Override
     public int compare(int slot1, int slot2) {
-      // TODO: there are sneaky non-branch ways to compute
-      // -1/+1/0 sign
-      // Cannot return values[slot1] - values[slot2] because that
-      // may overflow
-      final int v1 = values[slot1];
-      final int v2 = values[slot2];
-      if (v1 > v2) {
-        return 1;
-      } else if (v1 < v2) {
-        return -1;
-      } else {
-        return 0;
-      }
+      return Integer.compare(values[slot1], values[slot2]);
     }
 
     @Override
     public int compareBottom(int doc) {
-      // TODO: there are sneaky non-branch ways to compute
-      // -1/+1/0 sign
-      // Cannot return bottom - values[slot2] because that
-      // may overflow
       int v2 = currentReaderValues.get(doc);
       // Test for v2 == 0 to save Bits.get method call for
       // the common case (doc has value and value is non-zero):
@@ -600,13 +541,7 @@ public abstract class FieldComparator<T>
         v2 = missingValue;
       }
 
-      if (bottom > v2) {
-        return 1;
-      } else if (bottom < v2) {
-        return -1;
-      } else {
-        return 0;
-      }
+      return Integer.compare(bottom, v2);
     }
 
     @Override
@@ -648,13 +583,7 @@ public abstract class FieldComparator<T>
       if (docsWithField != null && docValue == 0 && !docsWithField.get(doc)) {
         docValue = missingValue;
       }
-      if (docValue < value) {
-        return -1;
-      } else if (docValue > value) {
-        return 1;
-      } else {
-        return 0;
-      }
+      return Integer.compare(docValue, value);
     }
   }
 
@@ -674,17 +603,7 @@ public abstract class FieldComparator<T>
 
     @Override
     public int compare(int slot1, int slot2) {
-      // TODO: there are sneaky non-branch ways to compute
-      // -1/+1/0 sign
-      final long v1 = values[slot1];
-      final long v2 = values[slot2];
-      if (v1 > v2) {
-        return 1;
-      } else if (v1 < v2) {
-        return -1;
-      } else {
-        return 0;
-      }
+      return Long.compare(values[slot1], values[slot2]);
     }
 
     @Override
@@ -698,13 +617,7 @@ public abstract class FieldComparator<T>
         v2 = missingValue;
       }
 
-      if (bottom > v2) {
-        return 1;
-      } else if (bottom < v2) {
-        return -1;
-      } else {
-        return 0;
-      }
+      return Long.compare(bottom, v2);
     }
 
     @Override
@@ -746,13 +659,7 @@ public abstract class FieldComparator<T>
       if (docsWithField != null && docValue == 0 && !docsWithField.get(doc)) {
         docValue = missingValue;
       }
-      if (docValue < value) {
-        return -1;
-      } else if (docValue > value) {
-        return 1;
-      } else {
-        return 0;
-      }
+      return Long.compare(docValue, value);
     }
   }
 
@@ -773,16 +680,14 @@ public abstract class FieldComparator<T>
 
     @Override
     public int compare(int slot1, int slot2) {
-      final float score1 = scores[slot1];
-      final float score2 = scores[slot2];
-      return score1 > score2 ? -1 : (score1 < score2 ? 1 : 0);
+      return Float.compare(scores[slot2], scores[slot1]);
     }
 
     @Override
     public int compareBottom(int doc) throws IOException {
       float score = scorer.score();
       assert !Float.isNaN(score);
-      return bottom > score ? -1 : (bottom < score ? 1 : 0);
+      return Float.compare(score, bottom);
     }
 
     @Override
@@ -831,15 +736,7 @@ public abstract class FieldComparator<T>
       final float value = valueObj.floatValue();
       float docValue = scorer.score();
       assert !Float.isNaN(docValue);
-      if (docValue < value) {
-        // reverse of FloatComparator
-        return 1;
-      } else if (docValue > value) {
-        // reverse of FloatComparator
-        return -1;
-      } else {
-        return 0;
-      }
+      return Float.compare(value, docValue);
     }
   }
 
@@ -893,13 +790,7 @@ public abstract class FieldComparator<T>
     public int compareDocToValue(int doc, Integer valueObj) {
       final int value = valueObj.intValue();
       int docValue = docBase + doc;
-      if (docValue < value) {
-        return -1;
-      } else if (docValue > value) {
-        return 1;
-      } else {
-        return 0;
-      }
+      return Integer.compare(docValue, value);
     }
   }
   

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/Filter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/Filter.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/Filter.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/Filter.java Thu May 30 07:53:18 2013
@@ -21,7 +21,6 @@ import java.io.IOException;
 
 import org.apache.lucene.index.AtomicReader; // javadocs
 import org.apache.lucene.index.AtomicReaderContext;
-import org.apache.lucene.index.IndexReader; // javadocs
 import org.apache.lucene.util.Bits;
 
 /** 
@@ -53,8 +52,9 @@ public abstract class Filter {
    *          but possibly filtering other documents)
    *          
    * @return a DocIdSet that provides the documents which should be permitted or
-   *         prohibited in search results. <b>NOTE:</b> null can be returned if
-   *         no documents will be accepted by this Filter.
+   *         prohibited in search results. <b>NOTE:</b> <code>null</code> should be returned if
+   *         the filter doesn't accept any documents otherwise internal optimization might not apply
+   *         in the case an <i>empty</i> {@link DocIdSet} is returned.
    */
   public abstract DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException;
 }

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FilteredDocIdSetIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FilteredDocIdSetIterator.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FilteredDocIdSetIterator.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FilteredDocIdSetIterator.java Thu May 30 07:53:18 2013
@@ -81,5 +81,9 @@ public abstract class FilteredDocIdSetIt
     }
     return doc;
   }
-  
+
+  @Override
+  public long cost() {
+    return _innerIter.cost();
+  }
 }

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java Thu May 30 07:53:18 2013
@@ -102,9 +102,9 @@ public class FilteredQuery extends Query
         Explanation inner = weight.explain (ir, i);
         Filter f = FilteredQuery.this.filter;
         DocIdSet docIdSet = f.getDocIdSet(ir, ir.reader().getLiveDocs());
-        DocIdSetIterator docIdSetIterator = docIdSet == null ? DocIdSet.EMPTY_DOCIDSET.iterator() : docIdSet.iterator();
+        DocIdSetIterator docIdSetIterator = docIdSet == null ? DocIdSetIterator.empty() : docIdSet.iterator();
         if (docIdSetIterator == null) {
-          docIdSetIterator = DocIdSet.EMPTY_DOCIDSET.iterator();
+          docIdSetIterator = DocIdSetIterator.empty();
         }
         if (docIdSetIterator.advance(i) == i) {
           return inner;
@@ -210,6 +210,11 @@ public class FilteredQuery extends Query
     public Collection<ChildScorer> getChildren() {
       return Collections.singleton(new ChildScorer(scorer, "FILTERED"));
     }
+
+    @Override
+    public long cost() {
+      return scorer.cost();
+    }
   }
   
   /**
@@ -235,11 +240,11 @@ public class FilteredQuery extends Query
     // 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);
+      int primDoc = primaryNext();
+      int secDoc = secondary.advance(primDoc);
       for (;;) {
         if (primDoc == secDoc) {
           // Check if scorer has exhausted, only before collecting.
@@ -304,6 +309,11 @@ public class FilteredQuery extends Query
     public final Collection<ChildScorer> getChildren() {
       return Collections.singleton(new ChildScorer(scorer, "FILTERED"));
     }
+
+    @Override
+    public long cost() {
+      return Math.min(primary.cost(), secondary.cost());
+    }
   }
   
   // TODO once we have way to figure out if we use RA or LeapFrog we can remove this scorer

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java Thu May 30 07:53:18 2013
@@ -32,7 +32,6 @@ import java.util.concurrent.ExecutorServ
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 
-import org.apache.lucene.document.Document;
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.DirectoryReader; // javadocs
 import org.apache.lucene.index.IndexReader;
@@ -427,6 +426,12 @@ public class IndexSearcher {
    *         {@link BooleanQuery#getMaxClauseCount()} clauses.
    */
   protected TopDocs search(Weight weight, ScoreDoc after, int nDocs) throws IOException {
+    int limit = reader.maxDoc();
+    if (limit == 0) {
+      limit = 1;
+    }
+    nDocs = Math.min(nDocs, limit);
+    
     if (executor == null) {
       return search(leafContexts, weight, after, nDocs);
     } else {
@@ -512,6 +517,12 @@ public class IndexSearcher {
 
     if (sort == null) throw new NullPointerException("Sort must not be null");
     
+    int limit = reader.maxDoc();
+    if (limit == 0) {
+      limit = 1;
+    }
+    nDocs = Math.min(nDocs, limit);
+
     if (executor == null) {
       // use all leaves here!
       return search(leafContexts, weight, after, nDocs, sort, fillFields, doDocScores, doMaxScore);
@@ -592,10 +603,21 @@ public class IndexSearcher {
     // threaded...?  the Collector could be sync'd?
     // always use single thread:
     for (AtomicReaderContext ctx : leaves) { // search each subreader
-      collector.setNextReader(ctx);
+      try {
+        collector.setNextReader(ctx);
+      } catch (CollectionTerminatedException e) {
+        // there is no doc of interest in this reader context
+        // continue with the following leaf
+        continue;
+      }
       Scorer scorer = weight.scorer(ctx, !collector.acceptsDocsOutOfOrder(), true, ctx.reader().getLiveDocs());
       if (scorer != null) {
-        scorer.score(collector);
+        try {
+          scorer.score(collector);
+        } catch (CollectionTerminatedException e) {
+          // collection was terminated prematurely
+          // continue with the following leaf
+        }
       }
     }
   }
@@ -781,6 +803,11 @@ public class IndexSearcher {
       public float score() {
         return score;
       }
+
+      @Override
+      public long cost() {
+        return 1;
+      }
     }
 
     private final FakeScorer fakeScorer = new FakeScorer();
@@ -789,7 +816,7 @@ public class IndexSearcher {
     public TopFieldDocs call() throws IOException {
       assert slice.leaves.length == 1;
       final TopFieldDocs docs = searcher.search(Arrays.asList(slice.leaves),
-          weight, after, nDocs, sort, true, doDocScores, doMaxScore);
+          weight, after, nDocs, sort, true, doDocScores || sort.needsScores(), doMaxScore);
       lock.lock();
       try {
         final AtomicReaderContext ctx = slice.leaves[0];

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/LiveFieldValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/LiveFieldValues.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/LiveFieldValues.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/LiveFieldValues.java Thu May 30 07:53:18 2013
@@ -33,14 +33,14 @@ import java.util.concurrent.ConcurrentHa
  *  the same time by two threads, because in this case you
  *  cannot in general know which thread "won". */
 
-public abstract class LiveFieldValues<T> implements ReferenceManager.RefreshListener, Closeable {
+public abstract class LiveFieldValues<S,T> implements ReferenceManager.RefreshListener, Closeable {
 
   private volatile Map<String,T> current = new ConcurrentHashMap<String,T>();
   private volatile Map<String,T> old = new ConcurrentHashMap<String,T>();
-  private final ReferenceManager<IndexSearcher> mgr;
+  private final ReferenceManager<S> mgr;
   private final T missingValue;
 
-  public LiveFieldValues(ReferenceManager<IndexSearcher> mgr, T missingValue) {
+  public LiveFieldValues(ReferenceManager<S> mgr, T missingValue) {
     this.missingValue = missingValue;
     this.mgr = mgr;
     mgr.addListener(this);
@@ -114,7 +114,7 @@ public abstract class LiveFieldValues<T>
         // It either does not exist in the index, or, it was
         // already flushed & NRT reader was opened on the
         // segment, so fallback to current searcher:
-        IndexSearcher s = mgr.acquire();
+        S s = mgr.acquire();
         try {
           return lookupFromSearcher(s, id);
         } finally {
@@ -128,6 +128,6 @@ public abstract class LiveFieldValues<T>
    *  in an NRT IndexSearcher.  You must implement this to
    *  go look up the value (eg, via doc values, field cache,
    *  stored fields, etc.). */
-  protected abstract T lookupFromSearcher(IndexSearcher s, String id) throws IOException;
+  protected abstract T lookupFromSearcher(S s, String id) throws IOException;
 }
 

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java Thu May 30 07:53:18 2013
@@ -77,6 +77,11 @@ public class MatchAllDocsQuery extends Q
       doc = target-1;
       return nextDoc();
     }
+
+    @Override
+    public long cost() {
+      return maxDoc;
+    }
   }
 
   private class MatchAllDocsWeight extends Weight {

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java Thu May 30 07:53:18 2013
@@ -241,7 +241,7 @@ public class MultiPhraseQuery extends Qu
 
       // sort by increasing docFreq order
       if (slop == 0) {
-        ArrayUtil.mergeSort(postingsFreqs);
+        ArrayUtil.timSort(postingsFreqs);
       }
 
       if (slop == 0) {
@@ -472,6 +472,7 @@ class UnionDocsAndPositionsEnum extends 
   private int _freq;
   private DocsQueue _queue;
   private IntQueue _posList;
+  private long cost;
 
   public UnionDocsAndPositionsEnum(Bits liveDocs, AtomicReaderContext context, Term[] terms, Map<Term,TermContext> termContexts, TermsEnum termsEnum) throws IOException {
     List<DocsAndPositionsEnum> docsEnums = new LinkedList<DocsAndPositionsEnum>();
@@ -488,6 +489,7 @@ class UnionDocsAndPositionsEnum extends 
         // term does exist, but has no positions
         throw new IllegalStateException("field \"" + term.field() + "\" was indexed without position data; cannot run PhraseQuery (term=" + term.text() + ")");
       }
+      cost += postings.cost();
       docsEnums.add(postings);
     }
 
@@ -570,4 +572,9 @@ class UnionDocsAndPositionsEnum extends 
   public final int docID() {
     return _doc;
   }
+
+  @Override
+  public long cost() {
+    return cost;
+  }
 }

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/MultiTermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/MultiTermQuery.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/MultiTermQuery.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/MultiTermQuery.java Thu May 30 07:53:18 2013
@@ -248,8 +248,10 @@ public abstract class MultiTermQuery ext
    * Term.
    */
   public MultiTermQuery(final String field) {
+    if (field == null) {
+      throw new IllegalArgumentException("field must not be null");
+    }
     this.field = field;
-    assert field != null;
   }
 
   /** Returns the field name for this query */

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java Thu May 30 07:53:18 2013
@@ -88,13 +88,13 @@ public class MultiTermQueryWrapperFilter
     final Fields fields = reader.fields();
     if (fields == null) {
       // reader has no fields
-      return DocIdSet.EMPTY_DOCIDSET;
+      return null;
     }
 
     final Terms terms = fields.terms(query.field);
     if (terms == null) {
       // field does not exist
-      return DocIdSet.EMPTY_DOCIDSET;
+      return null;
     }
 
     final TermsEnum termsEnum = query.getTermsEnum(terms);
@@ -116,7 +116,7 @@ public class MultiTermQueryWrapperFilter
 
       return bitSet;
     } else {
-      return DocIdSet.EMPTY_DOCIDSET;
+      return null;
     }
   }
 }

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/NGramPhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/NGramPhraseQuery.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/NGramPhraseQuery.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/NGramPhraseQuery.java Thu May 30 07:53:18 2013
@@ -64,6 +64,7 @@ public class NGramPhraseQuery extends Ph
 
     // now create the new optimized phrase query for n-gram
     PhraseQuery optimized = new PhraseQuery();
+    optimized.setBoost(getBoost());
     int pos = 0;
     final int lastPos = terms.length - 1;
     for(int i = 0; i < terms.length; i++){

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java Thu May 30 07:53:18 2013
@@ -278,7 +278,7 @@ public class PhraseQuery extends Query {
 
       // sort by increasing docFreq order
       if (slop == 0) {
-        ArrayUtil.mergeSort(postingsFreqs);
+        ArrayUtil.timSort(postingsFreqs);
       }
 
       if (slop == 0) {  // optimize exact case

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/ReferenceManager.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/ReferenceManager.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/ReferenceManager.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/ReferenceManager.java Thu May 30 07:53:18 2013
@@ -81,7 +81,7 @@ public abstract class ReferenceManager<G
    * the operation was successful.
    * @throws AlreadyClosedException if the reference manager has been {@link #close() closed}. 
    */
-  protected abstract boolean tryIncRef(G reference);
+  protected abstract boolean tryIncRef(G reference) throws IOException;
 
   /**
    * Obtain the current reference. You must match every call to acquire with one
@@ -90,7 +90,7 @@ public abstract class ReferenceManager<G
    * released.
    * @throws AlreadyClosedException if the reference manager has been {@link #close() closed}. 
    */
-  public final G acquire() {
+  public final G acquire() throws IOException {
     G ref;
     do {
       if ((ref = current) == null) {

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/ReqExclScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/ReqExclScorer.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/ReqExclScorer.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/ReqExclScorer.java Thu May 30 07:53:18 2013
@@ -128,4 +128,9 @@ class ReqExclScorer extends Scorer {
     }
     return doc = toNonExcluded();
   }
+
+  @Override
+  public long cost() {
+    return reqScorer.cost();
+  }
 }

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java Thu May 30 07:53:18 2013
@@ -99,5 +99,10 @@ class ReqOptSumScorer extends Scorer {
     children.add(new ChildScorer(optScorer, "SHOULD"));
     return children;
   }
+
+  @Override
+  public long cost() {
+    return reqScorer.cost();
+  }
 }
 

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java Thu May 30 07:53:18 2013
@@ -89,4 +89,9 @@ public class ScoreCachingWrappingScorer 
   public Collection<ChildScorer> getChildren() {
     return Collections.singleton(new ChildScorer(scorer, "CACHED"));
   }
+
+  @Override
+  public long cost() {
+    return scorer.cost();
+  }
 }

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/Scorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/Scorer.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/Scorer.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/Scorer.java Thu May 30 07:53:18 2013
@@ -58,6 +58,7 @@ public abstract class Scorer extends Doc
    * @param collector The collector to which all matching documents are passed.
    */
   public void score(Collector collector) throws IOException {
+    assert docID() == -1; // not started
     collector.setScorer(this);
     int doc;
     while ((doc = nextDoc()) != NO_MORE_DOCS) {
@@ -80,11 +81,11 @@ public abstract class Scorer extends Doc
    * @return true if more matching documents may remain.
    */
   public boolean score(Collector collector, int max, int firstDocID) throws IOException {
+    assert docID() == firstDocID;
     collector.setScorer(this);
-    int doc = firstDocID;
-    while (doc < max) {
+    int doc;
+    for (doc = firstDocID; doc < max; doc = nextDoc()) {
       collector.collect(doc);
-      doc = nextDoc();
     }
     return doc != NO_MORE_DOCS;
   }

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/SearcherFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/SearcherFactory.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/SearcherFactory.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/SearcherFactory.java Thu May 30 07:53:18 2013
@@ -26,7 +26,7 @@ import org.apache.lucene.index.IndexWrit
 import org.apache.lucene.search.similarities.Similarity; // javadocs
 
 /**
- * Factory class used by {@link SearcherManager} and {@link NRTManager} to
+ * Factory class used by {@link SearcherManager} to
  * create new IndexSearchers. The default implementation just creates 
  * an IndexSearcher with no custom behavior:
  * 

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/SearcherLifetimeManager.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/SearcherLifetimeManager.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/SearcherLifetimeManager.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/SearcherLifetimeManager.java Thu May 30 07:53:18 2013
@@ -24,7 +24,6 @@ import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.ConcurrentHashMap;
 
-import org.apache.lucene.search.NRTManager;        // javadocs
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.util.IOUtils;
@@ -41,8 +40,8 @@ import org.apache.lucene.util.IOUtils;
  *
  * Per search-request, if it's a "new" search request, then
  * obtain the latest searcher you have (for example, by
- * using {@link SearcherManager} or {@link NRTManager}), and
- * then record this searcher:
+ * using {@link SearcherManager}), and then record this
+ * searcher:
  *
  * <pre class="prettyprint">
  *   // Record the current searcher, and save the returend
@@ -119,16 +118,7 @@ public class SearcherLifetimeManager imp
     // Newer searchers are sort before older ones:
     @Override
     public int compareTo(SearcherTracker other) {
-      // Be defensive: cannot subtract since it could
-      // technically overflow long, though, we'd never hit
-      // that in practice:
-      if (recordTimeSec < other.recordTimeSec) {
-        return 1;
-      } else if (other.recordTimeSec < recordTimeSec) {
-        return -1;
-      } else {
-        return 0;
-      }
+      return Double.compare(other.recordTimeSec, recordTimeSec);
     }
 
     @Override
@@ -152,8 +142,7 @@ public class SearcherLifetimeManager imp
 
   /** Records that you are now using this IndexSearcher.
    *  Always call this when you've obtained a possibly new
-   *  {@link IndexSearcher}, for example from one of the
-   *  <code>get</code> methods in {@link NRTManager} or {@link
+   *  {@link IndexSearcher}, for example from {@link
    *  SearcherManager}.  It's fine if you already passed the
    *  same searcher to this method before.
    *

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/SearcherManager.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/SearcherManager.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/SearcherManager.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/SearcherManager.java Thu May 30 07:53:18 2013
@@ -144,8 +144,11 @@ public final class SearcherManager exten
     }
   }
 
-  // NOTE: decRefs incoming reader on throwing an exception
-  static IndexSearcher getSearcher(SearcherFactory searcherFactory, IndexReader reader) throws IOException {
+  /** Expert: creates a searcher from the provided {@link
+   *  IndexReader} using the provided {@link
+   *  SearcherFactory}.  NOTE: this decRefs incoming reader
+   * on throwing an exception. */
+  public static IndexSearcher getSearcher(SearcherFactory searcherFactory, IndexReader reader) throws IOException {
     boolean success = false;
     final IndexSearcher searcher;
     try {

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java Thu May 30 07:53:18 2013
@@ -49,6 +49,7 @@ final class SloppyPhraseScorer extends S
   private PhrasePositions[] rptStack; // temporary stack for switching colliding repeating pps 
   
   private int numMatches;
+  private final long cost;
   
   SloppyPhraseScorer(Weight weight, PhraseQuery.PostingsAndFreq[] postings,
       int slop, Similarity.SloppySimScorer docScorer) {
@@ -57,6 +58,8 @@ final class SloppyPhraseScorer extends S
     this.slop = slop;
     this.numPostings = postings==null ? 0 : postings.length;
     pq = new PhraseQueue(postings.length);
+    // min(cost)
+    cost = postings[0].postings.cost();
     // convert tps to a list of phrase positions.
     // note: phrase-position differs from term-position in that its position
     // reflects the phrase offset: pp.pos = tp.pos - offset.
@@ -558,7 +561,7 @@ final class SloppyPhraseScorer extends S
 
   @Override
   public int nextDoc() throws IOException {
-    return advance(max.doc);
+    return advance(max.doc + 1); // advance to the next doc after #docID()
   }
   
   @Override
@@ -568,27 +571,30 @@ final class SloppyPhraseScorer extends S
 
   @Override
   public int advance(int target) throws IOException {
-    sloppyFreq = 0.0f;
-    if (!advanceMin(target)) {
-      return NO_MORE_DOCS;
-    }        
-    boolean restart=false;
-    while (sloppyFreq == 0.0f) {
-      while (min.doc < max.doc || restart) {
-        restart = false;
+    assert target > docID();
+    do {
+      if (!advanceMin(target)) {
+        return NO_MORE_DOCS;
+      }
+      while (min.doc < max.doc) {
         if (!advanceMin(max.doc)) {
           return NO_MORE_DOCS;
-        }        
+        }
       }
       // found a doc with all of the terms
       sloppyFreq = phraseFreq(); // check for phrase
-      restart = true;
-    } 
+      target = min.doc + 1; // next target in case sloppyFreq is still 0
+    } while (sloppyFreq == 0f);
 
     // found a match
     return max.doc;
   }
-  
+
+  @Override
+  public long cost() {
+    return cost;
+  }
+
   @Override
   public String toString() { return "scorer(" + weight + ")"; }
 }

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/Sort.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/Sort.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/Sort.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/Sort.java Thu May 30 07:53:18 2013
@@ -201,4 +201,15 @@ public class Sort {
   public int hashCode() {
     return 0x45aaf665 + Arrays.hashCode(fields);
   }
+
+  /** Whether the relevance score is needed to sort documents. */
+  boolean needsScores() {
+    for (SortField sortField : fields) {
+      if (sortField.getType() == SortField.Type.SCORE) {
+        return true;
+      }
+    }
+    return false;
+  }
+
 }

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/TermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/TermQuery.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/TermQuery.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/TermQuery.java Thu May 30 07:53:18 2013
@@ -32,7 +32,6 @@ import org.apache.lucene.index.TermsEnum
 import org.apache.lucene.search.similarities.Similarity.ExactSimScorer;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.ToStringUtils;
 
 /** A Query that matches documents containing a term.
@@ -85,7 +84,7 @@ public class TermQuery extends Query {
       }
       DocsEnum docs = termsEnum.docs(acceptDocs, null);
       assert docs != null;
-      return new TermScorer(this, docs, similarity.exactSimScorer(stats, context), termsEnum.docFreq());
+      return new TermScorer(this, docs, similarity.exactSimScorer(stats, context));
     }
     
     /**

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/TermRangeQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/TermRangeQuery.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/TermRangeQuery.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/TermRangeQuery.java Thu May 30 07:53:18 2013
@@ -19,6 +19,7 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 
+import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.AttributeSource;
@@ -122,9 +123,9 @@ public class TermRangeQuery extends Mult
       }
       buffer.append(includeLower ? '[' : '{');
       // TODO: all these toStrings for queries should just output the bytes, it might not be UTF-8!
-      buffer.append(lowerTerm != null ? ("*".equals(lowerTerm.utf8ToString()) ? "\\*" : lowerTerm.utf8ToString())  : "*");
+      buffer.append(lowerTerm != null ? ("*".equals(Term.toString(lowerTerm)) ? "\\*" : Term.toString(lowerTerm))  : "*");
       buffer.append(" TO ");
-      buffer.append(upperTerm != null ? ("*".equals(upperTerm.utf8ToString()) ? "\\*" : upperTerm.utf8ToString()) : "*");
+      buffer.append(upperTerm != null ? ("*".equals(Term.toString(upperTerm)) ? "\\*" : Term.toString(upperTerm)) : "*");
       buffer.append(includeUpper ? ']' : '}');
       buffer.append(ToStringUtils.boost(getBoost()));
       return buffer.toString();

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/TermScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/TermScorer.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/TermScorer.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/TermScorer.java Thu May 30 07:53:18 2013
@@ -27,7 +27,6 @@ import org.apache.lucene.search.similari
 final class TermScorer extends Scorer {
   private final DocsEnum docsEnum;
   private final Similarity.ExactSimScorer docScorer;
-  private final int docFreq;
   
   /**
    * Construct a <code>TermScorer</code>.
@@ -39,14 +38,11 @@ final class TermScorer extends Scorer {
    * @param docScorer
    *          The </code>Similarity.ExactSimScorer</code> implementation 
    *          to be used for score computations.
-   * @param docFreq
-   *          per-segment docFreq of this term
    */
-  TermScorer(Weight weight, DocsEnum td, Similarity.ExactSimScorer docScorer, int docFreq) {
+  TermScorer(Weight weight, DocsEnum td, Similarity.ExactSimScorer docScorer) {
     super(weight);
     this.docScorer = docScorer;
     this.docsEnum = td;
-    this.docFreq = docFreq;
   }
 
   @Override
@@ -88,22 +84,13 @@ final class TermScorer extends Scorer {
   public int advance(int target) throws IOException {
     return docsEnum.advance(target);
   }
+  
+  @Override
+  public long cost() {
+    return docsEnum.cost();
+  }
 
   /** Returns a string representation of this <code>TermScorer</code>. */
   @Override
   public String toString() { return "scorer(" + weight + ")"; }
-
-  // TODO: benchmark if the specialized conjunction really benefits
-  // from this, or if instead its from sorting by docFreq, or both
-
-  DocsEnum getDocsEnum() {
-    return docsEnum;
-  }
-  
-  // TODO: generalize something like this for scorers?
-  // even this is just an estimation...
-  
-  int getDocFreq() {
-    return docFreq;
-  }
 }

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/TopTermsRewrite.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/TopTermsRewrite.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/TopTermsRewrite.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/TopTermsRewrite.java Thu May 30 07:53:18 2013
@@ -156,7 +156,7 @@ public abstract class TopTermsRewrite<Q 
     
     final Q q = getTopLevelQuery();
     final ScoreTerm[] scoreTerms = stQueue.toArray(new ScoreTerm[stQueue.size()]);
-    ArrayUtil.mergeSort(scoreTerms, scoreTermSortByTermComp);
+    ArrayUtil.timSort(scoreTerms, scoreTermSortByTermComp);
     
     for (final ScoreTerm st : scoreTerms) {
       final Term term = new Term(query.field, st.bytes);

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java Thu May 30 07:53:18 2013
@@ -134,6 +134,15 @@ public class NearSpansOrdered extends Sp
     return matchPayload.isEmpty() == false;
   }
 
+  @Override
+  public long cost() {
+    long minCost = Long.MAX_VALUE;
+    for (int i = 0; i < subSpans.length; i++) {
+      minCost = Math.min(minCost, subSpans[i].cost());
+    }
+    return minCost;
+  }
+
   // inherit javadocs
   @Override
   public boolean next() throws IOException {
@@ -195,7 +204,7 @@ public class NearSpansOrdered extends Sp
 
   /** Advance the subSpans to the same document */
   private boolean toSameDoc() throws IOException {
-    ArrayUtil.mergeSort(subSpansByDoc, spanDocComparator);
+    ArrayUtil.timSort(subSpansByDoc, spanDocComparator);
     int firstIndex = 0;
     int maxDoc = subSpansByDoc[subSpansByDoc.length - 1].doc();
     while (subSpansByDoc[firstIndex].doc() != maxDoc) {

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java Thu May 30 07:53:18 2013
@@ -131,6 +131,11 @@ public class NearSpansUnordered extends 
     }
 
     @Override
+    public long cost() {
+      return spans.cost();
+    }
+
+    @Override
     public String toString() { return spans.toString() + "#" + index; }
   }
 
@@ -267,6 +272,15 @@ public class NearSpansUnordered extends 
 
     return false;
   }
+  
+  @Override
+  public long cost() {
+    long minCost = Long.MAX_VALUE;
+    for (int i = 0; i < subSpans.length; i++) {
+      minCost = Math.min(minCost, subSpans[i].cost());
+    }
+    return minCost;
+  }
 
   @Override
   public String toString() {

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/spans/SpanNotQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/spans/SpanNotQuery.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/spans/SpanNotQuery.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/spans/SpanNotQuery.java Thu May 30 07:53:18 2013
@@ -162,6 +162,11 @@ public class SpanNotQuery extends SpanQu
       }
 
       @Override
+      public long cost() {
+        return includeSpans.cost();
+      }
+
+      @Override
       public String toString() {
           return "spans(" + SpanNotQuery.this.toString() + ")";
         }

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/spans/SpanOrQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/spans/SpanOrQuery.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/spans/SpanOrQuery.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/spans/SpanOrQuery.java Thu May 30 07:53:18 2013
@@ -172,12 +172,14 @@ public class SpanOrQuery extends SpanQue
 
     return new Spans() {
         private SpanQueue queue = null;
+        private long cost;
 
         private boolean initSpanQueue(int target) throws IOException {
           queue = new SpanQueue(clauses.size());
           Iterator<SpanQuery> i = clauses.iterator();
           while (i.hasNext()) {
             Spans spans = i.next().getSpans(context, acceptDocs, termContexts);
+            cost += spans.cost();
             if (   ((target == -1) && spans.next())
                 || ((target != -1) && spans.skipTo(target))) {
               queue.add(spans);
@@ -259,6 +261,11 @@ public class SpanOrQuery extends SpanQue
              :(queue.size()>0?(doc()+":"+start()+"-"+end()):"END"));
         }
 
+      @Override
+      public long cost() {
+        return cost;
+      }
+      
       };
   }