Return-Path: Delivered-To: apmail-lucene-java-commits-archive@www.apache.org Received: (qmail 65684 invoked from network); 27 Oct 2006 22:04:18 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.2) by minotaur.apache.org with SMTP; 27 Oct 2006 22:04:18 -0000 Received: (qmail 58673 invoked by uid 500); 27 Oct 2006 22:04:29 -0000 Delivered-To: apmail-lucene-java-commits-archive@lucene.apache.org Received: (qmail 58659 invoked by uid 500); 27 Oct 2006 22:04:29 -0000 Mailing-List: contact java-commits-help@lucene.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: java-dev@lucene.apache.org Delivered-To: mailing list java-commits@lucene.apache.org Received: (qmail 58648 invoked by uid 99); 27 Oct 2006 22:04:29 -0000 Received: from herse.apache.org (HELO herse.apache.org) (140.211.11.133) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 27 Oct 2006 15:04:29 -0700 X-ASF-Spam-Status: No, hits=0.6 required=10.0 tests=NO_REAL_NAME X-Spam-Check-By: apache.org Received: from [140.211.11.3] (HELO eris.apache.org) (140.211.11.3) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 27 Oct 2006 15:04:16 -0700 Received: by eris.apache.org (Postfix, from userid 65534) id A70591A9846; Fri, 27 Oct 2006 15:03:54 -0700 (PDT) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r468554 - in /lucene/java/trunk: ./ src/java/org/apache/lucene/search/spans/ src/test/org/apache/lucene/search/spans/ Date: Fri, 27 Oct 2006 22:03:54 -0000 To: java-commits@lucene.apache.org From: hossman@apache.org X-Mailer: svnmailer-1.1.0 Message-Id: <20061027220354.A70591A9846@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: hossman Date: Fri Oct 27 15:03:53 2006 New Revision: 468554 URL: http://svn.apache.org/viewvc?view=rev&rev=468554 Log: LUCENE-569: Fixed bug in ordered SpanNearQueries Added: lucene/java/trunk/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java (with props) lucene/java/trunk/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java (with props) lucene/java/trunk/src/test/org/apache/lucene/search/spans/TestNearSpansOrdered.java (with props) Removed: lucene/java/trunk/src/java/org/apache/lucene/search/spans/NearSpans.java Modified: lucene/java/trunk/CHANGES.txt lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanNearQuery.java lucene/java/trunk/src/test/org/apache/lucene/search/spans/TestSpanExplanations.java Modified: lucene/java/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/lucene/java/trunk/CHANGES.txt?view=diff&rev=468554&r1=468553&r2=468554 ============================================================================== --- lucene/java/trunk/CHANGES.txt (original) +++ lucene/java/trunk/CHANGES.txt Fri Oct 27 15:03:53 2006 @@ -161,6 +161,10 @@ 18. LUCENE-696: Fix bug when scorer for DisjunctionMaxQuery has skipTo() called on it before next(). (Yonik Seeley) +19. LUCENE-569: Fixed SpanNearQuery bug, for 'inOrder' queries it would fail + to recognize ordered spans if they overlaped with unordered spans. + (Paul Elschot via Chris Hostetter) + Optimizations 1. LUCENE-586: TermDocs.skipTo() is now more efficient for Added: lucene/java/trunk/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java?view=auto&rev=468554 ============================================================================== --- lucene/java/trunk/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java (added) +++ lucene/java/trunk/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java Fri Oct 27 15:03:53 2006 @@ -0,0 +1,265 @@ +package org.apache.lucene.search.spans; + +/** + * Copyright 2006 The Apache Software Foundation + * + * 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 + * + * 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. + */ + +import java.io.IOException; + +import java.util.Arrays; +import java.util.Comparator; + +import org.apache.lucene.index.IndexReader; + +/** A Spans that is formed from the ordered subspans of a SpanNearQuery + * where the subspans do not overlap and have a maximum slop between them. + *

+ * The formed spans only contains minimum slop matches.
+ * The matching slop is computed from the distance(s) between + * the non overlapping matching Spans.
+ * Successive matches are always formed from the successive Spans + * of the SpanNearQuery. + *

+ * The formed spans may contain overlaps when the slop is at least 1. + * For example, when querying using + *

t1 t2 t3
+ * with slop at least 1, the fragment: + *
t1 t2 t1 t3 t2 t3
+ * matches twice: + *
t1 t2 .. t3      
+ *
      t1 .. t2 t3
+ */ +class NearSpansOrdered implements Spans { + private final int allowedSlop; + private boolean firstTime = true; + private boolean more = false; + + /** The spans in the same order as the SpanNearQuery */ + private final Spans[] subSpans; + + /** Indicates that all subSpans have same doc() */ + private boolean inSameDoc = false; + + private int matchDoc = -1; + private int matchStart = -1; + private int matchEnd = -1; + + private final Spans[] subSpansByDoc; + private final Comparator spanDocComparator = new Comparator() { + public int compare(Object o1, Object o2) { + return ((Spans)o1).doc() - ((Spans)o2).doc(); + } + }; + + private SpanNearQuery query; + + public NearSpansOrdered(SpanNearQuery spanNearQuery, IndexReader reader) + throws IOException { + if (spanNearQuery.getClauses().length < 2) { + throw new IllegalArgumentException("Less than 2 clauses: " + + spanNearQuery); + } + allowedSlop = spanNearQuery.getSlop(); + SpanQuery[] clauses = spanNearQuery.getClauses(); + subSpans = new Spans[clauses.length]; + subSpansByDoc = new Spans[clauses.length]; + for (int i = 0; i < clauses.length; i++) { + subSpans[i] = clauses[i].getSpans(reader); + subSpansByDoc[i] = subSpans[i]; // used in toSameDoc() + } + query = spanNearQuery; // kept for toString() only. + } + + // inherit javadocs + public int doc() { return matchDoc; } + + // inherit javadocs + public int start() { return matchStart; } + + // inherit javadocs + public int end() { return matchEnd; } + + // inherit javadocs + public boolean next() throws IOException { + if (firstTime) { + firstTime = false; + for (int i = 0; i < subSpans.length; i++) { + if (! subSpans[i].next()) { + more = false; + return false; + } + } + more = true; + } + return advanceAfterOrdered(); + } + + // inherit javadocs + public boolean skipTo(int target) throws IOException { + if (firstTime) { + firstTime = false; + for (int i = 0; i < subSpans.length; i++) { + if (! subSpans[i].skipTo(target)) { + more = false; + return false; + } + } + more = true; + } else if (more && (subSpans[0].doc() < target)) { + if (subSpans[0].skipTo(target)) { + inSameDoc = false; + } else { + more = false; + return false; + } + } + return advanceAfterOrdered(); + } + + /** Advances the subSpans to just after an ordered match with a minimum slop + * that is smaller than the slop allowed by the SpanNearQuery. + * @return true iff there is such a match. + */ + private boolean advanceAfterOrdered() throws IOException { + while (more && (inSameDoc || toSameDoc())) { + if (stretchToOrder() && shrinkToAfterShortestMatch()) { + return true; + } + } + return false; // no more matches + } + + + /** Advance the subSpans to the same document */ + private boolean toSameDoc() throws IOException { + Arrays.sort(subSpansByDoc, spanDocComparator); + int firstIndex = 0; + int maxDoc = subSpansByDoc[subSpansByDoc.length - 1].doc(); + while (subSpansByDoc[firstIndex].doc() != maxDoc) { + if (! subSpansByDoc[firstIndex].skipTo(maxDoc)) { + more = false; + inSameDoc = false; + return false; + } + maxDoc = subSpansByDoc[firstIndex].doc(); + if (++firstIndex == subSpansByDoc.length) { + firstIndex = 0; + } + } + for (int i = 0; i < subSpansByDoc.length; i++) { + assert (subSpansByDoc[i].doc() == maxDoc) + : " NearSpansOrdered.toSameDoc() spans " + subSpansByDoc[0] + + "\n at doc " + subSpansByDoc[i].doc() + + ", but should be at " + maxDoc; + } + inSameDoc = true; + return true; + } + + /** Check whether two Spans in the same document are ordered. + * @param spans1 + * @param spans2 + * @return true iff spans1 starts before spans2 + * or the spans start at the same position, + * and spans1 ends before spans2. + */ + static final boolean docSpansOrdered(Spans spans1, Spans spans2) { + assert spans1.doc() == spans2.doc() : "doc1 " + spans1.doc() + " != doc2 " + spans2.doc(); + int start1 = spans1.start(); + int start2 = spans2.start(); + /* Do not call docSpansOrdered(int,int,int,int) to avoid invoking .end() : */ + return (start1 == start2) ? (spans1.end() < spans2.end()) : (start1 < start2); + } + + /** Like {@link #docSpansOrdered(Spans,Spans)}, but use the spans + * starts and ends as parameters. + */ + private static final boolean docSpansOrdered(int start1, int end1, int start2, int end2) { + return (start1 == start2) ? (end1 < end2) : (start1 < start2); + } + + /** Order the subSpans within the same document by advancing all later spans + * after the previous one. + */ + private boolean stretchToOrder() throws IOException { + matchDoc = subSpans[0].doc(); + for (int i = 1; inSameDoc && (i < subSpans.length); i++) { + while (! docSpansOrdered(subSpans[i-1], subSpans[i])) { + if (! subSpans[i].next()) { + inSameDoc = false; + more = false; + break; + } else if (matchDoc != subSpans[i].doc()) { + inSameDoc = false; + break; + } + } + } + return inSameDoc; + } + + /** The subSpans are ordered in the same doc, so there is a possible match. + * Compute the slop while making the match as short as possible by advancing + * all subSpans except the last one in reverse order. + */ + private boolean shrinkToAfterShortestMatch() throws IOException { + matchStart = subSpans[subSpans.length - 1].start(); + matchEnd = subSpans[subSpans.length - 1].end(); + int matchSlop = 0; + int lastStart = matchStart; + int lastEnd = matchEnd; + for (int i = subSpans.length - 2; i >= 0; i--) { + Spans prevSpans = subSpans[i]; + int prevStart = prevSpans.start(); + int prevEnd = prevSpans.end(); + while (true) { // Advance prevSpans until after (lastStart, lastEnd) + if (! prevSpans.next()) { + inSameDoc = false; + more = false; + break; // Check remaining subSpans for final match. + } else if (matchDoc != prevSpans.doc()) { + inSameDoc = false; // The last subSpans is not advanced here. + break; // Check remaining subSpans for last match in this document. + } else { + int ppStart = prevSpans.start(); + int ppEnd = prevSpans.end(); // Cannot avoid invoking .end() + if (! docSpansOrdered(ppStart, ppEnd, lastStart, lastEnd)) { + break; // Check remaining subSpans. + } else { // prevSpans still before (lastStart, lastEnd) + prevStart = ppStart; + prevEnd = ppEnd; + } + } + } + assert prevStart <= matchStart; + if (matchStart > prevEnd) { // Only non overlapping spans add to slop. + matchSlop += (matchStart - prevEnd); + } + /* Do not break on (matchSlop > allowedSlop) here to make sure + * that subSpans[0] is advanced after the match, if any. + */ + matchStart = prevStart; + lastStart = prevStart; + lastEnd = prevEnd; + } + return matchSlop <= allowedSlop; // ordered and allowed slop + } + + public String toString() { + return getClass().getName() + "("+query.toString()+")@"+ + (firstTime?"START":(more?(doc()+":"+start()+"-"+end()):"END")); + } +} + Propchange: lucene/java/trunk/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java ------------------------------------------------------------------------------ svn:eol-style = native Added: lucene/java/trunk/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java?view=auto&rev=468554 ============================================================================== --- lucene/java/trunk/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java (added) +++ lucene/java/trunk/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java Fri Oct 27 15:03:53 2006 @@ -0,0 +1,250 @@ +package org.apache.lucene.search.spans; + +/** + * Copyright 2006 The Apache Software Foundation + * + * 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 + * + * 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. + */ + +import java.io.IOException; + +import java.util.List; +import java.util.ArrayList; + +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.util.PriorityQueue; + +class NearSpansUnordered implements Spans { + private SpanNearQuery query; + + private List ordered = new ArrayList(); // spans in query order + private int slop; // from query + + private SpansCell first; // linked list of spans + private SpansCell last; // sorted by doc only + + private int totalLength; // sum of current lengths + + private CellQueue queue; // sorted queue of spans + private SpansCell max; // max element in queue + + private boolean more = true; // true iff not done + private boolean firstTime = true; // true before first next() + + private class CellQueue extends PriorityQueue { + public CellQueue(int size) { + initialize(size); + } + + protected final boolean lessThan(Object o1, Object o2) { + SpansCell spans1 = (SpansCell)o1; + SpansCell spans2 = (SpansCell)o2; + if (spans1.doc() == spans2.doc()) { + return NearSpansOrdered.docSpansOrdered(spans1, spans2); + } else { + return spans1.doc() < spans2.doc(); + } + } + } + + + /** Wraps a Spans, and can be used to form a linked list. */ + private class SpansCell implements Spans { + private Spans spans; + private SpansCell next; + private int length = -1; + private int index; + + public SpansCell(Spans spans, int index) { + this.spans = spans; + this.index = index; + } + + public boolean next() throws IOException { + return adjust(spans.next()); + } + + public boolean skipTo(int target) throws IOException { + return adjust(spans.skipTo(target)); + } + + private boolean adjust(boolean condition) { + if (length != -1) { + totalLength -= length; // subtract old length + } + if (condition) { + length = end() - start(); + totalLength += length; // add new length + + if (max == null || doc() > max.doc() + || (doc() == max.doc()) && (end() > max.end())) { + max = this; + } + } + more = condition; + return condition; + } + + public int doc() { return spans.doc(); } + public int start() { return spans.start(); } + public int end() { return spans.end(); } + + public String toString() { return spans.toString() + "#" + index; } + } + + + public NearSpansUnordered(SpanNearQuery query, IndexReader reader) + throws IOException { + this.query = query; + this.slop = query.getSlop(); + + SpanQuery[] clauses = query.getClauses(); + queue = new CellQueue(clauses.length); + for (int i = 0; i < clauses.length; i++) { + SpansCell cell = + new SpansCell(clauses[i].getSpans(reader), i); + ordered.add(cell); + } + } + + public boolean next() throws IOException { + if (firstTime) { + initList(true); + listToQueue(); // initialize queue + firstTime = false; + } else if (more) { + if (min().next()) { // trigger further scanning + queue.adjustTop(); // maintain queue + } else { + more = false; + } + } + + while (more) { + + boolean queueStale = false; + + if (min().doc() != max.doc()) { // maintain list + queueToList(); + queueStale = true; + } + + // skip to doc w/ all clauses + + while (more && first.doc() < last.doc()) { + more = first.skipTo(last.doc()); // skip first upto last + firstToLast(); // and move it to the end + queueStale = true; + } + + if (!more) return false; + + // found doc w/ all clauses + + if (queueStale) { // maintain the queue + listToQueue(); + queueStale = false; + } + + if (atMatch()) { + return true; + } + + more = min().next(); + if (more) { + queue.adjustTop(); // maintain queue + } + } + return false; // no more matches + } + + public boolean skipTo(int target) throws IOException { + if (firstTime) { // initialize + initList(false); + for (SpansCell cell = first; more && cell!=null; cell=cell.next) { + more = cell.skipTo(target); // skip all + } + if (more) { + listToQueue(); + } + firstTime = false; + } else { // normal case + while (more && min().doc() < target) { // skip as needed + if (min().skipTo(target)) { + queue.adjustTop(); + } else { + more = false; + } + } + } + return more && (atMatch() || next()); + } + + private SpansCell min() { return (SpansCell)queue.top(); } + + public int doc() { return min().doc(); } + public int start() { return min().start(); } + public int end() { return max.end(); } + + + public String toString() { + return getClass().getName() + "("+query.toString()+")@"+ + (firstTime?"START":(more?(doc()+":"+start()+"-"+end()):"END")); + } + + private void initList(boolean next) throws IOException { + for (int i = 0; more && i < ordered.size(); i++) { + SpansCell cell = (SpansCell)ordered.get(i); + if (next) + more = cell.next(); // move to first entry + if (more) { + addToList(cell); // add to list + } + } + } + + private void addToList(SpansCell cell) { + if (last != null) { // add next to end of list + last.next = cell; + } else + first = cell; + last = cell; + cell.next = null; + } + + private void firstToLast() { + last.next = first; // move first to end of list + last = first; + first = first.next; + last.next = null; + } + + private void queueToList() { + last = first = null; + while (queue.top() != null) { + addToList((SpansCell)queue.pop()); + } + } + + private void listToQueue() { + queue.clear(); // rebuild queue + for (SpansCell cell = first; cell != null; cell = cell.next) { + queue.put(cell); // add to queue from list + } + } + + private boolean atMatch() { + return (min().doc() == max.doc()) + && ((max.end() - min().start() - totalLength) <= slop); + } +} Propchange: lucene/java/trunk/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java ------------------------------------------------------------------------------ svn:eol-style = native Modified: lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanNearQuery.java URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanNearQuery.java?view=diff&rev=468554&r1=468553&r2=468554 ============================================================================== --- lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanNearQuery.java (original) +++ lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanNearQuery.java Fri Oct 27 15:03:53 2006 @@ -124,7 +124,9 @@ if (clauses.size() == 1) // optimize 1-clause case return ((SpanQuery)clauses.get(0)).getSpans(reader); - return new NearSpans(this, reader); + return inOrder + ? (Spans) new NearSpansOrdered(this, reader) + : (Spans) new NearSpansUnordered(this, reader); } public Query rewrite(IndexReader reader) throws IOException { Added: lucene/java/trunk/src/test/org/apache/lucene/search/spans/TestNearSpansOrdered.java URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/search/spans/TestNearSpansOrdered.java?view=auto&rev=468554 ============================================================================== --- lucene/java/trunk/src/test/org/apache/lucene/search/spans/TestNearSpansOrdered.java (added) +++ lucene/java/trunk/src/test/org/apache/lucene/search/spans/TestNearSpansOrdered.java Fri Oct 27 15:03:53 2006 @@ -0,0 +1,182 @@ +package org.apache.lucene.search.spans; + +/** + * Copyright 2006 Apache Software Foundation + * + * 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 + * + * 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. + */ + +import org.apache.lucene.search.Explanation; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.Weight; +import org.apache.lucene.search.CheckHits; +import org.apache.lucene.store.RAMDirectory; + +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.Term; + +import org.apache.lucene.analysis.WhitespaceAnalyzer; + +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field; + +import org.apache.lucene.queryParser.QueryParser; + +import junit.framework.TestCase; + +public class TestNearSpansOrdered extends TestCase { + protected IndexSearcher searcher; + + public static final String FIELD = "field"; + public static final QueryParser qp = + new QueryParser(FIELD, new WhitespaceAnalyzer()); + + public void tearDown() throws Exception { + searcher.close(); + } + + public void setUp() throws Exception { + RAMDirectory directory = new RAMDirectory(); + IndexWriter writer= new IndexWriter(directory, new WhitespaceAnalyzer(), true); + for (int i = 0; i < docFields.length; i++) { + Document doc = new Document(); + doc.add(new Field(FIELD, docFields[i], Field.Store.NO, Field.Index.TOKENIZED)); + writer.addDocument(doc); + } + writer.close(); + searcher = new IndexSearcher(directory); + } + + protected String[] docFields = { + "w1 w2 w3 w4 w5", + "w1 w3 w2 w3 zz", + "w1 xx w2 yy w3", + "w1 w3 xx w2 yy w3 zz" + }; + + protected SpanNearQuery makeQuery(String s1, String s2, String s3, + int slop, boolean inOrder) { + return new SpanNearQuery + (new SpanQuery[] { + new SpanTermQuery(new Term(FIELD, s1)), + new SpanTermQuery(new Term(FIELD, s2)), + new SpanTermQuery(new Term(FIELD, s3)) }, + slop, + inOrder); + } + protected SpanNearQuery makeQuery() { + return makeQuery("w1","w2","w3",1,true); + } + + public void testSpanNearQuery() throws Exception { + SpanNearQuery q = makeQuery(); + CheckHits.checkHits(q, FIELD, searcher, new int[] {0,1}); + } + + public String s(Spans span) { + return s(span.doc(), span.start(), span.end()); + } + public String s(int doc, int start, int end) { + return "s(" + doc + "," + start + "," + end +")"; + } + + public void testNearSpansNext() throws Exception { + SpanNearQuery q = makeQuery(); + Spans span = q.getSpans(searcher.getIndexReader()); + assertEquals(true, span.next()); + assertEquals(s(0,0,3), s(span)); + assertEquals(true, span.next()); + assertEquals(s(1,0,4), s(span)); + assertEquals(false, span.next()); + } + + /** + * test does not imply that skipTo(doc+1) should work exactly the + * same as next -- it's only applicable in this case since we know doc + * does not contain more than one span + */ + public void testNearSpansSkipToLikeNext() throws Exception { + SpanNearQuery q = makeQuery(); + Spans span = q.getSpans(searcher.getIndexReader()); + assertEquals(true, span.skipTo(0)); + assertEquals(s(0,0,3), s(span)); + assertEquals(true, span.skipTo(1)); + assertEquals(s(1,0,4), s(span)); + assertEquals(false, span.skipTo(2)); + } + + public void testNearSpansNextThenSkipTo() throws Exception { + SpanNearQuery q = makeQuery(); + Spans span = q.getSpans(searcher.getIndexReader()); + assertEquals(true, span.next()); + assertEquals(s(0,0,3), s(span)); + assertEquals(true, span.skipTo(1)); + assertEquals(s(1,0,4), s(span)); + assertEquals(false, span.next()); + } + + public void testNearSpansNextThenSkipPast() throws Exception { + SpanNearQuery q = makeQuery(); + Spans span = q.getSpans(searcher.getIndexReader()); + assertEquals(true, span.next()); + assertEquals(s(0,0,3), s(span)); + assertEquals(false, span.skipTo(2)); + } + + public void testNearSpansSkipPast() throws Exception { + SpanNearQuery q = makeQuery(); + Spans span = q.getSpans(searcher.getIndexReader()); + assertEquals(false, span.skipTo(2)); + } + + public void testNearSpansSkipTo0() throws Exception { + SpanNearQuery q = makeQuery(); + Spans span = q.getSpans(searcher.getIndexReader()); + assertEquals(true, span.skipTo(0)); + assertEquals(s(0,0,3), s(span)); + } + + public void testNearSpansSkipTo1() throws Exception { + SpanNearQuery q = makeQuery(); + Spans span = q.getSpans(searcher.getIndexReader()); + assertEquals(true, span.skipTo(1)); + assertEquals(s(1,0,4), s(span)); + } + + /** + * not a direct test of NearSpans, but a demonstration of how/when + * this causes problems + */ + public void testSpanNearScorerSkipTo1() throws Exception { + SpanNearQuery q = makeQuery(); + Weight w = q.createWeight(searcher); + Scorer s = w.scorer(searcher.getIndexReader()); + assertEquals(true, s.skipTo(1)); + assertEquals(1, s.doc()); + } + /** + * not a direct test of NearSpans, but a demonstration of how/when + * this causes problems + */ + public void testSpanNearScorerExplain() throws Exception { + SpanNearQuery q = makeQuery(); + Weight w = q.createWeight(searcher); + Scorer s = w.scorer(searcher.getIndexReader()); + Explanation e = s.explain(1); + assertTrue("Scorer explanation value for doc#1 isn't positive: " + + e.toString(), + 0.0f < e.getValue()); + } + +} Propchange: lucene/java/trunk/src/test/org/apache/lucene/search/spans/TestNearSpansOrdered.java ------------------------------------------------------------------------------ svn:eol-style = native Modified: lucene/java/trunk/src/test/org/apache/lucene/search/spans/TestSpanExplanations.java URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/search/spans/TestSpanExplanations.java?view=diff&rev=468554&r1=468553&r2=468554 ============================================================================== --- lucene/java/trunk/src/test/org/apache/lucene/search/spans/TestSpanExplanations.java (original) +++ lucene/java/trunk/src/test/org/apache/lucene/search/spans/TestSpanExplanations.java Fri Oct 27 15:03:53 2006 @@ -151,6 +151,10 @@ SpanQuery q = snear("xx","w1","w2",0,false); qtest(q, new int[] {2}); } + public void testSNear11() throws Exception { + SpanQuery q = snear("w1","w2","w3",1,true); + qtest(q, new int[] {0,1}); + } /* some SpanNotQueries */