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/05 10:13:34 UTC

svn commit: r1643199 - 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: Fri Dec  5 09:13:34 2014
New Revision: 1643199

URL: http://svn.apache.org/viewvc?rev=1643199&view=rev
Log:
PhraseQuery highlighing fixes

Added:
    lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/PhraseScorer.java   (with props)
Modified:
    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/SloppyPhraseScorer.java
    lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/search/TestSloppyPhraseQuery.java
    lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/search/posfilter/TestBasicIntervals.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/HighlightingIntervalCollector.java
    lucene/dev/branches/lucene2878/lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/PostingsHighlighter.java
    lucene/dev/branches/lucene2878/lucene/highlighter/src/test/org/apache/lucene/search/highlight/positions/IntervalHighlighterTest.java
    lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/index/AssertingLeafReader.java

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=1643199&r1=1643198&r2=1643199&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 Fri Dec  5 09:13:34 2014
@@ -23,9 +23,8 @@ import java.util.Arrays;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.search.PhraseQuery.TermDocsEnumFactory;
 import org.apache.lucene.search.similarities.Similarity;
-import org.apache.lucene.util.BytesRef;
 
-final class ExactPhraseScorer extends Scorer {
+final class ExactPhraseScorer extends PhraseScorer {
   private final int endMinus1;
   
   private final static int CHUNK = 4096;
@@ -33,9 +32,8 @@ final class ExactPhraseScorer extends Sc
   private int gen;
   private final int[] counts = new int[CHUNK];
   private final int[] gens = new int[CHUNK];
+  private final int[] offsets = new int[CHUNK];
 
-  
-  boolean noDocs;
   private final long cost;
 
   private final static class ChunkState {
@@ -59,8 +57,6 @@ final class ExactPhraseScorer extends Sc
 
   private int docID = -1;
 
-  private int freq = -1;
-
   private final Similarity.SimScorer docScorer;
 
   ExactPhraseScorer(Weight weight, PhraseQuery.PostingsAndFreq[] postings,
@@ -127,17 +123,6 @@ final class ExactPhraseScorer extends Sc
   }
   
   @Override
-  public int freq() throws IOException {
-    if (freq == -1) {
-      freq = 0;
-      while (nextPosition() != NO_MORE_DOCS) {
-        freq++;
-      }
-    }
-    return freq;
-  }
-  
-  @Override
   public int docID() {
     return docID;
   }
@@ -170,38 +155,33 @@ final class ExactPhraseScorer extends Sc
 
   private int firstPosition() throws IOException {
     resetPositions();
-    int pos = nextPosition();
+    int pos = doNextPosition();
     cached = true;
     return pos;
   }
 
   @Override
-  public int startPosition() throws IOException {
-    return posQueue[positionsInChunk - posRemaining];
-  }
-
-  @Override
-  public int startOffset() throws IOException {
-    return -1;
+  protected int doStartPosition() throws IOException {
+    return posQueue[positionsInChunk - posRemaining - 1];
   }
 
   @Override
-  public int endOffset() throws IOException {
-    return -1;
+  protected int doStartOffset() throws IOException {
+    return offsetQueue[(positionsInChunk - posRemaining - 1) / 2];
   }
 
   @Override
-  public BytesRef getPayload() throws IOException {
-    return null; // nocommit how to deal with payloads across multiple positions?
+  protected int doEndOffset() throws IOException {
+    return offsetQueue[(positionsInChunk - posRemaining - 1) / 2 + 1];
   }
 
   @Override
-  public int endPosition() throws IOException {
+  protected int doEndPosition() throws IOException {
     return startPosition() + chunkStates.length - 1;
   }
 
   @Override
-  public int nextPosition() throws IOException {
+  protected int doNextPosition() throws IOException {
     if (cached) {
       cached = false;
       return startPosition();
@@ -218,15 +198,21 @@ final class ExactPhraseScorer extends Sc
   }
 
   int[] posQueue = new int[8];
+  int[] offsetQueue = new int[16];
 
-  private void addPosition(int pos) {
+  private void addPosition(int pos, int beginOffset, int endOffset) {
     positionsInChunk++;
     if (posQueue.length < positionsInChunk) {
       int[] newQueue = new int[posQueue.length * 2];
       System.arraycopy(posQueue, 0, newQueue, 0, posQueue.length);
       posQueue = newQueue;
+      int[] newOffsets = new int[posQueue.length * 2];
+      System.arraycopy(offsetQueue, 0, newOffsets, 0, offsetQueue.length);
+      offsetQueue = newOffsets;
     }
-    posQueue[positionsInChunk] = pos;
+    posQueue[positionsInChunk - 1] = pos;
+    offsetQueue[(positionsInChunk - 1) * 2] = beginOffset;
+    offsetQueue[(positionsInChunk - 1) * 2 + 1] = endOffset;
   }
 
   private boolean findNextMatches() throws IOException {
@@ -259,6 +245,7 @@ final class ExactPhraseScorer extends Sc
             any = true;
             assert gens[posIndex] != gen;
             gens[posIndex] = gen;
+            offsets[posIndex] = cs.posEnum.startOffset();
           }
 
           if (cs.posUpto == cs.posLimit) {
@@ -318,7 +305,7 @@ final class ExactPhraseScorer extends Sc
             final int posIndex = cs.pos - chunkStart;
             if (posIndex >= 0 && gens[posIndex] == gen
                 && counts[posIndex] == endMinus1) {
-              addPosition(cs.pos);
+              addPosition(cs.pos, offsets[posIndex], cs.posEnum.endOffset());
               any = true;
             }
           }

Added: lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/PhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/PhraseScorer.java?rev=1643199&view=auto
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/PhraseScorer.java (added)
+++ lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/PhraseScorer.java Fri Dec  5 09:13:34 2014
@@ -0,0 +1,109 @@
+package org.apache.lucene.search;
+
+import java.io.IOException;
+
+import org.apache.lucene.search.posfilter.Interval;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Copyright (c) 2014 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 abstract class PhraseScorer extends Scorer {
+
+  protected PhraseScorer(Weight weight) {
+    super(weight);
+  }
+
+  protected int freq = -1;
+  protected Interval[] positionCache = new Interval[4];
+  private int currentPos = -1;
+
+  @Override
+  public final int freq() throws IOException {
+    if (freq == -1) {
+      cachePositions();
+    }
+    return freq;
+  }
+
+  private void cachePositions() throws IOException {
+    assert freq == -1;
+    int f = 0;
+    while (doNextPosition() != NO_MORE_POSITIONS) {
+      if (f >= positionCache.length) {
+        Interval[] newCache = new Interval[positionCache.length * 2];
+        System.arraycopy(positionCache, 0, newCache, 0, positionCache.length);
+        positionCache = newCache;
+      }
+      positionCache[f] = new Interval(this);
+      f++;
+    }
+    this.freq = f;
+  }
+
+  @Override
+  public final int nextPosition() throws IOException {
+    if (freq == -1)
+      return doNextPosition();
+    currentPos++;
+    if (currentPos >= freq)
+      return NO_MORE_POSITIONS;
+    return positionCache[currentPos].begin;
+  }
+
+  @Override
+  public final int startPosition() throws IOException {
+    if (freq == -1)
+      return doStartPosition();
+    return positionCache[currentPos].begin;
+  }
+
+  @Override
+  public final int endPosition() throws IOException {
+    if (freq == -1)
+      return doEndPosition();
+    return positionCache[currentPos].end;
+  }
+
+  @Override
+  public final int startOffset() throws IOException {
+    if (freq == -1)
+      return doStartOffset();
+    return positionCache[currentPos].offsetBegin;
+  }
+
+  @Override
+  public final int endOffset() throws IOException {
+    if (freq == -1)
+      return doEndOffset();
+    return positionCache[currentPos].offsetEnd;
+  }
+
+  @Override
+  public final BytesRef getPayload() throws IOException {
+    return null;  // TODO - how to deal with payloads on intervals?
+  }
+
+  protected abstract int doNextPosition() throws IOException;
+
+  protected abstract int doStartPosition() throws IOException;
+
+  protected abstract int doEndPosition() throws IOException;
+
+  protected abstract int doStartOffset() throws IOException;
+
+  protected abstract int doEndOffset() throws IOException;
+}

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=1643199&r1=1643198&r2=1643199&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 Fri Dec  5 09:13:34 2014
@@ -27,14 +27,12 @@ import java.util.LinkedHashMap;
 
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.similarities.Similarity;
-import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.FixedBitSet;
 
 
-final class SloppyPhraseScorer extends Scorer {
+final class SloppyPhraseScorer extends PhraseScorer {
   private PhrasePositions min, max;
 
-  private int freq;
   private float sloppyFreq;
 
   private final Similarity.SimScorer docScorer;
@@ -44,7 +42,12 @@ final class SloppyPhraseScorer extends S
   private final int numPostings;
   private final PhraseQueue pq; // for advancing min position
   
-  private int end; // current largest phrase position  
+  private int currentEnd; // current largest phrase position
+  private int currentRealEnd; // current largest phrase position, including phrase offset
+  private int currentEndOffset; // current largest phrase currentEnd offset
+
+  private int spanEnd;
+  private int spanOffsetEnd;
 
   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
@@ -88,12 +91,12 @@ final class SloppyPhraseScorer extends S
 
   private int matchLength;
   private int startpos = -1;
-  private int endpos = -1;
+  private int startoffset = -1;
   private int startoffset = -1;
   private int endoffset = -1;
 
   @Override
-  public int nextPosition() throws IOException {
+  protected int doNextPosition() throws IOException {
     if (cached) {
       cached = false;
       return this.startPosition();
@@ -103,40 +106,43 @@ final class SloppyPhraseScorer extends S
       return NO_MORE_POSITIONS;
 
     PhrasePositions top = pq.pop();
-    matchLength = end - top.position;
+    matchLength = currentEnd - top.position;
     int next = pq.top().position;
     int pos = top.position + top.phraseOffset;
     int startoffset = top.startOffset();
-    int endoffset = top.endOffset();
+    spanEnd = currentRealEnd;
+    spanOffsetEnd = currentEndOffset;
     while (advancePP(top)) {
       if (hasRpts && !advanceRpts(top))
         break; // pps exhausted
       if (top.position > next) { // done minimizing current match-length
         if (matchLength <= slop) {
-          setSpan(pos, startoffset, endoffset);
+          setSpanStart(pos, startoffset);
           pq.add(top);
           return startpos;
         }
         pq.add(top);
         top = pq.pop();
         next = pq.top().position;
-        matchLength = end - top.position;
+        matchLength = currentEnd - top.position;
         pos = top.position + top.phraseOffset;
         startoffset = top.startOffset();
-        endoffset = top.endOffset();
+        spanEnd = currentRealEnd;
+        spanOffsetEnd = currentEndOffset;
       }
       else {
-        int matchLength2 = end - top.position;
+        int matchLength2 = currentEnd - top.position;
         pos = top.position + top.phraseOffset;
         startoffset = top.startOffset();
-        endoffset = top.endOffset();
+        spanEnd = currentRealEnd;
+        spanOffsetEnd = currentEndOffset;
         if (matchLength2 < matchLength)
           matchLength = matchLength2;
       }
     }
 
     if (matchLength <= slop) {
-      setSpan(pos, startoffset, endoffset);
+      setSpanStart(pos, startoffset);
       return startpos;
     }
 
@@ -144,19 +150,19 @@ final class SloppyPhraseScorer extends S
 
   }
 
-  private void setSpan(int topPos, int topStartOffset, int topEndOffset) throws IOException {
+  private void setSpanStart(int topPos, int topStartOffset) throws IOException {
     startpos = topPos;
-    endpos = topPos;
+    startoffset = topStartOffset;
     startoffset = topStartOffset;
     endoffset = topEndOffset;
     for (Object o : pq.getPPs()) {
       if (o == null)
         continue;
       PhrasePositions pp = (PhrasePositions) o;
-      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());
+      if (pp.position != NO_MORE_POSITIONS) {
+        startpos = Math.min(startpos, pp.position + pp.phraseOffset);
+        startoffset = Math.min(startoffset, pp.startOffset());
+      }
     }
   }
 
@@ -169,18 +175,22 @@ final class SloppyPhraseScorer extends S
     freq = -1;
     sloppyFreq = -1;
     cached = false;
-    int pos = nextPosition();
+    int pos = doNextPosition();
     cached = true;
     return pos;
   }
 
-  /** advance a PhrasePosition and update 'end', return false if exhausted */
+  /** advance a PhrasePosition and update 'currentEnd', return false if exhausted */
   private boolean advancePP(PhrasePositions pp) throws IOException {
     if (!pp.nextPosition()) {
       return false;
     }
-    if (pp.position > end) {
-      end = pp.position;
+    if (pp.position > currentEnd) {
+      currentEnd = pp.position;
+    }
+    if (pp.position + pp.phraseOffset > currentRealEnd) {
+      currentRealEnd = pp.position + pp.phraseOffset;
+      currentEndOffset = pp.endOffset();
     }
     return true;
   }
@@ -265,7 +275,8 @@ final class SloppyPhraseScorer extends S
    * @return false if PPs are exhausted (and so current doc will not be a match) 
    */
   private boolean initPhrasePositions() throws IOException {
-    end = Integer.MIN_VALUE;
+    currentEnd = currentRealEnd = Integer.MIN_VALUE;
+    currentEndOffset = -1;
     if (!checkedRpts) {
       return initFirstTime();
     }
@@ -283,8 +294,12 @@ final class SloppyPhraseScorer extends S
     // position pps and build queue from list
     for (PhrasePositions pp=min,prev=null; prev!=max; pp=(prev=pp).next) {  // iterate cyclic list: done once handled max
       pp.firstPosition();
-      if (pp.position > end) {
-        end = pp.position;
+      if (pp.position > currentEnd) {
+        currentEnd = pp.position;
+      }
+      if (pp.position + pp.phraseOffset > currentRealEnd) {
+        currentRealEnd = pp.position + pp.phraseOffset;
+        currentEndOffset = pp.endOffset();
       }
       pq.add(pp);
     }
@@ -309,11 +324,15 @@ final class SloppyPhraseScorer extends S
   }
 
   /** Fill the queue (all pps are already placed */
-  private void fillQueue() {
+  private void fillQueue() throws IOException {
     pq.clear();
     for (PhrasePositions pp=min,prev=null; prev!=max; pp=(prev=pp).next) {  // iterate cyclic list: done once handled max
-      if (pp.position > end) {
-        end = pp.position;
+      if (pp.position > currentEnd) {
+        currentEnd = pp.position;
+      }
+      if (pp.position + pp.phraseOffset > currentRealEnd) {
+        currentRealEnd = pp.position + pp.phraseOffset;
+        currentEndOffset = pp.endOffset();
       }
       pq.add(pp);
     }
@@ -559,17 +578,6 @@ final class SloppyPhraseScorer extends S
     return tg;
   }
 
-  @Override
-  public int freq() throws IOException {
-    if (freq == -1) {
-      freq = 0;
-      while (nextPosition() != NO_MORE_POSITIONS) {
-        freq++;
-      }
-    }
-    return freq;
-  }
-
   /**
    * Score a candidate doc for all slop-valid position-combinations (matches)
    * encountered while traversing/hopping the PhrasePositions.
@@ -599,29 +607,24 @@ final class SloppyPhraseScorer extends S
   }
 
   @Override
-  public int startOffset() throws IOException {
+  protected int doStartOffset() throws IOException {
     return startoffset;
   }
 
   @Override
-  public int endOffset() throws IOException {
-    return endoffset;
+  protected int doEndOffset() throws IOException {
+    return spanOffsetEnd;
   }
 
   // TODO : getPayload on spans?
   @Override
-  public BytesRef getPayload() throws IOException {
-    return null;
-  }
-
-  @Override
-  public int startPosition() throws IOException {
+  protected int doStartPosition() throws IOException {
     return startpos;
   }
 
   @Override
-  public int endPosition() throws IOException {
-    return endpos;
+  protected int doEndPosition() throws IOException {
+    return spanEnd;
   }
 
   //  private void printQueue(PrintStream ps, PhrasePositions ext, String title) {

Modified: lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/search/TestSloppyPhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/search/TestSloppyPhraseQuery.java?rev=1643199&r1=1643198&r2=1643199&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/search/TestSloppyPhraseQuery.java (original)
+++ lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/search/TestSloppyPhraseQuery.java Fri Dec  5 09:13:34 2014
@@ -19,7 +19,7 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 
-import org.apache.lucene.util.LuceneTestCase;
+import com.carrotsearch.randomizedtesting.annotations.Seed;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.MockTokenizer;
 import org.apache.lucene.document.Document;
@@ -32,7 +32,9 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.store.RAMDirectory;
+import org.apache.lucene.util.LuceneTestCase;
 
+@Seed("2542F68A58928060")
 public class TestSloppyPhraseQuery extends LuceneTestCase {
 
   private static final String S_1 = "A A A";

Modified: lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/search/posfilter/TestBasicIntervals.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/search/posfilter/TestBasicIntervals.java?rev=1643199&r1=1643198&r2=1643199&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/search/posfilter/TestBasicIntervals.java (original)
+++ lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/search/posfilter/TestBasicIntervals.java Fri Dec  5 09:13:34 2014
@@ -17,6 +17,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;
@@ -24,8 +26,6 @@ import org.apache.lucene.search.BooleanC
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.Query;
 
-import java.io.IOException;
-
 public class TestBasicIntervals extends IntervalTestBase {
 
   public static final String field = "field";

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=1643199&r1=1643198&r2=1643199&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 Fri Dec  5 09:13:34 2014
@@ -55,6 +55,11 @@ public class TestPhraseQueryPositions ex
           TextField.TYPE_STORED));
       writer.addDocument(doc);
     }
+    {
+      Document doc = new Document();
+      doc.add(newField("sloppy", "x a x b a", TextField.TYPE_STORED));
+      writer.addDocument(doc);
+    }
   }
 
   public void testOutOfOrderSloppyPhraseQuery() throws IOException {
@@ -69,6 +74,17 @@ public class TestPhraseQueryPositions ex
     });
   }
 
+  public void testOverlappingOutOfOrderSloppyPhraseQuery() throws IOException {
+    PhraseQuery query = new PhraseQuery();
+    query.add(new Term("sloppy", "x"));
+    query.add(new Term("sloppy", "a"));
+    query.add(new Term("sloppy", "a"));
+    query.setSlop(2);
+    checkIntervals(query, searcher, new int[][]{
+        {2, 0, 4, 1, 4}
+    });
+  }
+
   public void testSloppyPhraseQuery() throws IOException {
     PhraseQuery query = new PhraseQuery();
     query.add(new Term("field", "pease"));
@@ -80,7 +96,7 @@ public class TestPhraseQueryPositions ex
     });
   }
 
-  public void testSloppyPhraseQueryWithRepears() throws IOException {
+  public void testSloppyPhraseQueryWithRepeats() throws IOException {
     PhraseQuery query = new PhraseQuery();
     query.add(new Term("field", "pease"));
     query.add(new Term("field", "porridge"));

Modified: lucene/dev/branches/lucene2878/lucene/highlighter/src/java/org/apache/lucene/search/highlight/positions/HighlightingIntervalCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/highlighter/src/java/org/apache/lucene/search/highlight/positions/HighlightingIntervalCollector.java?rev=1643199&r1=1643198&r2=1643199&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/highlighter/src/java/org/apache/lucene/search/highlight/positions/HighlightingIntervalCollector.java (original)
+++ lucene/dev/branches/lucene2878/lucene/highlighter/src/java/org/apache/lucene/search/highlight/positions/HighlightingIntervalCollector.java Fri Dec  5 09:13:34 2014
@@ -31,6 +31,7 @@ public class HighlightingIntervalCollect
   DocAndPositions docs[];
   
   public HighlightingIntervalCollector (int maxDocs) {
+    super(true);
     docs = new DocAndPositions[maxDocs];
   }
 

Modified: lucene/dev/branches/lucene2878/lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/PostingsHighlighter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/PostingsHighlighter.java?rev=1643199&r1=1643198&r2=1643199&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/PostingsHighlighter.java (original)
+++ lucene/dev/branches/lucene2878/lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/PostingsHighlighter.java Fri Dec  5 09:13:34 2014
@@ -740,10 +740,10 @@ public class PostingsHighlighter {
     }
 
     @Override
-    public int startOffset() throws IOException { throw new UnsupportedOperationException(); }
+    public int startOffset() throws IOException { return NO_MORE_POSITIONS; }
 
     @Override
-    public int endOffset() throws IOException { throw new UnsupportedOperationException(); }
+    public int endOffset() throws IOException { return NO_MORE_POSITIONS; }
 
     @Override
     public BytesRef getPayload() throws IOException { return null; }

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=1643199&r1=1643198&r2=1643199&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 Fri Dec  5 09:13:34 2014
@@ -19,7 +19,6 @@ package org.apache.lucene.search.highlig
 import java.io.IOException;
 import java.io.StringReader;
 
-import com.carrotsearch.randomizedtesting.annotations.Seed;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.MockTokenFilter;
 import org.apache.lucene.analysis.MockTokenizer;
@@ -53,7 +52,6 @@ import org.apache.lucene.search.posfilte
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
 
-@Seed("2C0AB6BC65255FAA")
 public class IntervalHighlighterTest extends LuceneTestCase {
   
   protected final static String F = "f";
@@ -276,7 +274,7 @@ public class IntervalHighlighterTest ext
     pq.add(new Term(F, "a"));
     String frags[] = doSearch(pq);
     // make sure we highlight the phrase, and not the terms outside the phrase
-    assertEquals("is it that this <B>is</B> <B>a</B> test, is it", frags[0]);
+    assertEquals("is it that this <B>is a</B> test, is it", frags[0]);
     close();
   }
   
@@ -290,7 +288,7 @@ public class IntervalHighlighterTest ext
     pq.add(new Term(F, "a"));
     pq.add(new Term(F, "test"));
     String frags[] = doSearch(pq);
-    assertEquals("This is <B>a</B> <B>test</B>", frags[0]);
+    assertEquals("This is <B>a test</B>", frags[0]);
     close();
   }
   
@@ -389,7 +387,7 @@ public class IntervalHighlighterTest ext
     NonOverlappingQuery bq = new NonOverlappingQuery(query, new TermQuery(new Term(F, "orange")));
 
     assertEquals(getHighlight(bq),
-                 "<B>the quick brown fox<B> jumps over the lazy dog with the quick orange fox");
+                 "<B>the quick brown fox</B> jumps over the lazy dog with the quick orange fox");
 
     close();
   }
@@ -405,7 +403,7 @@ public class IntervalHighlighterTest ext
     query.setSlop(1);
     
     String[] frags = doSearch(query, Integer.MAX_VALUE);
-    assertEquals("<B>pease</B> <B>porridge</B> <B>hot</B> but not too hot or otherwise <B>pease</B> <B>porridge</B> <B>cold</B>", frags[0]);
+    assertEquals("<B>pease porridge hot</B> but not too hot or otherwise <B>pease porridge cold</B>", frags[0]);
 
     close();
   }
@@ -421,7 +419,7 @@ public class IntervalHighlighterTest ext
     query.setSlop(1);
     
     String[] frags = doSearch(query, Integer.MAX_VALUE);
-    assertEquals("pease porridge hot but not too hot or otherwise <B>pease</B> <B>porridge</B> <B>porridge</B>", frags[0]);
+    assertEquals("pease porridge hot not too hot or otherwise <B>pease porridge porridge</B>", frags[0]);
 
     close();
   }
@@ -460,8 +458,8 @@ public class IntervalHighlighterTest ext
     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 A X 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 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");
@@ -469,12 +467,11 @@ public class IntervalHighlighterTest ext
     assertSloppyPhrase( "A X A", null, 1, "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");
+    assertSloppyPhrase( "A A X A X B A X B B A A X B A A", "A <B>A X A</B> X B A <B>X B B A A</B> <B>X B A A</B>", 2, "X", "A", "A");
 
-    assertSloppyPhrase( "A A X A X B A", "A <B>A</B> <B>X</B> <B>A</B> <B>X</B> B <B>A</B>", 2, "X", "A", "A");
-    assertSloppyPhrase( "A A Y A X B A", "A A Y <B>A</B> <B>X</B> B <B>A</B>", 2, "X", "A", "A");
-    assertSloppyPhrase( "A A Y A X B A A", "A A Y <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", "A <B>A X A</B> X B A", 2, "X", "A", "A");
+    assertSloppyPhrase( "A A Y A X B A", "A A Y <B>A X B A</B>", 2, "X", "A", "A");
+    assertSloppyPhrase( "A A Y A X B A A", "A A Y <B>A X B A</B> A", 2, "X", "A", "A");
     assertSloppyPhrase( "A A X A Y B A", null , 1, "X", "A", "A");
     close();
   }

Modified: lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/index/AssertingLeafReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/index/AssertingLeafReader.java?rev=1643199&r1=1643198&r2=1643199&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/index/AssertingLeafReader.java (original)
+++ lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/index/AssertingLeafReader.java Fri Dec  5 09:13:34 2014
@@ -322,9 +322,6 @@ public class AssertingLeafReader extends
       assert state != DocsEnumState.START : "freq() called before nextDoc()/advance()";
       assert state != DocsEnumState.FINISHED : "freq() called after NO_MORE_DOCS";
       int freq = super.freq();
-      if (freq == 0) {
-        System.out.println();
-      }
       assert freq > 0;
       return freq;
     }