Index: lucene/src/java/org/apache/lucene/search/BooleanQuery.java =================================================================== --- lucene/src/java/org/apache/lucene/search/BooleanQuery.java (revision 1135110) +++ lucene/src/java/org/apache/lucene/search/BooleanQuery.java (working copy) @@ -299,7 +299,7 @@ Iterator cIter = clauses.iterator(); for (Weight w : weights) { BooleanClause c = cIter.next(); - Scorer subScorer = w.scorer(context, ScorerContext.def()); + Scorer subScorer = w.scorer(context, scorerContext.topScorer(false)); if (subScorer == null) { if (c.isRequired()) { return null; @@ -329,7 +329,7 @@ } // Return a BooleanScorer2 - return new BooleanScorer2(this, disableCoord, minNrShouldMatch, required, prohibited, optional, maxCoord); + return new BooleanScorer2(this, disableCoord, minNrShouldMatch, required, prohibited, optional, maxCoord, scorerContext.needsPositions); } @Override Index: lucene/src/java/org/apache/lucene/search/BooleanScorer.java =================================================================== --- lucene/src/java/org/apache/lucene/search/BooleanScorer.java (revision 1135110) +++ lucene/src/java/org/apache/lucene/search/BooleanScorer.java (working copy) @@ -23,6 +23,7 @@ import org.apache.lucene.index.IndexReader.AtomicReaderContext; import org.apache.lucene.search.BooleanClause.Occur; import org.apache.lucene.search.BooleanQuery.BooleanWeight; +import org.apache.lucene.search.positions.PositionIntervalIterator; /* Description from Doug Cutting (excerpted from * LUCENE-1483): @@ -375,4 +376,9 @@ } } + @Override + public PositionIntervalIterator positions() throws IOException { + return super.positions(); + } + } Index: lucene/src/java/org/apache/lucene/search/BooleanScorer2.java =================================================================== --- lucene/src/java/org/apache/lucene/search/BooleanScorer2.java (revision 1135110) +++ lucene/src/java/org/apache/lucene/search/BooleanScorer2.java (working copy) @@ -23,6 +23,7 @@ import org.apache.lucene.search.BooleanClause.Occur; import org.apache.lucene.search.BooleanQuery.BooleanWeight; +import org.apache.lucene.search.positions.PositionIntervalIterator; /* See the description in BooleanScorer.java, comparing * BooleanScorer & BooleanScorer2 */ @@ -62,6 +63,7 @@ private final int minNrShouldMatch; private int doc = -1; + private boolean needsPositions; /** * Creates a {@link Scorer} with the given similarity and lists of required, @@ -85,12 +87,13 @@ * the list of optional scorers. */ public BooleanScorer2(BooleanWeight weight, boolean disableCoord, int minNrShouldMatch, - List required, List prohibited, List optional, int maxCoord) throws IOException { + List required, List prohibited, List optional, int maxCoord, boolean needsPositions) throws IOException { super(weight); if (minNrShouldMatch < 0) { throw new IllegalArgumentException("Minimum number of optional scorers should not be negative"); } coordinator = new Coordinator(); + this.needsPositions = needsPositions; this.minNrShouldMatch = minNrShouldMatch; coordinator.maxCoord = maxCoord; @@ -170,7 +173,7 @@ List requiredScorers) throws IOException { // each scorer from the list counted as a single matcher final int requiredNrMatchers = requiredScorers.size(); - return new ConjunctionScorer(weight, disableCoord ? 1.0f : ((BooleanWeight)weight).coord(requiredScorers.size(), requiredScorers.size()), requiredScorers) { + return new ConjunctionScorer(weight, disableCoord ? 1.0f : ((BooleanWeight)weight).coord(requiredScorers.size(), requiredScorers.size()), needsPositions, requiredScorers) { private int lastScoredDoc = -1; // Save the score of lastScoredDoc, so that we don't compute it more than // once in score(). @@ -195,7 +198,7 @@ private Scorer dualConjunctionSumScorer(boolean disableCoord, Scorer req1, Scorer req2) throws IOException { // non counting. - return new ConjunctionScorer(weight, disableCoord ? 1.0f : ((BooleanWeight)weight).coord(2, 2), req1, req2); + return new ConjunctionScorer(weight, disableCoord ? 1.0f : ((BooleanWeight)weight).coord(2, 2), needsPositions, req1, req2); // All scorers match, so defaultSimilarity always has 1 as // the coordination factor. // Therefore the sum of the scores of two scorers @@ -316,6 +319,13 @@ public int advance(int target) throws IOException { return doc = countingSumScorer.advance(target); } + + + + @Override + public PositionIntervalIterator positions() throws IOException { + return countingSumScorer.positions(); + } @Override protected void visitSubScorers(Query parent, Occur relationship, ScorerVisitor visitor) { Index: lucene/src/java/org/apache/lucene/search/ConjunctionScorer.java =================================================================== --- lucene/src/java/org/apache/lucene/search/ConjunctionScorer.java (revision 1135110) +++ lucene/src/java/org/apache/lucene/search/ConjunctionScorer.java (working copy) @@ -17,7 +17,10 @@ * limitations under the License. */ +import org.apache.lucene.search.positions.ConjunctionPositionIterator; +import org.apache.lucene.search.positions.PositionIntervalIterator; import org.apache.lucene.util.ArrayUtil; + import java.io.IOException; import java.util.Collection; import java.util.Comparator; @@ -25,16 +28,23 @@ /** Scorer for conjunctions, sets of queries, all of which are required. */ class ConjunctionScorer extends Scorer { + private final Scorer[] scorersOrdered; private final Scorer[] scorers; private final float coord; private int lastDoc = -1; - public ConjunctionScorer(Weight weight, float coord, Collection scorers) throws IOException { - this(weight, coord, scorers.toArray(new Scorer[scorers.size()])); + public ConjunctionScorer(Weight weight, float coord, boolean needsPositions, Collection scorers) throws IOException { + this(weight, coord, needsPositions, scorers.toArray(new Scorer[scorers.size()])); } - - public ConjunctionScorer(Weight weight, float coord, Scorer... scorers) throws IOException { + + public ConjunctionScorer(Weight weight, float coord, boolean needsPositions, Scorer... scorers) throws IOException { super(weight); + if (needsPositions) { + scorersOrdered = new Scorer[scorers.length]; + System.arraycopy(scorers, 0, scorersOrdered, 0, scorers.length); + } else { + scorersOrdered = null; + } this.scorers = scorers; this.coord = coord; @@ -136,4 +146,14 @@ } return sum * coord; } + + @Override + public PositionIntervalIterator positions() throws IOException { + if (scorersOrdered == null) + throw new IllegalStateException("no positons requested for this scorer"); + // only created if needed for this scorer - no penalty for non-positional queries + return new ConjunctionPositionIterator(this, scorersOrdered); + } + + } Index: lucene/src/java/org/apache/lucene/search/PositionTermScorer.java =================================================================== --- lucene/src/java/org/apache/lucene/search/PositionTermScorer.java (revision 0) +++ lucene/src/java/org/apache/lucene/search/PositionTermScorer.java (revision 0) @@ -0,0 +1,228 @@ +package org.apache.lucene.search; + +/** + * 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. + */ + +import java.io.IOException; + +import org.apache.lucene.index.DocsAndPositionsEnum; +import org.apache.lucene.search.positions.PositionIntervalIterator; +import org.apache.lucene.search.positions.PositionIntervalIterator.PositionInterval; +import org.apache.lucene.util.BytesRef; + +/** + * Expert: A Scorer for documents matching a Term. + */ +final class PositionTermScorer extends Scorer { + private final DocsAndPositionsEnum docsEnum; + private byte[] norms; + private float weightValue; + private int doc = -1; + private int freq; + private static final int SCORE_CACHE_SIZE = 32; + private float[] scoreCache = new float[SCORE_CACHE_SIZE]; + private final Similarity similarity; + private TermPositions positions; + + /** + * Construct a TermScorer. + * + * @param weight + * The weight of the Term in the query. + * @param td + * An iterator over the documents matching the Term. + * @param similarity + * The Similarity implementation to be used for score + * computations. + * @param norms + * The field norms of the document fields for the Term. + */ + PositionTermScorer(Weight weight, DocsAndPositionsEnum td, + Similarity similarity, byte[] norms, boolean doPayloads) { + super(weight); + this.positions = new TermPositions(td, doPayloads); + this.docsEnum = td; + this.norms = norms; + this.weightValue = weight.getValue(); + this.similarity = similarity; + + for (int i = 0; i < SCORE_CACHE_SIZE; i++) + scoreCache[i] = similarity.tf(i) * weightValue; + } + + @Override + public void score(Collector c) throws IOException { + score(c, Integer.MAX_VALUE, nextDoc()); + } + + // firstDocID is ignored since nextDoc() sets 'doc' + @Override + public boolean score(Collector c, int end, int firstDocID) throws IOException { + c.setScorer(this); + while (doc < end) { // for docs in window + c.collect(doc); // collect score + doc = docsEnum.nextDoc(); + if (doc != NO_MORE_DOCS) { + freq = docsEnum.freq(); + } + } + return true; + } + + @Override + public int docID() { + return doc; + } + + @Override + public float freq() { + return freq; + } + + /** + * Advances to the next document matching the query.
+ * The iterator over the matching documents is buffered using + * {@link TermDocs#read(int[],int[])}. + * + * @return the document matching the query or NO_MORE_DOCS if there are no + * more documents. + */ + @Override + public int nextDoc() throws IOException { + + doc = docsEnum.nextDoc(); + if (doc != NO_MORE_DOCS) { + positions.positionsPending = freq = docsEnum.freq(); + } + return doc; + } + + @Override + public float score() { + assert doc != -1; + float raw = // compute tf(f)*weight + freq < SCORE_CACHE_SIZE // check cache + ? scoreCache[freq] // cache hit + : similarity.tf(freq) * weightValue; // cache miss + + return norms == null ? raw : raw * similarity.decodeNormValue(norms[doc]); // normalize + // for + // field + } + + /** + * Advances to the first match beyond the current whose document number is + * greater than or equal to a given target.
+ * The implementation uses {@link TermDocs#skipTo(int)}. + * + * @param target + * The target document number. + * @return the matching document or NO_MORE_DOCS if none exist. + */ + @Override + public int advance(int target) throws IOException { + doc = docsEnum.advance(target); + if (doc != NO_MORE_DOCS) { + positions.positionsPending = freq = docsEnum.freq(); + } + return doc; + } + + /** Returns a string representation of this TermScorer. */ + @Override + public String toString() { + return "scorer(" + weight + ")"; + } + + @Override + public PositionIntervalIterator positions() throws IOException { + return positions; + } + + private final class TermPositions extends PositionIntervalIterator { + private final PositionInterval interval; + int positionsPending; + private final DocsAndPositionsEnum docsAndPos; + + public TermPositions(DocsAndPositionsEnum docsAndPos, boolean doPayloads) { + super(PositionTermScorer.this); + this.docsAndPos = docsAndPos; + this.interval = doPayloads ? new PayloadPosInterval(docsAndPos, this) + : new PositionInterval(); + + } + + @Override + public PositionInterval next() throws IOException { + if (--positionsPending >= 0) { + interval.begin = interval.end = docsAndPos.nextPosition(); + return interval; + } + interval.reset(); + positionsPending = 0; + return null; + } + + @Override + public int docID() { + return doc; + } + + @Override + public PositionIntervalIterator[] subs(boolean inOrder) { + return EMPTY; + } + } + + private static final class PayloadPosInterval extends PositionInterval { + private int pos = -1; + private final DocsAndPositionsEnum payloads; + private final TermPositions termPos; + + public PayloadPosInterval(DocsAndPositionsEnum payloads, TermPositions pos) { + this.payloads = payloads; + this.termPos = pos; + } + + @Override + public boolean payloadAvailable() { + return payloads.hasPayload(); + } + + @Override + public boolean nextPayload(BytesRef ref) throws IOException { + if (pos == termPos.positionsPending) { + return false; + } else { + pos = termPos.positionsPending; + final BytesRef payload = payloads.getPayload(); + ref.bytes = payload.bytes; + ref.length = payload.length; + ref.offset = payload.offset; + return true; + } + } + + @Override + public void reset() { + super.reset(); + pos = -1; + } + + } + +} Property changes on: lucene/src/java/org/apache/lucene/search/PositionTermScorer.java ___________________________________________________________________ Added: svn:eol-style + native Added: svn:keywords + Date Author Id Revision HeadURL Index: lucene/src/java/org/apache/lucene/search/Scorer.java =================================================================== --- lucene/src/java/org/apache/lucene/search/Scorer.java (revision 1135110) +++ lucene/src/java/org/apache/lucene/search/Scorer.java (working copy) @@ -20,6 +20,7 @@ import java.io.IOException; import org.apache.lucene.search.BooleanClause.Occur; +import org.apache.lucene.search.positions.PositionIntervalIterator; /** * Expert: Common scoring functionality for different types of queries. @@ -101,6 +102,11 @@ public float freq() throws IOException { throw new UnsupportedOperationException(this + " does not implement freq()"); } + + public PositionIntervalIterator positions() throws IOException { + // TODO make abstract? + throw new UnsupportedOperationException(); + } /** * A callback to gather information from a scorer and its sub-scorers. Each Index: lucene/src/java/org/apache/lucene/search/TermQuery.java =================================================================== --- lucene/src/java/org/apache/lucene/search/TermQuery.java (revision 1135110) +++ lucene/src/java/org/apache/lucene/search/TermQuery.java (working copy) @@ -20,6 +20,7 @@ import java.io.IOException; import java.util.Set; +import org.apache.lucene.index.DocsAndPositionsEnum; import org.apache.lucene.index.DocsEnum; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.TermState; @@ -96,9 +97,19 @@ assert termNotInReader(reader, field, term.bytes()) : "no termstate found but term exists in reader"; return null; } - final DocsEnum docs = reader.termDocsEnum(reader.getDeletedDocs(), field, term.bytes(), state); - assert docs != null; - return new TermScorer(this, docs, similarity, context.reader.norms(field)); + if (scorerContext.needsPositions) { + final DocsAndPositionsEnum docs = reader.termPositionsEnum(reader + .getDeletedDocs(), field, term.bytes(), state); + assert docs != null; + return new PositionTermScorer(this, docs, similarity, context.reader + .norms(field), scorerContext.needsPayloads); + } else { + final DocsEnum docs = reader.termDocsEnum(reader.getDeletedDocs(), + field, term.bytes(), state); + assert docs != null; + return new TermScorer(this, docs, similarity, context.reader + .norms(field)); + } } private boolean termNotInReader(IndexReader reader, String field, BytesRef bytes) throws IOException { Index: lucene/src/java/org/apache/lucene/search/Weight.java =================================================================== --- lucene/src/java/org/apache/lucene/search/Weight.java (revision 1135110) +++ lucene/src/java/org/apache/lucene/search/Weight.java (working copy) @@ -141,23 +141,39 @@ */ public final boolean topScorer; + /** + * if true Scorers / Queries supporting positional information will load + * positions. + */ + public final boolean needsPositions; - private static final ScorerContext DEFAULT_CONTEXT = new ScorerContext(true, false); + /** + * if true Scorers / Queries supporting payload information will load + * payloads. + */ + public final boolean needsPayloads; + + + private static final ScorerContext DEFAULT_CONTEXT = new ScorerContext(true, false, false, false); /** * Returns a default {@link ScorerContext} template initialized with: *
    *
  • {@link #scoreDocsInOrder} = true
  • *
  • {@link #topScorer} = false
  • + *
  • {@link #needsPositions} = false
  • + *
  • {@link #needsPayloads} = false
  • *
*/ public static ScorerContext def() { return DEFAULT_CONTEXT; } - private ScorerContext(boolean scoreDocsInOrder, boolean topScorer) { + private ScorerContext(boolean scoreDocsInOrder, boolean topScorer, boolean needsPostions, boolean needsPayloads) { this.scoreDocsInOrder = scoreDocsInOrder; this.topScorer = topScorer; + this.needsPositions = needsPostions; + this.needsPayloads = needsPayloads; } /** @@ -171,7 +187,7 @@ if (this.scoreDocsInOrder == scoreDocsInOrder) { return this; } - return new ScorerContext(scoreDocsInOrder, topScorer); + return new ScorerContext(scoreDocsInOrder, topScorer, needsPositions, needsPayloads); } /** @@ -185,7 +201,36 @@ if (this.topScorer == topScorer) { return this; } - return new ScorerContext(scoreDocsInOrder, topScorer); + return new ScorerContext(scoreDocsInOrder, topScorer, needsPositions, needsPayloads); + } + + + /** + * Creates and returns a copy of this context with the given value for + * {@link #needsPositions} and returns a new instance of + * {@link ScorerContext} iff the given value differs from the + * {@link #needsPositions}. Otherwise, this method has no effect and + * returns this instance. + */ + public ScorerContext needsPositions(boolean needsPositions) { + if (this.needsPositions == needsPositions) { + return this; + } + return new ScorerContext(scoreDocsInOrder, topScorer, needsPositions, needsPayloads); + } + + /** + * Creates and returns a copy of this context with the given value for + * {@link #needsPayloads} and returns a new instance of + * {@link ScorerContext} iff the given value differs from the + * {@link #needsPayloads}. Otherwise, this method has no effect and + * returns this instance. + */ + public ScorerContext needsPayloads(boolean needsPayloads) { + if (this.needsPayloads == needsPayloads) { + return this; + } + return new ScorerContext(scoreDocsInOrder, topScorer, needsPositions, needsPayloads); } } } Index: lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java =================================================================== --- lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java (revision 1135110) +++ lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java (working copy) @@ -19,17 +19,17 @@ import org.apache.lucene.index.IndexReader.AtomicReaderContext; import org.apache.lucene.index.Term; -import org.apache.lucene.index.DocsAndPositionsEnum; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Scorer; import org.apache.lucene.search.Weight; import org.apache.lucene.search.Similarity; import org.apache.lucene.search.Explanation; import org.apache.lucene.search.ComplexExplanation; -import org.apache.lucene.search.spans.TermSpans; +import org.apache.lucene.search.positions.PositionIntervalIterator.PositionInterval; import org.apache.lucene.search.spans.SpanTermQuery; import org.apache.lucene.search.spans.SpanWeight; import org.apache.lucene.search.spans.SpanScorer; +import org.apache.lucene.search.spans.SpansScorerWrapper; import org.apache.lucene.util.BytesRef; import java.io.IOException; @@ -75,17 +75,17 @@ @Override public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException { - return new PayloadTermSpanScorer((TermSpans) query.getSpans(context), + return new PayloadTermSpanScorer((SpansScorerWrapper) query.getSpans(context), this, similarity, context.reader.norms(query.getField())); } protected class PayloadTermSpanScorer extends SpanScorer { - protected BytesRef payload; + protected final BytesRef payload = new BytesRef(); protected float payloadScore; protected int payloadsSeen; - private final TermSpans termSpans; + private final SpansScorerWrapper termSpans; - public PayloadTermSpanScorer(TermSpans spans, Weight weight, + public PayloadTermSpanScorer(SpansScorerWrapper spans, Weight weight, Similarity similarity, byte[] norms) throws IOException { super(spans, weight, similarity, norms); termSpans = spans; @@ -113,10 +113,9 @@ } protected void processPayload(Similarity similarity) throws IOException { - final DocsAndPositionsEnum postings = termSpans.getPostings(); - if (postings.hasPayload()) { - payload = postings.getPayload(); - if (payload != null) { + if (termSpans.isPayloadAvailable()) { + final PositionInterval current = termSpans.current(); + if (current.nextPayload(payload) && payload.length != 0) { payloadScore = function.currentScore(doc, term.field(), spans.start(), spans.end(), payloadsSeen, payloadScore, similarity.scorePayload(doc, spans.start(), Index: lucene/src/java/org/apache/lucene/search/positions/BlockPositionIterator.java =================================================================== --- lucene/src/java/org/apache/lucene/search/positions/BlockPositionIterator.java (revision 0) +++ lucene/src/java/org/apache/lucene/search/positions/BlockPositionIterator.java (revision 0) @@ -0,0 +1,103 @@ +package org.apache.lucene.search.positions; + +/** + * 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. + */ +import java.io.IOException; +import java.util.Arrays; + +import org.apache.lucene.search.Scorer; + +public class BlockPositionIterator extends PositionIntervalIterator { + private final PositionIntervalIterator[] iterators; + private static final PositionInterval INFINITE_INTERVAL = new PositionInterval( + Integer.MIN_VALUE, Integer.MIN_VALUE); + private final PositionInterval[] intervals; + private int docID = -1; + private final PositionInterval interval = new PositionInterval( + Integer.MIN_VALUE, Integer.MIN_VALUE); + + private final int lastIter; + + public BlockPositionIterator(PositionIntervalIterator other) { + super(other.getScorer()); + assert other.subs(true) != null; + iterators = other.subs(true); + assert iterators.length > 1; + intervals = new PositionInterval[iterators.length]; + lastIter = iterators.length-1; + } + + public BlockPositionIterator(Scorer scorer, Scorer... subScorers) + throws IOException { + super(scorer); + assert subScorers.length > 1; + iterators = new PositionIntervalIterator[subScorers.length]; + intervals = new PositionInterval[subScorers.length]; + for (int i = 0; i < subScorers.length; i++) { + iterators[i] = subScorers[i].positions(); + assert iterators[i] != null; + } + lastIter = iterators.length-1; + } + + @Override + public PositionInterval next() throws IOException { + int currentDocID = scorer.docID(); + if (currentDocID != docID) { + Arrays.fill(intervals, INFINITE_INTERVAL); + docID = currentDocID; + } + if (advance()) { + interval.begin = intervals[0].begin; + interval.end = intervals[lastIter].end; + return interval; + } + return null; + } + + private boolean advance() throws IOException { + intervals[0] = iterators[0].next(); + if (intervals[0] == null) { + return false; + } + int i = 1; + final int len = iterators.length; + while (i < len ) { + while (intervals[i].begin <= intervals[i-1].end) { + intervals[i] = iterators[i].next(); + if (intervals[i] == null) { + return false; + } + } + if (intervals[i].begin == intervals[i-1].end+1) { + i++; + } else { + intervals[0] = iterators[0].next(); + if (intervals[0] == null) { + return false; + } + i = 1; + } + } + return true; + } + + @Override + public PositionIntervalIterator[] subs(boolean inOrder) { + return iterators; + } +} Property changes on: lucene/src/java/org/apache/lucene/search/positions/BlockPositionIterator.java ___________________________________________________________________ Added: svn:eol-style + native Added: svn:keywords + Date Author Id Revision HeadURL Index: lucene/src/java/org/apache/lucene/search/positions/ConjunctionPositionIterator.java =================================================================== --- lucene/src/java/org/apache/lucene/search/positions/ConjunctionPositionIterator.java (revision 0) +++ lucene/src/java/org/apache/lucene/search/positions/ConjunctionPositionIterator.java (revision 0) @@ -0,0 +1,94 @@ +package org.apache.lucene.search.positions; +/** + * 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. + */ +import java.io.IOException; + +import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.positions.IntervalQueue.IntervalRef; + +/** + * ConjuctionPositionIterator based on minimal interval semantics for AND + * operator + * + * "Efficient Optimally Lazy Algorithms for Minimal-Interval Semantic + * + */ +public final class ConjunctionPositionIterator extends PositionIntervalIterator { + private int docId = -1; + private final PositionIntervalIterator[] iterators; + private final IntervalQueue queue; + + public ConjunctionPositionIterator(Scorer scorer, Scorer[] subScorers) throws IOException { + super(scorer); + iterators = new PositionIntervalIterator[subScorers.length]; + queue = new IntervalQueue(subScorers.length); + for (int i = 0; i < subScorers.length; i++) { + iterators[i] = subScorers[i].positions(); + } + } + + private void advance() throws IOException { + final IntervalRef top = queue.top(); + PositionInterval interval = null; + if ((interval = iterators[top.index].next()) != null) { + top.interval = interval; + queue.updateRightExtreme(interval); + queue.updateTop(); + } else { + queue.pop(); + } + } + + @Override + public PositionInterval next() throws IOException { + if (docId != scorer.docID()) { + docId = scorer.docID(); + queue.reset(); + for (int i = 0; i < iterators.length; i++) { + final PositionInterval interval = iterators[i].next(); + if (interval == null) { + return null; + } + queue.updateRightExtreme(interval); + queue.add(new IntervalRef(interval, i)); + } + } + if (queue.size() != iterators.length) { + return null; + } + while (queue.topContainsQueueInterval()) { + advance(); + if (queue.size() != iterators.length) + return null; + } + do { + queue.updateQueueInterval(); + advance(); + if (queue.size() != iterators.length) + break; + } while (queue.topContainsQueueInterval()); + return queue.queueInterval; // TODO support payloads + } + + @Override + public PositionIntervalIterator[] subs(boolean inOrder) { + return iterators; + } + +} \ No newline at end of file Property changes on: lucene/src/java/org/apache/lucene/search/positions/ConjunctionPositionIterator.java ___________________________________________________________________ Added: svn:eol-style + native Added: svn:keywords + Date Author Id Revision HeadURL Index: lucene/src/java/org/apache/lucene/search/positions/IntervalQueue.java =================================================================== --- lucene/src/java/org/apache/lucene/search/positions/IntervalQueue.java (revision 0) +++ lucene/src/java/org/apache/lucene/search/positions/IntervalQueue.java (revision 0) @@ -0,0 +1,56 @@ +package org.apache.lucene.search.positions; + +import org.apache.lucene.search.positions.IntervalQueue.IntervalRef; +import org.apache.lucene.search.positions.PositionIntervalIterator.PositionInterval; +import org.apache.lucene.util.PriorityQueue; + +final class IntervalQueue extends PriorityQueue { + int rightExtreme = Integer.MIN_VALUE; + final PositionInterval queueInterval = new PositionInterval( + Integer.MIN_VALUE, Integer.MAX_VALUE); + + public void reset() { + clear(); + rightExtreme = Integer.MIN_VALUE; + queueInterval.begin = Integer.MIN_VALUE; + queueInterval.end = Integer.MIN_VALUE; + } + + public void updateRightExtreme(PositionInterval interval) { + rightExtreme = Math.max(rightExtreme, Math.max(interval.end, interval.end)); + } + + public boolean topContainsQueueInterval() { + PositionInterval interval = top().interval; + return interval.begin <= queueInterval.begin + && queueInterval.end <= rightExtreme; + } + + public void updateQueueInterval() { + PositionInterval interval = top().interval; + queueInterval.begin = interval.begin; + queueInterval.end = rightExtreme; + } + + public IntervalQueue(int size) { + super(size); + } + + @Override + protected boolean lessThan(IntervalRef left, IntervalRef right) { + final PositionInterval a = left.interval; + final PositionInterval b = right.interval; + return a.begin < b.begin || (a.begin == b.begin && a.end >= b.end); + } + + final static class IntervalRef { + PositionInterval interval; + int index; + + IntervalRef(PositionInterval interval, int index) { + super(); + this.interval = interval; + this.index = index; + } + } +} \ No newline at end of file Property changes on: lucene/src/java/org/apache/lucene/search/positions/IntervalQueue.java ___________________________________________________________________ Added: svn:eol-style + native Added: svn:keywords + Date Author Id Revision HeadURL Index: lucene/src/java/org/apache/lucene/search/positions/OrderedConjunctionPositionIterator.java =================================================================== --- lucene/src/java/org/apache/lucene/search/positions/OrderedConjunctionPositionIterator.java (revision 0) +++ lucene/src/java/org/apache/lucene/search/positions/OrderedConjunctionPositionIterator.java (revision 0) @@ -0,0 +1,98 @@ +package org.apache.lucene.search.positions; + +/** + * 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. + */ +import java.io.IOException; +import java.util.Arrays; + +public final class OrderedConjunctionPositionIterator extends + PositionIntervalIterator { + + private int docId = -1; + private final PositionIntervalIterator[] iterators; + private static final PositionInterval INFINITE_INTERVAL = new PositionInterval( + Integer.MIN_VALUE, Integer.MIN_VALUE); + private final PositionInterval[] intervals; + private final int lastIter; + private final PositionInterval interval = new PositionInterval( + Integer.MAX_VALUE, Integer.MAX_VALUE); + private int index = 1; + + public OrderedConjunctionPositionIterator(PositionIntervalIterator other) { + super(other.scorer); + assert other.subs(true) != null; + iterators = other.subs(true); + assert iterators.length > 1; + intervals = new PositionInterval[iterators.length]; + lastIter = iterators.length - 1; + } + + @Override + public PositionInterval next() throws IOException { + final int currentDocId = scorer.docID(); + if (docId != currentDocId) { + docId = currentDocId; + // TODO maybe use null instead? + Arrays.fill(intervals, 1, intervals.length, INFINITE_INTERVAL); + intervals[0] = iterators[0].next(); + index = 1; + } + + if(intervals[0] == null) { + return null; + } + + interval.begin = Integer.MAX_VALUE; + interval.end = Integer.MAX_VALUE; + int b = Integer.MAX_VALUE; + while (true) { + while (true) { + final PositionInterval previous = intervals[index - 1]; + if (previous.end >= b) { + return interval.begin == Integer.MAX_VALUE ? null : interval; + } + if (index == intervals.length || intervals[index].begin > previous.end) { + break; + } + PositionInterval current = intervals[index]; + do { + final PositionInterval next; + if (current.end >= b || (next = iterators[index].next()) == null) { + return interval.begin == Integer.MAX_VALUE ? null : interval; + } + current = intervals[index] = next; + } while (current.begin <= previous.end); + index++; + } + interval.begin = intervals[0].begin; + interval.end = intervals[lastIter].end; + b = intervals[lastIter].begin; + index = 1; + intervals[0] = iterators[0].next(); + if (intervals[0] == null) { + return interval.begin == Integer.MAX_VALUE ? null : interval; + } + } + + } + + @Override + public PositionIntervalIterator[] subs(boolean inOrder) { + return iterators; + } + +} Property changes on: lucene/src/java/org/apache/lucene/search/positions/OrderedConjunctionPositionIterator.java ___________________________________________________________________ Added: svn:eol-style + native Added: svn:keywords + Date Author Id Revision HeadURL Index: lucene/src/java/org/apache/lucene/search/positions/PositionFilterQuery.java =================================================================== --- lucene/src/java/org/apache/lucene/search/positions/PositionFilterQuery.java (revision 0) +++ lucene/src/java/org/apache/lucene/search/positions/PositionFilterQuery.java (revision 0) @@ -0,0 +1,176 @@ +package org.apache.lucene.search.positions; + +/** + * 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. + */ + +import java.io.IOException; +import java.util.Set; + +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexReader.AtomicReaderContext; +import org.apache.lucene.index.Term; +import org.apache.lucene.search.Explanation; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.Weight; +import org.apache.lucene.search.Weight.ScorerContext; +import org.apache.lucene.search.positions.PositionIntervalIterator.PositionIntervalFilter; + +/** + * + * + **/ +public class PositionFilterQuery extends Query implements Cloneable { + + private Query inner; + private PositionIntervalFilter filter; + + public PositionFilterQuery(Query inner, PositionIntervalFilter filter) { + this.inner = inner; + this.filter = filter; + } + + @Override + public void extractTerms(Set terms) { + inner.extractTerms(terms); + } + + @Override + public Query rewrite(IndexReader reader) throws IOException { + PositionFilterQuery clone = null; + + Query rewritten = (Query) inner.rewrite(reader); + if (rewritten != inner) { + clone = (PositionFilterQuery) this.clone(); + clone.inner = rewritten; + } + + if (clone != null) { + return clone; // some clauses rewrote + } else { + return this; // no clauses rewrote + } + } + + @Override + public Weight createWeight(IndexSearcher searcher) throws IOException { + return new PositionFilterWeight(inner.createWeight(searcher)); + } + + class PositionFilterWeight extends Weight { + + private final Weight other; + + public PositionFilterWeight(Weight other) { + this.other = other; + } + + @Override + public Explanation explain(AtomicReaderContext context, int doc) + throws IOException { + return other.explain(context, doc); + } + + @Override + public Query getQuery() { + return PositionFilterQuery.this; + } + + @Override + public float getValue() { + return other.getValue(); + } + + @Override + public void normalize(float norm) { + other.normalize(norm); + } + + @Override + public Scorer scorer(AtomicReaderContext context, + ScorerContext scorerContext) throws IOException { + Scorer scorer = other.scorer(context, + scorerContext.needsPositions(true)); + return scorer == null ? null : new PositionFilterScorer(this, scorer); + } + + @Override + public float sumOfSquaredWeights() throws IOException { + return other.sumOfSquaredWeights(); + } + + } + + class PositionFilterScorer extends Scorer { + + private final Scorer other; + private PositionIntervalIterator filter; + + public PositionFilterScorer(Weight weight, Scorer other) throws IOException { + super(weight); + this.other = other; + this.filter = PositionFilterQuery.this.filter != null ? PositionFilterQuery.this.filter.filter(other.positions()) + : other.positions(); + } + + @Override + public float score() throws IOException { + return other.score(); + } + + @Override + public PositionIntervalIterator positions() throws IOException { + return filter; + } + + @Override + public int docID() { + return other.docID(); + } + + @Override + public int nextDoc() throws IOException { + while (other.nextDoc() != Scorer.NO_MORE_DOCS) { + if (filter.next() != null) { // just check if there is a position that matches! + return other.docID(); + } + } + return Scorer.NO_MORE_DOCS; + } + + @Override + public int advance(int target) throws IOException { + int advance = other.advance(target); + if (advance == Scorer.NO_MORE_DOCS) + return NO_MORE_DOCS; + do { + if (filter.next() != null) { + return other.docID(); + } + } while (other.nextDoc() != Scorer.NO_MORE_DOCS); + return NO_MORE_DOCS; + } + + } + + @Override + public String toString(String field) { + return inner.toString(); + } + +} \ No newline at end of file Property changes on: lucene/src/java/org/apache/lucene/search/positions/PositionFilterQuery.java ___________________________________________________________________ Added: svn:eol-style + native Added: svn:keywords + Date Author Id Revision HeadURL Index: lucene/src/java/org/apache/lucene/search/positions/PositionIntervalIterator.java =================================================================== --- lucene/src/java/org/apache/lucene/search/positions/PositionIntervalIterator.java (revision 0) +++ lucene/src/java/org/apache/lucene/search/positions/PositionIntervalIterator.java (revision 0) @@ -0,0 +1,97 @@ +package org.apache.lucene.search.positions; + +/** + * 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. + */ +import java.io.IOException; +import java.io.Serializable; + +import org.apache.lucene.search.Scorer; +import org.apache.lucene.util.BytesRef; + +/** + * + * TODO add documentation + */ +public abstract class PositionIntervalIterator implements Serializable{ + + public static final PositionIntervalIterator[] EMPTY = new PositionIntervalIterator[0]; + protected final Scorer scorer; + + public PositionIntervalIterator(Scorer scorer) { + this.scorer = scorer; + } + + public abstract PositionInterval next() throws IOException; + + public abstract PositionIntervalIterator[] subs(boolean inOrder); + + public int docID() { + return scorer.docID(); + } + + public Scorer getScorer() { + return scorer; + } + + public static interface PositionIntervalFilter extends Serializable { + public abstract PositionIntervalIterator filter( + PositionIntervalIterator iter); + } + + public static class PositionInterval implements Cloneable { + + + public int begin; + public int end; + + public PositionInterval(int begin, int end) { + this.begin = begin; + this.end = end; + } + + public PositionInterval() { + this(0, 0); + } + + public boolean nextPayload(BytesRef ref) throws IOException { + return false; + } + + public boolean payloadAvailable() { + return false; + } + + public void reset() { + begin = end = 0; + } + + @Override + public Object clone() { + try { + return super.clone(); + } catch (CloneNotSupportedException e) { + throw new RuntimeException(); // should not happen + } + } + + @Override + public String toString() { + return "PositionInterval [begin=" + begin + ", end=" + end + "]"; + } + + } +} Property changes on: lucene/src/java/org/apache/lucene/search/positions/PositionIntervalIterator.java ___________________________________________________________________ Added: svn:eol-style + native Added: svn:keywords + Date Author Id Revision HeadURL Index: lucene/src/java/org/apache/lucene/search/positions/RangePositionsIterator.java =================================================================== --- lucene/src/java/org/apache/lucene/search/positions/RangePositionsIterator.java (revision 0) +++ lucene/src/java/org/apache/lucene/search/positions/RangePositionsIterator.java (revision 0) @@ -0,0 +1,66 @@ +package org.apache.lucene.search.positions; +/** + * 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. + */ + + +import java.io.IOException; + +import org.apache.lucene.search.positions.PositionIntervalIterator.PositionIntervalFilter; + + +public class RangePositionsIterator extends PositionIntervalIterator implements PositionIntervalFilter { + + private final PositionIntervalIterator iterator; + private int start; + private int end; + + public RangePositionsIterator(int start, int end) { + this(start, end, null); + } + + public RangePositionsIterator(int start, int end, PositionIntervalIterator iterator) { + super(iterator != null ? iterator.scorer : null); + this.iterator = iterator; + this.start = start; + this.end = end; + } + + public PositionIntervalIterator filter(PositionIntervalIterator iter) { + return new RangePositionsIterator(start, end, iter); + } + + @Override + public PositionInterval next() throws IOException { + PositionInterval interval = null; + while ((interval = iterator.next()) != null) { + if(interval.end > end) { + return null; + } else if (interval.begin >= start) { + return interval; + } + } + return null; + } + + @Override + public PositionIntervalIterator[] subs(boolean inOrder) { + return new PositionIntervalIterator[] { iterator }; + } + + + +} \ No newline at end of file Property changes on: lucene/src/java/org/apache/lucene/search/positions/RangePositionsIterator.java ___________________________________________________________________ Added: svn:eol-style + native Added: svn:keywords + Date Author Id Revision HeadURL Index: lucene/src/java/org/apache/lucene/search/positions/WithinPositionIterator.java =================================================================== --- lucene/src/java/org/apache/lucene/search/positions/WithinPositionIterator.java (revision 0) +++ lucene/src/java/org/apache/lucene/search/positions/WithinPositionIterator.java (revision 0) @@ -0,0 +1,41 @@ +package org.apache.lucene.search.positions; + +import java.io.IOException; + +import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.positions.PositionIntervalIterator.PositionIntervalFilter; + + +public class WithinPositionIterator extends PositionIntervalIterator implements PositionIntervalFilter { + private int howMany; + private PositionIntervalIterator iterator; + public WithinPositionIterator(int howMany, PositionIntervalIterator iterator) { + super(iterator != null ? iterator.scorer : null); + this.howMany = howMany; + this.iterator = iterator; + } + + public WithinPositionIterator(int howMany) { + this(howMany, null); // use this instance as a filter template + } + @Override + public PositionInterval next() throws IOException { + PositionInterval interval = null; + while ((interval = iterator.next()) != null) { + if((interval.end - interval.begin) <= howMany){ + return interval; + } + } + return null; + } + + @Override + public PositionIntervalIterator[] subs(boolean inOrder) { + return new PositionIntervalIterator[] {iterator}; + } + + public PositionIntervalIterator filter(PositionIntervalIterator iter) { + return new WithinPositionIterator(howMany, iter); + } + +} Property changes on: lucene/src/java/org/apache/lucene/search/positions/WithinPositionIterator.java ___________________________________________________________________ Added: svn:eol-style + native Added: svn:keywords + Date Author Id Revision HeadURL Index: lucene/src/java/org/apache/lucene/search/spans/MockSpanQuery.java =================================================================== --- lucene/src/java/org/apache/lucene/search/spans/MockSpanQuery.java (revision 0) +++ lucene/src/java/org/apache/lucene/search/spans/MockSpanQuery.java (revision 0) @@ -0,0 +1,148 @@ +package org.apache.lucene.search.spans; + +import java.io.IOException; +import java.util.Collection; +import java.util.Set; + +import org.apache.lucene.index.IndexReader.AtomicReaderContext; +import org.apache.lucene.index.IndexReader.ReaderContext; +import org.apache.lucene.index.Term; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.Weight; +import org.apache.lucene.search.Weight.ScorerContext; +import org.apache.lucene.search.positions.PositionIntervalIterator.PositionIntervalFilter; + +public class MockSpanQuery extends SpanQuery { + + private Query other; + private boolean needsPayloads; + private String field; + private PositionIntervalFilter filter; + + public MockSpanQuery(Query other, boolean needsPayloads, String field, PositionIntervalFilter filter) { + this.other = other; + this.needsPayloads = needsPayloads; + this.field = field; + this.filter = filter; + } + + public MockSpanQuery(SpanQuery other, boolean needsPayloads) { + this(other, needsPayloads, other.getField(), null); + } + + @Override + public Spans getSpans(AtomicReaderContext context) throws IOException { + + if(other instanceof SpanQuery) { + return ((SpanQuery) other).getSpans(context); + } + + ReaderContext topReaderContext = context.reader.getTopReaderContext(); + + Weight weight = other.weight(new IndexSearcher(topReaderContext)); + Scorer scorer = weight.scorer((AtomicReaderContext) topReaderContext, ScorerContext.def().needsPositions(true).needsPayloads(needsPayloads)); + if (scorer == null) { + return EMPTY_SPANS; + } + return new SpansScorerWrapper(scorer, filter == null ? scorer.positions() : filter.filter(scorer.positions())); + } + + + @Override + public void extractTerms(Set terms) { + other.extractTerms(terms); + } + + @Override + public String getField() { + return field; + } + + @Override + public String toString(String field) { + return other.toString(); + } + + @Override + public int hashCode() { + final int prime = 31; + int result = super.hashCode(); + result = prime * result + ((field == null) ? 0 : field.hashCode()); + result = prime * result + (needsPayloads ? 1231 : 1237); + result = prime * result + ((other == null) ? 0 : other.hashCode()); + return result; + } + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (!super.equals(obj)) + return false; + if (getClass() != obj.getClass()) + return false; + MockSpanQuery other = (MockSpanQuery) obj; + if (field == null) { + if (other.field != null) + return false; + } else if (!field.equals(other.field)) + return false; + if (needsPayloads != other.needsPayloads) + return false; + if (this.other == null) { + if (other.other != null) + return false; + } else if (!this.other.equals(other.other)) + return false; + return true; + } + + + + + private static final class EmptySpans extends SpansScorerWrapper { + + public EmptySpans() { + super(null, null); + } + + @Override + public boolean next() { + return false; + } + + @Override + public boolean skipTo(int target) { + return false; + } + + @Override + public int doc() { + return DocIdSetIterator.NO_MORE_DOCS; + } + + @Override + public int start() { + return -1; + } + + @Override + public int end() { + return -1; + } + + @Override + public Collection getPayload() { + return null; + } + + @Override + public boolean isPayloadAvailable() { + return false; + } + } + + public static final Spans EMPTY_SPANS = new EmptySpans(); +} Property changes on: lucene/src/java/org/apache/lucene/search/spans/MockSpanQuery.java ___________________________________________________________________ Added: svn:eol-style + native Added: svn:keywords + Date Author Id Revision HeadURL Index: lucene/src/java/org/apache/lucene/search/spans/SpanTermQuery.java =================================================================== --- lucene/src/java/org/apache/lucene/search/spans/SpanTermQuery.java (revision 1135110) +++ lucene/src/java/org/apache/lucene/search/spans/SpanTermQuery.java (working copy) @@ -17,86 +17,30 @@ * limitations under the License. */ -import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.IndexReader.AtomicReaderContext; -import org.apache.lucene.index.Term; -import org.apache.lucene.index.DocsAndPositionsEnum; -import org.apache.lucene.util.ToStringUtils; - -import java.io.IOException; import java.util.Set; +import org.apache.lucene.index.Term; +import org.apache.lucene.search.TermQuery; + /** Matches spans containing a term. */ -public class SpanTermQuery extends SpanQuery { +public class SpanTermQuery extends MockSpanQuery { protected Term term; - /** Construct a SpanTermQuery matching the named term's spans. */ - public SpanTermQuery(Term term) { this.term = term; } - + public SpanTermQuery(Term term) { this(term, new TermQuery(term), true); } + public SpanTermQuery(Term term, boolean needsPayloads) { this(term, new TermQuery(term), needsPayloads); } + + private SpanTermQuery(Term term, TermQuery query, boolean needsPayloads) { + super(query, needsPayloads, term.field(), null); + this.term = term; + } + /** Return the term whose spans are matched. */ public Term getTerm() { return term; } - @Override - public String getField() { return term.field(); } @Override public void extractTerms(Set terms) { terms.add(term); } - @Override - public String toString(String field) { - StringBuilder buffer = new StringBuilder(); - if (term.field().equals(field)) - buffer.append(term.text()); - else - buffer.append(term.toString()); - buffer.append(ToStringUtils.boost(getBoost())); - return buffer.toString(); - } - - @Override - public int hashCode() { - final int prime = 31; - int result = super.hashCode(); - result = prime * result + ((term == null) ? 0 : term.hashCode()); - return result; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) - return true; - if (!super.equals(obj)) - return false; - if (getClass() != obj.getClass()) - return false; - SpanTermQuery other = (SpanTermQuery) obj; - if (term == null) { - if (other.term != null) - return false; - } else if (!term.equals(other.term)) - return false; - return true; - } - - @Override - public Spans getSpans(final AtomicReaderContext context) throws IOException { - final IndexReader reader = context.reader; - final DocsAndPositionsEnum postings = reader.termPositionsEnum(reader.getDeletedDocs(), - term.field(), - term.bytes()); - - if (postings != null) { - return new TermSpans(postings, term); - } else { - if (reader.termDocsEnum(reader.getDeletedDocs(), term.field(), term.bytes()) != null) { - // term does exist, but has no positions - throw new IllegalStateException("field \"" + term.field() + "\" was indexed with Field.omitTermFreqAndPositions=true; cannot run SpanTermQuery (term=" + term.text() + ")"); - } else { - // term does not exist - return TermSpans.EMPTY_TERM_SPANS; - } - } - } } Index: lucene/src/java/org/apache/lucene/search/spans/SpansScorerWrapper.java =================================================================== --- lucene/src/java/org/apache/lucene/search/spans/SpansScorerWrapper.java (revision 0) +++ lucene/src/java/org/apache/lucene/search/spans/SpansScorerWrapper.java (revision 0) @@ -0,0 +1,108 @@ +package org.apache.lucene.search.spans; + +/** + * 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. + */ +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; + +import org.apache.lucene.search.positions.PositionIntervalIterator; +import org.apache.lucene.search.positions.PositionIntervalIterator.PositionInterval; +import org.apache.lucene.search.Scorer; +import org.apache.lucene.util.BytesRef; + +public class SpansScorerWrapper extends Spans { + private final Scorer scorer; + private final PositionIntervalIterator positions; + private PositionInterval current; + private int doc = -1; + + public SpansScorerWrapper(Scorer scorer, PositionIntervalIterator positions) { + this.scorer = scorer; + this.positions = positions; + } + + @Override + public boolean next() throws IOException { + if (doc == -1) { + doc = scorer.nextDoc(); + } + + if (doc == Scorer.NO_MORE_DOCS) { + return false; + } + + if ((current = positions.next()) == null) { + doc = scorer.nextDoc(); + if (doc == Scorer.NO_MORE_DOCS) { + return false; + } + return (current = positions.next()) != null; + } + return true; + } + + @Override + public boolean skipTo(int target) throws IOException { + doc = scorer.advance(target); + if (doc == Scorer.NO_MORE_DOCS) { + return false; + } + return (current = positions.next()) != null; + } + + @Override + public int doc() { + return doc; + } + + @Override + public int start() { + return current.begin; + } + + @Override + public int end() { + return current.end + 1; + } + + @Override + public Collection getPayload() throws IOException { + BytesRef ref = new BytesRef(); + final Collection payloads = new ArrayList(); + while(current.nextPayload(ref)) { + if (ref.length > 0) { + byte[] retVal = new byte[ref.length]; + System.arraycopy(ref.bytes, ref.offset, retVal, 0, + ref.length); + payloads.add(retVal); + } + } + + return payloads; + } + + @Override + public boolean isPayloadAvailable() { + return current != null && current.payloadAvailable(); + } + + public PositionInterval current() { + return current; + } + +} \ No newline at end of file Property changes on: lucene/src/java/org/apache/lucene/search/spans/SpansScorerWrapper.java ___________________________________________________________________ Added: svn:eol-style + native Added: svn:keywords + Date Author Id Revision HeadURL Index: lucene/src/java/org/apache/lucene/search/spans/TermSpans.java =================================================================== --- lucene/src/java/org/apache/lucene/search/spans/TermSpans.java (revision 1135110) +++ lucene/src/java/org/apache/lucene/search/spans/TermSpans.java (working copy) @@ -1,169 +0,0 @@ -package org.apache.lucene.search.spans; -/** - * Copyright 2005 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 org.apache.lucene.index.Term; -import org.apache.lucene.index.DocsAndPositionsEnum; -import org.apache.lucene.search.DocIdSetIterator; -import org.apache.lucene.util.BytesRef; - -import java.io.IOException; -import java.util.Collections; -import java.util.Collection; - -/** - * Expert: - * Public for extension only - */ -public class TermSpans extends Spans { - protected final DocsAndPositionsEnum postings; - protected final Term term; - protected int doc; - protected int freq; - protected int count; - protected int position; - - public TermSpans(DocsAndPositionsEnum postings, Term term) throws IOException { - this.postings = postings; - this.term = term; - doc = -1; - } - - // only for EmptyTermSpans (below) - TermSpans() { - term = null; - postings = null; - } - - @Override - public boolean next() throws IOException { - if (count == freq) { - if (postings == null) { - return false; - } - doc = postings.nextDoc(); - if (doc == DocsAndPositionsEnum.NO_MORE_DOCS) { - return false; - } - freq = postings.freq(); - count = 0; - } - position = postings.nextPosition(); - count++; - return true; - } - - @Override - public boolean skipTo(int target) throws IOException { - doc = postings.advance(target); - if (doc == DocsAndPositionsEnum.NO_MORE_DOCS) { - return false; - } - - freq = postings.freq(); - count = 0; - position = postings.nextPosition(); - count++; - - return true; - } - - @Override - public int doc() { - return doc; - } - - @Override - public int start() { - return position; - } - - @Override - public int end() { - return position + 1; - } - - // TODO: Remove warning after API has been finalized - @Override - public Collection getPayload() throws IOException { - final BytesRef payload = postings.getPayload(); - final byte[] bytes; - if (payload != null) { - bytes = new byte[payload.length]; - System.arraycopy(payload.bytes, payload.offset, bytes, 0, payload.length); - } else { - bytes = null; - } - return Collections.singletonList(bytes); - } - - // TODO: Remove warning after API has been finalized - @Override - public boolean isPayloadAvailable() { - return postings.hasPayload(); - } - - @Override - public String toString() { - return "spans(" + term.toString() + ")@" + - (doc == -1 ? "START" : (doc == Integer.MAX_VALUE) ? "END" : doc + "-" + position); - } - - public DocsAndPositionsEnum getPostings() { - return postings; - } - - private static final class EmptyTermSpans extends TermSpans { - - @Override - public boolean next() { - return false; - } - - @Override - public boolean skipTo(int target) { - return false; - } - - @Override - public int doc() { - return DocIdSetIterator.NO_MORE_DOCS; - } - - @Override - public int start() { - return -1; - } - - @Override - public int end() { - return -1; - } - - @Override - public Collection getPayload() { - return null; - } - - @Override - public boolean isPayloadAvailable() { - return false; - } - } - - public static final TermSpans EMPTY_TERM_SPANS = new EmptyTermSpans(); -} Index: lucene/src/test/org/apache/lucene/search/TestBooleanQuery.java =================================================================== --- lucene/src/test/org/apache/lucene/search/TestBooleanQuery.java (revision 1135110) +++ lucene/src/test/org/apache/lucene/search/TestBooleanQuery.java (working copy) @@ -17,6 +17,7 @@ * limitations under the License. */ +import java.io.IOException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; @@ -29,6 +30,10 @@ import org.apache.lucene.index.RandomIndexWriter; import org.apache.lucene.index.Term; import org.apache.lucene.queryParser.QueryParser; +import org.apache.lucene.search.BooleanClause.Occur; +import org.apache.lucene.search.positions.PositionFilterQuery; +import org.apache.lucene.search.positions.RangePositionsIterator; +import org.apache.lucene.search.positions.WithinPositionIterator; import org.apache.lucene.store.Directory; import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.NamedThreadFactory; @@ -164,6 +169,59 @@ dir1.close(); dir2.close(); } + + public void testConjunctionPositions() throws IOException { + Directory directory = newDirectory(); + RandomIndexWriter writer = new RandomIndexWriter(random, directory, + newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random))); + { + Document doc = new Document(); + doc.add(newField( + "field", + "Pease porridge hot! Pease porridge cold! Pease porridge in the pot nine days old! Some like it hot, some" + + " like it cold, Some like it in the pot nine days old! Pease porridge hot! Pease porridge cold!", + Field.Store.YES, Field.Index.ANALYZED)); + writer.addDocument(doc); + } + + { + Document doc = new Document(); + doc.add(newField( + "field", + "Pease porridge cold! Pease porridge hot! Pease porridge in the pot nine days old! Some like it cold, some" + + " like it hot, Some like it in the pot nine days old! Pease porridge cold! Pease porridge hot!", + Field.Store.YES, Field.Index.ANALYZED)); + writer.addDocument(doc); + } + + IndexReader reader = writer.getReader(); + IndexSearcher searcher = new IndexSearcher(reader); + writer.close(); + BooleanQuery query = new BooleanQuery(); + query.add(new BooleanClause(new TermQuery(new Term("field", "porridge")), Occur.MUST)); + query.add(new BooleanClause(new TermQuery(new Term("field", "pease")), Occur.MUST)); + query.add(new BooleanClause(new TermQuery(new Term("field", "hot!")), Occur.MUST)); + + { + PositionFilterQuery filter = new PositionFilterQuery(query, new RangePositionsIterator(0,3)); + TopDocs search = searcher.search(filter, 10); + ScoreDoc[] scoreDocs = search.scoreDocs; + assertEquals(1, search.totalHits); + assertEquals(0, scoreDocs[0].doc); + } + { + PositionFilterQuery filter = new PositionFilterQuery(query, new WithinPositionIterator(3)); + TopDocs search = searcher.search(filter, 10); + ScoreDoc[] scoreDocs = search.scoreDocs; + assertEquals(2, search.totalHits); + assertEquals(0, scoreDocs[0].doc); + assertEquals(1, scoreDocs[1].doc); + } + + searcher.close(); + reader.close(); + directory.close(); + } } Index: lucene/src/test/org/apache/lucene/search/TestTermQuery.java =================================================================== --- lucene/src/test/org/apache/lucene/search/TestTermQuery.java (revision 0) +++ lucene/src/test/org/apache/lucene/search/TestTermQuery.java (revision 0) @@ -0,0 +1,299 @@ +package org.apache.lucene.search; + +/** + * 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. + */ +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; + +import org.apache.lucene.analysis.Analyzer; +import org.apache.lucene.analysis.MockAnalyzer; +import org.apache.lucene.analysis.MockPayloadAnalyzer; +import org.apache.lucene.analysis.MockTokenFilter; +import org.apache.lucene.analysis.MockTokenizer; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.RandomIndexWriter; +import org.apache.lucene.index.Term; +import org.apache.lucene.index.IndexReader.AtomicReaderContext; +import org.apache.lucene.index.IndexReader.ReaderContext; +import org.apache.lucene.search.Weight.ScorerContext; +import org.apache.lucene.search.positions.PositionIntervalIterator; +import org.apache.lucene.search.positions.PositionIntervalIterator.PositionInterval; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.LuceneTestCase; +import org.apache.lucene.util.ReaderUtil; + +/** + * This class contains tests related to {@link TermQuery} + */ +public class TestTermQuery extends LuceneTestCase { + + private String fieldName; + private boolean usePayload; + + public void setUp() throws Exception { + super.setUp(); + fieldName = "field" + random.nextInt(); + usePayload = random.nextBoolean(); + } + + /** + * Simple testcase for {@link TermScorer#positions()} + */ + public void testPositionsSimple() throws IOException { + Directory directory = newDirectory(); + + final Analyzer analyzer = new MockAnalyzer(random, MockTokenizer.WHITESPACE, false, MockTokenFilter.EMPTY_STOPSET, true); + RandomIndexWriter writer = new RandomIndexWriter(random, directory, + newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer )); + for (int i = 0; i < 39; i++) { + Document doc = new Document(); + doc.add(newField(fieldName, "1 2 3 4 5 6 7 8 9 10 " + + "1 2 3 4 5 6 7 8 9 10 " + "1 2 3 4 5 6 7 8 9 10 " + + "1 2 3 4 5 6 7 8 9 10", Field.Store.YES, Field.Index.ANALYZED)); + writer.addDocument(doc); + } + IndexReader reader = writer.getReader(); + IndexSearcher searcher = new IndexSearcher(reader); + writer.close(); + + for (int i = 0; i < 39 * RANDOM_MULTIPLIER; i++) { + TermQuery one = new TermQuery(new Term(fieldName, "1")); + ReaderContext topReaderContext = reader.getTopReaderContext(); + AtomicReaderContext[] leaves = ReaderUtil.leaves(topReaderContext); + Weight weight = one.createWeight(searcher); + for (AtomicReaderContext atomicReaderContext : leaves) { + Scorer scorer = weight.scorer(atomicReaderContext, ScorerContext.def() + .needsPositions(true).needsPayloads(usePayload)); + assertNotNull(scorer); + final int advance = scorer.advance(1 + random.nextInt(27)); + PositionIntervalIterator positions = scorer.positions(); + + do { + PositionInterval interval = null; + String msg = "Advanced to: " + advance + " current doc: " + + scorer.docID() + " usePayloads: " + usePayload; + assertNotNull(msg, (interval = positions.next())); + assertEquals(msg, 4.0f, positions.getScorer().freq(), 0.0f); + + assertEquals(msg, 0, interval.begin); + assertEquals(msg, 0, interval.end); + checkPayload(0, interval); + + assertNotNull(msg, (interval = positions.next())); + assertEquals(msg, 4.0f, positions.getScorer().freq(), 0.0f); + assertEquals(msg, 10, interval.begin); + assertEquals(msg, 10, interval.end); + checkPayload(10, interval); + + assertNotNull(msg, (interval = positions.next())); + assertEquals(msg, 4.0f, positions.getScorer().freq(), 0.0f); + assertEquals(msg, 20, interval.begin); + assertEquals(msg, 20, interval.end); + checkPayload(20, interval); + + assertNotNull(msg, (interval = positions.next())); + assertEquals(msg, 4.0f, positions.getScorer().freq(), 0.0f); + assertEquals(msg, 30, interval.begin); + assertEquals(msg, 30, interval.end); + checkPayload(30, interval); + + assertNull(msg, (interval = positions.next())); + + } while (scorer.nextDoc() != Scorer.NO_MORE_DOCS); + } + } + searcher.close(); + reader.close(); + directory.close(); + } + + public final void checkPayload(int pos, PositionInterval interval) throws IOException { + if (usePayload) { + // nocommit enable this with mockanalyzer +// assertTrue(interval.payloadAvailable()); +// BytesRef bytes = new BytesRef(); +// assertTrue(interval.nextPayload(bytes)); +// assertFalse(interval.payloadAvailable()); +// assertEquals("pos: " + pos, bytes.utf8ToString()); + } else { + assertFalse(interval.payloadAvailable()); + } + } + + /** + * this test indexes random numbers within a range into a field and checks + * their occurrences by searching for a number from that range selected at + * random. All positions for that number are safed up front and compared to + * the terms scorers positions. + * + */ + public void testRandomPositons() throws IOException { + Directory dir = newDirectory(); + RandomIndexWriter writer = new RandomIndexWriter(random, dir, + newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random))); + int numDocs = 131; + int max = 1051; + int term = random.nextInt(max); + Integer[][] positionsInDoc = new Integer[numDocs][]; + for (int i = 0; i < numDocs; i++) { + Document doc = new Document(); + ArrayList positions = new ArrayList(); + StringBuilder builder = new StringBuilder(); + for (int j = 0; j < 3049; j++) { + int nextInt = random.nextInt(max); + builder.append(nextInt).append(" "); + if (nextInt == term) { + positions.add(Integer.valueOf(j)); + } + } + doc.add(newField(fieldName, builder.toString(), Field.Store.YES, + Field.Index.ANALYZED)); + positionsInDoc[i] = positions.toArray(new Integer[0]); + writer.addDocument(doc); + } + + IndexReader reader = writer.getReader(); + writer.close(); + IndexSearcher searcher = new IndexSearcher(reader); + + for (int i = 0; i < 39 * RANDOM_MULTIPLIER; i++) { + TermQuery one = new TermQuery(new Term(fieldName, "" + term)); + ReaderContext topReaderContext = reader.getTopReaderContext(); + AtomicReaderContext[] leaves = ReaderUtil.leaves(topReaderContext); + Weight weight = one.createWeight(searcher); + for (AtomicReaderContext atomicReaderContext : leaves) { + Scorer scorer = weight.scorer(atomicReaderContext, ScorerContext.def() + .needsPositions(true)); + assertNotNull(scorer); + int initDoc = 0; + int maxDoc = atomicReaderContext.reader.maxDoc(); + // initially advance or do next doc + if (random.nextBoolean()) { + initDoc = scorer.nextDoc(); + } else { + initDoc = scorer.advance(random.nextInt(maxDoc)); + } + // now run through the scorer and check if all positions are there... + do { + int docID = scorer.docID(); + if (docID == Scorer.NO_MORE_DOCS) { + break; + } + PositionIntervalIterator positions = scorer.positions(); + Integer[] pos = positionsInDoc[atomicReaderContext.docBase + docID]; + + assertEquals((float)pos.length, positions.getScorer().freq(), 0.0f); + // number of positions read should be random - don't read all of them + // allways + final int howMany = random.nextInt(20) == 0 ? pos.length + - random.nextInt(pos.length) : pos.length; + PositionInterval interval = null; + + for (int j = 0; j < howMany; j++) { + assertNotNull((interval = positions.next())); + assertEquals("iteration: " + i + " initDoc: " + initDoc + " doc: " + + docID + " base: " + atomicReaderContext.docBase + + " positions: " + Arrays.toString(pos) + " usePayloads: " + + usePayload, pos[j].intValue(), interval.begin); + assertEquals(pos[j].intValue(), interval.end); + } + if (howMany == pos.length) { + assertNull((interval = positions.next())); + } + + if (random.nextInt(10) == 0) { // once is a while advance + scorer.advance(docID + 1 + random.nextInt((maxDoc - docID))); + } + + } while (scorer.nextDoc() != Scorer.NO_MORE_DOCS); + } + + } + reader.close(); + dir.close(); + } + + /** + * tests retrieval of positions for terms that have a large number of + * occurrences to force test of buffer refill during positions iteration. + */ + public void testLargeNumberOfPositions() throws IOException { + Directory dir = newDirectory(); + RandomIndexWriter writer = new RandomIndexWriter(random, dir, + newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random))); + int howMany = 1000; + for (int i = 0; i < 39; i++) { + Document doc = new Document(); + StringBuilder builder = new StringBuilder(); + for (int j = 0; j < howMany; j++) { + if (j % 2 == 0) { + builder.append("even "); + } else { + builder.append("odd "); + } + } + doc.add(newField(fieldName, builder.toString(), Field.Store.YES, + Field.Index.ANALYZED)); + writer.addDocument(doc); + } + + // now do seaches + IndexReader reader = writer.getReader(); + writer.close(); + IndexSearcher searcher = new IndexSearcher(reader); + + for (int i = 0; i < 39 * RANDOM_MULTIPLIER; i++) { + TermQuery one = new TermQuery(new Term(fieldName, "even")); + ReaderContext topReaderContext = reader.getTopReaderContext(); + AtomicReaderContext[] leaves = ReaderUtil.leaves(topReaderContext); + Weight weight = one.createWeight(searcher); + PositionInterval interval = null; + for (AtomicReaderContext atomicReaderContext : leaves) { + Scorer scorer = weight.scorer(atomicReaderContext, ScorerContext.def() + .needsPositions(true)); + assertNotNull(scorer); + + int initDoc = 0; + int maxDoc = atomicReaderContext.reader.maxDoc(); + // initially advance or do next doc + if (random.nextBoolean()) { + initDoc = scorer.nextDoc(); + } else { + initDoc = scorer.advance(random.nextInt(maxDoc)); + } + String msg = "Iteration: " + i + " initDoc: " + initDoc + " payloads: " + + usePayload; + PositionIntervalIterator positions = scorer.positions(); + assertEquals(howMany / 2.f, positions.getScorer().freq(), 0.0); + for (int j = 0; j < howMany; j += 2) { + assertNotNull("next returned nullat index: " + j + " with freq: " + + positions.getScorer().freq() + " -- " + msg,(interval = positions.next())); + assertEquals("position missmatch index: " + j + " with freq: " + + positions.getScorer().freq() + " -- " + msg, j, interval.begin); + } + assertNull("next returned nonNull -- " + msg,(interval = positions.next())); + + } + } + reader.close(); + dir.close(); + } + +} \ No newline at end of file Property changes on: lucene/src/test/org/apache/lucene/search/TestTermQuery.java ___________________________________________________________________ Added: svn:eol-style + native Added: svn:keywords + Date Author Id Revision HeadURL Index: lucene/src/test/org/apache/lucene/search/positions/TestBlockPositionsIterator.java =================================================================== --- lucene/src/test/org/apache/lucene/search/positions/TestBlockPositionsIterator.java (revision 0) +++ lucene/src/test/org/apache/lucene/search/positions/TestBlockPositionsIterator.java (revision 0) @@ -0,0 +1,186 @@ +package org.apache.lucene.search.positions; +/** + * 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. + */ +import java.io.IOException; + +import org.apache.lucene.analysis.MockAnalyzer; +import org.apache.lucene.analysis.MockTokenizer; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexReader.AtomicReaderContext; +import org.apache.lucene.index.IndexReader.ReaderContext; +import org.apache.lucene.index.RandomIndexWriter; +import org.apache.lucene.index.Term; +import org.apache.lucene.search.BooleanClause; +import org.apache.lucene.search.BooleanClause.Occur; +import org.apache.lucene.search.BooleanQuery; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.ScoreDoc; +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.search.Weight.ScorerContext; +import org.apache.lucene.search.positions.PositionIntervalIterator.PositionInterval; +import org.apache.lucene.search.positions.PositionIntervalIterator.PositionIntervalFilter; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.LuceneTestCase; +import org.apache.lucene.util.ReaderUtil; + +public class TestBlockPositionsIterator extends LuceneTestCase { + + private static final void addDocs(RandomIndexWriter writer) throws CorruptIndexException, IOException { + { + Document doc = new Document(); + doc.add(newField( + "field", + "Pease porridge hot! Pease porridge cold! Pease porridge in the pot nine days old! Some like it hot, some" + + " like it cold, Some like it in the pot nine days old! Pease porridge hot! Pease porridge cold!", + Field.Store.YES, Field.Index.ANALYZED)); + writer.addDocument(doc); + } + + { + Document doc = new Document(); + doc.add(newField( + "field", + "Pease porridge cold! Pease porridge hot! Pease porridge in the pot nine days old! Some like it cold, some" + + " like it hot, Some like it in the pot nine days old! Pease porridge cold! Pease porridge hot!", + Field.Store.YES, Field.Index.ANALYZED)); + writer.addDocument(doc); + } + } + public void testExactPhraseBooleanConjunction() throws IOException { + Directory directory = newDirectory(); + RandomIndexWriter writer = new RandomIndexWriter(random, directory, + newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random))); + addDocs(writer); + IndexReader reader = writer.getReader(); + IndexSearcher searcher = new IndexSearcher(reader); + writer.close(); + BooleanQuery query = new BooleanQuery(); + query.add(new BooleanClause(new TermQuery(new Term("field", "pease")), Occur.MUST)); + query.add(new BooleanClause(new TermQuery(new Term("field", "porridge")), Occur.MUST)); + query.add(new BooleanClause(new TermQuery(new Term("field", "hot!")), Occur.MUST)); + { + PositionFilterQuery filter = new PositionFilterQuery(query, new BlockPositionIteratorFilter()); + TopDocs search = searcher.search(filter, 10); + ScoreDoc[] scoreDocs = search.scoreDocs; + assertEquals(2, search.totalHits); + assertEquals(0, scoreDocs[0].doc); + assertEquals(1, scoreDocs[1].doc); + } + query.add(new BooleanClause(new TermQuery(new Term("field", "pease")), Occur.MUST)); + query.add(new BooleanClause(new TermQuery(new Term("field", "porridge")), Occur.MUST)); + query.add(new BooleanClause(new TermQuery(new Term("field", "cold!")), Occur.MUST)); + + { + PositionFilterQuery filter = new PositionFilterQuery(query, new BlockPositionIteratorFilter()); + TopDocs search = searcher.search(filter, 10); + ScoreDoc[] scoreDocs = search.scoreDocs; + assertEquals(1, search.totalHits); + assertEquals(0, scoreDocs[0].doc); + } + + query = new BooleanQuery(); + query.add(new BooleanClause(new TermQuery(new Term("field", "pease")), Occur.MUST)); + query.add(new BooleanClause(new TermQuery(new Term("field", "hot!")), Occur.MUST)); + { + PositionFilterQuery filter = new PositionFilterQuery(query, new BlockPositionIteratorFilter()); + TopDocs search = searcher.search(filter, 10); + assertEquals(0, search.totalHits); + } + searcher.close(); + reader.close(); + directory.close(); + } + + public void testBlockPositionIterator() throws IOException { + Directory directory = newDirectory(); + RandomIndexWriter writer = new RandomIndexWriter(random, directory, + newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random))); + addDocs(writer); + IndexReader reader = writer.getReader(); + IndexSearcher searcher = new IndexSearcher(reader); + writer.close(); + BooleanQuery query = new BooleanQuery(); + query.add(new BooleanClause(new TermQuery(new Term("field", "pease")), Occur.MUST)); + query.add(new BooleanClause(new TermQuery(new Term("field", "porridge")), Occur.MUST)); + query.add(new BooleanClause(new TermQuery(new Term("field", "hot!")), Occur.MUST)); + + Weight weight = query.weight(searcher); + ReaderContext topReaderContext = searcher.getTopReaderContext(); + AtomicReaderContext[] leaves = ReaderUtil.leaves(topReaderContext); + assertEquals(1, leaves.length); + for (int i = 0; i < leaves.length; i++) { + Scorer scorer = weight.scorer(leaves[0], ScorerContext.def().topScorer(true).needsPositions(true)); + { + int nextDoc = scorer.nextDoc(); + assertEquals(0, nextDoc); + PositionIntervalIterator positions = new BlockPositionIterator(scorer.positions()); + PositionInterval interval = null; + int[] start = new int[] {0, 31}; + int[] end = new int[] {2, 33}; + // {start}term{end} - end is pos+1 + // {0}Pease porridge hot!{0} Pease porridge cold! Pease porridge in the pot nine days old! Some like it hot, some" + // like it cold, Some like it in the pot nine days old! {1}Pease porridge hot!{1} Pease porridge cold!", + for (int j = 0; j < end.length; j++) { + interval = positions.next(); + assertNotNull(interval); + assertEquals(start[j], interval.begin); + assertEquals(end[j], interval.end); + } + assertNull(positions.next()); + } + { + int nextDoc = scorer.nextDoc(); + assertEquals(1, nextDoc); + PositionIntervalIterator positions = new BlockPositionIterator(scorer.positions()); + PositionInterval interval = null; + int[] start = new int[] {3, 34}; + int[] end = new int[] {5, 36}; + // {start}term{end} - end is pos+1 + // Pease porridge cold! {0}Pease porridge hot!{0} Pease porridge in the pot nine days old! Some like it cold, some + // like it hot, Some like it in the pot nine days old! Pease porridge cold! {1}Pease porridge hot{1}! + for (int j = 0; j < end.length; j++) { + interval = positions.next(); + assertNotNull(interval); + assertEquals(j + "", start[j], interval.begin); + assertEquals(j+ "", end[j], interval.end); + } + assertNull(positions.next()); + } + } + + + searcher.close(); + reader.close(); + directory.close(); + } + + + public static class BlockPositionIteratorFilter implements PositionIntervalFilter { + + @Override + public PositionIntervalIterator filter(PositionIntervalIterator iter) { + return new BlockPositionIterator(iter); + } + + } +} Property changes on: lucene/src/test/org/apache/lucene/search/positions/TestBlockPositionsIterator.java ___________________________________________________________________ Added: svn:eol-style + native Added: svn:keywords + Date Author Id Revision HeadURL Index: lucene/src/test/org/apache/lucene/search/positions/TestConjunctionPositionsIterator.java =================================================================== --- lucene/src/test/org/apache/lucene/search/positions/TestConjunctionPositionsIterator.java (revision 0) +++ lucene/src/test/org/apache/lucene/search/positions/TestConjunctionPositionsIterator.java (revision 0) @@ -0,0 +1,161 @@ +package org.apache.lucene.search.positions; +/** + * 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. + */ +import java.io.IOException; + +import org.apache.lucene.analysis.MockAnalyzer; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexReader.AtomicReaderContext; +import org.apache.lucene.index.IndexReader.ReaderContext; +import org.apache.lucene.index.RandomIndexWriter; +import org.apache.lucene.index.Term; +import org.apache.lucene.search.BooleanClause; +import org.apache.lucene.search.BooleanClause.Occur; +import org.apache.lucene.search.BooleanQuery; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.ScoreDoc; +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.search.Weight.ScorerContext; +import org.apache.lucene.search.positions.PositionIntervalIterator.PositionInterval; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.LuceneTestCase; +import org.apache.lucene.util.ReaderUtil; + +public class TestConjunctionPositionsIterator extends LuceneTestCase { + + private static final void addDocs(RandomIndexWriter writer) throws CorruptIndexException, IOException { + { + Document doc = new Document(); + doc.add(newField( + "field", + "Pease porridge hot! Pease porridge cold! Pease porridge in the pot nine days old! Some like it hot, some" + + " like it cold, Some like it in the pot nine days old! Pease porridge hot! Pease porridge cold!", + Field.Store.YES, Field.Index.ANALYZED)); + writer.addDocument(doc); + } + + { + Document doc = new Document(); + doc.add(newField( + "field", + "Pease porridge cold! Pease porridge hot! Pease porridge in the pot nine days old! Some like it cold, some" + + " like it hot, Some like it in the pot nine days old! Pease porridge cold! Pease porridge hot!", + Field.Store.YES, Field.Index.ANALYZED)); + writer.addDocument(doc); + } + } + public void testConjunctionPositionsBooleanQuery() throws IOException { + Directory directory = newDirectory(); + RandomIndexWriter writer = new RandomIndexWriter(random, directory, + newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random))); + addDocs(writer); + + IndexReader reader = writer.getReader(); + IndexSearcher searcher = new IndexSearcher(reader); + writer.close(); + BooleanQuery query = new BooleanQuery(); + query.add(new BooleanClause(new TermQuery(new Term("field", "porridge")), Occur.MUST)); + query.add(new BooleanClause(new TermQuery(new Term("field", "pease")), Occur.MUST)); + query.add(new BooleanClause(new TermQuery(new Term("field", "hot!")), Occur.MUST)); + { + PositionFilterQuery filter = new PositionFilterQuery(query, new RangePositionsIterator(0,3)); + TopDocs search = searcher.search(filter, 10); + ScoreDoc[] scoreDocs = search.scoreDocs; + assertEquals(1, search.totalHits); + assertEquals(0, scoreDocs[0].doc); + } + { + PositionFilterQuery filter = new PositionFilterQuery(query, new WithinPositionIterator(3)); + TopDocs search = searcher.search(filter, 10); + ScoreDoc[] scoreDocs = search.scoreDocs; + assertEquals(2, search.totalHits); + assertEquals(0, scoreDocs[0].doc); + assertEquals(1, scoreDocs[1].doc); + } + + searcher.close(); + reader.close(); + directory.close(); + } + + public void testConjuctionPositionIterator() throws IOException { + Directory directory = newDirectory(); + RandomIndexWriter writer = new RandomIndexWriter(random, directory, + newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random))); + addDocs(writer); + + IndexReader reader = writer.getReader(); + IndexSearcher searcher = new IndexSearcher(reader); + writer.close(); + BooleanQuery query = new BooleanQuery(); + query.add(new BooleanClause(new TermQuery(new Term("field", "porridge")), Occur.MUST)); + query.add(new BooleanClause(new TermQuery(new Term("field", "pease")), Occur.MUST)); + query.add(new BooleanClause(new TermQuery(new Term("field", "hot!")), Occur.MUST)); + Weight weight = query.weight(searcher); + ReaderContext topReaderContext = searcher.getTopReaderContext(); + AtomicReaderContext[] leaves = ReaderUtil.leaves(topReaderContext); + assertEquals(1, leaves.length); + for (int i = 0; i < leaves.length; i++) { + Scorer scorer = weight.scorer(leaves[0], ScorerContext.def().topScorer(true).needsPositions(true)); + { + int nextDoc = scorer.nextDoc(); + assertEquals(0, nextDoc); + PositionIntervalIterator positions = scorer.positions(); + PositionInterval interval = null; + int[] start = new int[] {0, 1, 2, 3, 4, 6, 7, 31, 32, 33}; + int[] end = new int[] {2, 3, 4, 33, 33, 33, 33, 33, 34, 35}; + // {start}term{end} - end is pos+1 + // {0}Pease {1}porridge {2}hot!{0} {3}Pease{1} {4}porridge{2} cold! {5}Pease {6}porridge in the pot nine days old! Some like it hot, some" + // like it cold, Some like it in the pot nine days old! {7}Pease {8}porridge {9}hot!{3,4,5,6,7} Pease{8} porridge{9} cold!", + for (int j = 0; j < end.length; j++) { + interval = positions.next(); + assertNotNull(interval); + assertEquals(start[j], interval.begin); + assertEquals(end[j], interval.end); + } + assertNull(positions.next()); + } + { + int nextDoc = scorer.nextDoc(); + assertEquals(1, nextDoc); + PositionIntervalIterator positions = scorer.positions(); + PositionInterval interval = null; + int[] start = new int[] {0, 1, 3, 4, 5, 6, 7, 31, 32, 34 }; + int[] end = new int[] {5, 5, 5, 6, 7, 36, 36, 36, 36, 36 }; + // {start}term{end} - end is pos+1 + // {0}Pease {1}porridge cold! {2}Pease {3}porridge {4}hot!{0, 1, 2, 3} {5}Pease {4, 6}porridge in the pot nine days old! Some like it cold, some + // like it hot, Some like it in the pot nine days old! {7}Pease {8}porridge cold! {9}Pease porridge hot{5, 6, 7, 8, 9}! + for (int j = 0; j < end.length; j++) { + interval = positions.next(); + assertNotNull(interval); + assertEquals(j + "", start[j], interval.begin); + assertEquals(j+ "", end[j], interval.end); + } + assertNull(positions.next()); + } + } + searcher.close(); + reader.close(); + directory.close(); + } +} Property changes on: lucene/src/test/org/apache/lucene/search/positions/TestConjunctionPositionsIterator.java ___________________________________________________________________ Added: svn:eol-style + native Added: svn:keywords + Date Author Id Revision HeadURL Index: lucene/src/test/org/apache/lucene/search/positions/TestOrderedConjunctionPositionsIterator.java =================================================================== --- lucene/src/test/org/apache/lucene/search/positions/TestOrderedConjunctionPositionsIterator.java (revision 0) +++ lucene/src/test/org/apache/lucene/search/positions/TestOrderedConjunctionPositionsIterator.java (revision 0) @@ -0,0 +1,177 @@ +package org.apache.lucene.search.positions; + +import java.io.IOException; + +import org.apache.lucene.analysis.MockAnalyzer; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexReader.AtomicReaderContext; +import org.apache.lucene.index.IndexReader.ReaderContext; +import org.apache.lucene.index.RandomIndexWriter; +import org.apache.lucene.index.Term; +import org.apache.lucene.search.BooleanClause; +import org.apache.lucene.search.BooleanClause.Occur; +import org.apache.lucene.search.BooleanQuery; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.ScoreDoc; +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.search.Weight.ScorerContext; +import org.apache.lucene.search.positions.PositionIntervalIterator.PositionInterval; +import org.apache.lucene.search.positions.PositionIntervalIterator.PositionIntervalFilter; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.LuceneTestCase; +import org.apache.lucene.util.ReaderUtil; + +public class TestOrderedConjunctionPositionsIterator extends LuceneTestCase { + + private static final void addDocs(RandomIndexWriter writer) throws CorruptIndexException, IOException { + { + Document doc = new Document(); + doc.add(newField( + "field", + "Pease porridge hot! Pease porridge cold! Pease porridge in the pot nine days old! Some like it hot, some" + + " like it cold, Some like it in the pot nine days old! Pease porridge hot! Pease porridge cold!", + Field.Store.YES, Field.Index.ANALYZED)); + writer.addDocument(doc); + } + + { + Document doc = new Document(); + doc.add(newField( + "field", + "Pease porridge cold! Pease porridge hot! Pease porridge in the pot nine days old! Some like it cold, some" + + " like it hot, Some like it in the pot nine days old! Pease porridge cold! Pease porridge hot!", + Field.Store.YES, Field.Index.ANALYZED)); + writer.addDocument(doc); + } + + { + Document doc = new Document(); + doc.add(newField( + "field", + "Pease porridge cold! Pease porridge hot! Pease porridge in the pot nine days old!", + Field.Store.YES, Field.Index.ANALYZED)); + writer.addDocument(doc); + } + } + public void testConjunctionPositionsBooleanQuery() throws IOException { + Directory directory = newDirectory(); + RandomIndexWriter writer = new RandomIndexWriter(random, directory, + newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random))); + addDocs(writer); + + IndexReader reader = writer.getReader(); + IndexSearcher searcher = new IndexSearcher(reader); + writer.close(); + BooleanQuery query = new BooleanQuery(); + query.add(new BooleanClause(new TermQuery(new Term("field", "hot!")), Occur.MUST)); + query.add(new BooleanClause(new TermQuery(new Term("field", "pease")), Occur.MUST)); + query.add(new BooleanClause(new TermQuery(new Term("field", "porridge")), Occur.MUST)); + { + PositionFilterQuery filter = new PositionFilterQuery(query, new OrderedConjunctionPositionIteratorFilter()); + TopDocs search = searcher.search(filter, 10); + ScoreDoc[] scoreDocs = search.scoreDocs; + assertEquals(3, search.totalHits); + assertEquals(2, scoreDocs[0].doc); + assertEquals(0, scoreDocs[1].doc); + assertEquals(1, scoreDocs[2].doc); + } + + query = new BooleanQuery(); + query.add(new BooleanClause(new TermQuery(new Term("field", "old!")), Occur.MUST)); + query.add(new BooleanClause(new TermQuery(new Term("field", "pease")), Occur.MUST)); + query.add(new BooleanClause(new TermQuery(new Term("field", "porridge")), Occur.MUST)); + query.add(new BooleanClause(new TermQuery(new Term("field", "cold!")), Occur.MUST)); + + { + PositionFilterQuery filter = new PositionFilterQuery(query, new OrderedConjunctionPositionIteratorFilter()); + TopDocs search = searcher.search(filter, 10); + ScoreDoc[] scoreDocs = search.scoreDocs; + assertEquals(2, search.totalHits); + assertEquals(0, scoreDocs[0].doc); + assertEquals(1, scoreDocs[1].doc); + } + + searcher.close(); + reader.close(); + directory.close(); + } + + public void testConjuctionPositionIterator() throws IOException { + Directory directory = newDirectory(); + RandomIndexWriter writer = new RandomIndexWriter(random, directory, + newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random))); + addDocs(writer); + + IndexReader reader = writer.getReader(); + writer.optimize(); + IndexSearcher searcher = new IndexSearcher(reader); + writer.close(); + BooleanQuery query = new BooleanQuery(); + query.add(new BooleanClause(new TermQuery(new Term("field", "pease")), Occur.MUST)); + query.add(new BooleanClause(new TermQuery(new Term("field", "porridge")), Occur.MUST)); + query.add(new BooleanClause(new TermQuery(new Term("field", "hot!")), Occur.MUST)); + Weight weight = query.weight(searcher); + ReaderContext topReaderContext = searcher.getTopReaderContext(); + AtomicReaderContext[] leaves = ReaderUtil.leaves(topReaderContext); + assertEquals(1, leaves.length); + for (int i = 0; i < leaves.length; i++) { + Scorer scorer = weight.scorer(leaves[0], ScorerContext.def().topScorer(true).needsPositions(true)); + { + int nextDoc = scorer.nextDoc(); + assertEquals(0, nextDoc); + PositionIntervalIterator positions = new OrderedConjunctionPositionIterator(scorer.positions()); + PositionInterval interval = null; + int[] start = new int[] {0, 31}; + int[] end = new int[] {2, 33}; + // {start}term{end} - end is pos+1 + // {0}Pease porridge hot!{0} Pease porridge cold! Pease porridge in the pot nine days old! Some like it hot, some" + // like it cold, Some like it in the pot nine days old! {1}Pease porridge hot!{1} Pease porridge cold!", + + for (int j = 0; j < end.length; j++) { + interval = positions.next(); + + assertNotNull(interval); + assertEquals(start[j], interval.begin); + assertEquals(end[j], interval.end); + } + assertNull(positions.next()); + } + { + int nextDoc = scorer.nextDoc(); + assertEquals(1, nextDoc); + PositionIntervalIterator positions = new OrderedConjunctionPositionIterator(scorer.positions()); + PositionInterval interval = null; + int[] start = new int[] {3, 34}; + int[] end = new int[] {5, 36}; + // {start}term{end} - end is pos+1 + // Pease porridge cold! {0}Pease porridge hot!{0} Pease porridge in the pot nine days old! Some like it cold, some + // like it hot, Some like it in the pot nine days old! Pease porridge cold! {1}Pease porridge hot{1}! + for (int j = 0; j < end.length; j++) { + interval = positions.next(); + assertNotNull(interval); + assertEquals(j + "", start[j], interval.begin); + assertEquals(j+ "", end[j], interval.end); + } + assertNull(positions.next()); + } + } + searcher.close(); + reader.close(); + directory.close(); + } + + public static class OrderedConjunctionPositionIteratorFilter implements PositionIntervalFilter { + + @Override + public PositionIntervalIterator filter(PositionIntervalIterator iter) { + return new OrderedConjunctionPositionIterator(iter); + } + + } +} Property changes on: lucene/src/test/org/apache/lucene/search/positions/TestOrderedConjunctionPositionsIterator.java ___________________________________________________________________ Added: svn:eol-style + native Added: svn:keywords + Date Author Id Revision HeadURL Index: lucene/src/test/org/apache/lucene/search/spans/TestNearSpansOrdered.java =================================================================== --- lucene/src/test/org/apache/lucene/search/spans/TestNearSpansOrdered.java (revision 1135110) +++ lucene/src/test/org/apache/lucene/search/spans/TestNearSpansOrdered.java (working copy) @@ -26,12 +26,22 @@ import org.apache.lucene.index.RandomIndexWriter; import org.apache.lucene.index.Term; import org.apache.lucene.queryParser.QueryParser; +import org.apache.lucene.search.BooleanClause; +import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.CheckHits; import org.apache.lucene.search.Explanation; import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.Weight; import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.BooleanClause.Occur; import org.apache.lucene.search.Weight.ScorerContext; +import org.apache.lucene.search.positions.OrderedConjunctionPositionIterator; +import org.apache.lucene.search.positions.PositionFilterQuery; +import org.apache.lucene.search.positions.PositionIntervalIterator; +import org.apache.lucene.search.positions.WithinPositionIterator; +import org.apache.lucene.search.positions.PositionIntervalIterator.PositionIntervalFilter; import org.apache.lucene.store.Directory; import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.ReaderUtil; @@ -75,22 +85,34 @@ "w1 w3 xx w2 yy w3 zz" }; - protected SpanNearQuery makeQuery(String s1, String s2, String s3, + protected SpanQuery 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); + + BooleanQuery query = new BooleanQuery(); + query.add(new BooleanClause(new TermQuery(new Term(FIELD, s1)), Occur.MUST)); + query.add(new BooleanClause(new TermQuery(new Term(FIELD, s2)), Occur.MUST)); + query.add(new BooleanClause(new TermQuery(new Term(FIELD, s3)), Occur.MUST)); + return new MockSpanQuery(query, false, FIELD, new Filter(3 + slop-1)); } - protected SpanNearQuery makeQuery() { + + public static class Filter implements PositionIntervalFilter { + + private int slop; + public Filter(int slop) { + this.slop = slop; + } + @Override + public PositionIntervalIterator filter(PositionIntervalIterator iter) { + return new WithinPositionIterator(slop, new OrderedConjunctionPositionIterator(iter)); + } + + } + protected SpanQuery makeQuery() { return makeQuery("w1","w2","w3",1,true); } public void testSpanNearQuery() throws Exception { - SpanNearQuery q = makeQuery(); + Query q = makeQuery(); CheckHits.checkHits(random, q, FIELD, searcher, new int[] {0,1}); } @@ -102,7 +124,7 @@ } public void testNearSpansNext() throws Exception { - SpanNearQuery q = makeQuery(); + SpanQuery q = makeQuery(); Spans span = MultiSpansWrapper.wrap(searcher.getTopReaderContext(), q); assertEquals(true, span.next()); assertEquals(s(0,0,3), s(span)); @@ -117,7 +139,7 @@ * does not contain more than one span */ public void testNearSpansSkipToLikeNext() throws Exception { - SpanNearQuery q = makeQuery(); + SpanQuery q = makeQuery(); Spans span = MultiSpansWrapper.wrap(searcher.getTopReaderContext(), q); assertEquals(true, span.skipTo(0)); assertEquals(s(0,0,3), s(span)); @@ -127,7 +149,7 @@ } public void testNearSpansNextThenSkipTo() throws Exception { - SpanNearQuery q = makeQuery(); + SpanQuery q = makeQuery(); Spans span = MultiSpansWrapper.wrap(searcher.getTopReaderContext(), q); assertEquals(true, span.next()); assertEquals(s(0,0,3), s(span)); @@ -137,7 +159,7 @@ } public void testNearSpansNextThenSkipPast() throws Exception { - SpanNearQuery q = makeQuery(); + SpanQuery q = makeQuery(); Spans span = MultiSpansWrapper.wrap(searcher.getTopReaderContext(), q); assertEquals(true, span.next()); assertEquals(s(0,0,3), s(span)); @@ -145,20 +167,20 @@ } public void testNearSpansSkipPast() throws Exception { - SpanNearQuery q = makeQuery(); + SpanQuery q = makeQuery(); Spans span = MultiSpansWrapper.wrap(searcher.getTopReaderContext(), q); assertEquals(false, span.skipTo(2)); } public void testNearSpansSkipTo0() throws Exception { - SpanNearQuery q = makeQuery(); + SpanQuery q = makeQuery(); Spans span = MultiSpansWrapper.wrap(searcher.getTopReaderContext(), q); assertEquals(true, span.skipTo(0)); assertEquals(s(0,0,3), s(span)); } public void testNearSpansSkipTo1() throws Exception { - SpanNearQuery q = makeQuery(); + SpanQuery q = makeQuery(); Spans span = MultiSpansWrapper.wrap(searcher.getTopReaderContext(), q); assertEquals(true, span.skipTo(1)); assertEquals(s(1,0,4), s(span)); @@ -169,7 +191,7 @@ * this causes problems */ public void testSpanNearScorerSkipTo1() throws Exception { - SpanNearQuery q = makeQuery(); + SpanQuery q = makeQuery(); Weight w = q.weight(searcher); ReaderContext topReaderContext = searcher.getTopReaderContext(); AtomicReaderContext[] leaves = ReaderUtil.leaves(topReaderContext); @@ -181,7 +203,7 @@ * this causes problems */ public void testSpanNearScorerExplain() throws Exception { - SpanNearQuery q = makeQuery(); + SpanQuery q = makeQuery(); ReaderContext topReaderContext = searcher.getTopReaderContext(); AtomicReaderContext[] leaves = ReaderUtil.leaves(topReaderContext); Index: lucene/src/test/org/apache/lucene/search/spans/TestSpanFirstQuery.java =================================================================== --- lucene/src/test/org/apache/lucene/search/spans/TestSpanFirstQuery.java (revision 1135110) +++ lucene/src/test/org/apache/lucene/search/spans/TestSpanFirstQuery.java (working copy) @@ -26,6 +26,8 @@ import org.apache.lucene.index.RandomIndexWriter; import org.apache.lucene.index.Term; import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.TermQuery; +import org.apache.lucene.search.positions.RangePositionsIterator; import org.apache.lucene.store.Directory; import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.automaton.CharacterRunAutomaton; @@ -64,4 +66,37 @@ reader.close(); dir.close(); } + + public void testStartPositionsAlt() throws Exception { + Directory dir = newDirectory(); + + // mimic StopAnalyzer + CharacterRunAutomaton stopSet = new CharacterRunAutomaton(new RegExp("the|a|of").toAutomaton()); + Analyzer analyzer = new MockAnalyzer(random, MockTokenizer.SIMPLE, true, stopSet, true); + + RandomIndexWriter writer = new RandomIndexWriter(random, dir, analyzer); + Document doc = new Document(); + doc.add(newField("field", "the quick brown fox", Field.Index.ANALYZED)); + writer.addDocument(doc); + Document doc2 = new Document(); + doc2.add(newField("field", "quick brown fox", Field.Index.ANALYZED)); + writer.addDocument(doc2); + + IndexReader reader = writer.getReader(); + IndexSearcher searcher = new IndexSearcher(reader); + + // user queries on "starts-with quick" + SpanQuery sfq = new MockSpanQuery(new TermQuery(new Term("field", "quick")), false,"field", new RangePositionsIterator(0, 0)); + assertEquals(1, searcher.search(sfq, 10).totalHits); + + // user queries on "starts-with the quick" + SpanQuery include = new MockSpanQuery(new TermQuery(new Term("field", "quick")), false,"field", new RangePositionsIterator(0, 1)); + sfq = new SpanNotQuery(include, sfq); + assertEquals(1, searcher.search(sfq, 10).totalHits); + + writer.close(); + searcher.close(); + reader.close(); + dir.close(); + } } Index: lucene/src/test/org/apache/lucene/search/spans/TestSpans.java =================================================================== --- lucene/src/test/org/apache/lucene/search/spans/TestSpans.java (revision 1135110) +++ lucene/src/test/org/apache/lucene/search/spans/TestSpans.java (working copy) @@ -17,6 +17,8 @@ * limitations under the License. */ +import org.apache.lucene.search.BooleanClause; +import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.DefaultSimilarityProvider; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.Query; @@ -27,7 +29,12 @@ import org.apache.lucene.search.SimilarityProvider; import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.BooleanClause.Occur; import org.apache.lucene.search.Weight.ScorerContext; +import org.apache.lucene.search.positions.OrderedConjunctionPositionIterator; +import org.apache.lucene.search.positions.PositionIntervalIterator; +import org.apache.lucene.search.positions.WithinPositionIterator; +import org.apache.lucene.search.positions.PositionIntervalIterator.PositionIntervalFilter; import org.apache.lucene.store.Directory; import org.apache.lucene.analysis.MockAnalyzer; import org.apache.lucene.index.IndexReader.ReaderContext; @@ -194,13 +201,19 @@ public void testSpanNearOrderedOverlap() throws Exception { boolean ordered = true; int slop = 1; - SpanNearQuery snq = new SpanNearQuery( - new SpanQuery[] { - makeSpanTermQuery("t1"), - makeSpanTermQuery("t2"), - makeSpanTermQuery("t3") }, - slop, - ordered); + + BooleanQuery query = new BooleanQuery(); + query.add(new BooleanClause(new TermQuery(new Term(field, "t1")), Occur.MUST)); + query.add(new BooleanClause(new TermQuery(new Term(field, "t2")), Occur.MUST)); + query.add(new BooleanClause(new TermQuery(new Term(field, "t3")), Occur.MUST)); + SpanQuery snq = new MockSpanQuery(query, false, field, new Filter(3 + slop-1)); +// SpanNearQuery snq = new SpanNearQuery( +// new SpanQuery[] { +// makeSpanTermQuery("t1"), +// makeSpanTermQuery("t2"), +// makeSpanTermQuery("t3") }, +// slop, +// ordered); Spans spans = MultiSpansWrapper.wrap(searcher.getTopReaderContext(), snq); assertTrue("first range", spans.next()); @@ -215,7 +228,19 @@ assertFalse("third range", spans.next()); } + public static class Filter implements PositionIntervalFilter { + private int slop; + public Filter(int slop) { + this.slop = slop; + } + @Override + public PositionIntervalIterator filter(PositionIntervalIterator iter) { + return new WithinPositionIterator(slop, new OrderedConjunctionPositionIterator(iter)); +// return new OrderedConjunctionPositionIterator(iter); + } + + } public void testSpanNearUnOrdered() throws Exception {