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 2014/12/15 18:12:37 UTC

svn commit: r1643113 - in /lucene/dev/branches/lucene2878/lucene: core/src/java/org/apache/lucene/search/ core/src/test/org/apache/lucene/search/ core/src/test/org/apache/lucene/search/posfilter/ highlighter/src/java/org/apache/lucene/search/highlight/...

Author: romseygeek
Date: Wed Dec  3 13:40:16 2014
New Revision: 1643113

URL: http://svn.apache.org/r1643113
Log:
Checkpoint sloppyphrasescorer fixes

Modified:
    lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java
    lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java
    lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/PhrasePositions.java
    lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/PhraseQueue.java
    lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java
    lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/search/TestConstantScoreQuery.java
    lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/search/posfilter/TestPhraseQueryPositions.java
    lucene/dev/branches/lucene2878/lucene/highlighter/src/java/org/apache/lucene/search/highlight/positions/IntervalTokenStream.java
    lucene/dev/branches/lucene2878/lucene/highlighter/src/test/org/apache/lucene/search/highlight/positions/IntervalHighlighterTest.java

Modified: lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java?rev=1643113&r1=1643112&r2=1643113&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java (original)
+++ lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java Wed Dec  3 13:40:16 2014
@@ -153,23 +153,26 @@ public class ConstantScoreQuery extends
 
     @Override
     public Scorer scorer(LeafReaderContext context, int flags, Bits acceptDocs) throws IOException {
-      final DocIdSetIterator disi;
       if (filter != null) {
         assert query == null;
         final DocIdSet dis = filter.getDocIdSet(context, acceptDocs);
         if (dis == null) {
           return null;
         }
-        disi = dis.iterator();
+        final DocIdSetIterator disi = dis.iterator();
+        if (disi == null)
+          return null;
+        return new ConstantDocIdSetIteratorScorer(disi, this, queryWeight);
       } else {
         assert query != null && innerWeight != null;
-        disi = innerWeight.scorer(context, flags, acceptDocs);
+        Scorer scorer = innerWeight.scorer(context, flags, acceptDocs);
+        if (scorer == null) {
+          return null;
+        }
+        return new ConstantScoreScorer(scorer, queryWeight);
       }
 
-      if (disi == null) {
-        return null;
-      }
-      return new ConstantScorer(disi, this, queryWeight);
+
     }
 
     @Override
@@ -223,17 +226,46 @@ public class ConstantScoreQuery extends
         @Override
         public void setScorer(Scorer scorer) throws IOException {
           // we must wrap again here, but using the scorer passed in as parameter:
-          in.setScorer(new ConstantScorer(scorer, weight, theScore));
+          in.setScorer(new ConstantScoreScorer(scorer, theScore));
         }
       };
     }
   }
 
-  protected class ConstantScorer extends Scorer {
+  protected class ConstantScoreScorer extends FilterScorer {
+
+    private final float score;
+
+    public ConstantScoreScorer(Scorer wrapped, float score) {
+      super(wrapped);
+      this.score = score;
+    }
+
+    @Override
+    public int freq() throws IOException {
+      return 1;
+    }
+
+    @Override
+    public float score() throws IOException {
+      return score;
+    }
+
+    @Override
+    public Collection<ChildScorer> getChildren() {
+      if (query != null) {
+        return Collections.singletonList(new ChildScorer(in, "constant"));
+      } else {
+        return Collections.emptyList();
+      }
+    }
+  }
+
+  protected class ConstantDocIdSetIteratorScorer extends Scorer {
     final DocIdSetIterator docIdSetIterator;
     final float theScore;
 
-    public ConstantScorer(DocIdSetIterator docIdSetIterator, Weight w, float theScore) {
+    public ConstantDocIdSetIteratorScorer(DocIdSetIterator docIdSetIterator, Weight w, float theScore) {
       super(w);
       this.theScore = theScore;
       this.docIdSetIterator = docIdSetIterator;
@@ -260,11 +292,9 @@ public class ConstantScoreQuery extends
       return 1;
     }
 
-    // nocommit maybe split into ConstantFilterScorer and ConstantQueryScorer to make these available?
-
     @Override
     public int nextPosition() throws IOException {
-      return -1;
+      return NO_MORE_POSITIONS;
     }
 
     @Override

Modified: lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java?rev=1643113&r1=1643112&r2=1643113&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java (original)
+++ lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java Wed Dec  3 13:40:16 2014
@@ -231,7 +231,7 @@ final class ExactPhraseScorer extends Sc
 
   private boolean findNextMatches() throws IOException {
 
-    // TODO: we could fold in chunkStart into offset and
+    // TODO: we could fold in chunkStart into phraseOffset and
     // save one subtract per pos incr
 
     boolean exhausted = false;

Modified: lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/PhrasePositions.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/PhrasePositions.java?rev=1643113&r1=1643112&r2=1643113&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/PhrasePositions.java (original)
+++ lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/PhrasePositions.java Wed Dec  3 13:40:16 2014
@@ -17,19 +17,19 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
+import java.io.IOException;
+
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.Term;
 
-import java.io.IOException;
-
 /**
- * Position of a term in a document that takes into account the term offset within the phrase. 
+ * Position of a term in a document that takes into account the term phraseOffset within the phrase.
  */
 final class PhrasePositions {
   int doc;              // current doc
   int position;         // position in doc
-  int count;            // remaining pos in this doc
-  int offset;           // position in phrase
+  //int count;            // remaining pos in this doc
+  int phraseOffset;           // position in phrase
   final int ord;                                  // unique across all PhrasePositions instances
   final DocsEnum postings;            // stream of docs & positions
   PhrasePositions next;                           // used to make lists
@@ -39,7 +39,7 @@ final class PhrasePositions {
 
   PhrasePositions(DocsEnum postings, int o, int ord, Term[] terms) {
     this.postings = postings;
-    offset = o;
+    phraseOffset = o;
     this.ord = ord;
     this.terms = terms;
   }
@@ -62,28 +62,37 @@ final class PhrasePositions {
   }
 
   final void firstPosition() throws IOException {
-    count = postings.freq();  // read first pos
     nextPosition();
   }
 
   /**
    * Go to next location of this term current document, and set 
-   * <code>position</code> as <code>location - offset</code>, so that a 
+   * <code>position</code> as <code>location - phraseOffset</code>, so that a
    * matching exact phrase is easily identified when all PhrasePositions 
    * have exactly the same <code>position</code>.
    */
   final boolean nextPosition() throws IOException {
-    if (count-- > 0) {  // read subsequent pos's
-      position = postings.nextPosition() - offset;
-      return true;
-    } else
+    int nextPos = postings.nextPosition();
+    if (nextPos == DocsEnum.NO_MORE_POSITIONS) {
+      position = nextPos;
       return false;
+    }
+    position = nextPos - phraseOffset;
+    return true;
+  }
+
+  public final int startOffset() throws IOException {
+    return postings.startOffset();
+  }
+
+  public final int endOffset() throws IOException {
+    return postings.endOffset();
   }
   
   /** for debug purposes */
   @Override
   public String toString() {
-    String s = "d:"+doc+" offset:"+offset+" position:"+position+" c:"+count;
+    String s = "d:"+doc+" phraseOffset:"+ phraseOffset +" position:"+position;
     if (rptGroup >=0 ) {
       s += " rpt:"+rptGroup+",i"+rptInd;
     }

Modified: lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/PhraseQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/PhraseQueue.java?rev=1643113&r1=1643112&r2=1643113&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/PhraseQueue.java (original)
+++ lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/PhraseQueue.java Wed Dec  3 13:40:16 2014
@@ -29,11 +29,11 @@ final class PhraseQueue extends Priority
     if (pp1.doc == pp2.doc) 
       if (pp1.position == pp2.position)
         // same doc and pp.position, so decide by actual term positions. 
-        // rely on: pp.position == tp.position - offset. 
-        if (pp1.offset == pp2.offset) {
+        // rely on: pp.position == tp.position - phraseOffset.
+        if (pp1.phraseOffset == pp2.phraseOffset) {
           return pp1.ord < pp2.ord;
         } else {
-          return pp1.offset < pp2.offset;
+          return pp1.phraseOffset < pp2.phraseOffset;
         }
       else {
         return pp1.position < pp2.position;

Modified: lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java?rev=1643113&r1=1643112&r2=1643113&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java (original)
+++ lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java Wed Dec  3 13:40:16 2014
@@ -49,10 +49,9 @@ final class SloppyPhraseScorer extends S
   private boolean hasRpts; // flag indicating that there are repetitions (as checked in first candidate doc)
   private boolean checkedRpts; // flag to only check for repetitions in first candidate doc
   private boolean hasMultiTermRpts; //  
-  private PhrasePositions[][] rptGroups; // in each group are PPs that repeats each other (i.e. same term), sorted by (query) offset 
+  private PhrasePositions[][] rptGroups; // in each group are PPs that repeats each other (i.e. same term), sorted by (query) phraseOffset
   private PhrasePositions[] rptStack; // temporary stack for switching colliding repeating pps 
   
-  private int numMatches;
   private final long cost;
   
   SloppyPhraseScorer(Weight weight, PhraseQuery.PostingsAndFreq[] postings,
@@ -62,15 +61,15 @@ final class SloppyPhraseScorer extends S
     this.postings = postings;
     this.slop = slop;
     this.numPostings = postings==null ? 0 : postings.length;
-    pq = new PhraseQueue(postings.length);
-    // min(cost)
-    cost = postings[0].postings.cost();
+    pq = new PhraseQueue(this.numPostings);
     // 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.
+    // reflects the phrase phraseOffset: pp.pos = tp.pos - phraseOffset.
     // this allows to easily identify a matching (exact) phrase 
     // when all PhrasePositions have exactly the same position.
-    if (postings.length > 0) {
+    if (postings != null && postings.length > 0) {
+      // min(cost)
+      cost = postings[0].postings.cost();
       min = new PhrasePositions(postings[0].postings, postings[0].position, 0, postings[0].terms);
       max = min;
       max.doc = -1;
@@ -82,11 +81,16 @@ final class SloppyPhraseScorer extends S
       }
       max.next = min; // make it cyclic for easier manipulation
     }
+    else {
+      cost = 0;
+    }
   }
 
   private int matchLength;
   private int startpos = -1;
   private int endpos = -1;
+  private int startoffset = -1;
+  private int endoffset = -1;
 
   @Override
   public int nextPosition() throws IOException {
@@ -101,13 +105,15 @@ final class SloppyPhraseScorer extends S
     PhrasePositions top = pq.pop();
     matchLength = end - top.position;
     int next = pq.top().position;
-    int pos = top.position + top.offset;
+    int pos = top.position + top.phraseOffset;
+    int startoffset = top.startOffset();
+    int endoffset = top.endOffset();
     while (advancePP(top)) {
       if (hasRpts && !advanceRpts(top))
         break; // pps exhausted
       if (top.position > next) { // done minimizing current match-length
         if (matchLength <= slop) {
-          setSpan(pos);
+          setSpan(pos, startoffset, endoffset);
           pq.add(top);
           return startpos;
         }
@@ -115,18 +121,22 @@ final class SloppyPhraseScorer extends S
         top = pq.pop();
         next = pq.top().position;
         matchLength = end - top.position;
-        pos = top.position + top.offset;
+        pos = top.position + top.phraseOffset;
+        startoffset = top.startOffset();
+        endoffset = top.endOffset();
       }
       else {
         int matchLength2 = end - top.position;
-        pos = top.position + top.offset;
+        pos = top.position + top.phraseOffset;
+        startoffset = top.startOffset();
+        endoffset = top.endOffset();
         if (matchLength2 < matchLength)
           matchLength = matchLength2;
       }
     }
 
     if (matchLength <= slop) {
-      setSpan(pos);
+      setSpan(pos, startoffset, endoffset);
       return startpos;
     }
 
@@ -134,15 +144,19 @@ final class SloppyPhraseScorer extends S
 
   }
 
-  private void setSpan(int topPos) {
+  private void setSpan(int topPos, int topStartOffset, int topEndOffset) throws IOException {
     startpos = topPos;
     endpos = topPos;
+    startoffset = topStartOffset;
+    endoffset = topEndOffset;
     for (Object o : pq.getPPs()) {
       if (o == null)
         continue;
       PhrasePositions pp = (PhrasePositions) o;
-      startpos = Math.min(startpos, pp.position + pp.offset);
-      endpos = Math.max(endpos, pp.position + pp.offset);
+      startpos = Math.min(startpos, pp.position + pp.phraseOffset);
+      startoffset = Math.min(startoffset, pp.startOffset());
+      endpos = Math.max(endpos, pp.position + pp.phraseOffset);
+      endoffset = Math.max(endoffset, pp.endOffset());
     }
   }
 
@@ -213,10 +227,10 @@ final class SloppyPhraseScorer extends S
     return true;
   }
 
-  /** compare two pps, but only by position and offset */
+  /** compare two pps, but only by position and phraseOffset */
   private PhrasePositions lesser(PhrasePositions pp, PhrasePositions pp2) {
     if (pp.position < pp2.position ||
-        (pp.position == pp2.position && pp.offset < pp2.offset)) {
+        (pp.position == pp2.position && pp.phraseOffset < pp2.phraseOffset)) {
       return pp;
     }
     return pp2;
@@ -305,7 +319,7 @@ final class SloppyPhraseScorer extends S
     }
   }
 
-  /** At initialization (each doc), each repetition group is sorted by (query) offset.
+  /** At initialization (each doc), each repetition group is sorted by (query) phraseOffset.
    * This provides the start condition: no collisions.
    * <p>Case 1: no multi-term repeats<br>
    * It is sufficient to advance each pp in the group by one less than its group index.
@@ -325,7 +339,7 @@ final class SloppyPhraseScorer extends S
           int k;
           while((k=collide(pp)) >= 0) {
             PhrasePositions pp2 = lesser(pp, rg[k]);
-            if (!advancePP(pp2)) {  // at initialization always advance pp with higher offset
+            if (!advancePP(pp2)) {  // at initialization always advance pp with higher phraseOffset
               return false; // exhausted
             }
             if (pp2.rptInd < i) { // should not happen?
@@ -383,14 +397,14 @@ final class SloppyPhraseScorer extends S
     return true; // PPs available
   }
 
-  /** sort each repetition group by (query) offset. 
+  /** sort each repetition group by (query) phraseOffset.
    * Done only once (at first doc) and allows to initialize faster for each doc. */
   private void sortRptGroups(ArrayList<ArrayList<PhrasePositions>> rgs) {
     rptGroups = new PhrasePositions[rgs.size()][];
     Comparator<PhrasePositions> cmprtr = new Comparator<PhrasePositions>() {
       @Override
       public int compare(PhrasePositions pp1, PhrasePositions pp2) {
-        return pp1.offset - pp2.offset;
+        return pp1.phraseOffset - pp2.phraseOffset;
       }
     };
     for (int i=0; i<rptGroups.length; i++) {
@@ -417,7 +431,7 @@ final class SloppyPhraseScorer extends S
           PhrasePositions pp2 = rpp[j];
           if (
               pp2.rptGroup >=0        // already marked as a repetition
-              || pp2.offset == pp.offset // not a repetition: two PPs are originally in same offset in the query! 
+              || pp2.phraseOffset == pp.phraseOffset // not a repetition: two PPs are originally in same phraseOffset in the query!
               || tpPos(pp2) != tpPos) {  // not a repetition
             continue; 
           }
@@ -461,9 +475,9 @@ final class SloppyPhraseScorer extends S
     return res;
   }
 
-  /** Actual position in doc of a PhrasePosition, relies on that position = tpPos - offset) */
-  private final int tpPos(PhrasePositions pp) {
-    return pp.position + pp.offset;
+  /** Actual position in doc of a PhrasePosition, relies on that position = tpPos - phraseOffset) */
+  private int tpPos(PhrasePositions pp) {
+    return pp.position + pp.phraseOffset;
   }
 
   /** find repeating terms and assign them ordinal values */
@@ -473,7 +487,7 @@ final class SloppyPhraseScorer extends S
     for (PhrasePositions pp=min,prev=null; prev!=max; pp=(prev=pp).next) { // iterate cyclic list: done once handled max
       for (Term t : pp.terms) {
         Integer cnt0 = tcnt.get(t);
-        Integer cnt = cnt0==null ? new Integer(1) : new Integer(1+cnt0.intValue());
+        Integer cnt = cnt0==null ? new Integer(1) : new Integer(1 + cnt0);
         tcnt.put(t, cnt);
         if (cnt==2) {
           tord.put(t,tord.size());
@@ -495,7 +509,7 @@ final class SloppyPhraseScorer extends S
         }
       }
     }
-    return rp.toArray(new PhrasePositions[0]);
+    return rp.toArray(new PhrasePositions[rp.size()]);
   }
   
   /** bit-sets - for each repeating pp, for each of its repeating terms, the term ordinal values is set */
@@ -535,7 +549,7 @@ final class SloppyPhraseScorer extends S
   /** map each term to the single group that contains it */ 
   private HashMap<Term,Integer> termGroups(LinkedHashMap<Term,Integer> tord, ArrayList<FixedBitSet> bb) throws IOException {
     HashMap<Term,Integer> tg = new HashMap<>();
-    Term[] t = tord.keySet().toArray(new Term[0]);
+    Term[] t = tord.keySet().toArray(new Term[tord.size()]);
     for (int i=0; i<bb.size(); i++) { // i is the group no.
       FixedBitSet bits = bb.get(i);
       for (int ord = bits.nextSetBit(0); ord != DocIdSetIterator.NO_MORE_DOCS; ord = ord + 1 >= bits.length() ? DocIdSetIterator.NO_MORE_DOCS : bits.nextSetBit(ord + 1)) {
@@ -586,17 +600,18 @@ final class SloppyPhraseScorer extends S
 
   @Override
   public int startOffset() throws IOException {
-    return -1; // nocommit
+    return startoffset;
   }
 
   @Override
   public int endOffset() throws IOException {
-    return -1; // nocommit
+    return endoffset;
   }
 
+  // TODO : getPayload on spans?
   @Override
   public BytesRef getPayload() throws IOException {
-    return null; // nocommit
+    return null;
   }
 
   @Override

Modified: lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/search/TestConstantScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/search/TestConstantScoreQuery.java?rev=1643113&r1=1643112&r2=1643113&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/search/TestConstantScoreQuery.java (original)
+++ lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/search/TestConstantScoreQuery.java Wed Dec  3 13:40:16 2014
@@ -17,6 +17,8 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
+import java.io.IOException;
+
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.index.IndexReader;
@@ -26,8 +28,6 @@ import org.apache.lucene.search.similari
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
 
-import java.io.IOException;
-
 /** This class only tests some basic functionality in CSQ, the main parts are mostly
  * tested by MultiTermQuery tests, explanations seems to be tested in TestExplanations! */
 public class TestConstantScoreQuery extends LuceneTestCase {
@@ -56,9 +56,9 @@ public class TestConstantScoreQuery exte
       public void setScorer(Scorer scorer) {
         this.scorer = scorer;
         assertEquals("Scorer is implemented by wrong class", scorerClassName, scorer.getClass().getName());
-        if (innerScorerClassName != null && scorer instanceof ConstantScoreQuery.ConstantScorer) {
-          final ConstantScoreQuery.ConstantScorer innerScorer = (ConstantScoreQuery.ConstantScorer) scorer;
-          assertEquals("inner Scorer is implemented by wrong class", innerScorerClassName, innerScorer.docIdSetIterator.getClass().getName());
+        if (innerScorerClassName != null && scorer instanceof ConstantScoreQuery.ConstantScoreScorer) {
+          final ConstantScoreQuery.ConstantScoreScorer innerScorer = (ConstantScoreQuery.ConstantScoreScorer) scorer;
+          assertEquals("inner Scorer is implemented by wrong class", innerScorerClassName, innerScorer.in.getClass().getName());
         }
       }
       
@@ -113,13 +113,13 @@ public class TestConstantScoreQuery exte
       final Query csqbq = new ConstantScoreQuery(bq);
       csqbq.setBoost(17.0f);
       
-      checkHits(searcher, csq1, csq1.getBoost(), ConstantScoreQuery.ConstantScorer.class.getName(), null);
-      checkHits(searcher, csq2, csq2.getBoost(), ConstantScoreQuery.ConstantScorer.class.getName(), ConstantScoreQuery.ConstantScorer.class.getName());
+      checkHits(searcher, csq1, csq1.getBoost(), ConstantScoreQuery.ConstantScoreScorer.class.getName(), null);
+      checkHits(searcher, csq2, csq2.getBoost(), ConstantScoreQuery.ConstantScoreScorer.class.getName(), ConstantScoreQuery.ConstantScoreScorer.class.getName());
       
       // for the combined BQ, the scorer should always be BooleanScorer's BucketScorer, because our scorer supports out-of order collection!
       final String bucketScorerClass = FakeScorer.class.getName();
       checkHits(searcher, bq, csq1.getBoost() + csq2.getBoost(), bucketScorerClass, null);
-      checkHits(searcher, csqbq, csqbq.getBoost(), ConstantScoreQuery.ConstantScorer.class.getName(), bucketScorerClass);
+      checkHits(searcher, csqbq, csqbq.getBoost(), ConstantScoreQuery.ConstantScoreScorer.class.getName(), bucketScorerClass);
     } finally {
       if (reader != null) reader.close();
       if (directory != null) directory.close();

Modified: lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/search/posfilter/TestPhraseQueryPositions.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/search/posfilter/TestPhraseQueryPositions.java?rev=1643113&r1=1643112&r2=1643113&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/search/posfilter/TestPhraseQueryPositions.java (original)
+++ lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/search/posfilter/TestPhraseQueryPositions.java Wed Dec  3 13:40:16 2014
@@ -16,6 +16,8 @@ package org.apache.lucene.search.posfilt
  * limitations under the License.
  */
 
+import java.io.IOException;
+
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.RandomIndexWriter;
@@ -23,8 +25,6 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.search.MultiPhraseQuery;
 import org.apache.lucene.search.PhraseQuery;
 
-import java.io.IOException;
-
 public class TestPhraseQueryPositions extends IntervalTestBase {
   
   protected void addDocs(RandomIndexWriter writer) throws IOException {
@@ -80,6 +80,18 @@ public class TestPhraseQueryPositions ex
     });
   }
 
+  public void testSloppyPhraseQueryWithRepears() throws IOException {
+    PhraseQuery query = new PhraseQuery();
+    query.add(new Term("field", "pease"));
+    query.add(new Term("field", "porridge"));
+    query.add(new Term("field", "pease"));
+    query.setSlop(1);
+    checkIntervals(query, searcher, new int[][]{
+        {0, 0, 3, 3, 6, 31, 34},
+        {1, 0, 3, 3, 6, 31, 34}
+    });
+  }
+
   public void testManyTermSloppyPhraseQuery() throws IOException {
     PhraseQuery query = new PhraseQuery();
     query.add(new Term("field", "pease"));

Modified: lucene/dev/branches/lucene2878/lucene/highlighter/src/java/org/apache/lucene/search/highlight/positions/IntervalTokenStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/highlighter/src/java/org/apache/lucene/search/highlight/positions/IntervalTokenStream.java?rev=1643113&r1=1643112&r2=1643113&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/highlighter/src/java/org/apache/lucene/search/highlight/positions/IntervalTokenStream.java (original)
+++ lucene/dev/branches/lucene2878/lucene/highlighter/src/java/org/apache/lucene/search/highlight/positions/IntervalTokenStream.java Wed Dec  3 13:40:16 2014
@@ -27,8 +27,8 @@ import org.apache.lucene.search.posfilte
 
 /**
  * A TokenStream constructed from a stream of positions and their offsets.
- * The document is segmented into tokens at the start and end offset of each interval.  The intervals
- * are assumed to be non-overlapping.
+ * The document is segmented into tokens at the start and end offset of each interval.  If
+ * an interval overlaps the previous returned interval, it is skipped.
  * 
  * TODO: abstract the dependency on the current PositionOffsetMapper impl; 
  * allow for implementations of position-&gt;offset maps that don't rely on term vectors.
@@ -47,7 +47,10 @@ public class IntervalTokenStream extends
   private final Interval[] positions;
   
   // the index of the current position interval
-  private int pos = -1;
+  private int index = -1;
+
+  // last end offset returned (to avoid overlaps)
+  private int lastEndOffset = -1;
   
   public IntervalTokenStream (String text, Interval[] positions) {
     this.text = text;
@@ -56,14 +59,17 @@ public class IntervalTokenStream extends
   
   @Override
   public final boolean incrementToken() throws IOException {
-    if (++pos >= positions.length)
-      return false;
-    if (positions[pos] == null)
-      return false;
-    int b, e;
-    b = positions[pos].offsetBegin;
-    e = positions[pos].offsetEnd;
-    assert b >=0;
+    int b = -1, e = -1;
+    while (b <= lastEndOffset) {
+      if (++index >= positions.length)
+        return false;
+      if (positions[index] == null)
+        return false;
+      b = positions[index].offsetBegin;
+      e = positions[index].offsetEnd;
+      assert b >= 0;
+    }
+    lastEndOffset = e;
     termAtt.append(text, b, e);
     offsetAtt.setOffset(b, e);
     posIncrAtt.setPositionIncrement(1);

Modified: lucene/dev/branches/lucene2878/lucene/highlighter/src/test/org/apache/lucene/search/highlight/positions/IntervalHighlighterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/highlighter/src/test/org/apache/lucene/search/highlight/positions/IntervalHighlighterTest.java?rev=1643113&r1=1643112&r2=1643113&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/highlighter/src/test/org/apache/lucene/search/highlight/positions/IntervalHighlighterTest.java (original)
+++ lucene/dev/branches/lucene2878/lucene/highlighter/src/test/org/apache/lucene/search/highlight/positions/IntervalHighlighterTest.java Wed Dec  3 13:40:16 2014
@@ -19,7 +19,7 @@ package org.apache.lucene.search.highlig
 import java.io.IOException;
 import java.io.StringReader;
 
-import org.apache.lucene.analysis.Analyzer;
+import com.carrotsearch.randomizedtesting.annotations.Seed;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.MockTokenFilter;
 import org.apache.lucene.analysis.MockTokenizer;
@@ -52,28 +52,19 @@ import org.apache.lucene.search.posfilte
 import org.apache.lucene.search.posfilter.OrderedNearQuery;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
-import org.junit.Ignore;
 
-/**
- * TODO: FIX THIS TEST Phrase and Span Queries positions callback API
- */
-@SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom"})
+@Seed("2C0AB6BC65255FAA")
 public class IntervalHighlighterTest extends LuceneTestCase {
   
   protected final static String F = "f";
-  protected Analyzer analyzer;
   protected Directory dir;
   protected IndexSearcher searcher;
-  private IndexWriterConfig iwc;
-  
+
   private static final String PORRIDGE_VERSE = "Pease porridge hot! Pease porridge cold! Pease porridge in the pot nine days old! Some like it hot, some"
       + " like it cold, Some like it in the pot nine days old! Pease porridge hot! Pease porridge cold!";
   
   public void setUp() throws Exception {
     super.setUp();
-    iwc = newIndexWriterConfig(new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false)).setOpenMode(OpenMode.CREATE);
-    analyzer = iwc.getAnalyzer();
     dir = newDirectory();
   }
   
@@ -86,8 +77,9 @@ public class IntervalHighlighterTest ext
   }
   
   // make several docs
-  protected void insertDocs(Analyzer analyzer, String... values)
+  protected void insertDocs(String... values)
       throws Exception {
+    IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false)).setOpenMode(OpenMode.CREATE);
     IndexWriter writer = new IndexWriter(dir, iwc);
     FieldType type = new FieldType();
     type.setTokenized(true);
@@ -184,7 +176,7 @@ public class IntervalHighlighterTest ext
   }
   
   public void testTerm() throws Exception {
-    insertDocs(analyzer, "This is a test test");
+    insertDocs("This is a test test");
     String frags[] = doSearch(termQuery("test"));
     assertEquals("This is a <B>test</B> <B>test</B>", frags[0]);
     close();
@@ -195,14 +187,14 @@ public class IntervalHighlighterTest ext
         + "Let us see what happens to long in this case.";
     String gold = "this is some <B>long</B> text.  It has the word <B>long</B> in many places.  In fact, it has <B>long</B> on some different fragments.  "
         + "Let us see what happens to <B>long</B> in this case.";
-    insertDocs(analyzer, input);
+    insertDocs(input);
     String frags[] = doSearch(termQuery("long"), input.length());
     assertEquals(gold, frags[0]);
     close();
   }
   
   public void testBooleanAnd() throws Exception {
-    insertDocs(analyzer, "This is a test");
+    insertDocs("This is a test");
     BooleanQuery bq = new BooleanQuery();
     bq.add(new BooleanClause(termQuery("This"), Occur.MUST));
     bq.add(new BooleanClause(termQuery("test"), Occur.MUST));
@@ -212,7 +204,7 @@ public class IntervalHighlighterTest ext
   }
   
   public void testConstantScore() throws Exception {
-    insertDocs(analyzer, "This is a test");
+    insertDocs("This is a test");
     BooleanQuery bq = new BooleanQuery();
     bq.add(new BooleanClause(termQuery("This"), Occur.MUST));
     bq.add(new BooleanClause(termQuery("test"), Occur.MUST));
@@ -222,7 +214,7 @@ public class IntervalHighlighterTest ext
   }
   
   public void testBooleanAndOtherOrder() throws Exception {
-    insertDocs(analyzer, "This is a test");
+    insertDocs("This is a test");
     BooleanQuery bq = new BooleanQuery();
     bq.add(new BooleanClause(new TermQuery(new Term(F, "test")), Occur.MUST));
     bq.add(new BooleanClause(new TermQuery(new Term(F, "This")), Occur.MUST));
@@ -232,7 +224,7 @@ public class IntervalHighlighterTest ext
   }
   
   public void testBooleanOr() throws Exception {
-    insertDocs(analyzer, "This is a test");
+    insertDocs("This is a test");
     BooleanQuery bq = new BooleanQuery();
     bq.add(new BooleanClause(new TermQuery(new Term(F, "test")), Occur.SHOULD));
     bq.add(new BooleanClause(new TermQuery(new Term(F, "This")), Occur.SHOULD));
@@ -242,7 +234,7 @@ public class IntervalHighlighterTest ext
   }
   
   public void testSingleMatchScorer() throws Exception {
-    insertDocs(analyzer, "This is a test");
+    insertDocs("This is a test");
     BooleanQuery bq = new BooleanQuery();
     bq.add(new BooleanClause(new TermQuery(new Term(F, "test")), Occur.SHOULD));
     bq.add(new BooleanClause(new TermQuery(new Term(F, "notoccurringterm")),
@@ -253,7 +245,7 @@ public class IntervalHighlighterTest ext
   }
   
   public void testBooleanNrShouldMatch() throws Exception {
-    insertDocs(analyzer, "a b c d e f g h i");
+    insertDocs("a b c d e f g h i");
     BooleanQuery bq = new BooleanQuery();
     bq.add(new BooleanClause(new TermQuery(new Term(F, "a")), Occur.SHOULD));
     bq.add(new BooleanClause(new TermQuery(new Term(F, "b")), Occur.SHOULD));
@@ -278,7 +270,7 @@ public class IntervalHighlighterTest ext
   }
   
   public void testPhrase() throws Exception {
-    insertDocs(analyzer, "is it that this is a test, is it");
+    insertDocs("is it that this is a test, is it");
     PhraseQuery pq = new PhraseQuery();
     pq.add(new Term(F, "is"));
     pq.add(new Term(F, "a"));
@@ -293,7 +285,7 @@ public class IntervalHighlighterTest ext
    */
   //@Ignore
   public void testPhraseOriginal() throws Exception {
-    insertDocs(analyzer, "This is a test");
+    insertDocs("This is a test");
     PhraseQuery pq = new PhraseQuery();
     pq.add(new Term(F, "a"));
     pq.add(new Term(F, "test"));
@@ -303,7 +295,7 @@ public class IntervalHighlighterTest ext
   }
   
   public void testNestedBoolean() throws Exception {
-    insertDocs(analyzer, "This is a test");
+    insertDocs("This is a test");
     BooleanQuery bq = new BooleanQuery();
     bq.add(new BooleanClause(new TermQuery(new Term(F, "test")), Occur.SHOULD));
     BooleanQuery bq2 = new BooleanQuery();
@@ -316,14 +308,14 @@ public class IntervalHighlighterTest ext
   }
   
   public void testWildcard() throws Exception {
-    insertDocs(analyzer, "This is a test");
+    insertDocs("This is a test");
     String frags[] = doSearch(new WildcardQuery(new Term(F, "t*t")));
     assertEquals("This is a <B>test</B>", frags[0]);
     close();
   }
 
   public void testMixedBooleanNot() throws Exception {
-    insertDocs(analyzer, "this is a test", "that is an elephant");
+    insertDocs("this is a test", "that is an elephant");
     BooleanQuery bq = new BooleanQuery();
     bq.add(new BooleanClause(new TermQuery(new Term(F, "test")), Occur.MUST));
     bq.add(new BooleanClause(new TermQuery(new Term(F, "that")), Occur.MUST_NOT));
@@ -333,7 +325,7 @@ public class IntervalHighlighterTest ext
   }
 
   public void testMixedBooleanShould() throws Exception {
-    insertDocs(analyzer, "this is a test", "that is an elephant", "the other was a rhinoceros");
+    insertDocs("this is a test", "that is an elephant", "the other was a rhinoceros");
     BooleanQuery bq = new BooleanQuery();
     bq.add(new BooleanClause(new TermQuery(new Term(F, "is")), Occur.MUST));
     bq.add(new BooleanClause(new TermQuery(new Term(F, "test")), Occur.SHOULD));
@@ -351,7 +343,7 @@ public class IntervalHighlighterTest ext
   }
   
   public void testMultipleDocumentsAnd() throws Exception {
-    insertDocs(analyzer, "This document has no matches", PORRIDGE_VERSE,
+    insertDocs("This document has no matches", PORRIDGE_VERSE,
         "This document has some Pease porridge in it");
     BooleanQuery bq = new BooleanQuery();
     bq.add(new BooleanClause(new TermQuery(new Term(F, "Pease")), Occur.MUST));
@@ -368,7 +360,7 @@ public class IntervalHighlighterTest ext
   
 
   public void testMultipleDocumentsOr() throws Exception {
-    insertDocs(analyzer, "This document has no matches", PORRIDGE_VERSE,
+    insertDocs("This document has no matches", PORRIDGE_VERSE,
         "This document has some Pease porridge in it");
     BooleanQuery bq = new BooleanQuery();
     bq.add(new BooleanClause(new TermQuery(new Term(F, "Pease")), Occur.SHOULD));
@@ -386,30 +378,26 @@ public class IntervalHighlighterTest ext
   
   public void testBrouwerianQuery() throws Exception {
 
-    insertDocs(analyzer, "the quick brown duck jumps over the lazy dog with the quick brown fox");
+    insertDocs("the quick brown fox jumps over the lazy dog with the quick orange fox");
 
-    BooleanQuery query = new BooleanQuery();
-    query.add(new BooleanClause(new TermQuery(new Term(F, "the")), Occur.MUST));
-    query.add(new BooleanClause(new TermQuery(new Term(F, "quick")), Occur.MUST));
-    query.add(new BooleanClause(new TermQuery(new Term(F, "jumps")), Occur.MUST));
+    OrderedNearQuery query = new OrderedNearQuery(1,
+        new TermQuery(new Term(F, "the")), new TermQuery(new Term(F, "quick")), new TermQuery(new Term(F, "fox")));
 
     assertEquals(getHighlight(query),
-                 "<B>the</B> <B>quick</B> brown duck <B>jumps</B> over <B>the</B> lazy dog with the <B>quick</B> brown fox");
+                 "<B>the quick brown fox</B> jumps over the lazy dog with <B>the quick orange fox</B>");
 
-    BooleanQuery sub = new BooleanQuery();
-    sub.add(new BooleanClause(new TermQuery(new Term(F, "duck")), Occur.MUST));
-    NonOverlappingQuery bq = new NonOverlappingQuery(query, sub);
+    NonOverlappingQuery bq = new NonOverlappingQuery(query, new TermQuery(new Term(F, "orange")));
 
     assertEquals(getHighlight(bq),
-                 "the quick brown duck <B>jumps</B> over <B>the</B> lazy dog with the <B>quick</B> brown fox");
+                 "<B>the quick brown fox<B> jumps over the lazy dog with the quick orange fox");
 
     close();
   }
   
-  @Ignore("not implemented yet - unsupported")
+  //@Ignore("not implemented yet - unsupported")
   public void testMultiPhraseQuery() throws Exception {
     MultiPhraseQuery query = new MultiPhraseQuery();
-    insertDocs(analyzer, "pease porridge hot but not too hot or otherwise pease porridge cold");
+    insertDocs("pease porridge hot but not too hot or otherwise pease porridge cold");
 
     query.add(terms(F, "pease"), 0);
     query.add(terms(F, "porridge"), 1);
@@ -422,10 +410,10 @@ public class IntervalHighlighterTest ext
     close();
   }
   
-  @Ignore("not implemented yet - unsupported")
+  //@Ignore("not implemented yet - unsupported")
   public void testMultiPhraseQueryCollisions() throws Exception {
     MultiPhraseQuery query = new MultiPhraseQuery();
-    insertDocs(analyzer, "pease porridge hot not too hot or otherwise pease porridge porridge");
+    insertDocs("pease porridge hot not too hot or otherwise pease porridge porridge");
 
     query.add(terms(F, "pease"), 0);
     query.add(terms(F, "porridge"), 1);
@@ -440,12 +428,12 @@ public class IntervalHighlighterTest ext
 
   public void testNearPhraseQuery() throws Exception {
 
-    insertDocs(analyzer, "pease porridge rather hot and pease porridge fairly cold");
+    insertDocs("pease porridge rather hot and pease porridge fairly cold");
 
     Query firstQ = new OrderedNearQuery(4, termQuery("pease"), termQuery("porridge"), termQuery("hot"));
     {
       String frags[] = doSearch(firstQ, Integer.MAX_VALUE);
-      assertEquals("<B>pease</B> <B>porridge</B> rather <B>hot</B> and pease porridge fairly cold", frags[0]);
+      assertEquals("<B>pease porridge rather hot</B> and pease porridge fairly cold", frags[0]);
     }
 
     // near.3(near.4(pease, porridge, hot), near.4(pease, porridge, cold))
@@ -454,7 +442,7 @@ public class IntervalHighlighterTest ext
                 new OrderedNearQuery(4, termQuery("pease"), termQuery("porridge"), termQuery("cold")));
 
     String frags[] = doSearch(q, Integer.MAX_VALUE);
-    assertEquals("<B>pease</B> <B>porridge</B> rather <B>hot</B> and <B>pease</B> <B>porridge</B> fairly <B>cold</B>",
+    assertEquals("<B>pease porridge rather hot and pease porridge fairly cold</B>",
                  frags[0]);
 
     close();
@@ -469,18 +457,18 @@ public class IntervalHighlighterTest ext
     }
 
   public void testSloppyPhraseQuery() throws Exception {
-    assertSloppyPhrase( "a b c d a b c d e f", "a b <B>c</B> d <B>a</B> b c d e f", 2, "c", "a");
-    assertSloppyPhrase( "a c e b d e f a b","<B>a</B> c e <B>b</B> d e f <B>a</B> <B>b</B>", 2, "a", "b");
-    assertSloppyPhrase( "Y A X B A", "Y <B>A</B> <B>X</B> B <B>A</B>", 2, "X", "A", "A");
+    assertSloppyPhrase( "a c e b d e f a b", "<B>a c e b</B> d e f <B>a b</B>", 2, "a", "b");
+    assertSloppyPhrase( "a b c d a b c d e f", "a b <B>c d a</B> b c d e f", 2, "c", "a");
+    assertSloppyPhrase( "Y A X B A", "Y <B>A X B A</B>", 2, "X", "A", "A");
 
-    assertSloppyPhrase( "X A X B A","<B>X</B> <B>A</B> X B <B>A</B>", 2, "X", "A", "A"); // non overlapping minmal!!
+    assertSloppyPhrase( "X A X B A","X <B>A X B A</B>", 2, "X", "A", "A"); // non overlapping minmal!!
     assertSloppyPhrase( "A A A X",null, 2, "X", "A", "A");
-    assertSloppyPhrase( "A A X A",  "A <B>A</B> <B>X</B> <B>A</B>", 2, "X", "A", "A");
-    assertSloppyPhrase( "A A X A Y B A", "A <B>A</B> <B>X</B> <B>A</B> Y B <B>A</B>", 2, "X", "A", "A");
+    assertSloppyPhrase( "A A X A",  "A <B>A X A</B>", 2, "X", "A", "A");
+    assertSloppyPhrase( "A A X A Y B A", "A <B>A X A</B> Y B A", 2, "X", "A", "A");
     assertSloppyPhrase( "A A X", null, 2, "X", "A", "A");
     assertSloppyPhrase( "A X A", null, 1, "X", "A", "A");
 
-    assertSloppyPhrase( "A X B A", "<B>A</B> <B>X</B> B <B>A</B>", 2, "X", "A", "A");
+    assertSloppyPhrase( "A X B A", "<B>A X B A</B>", 2, "X", "A", "A");
     assertSloppyPhrase( "A A X A X B A X B B A A X B A A", "A <B>A</B> <B>X</B> <B>A</B> <B>X</B> B <B>A</B> <B>X</B> B B <B>A</B> <B>A</B> <B>X</B> B <B>A</B> <B>A</B>", 2, "X", "A", "A");
     assertSloppyPhrase( "A A X A X B A X B B A A X B A A", "A <B>A</B> <B>X</B> <B>A</B> <B>X</B> B <B>A</B> <B>X</B> B B <B>A</B> <B>A</B> <B>X</B> B <B>A</B> <B>A</B>", 2, "X", "A", "A");
 
@@ -493,7 +481,7 @@ public class IntervalHighlighterTest ext
 
 
   private void assertSloppyPhrase(String doc, String expected, int slop, String...query) throws Exception {
-    insertDocs(analyzer, doc);
+    insertDocs(doc);
     PhraseQuery pq = new PhraseQuery();
     for (String string : query) {
       pq.add(new Term(F, string));