LUCENE-1001: Make payloads accessible from SpanQuery's

git-svn-id: https://svn.apache.org/repos/asf/lucene/java/trunk@687379 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Grant Ingersoll 2008-08-20 16:06:36 +00:00
parent bb6b711718
commit 09467ad18e
17 changed files with 1062 additions and 61 deletions

View File

@ -232,6 +232,8 @@ New features
17. LUCENE-1297: Allow other string distance measures for the SpellChecker 17. LUCENE-1297: Allow other string distance measures for the SpellChecker
(Thomas Morton via Otis Gospodnetic) (Thomas Morton via Otis Gospodnetic)
18. LUCENE-1001: Provide access to Payloads via Spans. All existing Span Query implementations in Lucene implement. (Mark Miller, Grant Ingersoll)
Optimizations Optimizations
1. LUCENE-705: When building a compound file, use 1. LUCENE-705: When building a compound file, use

View File

@ -27,6 +27,7 @@ import org.apache.lucene.search.spans.SpanOrQuery;
import org.apache.lucene.search.spans.SpanQuery; import org.apache.lucene.search.spans.SpanQuery;
import org.apache.lucene.search.spans.SpanTermQuery; import org.apache.lucene.search.spans.SpanTermQuery;
import org.apache.lucene.search.spans.Spans; import org.apache.lucene.search.spans.Spans;
import org.apache.lucene.search.spans.PayloadSpans;
import org.apache.lucene.util.ToStringUtils; import org.apache.lucene.util.ToStringUtils;
import java.io.IOException; import java.io.IOException;
@ -76,6 +77,10 @@ public class SpanRegexQuery extends SpanQuery implements RegexQueryCapable {
throw new UnsupportedOperationException("Query should have been rewritten"); throw new UnsupportedOperationException("Query should have been rewritten");
} }
public PayloadSpans getPayloadSpans(IndexReader reader) throws IOException {
throw new UnsupportedOperationException("Query should have been rewritten");
}
public String getField() { public String getField() {
return term.field(); return term.field();
} }

View File

@ -0,0 +1,190 @@
package org.apache.lucene.search.payloads;
import org.apache.lucene.search.BooleanClause;
/**
* 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 java.util.Iterator;
import java.util.List;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.DisjunctionMaxQuery;
import org.apache.lucene.search.FilteredQuery;
import org.apache.lucene.search.MultiPhraseQuery;
import org.apache.lucene.search.PhraseQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.spans.PayloadSpans;
import org.apache.lucene.search.spans.SpanNearQuery;
import org.apache.lucene.search.spans.SpanOrQuery;
import org.apache.lucene.search.spans.SpanQuery;
import org.apache.lucene.search.spans.SpanTermQuery;
/**
* Experimental class to get set of payloads for most standard Lucene queries.
* Operates like Highlighter - IndexReader should only contain doc of interest,
* best to use MemoryIndex.
*
* <p/>
* <font color="#FF0000">
* WARNING: The status of the <b>Payloads</b> feature is experimental.
* The APIs introduced here might change in the future and will not be
* supported anymore in such a case.</font>
*
*/
public class PayloadSpanUtil {
private IndexReader reader;
/**
* @param reader
* that contains doc with payloads to extract
*/
public PayloadSpanUtil(IndexReader reader) {
this.reader = reader;
}
/**
* Query should be rewritten for wild/fuzzy support.
*
* @param query
* @return
* @throws IOException
*/
public Collection getPayloadsForQuery(Query query) throws IOException {
Collection payloads = new ArrayList();
queryToSpanQuery(query, payloads);
return payloads;
}
private void queryToSpanQuery(Query query, Collection payloads)
throws IOException {
if (query instanceof BooleanQuery) {
BooleanClause[] queryClauses = ((BooleanQuery) query).getClauses();
for (int i = 0; i < queryClauses.length; i++) {
if (!queryClauses[i].isProhibited()) {
queryToSpanQuery(queryClauses[i].getQuery(), payloads);
}
}
} else if (query instanceof PhraseQuery) {
Term[] phraseQueryTerms = ((PhraseQuery) query).getTerms();
SpanQuery[] clauses = new SpanQuery[phraseQueryTerms.length];
for (int i = 0; i < phraseQueryTerms.length; i++) {
clauses[i] = new SpanTermQuery(phraseQueryTerms[i]);
}
int slop = ((PhraseQuery) query).getSlop();
boolean inorder = false;
if (slop == 0) {
inorder = true;
}
SpanNearQuery sp = new SpanNearQuery(clauses, slop, inorder);
sp.setBoost(query.getBoost());
getPayloads(payloads, sp);
} else if (query instanceof TermQuery) {
SpanTermQuery stq = new SpanTermQuery(((TermQuery) query).getTerm());
stq.setBoost(query.getBoost());
getPayloads(payloads, stq);
} else if (query instanceof SpanQuery) {
getPayloads(payloads, (SpanQuery) query);
} else if (query instanceof FilteredQuery) {
queryToSpanQuery(((FilteredQuery) query).getQuery(), payloads);
} else if (query instanceof DisjunctionMaxQuery) {
for (Iterator iterator = ((DisjunctionMaxQuery) query).iterator(); iterator
.hasNext();) {
queryToSpanQuery((Query) iterator.next(), payloads);
}
} else if (query instanceof MultiPhraseQuery) {
final MultiPhraseQuery mpq = (MultiPhraseQuery) query;
final List termArrays = mpq.getTermArrays();
final int[] positions = mpq.getPositions();
if (positions.length > 0) {
int maxPosition = positions[positions.length - 1];
for (int i = 0; i < positions.length - 1; ++i) {
if (positions[i] > maxPosition) {
maxPosition = positions[i];
}
}
final List[] disjunctLists = new List[maxPosition + 1];
int distinctPositions = 0;
for (int i = 0; i < termArrays.size(); ++i) {
final Term[] termArray = (Term[]) termArrays.get(i);
List disjuncts = disjunctLists[positions[i]];
if (disjuncts == null) {
disjuncts = (disjunctLists[positions[i]] = new ArrayList(
termArray.length));
++distinctPositions;
}
for (int j = 0; j < termArray.length; ++j) {
disjuncts.add(new SpanTermQuery(termArray[j]));
}
}
int positionGaps = 0;
int position = 0;
final SpanQuery[] clauses = new SpanQuery[distinctPositions];
for (int i = 0; i < disjunctLists.length; ++i) {
List disjuncts = disjunctLists[i];
if (disjuncts != null) {
clauses[position++] = new SpanOrQuery((SpanQuery[]) disjuncts
.toArray(new SpanQuery[disjuncts.size()]));
} else {
++positionGaps;
}
}
final int slop = mpq.getSlop();
final boolean inorder = (slop == 0);
SpanNearQuery sp = new SpanNearQuery(clauses, slop + positionGaps,
inorder);
sp.setBoost(query.getBoost());
getPayloads(payloads, sp);
}
}
}
private void getPayloads(Collection payloads, SpanQuery query)
throws IOException {
PayloadSpans spans = query.getPayloadSpans(reader);
while (spans.next() == true) {
if (spans.isPayloadAvailable()) {
Collection payload = spans.getPayload();
Iterator it = payload.iterator();
while (it.hasNext()) {
byte[] bytes = (byte[]) it.next();
payloads.add(bytes);
}
}
}
}
}

View File

@ -17,12 +17,14 @@ package org.apache.lucene.search.spans;
* limitations under the License. * limitations under the License.
*/ */
import java.io.IOException; import org.apache.lucene.index.IndexReader;
import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.Comparator; import java.util.Comparator;
import java.util.LinkedList;
import org.apache.lucene.index.IndexReader; import java.util.List;
import java.util.Collection;
/** A Spans that is formed from the ordered subspans of a SpanNearQuery /** A Spans that is formed from the ordered subspans of a SpanNearQuery
* where the subspans do not overlap and have a maximum slop between them. * where the subspans do not overlap and have a maximum slop between them.
@ -42,13 +44,13 @@ import org.apache.lucene.index.IndexReader;
* <pre>t1 t2 .. t3 </pre> * <pre>t1 t2 .. t3 </pre>
* <pre> t1 .. t2 t3</pre> * <pre> t1 .. t2 t3</pre>
*/ */
class NearSpansOrdered implements Spans { class NearSpansOrdered implements PayloadSpans {
private final int allowedSlop; private final int allowedSlop;
private boolean firstTime = true; private boolean firstTime = true;
private boolean more = false; private boolean more = false;
/** The spans in the same order as the SpanNearQuery */ /** The spans in the same order as the SpanNearQuery */
private final Spans[] subSpans; private final PayloadSpans[] subSpans;
/** Indicates that all subSpans have same doc() */ /** Indicates that all subSpans have same doc() */
private boolean inSameDoc = false; private boolean inSameDoc = false;
@ -56,8 +58,9 @@ class NearSpansOrdered implements Spans {
private int matchDoc = -1; private int matchDoc = -1;
private int matchStart = -1; private int matchStart = -1;
private int matchEnd = -1; private int matchEnd = -1;
private List/*<byte[]>*/ matchPayload;
private final Spans[] subSpansByDoc; private final PayloadSpans[] subSpansByDoc;
private final Comparator spanDocComparator = new Comparator() { private final Comparator spanDocComparator = new Comparator() {
public int compare(Object o1, Object o2) { public int compare(Object o1, Object o2) {
return ((Spans)o1).doc() - ((Spans)o2).doc(); return ((Spans)o1).doc() - ((Spans)o2).doc();
@ -74,10 +77,11 @@ class NearSpansOrdered implements Spans {
} }
allowedSlop = spanNearQuery.getSlop(); allowedSlop = spanNearQuery.getSlop();
SpanQuery[] clauses = spanNearQuery.getClauses(); SpanQuery[] clauses = spanNearQuery.getClauses();
subSpans = new Spans[clauses.length]; subSpans = new PayloadSpans[clauses.length];
subSpansByDoc = new Spans[clauses.length]; matchPayload = new LinkedList();
subSpansByDoc = new PayloadSpans[clauses.length];
for (int i = 0; i < clauses.length; i++) { for (int i = 0; i < clauses.length; i++) {
subSpans[i] = clauses[i].getSpans(reader); subSpans[i] = clauses[i].getPayloadSpans(reader);
subSpansByDoc[i] = subSpans[i]; // used in toSameDoc() subSpansByDoc[i] = subSpans[i]; // used in toSameDoc()
} }
query = spanNearQuery; // kept for toString() only. query = spanNearQuery; // kept for toString() only.
@ -92,6 +96,16 @@ class NearSpansOrdered implements Spans {
// inherit javadocs // inherit javadocs
public int end() { return matchEnd; } public int end() { return matchEnd; }
// TODO: Remove warning after API has been finalized
public Collection/*<byte[]>*/ getPayload() throws IOException {
return matchPayload;
}
// TODO: Remove warning after API has been finalized
public boolean isPayloadAvailable() {
return matchPayload.isEmpty() == false;
}
// inherit javadocs // inherit javadocs
public boolean next() throws IOException { public boolean next() throws IOException {
if (firstTime) { if (firstTime) {
@ -104,6 +118,7 @@ class NearSpansOrdered implements Spans {
} }
more = true; more = true;
} }
matchPayload.clear();
return advanceAfterOrdered(); return advanceAfterOrdered();
} }
@ -126,6 +141,7 @@ class NearSpansOrdered implements Spans {
return false; return false;
} }
} }
matchPayload.clear();
return advanceAfterOrdered(); return advanceAfterOrdered();
} }
@ -218,11 +234,19 @@ class NearSpansOrdered implements Spans {
private boolean shrinkToAfterShortestMatch() throws IOException { private boolean shrinkToAfterShortestMatch() throws IOException {
matchStart = subSpans[subSpans.length - 1].start(); matchStart = subSpans[subSpans.length - 1].start();
matchEnd = subSpans[subSpans.length - 1].end(); matchEnd = subSpans[subSpans.length - 1].end();
if (subSpans[subSpans.length - 1].isPayloadAvailable()) {
matchPayload.addAll(subSpans[subSpans.length - 1].getPayload());
}
int matchSlop = 0; int matchSlop = 0;
int lastStart = matchStart; int lastStart = matchStart;
int lastEnd = matchEnd; int lastEnd = matchEnd;
for (int i = subSpans.length - 2; i >= 0; i--) { for (int i = subSpans.length - 2; i >= 0; i--) {
Spans prevSpans = subSpans[i]; PayloadSpans prevSpans = subSpans[i];
if (subSpans[i].isPayloadAvailable()) {
matchPayload.addAll(0, subSpans[i].getPayload());
}
int prevStart = prevSpans.start(); int prevStart = prevSpans.start();
int prevEnd = prevSpans.end(); int prevEnd = prevSpans.end();
while (true) { // Advance prevSpans until after (lastStart, lastEnd) while (true) { // Advance prevSpans until after (lastStart, lastEnd)
@ -248,6 +272,7 @@ class NearSpansOrdered implements Spans {
if (matchStart > prevEnd) { // Only non overlapping spans add to slop. if (matchStart > prevEnd) { // Only non overlapping spans add to slop.
matchSlop += (matchStart - prevEnd); matchSlop += (matchStart - prevEnd);
} }
/* Do not break on (matchSlop > allowedSlop) here to make sure /* Do not break on (matchSlop > allowedSlop) here to make sure
* that subSpans[0] is advanced after the match, if any. * that subSpans[0] is advanced after the match, if any.
*/ */

View File

@ -17,15 +17,17 @@ package org.apache.lucene.search.spans;
* limitations under the License. * limitations under the License.
*/ */
import java.io.IOException;
import java.util.List;
import java.util.ArrayList;
import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexReader;
import org.apache.lucene.util.PriorityQueue; import org.apache.lucene.util.PriorityQueue;
class NearSpansUnordered implements Spans { import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.Set;
import java.util.HashSet;
class NearSpansUnordered implements PayloadSpans {
private SpanNearQuery query; private SpanNearQuery query;
private List ordered = new ArrayList(); // spans in query order private List ordered = new ArrayList(); // spans in query order
@ -60,13 +62,13 @@ class NearSpansUnordered implements Spans {
/** Wraps a Spans, and can be used to form a linked list. */ /** Wraps a Spans, and can be used to form a linked list. */
private class SpansCell implements Spans { private class SpansCell implements PayloadSpans {
private Spans spans; private PayloadSpans spans;
private SpansCell next; private SpansCell next;
private int length = -1; private int length = -1;
private int index; private int index;
public SpansCell(Spans spans, int index) { public SpansCell(PayloadSpans spans, int index) {
this.spans = spans; this.spans = spans;
this.index = index; this.index = index;
} }
@ -99,6 +101,15 @@ class NearSpansUnordered implements Spans {
public int doc() { return spans.doc(); } public int doc() { return spans.doc(); }
public int start() { return spans.start(); } public int start() { return spans.start(); }
public int end() { return spans.end(); } public int end() { return spans.end(); }
// TODO: Remove warning after API has been finalized
public Collection/*<byte[]>*/ getPayload() throws IOException {
return new ArrayList(spans.getPayload());
}
// TODO: Remove warning after API has been finalized
public boolean isPayloadAvailable() {
return spans.isPayloadAvailable();
}
public String toString() { return spans.toString() + "#" + index; } public String toString() { return spans.toString() + "#" + index; }
} }
@ -113,7 +124,7 @@ class NearSpansUnordered implements Spans {
queue = new CellQueue(clauses.length); queue = new CellQueue(clauses.length);
for (int i = 0; i < clauses.length; i++) { for (int i = 0; i < clauses.length; i++) {
SpansCell cell = SpansCell cell =
new SpansCell(clauses[i].getSpans(reader), i); new SpansCell(clauses[i].getPayloadSpans(reader), i);
ordered.add(cell); ordered.add(cell);
} }
} }
@ -197,6 +208,34 @@ class NearSpansUnordered implements Spans {
public int start() { return min().start(); } public int start() { return min().start(); }
public int end() { return max.end(); } public int end() { return max.end(); }
// TODO: Remove warning after API has been finalized
/**
* WARNING: The List is not necessarily in order of the the positions
* @return
* @throws IOException
*/
public Collection/*<byte[]>*/ getPayload() throws IOException {
Set/*<byte[]*/ matchPayload = new HashSet();
for (SpansCell cell = first; cell != null; cell = cell.next) {
if (cell.isPayloadAvailable()) {
matchPayload.addAll(cell.getPayload());
}
}
return matchPayload;
}
// TODO: Remove warning after API has been finalized
public boolean isPayloadAvailable() {
SpansCell pointer = min();
do {
if(pointer.isPayloadAvailable()) {
return true;
}
pointer = pointer.next;
} while(pointer.next != null);
return false;
}
public String toString() { public String toString() {
return getClass().getName() + "("+query.toString()+")@"+ return getClass().getName() + "("+query.toString()+")@"+
@ -214,7 +253,7 @@ class NearSpansUnordered implements Spans {
} }
} }
private void addToList(SpansCell cell) { private void addToList(SpansCell cell) throws IOException {
if (last != null) { // add next to end of list if (last != null) { // add next to end of list
last.next = cell; last.next = cell;
} else } else
@ -230,7 +269,7 @@ class NearSpansUnordered implements Spans {
last.next = null; last.next = null;
} }
private void queueToList() { private void queueToList() throws IOException {
last = first = null; last = first = null;
while (queue.top() != null) { while (queue.top() != null) {
addToList((SpansCell)queue.pop()); addToList((SpansCell)queue.pop());

View File

@ -0,0 +1,69 @@
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.util.Collection;
import java.io.IOException;
/**
* <font color="#FF0000">
* WARNING: The status of the <b>Payloads</b> feature is experimental.
* The APIs introduced here might change in the future and will not be
* supported anymore in such a case.</font>
*
**/
public interface PayloadSpans extends Spans{
/**
* Returns the payload data for the current span.
* This is invalid until {@link #next()} is called for
* the first time.
* This method must not be called more than once after each call
* of {@link #next()}. However, payloads are loaded lazily,
* so if the payload data for the current position is not needed,
* this method may not be called at all for performance reasons.<br>
* <br>
* Note that the return type is a collection, thus the ordering should not be relied upon.
* <br/>
* <p><font color="#FF0000">
* WARNING: The status of the <b>Payloads</b> feature is experimental.
* The APIs introduced here might change in the future and will not be
* supported anymore in such a case.</font>
*
* @return a List of byte arrays containing the data of this payload, otherwise null if isPayloadAvailable is false
* @throws java.io.IOException
*/
// TODO: Remove warning after API has been finalized
Collection/*<byte[]>*/ getPayload() throws IOException;
/**
* Checks if a payload can be loaded at this position.
* <p/>
* Payloads can only be loaded once per call to
* {@link #next()}.
* <p/>
* <p><font color="#FF0000">
* WARNING: The status of the <b>Payloads</b> feature is experimental.
* The APIs introduced here might change in the future and will not be
* supported anymore in such a case.</font>
*
* @return true if there is a payload available at this position that can be loaded
*/
// TODO: Remove warning after API has been finalized
public boolean isPayloadAvailable();
}

View File

@ -21,6 +21,7 @@ import java.io.IOException;
import java.util.Collection; import java.util.Collection;
import java.util.Set; import java.util.Set;
import java.util.ArrayList;
import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexReader;
import org.apache.lucene.search.Query; import org.apache.lucene.search.Query;
@ -65,11 +66,15 @@ public class SpanFirstQuery extends SpanQuery {
public void extractTerms(Set terms) { public void extractTerms(Set terms) {
match.extractTerms(terms); match.extractTerms(terms);
} }
public PayloadSpans getPayloadSpans(IndexReader reader) throws IOException {
return (PayloadSpans) getSpans(reader);
}
public Spans getSpans(final IndexReader reader) throws IOException { public Spans getSpans(final IndexReader reader) throws IOException {
return new Spans() { return new PayloadSpans() {
private Spans spans = match.getSpans(reader); private PayloadSpans spans = match.getPayloadSpans(reader);
public boolean next() throws IOException { public boolean next() throws IOException {
while (spans.next()) { // scan to next match while (spans.next()) { // scan to next match
@ -83,17 +88,29 @@ public class SpanFirstQuery extends SpanQuery {
if (!spans.skipTo(target)) if (!spans.skipTo(target))
return false; return false;
if (spans.end() <= end) // there is a match return spans.end() <= end || next();
return true;
return next(); // scan to next match
} }
public int doc() { return spans.doc(); } public int doc() { return spans.doc(); }
public int start() { return spans.start(); } public int start() { return spans.start(); }
public int end() { return spans.end(); } public int end() { return spans.end(); }
public String toString() { // TODO: Remove warning after API has been finalized
public Collection/*<byte[]>*/ getPayload() throws IOException {
ArrayList result = null;
if (spans.isPayloadAvailable()) {
result = new ArrayList(spans.getPayload());
}
return result;//TODO: any way to avoid the new construction?
}
// TODO: Remove warning after API has been finalized
public boolean isPayloadAvailable() {
return spans.isPayloadAvailable();
}
public String toString() {
return "spans(" + SpanFirstQuery.this.toString() + ")"; return "spans(" + SpanFirstQuery.this.toString() + ")";
} }

View File

@ -120,14 +120,18 @@ public class SpanNearQuery extends SpanQuery {
public Spans getSpans(final IndexReader reader) throws IOException { public Spans getSpans(final IndexReader reader) throws IOException {
if (clauses.size() == 0) // optimize 0-clause case if (clauses.size() == 0) // optimize 0-clause case
return new SpanOrQuery(getClauses()).getSpans(reader); return new SpanOrQuery(getClauses()).getPayloadSpans(reader);
if (clauses.size() == 1) // optimize 1-clause case if (clauses.size() == 1) // optimize 1-clause case
return ((SpanQuery)clauses.get(0)).getSpans(reader); return ((SpanQuery)clauses.get(0)).getPayloadSpans(reader);
return inOrder return inOrder
? (Spans) new NearSpansOrdered(this, reader) ? (PayloadSpans) new NearSpansOrdered(this, reader)
: (Spans) new NearSpansUnordered(this, reader); : (PayloadSpans) new NearSpansUnordered(this, reader);
}
public PayloadSpans getPayloadSpans(IndexReader reader) throws IOException {
return (PayloadSpans) getSpans(reader);
} }
public Query rewrite(IndexReader reader) throws IOException { public Query rewrite(IndexReader reader) throws IOException {

View File

@ -17,15 +17,15 @@ package org.apache.lucene.search.spans;
* limitations under the License. * limitations under the License.
*/ */
import java.io.IOException;
import java.util.Collection;
import java.util.Set;
import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexReader;
import org.apache.lucene.search.Query; import org.apache.lucene.search.Query;
import org.apache.lucene.util.ToStringUtils; import org.apache.lucene.util.ToStringUtils;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Set;
/** Removes matches which overlap with another SpanQuery. */ /** Removes matches which overlap with another SpanQuery. */
public class SpanNotQuery extends SpanQuery { public class SpanNotQuery extends SpanQuery {
private SpanQuery include; private SpanQuery include;
@ -70,8 +70,8 @@ public class SpanNotQuery extends SpanQuery {
public Spans getSpans(final IndexReader reader) throws IOException { public Spans getSpans(final IndexReader reader) throws IOException {
return new Spans() { return new PayloadSpans() {
private Spans includeSpans = include.getSpans(reader); private PayloadSpans includeSpans = include.getPayloadSpans(reader);
private boolean moreInclude = true; private boolean moreInclude = true;
private Spans excludeSpans = exclude.getSpans(reader); private Spans excludeSpans = exclude.getSpans(reader);
@ -131,13 +131,31 @@ public class SpanNotQuery extends SpanQuery {
public int start() { return includeSpans.start(); } public int start() { return includeSpans.start(); }
public int end() { return includeSpans.end(); } public int end() { return includeSpans.end(); }
public String toString() { // TODO: Remove warning after API has been finalizedb
public Collection/*<byte[]>*/ getPayload() throws IOException {
ArrayList result = null;
if (includeSpans.isPayloadAvailable()) {
result = new ArrayList(includeSpans.getPayload());
}
return result;
}
// TODO: Remove warning after API has been finalized
public boolean isPayloadAvailable() {
return includeSpans.isPayloadAvailable();
}
public String toString() {
return "spans(" + SpanNotQuery.this.toString() + ")"; return "spans(" + SpanNotQuery.this.toString() + ")";
} }
}; };
} }
public PayloadSpans getPayloadSpans(IndexReader reader) throws IOException {
return (PayloadSpans) getSpans(reader);
}
public Query rewrite(IndexReader reader) throws IOException { public Query rewrite(IndexReader reader) throws IOException {
SpanNotQuery clone = null; SpanNotQuery clone = null;

View File

@ -154,19 +154,22 @@ public class SpanOrQuery extends SpanQuery {
} }
} }
public PayloadSpans getPayloadSpans(final IndexReader reader) throws IOException {
return (PayloadSpans)getSpans(reader);
}
public Spans getSpans(final IndexReader reader) throws IOException { public Spans getSpans(final IndexReader reader) throws IOException {
if (clauses.size() == 1) // optimize 1-clause case if (clauses.size() == 1) // optimize 1-clause case
return ((SpanQuery)clauses.get(0)).getSpans(reader); return ((SpanQuery)clauses.get(0)).getPayloadSpans(reader);
return new Spans() { return new PayloadSpans() {
private SpanQueue queue = null; private SpanQueue queue = null;
private boolean initSpanQueue(int target) throws IOException { private boolean initSpanQueue(int target) throws IOException {
queue = new SpanQueue(clauses.size()); queue = new SpanQueue(clauses.size());
Iterator i = clauses.iterator(); Iterator i = clauses.iterator();
while (i.hasNext()) { while (i.hasNext()) {
Spans spans = ((SpanQuery)i.next()).getSpans(reader); PayloadSpans spans = ((SpanQuery)i.next()).getPayloadSpans(reader);
if ( ((target == -1) && spans.next()) if ( ((target == -1) && spans.next())
|| ((target != -1) && spans.skipTo(target))) { || ((target != -1) && spans.skipTo(target))) {
queue.put(spans); queue.put(spans);
@ -193,7 +196,7 @@ public class SpanOrQuery extends SpanQuery {
return queue.size() != 0; return queue.size() != 0;
} }
private Spans top() { return (Spans)queue.top(); } private PayloadSpans top() { return (PayloadSpans)queue.top(); }
public boolean skipTo(int target) throws IOException { public boolean skipTo(int target) throws IOException {
if (queue == null) { if (queue == null) {
@ -215,7 +218,23 @@ public class SpanOrQuery extends SpanQuery {
public int start() { return top().start(); } public int start() { return top().start(); }
public int end() { return top().end(); } public int end() { return top().end(); }
public String toString() { // TODO: Remove warning after API has been finalized
public Collection/*<byte[]>*/ getPayload() throws IOException {
ArrayList result = null;
PayloadSpans theTop = top();
if (theTop != null && theTop.isPayloadAvailable()) {
result = new ArrayList(theTop.getPayload());
}
return result;
}
// TODO: Remove warning after API has been finalized
public boolean isPayloadAvailable() {
PayloadSpans top = top();
return top != null && top.isPayloadAvailable();
}
public String toString() {
return "spans("+SpanOrQuery.this+")@"+ return "spans("+SpanOrQuery.this+")@"+
((queue == null)?"START" ((queue == null)?"START"
:(queue.size()>0?(doc()+":"+start()+"-"+end()):"END")); :(queue.size()>0?(doc()+":"+start()+"-"+end()):"END"));

View File

@ -17,15 +17,14 @@ package org.apache.lucene.search.spans;
* limitations under the License. * limitations under the License.
*/ */
import java.io.IOException;
import java.util.Collection;
import java.util.Set;
import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexReader;
import org.apache.lucene.search.Query; import org.apache.lucene.search.Query;
import org.apache.lucene.search.Weight;
import org.apache.lucene.search.Searcher; import org.apache.lucene.search.Searcher;
import org.apache.lucene.search.Weight;
import java.io.IOException;
import java.util.Collection;
import java.util.Set;
/** Base class for span-based queries. */ /** Base class for span-based queries. */
public abstract class SpanQuery extends Query { public abstract class SpanQuery extends Query {
@ -33,6 +32,22 @@ public abstract class SpanQuery extends Query {
* to search for spans. */ * to search for spans. */
public abstract Spans getSpans(IndexReader reader) throws IOException; public abstract Spans getSpans(IndexReader reader) throws IOException;
/**
* Returns the matches for this query in an index, including access to any {@link org.apache.lucene.index.Payload}s at those
* positions. Implementing classes that want access to the payloads will need to implement this.
* @param reader The {@link org.apache.lucene.index.IndexReader} to use to get spans/payloads
* @return null
* @throws IOException if there is an error accessing the payload
*
* <font color="#FF0000">
* WARNING: The status of the <b>Payloads</b> feature is experimental.
* The APIs introduced here might change in the future and will not be
* supported anymore in such a case.</font>
*/
public PayloadSpans getPayloadSpans(IndexReader reader) throws IOException{
return null;
};
/** Returns the name of the field matched by this query.*/ /** Returns the name of the field matched by this query.*/
public abstract String getField(); public abstract String getField();

View File

@ -79,4 +79,9 @@ public class SpanTermQuery extends SpanQuery {
return new TermSpans(reader.termPositions(term), term); return new TermSpans(reader.termPositions(term), term);
} }
public PayloadSpans getPayloadSpans(IndexReader reader) throws IOException {
return (PayloadSpans) getSpans(reader);
}
} }

View File

@ -20,12 +20,14 @@ import org.apache.lucene.index.Term;
import org.apache.lucene.index.TermPositions; import org.apache.lucene.index.TermPositions;
import java.io.IOException; import java.io.IOException;
import java.util.Collections;
import java.util.Collection;
/** /**
* Expert: * Expert:
* Public for extension only * Public for extension only
*/ */
public class TermSpans implements Spans { public class TermSpans implements PayloadSpans {
protected TermPositions positions; protected TermPositions positions;
protected Term term; protected Term term;
protected int doc; protected int doc;
@ -89,6 +91,18 @@ public class TermSpans implements Spans {
return position + 1; return position + 1;
} }
// TODO: Remove warning after API has been finalized
public Collection/*<byte[]>*/ getPayload() throws IOException {
byte [] bytes = new byte[positions.getPayloadLength()];
bytes = positions.getPayload(bytes, 0);
return Collections.singletonList(bytes);
}
// TODO: Remove warning after API has been finalized
public boolean isPayloadAvailable() {
return positions.isPayloadAvailable();
}
public String toString() { public String toString() {
return "spans(" + term.toString() + ")@" + return "spans(" + term.toString() + ")@" +
(doc == -1 ? "START" : (doc == Integer.MAX_VALUE) ? "END" : doc + "-" + position); (doc == -1 ? "START" : (doc == Integer.MAX_VALUE) ? "END" : doc + "-" + position);

View File

@ -0,0 +1,104 @@
package org.apache.lucene.search.payloads;
import org.apache.lucene.analysis.*;
import org.apache.lucene.index.Payload;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.store.RAMDirectory;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.util.English;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Similarity;
import java.io.Reader;
import java.io.IOException;
/**
*
*
**/
public class PayloadHelper {
private byte[] payloadField = new byte[]{1};
private byte[] payloadMultiField1 = new byte[]{2};
private byte[] payloadMultiField2 = new byte[]{4};
public static final String NO_PAYLOAD_FIELD = "noPayloadField";
public static final String MULTI_FIELD = "multiField";
public static final String FIELD = "field";
public class PayloadAnalyzer extends Analyzer {
public TokenStream tokenStream(String fieldName, Reader reader) {
TokenStream result = new LowerCaseTokenizer(reader);
result = new PayloadFilter(result, fieldName);
return result;
}
}
public class PayloadFilter extends TokenFilter {
String fieldName;
int numSeen = 0;
public PayloadFilter(TokenStream input, String fieldName) {
super(input);
this.fieldName = fieldName;
}
public Token next() throws IOException {
Token result = input.next();
if (result != null) {
if (fieldName.equals(FIELD))
{
result.setPayload(new Payload(payloadField));
}
else if (fieldName.equals(MULTI_FIELD))
{
if (numSeen % 2 == 0)
{
result.setPayload(new Payload(payloadMultiField1));
}
else
{
result.setPayload(new Payload(payloadMultiField2));
}
numSeen++;
}
}
return result;
}
}
/**
* Sets up a RAMDirectory, and adds documents (using English.intToEnglish()) with two fields: field and multiField
* and analyzes them using the PayloadAnalyzer
* @param similarity The Similarity class to use in the Searcher
* @param numDocs The num docs to add
* @return An IndexSearcher
* @throws IOException
*/
public IndexSearcher setUp(Similarity similarity, int numDocs) throws IOException {
RAMDirectory directory = new RAMDirectory();
PayloadAnalyzer analyzer = new PayloadAnalyzer();
IndexWriter writer
= new IndexWriter(directory, analyzer, true);
writer.setSimilarity(similarity);
//writer.infoStream = System.out;
for (int i = 0; i < numDocs; i++) {
Document doc = new Document();
doc.add(new Field(FIELD, English.intToEnglish(i), Field.Store.YES, Field.Index.TOKENIZED));
doc.add(new Field(MULTI_FIELD, English.intToEnglish(i) + " " + English.intToEnglish(i), Field.Store.YES, Field.Index.TOKENIZED));
doc.add(new Field(NO_PAYLOAD_FIELD, English.intToEnglish(i), Field.Store.YES, Field.Index.TOKENIZED));
writer.addDocument(doc);
}
//writer.optimize();
writer.close();
IndexSearcher searcher = new IndexSearcher(directory);
searcher.setSimilarity(similarity);
return searcher;
}
}

View File

@ -102,7 +102,7 @@ public class TestBoostingTermQuery extends LuceneTestCase {
//writer.infoStream = System.out; //writer.infoStream = System.out;
for (int i = 0; i < 1000; i++) { for (int i = 0; i < 1000; i++) {
Document doc = new Document(); Document doc = new Document();
Field noPayloadField = new Field("noPayLoad", English.intToEnglish(i), Field.Store.YES, Field.Index.TOKENIZED); Field noPayloadField = new Field(PayloadHelper.NO_PAYLOAD_FIELD, English.intToEnglish(i), Field.Store.YES, Field.Index.TOKENIZED);
//noPayloadField.setBoost(0); //noPayloadField.setBoost(0);
doc.add(noPayloadField); doc.add(noPayloadField);
doc.add(new Field("field", English.intToEnglish(i), Field.Store.YES, Field.Index.TOKENIZED)); doc.add(new Field("field", English.intToEnglish(i), Field.Store.YES, Field.Index.TOKENIZED));
@ -130,7 +130,7 @@ public class TestBoostingTermQuery extends LuceneTestCase {
ScoreDoc doc = hits.scoreDocs[i]; ScoreDoc doc = hits.scoreDocs[i];
assertTrue(doc.score + " does not equal: " + 1, doc.score == 1); assertTrue(doc.score + " does not equal: " + 1, doc.score == 1);
} }
CheckHits.checkExplanations(query, "field", searcher, true); CheckHits.checkExplanations(query, PayloadHelper.FIELD, searcher, true);
Spans spans = query.getSpans(searcher.getIndexReader()); Spans spans = query.getSpans(searcher.getIndexReader());
assertTrue("spans is null and it shouldn't be", spans != null); assertTrue("spans is null and it shouldn't be", spans != null);
assertTrue("spans is not an instanceof " + TermSpans.class, spans instanceof TermSpans); assertTrue("spans is not an instanceof " + TermSpans.class, spans instanceof TermSpans);
@ -143,7 +143,7 @@ public class TestBoostingTermQuery extends LuceneTestCase {
} }
public void testMultipleMatchesPerDoc() throws Exception { public void testMultipleMatchesPerDoc() throws Exception {
BoostingTermQuery query = new BoostingTermQuery(new Term("multiField", "seventy")); BoostingTermQuery query = new BoostingTermQuery(new Term(PayloadHelper.MULTI_FIELD, "seventy"));
TopDocs hits = searcher.search(query, null, 100); TopDocs hits = searcher.search(query, null, 100);
assertTrue("hits is null and it shouldn't be", hits != null); assertTrue("hits is null and it shouldn't be", hits != null);
assertTrue("hits Size: " + hits.totalHits + " is not: " + 100, hits.totalHits == 100); assertTrue("hits Size: " + hits.totalHits + " is not: " + 100, hits.totalHits == 100);
@ -180,7 +180,7 @@ public class TestBoostingTermQuery extends LuceneTestCase {
} }
public void testNoMatch() throws Exception { public void testNoMatch() throws Exception {
BoostingTermQuery query = new BoostingTermQuery(new Term("field", "junk")); BoostingTermQuery query = new BoostingTermQuery(new Term(PayloadHelper.FIELD, "junk"));
TopDocs hits = searcher.search(query, null, 100); TopDocs hits = searcher.search(query, null, 100);
assertTrue("hits is null and it shouldn't be", hits != null); assertTrue("hits is null and it shouldn't be", hits != null);
assertTrue("hits Size: " + hits.totalHits + " is not: " + 0, hits.totalHits == 0); assertTrue("hits Size: " + hits.totalHits + " is not: " + 0, hits.totalHits == 0);
@ -188,8 +188,8 @@ public class TestBoostingTermQuery extends LuceneTestCase {
} }
public void testNoPayload() throws Exception { public void testNoPayload() throws Exception {
BoostingTermQuery q1 = new BoostingTermQuery(new Term("noPayLoad", "zero")); BoostingTermQuery q1 = new BoostingTermQuery(new Term(PayloadHelper.NO_PAYLOAD_FIELD, "zero"));
BoostingTermQuery q2 = new BoostingTermQuery(new Term("noPayLoad", "foo")); BoostingTermQuery q2 = new BoostingTermQuery(new Term(PayloadHelper.NO_PAYLOAD_FIELD, "foo"));
BooleanClause c1 = new BooleanClause(q1, BooleanClause.Occur.MUST); BooleanClause c1 = new BooleanClause(q1, BooleanClause.Occur.MUST);
BooleanClause c2 = new BooleanClause(q2, BooleanClause.Occur.MUST_NOT); BooleanClause c2 = new BooleanClause(q2, BooleanClause.Occur.MUST_NOT);
BooleanQuery query = new BooleanQuery(); BooleanQuery query = new BooleanQuery();
@ -200,7 +200,7 @@ public class TestBoostingTermQuery extends LuceneTestCase {
assertTrue("hits Size: " + hits.totalHits + " is not: " + 1, hits.totalHits == 1); assertTrue("hits Size: " + hits.totalHits + " is not: " + 1, hits.totalHits == 1);
int[] results = new int[1]; int[] results = new int[1];
results[0] = 0;//hits.scoreDocs[0].doc; results[0] = 0;//hits.scoreDocs[0].doc;
CheckHits.checkHitCollector(query, "noPayLoad", searcher, results); CheckHits.checkHitCollector(query, PayloadHelper.NO_PAYLOAD_FIELD, searcher, results);
} }
// must be static for weight serialization tests // must be static for weight serialization tests

View File

@ -0,0 +1,377 @@
package org.apache.lucene.search.spans;
/**
* Copyright 2004 The Apache Software Foundation
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import java.io.Reader;
import java.util.Collection;
import java.util.HashSet;
import java.util.Iterator;
import java.util.Set;
import junit.framework.TestCase;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.LowerCaseTokenizer;
import org.apache.lucene.analysis.Token;
import org.apache.lucene.analysis.TokenFilter;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.Payload;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.DefaultSimilarity;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Similarity;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.payloads.PayloadHelper;
import org.apache.lucene.search.payloads.PayloadSpanUtil;
import org.apache.lucene.store.RAMDirectory;
public class TestPayloadSpans extends TestCase {
private final static boolean DEBUG = false;
private IndexSearcher searcher;
private Similarity similarity = new DefaultSimilarity();
protected IndexReader indexReader;
public TestPayloadSpans(String s) {
super(s);
}
protected void setUp() throws IOException {
PayloadHelper helper = new PayloadHelper();
searcher = helper.setUp(similarity, 1000);
indexReader = searcher.getIndexReader();
}
protected void tearDown() {
}
public void testSpanTermQuery() throws Exception {
SpanTermQuery stq;
PayloadSpans spans;
stq = new SpanTermQuery(new Term(PayloadHelper.FIELD, "seventy"));
spans = stq.getPayloadSpans(indexReader);
assertTrue("spans is null and it shouldn't be", spans != null);
checkSpans(spans, 100, 1, 1, 1);
stq = new SpanTermQuery(new Term(PayloadHelper.NO_PAYLOAD_FIELD, "seventy"));
spans = stq.getPayloadSpans(indexReader);
assertTrue("spans is null and it shouldn't be", spans != null);
checkSpans(spans, 100, 0, 0, 0);
}
public void testSpanFirst() throws IOException {
SpanQuery match;
SpanFirstQuery sfq;
match = new SpanTermQuery(new Term(PayloadHelper.FIELD, "one"));
sfq = new SpanFirstQuery(match, 2);
PayloadSpans spans = sfq.getPayloadSpans(indexReader);
checkSpans(spans, 109, 1, 1, 1);
//Test more complicated subclause
SpanQuery[] clauses = new SpanQuery[2];
clauses[0] = new SpanTermQuery(new Term(PayloadHelper.FIELD, "one"));
clauses[1] = new SpanTermQuery(new Term(PayloadHelper.FIELD, "hundred"));
match = new SpanNearQuery(clauses, 0, true);
sfq = new SpanFirstQuery(match, 2);
checkSpans(sfq.getPayloadSpans(indexReader), 100, 2, 1, 1);
match = new SpanNearQuery(clauses, 0, false);
sfq = new SpanFirstQuery(match, 2);
checkSpans(sfq.getPayloadSpans(indexReader), 100, 2, 1, 1);
}
public void testNestedSpans() throws Exception {
SpanTermQuery stq;
PayloadSpans spans;
IndexSearcher searcher = getSearcher();
stq = new SpanTermQuery(new Term(PayloadHelper.FIELD, "mark"));
spans = stq.getPayloadSpans(searcher.getIndexReader());
assertTrue("spans is null and it shouldn't be", spans != null);
checkSpans(spans, 0, null);
SpanQuery[] clauses = new SpanQuery[3];
clauses[0] = new SpanTermQuery(new Term(PayloadHelper.FIELD, "rr"));
clauses[1] = new SpanTermQuery(new Term(PayloadHelper.FIELD, "yy"));
clauses[2] = new SpanTermQuery(new Term(PayloadHelper.FIELD, "xx"));
SpanNearQuery spanNearQuery = new SpanNearQuery(clauses, 12, false);
spans = spanNearQuery.getPayloadSpans(searcher.getIndexReader());
assertTrue("spans is null and it shouldn't be", spans != null);
checkSpans(spans, 2, new int[]{3,3});
clauses[0] = new SpanTermQuery(new Term(PayloadHelper.FIELD, "xx"));
clauses[1] = new SpanTermQuery(new Term(PayloadHelper.FIELD, "rr"));
clauses[2] = new SpanTermQuery(new Term(PayloadHelper.FIELD, "yy"));
spanNearQuery = new SpanNearQuery(clauses, 6, true);
spans = spanNearQuery.getPayloadSpans(searcher.getIndexReader());
assertTrue("spans is null and it shouldn't be", spans != null);
checkSpans(spans, 1, new int[]{3});
clauses = new SpanQuery[2];
clauses[0] = new SpanTermQuery(new Term(PayloadHelper.FIELD, "xx"));
clauses[1] = new SpanTermQuery(new Term(PayloadHelper.FIELD, "rr"));
spanNearQuery = new SpanNearQuery(clauses, 6, true);
SpanQuery[] clauses2 = new SpanQuery[2];
clauses2[0] = new SpanTermQuery(new Term(PayloadHelper.FIELD, "yy"));
clauses2[1] = spanNearQuery;
SpanNearQuery nestedSpanNearQuery = new SpanNearQuery(clauses2, 6, false);
spans = nestedSpanNearQuery.getPayloadSpans(searcher.getIndexReader());
assertTrue("spans is null and it shouldn't be", spans != null);
checkSpans(spans, 2, new int[]{3,3});
}
public void testFirstClauseWithoutPayload() throws Exception {
PayloadSpans spans;
IndexSearcher searcher = getSearcher();
SpanQuery[] clauses = new SpanQuery[3];
clauses[0] = new SpanTermQuery(new Term(PayloadHelper.FIELD, "nopayload"));
clauses[1] = new SpanTermQuery(new Term(PayloadHelper.FIELD, "qq"));
clauses[2] = new SpanTermQuery(new Term(PayloadHelper.FIELD, "ss"));
SpanNearQuery spanNearQuery = new SpanNearQuery(clauses, 6, true);
SpanQuery[] clauses2 = new SpanQuery[2];
clauses2[0] = new SpanTermQuery(new Term(PayloadHelper.FIELD, "pp"));
clauses2[1] = spanNearQuery;
SpanNearQuery snq = new SpanNearQuery(clauses2, 6, false);
SpanQuery[] clauses3 = new SpanQuery[2];
clauses3[0] = new SpanTermQuery(new Term(PayloadHelper.FIELD, "np"));
clauses3[1] = snq;
SpanNearQuery nestedSpanNearQuery = new SpanNearQuery(clauses3, 6, false);
spans = nestedSpanNearQuery.getPayloadSpans(searcher.getIndexReader());
assertTrue("spans is null and it shouldn't be", spans != null);
checkSpans(spans, 1, new int[]{3});
}
public void testHeavilyNestedSpanQuery() throws Exception {
PayloadSpans spans;
IndexSearcher searcher = getSearcher();
SpanQuery[] clauses = new SpanQuery[3];
clauses[0] = new SpanTermQuery(new Term(PayloadHelper.FIELD, "one"));
clauses[1] = new SpanTermQuery(new Term(PayloadHelper.FIELD, "two"));
clauses[2] = new SpanTermQuery(new Term(PayloadHelper.FIELD, "three"));
SpanNearQuery spanNearQuery = new SpanNearQuery(clauses, 5, true);
clauses = new SpanQuery[3];
clauses[0] = spanNearQuery;
clauses[1] = new SpanTermQuery(new Term(PayloadHelper.FIELD, "five"));
clauses[2] = new SpanTermQuery(new Term(PayloadHelper.FIELD, "six"));
SpanNearQuery spanNearQuery2 = new SpanNearQuery(clauses, 6, true);
SpanQuery[] clauses2 = new SpanQuery[2];
clauses2[0] = new SpanTermQuery(new Term(PayloadHelper.FIELD, "eleven"));
clauses2[1] = new SpanTermQuery(new Term(PayloadHelper.FIELD, "ten"));
SpanNearQuery spanNearQuery3 = new SpanNearQuery(clauses2, 2, false);
SpanQuery[] clauses3 = new SpanQuery[3];
clauses3[0] = new SpanTermQuery(new Term(PayloadHelper.FIELD, "nine"));
clauses3[1] = spanNearQuery2;
clauses3[2] = spanNearQuery3;
SpanNearQuery nestedSpanNearQuery = new SpanNearQuery(clauses3, 6, false);
spans = nestedSpanNearQuery.getPayloadSpans(searcher.getIndexReader());
assertTrue("spans is null and it shouldn't be", spans != null);
checkSpans(spans, 2, new int[]{8, 8});
}
public void testPayloadSpanUtil() throws Exception {
RAMDirectory directory = new RAMDirectory();
PayloadAnalyzer analyzer = new PayloadAnalyzer();
String[] docs = new String[]{};
IndexWriter writer = new IndexWriter(directory, analyzer, true);
writer.setSimilarity(similarity);
Document doc = new Document();
doc.add(new Field(PayloadHelper.FIELD,"xx rr yy mm pp", Field.Store.YES, Field.Index.TOKENIZED));
writer.addDocument(doc);
writer.close();
IndexSearcher searcher = new IndexSearcher(directory);
IndexReader reader = searcher.getIndexReader();
PayloadSpanUtil psu = new PayloadSpanUtil(reader);
Collection payloads = psu.getPayloadsForQuery(new TermQuery(new Term(PayloadHelper.FIELD, "rr")));
if(DEBUG)
System.out.println("Num payloads:" + payloads.size());
Iterator it = payloads.iterator();
while(it.hasNext()) {
byte[] bytes = (byte[]) it.next();
if(DEBUG)
System.out.println(new String(bytes));
}
}
private void checkSpans(PayloadSpans spans, int expectedNumSpans, int expectedNumPayloads,
int expectedPayloadLength, int expectedFirstByte) throws IOException {
assertTrue("spans is null and it shouldn't be", spans != null);
//each position match should have a span associated with it, since there is just one underlying term query, there should
//only be one entry in the span
int seen = 0;
while (spans.next() == true)
{
//if we expect payloads, then isPayloadAvailable should be true
if (expectedNumPayloads > 0) {
assertTrue("isPayloadAvailable is not returning the correct value: " + spans.isPayloadAvailable()
+ " and it should be: " + (expectedNumPayloads > 0),
spans.isPayloadAvailable() == true);
} else {
assertTrue("isPayloadAvailable should be false", spans.isPayloadAvailable() == false);
}
//See payload helper, for the PayloadHelper.FIELD field, there is a single byte payload at every token
if (spans.isPayloadAvailable()) {
Collection payload = spans.getPayload();
assertTrue("payload Size: " + payload.size() + " is not: " + expectedNumPayloads, payload.size() == expectedNumPayloads);
for (Iterator iterator = payload.iterator(); iterator.hasNext();) {
byte[] thePayload = (byte[]) iterator.next();
assertTrue("payload[0] Size: " + thePayload.length + " is not: " + expectedPayloadLength,
thePayload.length == expectedPayloadLength);
assertTrue(thePayload[0] + " does not equal: " + expectedFirstByte, thePayload[0] == expectedFirstByte);
}
}
seen++;
}
assertTrue(seen + " does not equal: " + expectedNumSpans, seen == expectedNumSpans);
}
private IndexSearcher getSearcher() throws Exception {
RAMDirectory directory = new RAMDirectory();
PayloadAnalyzer analyzer = new PayloadAnalyzer();
String[] docs = new String[]{"xx rr yy mm pp","xx yy mm rr pp", "nopayload qq ss pp np", "one two three four five six seven eight nine ten eleven", "nine one two three four five six seven eight eleven ten"};
IndexWriter writer = new IndexWriter(directory, analyzer, true);
writer.setSimilarity(similarity);
Document doc = null;
for(int i = 0; i < docs.length; i++) {
doc = new Document();
String docText = docs[i];
doc.add(new Field(PayloadHelper.FIELD,docText, Field.Store.YES, Field.Index.TOKENIZED));
writer.addDocument(doc);
}
writer.close();
IndexSearcher searcher = new IndexSearcher(directory);
return searcher;
}
private void checkSpans(PayloadSpans spans, int numSpans, int[] numPayloads) throws IOException {
int cnt = 0;
while (spans.next() == true) {
if(DEBUG)
System.out.println("\nSpans Dump --");
if (spans.isPayloadAvailable()) {
Collection payload = spans.getPayload();
if(DEBUG)
System.out.println("payloads for span:" + payload.size());
Iterator it = payload.iterator();
while(it.hasNext()) {
byte[] bytes = (byte[]) it.next();
if(DEBUG)
System.out.println("doc:" + spans.doc() + " s:" + spans.start() + " e:" + spans.end() + " "
+ new String(bytes));
}
assertEquals(numPayloads[cnt],payload.size());
} else {
assertFalse("Expected spans:" + numPayloads[cnt] + " found: 0",numPayloads.length > 0 && numPayloads[cnt] > 0 );
}
cnt++;
}
assertEquals(numSpans, cnt);
}
class PayloadAnalyzer extends Analyzer {
public TokenStream tokenStream(String fieldName, Reader reader) {
TokenStream result = new LowerCaseTokenizer(reader);
result = new PayloadFilter(result, fieldName);
return result;
}
}
class PayloadFilter extends TokenFilter {
String fieldName;
int numSeen = 0;
Set entities = new HashSet();
Set nopayload = new HashSet();
int pos;
public PayloadFilter(TokenStream input, String fieldName) {
super(input);
this.fieldName = fieldName;
pos = 0;
entities.add("xx");
entities.add("one");
nopayload.add("nopayload");
nopayload.add("np");
}
public Token next() throws IOException {
Token result = input.next();
if (result != null) {
String token = new String(result.termBuffer(), 0, result.termLength());
if (!nopayload.contains(token)) {
if (entities.contains(token)) {
result.setPayload(new Payload((token + ":Entity:"+ pos ).getBytes()));
} else {
result.setPayload(new Payload((token + ":Noise:" + pos ).getBytes()));
}
}
pos += result.getPositionIncrement();
}
return result;
}
}
}

View File

@ -46,7 +46,6 @@ public class TestSpans extends LuceneTestCase {
} }
writer.close(); writer.close();
searcher = new IndexSearcher(directory); searcher = new IndexSearcher(directory);
//System.out.println("set up " + getName());
} }
private String[] docFields = { private String[] docFields = {
@ -192,6 +191,105 @@ public class TestSpans extends LuceneTestCase {
} }
public void testSpanNearUnOrdered() throws Exception {
//See http://www.gossamer-threads.com/lists/lucene/java-dev/52270 for discussion about this test
SpanNearQuery snq;
snq = new SpanNearQuery(
new SpanQuery[] {
makeSpanTermQuery("u1"),
makeSpanTermQuery("u2") },
0,
false);
Spans spans = snq.getSpans(searcher.getIndexReader());
assertTrue("Does not have next and it should", spans.next());
assertEquals("doc", 4, spans.doc());
assertEquals("start", 1, spans.start());
assertEquals("end", 3, spans.end());
assertTrue("Does not have next and it should", spans.next());
assertEquals("doc", 5, spans.doc());
assertEquals("start", 2, spans.start());
assertEquals("end", 4, spans.end());
assertTrue("Does not have next and it should", spans.next());
assertEquals("doc", 8, spans.doc());
assertEquals("start", 2, spans.start());
assertEquals("end", 4, spans.end());
assertTrue("Does not have next and it should", spans.next());
assertEquals("doc", 9, spans.doc());
assertEquals("start", 0, spans.start());
assertEquals("end", 2, spans.end());
assertTrue("Does not have next and it should", spans.next());
assertEquals("doc", 10, spans.doc());
assertEquals("start", 0, spans.start());
assertEquals("end", 2, spans.end());
assertTrue("Has next and it shouldn't: " + spans.doc(), spans.next() == false);
SpanNearQuery u1u2 = new SpanNearQuery(new SpanQuery[]{makeSpanTermQuery("u1"),
makeSpanTermQuery("u2")}, 0, false);
snq = new SpanNearQuery(
new SpanQuery[] {
u1u2,
makeSpanTermQuery("u2")
},
1,
false);
spans = snq.getSpans(searcher.getIndexReader());
assertTrue("Does not have next and it should", spans.next());
assertEquals("doc", 4, spans.doc());
assertEquals("start", 0, spans.start());
assertEquals("end", 3, spans.end());
assertTrue("Does not have next and it should", spans.next());
//unordered spans can be subsets
assertEquals("doc", 4, spans.doc());
assertEquals("start", 1, spans.start());
assertEquals("end", 3, spans.end());
assertTrue("Does not have next and it should", spans.next());
assertEquals("doc", 5, spans.doc());
assertEquals("start", 0, spans.start());
assertEquals("end", 4, spans.end());
assertTrue("Does not have next and it should", spans.next());
assertEquals("doc", 5, spans.doc());
assertEquals("start", 2, spans.start());
assertEquals("end", 4, spans.end());
assertTrue("Does not have next and it should", spans.next());
assertEquals("doc", 8, spans.doc());
assertEquals("start", 0, spans.start());
assertEquals("end", 4, spans.end());
assertTrue("Does not have next and it should", spans.next());
assertEquals("doc", 8, spans.doc());
assertEquals("start", 2, spans.start());
assertEquals("end", 4, spans.end());
assertTrue("Does not have next and it should", spans.next());
assertEquals("doc", 9, spans.doc());
assertEquals("start", 0, spans.start());
assertEquals("end", 2, spans.end());
assertTrue("Does not have next and it should", spans.next());
assertEquals("doc", 9, spans.doc());
assertEquals("start", 0, spans.start());
assertEquals("end", 4, spans.end());
assertTrue("Does not have next and it should", spans.next());
assertEquals("doc", 10, spans.doc());
assertEquals("start", 0, spans.start());
assertEquals("end", 2, spans.end());
assertTrue("Has next and it shouldn't", spans.next() == false);
}
private Spans orSpans(String[] terms) throws Exception { private Spans orSpans(String[] terms) throws Exception {
SpanQuery[] sqa = new SpanQuery[terms.length]; SpanQuery[] sqa = new SpanQuery[terms.length];
for (int i = 0; i < terms.length; i++) { for (int i = 0; i < terms.length; i++) {