You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ro...@apache.org on 2013/02/11 23:17:57 UTC

svn commit: r1444980 - in /lucene/dev/branches/LUCENE-2878/lucene/core/src: java/org/apache/lucene/search/ java/org/apache/lucene/search/intervals/ test/org/apache/lucene/search/intervals/

Author: romseygeek
Date: Mon Feb 11 22:17:57 2013
New Revision: 1444980

URL: http://svn.apache.org/r1444980
Log:
LUCENE-2878: DisjunctionScorers implement next/start/endPosition()

Added:
    lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/PositionQueue.java   (with props)
Modified:
    lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java
    lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BooleanScorer2.java
    lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java
    lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java
    lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java
    lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/Scorer.java
    lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/TermScorer.java
    lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/intervals/Interval.java
    lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/search/intervals/TestBasicIntervals.java

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java?rev=1444980&r1=1444979&r2=1444980&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java Mon Feb 11 22:17:57 2013
@@ -342,7 +342,10 @@ public class BooleanQuery extends Query 
       // return BooleanScorer for topScorer):
 
       // Check if we can return a BooleanScorer
-      if (!scoreDocsInOrder && flags == PostingFeatures.DOCS_AND_FREQS && topScorer && required.size() == 0) {
+      // nocommit - we need to somehow detect if we need to iterate positions
+      // for now, always return BS2
+      boolean needsPositions = true;
+      if (!needsPositions && !scoreDocsInOrder && flags == PostingFeatures.DOCS_AND_FREQS && topScorer && required.size() == 0) {
         return new BooleanScorer(this, disableCoord, minNrShouldMatch, optional, prohibited, maxCoord);
       }
       

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BooleanScorer2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BooleanScorer2.java?rev=1444980&r1=1444979&r2=1444980&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BooleanScorer2.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BooleanScorer2.java Mon Feb 11 22:17:57 2013
@@ -17,16 +17,15 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
+import org.apache.lucene.search.BooleanQuery.BooleanWeight;
+import org.apache.lucene.search.intervals.IntervalIterator;
+import org.apache.lucene.search.similarities.Similarity;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 
-import org.apache.lucene.search.BooleanClause.Occur;
-import org.apache.lucene.search.BooleanQuery.BooleanWeight;
-import org.apache.lucene.search.intervals.IntervalIterator;
-import org.apache.lucene.search.similarities.Similarity;
-
 /* See the description in BooleanScorer.java, comparing
  * BooleanScorer & BooleanScorer2 */
 
@@ -334,6 +333,31 @@ class BooleanScorer2 extends Scorer {
   }
 
   @Override
+  public int nextPosition() throws IOException {
+    return countingSumScorer.nextPosition();
+  }
+
+  @Override
+  public int startPosition() throws IOException {
+    return countingSumScorer.startPosition();
+  }
+
+  @Override
+  public int endPosition() throws IOException {
+    return countingSumScorer.endPosition();
+  }
+
+  @Override
+  public int startOffset() throws IOException {
+    return countingSumScorer.startOffset();
+  }
+
+  @Override
+  public int endOffset() throws IOException {
+    return countingSumScorer.endOffset();
+  }
+
+  @Override
   public Collection<ChildScorer> getChildren() {
     ArrayList<ChildScorer> children = new ArrayList<ChildScorer>();
     for (Scorer s : optionalScorers) {

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java?rev=1444980&r1=1444979&r2=1444980&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java Mon Feb 11 22:17:57 2013
@@ -70,8 +70,9 @@ class DisjunctionMaxScorer extends Disju
         }
       }
     }
-    
-    return doc = subScorers[0].docID();
+    doc = subScorers[0].docID();
+    posQueue.advanceTo(doc);
+    return doc;
   }
 
   @Override
@@ -134,7 +135,9 @@ class DisjunctionMaxScorer extends Disju
         }
       }
     }
-    return doc = subScorers[0].docID();
+    doc = subScorers[0].docID();
+    posQueue.advanceTo(doc);
+    return doc;
   }
   
   @Override

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java?rev=1444980&r1=1444979&r2=1444980&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java Mon Feb 11 22:17:57 2013
@@ -17,6 +17,7 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 
@@ -27,11 +28,13 @@ import java.util.Collection;
 abstract class DisjunctionScorer extends Scorer {
   protected final Scorer subScorers[];
   protected int numScorers;
+  protected PositionQueue posQueue;
   
   protected DisjunctionScorer(Weight weight, Scorer subScorers[], int numScorers) {
     super(weight);
     this.subScorers = subScorers;
     this.numScorers = numScorers;
+    this.posQueue = new PositionQueue(subScorers);
     heapify();
   }
   
@@ -105,4 +108,29 @@ abstract class DisjunctionScorer extends
     }
     return children;
   }
+
+  @Override
+  public int nextPosition() throws IOException {
+    return posQueue.nextPosition();
+  }
+
+  @Override
+  public int startPosition() throws IOException {
+    return posQueue.startPosition();
+  }
+
+  @Override
+  public int endPosition() throws IOException {
+    return posQueue.endPosition();
+  }
+
+  @Override
+  public int startOffset() throws IOException {
+    return posQueue.startOffset();
+  }
+
+  @Override
+  public int endOffset() throws IOException {
+    return posQueue.endOffset();
+  }
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java?rev=1444980&r1=1444979&r2=1444980&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java Mon Feb 11 22:17:57 2013
@@ -17,13 +17,13 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
-import java.util.List;
-import java.io.IOException;
-
 import org.apache.lucene.search.intervals.ConjunctionIntervalIterator;
 import org.apache.lucene.search.intervals.DisjunctionIntervalIterator;
 import org.apache.lucene.search.intervals.IntervalIterator;
 
+import java.io.IOException;
+import java.util.List;
+
 /** 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. 
  */
@@ -89,7 +89,7 @@ class DisjunctionSumScorer extends Disju
         break;
       }
     }
-    
+    posQueue.advanceTo(doc);
     return doc;
   }
   
@@ -163,6 +163,7 @@ class DisjunctionSumScorer extends Disju
     }
     
     afterNext();
+    posQueue.advanceTo(doc);
 
     if (nrMatchers >= minimumNrMatchers) {
       return doc;

Added: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/PositionQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/PositionQueue.java?rev=1444980&view=auto
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/PositionQueue.java (added)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/PositionQueue.java Mon Feb 11 22:17:57 2013
@@ -0,0 +1,108 @@
+package org.apache.lucene.search;
+
+import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.search.intervals.Interval;
+import org.apache.lucene.util.PriorityQueue;
+
+import java.io.IOException;
+
+/**
+ * Copyright (c) 2013 Lemur Consulting Ltd.
+ * <p/>
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+public class PositionQueue extends PriorityQueue<PositionQueue.ScorerRef> {
+
+  class ScorerRef {
+
+    public final Scorer scorer;
+    public Interval interval = new Interval();
+
+    public ScorerRef(Scorer scorer) {
+      this.scorer = scorer;
+    }
+
+    public int nextPosition() throws IOException {
+      if (scorer.docID() == DocsEnum.NO_MORE_DOCS || scorer.docID() != docId
+            || scorer.nextPosition() == DocsEnum.NO_MORE_POSITIONS)
+        interval.update(Interval.EXHAUSTED_INTERVAL);
+      else
+        interval.update(this.scorer);
+      return interval.begin;
+    }
+
+  }
+
+  boolean positioned = false;
+  Interval current = new Interval();
+  int docId = -1;
+
+  public PositionQueue(Scorer[] subScorers) {
+    super(subScorers.length);
+    for (int i = 0; i < subScorers.length; i++) {
+      add(new ScorerRef(subScorers[i]));
+    }
+  }
+
+  public int nextPosition() throws IOException {
+    if (!positioned) {
+      for (Object scorerRef : getHeapArray()) {
+        if (scorerRef != null)
+          ((ScorerRef) scorerRef).nextPosition();
+      }
+      positioned = true;
+      updateTop();
+      current.update(top().interval);
+      return current.begin;
+    };
+    if (current.begin == DocsEnum.NO_MORE_POSITIONS)
+      return DocsEnum.NO_MORE_POSITIONS;
+    top().nextPosition();
+    updateTop();
+    current.update(top().interval);
+    return current.begin;
+  }
+
+  @Override
+  protected boolean lessThan(ScorerRef a, ScorerRef b) {
+    if (a.scorer.docID() < b.scorer.docID())
+      return true;
+    if (a.scorer.docID() > b.scorer.docID())
+      return false;
+    return a.interval.begin < b.interval.begin;
+  }
+
+  /**
+   * Must be called after the scorers have been advanced
+   */
+  public void advanceTo(int doc) {
+    positioned = false;
+    this.docId = doc;
+  }
+
+  public int startPosition() throws IOException {
+    return current.begin;
+  }
+
+  public int endPosition() throws IOException {
+    return current.end;
+  }
+
+  public int startOffset() throws IOException {
+    return current.offsetBegin;
+  }
+
+  public int endOffset() throws IOException {
+    return current.offsetEnd;
+  }
+}

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/Scorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/Scorer.java?rev=1444980&r1=1444979&r2=1444980&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/Scorer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/Scorer.java Mon Feb 11 22:17:57 2013
@@ -152,6 +152,15 @@ public abstract class Scorer extends Doc
   public int nextPosition() throws IOException {
     throw new UnsupportedOperationException("nextPosition() is not implemented on " + this.getClass());
   }
+
+  @Override
+  public String toString() {
+    try {
+      return String.format("%d:%d(%d)->%d(%d)", docID(), startPosition(), startOffset(), endPosition(), endOffset());
+    } catch (IOException e) {
+      return String.format("Cannot retrieve position due to IOException");
+    }
+  }
   
   /** Returns child sub-scorers
    * @lucene.experimental */

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/TermScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/TermScorer.java?rev=1444980&r1=1444979&r2=1444980&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/TermScorer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/TermScorer.java Mon Feb 11 22:17:57 2013
@@ -124,7 +124,7 @@ final class TermScorer extends Scorer {
 
   /** Returns a string representation of this <code>TermScorer</code>. */
   @Override
-  public String toString() { return "scorer(" + weight + ")"; }
+  public String toString() { return "scorer(" + weight + ")[" + super.toString() + "]"; }
   
   @Override
   public IntervalIterator intervals(boolean collectIntervals) throws IOException {

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/intervals/Interval.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/intervals/Interval.java?rev=1444980&r1=1444979&r2=1444980&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/intervals/Interval.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/intervals/Interval.java Mon Feb 11 22:17:57 2013
@@ -41,6 +41,9 @@ public class Interval implements Cloneab
   /** An interval that will always compare as less than any other interval */
   public static final Interval INFINITE_INTERVAL = new Interval();
 
+  /** An interval that will always compare as more than any other interval */
+  public static final Interval EXHAUSTED_INTERVAL = new Interval(Integer.MAX_VALUE, Integer.MAX_VALUE, -1, -1);
+
   /**
    * Constructs a new Interval
    * @param begin the start position
@@ -190,4 +193,11 @@ public class Interval implements Cloneab
     begin = scorer.startPosition();
     end = scorer.endPosition();
   }
+
+  public void update(Interval interval) {
+    this.begin = interval.begin;
+    this.end = interval.end;
+    this.offsetBegin = interval.offsetBegin;
+    this.offsetEnd = interval.offsetEnd;
+  }
 }
\ No newline at end of file

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/search/intervals/TestBasicIntervals.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/search/intervals/TestBasicIntervals.java?rev=1444980&r1=1444979&r2=1444980&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/search/intervals/TestBasicIntervals.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/search/intervals/TestBasicIntervals.java Mon Feb 11 22:17:57 2013
@@ -49,7 +49,34 @@ public class TestBasicIntervals extends 
       "u2 xx u1 u2",//8
       "u1 u2 xx u2",//9
       "u2 u1 xx u2",//10
-      "t1 t2 t1 t3 t2 t3"};//11
+      "t1 t2 t1 t3 t2 t3",//11
+      "v1 v2 v3",//12
+      "v1 v3 v2 v3 v4",//13
+      "v4 v2 v2 v4",//14
+      "v3 v4 v3"};//15
+
+  public void testSimpleTerm() throws IOException {
+    Query q = makeTermQuery("u2");
+    checkIntervals(q, searcher, new int[][]{
+        { 4, 0, 0, 1, 1 },
+        { 5, 0, 0, 2, 2 },
+        { 6, 0, 0, 1, 1 },
+        { 7, 0, 0, 2, 2 },
+        { 8, 0, 0, 3, 3 },
+        { 9, 1, 1, 3, 3 },
+        { 10, 0, 0, 3, 3 }
+    });
+  }
+
+  public void testBasicDisjunction() throws IOException {
+    Query q = makeOrQuery(makeTermQuery("v3"), makeTermQuery("v2"));
+    checkIntervals(q, searcher, new int[][]{
+        { 12, 1, 1, 2, 2 },
+        { 13, 1, 1, 2, 2, 3, 3 },
+        { 14, 1, 1, 2, 2 },
+        { 15, 0, 0, 2, 2 }
+    });
+  }
 
   /*