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/08 12:01:52 UTC

svn commit: r1643787 - in /lucene/dev/branches/lucene2878/lucene/core/src: java/org/apache/lucene/search/ java/org/apache/lucene/search/posfilter/ test/org/apache/lucene/search/posfilter/

Author: romseygeek
Date: Mon Dec  8 11:01:51 2014
New Revision: 1643787

URL: http://svn.apache.org/r1643787
Log:
LUCENE-2878: Make Interval package-private, remove PositionsCollector

Removed:
    lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/PositionsCollector.java
Modified:
    lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/PhraseScorer.java
    lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/PositionQueue.java
    lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/posfilter/Interval.java
    lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/posfilter/UnorderedNearQuery.java
    lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/search/posfilter/IntervalTestBase.java

Modified: 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=1643787&r1=1643786&r2=1643787&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/PhraseScorer.java (original)
+++ lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/PhraseScorer.java Mon Dec  8 11:01:51 2014
@@ -2,7 +2,7 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 
-import org.apache.lucene.search.posfilter.Interval;
+import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.util.BytesRef;
 
 /**
@@ -23,12 +23,26 @@ import org.apache.lucene.util.BytesRef;
 
 public abstract class PhraseScorer extends Scorer {
 
+  static class PositionSnapshot {
+    int begin;
+    int end;
+    int offsetBegin;
+    int offsetEnd;
+
+    public PositionSnapshot(DocsEnum docsEnum) throws IOException {
+      this.begin = docsEnum.startPosition();
+      this.end = docsEnum.endPosition();
+      this.offsetBegin = docsEnum.startOffset();
+      this.offsetEnd = docsEnum.endOffset();
+    }
+  }
+
   protected PhraseScorer(Weight weight) {
     super(weight);
   }
 
   protected int freq = -1;
-  protected Interval[] positionCache = new Interval[4];
+  protected PositionSnapshot[] positionCache = new PositionSnapshot[4];
   private int currentPos = -1;
 
   @Override
@@ -44,11 +58,11 @@ public abstract class PhraseScorer exten
     int f = 0;
     while (doNextPosition() != NO_MORE_POSITIONS) {
       if (f >= positionCache.length) {
-        Interval[] newCache = new Interval[positionCache.length * 2];
+        PositionSnapshot[] newCache = new PositionSnapshot[positionCache.length * 2];
         System.arraycopy(positionCache, 0, newCache, 0, positionCache.length);
         positionCache = newCache;
       }
-      positionCache[f] = new Interval(this);
+      positionCache[f] = new PositionSnapshot(this);
       f++;
     }
     this.freq = f;

Modified: lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/PositionQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/PositionQueue.java?rev=1643787&r1=1643786&r2=1643787&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/PositionQueue.java (original)
+++ lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/PositionQueue.java Mon Dec  8 11:01:51 2014
@@ -3,7 +3,6 @@ package org.apache.lucene.search;
 import java.io.IOException;
 
 import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.search.posfilter.Interval;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.PriorityQueue;
 
@@ -24,11 +23,12 @@ import org.apache.lucene.util.PriorityQu
  */
 public class PositionQueue extends PriorityQueue<PositionQueue.DocsEnumRef> {
 
-  public class DocsEnumRef {
+  public class DocsEnumRef implements Comparable<DocsEnumRef> {
 
     public final DocsEnum docsEnum;
     public final int ord;
-    public Interval interval = new Interval();
+    public int start;
+    public int end;
 
     public DocsEnumRef(DocsEnum docsEnum, int ord) {
       this.docsEnum = docsEnum;
@@ -38,17 +38,28 @@ public class PositionQueue extends Prior
     public int nextPosition() throws IOException {
       assert docsEnum.docID() != -1;
       if (docsEnum.docID() == DocsEnum.NO_MORE_DOCS || docsEnum.docID() != docId
-            || docsEnum.nextPosition() == DocsEnum.NO_MORE_POSITIONS)
-        interval.setMaximum();
-      else
-        interval.update(this.docsEnum);
-      return interval.begin;
+          || docsEnum.nextPosition() == DocsEnum.NO_MORE_POSITIONS) {
+        start = end = DocsEnum.NO_MORE_POSITIONS;
+      }
+      else {
+        start = docsEnum.startPosition();
+        end = docsEnum.endPosition();
+      }
+      return start;
     }
 
+    @Override
+    public int compareTo(DocsEnumRef o) {
+      if (this.docsEnum.docID() < o.docsEnum.docID())
+        return -1;
+      if (this.docsEnum.docID() > o.docsEnum.docID())
+        return 1;
+      return Integer.compare(this.start, o.start);
+    }
   }
 
   boolean positioned = false;
-  Interval current = new Interval();
+  int pos = -1;
   int docId = -1;
   protected int queuesize;
 
@@ -75,30 +86,23 @@ public class PositionQueue extends Prior
     if (!positioned) {
       init();
       positioned = true;
-      current.update(top().interval);
-      return current.begin;
+      return pos = top().start;
     }
-    if (current.begin == DocsEnum.NO_MORE_POSITIONS)
+    if (pos == DocsEnum.NO_MORE_POSITIONS)
       return DocsEnum.NO_MORE_POSITIONS;
     if (top().nextPosition() == DocsEnum.NO_MORE_POSITIONS)
       queuesize--;
     updateInternalIntervals();
     updateTop();
-    current.update(top().interval);
-    //System.out.println("PQ: " + current.toString());
-    return current.begin;
+    return pos = top().start;
   }
 
   @Override
   protected boolean lessThan(DocsEnumRef a, DocsEnumRef b) {
-    if (a.docsEnum.docID() < b.docsEnum.docID())
-      return true;
-    if (a.docsEnum.docID() > b.docsEnum.docID())
-      return false;
-    return a.interval.begin < b.interval.begin;
+    return a.compareTo(b) < 0;
   }
 
-  protected void updateInternalIntervals() {}
+  protected void updateInternalIntervals() throws IOException {}
 
   /**
    * Must be called after the scorers have been advanced
@@ -110,19 +114,19 @@ public class PositionQueue extends Prior
   }
 
   public int startPosition() throws IOException {
-    return current.begin;
+    return top().docsEnum.startPosition();
   }
 
   public int endPosition() throws IOException {
-    return current.end;
+    return top().docsEnum.endPosition();
   }
 
   public int startOffset() throws IOException {
-    return current.offsetBegin;
+    return top().docsEnum.startOffset();
   }
 
   public int endOffset() throws IOException {
-    return current.offsetEnd;
+    return top().docsEnum.endOffset();
   }
 
   public BytesRef getPayload() throws IOException {

Modified: lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/posfilter/Interval.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/posfilter/Interval.java?rev=1643787&r1=1643786&r2=1643787&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/posfilter/Interval.java (original)
+++ lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/posfilter/Interval.java Mon Dec  8 11:01:51 2014
@@ -16,14 +16,14 @@ package org.apache.lucene.search.posfilt
  * limitations under the License.
  */
 
-import org.apache.lucene.index.DocsEnum;
-
 import java.io.IOException;
 
+import org.apache.lucene.index.DocsEnum;
+
 /**
  * Represents a section of a document that matches a query
  */
-public class Interval implements Cloneable {
+class Interval implements Cloneable {
 
   /** The position of the start of this Interval */
   public int begin;
@@ -81,6 +81,20 @@ public class Interval implements Cloneab
     this.offsetEnd = end.offsetEnd;
   }
 
+  public void update(DocsEnum start, Interval end) throws IOException {
+    this.begin = start.startPosition();
+    this.offsetBegin = start.startOffset();
+    this.end = end.end;
+    this.offsetEnd = end.offsetEnd;
+  }
+
+  public void update(Interval start, DocsEnum end) throws IOException {
+    this.begin = start.begin;
+    this.offsetBegin = start.offsetBegin;
+    this.end = end.endPosition();
+    this.offsetEnd = end.endOffset();
+  }
+
   /**
    * Compare with another Interval.
    * @param other the comparator

Modified: lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/posfilter/UnorderedNearQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/posfilter/UnorderedNearQuery.java?rev=1643787&r1=1643786&r2=1643787&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/posfilter/UnorderedNearQuery.java (original)
+++ lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/posfilter/UnorderedNearQuery.java Mon Dec  8 11:01:51 2014
@@ -16,14 +16,14 @@ package org.apache.lucene.search.posfilt
  * limitations under the License.
  */
 
+import java.io.IOException;
+
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.search.PositionQueue;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.similarities.Similarity;
 
-import java.io.IOException;
-
 /**
  * A query that matches if a set of subqueries also match, and are within
  * a given distance of each other within the document.  The subqueries
@@ -31,7 +31,7 @@ import java.io.IOException;
  *
  * N.B. Positions must be included in the index for this query to work
  *
- * Implements the LOWPASS<sub>k</sub> operator as defined in <a href=
+ * Implements the AND operator as defined in <a href=
  * "http://vigna.dsi.unimi.it/ftp/papers/EfficientAlgorithmsMinimalIntervalSemantics"
  * >"Efficient Optimally Lazy Algorithms for Minimal-Interval Semantics"</a>
  *
@@ -86,8 +86,8 @@ public class UnorderedNearQuery extends
         return NO_MORE_POSITIONS;
       do {
         //current.update(posQueue.top().interval, posQueue.span);
-        posQueue.updateCurrent(current);
-        if (current.equals(posQueue.top().interval))
+        posQueue.updateCurrent();
+        if (current.equals(posQueue.top().docsEnum))
           return current.begin;
         matchDistance = posQueue.getMatchDistance();
         posQueue.nextPosition();
@@ -102,86 +102,81 @@ public class UnorderedNearQuery extends
       posQueue.advanceTo(doc);
     }
 
-  }
-
-  private static class SpanningPositionQueue extends PositionQueue {
 
-    Interval span = new Interval();
-    int scorerCount;
-    int firstIntervalEnd;
-    int lastIntervalBegin;
+    private class SpanningPositionQueue extends PositionQueue {
 
-    public SpanningPositionQueue(Scorer[] subScorers) {
-      super(subScorers);
-      scorerCount = subScorers.length;
-    }
+      Interval span = new Interval();
+      int scorerCount;
+      int firstIntervalEnd;
+      int lastIntervalBegin;
+
+      public SpanningPositionQueue(Scorer[] subScorers) {
+        super(subScorers);
+        scorerCount = subScorers.length;
+      }
 
-    public int getMatchDistance() {
-      return lastIntervalBegin - firstIntervalEnd - scorerCount + 1;
-    }
+      public int getMatchDistance() {
+        return lastIntervalBegin - firstIntervalEnd - scorerCount + 1;
+      }
 
-    public boolean isFull() {
-      return queuesize == scorerCount;
-    }
+      public boolean isFull() {
+        return queuesize == scorerCount;
+      }
 
-    public void updateCurrent(Interval current) {
-      final Interval top = this.top().interval;
-      current.update(top, span);
-      this.firstIntervalEnd = top.end;
-    }
+      public void updateCurrent() throws IOException {
+        current.update(this.top().docsEnum, span);
+        this.firstIntervalEnd = this.top().end;
+      }
 
-    private void updateRightExtreme(Interval newRight) {
-      if (span.end <= newRight.end) {
-        span.update(span, newRight);
-        this.lastIntervalBegin = newRight.begin;
+      private void updateRightExtreme(DocsEnum newRight) throws IOException {
+        if (span.end <= newRight.endPosition()) {
+          span.update(span, newRight);
+          this.lastIntervalBegin = newRight.startPosition();
+        }
       }
-    }
 
-    protected void updateInternalIntervals() {
-      updateRightExtreme(top().interval);
-    }
+      protected void updateInternalIntervals() throws IOException {
+        updateRightExtreme(top().docsEnum);
+      }
 
-    @Override
-    public int nextPosition() throws IOException {
-      int position;
-      if ((position = super.nextPosition()) == DocsEnum.NO_MORE_POSITIONS) {
-        return DocsEnum.NO_MORE_POSITIONS;
+      @Override
+      public int nextPosition() throws IOException {
+        int position;
+        if ((position = super.nextPosition()) == DocsEnum.NO_MORE_POSITIONS) {
+          return DocsEnum.NO_MORE_POSITIONS;
+        }
+        span.update(top().docsEnum, span);
+        return position;
       }
-      span.update(top().interval, span);
-      return position;
-    }
 
-    @Override
-    protected void init() throws IOException {
-      super.init();
-      for (Object docsEnumRef : getHeapArray()) {
-        if (docsEnumRef != null) {
-          final Interval i = ((DocsEnumRef) docsEnumRef).interval;
-          updateRightExtreme(i);
+      @Override
+      protected void init() throws IOException {
+        super.init();
+        for (Object docsEnumRef : getHeapArray()) {
+          if (docsEnumRef != null) {
+            updateRightExtreme(((DocsEnumRef) docsEnumRef).docsEnum);
+          }
         }
       }
-    }
 
-    @Override
-    public void advanceTo(int doc) {
-      super.advanceTo(doc);
-      span.reset();
-      firstIntervalEnd = lastIntervalBegin = span.begin;
-    }
+      @Override
+      public void advanceTo(int doc) {
+        super.advanceTo(doc);
+        span.reset();
+        firstIntervalEnd = lastIntervalBegin = span.begin;
+      }
 
-    @Override
-    protected boolean lessThan(DocsEnumRef left, DocsEnumRef right) {
-      final Interval a = left.interval;
-      final Interval b = right.interval;
-      return a.begin < b.begin || (a.begin == b.begin && a.end > b.end);
-    }
+      @Override
+      protected boolean lessThan(DocsEnumRef left, DocsEnumRef right) {
+        int c = left.compareTo(right);
+        if (c != 0)
+          return c < 0;
+        return left.end > right.end;
+      }
 
-    @Override
-    public String toString() {
-      return top().interval.toString();
     }
-  }
 
+  }
 
 }
 

Modified: lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/search/posfilter/IntervalTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/search/posfilter/IntervalTestBase.java?rev=1643787&r1=1643786&r2=1643787&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/search/posfilter/IntervalTestBase.java (original)
+++ lucene/dev/branches/lucene2878/lucene/core/src/test/org/apache/lucene/search/posfilter/IntervalTestBase.java Mon Dec  8 11:01:51 2014
@@ -17,107 +17,81 @@ package org.apache.lucene.search.posfilt
  * limitations under the License.
  */
 
+import java.io.IOException;
+
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.SlowCompositeReaderWrapper;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.CheckHits;
 import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.PositionsCollector;
 import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.Weight;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 
-import java.io.IOException;
-
 public abstract class IntervalTestBase extends LuceneTestCase {
 
   protected Directory directory;
   protected IndexReader reader;
   protected IndexSearcher searcher;
 
-  public static class AssertingPositionsCollector extends PositionsCollector {
-
-    enum AssertionType { POSITIONS, OFFSETS }
+  enum AssertionType { POSITIONS, OFFSETS }
 
-    private final int[][] expectedResults;
-    private final AssertionType type;
-
-    private int docUpto = -1;
-    private int posUpto = -1;
-
-    private int currentDoc = -1;
-    private int posRemaining = 0;
-
-    public AssertingPositionsCollector(int[][] expectedResults, AssertionType type) {
-      this.expectedResults = expectedResults;
-      this.type = type;
-    }
+  protected static void checkMatches(Scorer scorer, int[][] expectedResults, AssertionType type) throws IOException {
 
-    @Override
-    public int postingFeatures() {
-      if (type == AssertionType.POSITIONS)
-        return DocsEnum.FLAG_POSITIONS;
-      else
-        return DocsEnum.FLAG_OFFSETS;
-    }
+    int doc;
+    int docUpto = -1;
 
-    @Override
-    protected void collectPosition(int doc, Interval interval) {
+    while ((doc = scorer.nextDoc()) != DocsEnum.NO_MORE_DOCS) {
 
-      if (doc != currentDoc) {
-        if (posRemaining > 0) {
-          int missingPos = expectedResults[docUpto].length - (posRemaining * 2);
-          fail("Missing expected hit in document " + expectedResults[docUpto][0] + ": [" +
-              expectedResults[docUpto][missingPos] + ", " + expectedResults[docUpto][missingPos + 1] + "]");
+      docUpto++;
+      if (doc != expectedResults[docUpto][0])
+        fail("Expected next hit in document " + expectedResults[docUpto][0] + " but was in " + doc);
+
+      int posUpto = 0;
+      while (scorer.nextPosition() != DocsEnum.NO_MORE_POSITIONS) {
+
+        if (posUpto > ((expectedResults[docUpto].length - 1) / 2) - 1)
+          fail("Unexpected hit in document " + doc + ": " + scorer.toString());
+
+        if (type == AssertionType.POSITIONS) {
+          if (expectedResults[docUpto][posUpto * 2 + 1] != scorer.startPosition() ||
+              expectedResults[docUpto][posUpto * 2 + 2] != scorer.endPosition())
+            fail("Expected next position in document " + doc + " to be [" + expectedResults[docUpto][posUpto * 2 + 1] + ", " +
+                expectedResults[docUpto][posUpto * 2 + 2] + "] but was [" + scorer.startPosition() + ", " + scorer.endPosition() + "]");
+        } else {
+          // check offsets
+          if (expectedResults[docUpto][posUpto * 2 + 1] != scorer.startOffset() ||
+              expectedResults[docUpto][posUpto * 2 + 2] != scorer.endOffset())
+            fail("Expected next offset in document to be [" + expectedResults[docUpto][posUpto * 2 + 1] + ", " +
+                expectedResults[docUpto][posUpto * 2 + 2] + "] but was [" + scorer.startOffset() + ", " + scorer.endOffset() + "]");
         }
-        docUpto++;
-        if (docUpto > expectedResults.length - 1)
-          fail("Unexpected hit in document " + doc + ": " + interval.toString());
-
-        currentDoc = expectedResults[docUpto][0];
-        posUpto = -1;
-        posRemaining = (expectedResults[docUpto].length - 1) / 2;
-      }
-
-      if (doc != currentDoc)
-        fail("Expected next hit in document " + currentDoc + " but was in " + doc + ": " + interval.toString());
 
-      posUpto++;
-      posRemaining--;
+        posUpto++;
+      }
 
-      if (posUpto > ((expectedResults[docUpto].length - 1) / 2) - 1)
-        fail("Unexpected hit in document " + doc + ": " + interval.toString());
+      if (posUpto < (expectedResults[docUpto].length - 1) / 2)
+        fail("Missing expected hit in document " + expectedResults[docUpto][0] + ": [" +
+            expectedResults[docUpto][posUpto] + ", " + expectedResults[docUpto][posUpto + 1] + "]");
 
-      if (type == AssertionType.POSITIONS) {
-        if (expectedResults[docUpto][posUpto * 2 + 1] != interval.begin ||
-            expectedResults[docUpto][posUpto * 2 + 2] != interval.end)
-          fail("Expected next position in document to be [" + expectedResults[docUpto][posUpto * 2 + 1] + ", " +
-              expectedResults[docUpto][posUpto * 2 + 2] + "] but was [" + interval.begin + ", " + interval.end + "]");
-      }
-      else {
-        // check offsets
-        if (expectedResults[docUpto][posUpto * 2 + 1] != interval.offsetBegin ||
-            expectedResults[docUpto][posUpto * 2 + 2] != interval.offsetEnd)
-          fail("Expected next offset in document to be [" + expectedResults[docUpto][posUpto * 2 + 1] + ", " +
-              expectedResults[docUpto][posUpto * 2 + 2] + "] but was [" + interval.offsetBegin + ", " + interval.offsetEnd + "]");
-      }
     }
 
-    public void assertAllMatched() {
-      if (docUpto < expectedResults.length - 1) {
-        fail("Expected a hit in document " + expectedResults[docUpto + 1][0]);
-      }
-    }
+    if (docUpto < expectedResults.length - 1)
+      fail("Missing expected match to document " + expectedResults[docUpto + 1][0]);
+
   }
 
   /**
@@ -132,10 +106,10 @@ public abstract class IntervalTestBase e
    */
   public static void checkIntervalOffsets(Query q, IndexSearcher searcher, int[][] expectedResults) throws IOException {
 
-    //MatchCollector m = new MatchCollector();
-    AssertingPositionsCollector c = new AssertingPositionsCollector(expectedResults, AssertingPositionsCollector.AssertionType.OFFSETS);
-    searcher.search(q, c);
-    c.assertAllMatched();
+    Weight weight = searcher.createNormalizedWeight(q);
+    LeafReaderContext ctx = (LeafReaderContext) searcher.getTopReaderContext();
+    Scorer scorer = weight.scorer(ctx, DocsEnum.FLAG_OFFSETS, ctx.reader().getLiveDocs());
+    checkMatches(scorer, expectedResults, AssertionType.OFFSETS);
 
   }
 
@@ -151,9 +125,10 @@ public abstract class IntervalTestBase e
    */
   public static void checkIntervals(Query q, IndexSearcher searcher, int[][] expectedResults) throws IOException {
 
-    AssertingPositionsCollector c = new AssertingPositionsCollector(expectedResults, AssertingPositionsCollector.AssertionType.POSITIONS);
-    searcher.search(q, c);
-    c.assertAllMatched();
+    Weight weight = searcher.createNormalizedWeight(q);
+    LeafReaderContext ctx = (LeafReaderContext) searcher.getTopReaderContext();
+    Scorer scorer = weight.scorer(ctx, DocsEnum.FLAG_POSITIONS, ctx.reader().getLiveDocs());
+    checkMatches(scorer, expectedResults, AssertionType.POSITIONS);
 
   }
 
@@ -177,7 +152,7 @@ public abstract class IntervalTestBase e
     //config.setCodec(Codec.forName("Asserting"));
     RandomIndexWriter writer = new RandomIndexWriter(random(), directory, config);
     addDocs(writer);
-    reader = writer.getReader();
+    reader = SlowCompositeReaderWrapper.wrap(writer.getReader());
     writer.close();
     searcher = new IndexSearcher(reader);
   }