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

svn commit: r1444832 - in /lucene/dev/branches/LUCENE-2878/lucene: core/src/java/org/apache/lucene/codecs/lucene41/ core/src/java/org/apache/lucene/search/ core/src/java/org/apache/lucene/search/intervals/ core/src/test/org/apache/lucene/search/interva...

Author: romseygeek
Date: Mon Feb 11 15:39:43 2013
New Revision: 1444832

URL: http://svn.apache.org/r1444832
Log:
LUCENE-2878: Add PositionCollector to iterate through positions; TermScorer correctly returns positions

Added:
    lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/PositionsCollector.java   (with props)
Modified:
    lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java
    lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionTermScorer.java
    lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ExactIntervalPhraseScorer.java
    lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/Scorer.java
    lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/TermScorer.java
    lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/intervals/Interval.java
    lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/search/intervals/IntervalTestBase.java
    lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/search/intervals/TestBasicIntervals.java
    lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/index/AssertingAtomicReader.java

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java?rev=1444832&r1=1444831&r2=1444832&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java Mon Feb 11 15:39:43 2013
@@ -280,6 +280,7 @@ public final class Lucene41PostingsReade
     
   @Override
   public DocsEnum docs(FieldInfo fieldInfo, BlockTermState termState, Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
+    /*
     BlockDocsEnum docsEnum;
     if (reuse instanceof BlockDocsEnum) {
       docsEnum = (BlockDocsEnum) reuse;
@@ -290,6 +291,8 @@ public final class Lucene41PostingsReade
       docsEnum = new BlockDocsEnum(fieldInfo);
     }
     return docsEnum.reset(liveDocs, (IntBlockTermState) termState, flags);
+    */
+    return docsAndPositions(fieldInfo, termState, liveDocs, reuse, flags);
   }
 
   // TODO: specialize to liveDocs vs not
@@ -999,6 +1002,16 @@ public final class Lucene41PostingsReade
     }
 
     @Override
+    public int startPosition() {
+      return position;
+    }
+
+    @Override
+    public int endPosition() {
+      return position;
+    }
+
+    @Override
     public int startOffset() {
       return -1;
     }
@@ -1573,6 +1586,16 @@ public final class Lucene41PostingsReade
     }
 
     @Override
+    public int startPosition() {
+      return position;
+    }
+
+    @Override
+    public int endPosition() {
+      return position;
+    }
+
+    @Override
     public int startOffset() {
       return startOffset;
     }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionTermScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionTermScorer.java?rev=1444832&r1=1444831&r2=1444832&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionTermScorer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionTermScorer.java Mon Feb 11 15:39:43 2013
@@ -108,7 +108,7 @@ class ConjunctionTermScorer extends Scor
   }
   
   @Override
-  public int freq() {
+  public int freq() throws IOException {
     return docsAndFreqs.length;
   }
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ExactIntervalPhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ExactIntervalPhraseScorer.java?rev=1444832&r1=1444831&r2=1444832&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ExactIntervalPhraseScorer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ExactIntervalPhraseScorer.java Mon Feb 11 15:39:43 2013
@@ -23,29 +23,38 @@ import java.io.IOException;
  * limitations under the License.
  */
 
-public class ExactIntervalPhraseScorer extends Scorer {
+public class ExactIntervalPhraseScorer extends ConjunctionTermScorer {
 
-  private final Similarity.ExactSimScorer docScorer;
-  private final ChildScorer[] children;
+  private final TermScorer[] children;
   private final Interval[] intervals;
+  private final Similarity.ExactSimScorer docScorer;
   private final boolean matchOnly;
 
+  private boolean cached;
+
   /**
    * Constructs a Scorer
    *
    * @param weight The scorers <code>Weight</code>.
    */
   protected ExactIntervalPhraseScorer(Weight weight, Similarity.ExactSimScorer docScorer,
-                                      boolean matchOnly, Scorer... children) {
-    super(weight);
-    this.docScorer = docScorer;
-    this.children = new ChildScorer[children.length];
+                                      boolean matchOnly, TermScorer... children) {
+    super(weight, 1, wrapChildScorers(children));
+    this.children = children;
     this.intervals = new Interval[children.length];
     for (int i = 0; i < children.length; i++) {
-      this.children[i] = new ChildScorer(children[i], "subphrase");
       this.intervals[i] = new Interval();
     }
     this.matchOnly = matchOnly;
+    this.docScorer = docScorer;
+  }
+
+  private static DocsAndFreqs[] wrapChildScorers(TermScorer... children) {
+    DocsAndFreqs[] docsAndFreqs = new DocsAndFreqs[children.length];
+    for (int i = 0; i < children.length; i++) {
+      docsAndFreqs[i] = new DocsAndFreqs(children[i]);
+    }
+    return docsAndFreqs;
   }
 
   @Override
@@ -55,51 +64,74 @@ public class ExactIntervalPhraseScorer e
 
   @Override
   public float score() throws IOException {
-    return 0;  //To change body of implemented methods use File | Settings | File Templates.
+    return docScorer.score(docID(), freq());
   }
 
   @Override
   public int freq() throws IOException {
-    return 0;  //To change body of implemented methods use File | Settings | File Templates.
+    if (matchOnly)
+      return 1;
+    int freq = 0;
+    while (nextPosition() != NO_MORE_POSITIONS) //nocommit, should we try cacheing here?
+      freq++;
+    return freq;
   }
 
   @Override
   public int docID() {
-    return 0;  //To change body of implemented methods use File | Settings | File Templates.
+    return children[0].docID();
   }
 
   @Override
   public int nextDoc() throws IOException {
-    return 0;  //To change body of implemented methods use File | Settings | File Templates.
+    int doc;
+    resetIntervals();
+    while ((doc = super.nextDoc()) != NO_MORE_DOCS
+        && nextPosition() == NO_MORE_POSITIONS) {
+      resetIntervals();
+    }
+    cached = true;
+    return doc;
   }
 
   @Override
   public int advance(int target) throws IOException {
-    return 0;  //To change body of implemented methods use File | Settings | File Templates.
+    int doc = super.advance(target);
+    resetIntervals();
+    while (doc != NO_MORE_DOCS && nextPosition() == NO_MORE_POSITIONS) {
+      doc = super.nextDoc();
+      resetIntervals();
+    }
+    cached = true;
+    return doc;
   }
 
   public int nextPosition() throws IOException {
-    if (children[0].child.nextPosition() == NO_MORE_POSITIONS)
+    if (cached == true) {
+      cached = false;
+      return children[0].startPosition();
+    }
+    if (children[0].nextPosition() == NO_MORE_POSITIONS)
       return NO_MORE_POSITIONS;
-    intervals[0].update(children[0].child);
+    intervals[0].update(children[0]);
     int i = 1;
     while (i < children.length) {
       while (intervals[i].begin <= intervals[i - 1].end) {
-        if (children[i].child.nextPosition() == NO_MORE_POSITIONS)
+        if (children[i].nextPosition() == NO_MORE_POSITIONS)
           return NO_MORE_POSITIONS;
-        intervals[i].update(children[i].child);
+        intervals[i].update(children[i]);
       }
       if (intervals[i].begin == intervals[i - 1].end) {
         i++;
       }
       else {
-        if (children[0].child.nextPosition() == NO_MORE_POSITIONS)
+        if (children[0].nextPosition() == NO_MORE_POSITIONS)
           return NO_MORE_POSITIONS;
-        intervals[0].update(children[0].child);
+        intervals[0].update(children[0]);
         i = 1;
       }
     }
-    return children[0].child.startPosition();
+    return children[0].startPosition();
   }
 
   private void resetIntervals() {

Added: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/PositionsCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/PositionsCollector.java?rev=1444832&view=auto
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/PositionsCollector.java (added)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/PositionsCollector.java Mon Feb 11 15:39:43 2013
@@ -0,0 +1,88 @@
+package org.apache.lucene.search;
+
+import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.search.intervals.Interval;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+public class PositionsCollector extends Collector {
+
+  private Scorer scorer;
+  private int docsSeen = 0;
+
+  private final int numDocs;
+  private DocPositions[] positions;
+
+  public PositionsCollector(int numDocs) {
+    this.numDocs = numDocs;
+    this.positions = new DocPositions[this.numDocs];
+  }
+
+  @Override
+  public void setScorer(Scorer scorer) throws IOException {
+    this.scorer = scorer;
+  }
+
+  @Override
+  public void collect(int doc) throws IOException {
+    if (docsSeen >= numDocs)
+      return;
+    DocPositions dp = new DocPositions(doc);
+    while (scorer.nextPosition() != DocsEnum.NO_MORE_POSITIONS) {
+      dp.positions.add(new Interval(scorer));
+    }
+    positions[docsSeen] = dp;
+    docsSeen++;
+  }
+
+  @Override
+  public void setNextReader(AtomicReaderContext context) throws IOException {
+
+  }
+
+  @Override
+  public boolean acceptsDocsOutOfOrder() {
+    return true;
+  }
+
+  public DocPositions[] getPositions() {
+    return positions;
+  }
+
+  public int getNumDocs() {
+    return docsSeen;
+  }
+
+  public static class DocPositions {
+
+    public final int doc;
+    public final List<Interval> positions;
+
+    DocPositions(int doc) {
+      this.doc = doc;
+      this.positions = new ArrayList<Interval>();
+    }
+
+  }
+
+}

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/Scorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/Scorer.java?rev=1444832&r1=1444831&r2=1444832&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/Scorer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/Scorer.java Mon Feb 11 15:39:43 2013
@@ -17,14 +17,13 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
+import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.search.intervals.IntervalIterator;
 
 import java.io.IOException;
 import java.util.Collection;
 import java.util.Collections;
 
-import org.apache.lucene.index.DocsEnum;
-
 /**
  * Expert: Common scoring functionality for different types of queries.
  *
@@ -148,6 +147,11 @@ public abstract class Scorer extends Doc
   public Weight getWeight() {
     return weight;
   }
+
+  @Override
+  public int nextPosition() throws IOException {
+    throw new UnsupportedOperationException("nextPosition() is not implemented on " + this.getClass());
+  }
   
   /** Returns child sub-scorers
    * @lucene.experimental */

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/TermScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/TermScorer.java?rev=1444832&r1=1444831&r2=1444832&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/TermScorer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/TermScorer.java Mon Feb 11 15:39:43 2013
@@ -21,6 +21,7 @@ import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.search.intervals.IntervalIterator;
 import org.apache.lucene.search.intervals.TermIntervalIterator;
 import org.apache.lucene.search.similarities.Similarity;
+import org.apache.lucene.util.BytesRef;
 
 import java.io.IOException;
 
@@ -70,6 +71,36 @@ final class TermScorer extends Scorer {
   public int nextDoc() throws IOException {
     return docsEnum.nextDoc();
   }
+
+  @Override
+  public int nextPosition() throws IOException {
+    return docsEnum.nextPosition();
+  }
+
+  @Override
+  public int startPosition() throws IOException {
+    return docsEnum.startPosition();
+  }
+
+  @Override
+  public int endPosition() throws IOException {
+    return docsEnum.endPosition();
+  }
+
+  @Override
+  public int startOffset() throws IOException {
+    return docsEnum.startOffset();
+  }
+
+  @Override
+  public int endOffset() throws IOException {
+    return docsEnum.endOffset();
+  }
+
+  @Override
+  public BytesRef getPayload() throws IOException {
+    return docsEnum.getPayload();
+  }
   
   @Override
   public float score() throws IOException {

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/intervals/Interval.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/intervals/Interval.java?rev=1444832&r1=1444831&r2=1444832&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/intervals/Interval.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/intervals/Interval.java Mon Feb 11 15:39:43 2013
@@ -16,6 +16,7 @@ package org.apache.lucene.search.interva
  * limitations under the License.
  */
 
+import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.search.Scorer;
 
 import java.io.IOException;
@@ -62,6 +63,13 @@ public class Interval implements Cloneab
     this(Integer.MIN_VALUE, Integer.MIN_VALUE, -1, -1);
   }
 
+  public Interval(DocsEnum docsEnum) throws IOException {
+    this.begin = docsEnum.startPosition();
+    this.end = docsEnum.endPosition();
+    this.offsetBegin = docsEnum.startOffset();
+    this.offsetEnd = docsEnum.endOffset();
+  }
+
   /**
    * Update to span the range defined by two other Intervals.
    * @param start the first Interval

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/search/intervals/IntervalTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/search/intervals/IntervalTestBase.java?rev=1444832&r1=1444831&r2=1444832&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/search/intervals/IntervalTestBase.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/search/intervals/IntervalTestBase.java Mon Feb 11 15:39:43 2013
@@ -2,7 +2,6 @@ package org.apache.lucene.search.interva
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.RandomIndexWriter;
@@ -10,13 +9,11 @@ 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.Collector;
 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;
@@ -24,10 +21,7 @@ import org.junit.Assert;
 import org.junit.Before;
 
 import java.io.IOException;
-import java.util.Arrays;
 import java.util.Iterator;
-import java.util.Set;
-import java.util.TreeSet;
 
 /**
  * Copyright (c) 2012 Lemur Consulting Ltd.
@@ -56,32 +50,34 @@ public abstract class IntervalTestBase e
    * the expected results.
    * @param q the query
    * @param searcher the searcher
-   * @param expectedResults and int[][] detailing the expected results, in the format
+   * @param expectedResults an int[][] detailing the expected results, in the format
    *                        { { docid1, startoffset1, endoffset1, startoffset2, endoffset2, ... },
    *                          { docid2, startoffset1, endoffset1, startoffset2, endoffset2, ...}, ... }
    * @throws IOException
    */
   public static void checkIntervalOffsets(Query q, IndexSearcher searcher, int[][] expectedResults) throws IOException {
 
-    MatchCollector m = new MatchCollector();
-    searcher.search(q, m);
+    //MatchCollector m = new MatchCollector();
+    PositionsCollector c = new PositionsCollector(expectedResults.length + 1);
+    searcher.search(q, c);
 
-    Assert.assertEquals("Incorrect number of hits", expectedResults.length, m.getHitCount());
-    Iterator<Match> matchIt = m.getMatches().iterator();
+    PositionsCollector.DocPositions[] matches = c.getPositions();
+    Assert.assertEquals("Incorrect number of hits", expectedResults.length, matches.length);
     for (int i = 0; i < expectedResults.length; i++) {
-      int docMatches[] = expectedResults[i];
-      int docid = docMatches[0];
-      for (int j = 1; j < docMatches.length; j += 2) {
-        String expectation = "Expected match at docid " + docid + ", position " + docMatches[j];
+      int expectedDocMatches[] = expectedResults[i];
+      int docid = expectedDocMatches[0];
+      Iterator<Interval> matchIt = matches[i].positions.iterator();
+      for (int j = 1; j < expectedDocMatches.length; j += 2) {
+        String expectation = "Expected match at docid " + docid + ", position " + expectedDocMatches[j];
         Assert.assertTrue(expectation, matchIt.hasNext());
-        Match match = matchIt.next();
+        Interval match = matchIt.next();
         System.err.println(match);
-        Assert.assertEquals("Incorrect docid", match.docid, docid);
-        Assert.assertEquals("Incorrect match offset", docMatches[j], match.startOffset);
-        Assert.assertEquals("Incorrect match end offset", docMatches[j + 1], match.endOffset);
+        Assert.assertEquals("Incorrect docid", matches[i].doc, docid);
+        Assert.assertEquals("Incorrect match offset", expectedDocMatches[j], match.offsetBegin);
+        Assert.assertEquals("Incorrect match end offset", expectedDocMatches[j + 1], match.offsetEnd);
       }
+      Assert.assertFalse("Unexpected matches!", matchIt.hasNext());
     }
-    Assert.assertFalse("Unexpected matches!", matchIt.hasNext());
 
   }
 
@@ -90,41 +86,33 @@ public abstract class IntervalTestBase e
    * the expected results.
    * @param q the query
    * @param searcher the searcher
-   * @param expectedResults and int[][] detailing the expected results, in the format
+   * @param expectedResults an int[][] detailing the expected results, in the format
    *                        { { docid1, startpos1, endpos1, startpos2, endpos2, ... },
    *                          { docid2, startpos1, endpos1, startpos2, endpos2, ...}, ... }
    * @throws IOException
    */
   public static void checkIntervals(Query q, IndexSearcher searcher, int[][] expectedResults) throws IOException {
 
-    MatchCollector m = new MatchCollector();
-    searcher.search(q, m);
-    Assert.assertEquals("Incorrect number of hits in collecting query", expectedResults.length, m.getHitCount());
-
-    TopDocs td = searcher.search(q, expectedResults.length + 1); // + 1 because you can't pass 0 to search(q, n)
-    Assert.assertEquals("Incorrect number of hits in non-collecting query", expectedResults.length, td.totalHits);
-    int[] topdocsids = new int[td.scoreDocs.length];
-    for (int i = 0; i < topdocsids.length; i++) {
-      topdocsids[i] = td.scoreDocs[i].doc;
-    }
-    Arrays.sort(topdocsids);
+    PositionsCollector c = new PositionsCollector(expectedResults.length + 1);
+    searcher.search(q, c);
 
-    Iterator<Match> matchIt = m.getMatches().iterator();
+    PositionsCollector.DocPositions[] matches = c.getPositions();
+    Assert.assertEquals("Incorrect number of hits", expectedResults.length, c.getNumDocs());
     for (int i = 0; i < expectedResults.length; i++) {
-      int docMatches[] = expectedResults[i];
-      int docid = docMatches[0];
-      Assert.assertEquals("Didn't get a match in document " + docid, docid, topdocsids[i]);
-      for (int j = 1; j < docMatches.length; j += 2) {
-        String expectation = "Expected match at docid " + docid + ", position " + docMatches[j];
+      int expectedDocMatches[] = expectedResults[i];
+      int docid = expectedDocMatches[0];
+      Iterator<Interval> matchIt = matches[i].positions.iterator();
+      for (int j = 1; j < expectedDocMatches.length; j += 2) {
+        String expectation = "Expected match at docid " + docid + ", position " + expectedDocMatches[j];
         Assert.assertTrue(expectation, matchIt.hasNext());
-        Match match = matchIt.next();
-        System.out.println(match);
-        Assert.assertEquals("Incorrect docid", docid, match.docid);
-        Assert.assertEquals("Incorrect match start position", docMatches[j], match.start);
-        Assert.assertEquals("Incorrect match end position", docMatches[j + 1], match.end);
+        Interval match = matchIt.next();
+        System.err.println(match);
+        Assert.assertEquals("Incorrect docid", matches[i].doc, docid);
+        Assert.assertEquals("Incorrect match start position", expectedDocMatches[j], match.begin);
+        Assert.assertEquals("Incorrect match end position", expectedDocMatches[j + 1], match.end);
       }
+      Assert.assertFalse("Unexpected matches!", matchIt.hasNext());
     }
-    Assert.assertFalse("Unexpected matches!", matchIt.hasNext());
 
   }
 
@@ -206,58 +194,5 @@ public abstract class IntervalTestBase e
     }
   }
 
-  public static class MatchCollector extends Collector implements IntervalCollector {
-
-    private IntervalIterator intervals;
-    private Interval current;
-    private Set<Match> matches = new TreeSet<Match>();
-    private int hitCount;
-
-    @Override
-    public void setScorer(Scorer scorer) throws IOException {
-      this.intervals = scorer.intervals(true);
-    }
 
-    @Override
-    public void collect(int doc) throws IOException {
-      hitCount++;
-      intervals.scorerAdvanced(doc);
-      while ((current = intervals.next()) != null) {
-        //System.out.println(doc + ":" + current);
-        intervals.collect(this);
-      }
-    }
-
-    @Override
-    public void setNextReader(AtomicReaderContext context) throws IOException {
-    }
-
-    @Override
-    public boolean acceptsDocsOutOfOrder() {
-      return false;
-    }
-
-    @Override
-    public void collectLeafPosition(Scorer scorer, Interval interval, int docID) {
-      matches.add(new Match(docID, interval, false));
-    }
-
-    @Override
-    public void collectComposite(Scorer scorer, Interval interval, int docID) {
-      matches.add(new Match(docID, interval, true));
-    }
-
-    @Override
-    public Weight.PostingFeatures postingFeatures() {
-      return Weight.PostingFeatures.OFFSETS;
-    }
-
-    public Set<Match> getMatches() {
-      return matches;
-    }
-
-    public int getHitCount() {
-      return hitCount;
-    }
-  }
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/search/intervals/TestBasicIntervals.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/search/intervals/TestBasicIntervals.java?rev=1444832&r1=1444831&r2=1444832&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/search/intervals/TestBasicIntervals.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/search/intervals/TestBasicIntervals.java Mon Feb 11 15:39:43 2013
@@ -20,8 +20,6 @@ package org.apache.lucene.search.interva
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.search.BooleanClause.Occur;
-import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.Query;
 
 import java.io.IOException;
@@ -53,6 +51,8 @@ public class TestBasicIntervals extends 
       "u2 u1 xx u2",//10
       "t1 t2 t1 t3 t2 t3"};//11
 
+  /*
+
   public void testNearOrdered01() throws Exception {
     Query q = new OrderedNearQuery(0, false, makeTermQuery("w1"), makeTermQuery("w2"), makeTermQuery("w3"));
     checkIntervals(q, searcher, new int[][]{
@@ -192,6 +192,8 @@ public class TestBasicIntervals extends 
       "t1 t2 t1 t3 t2 t3"};//11
    */
 
+  /*
+
   // ((u1 near u2) near xx)
   public void testNestedNear() throws Exception {
 
@@ -207,7 +209,7 @@ public class TestBasicIntervals extends 
         { 10, 0, 2, 0, 1, 2, 2 }
     });
 
-  }
+  } */
   
   public void testOrSingle() throws Exception {
     Query q = makeOrQuery(makeTermQuery("w5"));

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/index/AssertingAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/index/AssertingAtomicReader.java?rev=1444832&r1=1444831&r2=1444832&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/index/AssertingAtomicReader.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/index/AssertingAtomicReader.java Mon Feb 11 15:39:43 2013
@@ -1,13 +1,13 @@
 package org.apache.lucene.index;
 
-import java.io.IOException;
-import java.util.Iterator;
-
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
 
+import java.io.IOException;
+import java.util.Iterator;
+
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -129,11 +129,11 @@ public class AssertingAtomicReader exten
 
       // TODO: should we give this thing a random to be super-evil,
       // and randomly *not* unwrap?
-      if (reuse instanceof AssertingDocsAndPositionsEnum) {
-        reuse = ((AssertingDocsAndPositionsEnum) reuse).in;
+      if (reuse instanceof AssertingDocsEnum) {
+        reuse = ((AssertingDocsEnum) reuse).in;
       }
       DocsEnum docs = super.docsAndPositions(liveDocs, reuse, flags);
-      return docs == null ? null : new AssertingDocsAndPositionsEnum(docs);
+      return docs == null ? null : new AssertingDocsEnum(docs);
     }
 
     // TODO: we should separately track if we are 'at the end' ?
@@ -218,6 +218,8 @@ public class AssertingAtomicReader exten
   static enum DocsEnumState { START, ITERATING, FINISHED };
   static class AssertingDocsEnum extends FilterDocsEnum {
     private DocsEnumState state = DocsEnumState.START;
+    int positionCount = 0;
+    int positionMax = 0;
     
     public AssertingDocsEnum(DocsEnum in) {
       super(in);
@@ -232,9 +234,12 @@ public class AssertingAtomicReader exten
       assert nextDoc >= 0 : "invalid doc id: " + nextDoc;
       if (nextDoc == DocIdSetIterator.NO_MORE_DOCS) {
         state = DocsEnumState.FINISHED;
+        positionMax = 0;
       } else {
         state = DocsEnumState.ITERATING;
+        positionMax = super.freq();
       }
+      positionCount = 0;
       return nextDoc;
     }
 
@@ -246,9 +251,12 @@ public class AssertingAtomicReader exten
       assert advanced >= target : "backwards advance from: " + target + " to: " + advanced;
       if (advanced == DocIdSetIterator.NO_MORE_DOCS) {
         state = DocsEnumState.FINISHED;
+        positionMax = 0;
       } else {
         state = DocsEnumState.ITERATING;
+        positionMax = super.freq();
       }
+      positionCount = 0;
       return advanced;
     }
 
@@ -267,6 +275,63 @@ public class AssertingAtomicReader exten
       assert freq > 0;
       return freq;
     }
+
+    @Override
+    public int nextPosition() throws IOException {
+      assert state != DocsEnumState.START : "nextPosition() called before nextDoc()/advance()";
+      assert state != DocsEnumState.FINISHED : "nextPosition() called after NO_MORE_DOCS";
+      int position = super.nextPosition();
+      assert position >= 0 || position == -1 : "invalid position: " + position;
+      if (positionCount++ >= positionMax)
+        assert position == NO_MORE_POSITIONS : "nextPosition() does not return NO_MORE_POSITIONS when exhausted";
+      return position;
+    }
+
+    @Override
+    public int startOffset() throws IOException {
+      assert state != DocsEnumState.START : "startOffset() called before nextDoc()/advance()";
+      assert state != DocsEnumState.FINISHED : "startOffset() called after NO_MORE_DOCS";
+      assert positionCount > 0 : "startOffset() called before nextPosition()!";
+      assert positionCount <= positionMax : "startOffset() called after NO_MORE_POSITIONS";
+      return super.startOffset();
+    }
+
+    @Override
+    public int endOffset() throws IOException {
+      assert state != DocsEnumState.START : "endOffset() called before nextDoc()/advance()";
+      assert state != DocsEnumState.FINISHED : "endOffset() called after NO_MORE_DOCS";
+      assert positionCount > 0 : "endOffset() called before nextPosition()!";
+      assert positionCount <= positionMax : "endOffset() called after NO_MORE_POSITIONS";
+      return super.endOffset();
+    }
+
+    @Override
+    public int startPosition() throws IOException {
+      assert state != DocsEnumState.START : "startPosition() called before nextDoc()/advance()";
+      assert state != DocsEnumState.FINISHED : "startPosition() called after NO_MORE_DOCS";
+      assert positionCount > 0 : "startPosition() called before nextPosition()!";
+      assert positionCount <= positionMax : "startPosition() called after NO_MORE_POSITIONS";
+      return super.startPosition();
+    }
+
+    @Override
+    public int endPosition() throws IOException {
+      assert state != DocsEnumState.START : "endPosition() called before nextDoc()/advance()";
+      assert state != DocsEnumState.FINISHED : "endPosition() called after NO_MORE_DOCS";
+      assert positionCount > 0 : "endPosition() called before nextPosition()!";
+      assert positionCount <= positionMax : "endPosition() called after NO_MORE_POSITIONS";
+      return super.endPosition();
+    }
+
+    @Override
+    public BytesRef getPayload() throws IOException {
+      assert state != DocsEnumState.START : "getPayload() called before nextDoc()/advance()";
+      assert state != DocsEnumState.FINISHED : "getPayload() called after NO_MORE_DOCS";
+      assert positionCount > 0 : "getPayload() called before nextPosition()!";
+      BytesRef payload = super.getPayload();
+      assert payload == null || payload.length > 0 : "getPayload() returned payload with invalid length!";
+      return payload;
+    }
   }
   
   static class AssertingDocsAndPositionsEnum extends FilterDocsEnum {