You are viewing a plain text version of this content. The canonical link for it is here.
Posted to java-commits@lucene.apache.org by mi...@apache.org on 2007/12/17 19:05:16 UTC

svn commit: r604949 - in /lucene/java/trunk: ./ src/java/org/apache/lucene/search/ src/java/org/apache/lucene/util/ src/test/org/apache/lucene/util/

Author: mikemccand
Date: Mon Dec 17 10:05:13 2007
New Revision: 604949

URL: http://svn.apache.org/viewvc?rev=604949&view=rev
Log:
LUCENE-1089: add new PriorityQueue.insertWithOverflow method to allow for re-use

Modified:
    lucene/java/trunk/CHANGES.txt
    lucene/java/trunk/src/java/org/apache/lucene/search/FieldSortedHitQueue.java
    lucene/java/trunk/src/java/org/apache/lucene/search/FuzzyQuery.java
    lucene/java/trunk/src/java/org/apache/lucene/search/TopDocCollector.java
    lucene/java/trunk/src/java/org/apache/lucene/search/TopFieldDocCollector.java
    lucene/java/trunk/src/java/org/apache/lucene/util/PriorityQueue.java
    lucene/java/trunk/src/test/org/apache/lucene/util/TestPriorityQueue.java

Modified: lucene/java/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/java/trunk/CHANGES.txt?rev=604949&r1=604948&r2=604949&view=diff
==============================================================================
--- lucene/java/trunk/CHANGES.txt (original)
+++ lucene/java/trunk/CHANGES.txt Mon Dec 17 10:05:13 2007
@@ -82,7 +82,11 @@
 11. LUCENE-1079: DocValues cleanup: constructor now has no params,
     and getInnerArray() now throws UnsupportedOperationException (Doron Cohen)
 
-    
+12. LUCENE-1089: Added PriorityQueue.insertWithOverflow, which returns
+    the Object (if any) that was bumped from the queue to allow
+    re-use.  (Shai Erera via Mike McCandless)
+
+
 Bug fixes
 
  1. LUCENE-933: QueryParser fixed to not produce empty sub 

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/FieldSortedHitQueue.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/FieldSortedHitQueue.java?rev=604949&r1=604948&r2=604949&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/FieldSortedHitQueue.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/FieldSortedHitQueue.java Mon Dec 17 10:05:13 2007
@@ -80,11 +80,16 @@
     return maxscore;
   }
 
+  // Update maxscore.
+  private final void updateMaxScore(FieldDoc fdoc) {
+    maxscore = Math.max(maxscore, fdoc.score);
+  }
+  
   // The signature of this method takes a FieldDoc in order to avoid
   // the unneeded cast to retrieve the score.
   // inherit javadoc
   public boolean insert(FieldDoc fdoc) {
-    maxscore = Math.max(maxscore,fdoc.score);
+    updateMaxScore(fdoc);
     return super.insert(fdoc);
   }
 
@@ -95,6 +100,14 @@
     return insert((FieldDoc)fdoc);
   }
 
+  // This overrides PriorityQueue.insertWithOverflow() so that
+  // updateMaxScore(FieldDoc) that keeps track of the score isn't accidentally
+  // bypassed.
+  public Object insertWithOverflow(Object element) {
+    updateMaxScore((FieldDoc) element);
+    return super.insertWithOverflow(element);
+  }
+  
   /**
    * Returns whether <code>a</code> is less relevant than <code>b</code>.
    * @param a ScoreDoc

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/FuzzyQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/FuzzyQuery.java?rev=604949&r1=604948&r2=604949&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/FuzzyQuery.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/FuzzyQuery.java Mon Dec 17 10:05:13 2007
@@ -104,20 +104,27 @@
     FilteredTermEnum enumerator = getEnum(reader);
     int maxClauseCount = BooleanQuery.getMaxClauseCount();
     ScoreTermQueue stQueue = new ScoreTermQueue(maxClauseCount);
-    
+    ScoreTerm reusableST = null;
+
     try {
       do {
-        float minScore = 0.0f;
         float score = 0.0f;
         Term t = enumerator.term();
         if (t != null) {
           score = enumerator.difference();
-          // terms come in alphabetical order, therefore if queue is full and score
-          // not bigger than minScore, we can skip
-          if(stQueue.size() < maxClauseCount || score > minScore){
-            stQueue.insert(new ScoreTerm(t, score));
-            minScore = ((ScoreTerm)stQueue.top()).score; // maintain minScore
+          if (reusableST == null) {
+            reusableST = new ScoreTerm(t, score);
+          } else if (score >= reusableST.score) {
+            // reusableST holds the last "rejected" entry, so, if
+            // this new score is not better than that, there's no
+            // need to try inserting it
+            reusableST.score = score;
+            reusableST.term = t;
+          } else {
+            continue;
           }
+
+          reusableST = (ScoreTerm) stQueue.insertWithOverflow(reusableST);
         }
       } while (enumerator.next());
     } finally {

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/TopDocCollector.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/TopDocCollector.java?rev=604949&r1=604948&r2=604949&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/TopDocCollector.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/TopDocCollector.java Mon Dec 17 10:05:13 2007
@@ -28,8 +28,8 @@
  * documents are collected.
  **/
 public class TopDocCollector extends HitCollector {
-  private int numHits;
-  private float minScore = 0.0f;
+
+  private ScoreDoc reusableSD;
 
   int totalHits;
   PriorityQueue hq;
@@ -42,7 +42,6 @@
   }
 
   TopDocCollector(int numHits, PriorityQueue hq) {
-    this.numHits = numHits;
     this.hq = hq;
   }
 
@@ -50,10 +49,18 @@
   public void collect(int doc, float score) {
     if (score > 0.0f) {
       totalHits++;
-      if (hq.size() < numHits || score >= minScore) {
-        hq.insert(new ScoreDoc(doc, score));
-        minScore = ((ScoreDoc)hq.top()).score; // maintain minScore
+      if (reusableSD == null) {
+        reusableSD = new ScoreDoc(doc, score);
+      } else if (score >= reusableSD.score) {
+        // reusableSD holds the last "rejected" entry, so, if
+        // this new score is not better than that, there's no
+        // need to try inserting it
+        reusableSD.doc = doc;
+        reusableSD.score = score;
+      } else {
+        return;
       }
+      reusableSD = (ScoreDoc) hq.insertWithOverflow(reusableSD);
     }
   }
 

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/TopFieldDocCollector.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/TopFieldDocCollector.java?rev=604949&r1=604948&r2=604949&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/TopFieldDocCollector.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/TopFieldDocCollector.java Mon Dec 17 10:05:13 2007
@@ -31,6 +31,8 @@
  **/
 public class TopFieldDocCollector extends TopDocCollector {
 
+  private FieldDoc reusableFD;
+
   /** Construct to collect a given number of hits.
    * @param reader the index to be searched
    * @param sort the sort criteria
@@ -45,7 +47,18 @@
   public void collect(int doc, float score) {
     if (score > 0.0f) {
       totalHits++;
-      hq.insert(new FieldDoc(doc, score));
+      if (reusableFD == null)
+        reusableFD = new FieldDoc(doc, score);
+      else {
+        // Whereas TopDocCollector can skip this if the
+        // score is not competitive, we cannot because the
+        // comparators in the FieldSortedHitQueue.lessThan
+        // aren't in general congruent with "higher score
+        // wins"
+        reusableFD.score = score;
+        reusableFD.doc = doc;
+      }
+      reusableFD = (FieldDoc) hq.insertWithOverflow(reusableFD);
     }
   }
 

Modified: lucene/java/trunk/src/java/org/apache/lucene/util/PriorityQueue.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/util/PriorityQueue.java?rev=604949&r1=604948&r2=604949&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/util/PriorityQueue.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/util/PriorityQueue.java Mon Dec 17 10:05:13 2007
@@ -21,9 +21,9 @@
   least element can always be found in constant time.  Put()'s and pop()'s
   require log(size) time. */
 public abstract class PriorityQueue {
-  private Object[] heap;
   private int size;
   private int maxSize;
+  protected Object[] heap;
 
   /** Determines the ordering of objects in this priority queue.  Subclasses
     must define this one method. */
@@ -32,7 +32,12 @@
   /** Subclass constructors must call this. */
   protected final void initialize(int maxSize) {
     size = 0;
-    int heapSize = maxSize + 1;
+    int heapSize;
+    if (0 == maxSize)
+      // We allocate 1 extra to avoid if statement in top()
+      heapSize = 2;
+    else
+      heapSize = maxSize + 1;
     heap = new Object[heapSize];
     this.maxSize = maxSize;
   }
@@ -54,26 +59,40 @@
    * @param element
    * @return true if element is added, false otherwise.
    */
-  public boolean insert(Object element){
-    if(size < maxSize){
+  public boolean insert(Object element) {
+    return insertWithOverflow(element) != element;
+  }
+
+  /**
+   * insertWithOverflow() is the same as insert() except its
+   * return value: it returns the object (if any) that was
+   * dropped off the heap because it was full. This can be
+   * the given parameter (in case it is smaller than the
+   * full heap's minimum, and couldn't be added), or another
+   * object that was previously the smallest value in the
+   * heap and now has been replaced by a larger one, or null
+   * if the queue wasn't yet full with maxSize elements.
+   */
+  public Object insertWithOverflow(Object element) {
+    if (size < maxSize) {
       put(element);
-      return true;
-    }
-    else if(size > 0 && !lessThan(element, top())){
+      return null;
+    } else if (size > 0 && !lessThan(element, heap[1])) {
+      Object ret = heap[1];
       heap[1] = element;
       adjustTop();
-      return true;
+      return ret;
+    } else {
+      return element;
     }
-    else
-      return false;
-   }
+  }
 
   /** Returns the least element of the PriorityQueue in constant time. */
   public final Object top() {
-    if (size > 0)
-      return heap[1];
-    else
-      return null;
+    // We don't need to check size here: if maxSize is 0,
+    // then heap is length 2 array with both entries null.
+    // If size is 0 then heap[1] is already null.
+    return heap[1];
   }
 
   /** Removes and returns the least element of the PriorityQueue in log(size)
@@ -101,7 +120,6 @@
     downHeap();
   }
 
-
   /** Returns the number of elements currently stored in the PriorityQueue. */
   public final int size() {
     return size;
@@ -140,7 +158,7 @@
       j = i << 1;
       k = j + 1;
       if (k <= size && lessThan(heap[k], heap[j])) {
-	j = k;
+        j = k;
       }
     }
     heap[i] = node;				  // install saved node

Modified: lucene/java/trunk/src/test/org/apache/lucene/util/TestPriorityQueue.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/util/TestPriorityQueue.java?rev=604949&r1=604948&r2=604949&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/util/TestPriorityQueue.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/util/TestPriorityQueue.java Mon Dec 17 10:05:13 2007
@@ -18,7 +18,6 @@
  */
 
 import java.util.Random;
-import org.apache.lucene.util.LuceneTestCase;
 
 public class TestPriorityQueue
     extends LuceneTestCase
@@ -107,4 +106,25 @@
         assertEquals(3, pq.size());
         assertEquals(3, ((Integer) pq.top()).intValue());
     }
+    
+  public void testInsertWithOverflow() {
+    int size = 4;
+    PriorityQueue pq = new IntegerQueue(size);
+    Integer i1 = new Integer(2);
+    Integer i2 = new Integer(3);
+    Integer i3 = new Integer(1);
+    Integer i4 = new Integer(5);
+    Integer i5 = new Integer(7);
+    Integer i6 = new Integer(1);
+    
+    assertNull(pq.insertWithOverflow(i1));
+    assertNull(pq.insertWithOverflow(i2));
+    assertNull(pq.insertWithOverflow(i3));
+    assertNull(pq.insertWithOverflow(i4));
+    assertTrue(pq.insertWithOverflow(i5) == i3); // i3 should have been dropped
+    assertTrue(pq.insertWithOverflow(i6) == i6); // i6 should not have been inserted
+    assertEquals(size, pq.size());
+    assertEquals(2, ((Integer) pq.top()).intValue());
+  }
+  
 }