mirror of https://github.com/apache/lucene.git
LUCENE-7438: New UnifiedHighlighter
(cherry picked from commit 722e827
)
This commit is contained in:
parent
f9e915b3da
commit
4b6794368d
|
@ -9,6 +9,12 @@ API Changes
|
|||
|
||||
New Features
|
||||
|
||||
* LUCENE-7438: New "UnifiedHighlighter" derivative of the PostingsHighlighter that
|
||||
can consume offsets from postings, term vectors, or analysis. It can highlight phrases
|
||||
as accurately as the standard Highlighter. Light term vectors can be used with offsets
|
||||
in postings for fast wildcard (MultiTermQuery) highlighting.
|
||||
(David Smiley, Timothy Rodriguez)
|
||||
|
||||
Bug Fixes
|
||||
|
||||
* LUCENE-7472: MultiFieldQueryParser.getFieldQuery() drops queries that are
|
||||
|
|
|
@ -0,0 +1,190 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.search.uhighlight;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.FilteringTokenFilter;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.memory.MemoryIndex;
|
||||
import org.apache.lucene.search.spans.SpanQuery;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.automaton.Automata;
|
||||
import org.apache.lucene.util.automaton.CharacterRunAutomaton;
|
||||
|
||||
|
||||
/**
|
||||
* Uses an {@link Analyzer} on content to get offsets. It may use a {@link MemoryIndex} too.
|
||||
*
|
||||
* @lucene.internal
|
||||
*/
|
||||
public class AnalysisOffsetStrategy extends FieldOffsetStrategy {
|
||||
|
||||
//TODO: Consider splitting this highlighter into a MemoryIndexFieldHighlighter and a TokenStreamFieldHighlighter
|
||||
private static final BytesRef[] ZERO_LEN_BYTES_REF_ARRAY = new BytesRef[0];
|
||||
private final Analyzer analyzer;
|
||||
private final MemoryIndex memoryIndex;
|
||||
private final LeafReader leafReader;
|
||||
private final CharacterRunAutomaton preMemIndexFilterAutomaton;
|
||||
|
||||
public AnalysisOffsetStrategy(String field, BytesRef[] extractedTerms, PhraseHelper phraseHelper, CharacterRunAutomaton[] automata, Analyzer analyzer) {
|
||||
super(field, extractedTerms, phraseHelper, automata);
|
||||
this.analyzer = analyzer;
|
||||
// Automata (Wildcards / MultiTermQuery):
|
||||
this.automata = automata;
|
||||
|
||||
if (terms.length > 0 && !strictPhrases.hasPositionSensitivity()) {
|
||||
this.automata = convertTermsToAutomata(terms, automata);
|
||||
// clear the terms array now that we've moved them to be expressed as automata
|
||||
terms = ZERO_LEN_BYTES_REF_ARRAY;
|
||||
}
|
||||
|
||||
if (terms.length > 0 || strictPhrases.willRewrite()) { //needs MemoryIndex
|
||||
// init MemoryIndex
|
||||
boolean storePayloads = strictPhrases.hasPositionSensitivity(); // might be needed
|
||||
memoryIndex = new MemoryIndex(true, storePayloads);//true==store offsets
|
||||
leafReader = (LeafReader) memoryIndex.createSearcher().getIndexReader();
|
||||
// preFilter for MemoryIndex
|
||||
preMemIndexFilterAutomaton = buildCombinedAutomaton(field, terms, this.automata, strictPhrases);
|
||||
} else {
|
||||
memoryIndex = null;
|
||||
leafReader = null;
|
||||
preMemIndexFilterAutomaton = null;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public UnifiedHighlighter.OffsetSource getOffsetSource() {
|
||||
return UnifiedHighlighter.OffsetSource.ANALYSIS;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<OffsetsEnum> getOffsetsEnums(IndexReader reader, int docId, String content) throws IOException {
|
||||
// note: don't need LimitTokenOffsetFilter since content is already truncated to maxLength
|
||||
TokenStream tokenStream = tokenStream(content);
|
||||
|
||||
if (memoryIndex != null) { // also handles automata.length > 0
|
||||
// We use a MemoryIndex and index the tokenStream so that later we have the PostingsEnum with offsets.
|
||||
|
||||
// note: An *alternative* strategy is to get PostingsEnums without offsets from the main index
|
||||
// and then marry this up with a fake PostingsEnum backed by a TokenStream (which has the offsets) and
|
||||
// can use that to filter applicable tokens? It would have the advantage of being able to exit
|
||||
// early and save some re-analysis. This would be an additional method/offset-source approach
|
||||
// since it's still useful to highlight without any index (so we build MemoryIndex).
|
||||
|
||||
// note: probably unwise to re-use TermsEnum on reset mem index so we don't. But we do re-use the
|
||||
// leaf reader, which is a bit more top level than in the guts.
|
||||
memoryIndex.reset();
|
||||
|
||||
// Filter the tokenStream to applicable terms
|
||||
if (preMemIndexFilterAutomaton != null) {
|
||||
tokenStream = newKeepWordFilter(tokenStream, preMemIndexFilterAutomaton);
|
||||
}
|
||||
memoryIndex.addField(field, tokenStream);//note: calls tokenStream.reset() & close()
|
||||
tokenStream = null; // it's consumed; done.
|
||||
docId = 0;
|
||||
|
||||
if (automata.length > 0) {
|
||||
Terms foundTerms = leafReader.terms(field);
|
||||
if (foundTerms == null) {
|
||||
return Collections.emptyList(); //No offsets for this field.
|
||||
}
|
||||
// Un-invert for the automata. Much more compact than a CachingTokenStream
|
||||
tokenStream = MultiTermHighlighting.uninvertAndFilterTerms(foundTerms, 0, automata, content.length());
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
return createOffsetsEnums(leafReader, docId, tokenStream);
|
||||
}
|
||||
|
||||
protected TokenStream tokenStream(String content) throws IOException {
|
||||
return MultiValueTokenStream.wrap(field, analyzer, content, UnifiedHighlighter.MULTIVAL_SEP_CHAR);
|
||||
}
|
||||
|
||||
private static CharacterRunAutomaton[] convertTermsToAutomata(BytesRef[] terms, CharacterRunAutomaton[] automata) {
|
||||
CharacterRunAutomaton[] newAutomata = new CharacterRunAutomaton[terms.length + automata.length];
|
||||
for (int i = 0; i < terms.length; i++) {
|
||||
newAutomata[i] = MultiTermHighlighting.makeStringMatchAutomata(terms[i]);
|
||||
}
|
||||
// Append existing automata (that which is used for MTQs)
|
||||
System.arraycopy(automata, 0, newAutomata, terms.length, automata.length);
|
||||
return newAutomata;
|
||||
}
|
||||
|
||||
private static FilteringTokenFilter newKeepWordFilter(final TokenStream tokenStream,
|
||||
final CharacterRunAutomaton charRunAutomaton) {
|
||||
// it'd be nice to use KeepWordFilter but it demands a CharArraySet. TODO File JIRA? Need a new interface?
|
||||
return new FilteringTokenFilter(tokenStream) {
|
||||
final CharTermAttribute charAtt = addAttribute(CharTermAttribute.class);
|
||||
|
||||
@Override
|
||||
protected boolean accept() throws IOException {
|
||||
return charRunAutomaton.run(charAtt.buffer(), 0, charAtt.length());
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Build one {@link CharacterRunAutomaton} matching any term the query might match.
|
||||
*/
|
||||
private static CharacterRunAutomaton buildCombinedAutomaton(String field, BytesRef[] terms,
|
||||
CharacterRunAutomaton[] automata,
|
||||
PhraseHelper strictPhrases) {
|
||||
List<CharacterRunAutomaton> allAutomata = new ArrayList<>();
|
||||
if (terms.length > 0) {
|
||||
allAutomata.add(new CharacterRunAutomaton(Automata.makeStringUnion(Arrays.asList(terms))));
|
||||
}
|
||||
Collections.addAll(allAutomata, automata);
|
||||
for (SpanQuery spanQuery : strictPhrases.getSpanQueries()) {
|
||||
Collections.addAll(allAutomata,
|
||||
MultiTermHighlighting.extractAutomata(spanQuery, field, true));//true==lookInSpan
|
||||
}
|
||||
|
||||
if (allAutomata.size() == 1) {
|
||||
return allAutomata.get(0);
|
||||
}
|
||||
//TODO it'd be nice if we could get at the underlying Automaton in CharacterRunAutomaton so that we
|
||||
// could union them all. But it's not exposed, and note TermRangeQuery isn't modelled as an Automaton
|
||||
// by MultiTermHighlighting.
|
||||
|
||||
// Return an aggregate CharacterRunAutomaton of others
|
||||
return new CharacterRunAutomaton(Automata.makeEmpty()) {// the makeEmpty() is bogus; won't be used
|
||||
@Override
|
||||
public boolean run(char[] chars, int offset, int length) {
|
||||
for (int i = 0; i < allAutomata.size(); i++) {// don't use foreach to avoid Iterator allocation
|
||||
if (allAutomata.get(i).run(chars, offset, length)) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,138 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.search.uhighlight;
|
||||
|
||||
|
||||
/**
|
||||
* Creates a formatted snippet from the top passages.
|
||||
* <p>
|
||||
* The default implementation marks the query terms as bold, and places
|
||||
* ellipses between unconnected passages.
|
||||
*/
|
||||
public class DefaultPassageFormatter extends PassageFormatter {
|
||||
/** text that will appear before highlighted terms */
|
||||
protected final String preTag;
|
||||
/** text that will appear after highlighted terms */
|
||||
protected final String postTag;
|
||||
/** text that will appear between two unconnected passages */
|
||||
protected final String ellipsis;
|
||||
/** true if we should escape for html */
|
||||
protected final boolean escape;
|
||||
|
||||
/**
|
||||
* Creates a new DefaultPassageFormatter with the default tags.
|
||||
*/
|
||||
public DefaultPassageFormatter() {
|
||||
this("<b>", "</b>", "... ", false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new DefaultPassageFormatter with custom tags.
|
||||
* @param preTag text which should appear before a highlighted term.
|
||||
* @param postTag text which should appear after a highlighted term.
|
||||
* @param ellipsis text which should be used to connect two unconnected passages.
|
||||
* @param escape true if text should be html-escaped
|
||||
*/
|
||||
public DefaultPassageFormatter(String preTag, String postTag, String ellipsis, boolean escape) {
|
||||
if (preTag == null || postTag == null || ellipsis == null) {
|
||||
throw new NullPointerException();
|
||||
}
|
||||
this.preTag = preTag;
|
||||
this.postTag = postTag;
|
||||
this.ellipsis = ellipsis;
|
||||
this.escape = escape;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String format(Passage passages[], String content) {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
int pos = 0;
|
||||
for (Passage passage : passages) {
|
||||
// don't add ellipsis if its the first one, or if its connected.
|
||||
if (passage.startOffset > pos && pos > 0) {
|
||||
sb.append(ellipsis);
|
||||
}
|
||||
pos = passage.startOffset;
|
||||
for (int i = 0; i < passage.numMatches; i++) {
|
||||
int start = passage.matchStarts[i];
|
||||
int end = passage.matchEnds[i];
|
||||
// its possible to have overlapping terms
|
||||
if (start > pos) {
|
||||
append(sb, content, pos, start);
|
||||
}
|
||||
if (end > pos) {
|
||||
sb.append(preTag);
|
||||
append(sb, content, Math.max(pos, start), end);
|
||||
sb.append(postTag);
|
||||
pos = end;
|
||||
}
|
||||
}
|
||||
// its possible a "term" from the analyzer could span a sentence boundary.
|
||||
append(sb, content, pos, Math.max(pos, passage.endOffset));
|
||||
pos = passage.endOffset;
|
||||
}
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Appends original text to the response.
|
||||
* @param dest resulting text, possibly transformed or encoded
|
||||
* @param content original text content
|
||||
* @param start index of the first character in content
|
||||
* @param end index of the character following the last character in content
|
||||
*/
|
||||
protected void append(StringBuilder dest, String content, int start, int end) {
|
||||
if (escape) {
|
||||
// note: these are the rules from owasp.org
|
||||
for (int i = start; i < end; i++) {
|
||||
char ch = content.charAt(i);
|
||||
switch(ch) {
|
||||
case '&':
|
||||
dest.append("&");
|
||||
break;
|
||||
case '<':
|
||||
dest.append("<");
|
||||
break;
|
||||
case '>':
|
||||
dest.append(">");
|
||||
break;
|
||||
case '"':
|
||||
dest.append(""");
|
||||
break;
|
||||
case '\'':
|
||||
dest.append("'");
|
||||
break;
|
||||
case '/':
|
||||
dest.append("/");
|
||||
break;
|
||||
default:
|
||||
if (ch >= 0x30 && ch <= 0x39 || ch >= 0x41 && ch <= 0x5A || ch >= 0x61 && ch <= 0x7A) {
|
||||
dest.append(ch);
|
||||
} else if (ch < 0xff) {
|
||||
dest.append("&#");
|
||||
dest.append((int)ch);
|
||||
dest.append(";");
|
||||
} else {
|
||||
dest.append(ch);
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
dest.append(content, start, end);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,276 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.search.uhighlight;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.text.BreakIterator;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.PriorityQueue;
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.PostingsEnum;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
/**
|
||||
* Internal highlighter abstraction that operates on a per field basis.
|
||||
*
|
||||
* @lucene.internal
|
||||
*/
|
||||
public class FieldHighlighter {
|
||||
|
||||
protected final String field;
|
||||
protected final FieldOffsetStrategy fieldOffsetStrategy;
|
||||
protected final BreakIterator breakIterator; // note: stateful!
|
||||
protected final PassageScorer passageScorer;
|
||||
protected final int maxPassages;
|
||||
protected final int maxNoHighlightPassages;
|
||||
protected final PassageFormatter passageFormatter;
|
||||
|
||||
public FieldHighlighter(String field, FieldOffsetStrategy fieldOffsetStrategy, BreakIterator breakIterator,
|
||||
PassageScorer passageScorer, int maxPassages, int maxNoHighlightPassages,
|
||||
PassageFormatter passageFormatter) {
|
||||
this.field = field;
|
||||
this.fieldOffsetStrategy = fieldOffsetStrategy;
|
||||
this.breakIterator = breakIterator;
|
||||
this.passageScorer = passageScorer;
|
||||
this.maxPassages = maxPassages;
|
||||
this.maxNoHighlightPassages = maxNoHighlightPassages;
|
||||
this.passageFormatter = passageFormatter;
|
||||
}
|
||||
|
||||
public String getField() {
|
||||
return field;
|
||||
}
|
||||
|
||||
public UnifiedHighlighter.OffsetSource getOffsetSource() {
|
||||
return fieldOffsetStrategy.getOffsetSource();
|
||||
}
|
||||
|
||||
/**
|
||||
* The primary method -- highlight this doc, assuming a specific field and given this content.
|
||||
*/
|
||||
public Object highlightFieldForDoc(IndexReader reader, int docId, String content) throws IOException {
|
||||
// TODO accept LeafReader instead?
|
||||
// note: it'd be nice to accept a CharSequence for content, but we need a CharacterIterator impl for it.
|
||||
if (content.length() == 0) {
|
||||
return null; // nothing to do
|
||||
}
|
||||
|
||||
breakIterator.setText(content);
|
||||
|
||||
List<OffsetsEnum> offsetsEnums = fieldOffsetStrategy.getOffsetsEnums(reader, docId, content);
|
||||
|
||||
Passage[] passages;
|
||||
try {
|
||||
// Highlight the offsetsEnum list against the content to produce Passages.
|
||||
passages = highlightOffsetsEnums(offsetsEnums);// and breakIterator & scorer
|
||||
} finally {
|
||||
// Ensure closeable resources get closed
|
||||
IOUtils.close(offsetsEnums);
|
||||
}
|
||||
|
||||
// Format the resulting Passages.
|
||||
if (passages.length == 0) {
|
||||
// no passages were returned, so ask for a default summary
|
||||
passages = getSummaryPassagesNoHighlight(maxNoHighlightPassages == -1 ? maxPassages : maxNoHighlightPassages);
|
||||
}
|
||||
|
||||
if (passages.length > 0) {
|
||||
return passageFormatter.format(passages, content);
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Called to summarize a document when no highlights were found.
|
||||
* By default this just returns the first
|
||||
* {@link #maxPassages} sentences; subclasses can override to customize.
|
||||
* The state of {@link #breakIterator} should be at the beginning.
|
||||
*/
|
||||
protected Passage[] getSummaryPassagesNoHighlight(int maxPassages) {
|
||||
assert breakIterator.current() == breakIterator.first();
|
||||
|
||||
List<Passage> passages = new ArrayList<>(Math.min(maxPassages, 10));
|
||||
int pos = breakIterator.current();
|
||||
assert pos == 0;
|
||||
while (passages.size() < maxPassages) {
|
||||
int next = breakIterator.next();
|
||||
if (next == BreakIterator.DONE) {
|
||||
break;
|
||||
}
|
||||
Passage passage = new Passage();
|
||||
passage.score = Float.NaN;
|
||||
passage.startOffset = pos;
|
||||
passage.endOffset = next;
|
||||
passages.add(passage);
|
||||
pos = next;
|
||||
}
|
||||
|
||||
return passages.toArray(new Passage[passages.size()]);
|
||||
}
|
||||
|
||||
// algorithm: treat sentence snippets as miniature documents
|
||||
// we can intersect these with the postings lists via BreakIterator.preceding(offset),s
|
||||
// score each sentence as norm(sentenceStartOffset) * sum(weight * tf(freq))
|
||||
protected Passage[] highlightOffsetsEnums(List<OffsetsEnum> offsetsEnums)
|
||||
throws IOException {
|
||||
PassageScorer scorer = passageScorer;
|
||||
BreakIterator breakIterator = this.breakIterator;
|
||||
final int contentLength = breakIterator.getText().getEndIndex();
|
||||
|
||||
PriorityQueue<OffsetsEnum> offsetsEnumQueue = new PriorityQueue<>(offsetsEnums.size() + 1);
|
||||
for (OffsetsEnum off : offsetsEnums) {
|
||||
off.weight = scorer.weight(contentLength, off.postingsEnum.freq());
|
||||
off.nextPosition(); // go to first position
|
||||
offsetsEnumQueue.add(off);
|
||||
}
|
||||
offsetsEnumQueue.add(new OffsetsEnum(null, EMPTY)); // a sentinel for termination
|
||||
|
||||
PriorityQueue<Passage> passageQueue = new PriorityQueue<>(Math.min(64, maxPassages + 1), (left, right) -> {
|
||||
if (left.score < right.score) {
|
||||
return -1;
|
||||
} else if (left.score > right.score) {
|
||||
return 1;
|
||||
} else {
|
||||
return left.startOffset - right.startOffset;
|
||||
}
|
||||
});
|
||||
Passage passage = new Passage(); // the current passage in-progress. Will either get reset or added to queue.
|
||||
|
||||
OffsetsEnum off;
|
||||
while ((off = offsetsEnumQueue.poll()) != null) {
|
||||
int start = off.startOffset();
|
||||
if (start == -1) {
|
||||
throw new IllegalArgumentException("field '" + field + "' was indexed without offsets, cannot highlight");
|
||||
}
|
||||
int end = off.endOffset();
|
||||
// LUCENE-5166: this hit would span the content limit... however more valid
|
||||
// hits may exist (they are sorted by start). so we pretend like we never
|
||||
// saw this term, it won't cause a passage to be added to passageQueue or anything.
|
||||
assert EMPTY.startOffset() == Integer.MAX_VALUE;
|
||||
if (start < contentLength && end > contentLength) {
|
||||
continue;
|
||||
}
|
||||
// See if this term should be part of a new passage.
|
||||
if (start >= passage.endOffset) {
|
||||
if (passage.startOffset >= 0) { // true if this passage has terms; otherwise couldn't find any (yet)
|
||||
// finalize passage
|
||||
passage.score *= scorer.norm(passage.startOffset);
|
||||
// new sentence: first add 'passage' to queue
|
||||
if (passageQueue.size() == maxPassages && passage.score < passageQueue.peek().score) {
|
||||
passage.reset(); // can't compete, just reset it
|
||||
} else {
|
||||
passageQueue.offer(passage);
|
||||
if (passageQueue.size() > maxPassages) {
|
||||
passage = passageQueue.poll();
|
||||
passage.reset();
|
||||
} else {
|
||||
passage = new Passage();
|
||||
}
|
||||
}
|
||||
}
|
||||
// if we exceed limit, we are done
|
||||
if (start >= contentLength) {
|
||||
break;
|
||||
}
|
||||
// advance breakIterator
|
||||
passage.startOffset = Math.max(breakIterator.preceding(start + 1), 0);
|
||||
passage.endOffset = Math.min(breakIterator.following(start), contentLength);
|
||||
}
|
||||
// Add this term to the passage.
|
||||
int tf = 0;
|
||||
while (true) {
|
||||
tf++;
|
||||
BytesRef term = off.getTerm();// a reference; safe to refer to
|
||||
assert term != null;
|
||||
passage.addMatch(start, end, term);
|
||||
// see if there are multiple occurrences of this term in this passage. If so, add them.
|
||||
if (!off.hasMorePositions()) {
|
||||
break; // No more in the entire text. Already removed from pq; move on
|
||||
}
|
||||
off.nextPosition();
|
||||
start = off.startOffset();
|
||||
end = off.endOffset();
|
||||
if (start >= passage.endOffset || end > contentLength) { // it's beyond this passage
|
||||
offsetsEnumQueue.offer(off);
|
||||
break;
|
||||
}
|
||||
}
|
||||
passage.score += off.weight * scorer.tf(tf, passage.endOffset - passage.startOffset);
|
||||
}
|
||||
|
||||
Passage[] passages = passageQueue.toArray(new Passage[passageQueue.size()]);
|
||||
for (Passage p : passages) {
|
||||
p.sort();
|
||||
}
|
||||
// sort in ascending order
|
||||
Arrays.sort(passages, (left, right) -> left.startOffset - right.startOffset);
|
||||
return passages;
|
||||
}
|
||||
|
||||
protected static final PostingsEnum EMPTY = new PostingsEnum() {
|
||||
|
||||
@Override
|
||||
public int nextPosition() throws IOException {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int startOffset() throws IOException {
|
||||
return Integer.MAX_VALUE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int endOffset() throws IOException {
|
||||
return Integer.MAX_VALUE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef getPayload() throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int freq() throws IOException {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return NO_MORE_DOCS;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextDoc() throws IOException {
|
||||
return NO_MORE_DOCS;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int advance(int target) throws IOException {
|
||||
return NO_MORE_DOCS;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long cost() {
|
||||
return 0;
|
||||
}
|
||||
};
|
||||
}
|
|
@ -0,0 +1,122 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.lucene.search.uhighlight;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.PostingsEnum;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.search.spans.Spans;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.automaton.CharacterRunAutomaton;
|
||||
|
||||
/**
|
||||
* Ultimately returns a list of {@link OffsetsEnum} yielding potentially highlightable words in the text. Needs
|
||||
* information about the query up front.
|
||||
*
|
||||
* @lucene.internal
|
||||
*/
|
||||
public abstract class FieldOffsetStrategy {
|
||||
|
||||
protected final String field;
|
||||
protected BytesRef[] terms; // Query: free-standing terms
|
||||
protected PhraseHelper strictPhrases; // Query: position-sensitive information TODO: rename
|
||||
protected CharacterRunAutomaton[] automata; // Query: free-standing wildcards (multi-term query)
|
||||
|
||||
public FieldOffsetStrategy(String field, BytesRef[] queryTerms, PhraseHelper phraseHelper, CharacterRunAutomaton[] automata) {
|
||||
this.field = field;
|
||||
this.terms = queryTerms;
|
||||
this.strictPhrases = phraseHelper;
|
||||
this.automata = automata;
|
||||
}
|
||||
|
||||
public String getField() {
|
||||
return field;
|
||||
}
|
||||
|
||||
public abstract UnifiedHighlighter.OffsetSource getOffsetSource();
|
||||
|
||||
/**
|
||||
* The primary method -- return offsets for highlightable words in the specified document.
|
||||
* IMPORTANT: remember to close them all.
|
||||
*/
|
||||
public abstract List<OffsetsEnum> getOffsetsEnums(IndexReader reader, int docId, String content) throws IOException;
|
||||
|
||||
protected List<OffsetsEnum> createOffsetsEnums(LeafReader leafReader, int doc, TokenStream tokenStream) throws IOException {
|
||||
List<OffsetsEnum> offsetsEnums = createOffsetsEnumsFromReader(leafReader, doc);
|
||||
if (automata.length > 0) {
|
||||
offsetsEnums.add(createOffsetsEnumFromTokenStream(doc, tokenStream));
|
||||
}
|
||||
return offsetsEnums;
|
||||
}
|
||||
|
||||
protected List<OffsetsEnum> createOffsetsEnumsFromReader(LeafReader atomicReader, int doc) throws IOException {
|
||||
// For strict positions, get a Map of term to Spans:
|
||||
// note: ScriptPhraseHelper.NONE does the right thing for these method calls
|
||||
final Map<BytesRef, Spans> strictPhrasesTermToSpans =
|
||||
strictPhrases.getTermToSpans(atomicReader, doc);
|
||||
// Usually simply wraps terms in a List; but if willRewrite() then can be expanded
|
||||
final List<BytesRef> sourceTerms =
|
||||
strictPhrases.expandTermsIfRewrite(terms, strictPhrasesTermToSpans);
|
||||
|
||||
final List<OffsetsEnum> offsetsEnums = new ArrayList<>(sourceTerms.size() + 1);
|
||||
|
||||
Terms termsIndex = atomicReader == null || sourceTerms.isEmpty() ? null : atomicReader.terms(field);
|
||||
if (termsIndex != null) {
|
||||
TermsEnum termsEnum = termsIndex.iterator();//does not return null
|
||||
for (BytesRef term : sourceTerms) {
|
||||
if (!termsEnum.seekExact(term)) {
|
||||
continue; // term not found
|
||||
}
|
||||
PostingsEnum postingsEnum = termsEnum.postings(null, PostingsEnum.OFFSETS);
|
||||
if (postingsEnum == null) {
|
||||
// no offsets or positions available
|
||||
throw new IllegalArgumentException("field '" + field + "' was indexed without offsets, cannot highlight");
|
||||
}
|
||||
if (doc != postingsEnum.advance(doc)) { // now it's positioned, although may be exhausted
|
||||
continue;
|
||||
}
|
||||
postingsEnum = strictPhrases.filterPostings(term, postingsEnum, strictPhrasesTermToSpans.get(term));
|
||||
if (postingsEnum == null) {
|
||||
continue;// completely filtered out
|
||||
}
|
||||
|
||||
offsetsEnums.add(new OffsetsEnum(term, postingsEnum));
|
||||
}
|
||||
}
|
||||
return offsetsEnums;
|
||||
}
|
||||
|
||||
protected OffsetsEnum createOffsetsEnumFromTokenStream(int doc, TokenStream tokenStream) throws IOException {
|
||||
// if there are automata (MTQ), we have to initialize the "fake" enum wrapping them.
|
||||
assert tokenStream != null;
|
||||
// TODO Opt: we sometimes evaluate the automata twice when this TS isn't the original; can we avoid?
|
||||
PostingsEnum mtqPostingsEnum = MultiTermHighlighting.getDocsEnum(tokenStream, automata);
|
||||
assert mtqPostingsEnum instanceof Closeable; // FYI we propagate close() later.
|
||||
mtqPostingsEnum.advance(doc);
|
||||
return new OffsetsEnum(null, mtqPostingsEnum);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,379 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.search.uhighlight;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.index.FilterLeafReader;
|
||||
import org.apache.lucene.index.FilteredTermsEnum;
|
||||
import org.apache.lucene.index.PostingsEnum;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.search.AutomatonQuery;
|
||||
import org.apache.lucene.search.BooleanClause;
|
||||
import org.apache.lucene.search.BooleanQuery;
|
||||
import org.apache.lucene.search.ConstantScoreQuery;
|
||||
import org.apache.lucene.search.DisjunctionMaxQuery;
|
||||
import org.apache.lucene.search.FuzzyQuery;
|
||||
import org.apache.lucene.search.PrefixQuery;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.TermRangeQuery;
|
||||
import org.apache.lucene.search.spans.SpanMultiTermQueryWrapper;
|
||||
import org.apache.lucene.search.spans.SpanNearQuery;
|
||||
import org.apache.lucene.search.spans.SpanNotQuery;
|
||||
import org.apache.lucene.search.spans.SpanOrQuery;
|
||||
import org.apache.lucene.search.spans.SpanPositionCheckQuery;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CharsRef;
|
||||
import org.apache.lucene.util.CharsRefBuilder;
|
||||
import org.apache.lucene.util.UnicodeUtil;
|
||||
import org.apache.lucene.util.automaton.Automata;
|
||||
import org.apache.lucene.util.automaton.Automaton;
|
||||
import org.apache.lucene.util.automaton.CharacterRunAutomaton;
|
||||
import org.apache.lucene.util.automaton.LevenshteinAutomata;
|
||||
import org.apache.lucene.util.automaton.Operations;
|
||||
|
||||
/**
|
||||
* Support for highlighting multi-term queries.
|
||||
*
|
||||
* @lucene.internal
|
||||
*/
|
||||
class MultiTermHighlighting {
|
||||
private MultiTermHighlighting() {
|
||||
}
|
||||
|
||||
/**
|
||||
* Extracts all MultiTermQueries for {@code field}, and returns equivalent
|
||||
* automata that will match terms.
|
||||
*/
|
||||
public static CharacterRunAutomaton[] extractAutomata(Query query, String field, boolean lookInSpan) {
|
||||
List<CharacterRunAutomaton> list = new ArrayList<>();
|
||||
if (query instanceof BooleanQuery) {
|
||||
for (BooleanClause clause : (BooleanQuery) query) {
|
||||
if (!clause.isProhibited()) {
|
||||
list.addAll(Arrays.asList(extractAutomata(clause.getQuery(), field, lookInSpan)));
|
||||
}
|
||||
}
|
||||
} else if (query instanceof ConstantScoreQuery) {
|
||||
list.addAll(Arrays.asList(extractAutomata(((ConstantScoreQuery) query).getQuery(), field, lookInSpan)));
|
||||
} else if (query instanceof DisjunctionMaxQuery) {
|
||||
for (Query sub : ((DisjunctionMaxQuery) query).getDisjuncts()) {
|
||||
list.addAll(Arrays.asList(extractAutomata(sub, field, lookInSpan)));
|
||||
}
|
||||
} else if (lookInSpan && query instanceof SpanOrQuery) {
|
||||
for (Query sub : ((SpanOrQuery) query).getClauses()) {
|
||||
list.addAll(Arrays.asList(extractAutomata(sub, field, lookInSpan)));
|
||||
}
|
||||
} else if (lookInSpan && query instanceof SpanNearQuery) {
|
||||
for (Query sub : ((SpanNearQuery) query).getClauses()) {
|
||||
list.addAll(Arrays.asList(extractAutomata(sub, field, lookInSpan)));
|
||||
}
|
||||
} else if (lookInSpan && query instanceof SpanNotQuery) {
|
||||
list.addAll(Arrays.asList(extractAutomata(((SpanNotQuery) query).getInclude(), field, lookInSpan)));
|
||||
} else if (lookInSpan && query instanceof SpanPositionCheckQuery) {
|
||||
list.addAll(Arrays.asList(extractAutomata(((SpanPositionCheckQuery) query).getMatch(), field, lookInSpan)));
|
||||
} else if (lookInSpan && query instanceof SpanMultiTermQueryWrapper) {
|
||||
list.addAll(Arrays.asList(extractAutomata(((SpanMultiTermQueryWrapper<?>) query).getWrappedQuery(), field, lookInSpan)));
|
||||
} else if (query instanceof AutomatonQuery) {
|
||||
final AutomatonQuery aq = (AutomatonQuery) query;
|
||||
if (aq.getField().equals(field)) {
|
||||
list.add(new CharacterRunAutomaton(aq.getAutomaton()) {
|
||||
@Override
|
||||
public String toString() {
|
||||
return aq.toString();
|
||||
}
|
||||
});
|
||||
}
|
||||
} else if (query instanceof PrefixQuery) {
|
||||
final PrefixQuery pq = (PrefixQuery) query;
|
||||
Term prefix = pq.getPrefix();
|
||||
if (prefix.field().equals(field)) {
|
||||
list.add(new CharacterRunAutomaton(Operations.concatenate(Automata.makeString(prefix.text()),
|
||||
Automata.makeAnyString())) {
|
||||
@Override
|
||||
public String toString() {
|
||||
return pq.toString();
|
||||
}
|
||||
});
|
||||
}
|
||||
} else if (query instanceof FuzzyQuery) {
|
||||
final FuzzyQuery fq = (FuzzyQuery) query;
|
||||
if (fq.getField().equals(field)) {
|
||||
String utf16 = fq.getTerm().text();
|
||||
int termText[] = new int[utf16.codePointCount(0, utf16.length())];
|
||||
for (int cp, i = 0, j = 0; i < utf16.length(); i += Character.charCount(cp)) {
|
||||
termText[j++] = cp = utf16.codePointAt(i);
|
||||
}
|
||||
int termLength = termText.length;
|
||||
int prefixLength = Math.min(fq.getPrefixLength(), termLength);
|
||||
String suffix = UnicodeUtil.newString(termText, prefixLength, termText.length - prefixLength);
|
||||
LevenshteinAutomata builder = new LevenshteinAutomata(suffix, fq.getTranspositions());
|
||||
String prefix = UnicodeUtil.newString(termText, 0, prefixLength);
|
||||
Automaton automaton = builder.toAutomaton(fq.getMaxEdits(), prefix);
|
||||
list.add(new CharacterRunAutomaton(automaton) {
|
||||
@Override
|
||||
public String toString() {
|
||||
return fq.toString();
|
||||
}
|
||||
});
|
||||
}
|
||||
} else if (query instanceof TermRangeQuery) {
|
||||
final TermRangeQuery tq = (TermRangeQuery) query;
|
||||
if (tq.getField().equals(field)) {
|
||||
final CharsRef lowerBound;
|
||||
if (tq.getLowerTerm() == null) {
|
||||
lowerBound = null;
|
||||
} else {
|
||||
lowerBound = new CharsRef(tq.getLowerTerm().utf8ToString());
|
||||
}
|
||||
|
||||
final CharsRef upperBound;
|
||||
if (tq.getUpperTerm() == null) {
|
||||
upperBound = null;
|
||||
} else {
|
||||
upperBound = new CharsRef(tq.getUpperTerm().utf8ToString());
|
||||
}
|
||||
|
||||
final boolean includeLower = tq.includesLower();
|
||||
final boolean includeUpper = tq.includesUpper();
|
||||
final CharsRef scratch = new CharsRef();
|
||||
|
||||
@SuppressWarnings("deprecation")
|
||||
final Comparator<CharsRef> comparator = CharsRef.getUTF16SortedAsUTF8Comparator();
|
||||
|
||||
// this is *not* an automaton, but its very simple
|
||||
list.add(new CharacterRunAutomaton(Automata.makeEmpty()) {
|
||||
@Override
|
||||
public boolean run(char[] s, int offset, int length) {
|
||||
scratch.chars = s;
|
||||
scratch.offset = offset;
|
||||
scratch.length = length;
|
||||
|
||||
if (lowerBound != null) {
|
||||
int cmp = comparator.compare(scratch, lowerBound);
|
||||
if (cmp < 0 || (!includeLower && cmp == 0)) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
if (upperBound != null) {
|
||||
int cmp = comparator.compare(scratch, upperBound);
|
||||
if (cmp > 0 || (!includeUpper && cmp == 0)) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return tq.toString();
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
return list.toArray(new CharacterRunAutomaton[list.size()]);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a "fake" DocsAndPositionsEnum over the tokenstream, returning offsets where {@code matchers}
|
||||
* matches tokens.
|
||||
* <p>
|
||||
* This is solely used internally by PostingsHighlighter: <b>DO NOT USE THIS METHOD!</b>
|
||||
*/
|
||||
public static PostingsEnum getDocsEnum(final TokenStream ts, final CharacterRunAutomaton[] matchers) throws IOException {
|
||||
return new TokenStreamPostingsEnum(ts, matchers);
|
||||
}
|
||||
|
||||
// TODO: we could use CachingWrapperFilter, (or consume twice) to allow us to have a true freq()
|
||||
// but this would have a performance cost for likely little gain in the user experience, it
|
||||
// would only serve to make this method less bogus.
|
||||
// instead, we always return freq() = Integer.MAX_VALUE and let the highlighter terminate based on offset...
|
||||
// TODO: DWS perhaps instead OffsetsEnum could become abstract and this would be an impl?
|
||||
private static class TokenStreamPostingsEnum extends PostingsEnum implements Closeable {
|
||||
TokenStream stream; // becomes null when closed
|
||||
final CharacterRunAutomaton[] matchers;
|
||||
final CharTermAttribute charTermAtt;
|
||||
final OffsetAttribute offsetAtt;
|
||||
|
||||
int currentDoc = -1;
|
||||
int currentMatch = -1;
|
||||
int currentStartOffset = -1;
|
||||
|
||||
int currentEndOffset = -1;
|
||||
|
||||
final BytesRef matchDescriptions[];
|
||||
|
||||
TokenStreamPostingsEnum(TokenStream ts, CharacterRunAutomaton[] matchers) throws IOException {
|
||||
this.stream = ts;
|
||||
this.matchers = matchers;
|
||||
matchDescriptions = new BytesRef[matchers.length];
|
||||
charTermAtt = ts.addAttribute(CharTermAttribute.class);
|
||||
offsetAtt = ts.addAttribute(OffsetAttribute.class);
|
||||
ts.reset();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextPosition() throws IOException {
|
||||
if (stream != null) {
|
||||
while (stream.incrementToken()) {
|
||||
for (int i = 0; i < matchers.length; i++) {
|
||||
if (matchers[i].run(charTermAtt.buffer(), 0, charTermAtt.length())) {
|
||||
currentStartOffset = offsetAtt.startOffset();
|
||||
currentEndOffset = offsetAtt.endOffset();
|
||||
currentMatch = i;
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
}
|
||||
stream.end();
|
||||
close();
|
||||
}
|
||||
// exhausted
|
||||
currentStartOffset = currentEndOffset = Integer.MAX_VALUE;
|
||||
return Integer.MAX_VALUE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int freq() throws IOException {
|
||||
return Integer.MAX_VALUE; // lie
|
||||
}
|
||||
|
||||
@Override
|
||||
public int startOffset() throws IOException {
|
||||
assert currentStartOffset >= 0;
|
||||
return currentStartOffset;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int endOffset() throws IOException {
|
||||
assert currentEndOffset >= 0;
|
||||
return currentEndOffset;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef getPayload() throws IOException {
|
||||
if (matchDescriptions[currentMatch] == null) {
|
||||
matchDescriptions[currentMatch] = new BytesRef(matchers[currentMatch].toString());
|
||||
}
|
||||
return matchDescriptions[currentMatch];
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return currentDoc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextDoc() throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int advance(int target) throws IOException {
|
||||
return currentDoc = target;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long cost() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (stream != null) {
|
||||
stream.close();
|
||||
stream = null;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Return a TokenStream un-inverted from the provided Terms, but filtered based on the automata. The
|
||||
* Terms must have exactly one doc count (e.g. term vector or MemoryIndex).
|
||||
*/
|
||||
//TODO: Alternatively, produce a list of OffsetsEnums from the Terms that match the automata.
|
||||
public static TokenStream uninvertAndFilterTerms(Terms termsIndex,
|
||||
int doc,
|
||||
final CharacterRunAutomaton[] automata,
|
||||
int offsetLength)
|
||||
throws IOException {
|
||||
assert automata.length > 0;
|
||||
//Note: if automata were plain Automaton (not CharacterRunAutomaton), we might instead use
|
||||
// TermsEnum.intersect(compiledAutomaton). But probably won't help due to O(N) TV impl so whatever.
|
||||
FilterLeafReader.FilterTerms filteredTermsIndex = new FilterLeafReader.FilterTerms(termsIndex) {
|
||||
@Override
|
||||
public TermsEnum iterator() throws IOException {
|
||||
return new FilteredTermsEnum(super.iterator(), false) {//false == no seek
|
||||
CharsRefBuilder tempCharsRefBuilder = new CharsRefBuilder();//reuse only for UTF8->UTF16 call
|
||||
|
||||
@Override
|
||||
protected AcceptStatus accept(BytesRef termBytesRef) throws IOException {
|
||||
//Grab the term (in same way as BytesRef.utf8ToString() but we don't want a String obj)
|
||||
tempCharsRefBuilder.grow(termBytesRef.length);
|
||||
final int charLen = UnicodeUtil.UTF8toUTF16(termBytesRef, tempCharsRefBuilder.chars());
|
||||
for (CharacterRunAutomaton runAutomaton : automata) {
|
||||
if (runAutomaton.run(tempCharsRefBuilder.chars(), 0, charLen)) {
|
||||
return AcceptStatus.YES;
|
||||
}
|
||||
}
|
||||
return AcceptStatus.NO;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public long size() throws IOException {
|
||||
return -1; // unknown
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getSumTotalTermFreq() throws IOException {
|
||||
return -1; // unknown
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getSumDocFreq() throws IOException {
|
||||
return -1; // unknown
|
||||
}
|
||||
};
|
||||
float loadFactor = 1f / 64f;
|
||||
return new TokenStreamFromTermVector(filteredTermsIndex, doc, offsetLength, loadFactor);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a simple automata that matches the specified term.
|
||||
*/
|
||||
public static CharacterRunAutomaton makeStringMatchAutomata(BytesRef term) {
|
||||
String termString = term.utf8ToString();
|
||||
return new CharacterRunAutomaton(Automata.makeString(termString)) {
|
||||
@Override
|
||||
public String toString() {
|
||||
return termString;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
|
@ -0,0 +1,148 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.search.uhighlight;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.TokenFilter;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
|
||||
/**
|
||||
* Wraps an {@link Analyzer} and string text that represents multiple values delimited by a specified character. This
|
||||
* exposes a TokenStream that matches what would get indexed considering the
|
||||
* {@link Analyzer#getPositionIncrementGap(String)}. Currently this assumes {@link Analyzer#getOffsetGap(String)} is
|
||||
* 1; an exception will be thrown if it isn't.
|
||||
* <br />
|
||||
* It would be more orthogonal for this to be an Analyzer since we're wrapping an Analyzer but doing so seems like
|
||||
* more work. The underlying components see a Reader not a String -- and the String is easy to
|
||||
* split up without redundant buffering.
|
||||
*
|
||||
* @lucene.internal
|
||||
*/
|
||||
final class MultiValueTokenStream extends TokenFilter {
|
||||
|
||||
private final String fieldName;
|
||||
private final Analyzer indexAnalyzer;
|
||||
private final String content;
|
||||
private final char splitChar;
|
||||
|
||||
private final PositionIncrementAttribute posIncAtt = addAttribute(PositionIncrementAttribute.class);
|
||||
private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
|
||||
|
||||
private int startValIdx = 0;
|
||||
private int endValIdx;
|
||||
private int remainingPosInc = 0;
|
||||
|
||||
/** note: The caller must remember to close the TokenStream eventually. */
|
||||
static TokenStream wrap(String fieldName, Analyzer indexAnalyzer, String content, char splitChar)
|
||||
throws IOException {
|
||||
if (indexAnalyzer.getOffsetGap(fieldName) != 1) { // note: 1 is the default. It is RARELY changed.
|
||||
throw new IllegalArgumentException(
|
||||
"offset gap of the provided analyzer should be 1 (field " + fieldName + ")");
|
||||
}
|
||||
// If there is no splitChar in content then we needn't wrap:
|
||||
int splitCharIdx = content.indexOf(splitChar);
|
||||
if (splitCharIdx == -1) {
|
||||
return indexAnalyzer.tokenStream(fieldName, content);
|
||||
}
|
||||
|
||||
TokenStream subTokenStream = indexAnalyzer.tokenStream(fieldName, content.substring(0, splitCharIdx));
|
||||
|
||||
return new MultiValueTokenStream(subTokenStream, fieldName, indexAnalyzer, content, splitChar, splitCharIdx);
|
||||
}
|
||||
|
||||
private MultiValueTokenStream(TokenStream subTokenStream, String fieldName, Analyzer indexAnalyzer,
|
||||
String content, char splitChar, int splitCharIdx) {
|
||||
super(subTokenStream); // subTokenStream is already initialized to operate on the first value
|
||||
this.fieldName = fieldName;
|
||||
this.indexAnalyzer = indexAnalyzer;
|
||||
this.content = content;
|
||||
this.splitChar = splitChar;
|
||||
this.endValIdx = splitCharIdx;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void reset() throws IOException {
|
||||
if (startValIdx != 0) {
|
||||
throw new IllegalStateException("This TokenStream wasn't developed to be re-used.");
|
||||
// ... although we could if a need for it arises.
|
||||
}
|
||||
super.reset();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean incrementToken() throws IOException {
|
||||
while (true) {
|
||||
|
||||
if (input.incrementToken()) {
|
||||
// Position tracking:
|
||||
if (remainingPosInc > 0) {//usually true first token of additional values (not first val)
|
||||
posIncAtt.setPositionIncrement(remainingPosInc + posIncAtt.getPositionIncrement());
|
||||
remainingPosInc = 0;//reset
|
||||
}
|
||||
// Offset tracking:
|
||||
offsetAtt.setOffset(
|
||||
startValIdx + offsetAtt.startOffset(),
|
||||
startValIdx + offsetAtt.endOffset()
|
||||
);
|
||||
return true;
|
||||
}
|
||||
|
||||
if (endValIdx == content.length()) {//no more
|
||||
return false;
|
||||
}
|
||||
|
||||
input.end(); // might adjust position increment
|
||||
remainingPosInc += posIncAtt.getPositionIncrement();
|
||||
input.close();
|
||||
remainingPosInc += indexAnalyzer.getPositionIncrementGap(fieldName);
|
||||
|
||||
// Get new tokenStream based on next segment divided by the splitChar
|
||||
startValIdx = endValIdx + 1;
|
||||
endValIdx = content.indexOf(splitChar, startValIdx);
|
||||
if (endValIdx == -1) {//EOF
|
||||
endValIdx = content.length();
|
||||
}
|
||||
TokenStream tokenStream = indexAnalyzer.tokenStream(fieldName, content.substring(startValIdx, endValIdx));
|
||||
if (tokenStream != input) {// (input is defined in TokenFilter set in the constructor)
|
||||
// This is a grand trick we do -- knowing that the analyzer's re-use strategy is going to produce the
|
||||
// very same tokenStream instance and thus have the same AttributeSource as this wrapping TokenStream
|
||||
// since we used it as our input in the constructor.
|
||||
// Were this not the case, we'd have to copy every attribute of interest since we can't alter the
|
||||
// AttributeSource of this wrapping TokenStream post-construction (it's all private/final).
|
||||
// If this is a problem, we could do that instead; maybe with a custom CharTermAttribute that allows
|
||||
// us to easily set the char[] reference without literally copying char by char.
|
||||
throw new IllegalStateException("Require TokenStream re-use. Unsupported re-use strategy?: " +
|
||||
indexAnalyzer.getReuseStrategy());
|
||||
}
|
||||
tokenStream.reset();
|
||||
} // while loop to increment token of this new value
|
||||
}
|
||||
|
||||
@Override
|
||||
public void end() throws IOException {
|
||||
super.end();
|
||||
// Offset tracking:
|
||||
offsetAtt.setOffset(
|
||||
startValIdx + offsetAtt.startOffset(),
|
||||
startValIdx + offsetAtt.endOffset());
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,50 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.search.uhighlight;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.automaton.CharacterRunAutomaton;
|
||||
|
||||
/**
|
||||
* Never returns offsets. Used when the query would highlight nothing.
|
||||
*
|
||||
* @lucene.internal
|
||||
*/
|
||||
public class NoOpOffsetStrategy extends FieldOffsetStrategy {
|
||||
|
||||
public static final NoOpOffsetStrategy INSTANCE = new NoOpOffsetStrategy();
|
||||
|
||||
private NoOpOffsetStrategy() {
|
||||
super("_ignored_", new BytesRef[0], PhraseHelper.NONE, new CharacterRunAutomaton[0]);
|
||||
}
|
||||
|
||||
@Override
|
||||
public UnifiedHighlighter.OffsetSource getOffsetSource() {
|
||||
return UnifiedHighlighter.OffsetSource.NONE_NEEDED;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<OffsetsEnum> getOffsetsEnums(IndexReader reader, int docId, String content) throws IOException {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,97 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.search.uhighlight;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.Objects;
|
||||
|
||||
import org.apache.lucene.index.PostingsEnum;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/**
|
||||
* Holds the term & PostingsEnum, and info for tracking the occurrences of a term within the text.
|
||||
* It is advanced with the underlying postings and is placed in a priority queue by highlightOffsetsEnums
|
||||
* based on the start offset.
|
||||
*
|
||||
* @lucene.internal
|
||||
*/
|
||||
public class OffsetsEnum implements Comparable<OffsetsEnum>, Closeable {
|
||||
private final BytesRef term;
|
||||
final PostingsEnum postingsEnum; // with offsets
|
||||
|
||||
float weight; // set once in highlightOffsetsEnums
|
||||
private int pos = 0; // the occurrence counter of this term within the text being highlighted.
|
||||
|
||||
public OffsetsEnum(BytesRef term, PostingsEnum postingsEnum) throws IOException {
|
||||
this.term = term; // can be null
|
||||
this.postingsEnum = Objects.requireNonNull(postingsEnum);
|
||||
}
|
||||
|
||||
// note: the ordering clearly changes as the postings enum advances
|
||||
@Override
|
||||
public int compareTo(OffsetsEnum other) {
|
||||
try {
|
||||
int cmp = Integer.compare(startOffset(), other.startOffset());
|
||||
if (cmp != 0) {
|
||||
return cmp; // vast majority of the time we return here.
|
||||
}
|
||||
if (this.term == null || other.term == null) {
|
||||
if (this.term == null && other.term == null) {
|
||||
return 0;
|
||||
} else if (this.term == null) {
|
||||
return 1; // put "this" (wildcard mtq enum) last
|
||||
} else {
|
||||
return -1;
|
||||
}
|
||||
}
|
||||
return term.compareTo(other.term);
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
BytesRef getTerm() throws IOException {
|
||||
// the dp.getPayload thing is a hack -- see MultiTermHighlighting
|
||||
return term != null ? term : postingsEnum.getPayload();
|
||||
// We don't deepcopy() because in this hack we know we don't have to.
|
||||
}
|
||||
|
||||
boolean hasMorePositions() throws IOException {
|
||||
return pos < postingsEnum.freq();
|
||||
}
|
||||
|
||||
void nextPosition() throws IOException {
|
||||
pos++;
|
||||
postingsEnum.nextPosition();
|
||||
}
|
||||
|
||||
int startOffset() throws IOException {
|
||||
return postingsEnum.startOffset();
|
||||
}
|
||||
|
||||
int endOffset() throws IOException {
|
||||
return postingsEnum.endOffset();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (postingsEnum instanceof Closeable) { // the one in MultiTermHighlighting is.
|
||||
((Closeable) postingsEnum).close();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,161 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.search.uhighlight;
|
||||
|
||||
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.InPlaceMergeSorter;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
/**
|
||||
* Represents a passage (typically a sentence of the document).
|
||||
* <p>
|
||||
* A passage contains {@link #getNumMatches} highlights from the query,
|
||||
* and the offsets and query terms that correspond with each match.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public final class Passage {
|
||||
int startOffset = -1;
|
||||
int endOffset = -1;
|
||||
float score = 0.0f;
|
||||
|
||||
int matchStarts[] = new int[8];
|
||||
int matchEnds[] = new int[8];
|
||||
BytesRef matchTerms[] = new BytesRef[8];
|
||||
int numMatches = 0;
|
||||
|
||||
void addMatch(int startOffset, int endOffset, BytesRef term) {
|
||||
assert startOffset >= this.startOffset && startOffset <= this.endOffset;
|
||||
if (numMatches == matchStarts.length) {
|
||||
int newLength = ArrayUtil.oversize(numMatches+1, RamUsageEstimator.NUM_BYTES_OBJECT_REF);
|
||||
int newMatchStarts[] = new int[newLength];
|
||||
int newMatchEnds[] = new int[newLength];
|
||||
BytesRef newMatchTerms[] = new BytesRef[newLength];
|
||||
System.arraycopy(matchStarts, 0, newMatchStarts, 0, numMatches);
|
||||
System.arraycopy(matchEnds, 0, newMatchEnds, 0, numMatches);
|
||||
System.arraycopy(matchTerms, 0, newMatchTerms, 0, numMatches);
|
||||
matchStarts = newMatchStarts;
|
||||
matchEnds = newMatchEnds;
|
||||
matchTerms = newMatchTerms;
|
||||
}
|
||||
assert matchStarts.length == matchEnds.length && matchEnds.length == matchTerms.length;
|
||||
matchStarts[numMatches] = startOffset;
|
||||
matchEnds[numMatches] = endOffset;
|
||||
matchTerms[numMatches] = term;
|
||||
numMatches++;
|
||||
}
|
||||
|
||||
void sort() {
|
||||
final int starts[] = matchStarts;
|
||||
final int ends[] = matchEnds;
|
||||
final BytesRef terms[] = matchTerms;
|
||||
new InPlaceMergeSorter() {
|
||||
@Override
|
||||
protected void swap(int i, int j) {
|
||||
int temp = starts[i];
|
||||
starts[i] = starts[j];
|
||||
starts[j] = temp;
|
||||
|
||||
temp = ends[i];
|
||||
ends[i] = ends[j];
|
||||
ends[j] = temp;
|
||||
|
||||
BytesRef tempTerm = terms[i];
|
||||
terms[i] = terms[j];
|
||||
terms[j] = tempTerm;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int compare(int i, int j) {
|
||||
return Integer.compare(starts[i], starts[j]);
|
||||
}
|
||||
|
||||
}.sort(0, numMatches);
|
||||
}
|
||||
|
||||
void reset() {
|
||||
startOffset = endOffset = -1;
|
||||
score = 0.0f;
|
||||
numMatches = 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* Start offset of this passage.
|
||||
* @return start index (inclusive) of the passage in the
|
||||
* original content: always >= 0.
|
||||
*/
|
||||
public int getStartOffset() {
|
||||
return startOffset;
|
||||
}
|
||||
|
||||
/**
|
||||
* End offset of this passage.
|
||||
* @return end index (exclusive) of the passage in the
|
||||
* original content: always >= {@link #getStartOffset()}
|
||||
*/
|
||||
public int getEndOffset() {
|
||||
return endOffset;
|
||||
}
|
||||
|
||||
/**
|
||||
* Passage's score.
|
||||
*/
|
||||
public float getScore() {
|
||||
return score;
|
||||
}
|
||||
|
||||
/**
|
||||
* Number of term matches available in
|
||||
* {@link #getMatchStarts}, {@link #getMatchEnds},
|
||||
* {@link #getMatchTerms}
|
||||
*/
|
||||
public int getNumMatches() {
|
||||
return numMatches;
|
||||
}
|
||||
|
||||
/**
|
||||
* Start offsets of the term matches, in increasing order.
|
||||
* <p>
|
||||
* Only {@link #getNumMatches} are valid. Note that these
|
||||
* offsets are absolute (not relative to {@link #getStartOffset()}).
|
||||
*/
|
||||
public int[] getMatchStarts() {
|
||||
return matchStarts;
|
||||
}
|
||||
|
||||
/**
|
||||
* End offsets of the term matches, corresponding with {@link #getMatchStarts}.
|
||||
* <p>
|
||||
* Only {@link #getNumMatches} are valid. Note that its possible that an end offset
|
||||
* could exceed beyond the bounds of the passage ({@link #getEndOffset()}), if the
|
||||
* Analyzer produced a term which spans a passage boundary.
|
||||
*/
|
||||
public int[] getMatchEnds() {
|
||||
return matchEnds;
|
||||
}
|
||||
|
||||
/**
|
||||
* BytesRef (term text) of the matches, corresponding with {@link #getMatchStarts()}.
|
||||
* <p>
|
||||
* Only {@link #getNumMatches()} are valid.
|
||||
*/
|
||||
public BytesRef[] getMatchTerms() {
|
||||
return matchTerms;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,40 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.search.uhighlight;
|
||||
|
||||
/**
|
||||
* Creates a formatted snippet from the top passages.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public abstract class PassageFormatter {
|
||||
|
||||
/**
|
||||
* Formats the top <code>passages</code> from <code>content</code>
|
||||
* into a human-readable text snippet.
|
||||
*
|
||||
* @param passages top-N passages for the field. Note these are sorted in
|
||||
* the order that they appear in the document for convenience.
|
||||
* @param content content for the field.
|
||||
* @return formatted highlight. Note that for the
|
||||
* non-expert APIs in {@link UnifiedHighlighter} that
|
||||
* return String, the toString method on the Object
|
||||
* returned by this method is used to compute the string.
|
||||
*/
|
||||
public abstract Object format(Passage passages[], String content);
|
||||
|
||||
}
|
|
@ -0,0 +1,113 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.search.uhighlight;
|
||||
|
||||
/**
|
||||
* Ranks passages found by {@link UnifiedHighlighter}.
|
||||
* <p>
|
||||
* Each passage is scored as a miniature document within the document.
|
||||
* The final score is computed as {@link #norm} * ∑ ({@link #weight} * {@link #tf}).
|
||||
* The default implementation is {@link #norm} * BM25.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class PassageScorer {
|
||||
|
||||
// TODO: this formula is completely made up. It might not provide relevant snippets!
|
||||
|
||||
/**
|
||||
* BM25 k1 parameter, controls term frequency normalization
|
||||
*/
|
||||
final float k1;
|
||||
/**
|
||||
* BM25 b parameter, controls length normalization.
|
||||
*/
|
||||
final float b;
|
||||
/**
|
||||
* A pivot used for length normalization.
|
||||
*/
|
||||
final float pivot;
|
||||
|
||||
/**
|
||||
* Creates PassageScorer with these default values:
|
||||
* <ul>
|
||||
* <li>{@code k1 = 1.2},
|
||||
* <li>{@code b = 0.75}.
|
||||
* <li>{@code pivot = 87}
|
||||
* </ul>
|
||||
*/
|
||||
public PassageScorer() {
|
||||
// 1.2 and 0.75 are well-known bm25 defaults (but maybe not the best here) ?
|
||||
// 87 is typical average english sentence length.
|
||||
this(1.2f, 0.75f, 87f);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates PassageScorer with specified scoring parameters
|
||||
*
|
||||
* @param k1 Controls non-linear term frequency normalization (saturation).
|
||||
* @param b Controls to what degree passage length normalizes tf values.
|
||||
* @param pivot Pivot value for length normalization (some rough idea of average sentence length in characters).
|
||||
*/
|
||||
public PassageScorer(float k1, float b, float pivot) {
|
||||
this.k1 = k1;
|
||||
this.b = b;
|
||||
this.pivot = pivot;
|
||||
}
|
||||
|
||||
/**
|
||||
* Computes term importance, given its in-document statistics.
|
||||
*
|
||||
* @param contentLength length of document in characters
|
||||
* @param totalTermFreq number of time term occurs in document
|
||||
* @return term importance
|
||||
*/
|
||||
public float weight(int contentLength, int totalTermFreq) {
|
||||
// approximate #docs from content length
|
||||
float numDocs = 1 + contentLength / pivot;
|
||||
// numDocs not numDocs - docFreq (ala DFR), since we approximate numDocs
|
||||
return (k1 + 1) * (float) Math.log(1 + (numDocs + 0.5D) / (totalTermFreq + 0.5D));
|
||||
}
|
||||
|
||||
/**
|
||||
* Computes term weight, given the frequency within the passage
|
||||
* and the passage's length.
|
||||
*
|
||||
* @param freq number of occurrences of within this passage
|
||||
* @param passageLen length of the passage in characters.
|
||||
* @return term weight
|
||||
*/
|
||||
public float tf(int freq, int passageLen) {
|
||||
float norm = k1 * ((1 - b) + b * (passageLen / pivot));
|
||||
return freq / (freq + norm);
|
||||
}
|
||||
|
||||
/**
|
||||
* Normalize a passage according to its position in the document.
|
||||
* <p>
|
||||
* Typically passages towards the beginning of the document are
|
||||
* more useful for summarizing the contents.
|
||||
* <p>
|
||||
* The default implementation is <code>1 + 1/log(pivot + passageStart)</code>
|
||||
*
|
||||
* @param passageStart start offset of the passage
|
||||
* @return a boost value multiplied into the passage's core.
|
||||
*/
|
||||
public float norm(int passageStart) {
|
||||
return 1 + 1 / (float) Math.log(pivot + passageStart);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,581 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.search.uhighlight;
|
||||
|
||||
import org.apache.lucene.index.*;
|
||||
import org.apache.lucene.search.*;
|
||||
import org.apache.lucene.search.highlight.WeightedSpanTerm;
|
||||
import org.apache.lucene.search.highlight.WeightedSpanTermExtractor;
|
||||
import org.apache.lucene.search.spans.*;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.*;
|
||||
import java.util.function.Function;
|
||||
|
||||
/**
|
||||
* Helps the {@link FieldOffsetStrategy} with strict position highlighting (e.g. highlight phrases correctly).
|
||||
* This is a stateful class holding information about the query, but it can (and is) re-used across highlighting
|
||||
* documents. Despite this state; it's immutable after construction. The approach taken in this class is very similar
|
||||
* to the standard Highlighter's {@link WeightedSpanTermExtractor} which is in fact re-used here. However, we ought to
|
||||
* completely rewrite it to use the SpanCollector interface to collect offsets directly. We'll get better
|
||||
* phrase accuracy.
|
||||
*
|
||||
* @lucene.internal
|
||||
*/
|
||||
public class PhraseHelper {
|
||||
|
||||
public static final PhraseHelper NONE = new PhraseHelper(new MatchAllDocsQuery(), "_ignored_",
|
||||
spanQuery -> null, true);
|
||||
|
||||
//TODO it seems this ought to be a general thing on Spans?
|
||||
private static final Comparator<? super Spans> SPANS_COMPARATOR = (o1, o2) -> {
|
||||
int cmp = Integer.compare(o1.docID(), o2.docID());
|
||||
if (cmp != 0) {
|
||||
return cmp;
|
||||
}
|
||||
if (o1.docID() == DocIdSetIterator.NO_MORE_DOCS) {
|
||||
return 0; // don't ask for start/end position; not sure if we can even call those methods
|
||||
}
|
||||
cmp = Integer.compare(o1.startPosition(), o2.startPosition());
|
||||
if (cmp != 0) {
|
||||
return cmp;
|
||||
} else {
|
||||
return Integer.compare(o1.endPosition(), o2.endPosition());
|
||||
}
|
||||
};
|
||||
|
||||
private final String fieldName; // if non-null, only look at queries/terms for this field
|
||||
private final Set<Term> positionInsensitiveTerms; // (TermQuery terms)
|
||||
private final Set<SpanQuery> spanQueries;
|
||||
private final boolean willRewrite;
|
||||
|
||||
/**
|
||||
* Constructor.
|
||||
* {@code rewriteQueryPred} is an extension hook to override the default choice of
|
||||
* {@link WeightedSpanTermExtractor#mustRewriteQuery(SpanQuery)}. By default unknown query types are rewritten,
|
||||
* so use this to return {@link Boolean#FALSE} if you know the query doesn't need to be rewritten.
|
||||
* {@code ignoreQueriesNeedingRewrite} effectively ignores any query clause that needs to be "rewritten", which is
|
||||
* usually limited to just a {@link SpanMultiTermQueryWrapper} but could be other custom ones.
|
||||
*/
|
||||
public PhraseHelper(Query query, String field, Function<SpanQuery, Boolean> rewriteQueryPred,
|
||||
boolean ignoreQueriesNeedingRewrite) {
|
||||
this.fieldName = field; // if null then don't require field match
|
||||
// filter terms to those we want
|
||||
positionInsensitiveTerms = field != null ? new FieldFilteringTermHashSet(field) : new HashSet<>();
|
||||
// requireFieldMatch optional
|
||||
spanQueries = new HashSet<>();
|
||||
|
||||
// TODO Have toSpanQuery(query) Function as an extension point for those with custom Query impls
|
||||
|
||||
boolean[] mustRewriteHolder = {false}; // boolean wrapped in 1-ary array so it's mutable from inner class
|
||||
|
||||
// For TermQueries or other position insensitive queries, collect the Terms.
|
||||
// For other Query types, WSTE will convert to an equivalent SpanQuery. NOT extracting position spans here.
|
||||
new WeightedSpanTermExtractor(field) {
|
||||
//anonymous constructor
|
||||
{
|
||||
setExpandMultiTermQuery(true); //necessary for mustRewriteQuery(spanQuery) to work.
|
||||
|
||||
try {
|
||||
extract(query, 1f, null); // null because we won't actually extract right now; we're not collecting
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean isQueryUnsupported(Class<? extends Query> clazz) {
|
||||
if (clazz.isAssignableFrom(MultiTermQuery.class)) {
|
||||
return true; //We do MTQ processing separately in MultiTermHighlighting.java
|
||||
}
|
||||
return true; //TODO set to false and provide a hook to customize certain queries.
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void extractWeightedTerms(Map<String, WeightedSpanTerm> terms, Query query, float boost)
|
||||
throws IOException {
|
||||
query.createWeight(UnifiedHighlighter.EMPTY_INDEXSEARCHER, false)
|
||||
.extractTerms(positionInsensitiveTerms);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void extractWeightedSpanTerms(Map<String, WeightedSpanTerm> terms, SpanQuery spanQuery,
|
||||
float boost) throws IOException {
|
||||
if (field != null) {
|
||||
// if this span query isn't for this field, skip it.
|
||||
Set<String> fieldNameSet = new HashSet<>();//TODO reuse. note: almost always size 1
|
||||
collectSpanQueryFields(spanQuery, fieldNameSet);
|
||||
if (!fieldNameSet.contains(field)) {
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
// TODO allow users to override the answer to mustRewriteQuery
|
||||
boolean mustRewriteQuery = mustRewriteQuery(spanQuery);
|
||||
if (ignoreQueriesNeedingRewrite && mustRewriteQuery) {
|
||||
return;// ignore this query
|
||||
}
|
||||
mustRewriteHolder[0] |= mustRewriteQuery;
|
||||
|
||||
spanQueries.add(spanQuery);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean mustRewriteQuery(SpanQuery spanQuery) {
|
||||
Boolean rewriteQ = rewriteQueryPred.apply(spanQuery);// allow to override
|
||||
return rewriteQ != null ? rewriteQ : super.mustRewriteQuery(spanQuery);
|
||||
}
|
||||
}; // calling the constructor triggered the extraction/visiting we want. Hacky; yes.
|
||||
|
||||
willRewrite = mustRewriteHolder[0];
|
||||
}
|
||||
|
||||
Set<SpanQuery> getSpanQueries() {
|
||||
return spanQueries;
|
||||
}
|
||||
|
||||
/**
|
||||
* If there is no position sensitivity then use of the instance of this class can be ignored.
|
||||
*/
|
||||
boolean hasPositionSensitivity() {
|
||||
return spanQueries.isEmpty() == false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Rewrite is needed for handling a {@link SpanMultiTermQueryWrapper} (MTQ / wildcards) or some
|
||||
* custom things. When true, the resulting term list will probably be different than what it was known
|
||||
* to be initially.
|
||||
*/
|
||||
boolean willRewrite() {
|
||||
return willRewrite;
|
||||
}
|
||||
|
||||
/**
|
||||
* Collect a list of pre-positioned {@link Spans} for each term, given a reader that has just one document.
|
||||
* It returns no mapping for query terms that occurs in a position insensitive way which therefore don't
|
||||
* need to be filtered.
|
||||
*/
|
||||
Map<BytesRef, Spans> getTermToSpans(LeafReader leafReader, int doc)
|
||||
throws IOException {
|
||||
if (spanQueries.isEmpty()) {
|
||||
return Collections.emptyMap();
|
||||
}
|
||||
// for each SpanQuery, collect the member spans into a map.
|
||||
Map<BytesRef, Spans> result = new HashMap<>();
|
||||
for (SpanQuery spanQuery : spanQueries) {
|
||||
getTermToSpans(spanQuery, leafReader.getContext(), doc, result);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
// code extracted & refactored from WSTE.extractWeightedSpanTerms()
|
||||
private void getTermToSpans(SpanQuery spanQuery, LeafReaderContext readerContext,
|
||||
int doc, Map<BytesRef, Spans> result)
|
||||
throws IOException {
|
||||
// note: in WSTE there was some field specific looping that seemed pointless so that isn't here.
|
||||
final IndexSearcher searcher = new IndexSearcher(readerContext);
|
||||
searcher.setQueryCache(null);
|
||||
if (willRewrite) {
|
||||
spanQuery = (SpanQuery) searcher.rewrite(spanQuery); // searcher.rewrite loops till done
|
||||
}
|
||||
|
||||
// Get the underlying query terms
|
||||
|
||||
TreeSet<Term> termSet = new TreeSet<>(); // sorted so we can loop over results in order shortly...
|
||||
searcher.createWeight(spanQuery, false).extractTerms(termSet);//needsScores==false
|
||||
|
||||
// Get Spans by running the query against the reader
|
||||
// TODO it might make sense to re-use/cache the Spans instance, to advance forward between docs
|
||||
SpanWeight spanWeight = (SpanWeight) searcher.createNormalizedWeight(spanQuery, false);
|
||||
Spans spans = spanWeight.getSpans(readerContext, SpanWeight.Postings.POSITIONS);
|
||||
if (spans == null) {
|
||||
return;
|
||||
}
|
||||
TwoPhaseIterator twoPhaseIterator = spans.asTwoPhaseIterator();
|
||||
if (twoPhaseIterator != null) {
|
||||
if (twoPhaseIterator.approximation().advance(doc) != doc || !twoPhaseIterator.matches()) {
|
||||
return;
|
||||
}
|
||||
} else if (spans.advance(doc) != doc) { // preposition, and return doing nothing if find none
|
||||
return;
|
||||
}
|
||||
|
||||
// Consume the Spans into a cache. This instance is used as a source for multiple cloned copies.
|
||||
// It's important we do this and not re-use the same original Spans instance since these will be iterated
|
||||
// independently later on; sometimes in ways that prevents sharing the original Spans.
|
||||
CachedSpans cachedSpansSource = new CachedSpans(spans); // consumes spans for this doc only and caches
|
||||
spans = null;// we don't use it below
|
||||
|
||||
// Map terms to a Spans instance (aggregate if necessary)
|
||||
for (final Term queryTerm : termSet) {
|
||||
// note: we expect that at least one query term will pass these filters. This is because the collected
|
||||
// spanQuery list were already filtered by these conditions.
|
||||
if (fieldName != null && fieldName.equals(queryTerm.field()) == false) {
|
||||
continue;
|
||||
}
|
||||
if (positionInsensitiveTerms.contains(queryTerm)) {
|
||||
continue;
|
||||
}
|
||||
// copy-constructor refers to same data (shallow) but has iteration state from the beginning
|
||||
CachedSpans cachedSpans = new CachedSpans(cachedSpansSource);
|
||||
// Add the span to whatever span may or may not exist
|
||||
Spans existingSpans = result.get(queryTerm.bytes());
|
||||
if (existingSpans != null) {
|
||||
if (existingSpans instanceof MultiSpans) {
|
||||
((MultiSpans) existingSpans).addSpans(cachedSpans);
|
||||
} else { // upgrade to MultiSpans
|
||||
MultiSpans multiSpans = new MultiSpans();
|
||||
multiSpans.addSpans(existingSpans);
|
||||
multiSpans.addSpans(cachedSpans);
|
||||
result.put(queryTerm.bytes(), multiSpans);
|
||||
}
|
||||
} else {
|
||||
result.put(queryTerm.bytes(), cachedSpans);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns terms as a List, but expanded to any terms in strictPhrases' keySet if present. That can only
|
||||
* happen if willRewrite() is true.
|
||||
*/
|
||||
List<BytesRef> expandTermsIfRewrite(BytesRef[] terms, Map<BytesRef, Spans> strictPhrasesTermToSpans) {
|
||||
if (willRewrite()) {
|
||||
Set<BytesRef> allTermSet = new LinkedHashSet<>(terms.length + strictPhrasesTermToSpans.size());
|
||||
Collections.addAll(allTermSet, terms);//FYI already sorted; will keep order
|
||||
if (allTermSet.addAll(strictPhrasesTermToSpans.keySet())) { // true if any were added
|
||||
List<BytesRef> sourceTerms = Arrays.asList(allTermSet.toArray(new BytesRef[allTermSet.size()]));
|
||||
sourceTerms.sort(Comparator.naturalOrder());
|
||||
return sourceTerms;
|
||||
}
|
||||
}
|
||||
return Arrays.asList(terms); // no rewrite; use original terms
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a filtered postings where the position must be in the given Spans.
|
||||
* The Spans must be in a positioned state (not initial) and should not be shared between other terms.
|
||||
* {@code postingsEnum} should be positioned at the
|
||||
* document (the same one as the spans) but it hasn't iterated the positions yet.
|
||||
* The Spans should be the result of a simple
|
||||
* lookup from {@link #getTermToSpans(LeafReader, int)}, and so it could be null which could mean
|
||||
* either it's completely filtered or that there should be no filtering; this class knows what to do.
|
||||
* <p>
|
||||
* Due to limitations in filtering, the {@link PostingsEnum#freq()} is un-changed even if some positions
|
||||
* get filtered. So when {@link PostingsEnum#nextPosition()} is called or {@code startOffset} or {@code
|
||||
* endOffset} beyond the "real" positions, these methods returns {@link Integer#MAX_VALUE}.
|
||||
* <p>
|
||||
* <b>This will return null if it's completely filtered out (i.e. effectively has no postings).</b>
|
||||
*/
|
||||
PostingsEnum filterPostings(BytesRef term, PostingsEnum postingsEnum, Spans spans)
|
||||
throws IOException {
|
||||
if (spans == null) {
|
||||
if (hasPositionSensitivity() == false || positionInsensitiveTerms.contains(new Term(fieldName, term))) {
|
||||
return postingsEnum; // no filtering
|
||||
} else {
|
||||
return null; // completely filtered out
|
||||
}
|
||||
}
|
||||
if (postingsEnum.docID() != spans.docID()) {
|
||||
throw new IllegalStateException("Spans & Postings doc ID misaligned or not positioned");
|
||||
}
|
||||
|
||||
return new FilterLeafReader.FilterPostingsEnum(postingsEnum) {
|
||||
// freq() is max times nextPosition can be called. We'll set this var to -1 when exhausted.
|
||||
int remainingPositions = postingsEnum.freq();
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
String where;
|
||||
try {
|
||||
where = "[" + startOffset() + ":" + endOffset() + "]";
|
||||
} catch (IOException e) {
|
||||
where = "[" + e + "]";
|
||||
}
|
||||
return "'" + term.utf8ToString() + "'@" + where + " filtered by " + spans;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextDoc() throws IOException {
|
||||
throw new IllegalStateException("not expected"); // don't need to implement; just used on one doc
|
||||
}
|
||||
|
||||
@Override
|
||||
public int advance(int target) throws IOException {
|
||||
throw new IllegalStateException("not expected"); // don't need to implement; just used on one doc
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextPosition() throws IOException {
|
||||
// loop over posting positions...
|
||||
NEXT_POS_LOOP:
|
||||
while (remainingPositions > 0) {
|
||||
final int thisPos = super.nextPosition();
|
||||
remainingPositions--;
|
||||
|
||||
// loop spans forward (if necessary) while the span end is behind thisPos
|
||||
while (spans.endPosition() <= thisPos) {
|
||||
if (spans.nextStartPosition() == Spans.NO_MORE_POSITIONS) { // advance
|
||||
break NEXT_POS_LOOP;
|
||||
}
|
||||
assert spans.docID() == postingsEnum.docID();
|
||||
}
|
||||
|
||||
// is this position within the span?
|
||||
if (thisPos >= spans.startPosition()) {
|
||||
assert thisPos < spans.endPosition(); // guaranteed by previous loop
|
||||
return thisPos; // yay!
|
||||
}
|
||||
// else continue and try the next position
|
||||
}
|
||||
remainingPositions = -1; // signify done
|
||||
return Integer.MAX_VALUE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int startOffset() throws IOException {
|
||||
return remainingPositions >= 0 ? super.startOffset() : Integer.MAX_VALUE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int endOffset() throws IOException {
|
||||
return remainingPositions >= 0 ? super.endOffset() : Integer.MAX_VALUE;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* Simple HashSet that filters out Terms not matching a desired field on {@code add()}.
|
||||
*/
|
||||
private static class FieldFilteringTermHashSet extends HashSet<Term> {
|
||||
private final String field;
|
||||
|
||||
FieldFilteringTermHashSet(String field) {
|
||||
this.field = field;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean add(Term term) {
|
||||
if (term.field().equals(field)) {
|
||||
return super.add(term);
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A single {@link Spans} view over multiple spans. At least one span is mandatory, but you should probably
|
||||
* supply more than one. Furthermore, the given spans are expected to be positioned to a document already
|
||||
* via a call to next or advance).
|
||||
*/ // TODO move to Lucene core as a Spans utility class?
|
||||
static class MultiSpans extends Spans {
|
||||
final PriorityQueue<Spans> spansQueue = new PriorityQueue<>(SPANS_COMPARATOR);
|
||||
long cost;
|
||||
|
||||
void addSpans(Spans spans) {
|
||||
if (spans.docID() < 0 || spans.docID() == NO_MORE_DOCS) {
|
||||
throw new IllegalArgumentException("Expecting given spans to be in a positioned state.");
|
||||
}
|
||||
spansQueue.add(spans);
|
||||
cost = Math.max(cost, spans.cost());
|
||||
}
|
||||
|
||||
// DocIdSetIterator methods:
|
||||
|
||||
@Override
|
||||
public int nextDoc() throws IOException {
|
||||
if (spansQueue.isEmpty()) {
|
||||
return NO_MORE_DOCS;
|
||||
}
|
||||
return advance(spansQueue.peek().docID() + 1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int advance(int target) throws IOException {
|
||||
if (spansQueue.isEmpty()) {
|
||||
return NO_MORE_DOCS;
|
||||
}
|
||||
while (true) {
|
||||
Spans spans = spansQueue.peek();
|
||||
if (spans.docID() >= target) {
|
||||
return spans.docID();
|
||||
}
|
||||
spansQueue.remove(); // must remove before modify state
|
||||
if (spans.advance(target) != NO_MORE_DOCS) { // ... otherwise it's not re-added
|
||||
spansQueue.add(spans);
|
||||
} else if (spansQueue.isEmpty()) {
|
||||
return NO_MORE_DOCS;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
if (spansQueue.isEmpty()) {
|
||||
return NO_MORE_DOCS;
|
||||
}
|
||||
return spansQueue.peek().docID();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long cost() {
|
||||
return cost;
|
||||
}
|
||||
|
||||
// Spans methods:
|
||||
|
||||
@Override
|
||||
public int nextStartPosition() throws IOException {
|
||||
// advance any spans at the initial position per document
|
||||
boolean atDocStart = false;
|
||||
while (spansQueue.peek().startPosition() == -1) {
|
||||
atDocStart = true;
|
||||
Spans headSpans = spansQueue.remove(); // remove because we will change state
|
||||
headSpans.nextStartPosition();
|
||||
spansQueue.add(headSpans);
|
||||
}
|
||||
if (!atDocStart) {
|
||||
Spans headSpans = spansQueue.remove(); // remove because we will change state
|
||||
headSpans.nextStartPosition();
|
||||
spansQueue.add(headSpans);
|
||||
}
|
||||
return startPosition();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int startPosition() {
|
||||
return spansQueue.peek().startPosition();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int endPosition() {
|
||||
return spansQueue.peek().endPosition();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int width() {
|
||||
return spansQueue.peek().width();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void collect(SpanCollector collector) throws IOException {
|
||||
spansQueue.peek().collect(collector);
|
||||
}
|
||||
|
||||
@Override
|
||||
public float positionsCost() {
|
||||
return 100f;// no idea; and we can't delegate due to not allowing to call it dependent on TwoPhaseIterator
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A Spans based on a list of cached spans for one doc. It is pre-positioned to this doc.
|
||||
*/
|
||||
private static class CachedSpans extends Spans {
|
||||
|
||||
private static class CachedSpan {
|
||||
final int start;
|
||||
final int end;
|
||||
|
||||
CachedSpan(int start, int end) {
|
||||
this.start = start;
|
||||
this.end = end;
|
||||
}
|
||||
}
|
||||
|
||||
final int docId;
|
||||
final ArrayList<CachedSpan> cachedSpanList;
|
||||
int index = -1;
|
||||
|
||||
CachedSpans(Spans spans) throws IOException {
|
||||
this.docId = spans.docID();
|
||||
assert this.docId != -1;
|
||||
// Consume the spans for this doc into a list. There's always at least one; the first/current one.
|
||||
cachedSpanList = new ArrayList<>();
|
||||
while (spans.nextStartPosition() != NO_MORE_POSITIONS) {
|
||||
cachedSpanList.add(new CachedSpan(spans.startPosition(), spans.endPosition()));
|
||||
}
|
||||
assert !cachedSpanList.isEmpty(); // bad Span impl?
|
||||
}
|
||||
|
||||
/**
|
||||
* Clone; reset iteration state.
|
||||
*/
|
||||
CachedSpans(CachedSpans cloneMe) {
|
||||
docId = cloneMe.docId;
|
||||
cachedSpanList = cloneMe.cachedSpanList;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextDoc() throws IOException {
|
||||
throw new UnsupportedOperationException("Not expected");
|
||||
}
|
||||
|
||||
@Override
|
||||
public int advance(int target) throws IOException {
|
||||
throw new UnsupportedOperationException("Not expected");
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return docId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long cost() {
|
||||
return 1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextStartPosition() throws IOException {
|
||||
index++;
|
||||
return startPosition();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int startPosition() {
|
||||
return index < 0 ?
|
||||
-1 : index >= cachedSpanList.size() ?
|
||||
NO_MORE_POSITIONS : cachedSpanList.get(index).start;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int endPosition() {
|
||||
return index < 0 ?
|
||||
-1 : index >= cachedSpanList.size() ?
|
||||
NO_MORE_POSITIONS : cachedSpanList.get(index).end;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int width() {
|
||||
return endPosition() - startPosition();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void collect(SpanCollector collector) throws IOException {
|
||||
throw new UnsupportedOperationException("Not expected");
|
||||
}
|
||||
|
||||
@Override
|
||||
public float positionsCost() {
|
||||
return 1f;
|
||||
}
|
||||
|
||||
} // class CachedSpans
|
||||
}
|
|
@ -0,0 +1,61 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.search.uhighlight;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.index.IndexOptions;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.ReaderUtil;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.automaton.CharacterRunAutomaton;
|
||||
|
||||
/**
|
||||
* Uses offsets in postings -- {@link IndexOptions#DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS}. This
|
||||
* does not support multi-term queries; the highlighter will fallback on analysis for that.
|
||||
*
|
||||
* @lucene.internal
|
||||
*/
|
||||
public class PostingsOffsetStrategy extends FieldOffsetStrategy {
|
||||
|
||||
public PostingsOffsetStrategy(String field, BytesRef[] queryTerms, PhraseHelper phraseHelper, CharacterRunAutomaton[] automata) {
|
||||
super(field, queryTerms, phraseHelper, automata);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<OffsetsEnum> getOffsetsEnums(IndexReader reader, int docId, String content) throws IOException {
|
||||
LeafReader leafReader;
|
||||
if (reader instanceof LeafReader) {
|
||||
leafReader = (LeafReader) reader;
|
||||
} else {
|
||||
List<LeafReaderContext> leaves = reader.leaves();
|
||||
LeafReaderContext leafReaderContext = leaves.get(ReaderUtil.subIndex(docId, leaves));
|
||||
leafReader = leafReaderContext.reader();
|
||||
docId -= leafReaderContext.docBase; // adjust 'doc' to be within this leaf reader
|
||||
}
|
||||
|
||||
return createOffsetsEnumsFromReader(leafReader, docId);
|
||||
}
|
||||
|
||||
@Override
|
||||
public UnifiedHighlighter.OffsetSource getOffsetSource() {
|
||||
return UnifiedHighlighter.OffsetSource.POSTINGS;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,71 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.search.uhighlight;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.ReaderUtil;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.automaton.CharacterRunAutomaton;
|
||||
|
||||
/**
|
||||
* Like {@link PostingsOffsetStrategy} but also uses term vectors (only terms needed) for multi-term queries.
|
||||
*
|
||||
* @lucene.internal
|
||||
*/
|
||||
public class PostingsWithTermVectorsOffsetStrategy extends FieldOffsetStrategy {
|
||||
|
||||
public PostingsWithTermVectorsOffsetStrategy(String field, BytesRef[] queryTerms, PhraseHelper phraseHelper, CharacterRunAutomaton[] automata) {
|
||||
super(field, queryTerms, phraseHelper, automata);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<OffsetsEnum> getOffsetsEnums(IndexReader reader, int docId, String content) throws IOException {
|
||||
LeafReader leafReader;
|
||||
if (reader instanceof LeafReader) {
|
||||
leafReader = (LeafReader) reader;
|
||||
} else {
|
||||
List<LeafReaderContext> leaves = reader.leaves();
|
||||
LeafReaderContext LeafReaderContext = leaves.get(ReaderUtil.subIndex(docId, leaves));
|
||||
leafReader = LeafReaderContext.reader();
|
||||
docId -= LeafReaderContext.docBase; // adjust 'doc' to be within this atomic reader
|
||||
}
|
||||
|
||||
Terms docTerms = leafReader.getTermVector(docId, field);
|
||||
if (docTerms == null) {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
leafReader = new TermVectorFilteredLeafReader(leafReader, docTerms);
|
||||
|
||||
TokenStream tokenStream = automata.length > 0 ? MultiTermHighlighting
|
||||
.uninvertAndFilterTerms(leafReader.terms(field), docId, this.automata, content.length()) : null;
|
||||
|
||||
return createOffsetsEnums(leafReader, docId, tokenStream);
|
||||
}
|
||||
|
||||
@Override
|
||||
public UnifiedHighlighter.OffsetSource getOffsetSource() {
|
||||
return UnifiedHighlighter.OffsetSource.POSTINGS;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,244 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.search.uhighlight;
|
||||
|
||||
import java.text.BreakIterator;
|
||||
import java.text.CharacterIterator;
|
||||
import java.text.StringCharacterIterator;
|
||||
|
||||
/**
|
||||
* Virtually slices the text on both sides of every occurrence of the specified character. If the slice is 0-length
|
||||
* which happens for adjacent slice characters or when they are at the beginning or end, that character is reported
|
||||
* as a boundary.
|
||||
* For every slice between the specified characters, it is further processed with a specified
|
||||
* BreakIterator. A consequence is that the enclosed BreakIterator will never "see" the splitting character.
|
||||
* <br>
|
||||
* <em>Note: {@link #setText(CharacterIterator)} is unsupported. Use the string version.</em>
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class SplittingBreakIterator extends BreakIterator {
|
||||
private final BreakIterator baseIter;
|
||||
private final char sliceChar;
|
||||
|
||||
private String text;
|
||||
private int sliceStartIdx;
|
||||
private int sliceEndIdx;
|
||||
private int current;
|
||||
|
||||
public SplittingBreakIterator(BreakIterator baseIter, char sliceChar) {
|
||||
this.baseIter = baseIter;
|
||||
this.sliceChar = sliceChar;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setText(CharacterIterator newText) {
|
||||
throw new UnsupportedOperationException("unexpected");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setText(String newText) {
|
||||
this.text = newText;
|
||||
first();
|
||||
}
|
||||
|
||||
@Override
|
||||
public CharacterIterator getText() {
|
||||
StringCharacterIterator charIter = new StringCharacterIterator(text);
|
||||
// API doesn't say what the state should be but it should probably be at the current index.
|
||||
charIter.setIndex(current());
|
||||
return charIter;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int current() {
|
||||
assert current != DONE;
|
||||
return current; // MUST be updated by the other methods when result isn't DONE.
|
||||
}
|
||||
|
||||
@Override
|
||||
public int first() {
|
||||
sliceStartIdx = 0;
|
||||
sliceEndIdx = text.indexOf(sliceChar);
|
||||
if (sliceEndIdx == -1) {
|
||||
sliceEndIdx = text.length();
|
||||
}
|
||||
if (sliceStartIdx == sliceEndIdx) {
|
||||
return current = sliceStartIdx;
|
||||
}
|
||||
baseIter.setText(text.substring(sliceStartIdx, sliceEndIdx));
|
||||
return current = sliceStartIdx + baseIter.current();// since setText() sets to first(), just grab current()
|
||||
}
|
||||
|
||||
@Override
|
||||
public int last() {
|
||||
sliceEndIdx = text.length();
|
||||
sliceStartIdx = text.lastIndexOf(sliceChar);
|
||||
if (sliceStartIdx == -1) {
|
||||
sliceStartIdx = 0;
|
||||
} else {
|
||||
sliceStartIdx++;//past sliceChar
|
||||
}
|
||||
if (sliceEndIdx == sliceStartIdx) {
|
||||
return current = sliceEndIdx;
|
||||
}
|
||||
baseIter.setText(text.substring(sliceStartIdx, sliceEndIdx));
|
||||
return current = sliceStartIdx + baseIter.last();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int next() {
|
||||
int prevCurrent = current;
|
||||
current = sliceStartIdx == sliceEndIdx ? DONE : baseIter.next();
|
||||
if (current != DONE) {
|
||||
return current = current + sliceStartIdx;
|
||||
}
|
||||
if (sliceEndIdx >= text.length()) {
|
||||
current = prevCurrent;//keep current where it is
|
||||
return DONE;
|
||||
}
|
||||
sliceStartIdx = sliceEndIdx + 1;
|
||||
sliceEndIdx = text.indexOf(sliceChar, sliceStartIdx);
|
||||
if (sliceEndIdx == -1) {
|
||||
sliceEndIdx = text.length();
|
||||
}
|
||||
if (sliceStartIdx == sliceEndIdx) {
|
||||
return current = sliceStartIdx;
|
||||
}
|
||||
baseIter.setText(text.substring(sliceStartIdx, sliceEndIdx));
|
||||
return current = sliceStartIdx + baseIter.current();//use current() since at first() already
|
||||
}
|
||||
|
||||
@Override
|
||||
public int previous() { // note: closely follows next() but reversed
|
||||
int prevCurrent = current;
|
||||
current = sliceStartIdx == sliceEndIdx ? DONE : baseIter.previous();
|
||||
if (current != DONE) {
|
||||
return current = current + sliceStartIdx;
|
||||
}
|
||||
if (sliceStartIdx == 0) {
|
||||
current = prevCurrent;//keep current where it is
|
||||
return DONE;
|
||||
}
|
||||
sliceEndIdx = sliceStartIdx - 1;
|
||||
sliceStartIdx = text.lastIndexOf(sliceChar, sliceEndIdx - 1);
|
||||
if (sliceStartIdx == -1) {
|
||||
sliceStartIdx = 0;
|
||||
} else {
|
||||
sliceStartIdx++;//past sliceChar
|
||||
}
|
||||
if (sliceStartIdx == sliceEndIdx) {
|
||||
return current = sliceStartIdx;
|
||||
}
|
||||
baseIter.setText(text.substring(sliceStartIdx, sliceEndIdx));
|
||||
return current = sliceStartIdx + baseIter.last();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int following(int offset) {
|
||||
// if the offset is not in this slice, update the slice
|
||||
if (offset + 1 < sliceStartIdx || offset + 1 > sliceEndIdx) {
|
||||
if (offset == text.length()) { // DONE condition
|
||||
last(); // because https://bugs.openjdk.java.net/browse/JDK-8015110
|
||||
return DONE;
|
||||
}
|
||||
sliceStartIdx = text.lastIndexOf(sliceChar, offset);//no +1
|
||||
if (sliceStartIdx == -1) {
|
||||
sliceStartIdx = 0;
|
||||
} else {
|
||||
sliceStartIdx++;//move past separator
|
||||
}
|
||||
sliceEndIdx = text.indexOf(sliceChar, Math.max(offset + 1, sliceStartIdx));
|
||||
if (sliceEndIdx == -1) {
|
||||
sliceEndIdx = text.length();
|
||||
}
|
||||
if (sliceStartIdx != sliceEndIdx) {//otherwise, adjacent separator or separator at end
|
||||
baseIter.setText(text.substring(sliceStartIdx, sliceEndIdx));
|
||||
}
|
||||
}
|
||||
|
||||
// lookup following() in this slice:
|
||||
if (sliceStartIdx == sliceEndIdx) {
|
||||
return current = offset + 1;
|
||||
} else {
|
||||
// note: following() can never be first() if the first character is a boundary (it usually is).
|
||||
// So we have to check if we should call first() instead of following():
|
||||
if (offset == sliceStartIdx - 1) {
|
||||
// the first boundary following this offset is the very first boundary in this slice
|
||||
return current = sliceStartIdx + baseIter.first();
|
||||
} else {
|
||||
return current = sliceStartIdx + baseIter.following(offset - sliceStartIdx);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int preceding(int offset) { // note: closely follows following() but reversed
|
||||
if (offset - 1 < sliceStartIdx || offset - 1 > sliceEndIdx) {
|
||||
if (offset == 0) { // DONE condition
|
||||
first(); // because https://bugs.openjdk.java.net/browse/JDK-8015110
|
||||
return DONE;
|
||||
}
|
||||
sliceEndIdx = text.indexOf(sliceChar, offset);//no -1
|
||||
if (sliceEndIdx == -1) {
|
||||
sliceEndIdx = text.length();
|
||||
}
|
||||
sliceStartIdx = text.lastIndexOf(sliceChar, offset - 1);
|
||||
if (sliceStartIdx == -1) {
|
||||
sliceStartIdx = 0;
|
||||
} else {
|
||||
sliceStartIdx = Math.min(sliceStartIdx + 1, sliceEndIdx);
|
||||
}
|
||||
if (sliceStartIdx != sliceEndIdx) {//otherwise, adjacent separator or separator at end
|
||||
baseIter.setText(text.substring(sliceStartIdx, sliceEndIdx));
|
||||
}
|
||||
}
|
||||
// lookup preceding() in this slice:
|
||||
if (sliceStartIdx == sliceEndIdx) {
|
||||
return current = offset - 1;
|
||||
} else {
|
||||
// note: preceding() can never be last() if the last character is a boundary (it usually is).
|
||||
// So we have to check if we should call last() instead of preceding():
|
||||
if (offset == sliceEndIdx + 1) {
|
||||
// the last boundary preceding this offset is the very last boundary in this slice
|
||||
return current = sliceStartIdx + baseIter.last();
|
||||
} else {
|
||||
return current = sliceStartIdx + baseIter.preceding(offset - sliceStartIdx);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int next(int n) {
|
||||
if (n < 0) {
|
||||
for (int i = 0; i < -n; i++) {
|
||||
if (previous() == DONE) {
|
||||
return DONE;
|
||||
}
|
||||
}
|
||||
} else {
|
||||
for (int i = 0; i < n; i++) {
|
||||
if (next() == DONE) {
|
||||
return DONE;
|
||||
}
|
||||
}
|
||||
}
|
||||
return current();
|
||||
}
|
||||
|
||||
|
||||
}
|
|
@ -0,0 +1,130 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.search.uhighlight;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.Fields;
|
||||
import org.apache.lucene.index.FilterLeafReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.PostingsEnum;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.automaton.CompiledAutomaton;
|
||||
|
||||
/**
|
||||
* A filtered LeafReader that only includes the terms that are also in a provided set of terms.
|
||||
* Certain methods may be unimplemented or cause large operations on the underlying reader
|
||||
* and be slow.
|
||||
*
|
||||
* @lucene.internal
|
||||
*/
|
||||
final class TermVectorFilteredLeafReader extends FilterLeafReader {
|
||||
// NOTE: super ("in") is baseLeafReader
|
||||
|
||||
private final Terms filterTerms;
|
||||
|
||||
/**
|
||||
* <p>Construct a FilterLeafReader based on the specified base reader.
|
||||
* <p>Note that base reader is closed if this FilterLeafReader is closed.</p>
|
||||
*
|
||||
* @param baseLeafReader full/original reader.
|
||||
* @param filterTerms set of terms to filter by -- probably from a TermVector or MemoryIndex.
|
||||
*/
|
||||
TermVectorFilteredLeafReader(LeafReader baseLeafReader, Terms filterTerms) {
|
||||
super(baseLeafReader);
|
||||
this.filterTerms = filterTerms;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Fields fields() throws IOException {
|
||||
return new TermVectorFilteredFields(in.fields(), filterTerms);
|
||||
}
|
||||
|
||||
private static final class TermVectorFilteredFields extends FilterLeafReader.FilterFields {
|
||||
// NOTE: super ("in") is baseFields
|
||||
|
||||
private final Terms filterTerms;
|
||||
|
||||
TermVectorFilteredFields(Fields baseFields, Terms filterTerms) {
|
||||
super(baseFields);
|
||||
this.filterTerms = filterTerms;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Terms terms(String field) throws IOException {
|
||||
return new TermsFilteredTerms(in.terms(field), filterTerms);
|
||||
}
|
||||
}
|
||||
|
||||
private static final class TermsFilteredTerms extends FilterLeafReader.FilterTerms {
|
||||
// NOTE: super ("in") is the baseTerms
|
||||
|
||||
private final Terms filterTerms;
|
||||
|
||||
TermsFilteredTerms(Terms baseTerms, Terms filterTerms) {
|
||||
super(baseTerms);
|
||||
this.filterTerms = filterTerms;
|
||||
}
|
||||
|
||||
//TODO delegate size() ?
|
||||
|
||||
//TODO delegate getMin, getMax to filterTerms
|
||||
|
||||
@Override
|
||||
public TermsEnum iterator() throws IOException {
|
||||
return new TermVectorFilteredTermsEnum(in.iterator(), filterTerms.iterator());
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermsEnum intersect(CompiledAutomaton compiled, BytesRef startTerm) throws IOException {
|
||||
return new TermVectorFilteredTermsEnum(in.iterator(), filterTerms.intersect(compiled, startTerm));
|
||||
}
|
||||
}
|
||||
|
||||
private static final class TermVectorFilteredTermsEnum extends FilterLeafReader.FilterTermsEnum {
|
||||
// NOTE: super ("in") is the filteredTermsEnum. This is different than wrappers above because we
|
||||
// navigate the terms using the filter.
|
||||
|
||||
//TODO: track the last term state from the term state method and do some potential optimizations
|
||||
private final TermsEnum baseTermsEnum;
|
||||
|
||||
TermVectorFilteredTermsEnum(TermsEnum baseTermsEnum, TermsEnum filteredTermsEnum) {
|
||||
super(filteredTermsEnum); // note this is reversed from constructors above
|
||||
this.baseTermsEnum = baseTermsEnum;
|
||||
}
|
||||
|
||||
//TODO delegate docFreq & ttf (moveToCurrentTerm() then call on full?
|
||||
|
||||
@Override
|
||||
public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
|
||||
moveToCurrentTerm();
|
||||
return baseTermsEnum.postings(reuse, flags);
|
||||
}
|
||||
|
||||
void moveToCurrentTerm() throws IOException {
|
||||
BytesRef currentTerm = in.term(); // from filteredTermsEnum
|
||||
boolean termInBothTermsEnum = baseTermsEnum.seekExact(currentTerm);
|
||||
|
||||
if (!termInBothTermsEnum) {
|
||||
throw new IllegalStateException("Term vector term " + currentTerm + " does not appear in full index.");
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
}
|
|
@ -0,0 +1,68 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.search.uhighlight;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.search.highlight.TermVectorLeafReader;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.automaton.CharacterRunAutomaton;
|
||||
|
||||
/**
|
||||
* Uses term vectors that contain offsets.
|
||||
*
|
||||
* @lucene.internal
|
||||
*/
|
||||
public class TermVectorOffsetStrategy extends FieldOffsetStrategy {
|
||||
|
||||
public TermVectorOffsetStrategy(String field, BytesRef[] queryTerms, PhraseHelper phraseHelper, CharacterRunAutomaton[] automata) {
|
||||
super(field, queryTerms, phraseHelper, automata);
|
||||
}
|
||||
|
||||
@Override
|
||||
public UnifiedHighlighter.OffsetSource getOffsetSource() {
|
||||
return UnifiedHighlighter.OffsetSource.TERM_VECTORS;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<OffsetsEnum> getOffsetsEnums(IndexReader reader, int docId, String content) throws IOException {
|
||||
Terms tvTerms = reader.getTermVector(docId, field);
|
||||
if (tvTerms == null) {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
LeafReader leafReader = null;
|
||||
if ((terms.length > 0) || strictPhrases.willRewrite()) {
|
||||
leafReader = new TermVectorLeafReader(field, tvTerms);
|
||||
docId = 0;
|
||||
}
|
||||
|
||||
TokenStream tokenStream = null;
|
||||
if (automata.length > 0) {
|
||||
tokenStream = MultiTermHighlighting.uninvertAndFilterTerms(tvTerms, 0, automata, content.length());
|
||||
}
|
||||
|
||||
return createOffsetsEnums(leafReader, docId, tokenStream);
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,395 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.search.uhighlight;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
import org.apache.lucene.index.PostingsEnum;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefArray;
|
||||
import org.apache.lucene.util.BytesRefBuilder;
|
||||
import org.apache.lucene.util.CharsRefBuilder;
|
||||
import org.apache.lucene.util.Counter;
|
||||
import org.apache.lucene.util.UnicodeUtil;
|
||||
|
||||
/**
|
||||
* TokenStream created from a term vector field. The term vector requires positions and/or offsets (either). If you
|
||||
* want payloads add PayloadAttributeImpl (as you would normally) but don't assume the attribute is already added just
|
||||
* because you know the term vector has payloads, since the first call to incrementToken() will observe if you asked
|
||||
* for them and if not then won't get them. This TokenStream supports an efficient {@link #reset()}, so there's
|
||||
* no need to wrap with a caching impl.
|
||||
*
|
||||
* @lucene.internal
|
||||
*/
|
||||
final class TokenStreamFromTermVector extends TokenStream {
|
||||
// note: differs from similar class in the standard highlighter. This one is optimized for sparse cases.
|
||||
|
||||
/**
|
||||
* content length divided by distinct positions; an average of dense text.
|
||||
*/
|
||||
private static final double AVG_CHARS_PER_POSITION = 6;
|
||||
|
||||
private static final int INSERTION_SORT_THRESHOLD = 16;
|
||||
|
||||
private final Terms vector;
|
||||
|
||||
private final int filteredDocId;
|
||||
|
||||
private final CharTermAttribute termAttribute;
|
||||
|
||||
private final PositionIncrementAttribute positionIncrementAttribute;
|
||||
|
||||
private final int offsetLength;
|
||||
|
||||
private final float loadFactor;
|
||||
|
||||
private OffsetAttribute offsetAttribute;//maybe null
|
||||
|
||||
private PayloadAttribute payloadAttribute;//maybe null
|
||||
|
||||
private CharsRefBuilder termCharsBuilder;//term data here
|
||||
|
||||
private BytesRefArray payloadsBytesRefArray;//only used when payloadAttribute is non-null
|
||||
private BytesRefBuilder spareBytesRefBuilder;//only used when payloadAttribute is non-null
|
||||
|
||||
private TokenLL firstToken = null; // the head of a linked-list
|
||||
|
||||
private TokenLL incrementToken = null;
|
||||
|
||||
private boolean initialized = false;//lazy
|
||||
|
||||
public TokenStreamFromTermVector(Terms vector, int offsetLength) throws IOException {
|
||||
this(vector, 0, offsetLength, 1f);
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructor.
|
||||
*
|
||||
* @param vector Terms that contains the data for
|
||||
* creating the TokenStream. Must have positions and/or offsets.
|
||||
* @param filteredDocId The docID we will process.
|
||||
* @param offsetLength Supply the character length of the text being uninverted, or a lower value if you don't want
|
||||
* to invert text beyond an offset (in so doing this will act as a filter). If you don't
|
||||
* know the length, pass -1. In conjunction with {@code loadFactor}, it's used to
|
||||
* determine how many buckets to create during uninversion.
|
||||
* It's also used to filter out tokens with a start offset exceeding this value.
|
||||
* @param loadFactor The percent of tokens from the original terms (by position count) that are
|
||||
* expected to be inverted. If they are filtered (e.g.
|
||||
* {@link org.apache.lucene.index.FilterLeafReader.FilterTerms})
|
||||
* then consider using less than 1.0 to avoid wasting space.
|
||||
* 1.0 means all, 1/64th would suggest 1/64th of all tokens coming from vector.
|
||||
*/
|
||||
TokenStreamFromTermVector(Terms vector, int filteredDocId, int offsetLength, float loadFactor) throws IOException {
|
||||
super();
|
||||
this.filteredDocId = filteredDocId;
|
||||
this.offsetLength = offsetLength == Integer.MAX_VALUE ? -1 : offsetLength;
|
||||
if (loadFactor <= 0f || loadFactor > 1f) {
|
||||
throw new IllegalArgumentException("loadFactor should be > 0 and <= 1");
|
||||
}
|
||||
this.loadFactor = loadFactor;
|
||||
assert !hasAttribute(PayloadAttribute.class) : "AttributeFactory shouldn't have payloads *yet*";
|
||||
if (!vector.hasPositions() && !vector.hasOffsets()) {
|
||||
throw new IllegalArgumentException("The term vector needs positions and/or offsets.");
|
||||
}
|
||||
assert vector.hasFreqs();
|
||||
this.vector = vector;
|
||||
termAttribute = addAttribute(CharTermAttribute.class);
|
||||
positionIncrementAttribute = addAttribute(PositionIncrementAttribute.class);
|
||||
}
|
||||
|
||||
public Terms getTermVectorTerms() {
|
||||
return vector;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void reset() throws IOException {
|
||||
incrementToken = null;
|
||||
super.reset();
|
||||
}
|
||||
|
||||
//We delay initialization because we can see which attributes the consumer wants, particularly payloads
|
||||
private void init() throws IOException {
|
||||
assert !initialized;
|
||||
int dpEnumFlags = 0;
|
||||
if (vector.hasOffsets()) {
|
||||
offsetAttribute = addAttribute(OffsetAttribute.class);
|
||||
dpEnumFlags |= PostingsEnum.OFFSETS;
|
||||
}
|
||||
if (vector.hasPayloads() && hasAttribute(PayloadAttribute.class)) {
|
||||
payloadAttribute = getAttribute(PayloadAttribute.class);
|
||||
payloadsBytesRefArray = new BytesRefArray(Counter.newCounter());
|
||||
spareBytesRefBuilder = new BytesRefBuilder();
|
||||
dpEnumFlags |= PostingsEnum.PAYLOADS;
|
||||
}
|
||||
|
||||
// We put term data here
|
||||
termCharsBuilder = new CharsRefBuilder();
|
||||
termCharsBuilder.grow(initTotalTermCharLen());
|
||||
|
||||
// Step 1: iterate termsEnum and create a token, placing into a bucketed array (given a load factor)
|
||||
|
||||
final TokenLL[] tokenBuckets = initTokenBucketsArray();
|
||||
final double OFFSET_TO_BUCKET_IDX = loadFactor / AVG_CHARS_PER_POSITION;
|
||||
final double POSITION_TO_BUCKET_IDX = loadFactor;
|
||||
|
||||
final TermsEnum termsEnum = vector.iterator();
|
||||
BytesRef termBytesRef;
|
||||
PostingsEnum dpEnum = null;
|
||||
final CharsRefBuilder tempCharsRefBuilder = new CharsRefBuilder();//only for UTF8->UTF16 call
|
||||
|
||||
TERM_LOOP:
|
||||
while ((termBytesRef = termsEnum.next()) != null) {
|
||||
//Grab the term (in same way as BytesRef.utf8ToString() but we don't want a String obj)
|
||||
// note: if term vectors supported seek by ord then we might just keep an int and seek by ord on-demand
|
||||
tempCharsRefBuilder.grow(termBytesRef.length);
|
||||
final int termCharsLen = UnicodeUtil.UTF8toUTF16(termBytesRef, tempCharsRefBuilder.chars());
|
||||
final int termCharsOff = termCharsBuilder.length();
|
||||
termCharsBuilder.append(tempCharsRefBuilder.chars(), 0, termCharsLen);
|
||||
dpEnum = termsEnum.postings(dpEnum, dpEnumFlags);
|
||||
assert dpEnum != null; // presumably checked by TokenSources.hasPositions earlier
|
||||
int currentDocId = dpEnum.advance(filteredDocId);
|
||||
if (currentDocId != filteredDocId) {
|
||||
continue; //Not expected
|
||||
}
|
||||
final int freq = dpEnum.freq();
|
||||
for (int j = 0; j < freq; j++) {
|
||||
TokenLL token = new TokenLL();
|
||||
token.position = dpEnum.nextPosition(); // can be -1 if not in the TV
|
||||
token.termCharsOff = termCharsOff;
|
||||
token.termCharsLen = (short) Math.min(termCharsLen, Short.MAX_VALUE);
|
||||
// copy offset (if it's there) and compute bucketIdx
|
||||
int bucketIdx;
|
||||
if (offsetAttribute != null) {
|
||||
token.startOffset = dpEnum.startOffset();
|
||||
if (offsetLength >= 0 && token.startOffset > offsetLength) {
|
||||
continue TERM_LOOP;//filter this token out; exceeds threshold
|
||||
}
|
||||
token.endOffsetInc = (short) Math.min(dpEnum.endOffset() - token.startOffset, Short.MAX_VALUE);
|
||||
bucketIdx = (int) (token.startOffset * OFFSET_TO_BUCKET_IDX);
|
||||
} else {
|
||||
bucketIdx = (int) (token.position * POSITION_TO_BUCKET_IDX);
|
||||
}
|
||||
if (bucketIdx >= tokenBuckets.length) {
|
||||
bucketIdx = tokenBuckets.length - 1;
|
||||
}
|
||||
|
||||
if (payloadAttribute != null) {
|
||||
final BytesRef payload = dpEnum.getPayload();
|
||||
token.payloadIndex = payload == null ? -1 : payloadsBytesRefArray.append(payload);
|
||||
}
|
||||
|
||||
//Add token to the head of the bucket linked list
|
||||
token.next = tokenBuckets[bucketIdx];
|
||||
tokenBuckets[bucketIdx] = token;
|
||||
}
|
||||
}
|
||||
|
||||
// Step 2: Link all Tokens into a linked-list and sort all tokens at the same position
|
||||
|
||||
firstToken = initLinkAndSortTokens(tokenBuckets);
|
||||
|
||||
// If the term vector didn't have positions, synthesize them
|
||||
if (!vector.hasPositions() && firstToken != null) {
|
||||
TokenLL prevToken = firstToken;
|
||||
prevToken.position = 0;
|
||||
for (TokenLL token = prevToken.next; token != null; prevToken = token, token = token.next) {
|
||||
if (prevToken.startOffset == token.startOffset) {
|
||||
token.position = prevToken.position;
|
||||
} else {
|
||||
token.position = prevToken.position + 1;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
initialized = true;
|
||||
}
|
||||
|
||||
private static TokenLL initLinkAndSortTokens(TokenLL[] tokenBuckets) {
|
||||
TokenLL firstToken = null;
|
||||
List<TokenLL> scratchTokenArray = new ArrayList<>(); // declare here for re-use. TODO use native array
|
||||
TokenLL prevToken = null;
|
||||
for (TokenLL tokenHead : tokenBuckets) {
|
||||
if (tokenHead == null) {
|
||||
continue;
|
||||
}
|
||||
//sort tokens at this position and link them; return the first
|
||||
TokenLL tokenTail;
|
||||
// just one token
|
||||
if (tokenHead.next == null) {
|
||||
tokenTail = tokenHead;
|
||||
} else {
|
||||
// add the linked list to a temporary array
|
||||
for (TokenLL cur = tokenHead; cur != null; cur = cur.next) {
|
||||
scratchTokenArray.add(cur);
|
||||
}
|
||||
// sort; and set tokenHead & tokenTail
|
||||
if (scratchTokenArray.size() < INSERTION_SORT_THRESHOLD) {
|
||||
// insertion sort by creating a linked list (leave scratchTokenArray alone)
|
||||
tokenHead = tokenTail = scratchTokenArray.get(0);
|
||||
tokenHead.next = null;
|
||||
for (int i = 1; i < scratchTokenArray.size(); i++) {
|
||||
TokenLL insertToken = scratchTokenArray.get(i);
|
||||
if (insertToken.compareTo(tokenHead) <= 0) {
|
||||
// takes the place of tokenHead
|
||||
insertToken.next = tokenHead;
|
||||
tokenHead = insertToken;
|
||||
} else {
|
||||
// goes somewhere after tokenHead
|
||||
for (TokenLL prev = tokenHead; true; prev = prev.next) {
|
||||
if (prev.next == null || insertToken.compareTo(prev.next) <= 0) {
|
||||
if (prev.next == null) {
|
||||
tokenTail = insertToken;
|
||||
}
|
||||
insertToken.next = prev.next;
|
||||
prev.next = insertToken;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
Collections.sort(scratchTokenArray);
|
||||
// take back out and create a linked list
|
||||
TokenLL prev = tokenHead = scratchTokenArray.get(0);
|
||||
for (int i = 1; i < scratchTokenArray.size(); i++) {
|
||||
prev.next = scratchTokenArray.get(i);
|
||||
prev = prev.next;
|
||||
}
|
||||
tokenTail = prev;
|
||||
tokenTail.next = null;
|
||||
}
|
||||
scratchTokenArray.clear();//too bad ArrayList nulls it out; we don't actually need that
|
||||
}
|
||||
|
||||
//link to previous
|
||||
if (prevToken != null) {
|
||||
assert prevToken.next == null;
|
||||
prevToken.next = tokenHead; //concatenate linked-list
|
||||
assert prevToken.compareTo(tokenHead) < 0 : "wrong offset / position ordering expectations";
|
||||
} else {
|
||||
assert firstToken == null;
|
||||
firstToken = tokenHead;
|
||||
}
|
||||
|
||||
prevToken = tokenTail;
|
||||
}
|
||||
return firstToken;
|
||||
}
|
||||
|
||||
private int initTotalTermCharLen() throws IOException {
|
||||
int guessNumTerms;
|
||||
if (vector.size() != -1) {
|
||||
guessNumTerms = (int) vector.size();
|
||||
} else if (offsetLength != -1) {
|
||||
guessNumTerms = (int) (offsetLength * 0.33);//guess 1/3rd
|
||||
} else {
|
||||
return 128;
|
||||
}
|
||||
return Math.max(64, (int) (guessNumTerms * loadFactor * 7.0));//7 is over-estimate of average term len
|
||||
}
|
||||
|
||||
private TokenLL[] initTokenBucketsArray() throws IOException {
|
||||
// Estimate the number of non-empty positions (number of tokens, excluding same-position synonyms).
|
||||
int positionsEstimate;
|
||||
if (offsetLength == -1) { // no clue what the char length is.
|
||||
// Estimate the number of position slots we need from term stats based on Wikipedia.
|
||||
int sumTotalTermFreq = (int) vector.getSumTotalTermFreq();
|
||||
if (sumTotalTermFreq == -1) {//unfortunately term vectors seem to not have this stat
|
||||
int size = (int) vector.size();
|
||||
if (size == -1) {//doesn't happen with term vectors, it seems, but pick a default any way
|
||||
size = 128;
|
||||
}
|
||||
sumTotalTermFreq = (int) (size * 2.4);
|
||||
}
|
||||
positionsEstimate = (int) (sumTotalTermFreq * 1.5);//less than 1 in 10 docs exceed this
|
||||
} else {
|
||||
// guess number of token positions by this factor.
|
||||
positionsEstimate = (int) (offsetLength / AVG_CHARS_PER_POSITION);
|
||||
}
|
||||
// apply the load factor.
|
||||
return new TokenLL[Math.max(1, (int) (positionsEstimate * loadFactor))];
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean incrementToken() throws IOException {
|
||||
int posInc;
|
||||
if (incrementToken == null) {
|
||||
if (!initialized) {
|
||||
init();
|
||||
assert initialized;
|
||||
}
|
||||
incrementToken = firstToken;
|
||||
if (incrementToken == null) {
|
||||
return false;
|
||||
}
|
||||
posInc = incrementToken.position + 1;//first token normally has pos 0; add 1 to get posInc
|
||||
} else if (incrementToken.next != null) {
|
||||
int lastPosition = incrementToken.position;
|
||||
incrementToken = incrementToken.next;
|
||||
posInc = incrementToken.position - lastPosition;
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
clearAttributes();
|
||||
termAttribute.copyBuffer(termCharsBuilder.chars(), incrementToken.termCharsOff, incrementToken.termCharsLen);
|
||||
|
||||
positionIncrementAttribute.setPositionIncrement(posInc);
|
||||
if (offsetAttribute != null) {
|
||||
offsetAttribute.setOffset(incrementToken.startOffset, incrementToken.startOffset + incrementToken.endOffsetInc);
|
||||
}
|
||||
if (payloadAttribute != null && incrementToken.payloadIndex >= 0) {
|
||||
payloadAttribute.setPayload(payloadsBytesRefArray.get(spareBytesRefBuilder, incrementToken.payloadIndex));
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
private static class TokenLL implements Comparable<TokenLL> {
|
||||
// This class should weigh 32 bytes, including object header
|
||||
|
||||
int termCharsOff; // see termCharsBuilder
|
||||
short termCharsLen;
|
||||
|
||||
int position;
|
||||
int startOffset;
|
||||
short endOffsetInc; // add to startOffset to get endOffset
|
||||
int payloadIndex;
|
||||
|
||||
TokenLL next;
|
||||
|
||||
@Override
|
||||
public int compareTo(TokenLL tokenB) {
|
||||
int cmp = Integer.compare(this.position, tokenB.position);
|
||||
if (cmp == 0) {
|
||||
cmp = Integer.compare(this.startOffset, tokenB.startOffset);
|
||||
if (cmp == 0) {
|
||||
cmp = Short.compare(this.endOffsetInc, tokenB.endOffsetInc);
|
||||
}
|
||||
}
|
||||
return cmp;
|
||||
}
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -0,0 +1,22 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
/**
|
||||
* The UnifiedHighlighter -- a flexible highlighter that can get offsets from postings, term vectors, or analysis.
|
||||
* @see org.apache.lucene.search.uhighlight.UnifiedHighlighter
|
||||
*/
|
||||
package org.apache.lucene.search.uhighlight;
|
File diff suppressed because one or more lines are too long
|
@ -0,0 +1,192 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.search.uhighlight;
|
||||
|
||||
import java.text.BreakIterator;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
||||
@LuceneTestCase.SuppressSysoutChecks(bugUrl = "")//Gradle interferes with this Lucene test rule
|
||||
public class TestSplittingBreakIterator extends LuceneTestCase {
|
||||
|
||||
|
||||
private static final BreakIterator LINE_BI = BreakIterator.getLineInstance(Locale.ROOT);
|
||||
private static final BreakIterator SPLIT_BI = new SplittingBreakIterator(LINE_BI, '|');
|
||||
|
||||
public void testLineBreakIterator() {
|
||||
testWithoutSplits(LINE_BI);
|
||||
}
|
||||
|
||||
private void testWithoutSplits(BreakIterator bi) {
|
||||
// these tests have no '|'
|
||||
testBreakIterator(bi,
|
||||
" a",
|
||||
"^^^");
|
||||
testBreakIterator(bi,
|
||||
"aa",
|
||||
"^ ^");
|
||||
testBreakIterator(bi,
|
||||
"aa a",
|
||||
"^ ^^");
|
||||
}
|
||||
|
||||
public void testWithoutSplits() {
|
||||
testWithoutSplits(SPLIT_BI);
|
||||
}
|
||||
|
||||
public void testOnlySingleSplitChar() {
|
||||
testBreakIterator(SPLIT_BI,
|
||||
"|",
|
||||
"^^");
|
||||
}
|
||||
|
||||
public void testSplitThenValue() {
|
||||
testBreakIterator(SPLIT_BI,
|
||||
"|a",
|
||||
"^^^");
|
||||
}
|
||||
|
||||
public void testValueThenSplit() {
|
||||
testBreakIterator(SPLIT_BI,
|
||||
"a|",
|
||||
"^^^");
|
||||
}
|
||||
|
||||
public void testValueThenSplitThenValue() {
|
||||
testBreakIterator(SPLIT_BI,
|
||||
"aa|aa",
|
||||
"^ ^^ ^");
|
||||
}
|
||||
|
||||
public void testValueThenDoubleSplitThenValue() {
|
||||
testBreakIterator(SPLIT_BI,
|
||||
"aa||aa",
|
||||
"^ ^^^ ^");
|
||||
}
|
||||
|
||||
public void testValueThenSplitThenDoubleValueThenSplitThenValue() {
|
||||
testBreakIterator(SPLIT_BI,
|
||||
"a|bb cc|d",
|
||||
"^^^ ^ ^^^");
|
||||
}
|
||||
|
||||
private void testBreakIterator(BreakIterator bi, String text, String boundaries) {
|
||||
bi.setText(text);
|
||||
|
||||
//Test first & last
|
||||
testFirstAndLast(bi, text, boundaries);
|
||||
|
||||
//Test if expected boundaries are consistent with reading them from next() in a loop:
|
||||
assertEquals(boundaries, readBoundariesToString(bi, text));
|
||||
|
||||
//Test following() and preceding():
|
||||
// get each index, randomized in case their is a sequencing bug:
|
||||
List<Integer> indexes = randomIntsBetweenInclusive(text.length() + 1);
|
||||
testFollowing(bi, text, boundaries, indexes);
|
||||
testPreceding(bi, text, boundaries, indexes);
|
||||
|
||||
//Test previous():
|
||||
testPrevious(bi, text, boundaries);
|
||||
}
|
||||
|
||||
private void testFirstAndLast(BreakIterator bi, String text, String boundaries) {
|
||||
String message = "Text: " + text;
|
||||
int current = bi.current();
|
||||
assertEquals(message, boundaries.indexOf('^'), current);
|
||||
assertEquals(message, current, bi.first());
|
||||
assertEquals(message, current, bi.current());
|
||||
current = bi.last();
|
||||
assertEquals(boundaries.lastIndexOf('^'), current);
|
||||
assertEquals(message, current, bi.current());
|
||||
}
|
||||
|
||||
private void testFollowing(BreakIterator bi, String text, String boundaries, List<Integer> indexes) {
|
||||
String message = "Text: " + text;
|
||||
for (Integer index : indexes) {
|
||||
int got = bi.following(index);
|
||||
if (index == boundaries.length()) {
|
||||
assertEquals(message, BreakIterator.DONE, got);
|
||||
assertEquals(boundaries.lastIndexOf('^'), bi.current());
|
||||
continue;
|
||||
}
|
||||
assertEquals(message + " index:" + index, boundaries.indexOf('^', index + 1), got);
|
||||
}
|
||||
}
|
||||
|
||||
private void testPreceding(BreakIterator bi, String text, String boundaries, List<Integer> indexes) {
|
||||
String message = "Text: " + text;
|
||||
for (Integer index : indexes) {
|
||||
int got = bi.preceding(index);
|
||||
if (index == 0) {
|
||||
assertEquals(message, BreakIterator.DONE, got);
|
||||
assertEquals(boundaries.indexOf('^'), bi.current());
|
||||
continue;
|
||||
}
|
||||
// if (index == text.length() && got == BreakIterator.DONE) {
|
||||
// continue;//hack to accept faulty default impl of BreakIterator.preceding()
|
||||
// }
|
||||
assertEquals(message + " index:" + index, boundaries.lastIndexOf('^', index - 1), got);
|
||||
}
|
||||
}
|
||||
|
||||
private List<Integer> randomIntsBetweenInclusive(int end) {
|
||||
List<Integer> indexes = new ArrayList<>(end);
|
||||
for (int i = 0; i < end; i++) {
|
||||
indexes.add(i);
|
||||
}
|
||||
Collections.shuffle(indexes, random());
|
||||
return indexes;
|
||||
}
|
||||
|
||||
private void testPrevious(BreakIterator bi, String text, String boundaries) {
|
||||
String message = "Text: " + text;
|
||||
|
||||
bi.setText(text);
|
||||
int idx = bi.last();//position at the end
|
||||
while (true) {
|
||||
idx = boundaries.lastIndexOf('^', idx - 1);
|
||||
if (idx == -1) {
|
||||
assertEquals(message, BreakIterator.DONE, bi.previous());
|
||||
break;
|
||||
}
|
||||
assertEquals(message, idx, bi.previous());
|
||||
}
|
||||
assertEquals(message, boundaries.indexOf('^'), bi.current());//finishes at first
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a string comprised of spaces and '^' only at the boundaries.
|
||||
*/
|
||||
private String readBoundariesToString(BreakIterator bi, String text) {
|
||||
// init markers to spaces
|
||||
StringBuilder markers = new StringBuilder();
|
||||
markers.setLength(text.length() + 1);
|
||||
for (int k = 0; k < markers.length(); k++) {
|
||||
markers.setCharAt(k, ' ');
|
||||
}
|
||||
|
||||
bi.setText(text);
|
||||
for (int boundary = bi.current(); boundary != BreakIterator.DONE; boundary = bi.next()) {
|
||||
markers.setCharAt(boundary, '^');
|
||||
}
|
||||
return markers.toString();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,962 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.search.uhighlight;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStreamReader;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.text.BreakIterator;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
|
||||
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.document.FieldType;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.BooleanClause;
|
||||
import org.apache.lucene.search.BooleanQuery;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.PhraseQuery;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.ScoreDoc;
|
||||
import org.apache.lucene.search.Sort;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.lucene.search.TopDocs;
|
||||
import org.apache.lucene.search.postingshighlight.WholeBreakIterator;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
|
||||
@SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom", "Lucene3x"})
|
||||
@LuceneTestCase.SuppressSysoutChecks(bugUrl = "")//Gradle interferes with this Lucene test rule
|
||||
public class TestUnifiedHighlighter extends LuceneTestCase {
|
||||
|
||||
private final FieldType fieldType; // for "body" generally, but not necessarily others. See constructor
|
||||
|
||||
private MockAnalyzer indexAnalyzer;
|
||||
private Directory dir;
|
||||
|
||||
@ParametersFactory
|
||||
public static Iterable<Object[]> parameters() {
|
||||
return UHTestHelper.parametersFactoryList();
|
||||
}
|
||||
|
||||
public TestUnifiedHighlighter(FieldType fieldType) {
|
||||
this.fieldType = fieldType;
|
||||
}
|
||||
|
||||
@Before
|
||||
public void doBefore() throws IOException {
|
||||
indexAnalyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);//whitespace, punctuation, lowercase
|
||||
dir = newDirectory();
|
||||
}
|
||||
|
||||
@After
|
||||
public void doAfter() throws IOException {
|
||||
dir.close();
|
||||
}
|
||||
|
||||
//
|
||||
// Tests below were ported from the PostingsHighlighter. Possibly augmented. Far below are newer tests.
|
||||
//
|
||||
|
||||
public void testBasics() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Field body = new Field("body", "", fieldType);
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
|
||||
body.setStringValue("This is a test. Just a test highlighting from postings. Feel free to ignore.");
|
||||
iw.addDocument(doc);
|
||||
body.setStringValue("Highlighting the first term. Hope it works.");
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
Query query = new TermQuery(new Term("body", "highlighting"));
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(2, topDocs.totalHits);
|
||||
String snippets[] = highlighter.highlight("body", query, topDocs);
|
||||
assertEquals(2, snippets.length);
|
||||
assertEquals("Just a test <b>highlighting</b> from postings. ", snippets[0]);
|
||||
assertEquals("<b>Highlighting</b> the first term. ", snippets[1]);
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
public void testFormatWithMatchExceedingContentLength2() throws Exception {
|
||||
|
||||
String bodyText = "123 TEST 01234 TEST";
|
||||
|
||||
String[] snippets = formatWithMatchExceedingContentLength(bodyText);
|
||||
|
||||
assertEquals(1, snippets.length);
|
||||
assertEquals("123 <b>TEST</b> 01234 TE", snippets[0]);
|
||||
}
|
||||
|
||||
public void testFormatWithMatchExceedingContentLength3() throws Exception {
|
||||
|
||||
String bodyText = "123 5678 01234 TEST TEST";
|
||||
|
||||
String[] snippets = formatWithMatchExceedingContentLength(bodyText);
|
||||
|
||||
assertEquals(1, snippets.length);
|
||||
assertEquals("123 5678 01234 TE", snippets[0]);
|
||||
}
|
||||
|
||||
public void testFormatWithMatchExceedingContentLength() throws Exception {
|
||||
|
||||
String bodyText = "123 5678 01234 TEST";
|
||||
|
||||
String[] snippets = formatWithMatchExceedingContentLength(bodyText);
|
||||
|
||||
assertEquals(1, snippets.length);
|
||||
// LUCENE-5166: no snippet
|
||||
assertEquals("123 5678 01234 TE", snippets[0]);
|
||||
}
|
||||
|
||||
private String[] formatWithMatchExceedingContentLength(String bodyText) throws IOException {
|
||||
|
||||
int maxLength = 17;
|
||||
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
final Field body = new Field("body", bodyText, fieldType);
|
||||
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
|
||||
Query query = new TermQuery(new Term("body", "test"));
|
||||
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(1, topDocs.totalHits);
|
||||
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
highlighter.setMaxLength(maxLength);
|
||||
String snippets[] = highlighter.highlight("body", query, topDocs);
|
||||
|
||||
|
||||
ir.close();
|
||||
return snippets;
|
||||
}
|
||||
|
||||
// simple test highlighting last word.
|
||||
public void testHighlightLastWord() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Field body = new Field("body", "", fieldType);
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
|
||||
body.setStringValue("This is a test");
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
Query query = new TermQuery(new Term("body", "test"));
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(1, topDocs.totalHits);
|
||||
String snippets[] = highlighter.highlight("body", query, topDocs);
|
||||
assertEquals(1, snippets.length);
|
||||
assertEquals("This is a <b>test</b>", snippets[0]);
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
// simple test with one sentence documents.
|
||||
public void testOneSentence() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Field body = new Field("body", "", fieldType);
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
|
||||
body.setStringValue("This is a test.");
|
||||
iw.addDocument(doc);
|
||||
body.setStringValue("Test a one sentence document.");
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
Query query = new TermQuery(new Term("body", "test"));
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(2, topDocs.totalHits);
|
||||
String snippets[] = highlighter.highlight("body", query, topDocs);
|
||||
assertEquals(2, snippets.length);
|
||||
assertEquals("This is a <b>test</b>.", snippets[0]);
|
||||
assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
// simple test with multiple values that make a result longer than maxLength.
|
||||
public void testMaxLengthWithMultivalue() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Document doc = new Document();
|
||||
|
||||
final String value = "This is a multivalued field. Sentencetwo field.";
|
||||
doc.add(new Field("body", value, fieldType));
|
||||
doc.add(new Field("body", value, fieldType));
|
||||
doc.add(new Field("body", value, fieldType));
|
||||
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
highlighter.setMaxLength(value.length() * 2 + 1);
|
||||
Query query = new TermQuery(new Term("body", "field"));
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(1, topDocs.totalHits);
|
||||
String snippets[] = highlighter.highlight("body", query, topDocs, 10);
|
||||
assertEquals(1, snippets.length);
|
||||
String highlightedValue = "This is a multivalued <b>field</b>. Sentencetwo <b>field</b>.";
|
||||
assertEquals(highlightedValue + "... " + highlightedValue, snippets[0]);
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
public void testMultipleFields() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Field body = new Field("body", "", fieldType);
|
||||
Field title = new Field("title", "", UHTestHelper.randomFieldType(random()));
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
doc.add(title);
|
||||
|
||||
body.setStringValue("This is a test. Just a test highlighting from postings. Feel free to ignore.");
|
||||
title.setStringValue("I am hoping for the best.");
|
||||
iw.addDocument(doc);
|
||||
body.setStringValue("Highlighting the first term. Hope it works.");
|
||||
title.setStringValue("But best may not be good enough.");
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
BooleanQuery query = new BooleanQuery.Builder()
|
||||
.add(new TermQuery(new Term("body", "highlighting")), BooleanClause.Occur.SHOULD)
|
||||
.add(new TermQuery(new Term("title", "best")), BooleanClause.Occur.SHOULD)
|
||||
.build();
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(2, topDocs.totalHits);
|
||||
Map<String, String[]> snippets = highlighter.highlightFields(new String[]{"body", "title"}, query, topDocs);
|
||||
assertEquals(2, snippets.size());
|
||||
assertEquals("Just a test <b>highlighting</b> from postings. ", snippets.get("body")[0]);
|
||||
assertEquals("<b>Highlighting</b> the first term. ", snippets.get("body")[1]);
|
||||
assertEquals("I am hoping for the <b>best</b>.", snippets.get("title")[0]);
|
||||
assertEquals("But <b>best</b> may not be good enough.", snippets.get("title")[1]);
|
||||
ir.close();
|
||||
}
|
||||
|
||||
public void testMultipleTerms() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Field body = new Field("body", "", fieldType);
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
|
||||
body.setStringValue("This is a test. Just a test highlighting from postings. Feel free to ignore.");
|
||||
iw.addDocument(doc);
|
||||
body.setStringValue("Highlighting the first term. Hope it works.");
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
BooleanQuery query = new BooleanQuery.Builder()
|
||||
.add(new TermQuery(new Term("body", "highlighting")), BooleanClause.Occur.SHOULD)
|
||||
.add(new TermQuery(new Term("body", "just")), BooleanClause.Occur.SHOULD)
|
||||
.add(new TermQuery(new Term("body", "first")), BooleanClause.Occur.SHOULD)
|
||||
.build();
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(2, topDocs.totalHits);
|
||||
String snippets[] = highlighter.highlight("body", query, topDocs);
|
||||
assertEquals(2, snippets.length);
|
||||
assertEquals("<b>Just</b> a test <b>highlighting</b> from postings. ", snippets[0]);
|
||||
assertEquals("<b>Highlighting</b> the <b>first</b> term. ", snippets[1]);
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
public void testMultiplePassages() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Field body = new Field("body", "", fieldType);
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
|
||||
body.setStringValue("This is a test. Just a test highlighting from postings. Feel free to ignore.");
|
||||
iw.addDocument(doc);
|
||||
body.setStringValue("This test is another test. Not a good sentence. Test test test test.");
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
Query query = new TermQuery(new Term("body", "test"));
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(2, topDocs.totalHits);
|
||||
String snippets[] = highlighter.highlight("body", query, topDocs, 2);
|
||||
assertEquals(2, snippets.length);
|
||||
assertEquals("This is a <b>test</b>. Just a <b>test</b> highlighting from postings. ", snippets[0]);
|
||||
assertEquals("This <b>test</b> is another <b>test</b>. ... <b>Test</b> <b>test</b> <b>test</b> <b>test</b>.", snippets[1]);
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
public void testBuddhism() throws Exception {
|
||||
String text = "This eight-volume set brings together seminal papers in Buddhist studies from a vast " +
|
||||
"range of academic disciplines published over the last forty years. With a new introduction " +
|
||||
"by the editor, this collection is a unique and unrivalled research resource for both " +
|
||||
"student and scholar. Coverage includes: - Buddhist origins; early history of Buddhism in " +
|
||||
"South and Southeast Asia - early Buddhist Schools and Doctrinal History; Theravada Doctrine " +
|
||||
"- the Origins and nature of Mahayana Buddhism; some Mahayana religious topics - Abhidharma " +
|
||||
"and Madhyamaka - Yogacara, the Epistemological tradition, and Tathagatagarbha - Tantric " +
|
||||
"Buddhism (Including China and Japan); Buddhism in Nepal and Tibet - Buddhism in South and " +
|
||||
"Southeast Asia, and - Buddhism in China, East Asia, and Japan.";
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Field body = new Field("body", text, fieldType);
|
||||
Document document = new Document();
|
||||
document.add(body);
|
||||
iw.addDocument(document);
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
PhraseQuery query = new PhraseQuery.Builder()
|
||||
.add(new Term("body", "buddhist"))
|
||||
.add(new Term("body", "origins"))
|
||||
.build();
|
||||
TopDocs topDocs = searcher.search(query, 10);
|
||||
assertEquals(1, topDocs.totalHits);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
highlighter.setHighlightPhrasesStrictly(false);
|
||||
String snippets[] = highlighter.highlight("body", query, topDocs, 2);
|
||||
assertEquals(1, snippets.length);
|
||||
assertTrue(snippets[0].contains("<b>Buddhist</b> <b>origins</b>"));
|
||||
ir.close();
|
||||
}
|
||||
|
||||
public void testCuriousGeorge() throws Exception {
|
||||
String text = "It’s the formula for success for preschoolers—Curious George and fire trucks! " +
|
||||
"Curious George and the Firefighters is a story based on H. A. and Margret Rey’s " +
|
||||
"popular primate and painted in the original watercolor and charcoal style. " +
|
||||
"Firefighters are a famously brave lot, but can they withstand a visit from one curious monkey?";
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Field body = new Field("body", text, fieldType);
|
||||
Document document = new Document();
|
||||
document.add(body);
|
||||
iw.addDocument(document);
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
PhraseQuery query = new PhraseQuery.Builder()
|
||||
.add(new Term("body", "curious"))
|
||||
.add(new Term("body", "george"))
|
||||
.build();
|
||||
TopDocs topDocs = searcher.search(query, 10);
|
||||
assertEquals(1, topDocs.totalHits);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
highlighter.setHighlightPhrasesStrictly(false);
|
||||
String snippets[] = highlighter.highlight("body", query, topDocs, 2);
|
||||
assertEquals(1, snippets.length);
|
||||
assertFalse(snippets[0].contains("<b>Curious</b>Curious"));
|
||||
ir.close();
|
||||
}
|
||||
|
||||
public void testCambridgeMA() throws Exception {
|
||||
BufferedReader r = new BufferedReader(new InputStreamReader(
|
||||
this.getClass().getResourceAsStream("CambridgeMA.utf8"), StandardCharsets.UTF_8));
|
||||
String text = r.readLine();
|
||||
r.close();
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
Field body = new Field("body", text, fieldType);
|
||||
Document document = new Document();
|
||||
document.add(body);
|
||||
iw.addDocument(document);
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
BooleanQuery query = new BooleanQuery.Builder()
|
||||
.add(new TermQuery(new Term("body", "porter")), BooleanClause.Occur.SHOULD)
|
||||
.add(new TermQuery(new Term("body", "square")), BooleanClause.Occur.SHOULD)
|
||||
.add(new TermQuery(new Term("body", "massachusetts")), BooleanClause.Occur.SHOULD)
|
||||
.build();
|
||||
TopDocs topDocs = searcher.search(query, 10);
|
||||
assertEquals(1, topDocs.totalHits);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
highlighter.setMaxLength(Integer.MAX_VALUE - 1);
|
||||
String snippets[] = highlighter.highlight("body", query, topDocs, 2);
|
||||
assertEquals(1, snippets.length);
|
||||
assertTrue(snippets[0].contains("<b>Square</b>"));
|
||||
assertTrue(snippets[0].contains("<b>Porter</b>"));
|
||||
ir.close();
|
||||
}
|
||||
|
||||
public void testPassageRanking() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Field body = new Field("body", "", fieldType);
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
|
||||
body.setStringValue("This is a test. Just highlighting from postings. This is also a much sillier test. Feel free to test test test test test test test.");
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
Query query = new TermQuery(new Term("body", "test"));
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(1, topDocs.totalHits);
|
||||
String snippets[] = highlighter.highlight("body", query, topDocs, 2);
|
||||
assertEquals(1, snippets.length);
|
||||
assertEquals("This is a <b>test</b>. ... Feel free to <b>test</b> <b>test</b> <b>test</b> <b>test</b> <b>test</b> <b>test</b> <b>test</b>.", snippets[0]);
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
public void testBooleanMustNot() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Field body = new Field("body", "This sentence has both terms. This sentence has only terms.", fieldType);
|
||||
Document document = new Document();
|
||||
document.add(body);
|
||||
iw.addDocument(document);
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
|
||||
BooleanQuery query2 = new BooleanQuery.Builder()
|
||||
.add(new TermQuery(new Term("body", "both")), BooleanClause.Occur.MUST_NOT)
|
||||
.build();
|
||||
|
||||
BooleanQuery query = new BooleanQuery.Builder()
|
||||
.add(new TermQuery(new Term("body", "terms")), BooleanClause.Occur.SHOULD)
|
||||
.add(query2, BooleanClause.Occur.SHOULD)
|
||||
.build();
|
||||
|
||||
TopDocs topDocs = searcher.search(query, 10);
|
||||
assertEquals(1, topDocs.totalHits);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
highlighter.setMaxLength(Integer.MAX_VALUE - 1);
|
||||
String snippets[] = highlighter.highlight("body", query, topDocs, 2);
|
||||
assertEquals(1, snippets.length);
|
||||
assertFalse(snippets[0].contains("<b>both</b>"));
|
||||
ir.close();
|
||||
}
|
||||
|
||||
public void testHighlightAllText() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Field body = new Field("body", "", fieldType);
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
|
||||
body.setStringValue("This is a test. Just highlighting from postings. This is also a much sillier test. Feel free to test test test test test test test.");
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer) {
|
||||
@Override
|
||||
protected BreakIterator getBreakIterator(String field) {
|
||||
return new WholeBreakIterator();
|
||||
}
|
||||
};
|
||||
highlighter.setMaxLength(10000);
|
||||
Query query = new TermQuery(new Term("body", "test"));
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(1, topDocs.totalHits);
|
||||
String snippets[] = highlighter.highlight("body", query, topDocs, 2);
|
||||
assertEquals(1, snippets.length);
|
||||
assertEquals("This is a <b>test</b>. Just highlighting from postings. This is also a much sillier <b>test</b>. Feel free to <b>test</b> <b>test</b> <b>test</b> <b>test</b> <b>test</b> <b>test</b> <b>test</b>.", snippets[0]);
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
public void testSpecificDocIDs() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Field body = new Field("body", "", fieldType);
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
|
||||
body.setStringValue("This is a test. Just a test highlighting from postings. Feel free to ignore.");
|
||||
iw.addDocument(doc);
|
||||
body.setStringValue("Highlighting the first term. Hope it works.");
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
Query query = new TermQuery(new Term("body", "highlighting"));
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(2, topDocs.totalHits);
|
||||
ScoreDoc[] hits = topDocs.scoreDocs;
|
||||
int[] docIDs = new int[2];
|
||||
docIDs[0] = hits[0].doc;
|
||||
docIDs[1] = hits[1].doc;
|
||||
String snippets[] = highlighter.highlightFields(new String[]{"body"}, query, docIDs, new int[]{1}).get("body");
|
||||
assertEquals(2, snippets.length);
|
||||
assertEquals("Just a test <b>highlighting</b> from postings. ", snippets[0]);
|
||||
assertEquals("<b>Highlighting</b> the first term. ", snippets[1]);
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
public void testCustomFieldValueSource() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Document doc = new Document();
|
||||
|
||||
final String text = "This is a test. Just highlighting from postings. This is also a much sillier test. Feel free to test test test test test test test.";
|
||||
Field body = new Field("body", text, fieldType);
|
||||
doc.add(body);
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer) {
|
||||
@Override
|
||||
protected List<CharSequence[]> loadFieldValues(String[] fields,
|
||||
DocIdSetIterator docIter, int cacheCharsThreshold) throws IOException {
|
||||
assert fields.length == 1;
|
||||
assert docIter.cost() == 1;
|
||||
docIter.nextDoc();
|
||||
return Collections.singletonList(new CharSequence[]{text});
|
||||
}
|
||||
|
||||
@Override
|
||||
protected BreakIterator getBreakIterator(String field) {
|
||||
return new WholeBreakIterator();
|
||||
}
|
||||
};
|
||||
|
||||
Query query = new TermQuery(new Term("body", "test"));
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(1, topDocs.totalHits);
|
||||
String snippets[] = highlighter.highlight("body", query, topDocs, 2);
|
||||
assertEquals(1, snippets.length);
|
||||
assertEquals("This is a <b>test</b>. Just highlighting from postings. This is also a much sillier <b>test</b>. Feel free to <b>test</b> <b>test</b> <b>test</b> <b>test</b> <b>test</b> <b>test</b> <b>test</b>.", snippets[0]);
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* Make sure highlighter returns first N sentences if
|
||||
* there were no hits.
|
||||
*/
|
||||
public void testEmptyHighlights() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Document doc = new Document();
|
||||
|
||||
Field body = new Field("body", "test this is. another sentence this test has. far away is that planet.", fieldType);
|
||||
doc.add(body);
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
Query query = new TermQuery(new Term("body", "highlighting"));
|
||||
int[] docIDs = new int[]{0};
|
||||
String snippets[] = highlighter.highlightFields(new String[]{"body"}, query, docIDs, new int[]{2}).get("body");
|
||||
assertEquals(1, snippets.length);
|
||||
assertEquals("test this is. another sentence this test has. ", snippets[0]);
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* Not empty but nothing analyzes. Ensures we address null term-vectors.
|
||||
*/
|
||||
public void testNothingAnalyzes() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Document doc = new Document();
|
||||
doc.add(new Field("body", " ", fieldType));// just a space! (thus not empty)
|
||||
doc.add(newTextField("id", "id", Field.Store.YES));
|
||||
iw.addDocument(doc);
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new Field("body", "something", fieldType));
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
int docID = searcher.search(new TermQuery(new Term("id", "id")), 1).scoreDocs[0].doc;
|
||||
|
||||
Query query = new TermQuery(new Term("body", "highlighting"));
|
||||
int[] docIDs = new int[1];
|
||||
docIDs[0] = docID;
|
||||
String snippets[] = highlighter.highlightFields(new String[]{"body"}, query, docIDs, new int[]{2}).get("body");
|
||||
assertEquals(1, snippets.length);
|
||||
assertEquals(" ", snippets[0]);
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* Make sure highlighter we can customize how emtpy
|
||||
* highlight is returned.
|
||||
*/
|
||||
public void testCustomEmptyHighlights() throws Exception {
|
||||
indexAnalyzer.setPositionIncrementGap(10);
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Document doc = new Document();
|
||||
|
||||
Field body = new Field("body", "test this is. another sentence this test has. far away is that planet.", fieldType);
|
||||
doc.add(body);
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
highlighter.setMaxNoHighlightPassages(0);// don't want any default summary
|
||||
Query query = new TermQuery(new Term("body", "highlighting"));
|
||||
int[] docIDs = new int[]{0};
|
||||
String snippets[] = highlighter.highlightFields(new String[]{"body"}, query, docIDs, new int[]{2}).get("body");
|
||||
assertEquals(1, snippets.length);
|
||||
assertNull(snippets[0]);
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* Make sure highlighter returns whole text when there
|
||||
* are no hits and BreakIterator is null.
|
||||
*/
|
||||
public void testEmptyHighlightsWhole() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Document doc = new Document();
|
||||
|
||||
Field body = new Field("body", "test this is. another sentence this test has. far away is that planet.", fieldType);
|
||||
doc.add(body);
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer) {
|
||||
@Override
|
||||
protected BreakIterator getBreakIterator(String field) {
|
||||
return new WholeBreakIterator();
|
||||
}
|
||||
};
|
||||
Query query = new TermQuery(new Term("body", "highlighting"));
|
||||
int[] docIDs = new int[]{0};
|
||||
String snippets[] = highlighter.highlightFields(new String[]{"body"}, query, docIDs, new int[]{2}).get("body");
|
||||
assertEquals(1, snippets.length);
|
||||
assertEquals("test this is. another sentence this test has. far away is that planet.", snippets[0]);
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* Make sure highlighter is OK with entirely missing
|
||||
* field.
|
||||
*/
|
||||
public void testFieldIsMissing() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Document doc = new Document();
|
||||
|
||||
Field body = new Field("body", "test this is. another sentence this test has. far away is that planet.", fieldType);
|
||||
doc.add(body);
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
Query query = new TermQuery(new Term("bogus", "highlighting"));
|
||||
int[] docIDs = new int[]{0};
|
||||
String snippets[] = highlighter.highlightFields(new String[]{"bogus"}, query, docIDs, new int[]{2}).get("bogus");
|
||||
assertEquals(1, snippets.length);
|
||||
assertNull(snippets[0]);
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
public void testFieldIsJustSpace() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Document doc = new Document();
|
||||
doc.add(new Field("body", " ", fieldType));
|
||||
doc.add(newTextField("id", "id", Field.Store.YES));
|
||||
iw.addDocument(doc);
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new Field("body", "something", fieldType));
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
int docID = searcher.search(new TermQuery(new Term("id", "id")), 1).scoreDocs[0].doc;
|
||||
|
||||
Query query = new TermQuery(new Term("body", "highlighting"));
|
||||
int[] docIDs = new int[1];
|
||||
docIDs[0] = docID;
|
||||
String snippets[] = highlighter.highlightFields(new String[]{"body"}, query, docIDs, new int[]{2}).get("body");
|
||||
assertEquals(1, snippets.length);
|
||||
assertEquals(" ", snippets[0]);
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
public void testFieldIsEmptyString() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Document doc = new Document();
|
||||
doc.add(new Field("body", "", fieldType));
|
||||
doc.add(newTextField("id", "id", Field.Store.YES));
|
||||
iw.addDocument(doc);
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new Field("body", "something", fieldType));
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
int docID = searcher.search(new TermQuery(new Term("id", "id")), 1).scoreDocs[0].doc;
|
||||
|
||||
Query query = new TermQuery(new Term("body", "highlighting"));
|
||||
int[] docIDs = new int[1];
|
||||
docIDs[0] = docID;
|
||||
String snippets[] = highlighter.highlightFields(new String[]{"body"}, query, docIDs, new int[]{2}).get("body");
|
||||
assertEquals(1, snippets.length);
|
||||
assertNull(snippets[0]);
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
public void testMultipleDocs() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
int numDocs = atLeast(100);
|
||||
for (int i = 0; i < numDocs; i++) {
|
||||
Document doc = new Document();
|
||||
String content = "the answer is " + i;
|
||||
if ((i & 1) == 0) {
|
||||
content += " some more terms";
|
||||
}
|
||||
doc.add(new Field("body", content, fieldType));
|
||||
doc.add(newStringField("id", "" + i, Field.Store.YES));
|
||||
iw.addDocument(doc);
|
||||
|
||||
if (random().nextInt(10) == 2) {
|
||||
iw.commit();
|
||||
}
|
||||
}
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
highlighter.setCacheFieldValCharsThreshold(random().nextInt(10) * 10);// 0 thru 90 intervals of 10
|
||||
Query query = new TermQuery(new Term("body", "answer"));
|
||||
TopDocs hits = searcher.search(query, numDocs);
|
||||
assertEquals(numDocs, hits.totalHits);
|
||||
|
||||
String snippets[] = highlighter.highlight("body", query, hits);
|
||||
assertEquals(numDocs, snippets.length);
|
||||
for (int hit = 0; hit < numDocs; hit++) {
|
||||
Document doc = searcher.doc(hits.scoreDocs[hit].doc);
|
||||
int id = Integer.parseInt(doc.get("id"));
|
||||
String expected = "the <b>answer</b> is " + id;
|
||||
if ((id & 1) == 0) {
|
||||
expected += " some more terms";
|
||||
}
|
||||
assertEquals(expected, snippets[hit]);
|
||||
}
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
public void testMultipleSnippetSizes() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Field body = new Field("body", "", fieldType);
|
||||
Field title = new Field("title", "", UHTestHelper.randomFieldType(random()));
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
doc.add(title);
|
||||
|
||||
body.setStringValue("This is a test. Just a test highlighting from postings. Feel free to ignore.");
|
||||
title.setStringValue("This is a test. Just a test highlighting from postings. Feel free to ignore.");
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
BooleanQuery query = new BooleanQuery.Builder()
|
||||
.add(new TermQuery(new Term("body", "test")), BooleanClause.Occur.SHOULD)
|
||||
.add(new TermQuery(new Term("title", "test")), BooleanClause.Occur.SHOULD)
|
||||
.build();
|
||||
Map<String, String[]> snippets = highlighter.highlightFields(new String[]{"title", "body"}, query, new int[]{0}, new int[]{1, 2});
|
||||
String titleHighlight = snippets.get("title")[0];
|
||||
String bodyHighlight = snippets.get("body")[0];
|
||||
assertEquals("This is a <b>test</b>. ", titleHighlight);
|
||||
assertEquals("This is a <b>test</b>. Just a <b>test</b> highlighting from postings. ", bodyHighlight);
|
||||
ir.close();
|
||||
}
|
||||
|
||||
public void testEncode() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Field body = new Field("body", "", fieldType);
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
|
||||
body.setStringValue("This is a test. Just a test highlighting from <i>postings</i>. Feel free to ignore.");
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer) {
|
||||
@Override
|
||||
protected PassageFormatter getFormatter(String field) {
|
||||
return new DefaultPassageFormatter("<b>", "</b>", "... ", true);
|
||||
}
|
||||
};
|
||||
Query query = new TermQuery(new Term("body", "highlighting"));
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(1, topDocs.totalHits);
|
||||
String snippets[] = highlighter.highlight("body", query, topDocs);
|
||||
assertEquals(1, snippets.length);
|
||||
assertEquals("Just a test <b>highlighting</b> from <i>postings</i>. ", snippets[0]);
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
// LUCENE-4906
|
||||
public void testObjectFormatter() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Field body = new Field("body", "", fieldType);
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
|
||||
body.setStringValue("This is a test. Just a test highlighting from postings. Feel free to ignore.");
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer) {
|
||||
@Override
|
||||
protected PassageFormatter getFormatter(String field) {
|
||||
return new PassageFormatter() {
|
||||
PassageFormatter defaultFormatter = new DefaultPassageFormatter();
|
||||
|
||||
@Override
|
||||
public String[] format(Passage passages[], String content) {
|
||||
// Just turns the String snippet into a length 2
|
||||
// array of String
|
||||
return new String[]{"blah blah", defaultFormatter.format(passages, content).toString()};
|
||||
}
|
||||
};
|
||||
}
|
||||
};
|
||||
|
||||
Query query = new TermQuery(new Term("body", "highlighting"));
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(1, topDocs.totalHits);
|
||||
int[] docIDs = new int[1];
|
||||
docIDs[0] = topDocs.scoreDocs[0].doc;
|
||||
Map<String, Object[]> snippets = highlighter.highlightFieldsAsObjects(new String[]{"body"}, query, docIDs, new int[]{1});
|
||||
Object[] bodySnippets = snippets.get("body");
|
||||
assertEquals(1, bodySnippets.length);
|
||||
assertTrue(Arrays.equals(new String[]{"blah blah", "Just a test <b>highlighting</b> from postings. "}, (String[]) bodySnippets[0]));
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,936 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.search.uhighlight;
|
||||
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
|
||||
import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.analysis.MockTokenizer;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.Tokenizer;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.FieldType;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.BooleanClause;
|
||||
import org.apache.lucene.search.BooleanQuery;
|
||||
import org.apache.lucene.search.ConstantScoreQuery;
|
||||
import org.apache.lucene.search.DisjunctionMaxQuery;
|
||||
import org.apache.lucene.search.FuzzyQuery;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.MatchAllDocsQuery;
|
||||
import org.apache.lucene.search.PhraseQuery;
|
||||
import org.apache.lucene.search.PrefixQuery;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.RegexpQuery;
|
||||
import org.apache.lucene.search.Sort;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.lucene.search.TermRangeQuery;
|
||||
import org.apache.lucene.search.TopDocs;
|
||||
import org.apache.lucene.search.WildcardQuery;
|
||||
import org.apache.lucene.search.spans.SpanFirstQuery;
|
||||
import org.apache.lucene.search.spans.SpanMultiTermQueryWrapper;
|
||||
import org.apache.lucene.search.spans.SpanNearQuery;
|
||||
import org.apache.lucene.search.spans.SpanNotQuery;
|
||||
import org.apache.lucene.search.spans.SpanOrQuery;
|
||||
import org.apache.lucene.search.spans.SpanQuery;
|
||||
import org.apache.lucene.search.spans.SpanTermQuery;
|
||||
import org.apache.lucene.store.BaseDirectoryWrapper;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
|
||||
/**
|
||||
* Some tests that highlight wildcard, fuzzy, etc queries.
|
||||
*/
|
||||
@SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom", "Lucene3x"})
|
||||
@LuceneTestCase.SuppressSysoutChecks(bugUrl = "")//Gradle interferes with this Lucene test rule
|
||||
public class TestUnifiedHighlighterMTQ extends LuceneTestCase {
|
||||
|
||||
final FieldType fieldType;
|
||||
|
||||
BaseDirectoryWrapper dir;
|
||||
Analyzer indexAnalyzer;
|
||||
|
||||
@ParametersFactory
|
||||
public static Iterable<Object[]> parameters() {
|
||||
return UHTestHelper.parametersFactoryList();
|
||||
}
|
||||
|
||||
public TestUnifiedHighlighterMTQ(FieldType fieldType) {
|
||||
this.fieldType = fieldType;
|
||||
}
|
||||
|
||||
@Before
|
||||
public void doBefore() throws IOException {
|
||||
dir = newDirectory();
|
||||
indexAnalyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);//whitespace, punctuation, lowercase
|
||||
}
|
||||
|
||||
@After
|
||||
public void doAfter() throws IOException {
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testWildcards() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Field body = new Field("body", "", fieldType);
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
|
||||
body.setStringValue("This is a test.");
|
||||
iw.addDocument(doc);
|
||||
body.setStringValue("Test a one sentence document.");
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
Query query = new WildcardQuery(new Term("body", "te*"));
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(2, topDocs.totalHits);
|
||||
String snippets[] = highlighter.highlight("body", query, topDocs);
|
||||
assertEquals(2, snippets.length);
|
||||
assertEquals("This is a <b>test</b>.", snippets[0]);
|
||||
assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
|
||||
|
||||
// disable MTQ; won't highlight
|
||||
highlighter.setHandleMultiTermQuery(false);
|
||||
snippets = highlighter.highlight("body", query, topDocs);
|
||||
assertEquals(2, snippets.length);
|
||||
assertEquals("This is a test.", snippets[0]);
|
||||
assertEquals("Test a one sentence document.", snippets[1]);
|
||||
highlighter.setHandleMultiTermQuery(true);//reset
|
||||
|
||||
// wrong field
|
||||
BooleanQuery bq = new BooleanQuery.Builder()
|
||||
.add(new MatchAllDocsQuery(), BooleanClause.Occur.SHOULD)
|
||||
.add(new WildcardQuery(new Term("bogus", "te*")), BooleanClause.Occur.SHOULD)
|
||||
.build();
|
||||
topDocs = searcher.search(bq, 10, Sort.INDEXORDER);
|
||||
assertEquals(2, topDocs.totalHits);
|
||||
snippets = highlighter.highlight("body", bq, topDocs);
|
||||
assertEquals(2, snippets.length);
|
||||
assertEquals("This is a test.", snippets[0]);
|
||||
assertEquals("Test a one sentence document.", snippets[1]);
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
public void testOnePrefix() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Field body = new Field("body", "", fieldType);
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
|
||||
body.setStringValue("This is a test.");
|
||||
iw.addDocument(doc);
|
||||
body.setStringValue("Test a one sentence document.");
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
Query query = new PrefixQuery(new Term("body", "te"));
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(2, topDocs.totalHits);
|
||||
String snippets[] = highlighter.highlight("body", query, topDocs);
|
||||
assertEquals(2, snippets.length);
|
||||
assertEquals("This is a <b>test</b>.", snippets[0]);
|
||||
assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
|
||||
|
||||
// wrong field
|
||||
BooleanQuery bq = new BooleanQuery.Builder()
|
||||
.add(new MatchAllDocsQuery(), BooleanClause.Occur.SHOULD)
|
||||
.add(new PrefixQuery(new Term("bogus", "te")), BooleanClause.Occur.SHOULD)
|
||||
.build();
|
||||
topDocs = searcher.search(bq, 10, Sort.INDEXORDER);
|
||||
assertEquals(2, topDocs.totalHits);
|
||||
snippets = highlighter.highlight("body", bq, topDocs);
|
||||
assertEquals(2, snippets.length);
|
||||
assertEquals("This is a test.", snippets[0]);
|
||||
assertEquals("Test a one sentence document.", snippets[1]);
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
public void testOneRegexp() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Field body = new Field("body", "", fieldType);
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
|
||||
body.setStringValue("This is a test.");
|
||||
iw.addDocument(doc);
|
||||
body.setStringValue("Test a one sentence document.");
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
Query query = new RegexpQuery(new Term("body", "te.*"));
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(2, topDocs.totalHits);
|
||||
String snippets[] = highlighter.highlight("body", query, topDocs);
|
||||
assertEquals(2, snippets.length);
|
||||
assertEquals("This is a <b>test</b>.", snippets[0]);
|
||||
assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
|
||||
|
||||
// wrong field
|
||||
BooleanQuery bq = new BooleanQuery.Builder()
|
||||
.add(new MatchAllDocsQuery(), BooleanClause.Occur.SHOULD)
|
||||
.add(new RegexpQuery(new Term("bogus", "te.*")), BooleanClause.Occur.SHOULD)
|
||||
.build();
|
||||
topDocs = searcher.search(bq, 10, Sort.INDEXORDER);
|
||||
assertEquals(2, topDocs.totalHits);
|
||||
snippets = highlighter.highlight("body", bq, topDocs);
|
||||
assertEquals(2, snippets.length);
|
||||
assertEquals("This is a test.", snippets[0]);
|
||||
assertEquals("Test a one sentence document.", snippets[1]);
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
public void testOneFuzzy() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Field body = new Field("body", "", fieldType);
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
|
||||
body.setStringValue("This is a test.");
|
||||
iw.addDocument(doc);
|
||||
body.setStringValue("Test a one sentence document.");
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
Query query = new FuzzyQuery(new Term("body", "tets"), 1);
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(2, topDocs.totalHits);
|
||||
String snippets[] = highlighter.highlight("body", query, topDocs);
|
||||
assertEquals(2, snippets.length);
|
||||
assertEquals("This is a <b>test</b>.", snippets[0]);
|
||||
assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
|
||||
|
||||
// with prefix
|
||||
query = new FuzzyQuery(new Term("body", "tets"), 1, 2);
|
||||
topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(2, topDocs.totalHits);
|
||||
snippets = highlighter.highlight("body", query, topDocs);
|
||||
assertEquals(2, snippets.length);
|
||||
assertEquals("This is a <b>test</b>.", snippets[0]);
|
||||
assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
|
||||
|
||||
// wrong field
|
||||
BooleanQuery bq = new BooleanQuery.Builder()
|
||||
.add(new MatchAllDocsQuery(), BooleanClause.Occur.SHOULD)
|
||||
.add(new FuzzyQuery(new Term("bogus", "tets"), 1), BooleanClause.Occur.SHOULD)
|
||||
.build();
|
||||
topDocs = searcher.search(bq, 10, Sort.INDEXORDER);
|
||||
assertEquals(2, topDocs.totalHits);
|
||||
snippets = highlighter.highlight("body", bq, topDocs);
|
||||
assertEquals(2, snippets.length);
|
||||
assertEquals("This is a test.", snippets[0]);
|
||||
assertEquals("Test a one sentence document.", snippets[1]);
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
public void testRanges() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Field body = new Field("body", "", fieldType);
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
|
||||
body.setStringValue("This is a test.");
|
||||
iw.addDocument(doc);
|
||||
body.setStringValue("Test a one sentence document.");
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
Query query = TermRangeQuery.newStringRange("body", "ta", "tf", true, true);
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(2, topDocs.totalHits);
|
||||
String snippets[] = highlighter.highlight("body", query, topDocs);
|
||||
assertEquals(2, snippets.length);
|
||||
assertEquals("This is a <b>test</b>.", snippets[0]);
|
||||
assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
|
||||
|
||||
// null start
|
||||
query = TermRangeQuery.newStringRange("body", null, "tf", true, true);
|
||||
topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(2, topDocs.totalHits);
|
||||
snippets = highlighter.highlight("body", query, topDocs);
|
||||
assertEquals(2, snippets.length);
|
||||
assertEquals("This <b>is</b> <b>a</b> <b>test</b>.", snippets[0]);
|
||||
assertEquals("<b>Test</b> <b>a</b> <b>one</b> <b>sentence</b> <b>document</b>.", snippets[1]);
|
||||
|
||||
// null end
|
||||
query = TermRangeQuery.newStringRange("body", "ta", null, true, true);
|
||||
topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(2, topDocs.totalHits);
|
||||
snippets = highlighter.highlight("body", query, topDocs);
|
||||
assertEquals(2, snippets.length);
|
||||
assertEquals("<b>This</b> is a <b>test</b>.", snippets[0]);
|
||||
assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
|
||||
|
||||
// exact start inclusive
|
||||
query = TermRangeQuery.newStringRange("body", "test", "tf", true, true);
|
||||
topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(2, topDocs.totalHits);
|
||||
snippets = highlighter.highlight("body", query, topDocs);
|
||||
assertEquals(2, snippets.length);
|
||||
assertEquals("This is a <b>test</b>.", snippets[0]);
|
||||
assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
|
||||
|
||||
// exact end inclusive
|
||||
query = TermRangeQuery.newStringRange("body", "ta", "test", true, true);
|
||||
topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(2, topDocs.totalHits);
|
||||
snippets = highlighter.highlight("body", query, topDocs);
|
||||
assertEquals(2, snippets.length);
|
||||
assertEquals("This is a <b>test</b>.", snippets[0]);
|
||||
assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
|
||||
|
||||
// exact start exclusive
|
||||
BooleanQuery bq = new BooleanQuery.Builder()
|
||||
.add(new MatchAllDocsQuery(), BooleanClause.Occur.SHOULD)
|
||||
.add(TermRangeQuery.newStringRange("body", "test", "tf", false, true), BooleanClause.Occur.SHOULD)
|
||||
.build();
|
||||
topDocs = searcher.search(bq, 10, Sort.INDEXORDER);
|
||||
assertEquals(2, topDocs.totalHits);
|
||||
snippets = highlighter.highlight("body", bq, topDocs);
|
||||
assertEquals(2, snippets.length);
|
||||
assertEquals("This is a test.", snippets[0]);
|
||||
assertEquals("Test a one sentence document.", snippets[1]);
|
||||
|
||||
// exact end exclusive
|
||||
bq = new BooleanQuery.Builder()
|
||||
.add(new MatchAllDocsQuery(), BooleanClause.Occur.SHOULD)
|
||||
.add(TermRangeQuery.newStringRange("body", "ta", "test", true, false), BooleanClause.Occur.SHOULD)
|
||||
.build();
|
||||
topDocs = searcher.search(bq, 10, Sort.INDEXORDER);
|
||||
assertEquals(2, topDocs.totalHits);
|
||||
snippets = highlighter.highlight("body", bq, topDocs);
|
||||
assertEquals(2, snippets.length);
|
||||
assertEquals("This is a test.", snippets[0]);
|
||||
assertEquals("Test a one sentence document.", snippets[1]);
|
||||
|
||||
// wrong field
|
||||
bq = new BooleanQuery.Builder()
|
||||
.add(new MatchAllDocsQuery(), BooleanClause.Occur.SHOULD)
|
||||
.add(TermRangeQuery.newStringRange("bogus", "ta", "tf", true, true), BooleanClause.Occur.SHOULD)
|
||||
.build();
|
||||
topDocs = searcher.search(bq, 10, Sort.INDEXORDER);
|
||||
assertEquals(2, topDocs.totalHits);
|
||||
snippets = highlighter.highlight("body", bq, topDocs);
|
||||
assertEquals(2, snippets.length);
|
||||
assertEquals("This is a test.", snippets[0]);
|
||||
assertEquals("Test a one sentence document.", snippets[1]);
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
public void testWildcardInBoolean() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Field body = new Field("body", "", fieldType);
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
|
||||
body.setStringValue("This is a test.");
|
||||
iw.addDocument(doc);
|
||||
body.setStringValue("Test a one sentence document.");
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
BooleanQuery query = new BooleanQuery.Builder()
|
||||
.add(new WildcardQuery(new Term("body", "te*")), BooleanClause.Occur.SHOULD)
|
||||
.build();
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(2, topDocs.totalHits);
|
||||
String snippets[] = highlighter.highlight("body", query, topDocs);
|
||||
assertEquals(2, snippets.length);
|
||||
assertEquals("This is a <b>test</b>.", snippets[0]);
|
||||
assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
|
||||
|
||||
// must not
|
||||
query = new BooleanQuery.Builder()
|
||||
.add(new MatchAllDocsQuery(), BooleanClause.Occur.SHOULD)
|
||||
.add(new WildcardQuery(new Term("bogus", "te*")), BooleanClause.Occur.MUST_NOT)
|
||||
.build();
|
||||
topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(2, topDocs.totalHits);
|
||||
snippets = highlighter.highlight("body", query, topDocs);
|
||||
assertEquals(2, snippets.length);
|
||||
assertEquals("This is a test.", snippets[0]);
|
||||
assertEquals("Test a one sentence document.", snippets[1]);
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
public void testWildcardInFiltered() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Field body = new Field("body", "", fieldType);
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
|
||||
body.setStringValue("This is a test.");
|
||||
iw.addDocument(doc);
|
||||
body.setStringValue("Test a one sentence document.");
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
BooleanQuery query = new BooleanQuery.Builder()
|
||||
.add(new WildcardQuery(new Term("body", "te*")), BooleanClause.Occur.MUST)
|
||||
.add(new TermQuery(new Term("body", "test")), BooleanClause.Occur.FILTER)
|
||||
.build();
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(2, topDocs.totalHits);
|
||||
String snippets[] = highlighter.highlight("body", query, topDocs);
|
||||
assertEquals(2, snippets.length);
|
||||
assertEquals("This is a <b>test</b>.", snippets[0]);
|
||||
assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
public void testWildcardInConstantScore() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Field body = new Field("body", "", fieldType);
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
|
||||
body.setStringValue("This is a test.");
|
||||
iw.addDocument(doc);
|
||||
body.setStringValue("Test a one sentence document.");
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
ConstantScoreQuery query = new ConstantScoreQuery(new WildcardQuery(new Term("body", "te*")));
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(2, topDocs.totalHits);
|
||||
String snippets[] = highlighter.highlight("body", query, topDocs);
|
||||
assertEquals(2, snippets.length);
|
||||
assertEquals("This is a <b>test</b>.", snippets[0]);
|
||||
assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
public void testWildcardInDisjunctionMax() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Field body = new Field("body", "", fieldType);
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
|
||||
body.setStringValue("This is a test.");
|
||||
iw.addDocument(doc);
|
||||
body.setStringValue("Test a one sentence document.");
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
DisjunctionMaxQuery query = new DisjunctionMaxQuery(
|
||||
Collections.singleton(new WildcardQuery(new Term("body", "te*"))), 0);
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(2, topDocs.totalHits);
|
||||
String snippets[] = highlighter.highlight("body", query, topDocs);
|
||||
assertEquals(2, snippets.length);
|
||||
assertEquals("This is a <b>test</b>.", snippets[0]);
|
||||
assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
public void testSpanWildcard() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Field body = new Field("body", "", fieldType);
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
|
||||
body.setStringValue("This is a test.");
|
||||
iw.addDocument(doc);
|
||||
body.setStringValue("Test a one sentence document.");
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
Query query = new SpanMultiTermQueryWrapper<>(new WildcardQuery(new Term("body", "te*")));
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(2, topDocs.totalHits);
|
||||
String snippets[] = highlighter.highlight("body", query, topDocs);
|
||||
assertEquals(2, snippets.length);
|
||||
assertEquals("This is a <b>test</b>.", snippets[0]);
|
||||
assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
public void testSpanOr() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Field body = new Field("body", "", fieldType);
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
|
||||
body.setStringValue("This is a test.");
|
||||
iw.addDocument(doc);
|
||||
body.setStringValue("Test a one sentence document.");
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
SpanQuery childQuery = new SpanMultiTermQueryWrapper<>(new WildcardQuery(new Term("body", "te*")));
|
||||
Query query = new SpanOrQuery(new SpanQuery[]{childQuery});
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(2, topDocs.totalHits);
|
||||
String snippets[] = highlighter.highlight("body", query, topDocs);
|
||||
assertEquals(2, snippets.length);
|
||||
assertEquals("This is a <b>test</b>.", snippets[0]);
|
||||
assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
public void testSpanNear() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Field body = new Field("body", "", fieldType);
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
|
||||
body.setStringValue("This is a test.");
|
||||
iw.addDocument(doc);
|
||||
body.setStringValue("Test a one sentence document.");
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
SpanQuery childQuery = new SpanMultiTermQueryWrapper<>(new WildcardQuery(new Term("body", "te*")));
|
||||
Query query = new SpanNearQuery(new SpanQuery[]{childQuery, childQuery}, 0, false);
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(2, topDocs.totalHits);
|
||||
String snippets[] = highlighter.highlight("body", query, topDocs);
|
||||
assertEquals(2, snippets.length);
|
||||
assertEquals("This is a <b>test</b>.", snippets[0]);
|
||||
assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
public void testSpanNot() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Field body = new Field("body", "", fieldType);
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
|
||||
body.setStringValue("This is a test.");
|
||||
iw.addDocument(doc);
|
||||
body.setStringValue("Test a one sentence document.");
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
SpanQuery include = new SpanMultiTermQueryWrapper<>(new WildcardQuery(new Term("body", "te*")));
|
||||
SpanQuery exclude = new SpanTermQuery(new Term("body", "bogus"));
|
||||
Query query = new SpanNotQuery(include, exclude);
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(2, topDocs.totalHits);
|
||||
String snippets[] = highlighter.highlight("body", query, topDocs);
|
||||
assertEquals(2, snippets.length);
|
||||
assertEquals("This is a <b>test</b>.", snippets[0]);
|
||||
assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
public void testSpanPositionCheck() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Field body = new Field("body", "", fieldType);
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
|
||||
body.setStringValue("This is a test.");
|
||||
iw.addDocument(doc);
|
||||
body.setStringValue("Test a one sentence document.");
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
SpanQuery childQuery = new SpanMultiTermQueryWrapper<>(new WildcardQuery(new Term("body", "te*")));
|
||||
Query query = new SpanFirstQuery(childQuery, 1000000);
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(2, topDocs.totalHits);
|
||||
String snippets[] = highlighter.highlight("body", query, topDocs);
|
||||
assertEquals(2, snippets.length);
|
||||
assertEquals("This is a <b>test</b>.", snippets[0]);
|
||||
assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* Runs a query with two MTQs and confirms the formatter
|
||||
* can tell which query matched which hit.
|
||||
*/
|
||||
public void testWhichMTQMatched() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Field body = new Field("body", "", fieldType);
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
|
||||
body.setStringValue("Test a one sentence document.");
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
BooleanQuery query = new BooleanQuery.Builder()
|
||||
.add(new WildcardQuery(new Term("body", "te*")), BooleanClause.Occur.SHOULD)
|
||||
.add(new WildcardQuery(new Term("body", "one")), BooleanClause.Occur.SHOULD)
|
||||
.add(new WildcardQuery(new Term("body", "se*")), BooleanClause.Occur.SHOULD)
|
||||
.build();
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(1, topDocs.totalHits);
|
||||
String snippets[] = highlighter.highlight("body", query, topDocs);
|
||||
assertEquals(1, snippets.length);
|
||||
|
||||
// Default formatter just bolds each hit:
|
||||
assertEquals("<b>Test</b> a <b>one</b> <b>sentence</b> document.", snippets[0]);
|
||||
|
||||
// Now use our own formatter, that also stuffs the
|
||||
// matching term's text into the result:
|
||||
highlighter = new UnifiedHighlighter(searcher, indexAnalyzer) {
|
||||
|
||||
@Override
|
||||
protected PassageFormatter getFormatter(String field) {
|
||||
return new PassageFormatter() {
|
||||
|
||||
@Override
|
||||
public Object format(Passage passages[], String content) {
|
||||
// Copied from DefaultPassageFormatter, but
|
||||
// tweaked to include the matched term:
|
||||
StringBuilder sb = new StringBuilder();
|
||||
int pos = 0;
|
||||
for (Passage passage : passages) {
|
||||
// don't add ellipsis if its the first one, or if its connected.
|
||||
if (passage.startOffset > pos && pos > 0) {
|
||||
sb.append("... ");
|
||||
}
|
||||
pos = passage.startOffset;
|
||||
for (int i = 0; i < passage.numMatches; i++) {
|
||||
int start = passage.matchStarts[i];
|
||||
int end = passage.matchEnds[i];
|
||||
// its possible to have overlapping terms
|
||||
if (start > pos) {
|
||||
sb.append(content, pos, start);
|
||||
}
|
||||
if (end > pos) {
|
||||
sb.append("<b>");
|
||||
sb.append(content, Math.max(pos, start), end);
|
||||
sb.append('(');
|
||||
sb.append(passage.getMatchTerms()[i].utf8ToString());
|
||||
sb.append(')');
|
||||
sb.append("</b>");
|
||||
pos = end;
|
||||
}
|
||||
}
|
||||
// its possible a "term" from the analyzer could span a sentence boundary.
|
||||
sb.append(content, pos, Math.max(pos, passage.endOffset));
|
||||
pos = passage.endOffset;
|
||||
}
|
||||
return sb.toString();
|
||||
}
|
||||
};
|
||||
}
|
||||
};
|
||||
|
||||
assertEquals(1, topDocs.totalHits);
|
||||
snippets = highlighter.highlight("body", query, topDocs);
|
||||
assertEquals(1, snippets.length);
|
||||
|
||||
// Default formatter bolds each hit:
|
||||
assertEquals("<b>Test(body:te*)</b> a <b>one(body:one)</b> <b>sentence(body:se*)</b> document.", snippets[0]);
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
|
||||
//
|
||||
// All tests below were *not* ported from the PostingsHighlighter; they are new to the U.H.
|
||||
//
|
||||
|
||||
public void testWithMaxLen() throws IOException {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Field body = new Field("body", "", fieldType);
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
|
||||
body.setStringValue("Alpha Bravo foo foo foo. Foo foo Alpha Bravo");//44 char long, 2 sentences
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
highlighter.setMaxLength(25);//a little past first sentence
|
||||
|
||||
BooleanQuery query = new BooleanQuery.Builder()
|
||||
.add(new TermQuery(new Term("body", "alpha")), BooleanClause.Occur.MUST)
|
||||
.add(new PrefixQuery(new Term("body", "bra")), BooleanClause.Occur.MUST)
|
||||
.build();
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
String snippets[] = highlighter.highlight("body", query, topDocs, 2);//ask for 2 but we'll only get 1
|
||||
assertArrayEquals(
|
||||
new String[]{"<b>Alpha</b> <b>Bravo</b> foo foo foo. "}, snippets
|
||||
);
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
public void testTokenStreamIsClosed() throws IOException {
|
||||
// note: test is a derivative of testWithMaxLen()
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Field body = new Field("body", "", fieldType);
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
|
||||
body.setStringValue("Alpha Bravo foo foo foo. Foo foo Alpha Bravo");
|
||||
if (random().nextBoolean()) { // sometimes add a 2nd value (maybe matters?)
|
||||
doc.add(new Field("body", "2nd value Alpha Bravo", fieldType));
|
||||
}
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
// use this buggy Analyzer at highlight time
|
||||
Analyzer buggyAnalyzer = new Analyzer() {
|
||||
@Override
|
||||
protected TokenStreamComponents createComponents(String fieldName) {
|
||||
Tokenizer buggyTokenizer = new Tokenizer() {
|
||||
@Override
|
||||
public boolean incrementToken() throws IOException {
|
||||
throw new IOException("EXPECTED");
|
||||
}
|
||||
};
|
||||
return new TokenStreamComponents(buggyTokenizer);
|
||||
}
|
||||
};
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, buggyAnalyzer);
|
||||
highlighter.setHandleMultiTermQuery(true);
|
||||
if (rarely()) {
|
||||
highlighter.setMaxLength(25);//a little past first sentence
|
||||
}
|
||||
|
||||
boolean hasClauses = false;
|
||||
BooleanQuery.Builder queryBuilder = new BooleanQuery.Builder();
|
||||
if (random().nextBoolean()) {
|
||||
hasClauses = true;
|
||||
queryBuilder.add(new TermQuery(new Term("body", "alpha")), BooleanClause.Occur.MUST);
|
||||
}
|
||||
if (!hasClauses || random().nextBoolean()) {
|
||||
queryBuilder.add(new PrefixQuery(new Term("body", "bra")), BooleanClause.Occur.MUST);
|
||||
}
|
||||
BooleanQuery query = queryBuilder.build();
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
try {
|
||||
String snippets[] = highlighter.highlight("body", query, topDocs, 2);
|
||||
// don't even care what the results are; just want to test exception behavior
|
||||
if (fieldType == UHTestHelper.reanalysisType) {
|
||||
fail("Expecting EXPECTED IOException");
|
||||
}
|
||||
} catch (IOException e) {
|
||||
if (!e.getMessage().equals("EXPECTED")) {
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
ir.close();
|
||||
|
||||
// Now test we can get the tokenStream without it puking due to IllegalStateException for not calling close()
|
||||
|
||||
try (TokenStream ts = buggyAnalyzer.tokenStream("body", "anything")) {
|
||||
ts.reset();// hopefully doesn't throw
|
||||
// don't call incrementToken; we know it's buggy ;-)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Not empty but nothing analyzes. Ensures we address null term-vectors.
|
||||
*/
|
||||
public void testNothingAnalyzes() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Document doc = new Document();
|
||||
doc.add(new Field("body", " ", fieldType));// just a space! (thus not empty)
|
||||
doc.add(newTextField("id", "id", Field.Store.YES));
|
||||
iw.addDocument(doc);
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new Field("body", "something", fieldType));
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
int docID = searcher.search(new TermQuery(new Term("id", "id")), 1).scoreDocs[0].doc;
|
||||
|
||||
Query query = new PrefixQuery(new Term("body", "nonexistent"));
|
||||
int[] docIDs = new int[1];
|
||||
docIDs[0] = docID;
|
||||
String snippets[] = highlighter.highlightFields(new String[]{"body"}, query, docIDs, new int[]{2}).get("body");
|
||||
assertEquals(1, snippets.length);
|
||||
assertEquals(" ", snippets[0]);
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
public void testMultiSegment() throws Exception {
|
||||
// If we incorrectly got the term vector from mis-matched global/leaf doc ID, this test may fail
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Document doc = new Document();
|
||||
doc.add(new Field("body", "word aberration", fieldType));
|
||||
iw.addDocument(doc);
|
||||
|
||||
iw.commit(); // make segment
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new Field("body", "word absolve", fieldType));
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
Query query = new PrefixQuery(new Term("body", "ab"));
|
||||
TopDocs topDocs = searcher.search(query, 10);
|
||||
|
||||
String snippets[] = highlighter.highlightFields(new String[]{"body"}, query, topDocs).get("body");
|
||||
Arrays.sort(snippets);
|
||||
assertEquals("[word <b>aberration</b>, word <b>absolve</b>]", Arrays.toString(snippets));
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
public void testPositionSensitiveWithWildcardDoesNotHighlight() throws Exception {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
Document doc = new Document();
|
||||
doc.add(new Field("body", "iterate insect ipswitch illinois indirect", fieldType));
|
||||
doc.add(newTextField("id", "id", Field.Store.YES));
|
||||
|
||||
iw.addDocument(doc);
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
int docID = searcher.search(new TermQuery(new Term("id", "id")), 1).scoreDocs[0].doc;
|
||||
|
||||
PhraseQuery pq = new PhraseQuery.Builder()
|
||||
.add(new Term("body", "consent"))
|
||||
.add(new Term("body", "order"))
|
||||
.build();
|
||||
|
||||
BooleanQuery query = new BooleanQuery.Builder()
|
||||
.add(new WildcardQuery(new Term("body", "enforc*")), BooleanClause.Occur.MUST)
|
||||
.add(pq, BooleanClause.Occur.MUST)
|
||||
.build();
|
||||
|
||||
int[] docIds = new int[]{docID};
|
||||
|
||||
String snippets[] = highlighter.highlightFields(new String[]{"body"}, query, docIds, new int[]{2}).get("body");
|
||||
assertEquals(1, snippets.length);
|
||||
assertEquals("iterate insect ipswitch illinois indirect", snippets[0]);
|
||||
ir.close();
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,339 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.search.uhighlight;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
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.document.FieldType;
|
||||
import org.apache.lucene.document.StringField;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexWriterConfig;
|
||||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.BooleanClause;
|
||||
import org.apache.lucene.search.BooleanQuery;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.Sort;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.lucene.search.TopDocs;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashSet;
|
||||
import java.util.Random;
|
||||
|
||||
@SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom", "Lucene3x"})
|
||||
@LuceneTestCase.SuppressSysoutChecks(bugUrl = "")//Gradle interferes with this Lucene test rule
|
||||
public class TestUnifiedHighlighterRanking extends LuceneTestCase {
|
||||
|
||||
Analyzer indexAnalyzer;
|
||||
|
||||
// note: don't choose reanalysis because it doesn't always know the term frequency, which is a statistic used
|
||||
// in passage ranking. Sometimes it does (e.g. when it builds a MemoryIndex) but not necessarily.
|
||||
final FieldType fieldType = UHTestHelper.randomFieldType(random(), UHTestHelper.postingsType, UHTestHelper.tvType);
|
||||
|
||||
/**
|
||||
* indexes a bunch of gibberish, and then highlights top(n).
|
||||
* asserts that top(n) highlights is a subset of top(n+1) up to some max N
|
||||
*/
|
||||
// TODO: this only tests single-valued fields. we should also index multiple values per field!
|
||||
public void testRanking() throws Exception {
|
||||
// number of documents: we will check each one
|
||||
final int numDocs = atLeast(100);
|
||||
// number of top-N snippets, we will check 1 .. N
|
||||
final int maxTopN = 5;
|
||||
// maximum number of elements to put in a sentence.
|
||||
final int maxSentenceLength = 10;
|
||||
// maximum number of sentences in a document
|
||||
final int maxNumSentences = 20;
|
||||
|
||||
Directory dir = newDirectory();
|
||||
indexAnalyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
Document document = new Document();
|
||||
Field id = new StringField("id", "", Field.Store.NO);
|
||||
Field body = new Field("body", "", fieldType);
|
||||
document.add(id);
|
||||
document.add(body);
|
||||
|
||||
for (int i = 0; i < numDocs; i++) {
|
||||
StringBuilder bodyText = new StringBuilder();
|
||||
int numSentences = TestUtil.nextInt(random(), 1, maxNumSentences);
|
||||
for (int j = 0; j < numSentences; j++) {
|
||||
bodyText.append(newSentence(random(), maxSentenceLength));
|
||||
}
|
||||
body.setStringValue(bodyText.toString());
|
||||
id.setStringValue(Integer.toString(i));
|
||||
iw.addDocument(document);
|
||||
}
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
for (int i = 0; i < numDocs; i++) {
|
||||
checkDocument(searcher, i, maxTopN);
|
||||
}
|
||||
iw.close();
|
||||
ir.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
private void checkDocument(IndexSearcher is, int doc, int maxTopN) throws IOException {
|
||||
for (int ch = 'a'; ch <= 'z'; ch++) {
|
||||
Term term = new Term("body", "" + (char) ch);
|
||||
// check a simple term query
|
||||
checkQuery(is, new TermQuery(term), doc, maxTopN);
|
||||
// check a boolean query
|
||||
Term nextTerm = new Term("body", "" + (char) (ch + 1));
|
||||
BooleanQuery bq = new BooleanQuery.Builder()
|
||||
.add(new TermQuery(term), BooleanClause.Occur.SHOULD)
|
||||
.add(new TermQuery(nextTerm), BooleanClause.Occur.SHOULD)
|
||||
.build();
|
||||
checkQuery(is, bq, doc, maxTopN);
|
||||
}
|
||||
}
|
||||
|
||||
private void checkQuery(IndexSearcher is, Query query, int doc, int maxTopN) throws IOException {
|
||||
for (int n = 1; n < maxTopN; n++) {
|
||||
final FakePassageFormatter f1 = new FakePassageFormatter();
|
||||
UnifiedHighlighter p1 = new UnifiedHighlighter(is, indexAnalyzer) {
|
||||
@Override
|
||||
protected PassageFormatter getFormatter(String field) {
|
||||
assertEquals("body", field);
|
||||
return f1;
|
||||
}
|
||||
};
|
||||
p1.setMaxLength(Integer.MAX_VALUE - 1);
|
||||
|
||||
final FakePassageFormatter f2 = new FakePassageFormatter();
|
||||
UnifiedHighlighter p2 = new UnifiedHighlighter(is, indexAnalyzer) {
|
||||
@Override
|
||||
protected PassageFormatter getFormatter(String field) {
|
||||
assertEquals("body", field);
|
||||
return f2;
|
||||
}
|
||||
};
|
||||
p2.setMaxLength(Integer.MAX_VALUE - 1);
|
||||
|
||||
BooleanQuery.Builder queryBuilder = new BooleanQuery.Builder();
|
||||
queryBuilder.add(query, BooleanClause.Occur.MUST);
|
||||
queryBuilder.add(new TermQuery(new Term("id", Integer.toString(doc))), BooleanClause.Occur.MUST);
|
||||
BooleanQuery bq = queryBuilder.build();
|
||||
TopDocs td = is.search(bq, 1);
|
||||
p1.highlight("body", bq, td, n);
|
||||
p2.highlight("body", bq, td, n + 1);
|
||||
assertTrue(f2.seen.containsAll(f1.seen));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* returns a new random sentence, up to maxSentenceLength "words" in length.
|
||||
* each word is a single character (a-z). The first one is capitalized.
|
||||
*/
|
||||
private String newSentence(Random r, int maxSentenceLength) {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
int numElements = TestUtil.nextInt(r, 1, maxSentenceLength);
|
||||
for (int i = 0; i < numElements; i++) {
|
||||
if (sb.length() > 0) {
|
||||
sb.append(' ');
|
||||
sb.append((char) TestUtil.nextInt(r, 'a', 'z'));
|
||||
} else {
|
||||
// capitalize the first word to help breakiterator
|
||||
sb.append((char) TestUtil.nextInt(r, 'A', 'Z'));
|
||||
}
|
||||
}
|
||||
sb.append(". "); // finalize sentence
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* a fake formatter that doesn't actually format passages.
|
||||
* instead it just collects them for asserts!
|
||||
*/
|
||||
static class FakePassageFormatter extends PassageFormatter {
|
||||
HashSet<Pair> seen = new HashSet<>();
|
||||
|
||||
@Override
|
||||
public String format(Passage passages[], String content) {
|
||||
for (Passage p : passages) {
|
||||
// verify some basics about the passage
|
||||
assertTrue(p.getScore() >= 0);
|
||||
assertTrue(p.getNumMatches() > 0);
|
||||
assertTrue(p.getStartOffset() >= 0);
|
||||
assertTrue(p.getStartOffset() <= content.length());
|
||||
assertTrue(p.getEndOffset() >= p.getStartOffset());
|
||||
assertTrue(p.getEndOffset() <= content.length());
|
||||
// we use a very simple analyzer. so we can assert the matches are correct
|
||||
int lastMatchStart = -1;
|
||||
for (int i = 0; i < p.getNumMatches(); i++) {
|
||||
BytesRef term = p.getMatchTerms()[i];
|
||||
int matchStart = p.getMatchStarts()[i];
|
||||
assertTrue(matchStart >= 0);
|
||||
// must at least start within the passage
|
||||
assertTrue(matchStart < p.getEndOffset());
|
||||
int matchEnd = p.getMatchEnds()[i];
|
||||
assertTrue(matchEnd >= 0);
|
||||
// always moving forward
|
||||
assertTrue(matchStart >= lastMatchStart);
|
||||
lastMatchStart = matchStart;
|
||||
// single character terms
|
||||
assertEquals(matchStart + 1, matchEnd);
|
||||
// and the offsets must be correct...
|
||||
assertEquals(1, term.length);
|
||||
assertEquals((char) term.bytes[term.offset], Character.toLowerCase(content.charAt(matchStart)));
|
||||
}
|
||||
// record just the start/end offset for simplicity
|
||||
seen.add(new Pair(p.getStartOffset(), p.getEndOffset()));
|
||||
}
|
||||
return "bogus!!!!!!";
|
||||
}
|
||||
}
|
||||
|
||||
static class Pair {
|
||||
final int start;
|
||||
final int end;
|
||||
|
||||
Pair(int start, int end) {
|
||||
this.start = start;
|
||||
this.end = end;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
final int prime = 31;
|
||||
int result = 1;
|
||||
result = prime * result + end;
|
||||
result = prime * result + start;
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (this == obj) {
|
||||
return true;
|
||||
}
|
||||
if (obj == null) {
|
||||
return false;
|
||||
}
|
||||
if (getClass() != obj.getClass()) {
|
||||
return false;
|
||||
}
|
||||
Pair other = (Pair) obj;
|
||||
if (end != other.end) {
|
||||
return false;
|
||||
}
|
||||
if (start != other.start) {
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "Pair [start=" + start + ", end=" + end + "]";
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* sets b=0 to disable passage length normalization
|
||||
*/
|
||||
public void testCustomB() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
indexAnalyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);
|
||||
IndexWriterConfig iwc = newIndexWriterConfig(indexAnalyzer);
|
||||
iwc.setMergePolicy(newLogMergePolicy());
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
|
||||
|
||||
Field body = new Field("body", "", fieldType);
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
|
||||
body.setStringValue("This is a test. This test is a better test but the sentence is excruiatingly long, " +
|
||||
"you have no idea how painful it was for me to type this long sentence into my IDE.");
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer) {
|
||||
@Override
|
||||
protected PassageScorer getScorer(String field) {
|
||||
return new PassageScorer(1.2f, 0, 87);
|
||||
}
|
||||
};
|
||||
Query query = new TermQuery(new Term("body", "test"));
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(1, topDocs.totalHits);
|
||||
String snippets[] = highlighter.highlight("body", query, topDocs, 1);
|
||||
assertEquals(1, snippets.length);
|
||||
assertTrue(snippets[0].startsWith("This <b>test</b> is a better <b>test</b>"));
|
||||
|
||||
ir.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* sets k1=0 for simple coordinate-level match (# of query terms present)
|
||||
*/
|
||||
public void testCustomK1() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
indexAnalyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);
|
||||
IndexWriterConfig iwc = newIndexWriterConfig(indexAnalyzer);
|
||||
iwc.setMergePolicy(newLogMergePolicy());
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
|
||||
|
||||
Field body = new Field("body", "", fieldType);
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
|
||||
body.setStringValue("This has only foo foo. " +
|
||||
"On the other hand this sentence contains both foo and bar. " +
|
||||
"This has only bar bar bar bar bar bar bar bar bar bar bar bar.");
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer) {
|
||||
@Override
|
||||
protected PassageScorer getScorer(String field) {
|
||||
return new PassageScorer(0, 0.75f, 87);
|
||||
}
|
||||
};
|
||||
BooleanQuery query = new BooleanQuery.Builder()
|
||||
.add(new TermQuery(new Term("body", "foo")), BooleanClause.Occur.SHOULD)
|
||||
.add(new TermQuery(new Term("body", "bar")), BooleanClause.Occur.SHOULD)
|
||||
.build();
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(1, topDocs.totalHits);
|
||||
String snippets[] = highlighter.highlight("body", query, topDocs, 1);
|
||||
assertEquals(1, snippets.length);
|
||||
assertTrue(snippets[0].startsWith("On the other hand"));
|
||||
|
||||
ir.close();
|
||||
dir.close();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,74 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.search.uhighlight;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.analysis.MockTokenizer;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.BooleanClause;
|
||||
import org.apache.lucene.search.BooleanQuery;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.junit.Test;
|
||||
|
||||
@LuceneTestCase.SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom", "Lucene3x"})
|
||||
@LuceneTestCase.SuppressSysoutChecks(bugUrl = "")//Gradle interferes with this Lucene test rule
|
||||
public class TestUnifiedHighlighterReanalysis extends LuceneTestCase {
|
||||
|
||||
private MockAnalyzer indexAnalyzer =
|
||||
new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);//whitespace, punctuation, lowercase;
|
||||
|
||||
@Test
|
||||
public void testWithoutIndexSearcher() throws IOException {
|
||||
String text = "This is a test. Just a test highlighting without a searcher. Feel free to ignore.";
|
||||
BooleanQuery query = new BooleanQuery.Builder()
|
||||
.add(new TermQuery(new Term("body", "highlighting")), BooleanClause.Occur.SHOULD)
|
||||
.add(new TermQuery(new Term("title", "test")), BooleanClause.Occur.SHOULD)
|
||||
.build();
|
||||
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(null, indexAnalyzer);
|
||||
String snippet = highlighter.highlightWithoutSearcher("body", query, text, 1).toString();
|
||||
|
||||
assertEquals("Just a test <b>highlighting</b> without a searcher. ", snippet);
|
||||
|
||||
assertEquals("test single space", " ", highlighter.highlightWithoutSearcher("body", query, " ", 1));
|
||||
|
||||
assertEquals("Hello", highlighter.highlightWithoutSearcher("nonexistent", query, "Hello", 1));
|
||||
}
|
||||
|
||||
@Test(expected = IllegalStateException.class)
|
||||
public void testIndexSearcherNullness() throws IOException {
|
||||
String text = "This is a test. Just a test highlighting without a searcher. Feel free to ignore.";
|
||||
Query query = new TermQuery(new Term("body", "highlighting"));
|
||||
|
||||
try (Directory directory = newDirectory();
|
||||
RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory);
|
||||
IndexReader indexReader = indexWriter.getReader()) {
|
||||
IndexSearcher searcher = newSearcher(indexReader);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
highlighter.highlightWithoutSearcher("body", query, text, 1);//should throw
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,404 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.search.uhighlight;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
|
||||
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.document.FieldType;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.BooleanClause;
|
||||
import org.apache.lucene.search.BooleanQuery;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.MatchNoDocsQuery;
|
||||
import org.apache.lucene.search.MultiPhraseQuery;
|
||||
import org.apache.lucene.search.PhraseQuery;
|
||||
import org.apache.lucene.search.PrefixQuery;
|
||||
import org.apache.lucene.search.Sort;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.lucene.search.TopDocs;
|
||||
import org.apache.lucene.search.WildcardQuery;
|
||||
import org.apache.lucene.search.spans.SpanMultiTermQueryWrapper;
|
||||
import org.apache.lucene.search.spans.SpanNearQuery;
|
||||
import org.apache.lucene.search.spans.SpanQuery;
|
||||
import org.apache.lucene.search.spans.SpanTermQuery;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.QueryBuilder;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
|
||||
@LuceneTestCase.SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom", "Lucene3x"})
|
||||
@LuceneTestCase.SuppressSysoutChecks(bugUrl = "")//Gradle interferes with this Lucene test rule
|
||||
public class TestUnifiedHighlighterStrictPhrases extends LuceneTestCase {
|
||||
|
||||
final FieldType fieldType;
|
||||
|
||||
Directory dir;
|
||||
MockAnalyzer indexAnalyzer;
|
||||
RandomIndexWriter indexWriter;
|
||||
IndexSearcher searcher;
|
||||
UnifiedHighlighter highlighter;
|
||||
IndexReader indexReader;
|
||||
|
||||
@ParametersFactory
|
||||
public static Iterable<Object[]> parameters() {
|
||||
return UHTestHelper.parametersFactoryList();
|
||||
}
|
||||
|
||||
public TestUnifiedHighlighterStrictPhrases(FieldType fieldType) {
|
||||
this.fieldType = fieldType;
|
||||
}
|
||||
|
||||
@Before
|
||||
public void doBefore() throws IOException {
|
||||
indexAnalyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);//whitespace, punctuation, lowercase
|
||||
indexAnalyzer.setPositionIncrementGap(3);// more than default
|
||||
dir = newDirectory();
|
||||
indexWriter = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
}
|
||||
|
||||
@After
|
||||
public void doAfter() throws IOException {
|
||||
IOUtils.close(indexReader, indexWriter, dir);
|
||||
}
|
||||
|
||||
private Document newDoc(String... bodyVals) {
|
||||
Document doc = new Document();
|
||||
for (String bodyVal : bodyVals) {
|
||||
doc.add(new Field("body", bodyVal, fieldType));
|
||||
}
|
||||
return doc;
|
||||
}
|
||||
|
||||
private void initReaderSearcherHighlighter() throws IOException {
|
||||
indexReader = indexWriter.getReader();
|
||||
searcher = newSearcher(indexReader);
|
||||
highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
highlighter.setHighlightPhrasesStrictly(true);
|
||||
}
|
||||
|
||||
private PhraseQuery newPhraseQuery(String field, String phrase) {
|
||||
return (PhraseQuery) new QueryBuilder(indexAnalyzer).createPhraseQuery(field, phrase);
|
||||
}
|
||||
|
||||
private PhraseQuery setSlop(PhraseQuery query, int slop) {
|
||||
PhraseQuery.Builder builder = new PhraseQuery.Builder();
|
||||
Term[] terms = query.getTerms();
|
||||
int[] positions = query.getPositions();
|
||||
for (int i = 0; i < terms.length; i++) {
|
||||
builder.add(terms[i], positions[i]);
|
||||
}
|
||||
builder.setSlop(slop);
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
public void testBasics() throws IOException {
|
||||
indexWriter.addDocument(newDoc("Yin yang, filter")); // filter out. test getTermToSpanLists reader 1-doc filter
|
||||
indexWriter.addDocument(newDoc("yin alone, Yin yang, yin gap yang"));
|
||||
initReaderSearcherHighlighter();
|
||||
|
||||
//query: -filter +"yin yang"
|
||||
BooleanQuery query = new BooleanQuery.Builder()
|
||||
.add(new TermQuery(new Term("body", "filter")), BooleanClause.Occur.MUST_NOT)
|
||||
.add(newPhraseQuery("body", "yin yang"), BooleanClause.Occur.MUST)
|
||||
.build();
|
||||
|
||||
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
String[] snippets = highlighter.highlight("body", query, topDocs);
|
||||
|
||||
assertArrayEquals(new String[]{"yin alone, <b>Yin</b> <b>yang</b>, yin gap yang"}, snippets);
|
||||
}
|
||||
|
||||
public void testWithSameTermQuery() throws IOException {
|
||||
indexWriter.addDocument(newDoc("Yin yang, yin gap yang"));
|
||||
initReaderSearcherHighlighter();
|
||||
|
||||
BooleanQuery query = new BooleanQuery.Builder()
|
||||
.add(new TermQuery(new Term("body", "yin")), BooleanClause.Occur.MUST)
|
||||
.add(newPhraseQuery("body", "yin yang"), BooleanClause.Occur.MUST)
|
||||
// add queries for other fields; we shouldn't highlight these because of that.
|
||||
.add(new TermQuery(new Term("title", "yang")), BooleanClause.Occur.SHOULD)
|
||||
.build();
|
||||
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
String[] snippets = highlighter.highlight("body", query, topDocs);
|
||||
|
||||
assertArrayEquals(new String[]{"<b>Yin</b> <b>yang</b>, <b>yin</b> gap yang"}, snippets);
|
||||
}
|
||||
|
||||
public void testPhraseNotInDoc() throws IOException {
|
||||
indexWriter.addDocument(newDoc("Whatever yin")); // query matches this; highlight it
|
||||
indexWriter.addDocument(newDoc("nextdoc yin"));// query does NOT match this, only the SHOULD clause does
|
||||
initReaderSearcherHighlighter();
|
||||
|
||||
BooleanQuery query = new BooleanQuery.Builder()
|
||||
//MUST:
|
||||
.add(new TermQuery(new Term("body", "whatever")), BooleanClause.Occur.MUST)
|
||||
//SHOULD: (yet won't)
|
||||
.add(newPhraseQuery("body", "nextdoc yin"), BooleanClause.Occur.SHOULD)
|
||||
.add(newPhraseQuery("body", "nonexistent yin"), BooleanClause.Occur.SHOULD)
|
||||
.build();
|
||||
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
String[] snippets = highlighter.highlight("body", query, topDocs);
|
||||
|
||||
assertArrayEquals(new String[]{"<b>Whatever</b> yin"}, snippets);
|
||||
}
|
||||
|
||||
public void testSubPhrases() throws IOException {
|
||||
indexWriter.addDocument(newDoc("alpha bravo charlie - charlie bravo alpha"));
|
||||
initReaderSearcherHighlighter();
|
||||
|
||||
BooleanQuery query = new BooleanQuery.Builder()
|
||||
.add(newPhraseQuery("body", "alpha bravo charlie"), BooleanClause.Occur.MUST)
|
||||
.add(newPhraseQuery("body", "alpha bravo"), BooleanClause.Occur.MUST)
|
||||
.build();
|
||||
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
String[] snippets = highlighter.highlight("body", query, topDocs);
|
||||
|
||||
assertArrayEquals(new String[]{"<b>alpha</b> <b>bravo</b> <b>charlie</b> - charlie bravo alpha"}, snippets);
|
||||
}
|
||||
|
||||
public void testSynonyms() throws IOException {
|
||||
indexWriter.addDocument(newDoc("mother father w mom father w dad"));
|
||||
initReaderSearcherHighlighter();
|
||||
|
||||
MultiPhraseQuery query = new MultiPhraseQuery.Builder()
|
||||
.add(new Term[]{new Term("body", "mom"), new Term("body", "mother")})
|
||||
.add(new Term[]{new Term("body", "dad"), new Term("body", "father")})
|
||||
.build();
|
||||
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
String[] snippets = highlighter.highlight("body", query, topDocs);
|
||||
|
||||
assertArrayEquals(new String[]{"<b>mother</b> <b>father</b> w <b>mom</b> <b>father</b> w dad"}, snippets);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test it does *not* highlight the same term's not next to the span-near. "charlie" in this case.
|
||||
* This particular example exercises "Rewrite" plus "MTQ" in the same query.
|
||||
*/
|
||||
public void testRewriteAndMtq() throws IOException {
|
||||
indexWriter.addDocument(newDoc("alpha bravo charlie - charlie bravo alpha"));
|
||||
initReaderSearcherHighlighter();
|
||||
|
||||
SpanNearQuery snq = new SpanNearQuery(
|
||||
new SpanQuery[]{
|
||||
new SpanTermQuery(new Term("body", "bravo")),
|
||||
new SpanMultiTermQueryWrapper<>(new PrefixQuery(new Term("body", "ch")))}, // REWRITES
|
||||
0, true);
|
||||
|
||||
BooleanQuery query = new BooleanQuery.Builder()
|
||||
.add(snq, BooleanClause.Occur.MUST)
|
||||
.add(new PrefixQuery(new Term("body", "al")), BooleanClause.Occur.MUST) // MTQ
|
||||
.add(newPhraseQuery("body", "alpha bravo"), BooleanClause.Occur.MUST)
|
||||
// add queries for other fields; we shouldn't highlight these because of that.
|
||||
.add(newPhraseQuery("title", "bravo alpha"), BooleanClause.Occur.SHOULD)
|
||||
.build();
|
||||
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
String[] snippets = highlighter.highlight("body", query, topDocs);
|
||||
|
||||
assertArrayEquals(new String[]{"<b>alpha</b> <b>bravo</b> <b>charlie</b> - charlie bravo <b>alpha</b>"},
|
||||
snippets);
|
||||
|
||||
// do again, this time with MTQ disabled. We should only find "alpha bravo".
|
||||
highlighter.setHandleMultiTermQuery(false);//disable but leave phrase processing enabled
|
||||
|
||||
topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
snippets = highlighter.highlight("body", query, topDocs);
|
||||
|
||||
assertArrayEquals(new String[]{"<b>alpha</b> <b>bravo</b> charlie - charlie bravo alpha"},
|
||||
snippets);
|
||||
}
|
||||
|
||||
/**
|
||||
* Like {@link #testRewriteAndMtq} but no freestanding MTQ
|
||||
*/
|
||||
public void testRewrite() throws IOException {
|
||||
indexWriter.addDocument(newDoc("alpha bravo charlie - charlie bravo alpha"));
|
||||
initReaderSearcherHighlighter();
|
||||
|
||||
SpanNearQuery snq = new SpanNearQuery(
|
||||
new SpanQuery[]{
|
||||
new SpanTermQuery(new Term("body", "bravo")),
|
||||
new SpanMultiTermQueryWrapper<>(new PrefixQuery(new Term("body", "ch")))}, // REWRITES
|
||||
0, true);
|
||||
BooleanQuery query = new BooleanQuery.Builder()
|
||||
.add(snq, BooleanClause.Occur.MUST)
|
||||
// .add(new PrefixQuery(new Term("body", "al")), BooleanClause.Occur.MUST) // MTQ
|
||||
.add(newPhraseQuery("body", "alpha bravo"), BooleanClause.Occur.MUST)
|
||||
// add queries for other fields; we shouldn't highlight these because of that.
|
||||
.add(newPhraseQuery("title", "bravo alpha"), BooleanClause.Occur.SHOULD)
|
||||
.build();
|
||||
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
String[] snippets = highlighter.highlight("body", query, topDocs);
|
||||
|
||||
assertArrayEquals(new String[]{"<b>alpha</b> <b>bravo</b> <b>charlie</b> - charlie bravo alpha"},
|
||||
snippets);
|
||||
|
||||
// do again, this time with MTQ disabled. We should only find "alpha bravo".
|
||||
highlighter.setHandleMultiTermQuery(false);//disable but leave phrase processing enabled
|
||||
|
||||
topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
snippets = highlighter.highlight("body", query, topDocs);
|
||||
|
||||
assertArrayEquals(new String[]{"<b>alpha</b> <b>bravo</b> charlie - charlie bravo alpha"},
|
||||
snippets);
|
||||
}
|
||||
|
||||
/**
|
||||
* Like {@link #testRewriteAndMtq} but no rewrite.
|
||||
*/
|
||||
public void testMtq() throws IOException {
|
||||
indexWriter.addDocument(newDoc("alpha bravo charlie - charlie bravo alpha"));
|
||||
initReaderSearcherHighlighter();
|
||||
|
||||
SpanNearQuery snq = new SpanNearQuery(
|
||||
new SpanQuery[]{
|
||||
new SpanTermQuery(new Term("body", "bravo")),
|
||||
new SpanTermQuery(new Term("body", "charlie"))}, // does NOT rewrite
|
||||
0, true);
|
||||
|
||||
BooleanQuery query = new BooleanQuery.Builder()
|
||||
.add(snq, BooleanClause.Occur.MUST)
|
||||
.add(new PrefixQuery(new Term("body", "al")), BooleanClause.Occur.MUST) // MTQ
|
||||
.add(newPhraseQuery("body", "alpha bravo"), BooleanClause.Occur.MUST)
|
||||
// add queries for other fields; we shouldn't highlight these because of that.
|
||||
.add(newPhraseQuery("title", "bravo alpha"), BooleanClause.Occur.SHOULD)
|
||||
.build();
|
||||
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
String[] snippets = highlighter.highlight("body", query, topDocs);
|
||||
|
||||
assertArrayEquals(new String[]{"<b>alpha</b> <b>bravo</b> <b>charlie</b> - charlie bravo <b>alpha</b>"},
|
||||
snippets);
|
||||
|
||||
// do again, this time with MTQ disabled.
|
||||
highlighter.setHandleMultiTermQuery(false);//disable but leave phrase processing enabled
|
||||
|
||||
topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
snippets = highlighter.highlight("body", query, topDocs);
|
||||
|
||||
assertArrayEquals(new String[]{"<b>alpha</b> <b>bravo</b> <b>charlie</b> - charlie bravo alpha"},
|
||||
snippets);
|
||||
}
|
||||
|
||||
public void testMultiValued() throws IOException {
|
||||
indexWriter.addDocument(newDoc("one bravo three", "four bravo six"));
|
||||
initReaderSearcherHighlighter();
|
||||
|
||||
BooleanQuery query = new BooleanQuery.Builder()
|
||||
.add(newPhraseQuery("body", "one bravo"), BooleanClause.Occur.MUST)
|
||||
.add(newPhraseQuery("body", "four bravo"), BooleanClause.Occur.MUST)
|
||||
.add(new PrefixQuery(new Term("body", "br")), BooleanClause.Occur.MUST)
|
||||
.build();
|
||||
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
String[] snippets = highlighter.highlight("body", query, topDocs, 2);
|
||||
|
||||
assertArrayEquals(new String[]{"<b>one</b> <b>bravo</b> three... <b>four</b> <b>bravo</b> six"},
|
||||
snippets);
|
||||
|
||||
|
||||
// now test phraseQuery won't span across values
|
||||
assert indexAnalyzer.getPositionIncrementGap("body") > 0;
|
||||
|
||||
PhraseQuery phraseQuery = newPhraseQuery("body", "three four");
|
||||
// 1 too little; won't span
|
||||
phraseQuery = setSlop(phraseQuery, indexAnalyzer.getPositionIncrementGap("body") - 1);
|
||||
|
||||
query = new BooleanQuery.Builder()
|
||||
.add(new TermQuery(new Term("body", "bravo")), BooleanClause.Occur.MUST)
|
||||
.add(phraseQuery, BooleanClause.Occur.SHOULD)
|
||||
.build();
|
||||
|
||||
topDocs = searcher.search(query, 10);
|
||||
snippets = highlighter.highlight("body", query, topDocs, 2);
|
||||
assertEquals("one <b>bravo</b> three... four <b>bravo</b> six", snippets[0]);
|
||||
|
||||
// and add just enough slop to cross the values:
|
||||
phraseQuery = newPhraseQuery("body", "three four");
|
||||
phraseQuery = setSlop(phraseQuery, indexAnalyzer.getPositionIncrementGap("body")); // just enough to span
|
||||
query = new BooleanQuery.Builder()
|
||||
.add(new TermQuery(new Term("body", "bravo")), BooleanClause.Occur.MUST)
|
||||
.add(phraseQuery, BooleanClause.Occur.MUST) // must match and it will
|
||||
.build();
|
||||
topDocs = searcher.search(query, 10);
|
||||
assertEquals(1, topDocs.totalHits);
|
||||
snippets = highlighter.highlight("body", query, topDocs, 2);
|
||||
assertEquals("one <b>bravo</b> <b>three</b>... <b>four</b> <b>bravo</b> six", snippets[0]);
|
||||
}
|
||||
|
||||
public void testMaxLen() throws IOException {
|
||||
indexWriter.addDocument(newDoc("alpha bravo charlie - gap alpha bravo")); // hyphen is at char 21
|
||||
initReaderSearcherHighlighter();
|
||||
highlighter.setMaxLength(21);
|
||||
|
||||
BooleanQuery query = new BooleanQuery.Builder()
|
||||
.add(newPhraseQuery("body", "alpha bravo"), BooleanClause.Occur.MUST)
|
||||
.add(newPhraseQuery("body", "gap alpha"), BooleanClause.Occur.MUST)
|
||||
.add(newPhraseQuery("body", "charlie gap"), BooleanClause.Occur.SHOULD)
|
||||
.build();
|
||||
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
String[] snippets = highlighter.highlight("body", query, topDocs);
|
||||
|
||||
if (fieldType == UHTestHelper.reanalysisType) {
|
||||
assertArrayEquals(new String[]{"<b>alpha</b> <b>bravo</b> charlie -"}, snippets);
|
||||
} else {
|
||||
assertArrayEquals(new String[]{"<b>alpha</b> <b>bravo</b> <b>charlie</b> -"}, snippets);
|
||||
}
|
||||
}
|
||||
|
||||
public void testFilteredOutSpan() throws IOException {
|
||||
indexWriter.addDocument(newDoc("freezing cold stuff like stuff freedom of speech"));
|
||||
initReaderSearcherHighlighter();
|
||||
|
||||
WildcardQuery wildcardQuery = new WildcardQuery(new Term("body", "free*"));
|
||||
SpanMultiTermQueryWrapper<WildcardQuery> wildcardSpanQuery = new SpanMultiTermQueryWrapper<>(wildcardQuery);
|
||||
SpanTermQuery termQuery = new SpanTermQuery(new Term("body", "speech"));
|
||||
SpanQuery spanQuery = new SpanNearQuery(new SpanQuery[]{wildcardSpanQuery, termQuery}, 3, false);
|
||||
|
||||
BooleanQuery query = new BooleanQuery.Builder()
|
||||
.add(spanQuery, BooleanClause.Occur.MUST)
|
||||
.build();
|
||||
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
String[] snippets = highlighter.highlight("body", query, topDocs);
|
||||
assertArrayEquals(new String[]{"freezing cold stuff like stuff <b>freedom</b> of <b>speech</b>"}, snippets);
|
||||
}
|
||||
|
||||
public void testMatchNoDocsQuery() throws IOException {
|
||||
highlighter = new UnifiedHighlighter(null, indexAnalyzer);
|
||||
highlighter.setHighlightPhrasesStrictly(true);
|
||||
String content = "whatever";
|
||||
Object o = highlighter.highlightWithoutSearcher("body", new MatchNoDocsQuery(), content, 1);
|
||||
assertEquals(content, o);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,182 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.search.uhighlight;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
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.document.FieldType;
|
||||
import org.apache.lucene.index.*;
|
||||
import org.apache.lucene.search.BooleanClause;
|
||||
import org.apache.lucene.search.BooleanQuery;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.Sort;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.lucene.search.TopDocs;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.BitSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Tests highlighting for matters *expressly* relating to term vectors.
|
||||
* <p>
|
||||
* This test DOES NOT represent all testing for highlighting when term vectors are used. Other tests pick the offset
|
||||
* source at random (to include term vectors) and in-effect test term vectors generally.
|
||||
*/
|
||||
@LuceneTestCase.SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom", "Lucene3x"})
|
||||
@LuceneTestCase.SuppressSysoutChecks(bugUrl = "")//Gradle interferes with this Lucene test rule
|
||||
public class TestUnifiedHighlighterTermVec extends LuceneTestCase {
|
||||
|
||||
private Analyzer indexAnalyzer;
|
||||
private Directory dir;
|
||||
|
||||
@Before
|
||||
public void doBefore() throws IOException {
|
||||
indexAnalyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);//whitespace, punctuation, lowercase
|
||||
dir = newDirectory();
|
||||
}
|
||||
|
||||
@After
|
||||
public void doAfter() throws IOException {
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testFetchTermVecsOncePerDoc() throws IOException {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
// Declare some number of fields with random field type; but at least one will have term vectors.
|
||||
final int numTvFields = 1 + random().nextInt(3);
|
||||
List<String> fields = new ArrayList<>(numTvFields);
|
||||
List<FieldType> fieldTypes = new ArrayList<>(numTvFields);
|
||||
for (int i = 0; i < numTvFields; i++) {
|
||||
fields.add("body" + i);
|
||||
fieldTypes.add(UHTestHelper.randomFieldType(random()));
|
||||
}
|
||||
//ensure at least one has TVs by setting one randomly to it:
|
||||
fieldTypes.set(random().nextInt(fieldTypes.size()), UHTestHelper.tvType);
|
||||
|
||||
final int numDocs = 1 + random().nextInt(3);
|
||||
for (int i = 0; i < numDocs; i++) {
|
||||
Document doc = new Document();
|
||||
for (String field : fields) {
|
||||
doc.add(new Field(field, "some test text", UHTestHelper.tvType));
|
||||
}
|
||||
iw.addDocument(doc);
|
||||
}
|
||||
|
||||
// Wrap the reader to ensure we only fetch TVs once per doc
|
||||
DirectoryReader originalReader = iw.getReader();
|
||||
IndexReader ir = new AssertOnceTermVecDirectoryReader(originalReader);
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
BooleanQuery.Builder queryBuilder = new BooleanQuery.Builder();
|
||||
for (String field : fields) {
|
||||
queryBuilder.add(new TermQuery(new Term(field, "test")), BooleanClause.Occur.MUST);
|
||||
}
|
||||
BooleanQuery query = queryBuilder.build();
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
assertEquals(numDocs, topDocs.totalHits);
|
||||
Map<String, String[]> fieldToSnippets =
|
||||
highlighter.highlightFields(fields.toArray(new String[numTvFields]), query, topDocs);
|
||||
String[] expectedSnippetsByDoc = new String[numDocs];
|
||||
Arrays.fill(expectedSnippetsByDoc, "some <b>test</b> text");
|
||||
for (String field : fields) {
|
||||
assertArrayEquals(expectedSnippetsByDoc, fieldToSnippets.get(field));
|
||||
}
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
private static class AssertOnceTermVecDirectoryReader extends FilterDirectoryReader {
|
||||
static final SubReaderWrapper SUB_READER_WRAPPER = new SubReaderWrapper() {
|
||||
@Override
|
||||
public LeafReader wrap(LeafReader reader) {
|
||||
return new FilterLeafReader(reader) {
|
||||
BitSet seenDocIDs = new BitSet();
|
||||
|
||||
@Override
|
||||
public Fields getTermVectors(int docID) throws IOException {
|
||||
// if we're invoked by ParallelLeafReader then we can't do our assertion. TODO see LUCENE-6868
|
||||
if (calledBy(ParallelLeafReader.class) == false
|
||||
&& calledBy(CheckIndex.class) == false) {
|
||||
assertFalse("Should not request TVs for doc more than once.", seenDocIDs.get(docID));
|
||||
seenDocIDs.set(docID);
|
||||
}
|
||||
|
||||
return super.getTermVectors(docID);
|
||||
}
|
||||
};
|
||||
}
|
||||
};
|
||||
|
||||
AssertOnceTermVecDirectoryReader(DirectoryReader in) throws IOException {
|
||||
super(in, SUB_READER_WRAPPER);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
|
||||
return new AssertOnceTermVecDirectoryReader(in);
|
||||
}
|
||||
}
|
||||
|
||||
private static boolean calledBy(Class<?> clazz) {
|
||||
for (StackTraceElement stackTraceElement : Thread.currentThread().getStackTrace()) {
|
||||
if (stackTraceElement.getClassName().equals(clazz.getName()))
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Test(expected = IllegalArgumentException.class)
|
||||
public void testUserFailedToIndexOffsets() throws IOException {
|
||||
FieldType fieldType = new FieldType(UHTestHelper.tvType); // note: it's indexed too
|
||||
fieldType.setStoreTermVectorPositions(random().nextBoolean());
|
||||
fieldType.setStoreTermVectorOffsets(false);
|
||||
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
Document doc = new Document();
|
||||
doc.add(new Field("body", "term vectors", fieldType));
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
TermQuery query = new TermQuery(new Term("body", "vectors"));
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
try {
|
||||
highlighter.highlight("body", query, topDocs, 1);//should throw
|
||||
} finally {
|
||||
ir.close();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,69 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
package org.apache.lucene.search.uhighlight;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.lucene.document.FieldType;
|
||||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.index.IndexOptions;
|
||||
|
||||
/**
|
||||
* Helper for {@link UnifiedHighlighter} tests.
|
||||
*/
|
||||
class UHTestHelper {
|
||||
|
||||
static final FieldType postingsType = new FieldType(TextField.TYPE_STORED);
|
||||
static final FieldType tvType = new FieldType(TextField.TYPE_STORED);
|
||||
static final FieldType postingsWithTvType = new FieldType(TextField.TYPE_STORED);
|
||||
static final FieldType reanalysisType = TextField.TYPE_STORED;
|
||||
|
||||
static {
|
||||
postingsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
|
||||
postingsType.freeze();
|
||||
|
||||
tvType.setStoreTermVectors(true);
|
||||
tvType.setStoreTermVectorPositions(true);
|
||||
tvType.setStoreTermVectorOffsets(true);
|
||||
tvType.freeze();
|
||||
|
||||
postingsWithTvType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
|
||||
postingsWithTvType.setStoreTermVectors(true);
|
||||
postingsWithTvType.freeze();
|
||||
|
||||
|
||||
//re-analysis type needs no further changes.
|
||||
}
|
||||
|
||||
public static FieldType randomFieldType(Random random, FieldType... typePossibilities) {
|
||||
if (typePossibilities == null || typePossibilities.length == 0) {
|
||||
typePossibilities = new FieldType[]{postingsType, tvType, postingsWithTvType, reanalysisType};
|
||||
}
|
||||
return typePossibilities[random.nextInt(typePossibilities.length)];
|
||||
}
|
||||
|
||||
/**
|
||||
* for {@link com.carrotsearch.randomizedtesting.annotations.ParametersFactory}
|
||||
*/
|
||||
// https://github.com/carrotsearch/randomizedtesting/blob/master/examples/maven/src/main/java/com/carrotsearch/examples/randomizedrunner/Test007ParameterizedTests.java
|
||||
static Iterable<Object[]> parametersFactoryList() {
|
||||
return Arrays.asList(new Object[][]{
|
||||
{postingsType}, {tvType}, {postingsWithTvType}, {reanalysisType}
|
||||
});
|
||||
}
|
||||
}
|
|
@ -0,0 +1,182 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
package org.apache.lucene.search.uhighlight.visibility;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.text.BreakIterator;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.SortedSet;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
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.spans.SpanQuery;
|
||||
import org.apache.lucene.search.uhighlight.FieldHighlighter;
|
||||
import org.apache.lucene.search.uhighlight.FieldOffsetStrategy;
|
||||
import org.apache.lucene.search.uhighlight.OffsetsEnum;
|
||||
import org.apache.lucene.search.uhighlight.Passage;
|
||||
import org.apache.lucene.search.uhighlight.PassageFormatter;
|
||||
import org.apache.lucene.search.uhighlight.PassageScorer;
|
||||
import org.apache.lucene.search.uhighlight.PhraseHelper;
|
||||
import org.apache.lucene.search.uhighlight.UnifiedHighlighter;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.automaton.CharacterRunAutomaton;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestUnifiedHighlighterExtensibility extends LuceneTestCase {
|
||||
|
||||
/**
|
||||
* This test is for maintaining the extensibility of the FieldOffsetStrategy
|
||||
* for customizations out of package.
|
||||
*/
|
||||
@Test
|
||||
public void testFieldOffsetStrategyExtensibility() {
|
||||
final UnifiedHighlighter.OffsetSource offsetSource = UnifiedHighlighter.OffsetSource.NONE_NEEDED;
|
||||
FieldOffsetStrategy strategy = new FieldOffsetStrategy("field",
|
||||
new BytesRef[0],
|
||||
PhraseHelper.NONE,
|
||||
new CharacterRunAutomaton[0]) {
|
||||
@Override
|
||||
public UnifiedHighlighter.OffsetSource getOffsetSource() {
|
||||
return offsetSource;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<OffsetsEnum> getOffsetsEnums(IndexReader reader, int docId, String content) throws IOException {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
};
|
||||
assertEquals(offsetSource, strategy.getOffsetSource());
|
||||
}
|
||||
|
||||
/**
|
||||
* This test is for maintaining the extensibility of the UnifiedHighlighter
|
||||
* for customizations out of package.
|
||||
*/
|
||||
@Test
|
||||
public void testUnifiedHighlighterExtensibility() {
|
||||
final int maxLength = 1000;
|
||||
UnifiedHighlighter uh = new UnifiedHighlighter(null, new MockAnalyzer(random())){
|
||||
|
||||
@Override
|
||||
protected Map<String, Object[]> highlightFieldsAsObjects(String[] fieldsIn, Query query, int[] docIdsIn, int[] maxPassagesIn) throws IOException {
|
||||
return super.highlightFieldsAsObjects(fieldsIn, query, docIdsIn, maxPassagesIn);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected OffsetSource getOffsetSource(String field) {
|
||||
return super.getOffsetSource(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected BreakIterator getBreakIterator(String field) {
|
||||
return super.getBreakIterator(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected PassageScorer getScorer(String field) {
|
||||
return super.getScorer(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected PassageFormatter getFormatter(String field) {
|
||||
return super.getFormatter(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Analyzer getIndexAnalyzer() {
|
||||
return super.getIndexAnalyzer();
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexSearcher getIndexSearcher() {
|
||||
return super.getIndexSearcher();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int getMaxNoHighlightPassages(String field) {
|
||||
return super.getMaxNoHighlightPassages(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Boolean requiresRewrite(SpanQuery spanQuery) {
|
||||
return super.requiresRewrite(spanQuery);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected LimitedStoredFieldVisitor newLimitedStoredFieldsVisitor(String[] fields) {
|
||||
return super.newLimitedStoredFieldsVisitor(fields);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<CharSequence[]> loadFieldValues(String[] fields, DocIdSetIterator docIter, int cacheCharsThreshold) throws IOException {
|
||||
return super.loadFieldValues(fields, docIter, cacheCharsThreshold);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected FieldHighlighter getFieldHighlighter(String field, Query query, SortedSet<Term> allTerms, int maxPassages) {
|
||||
return super.getFieldHighlighter(field, query, allTerms, maxPassages);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected FieldOffsetStrategy getOffsetStrategy(String field, Query query, SortedSet<Term> allTerms) {
|
||||
return super.getOffsetStrategy(field, query, allTerms);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxLength() {
|
||||
return maxLength;
|
||||
}
|
||||
};
|
||||
assertEquals(uh.getMaxLength(), maxLength);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPassageFormatterExtensibility() {
|
||||
final Object formattedResponse = new Object();
|
||||
PassageFormatter formatter = new PassageFormatter() {
|
||||
@Override
|
||||
public Object format(Passage[] passages, String content) {
|
||||
return formattedResponse;
|
||||
}
|
||||
};
|
||||
assertEquals(formattedResponse, formatter.format(new Passage[0], ""));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFieldHiglighterExtensibility() {
|
||||
final String fieldName = "fieldName";
|
||||
FieldHighlighter fieldHighlighter = new FieldHighlighter(fieldName, null, null, null, 1, 1, null) {
|
||||
@Override
|
||||
protected Passage[] highlightOffsetsEnums(List<OffsetsEnum> offsetsEnums) throws IOException {
|
||||
return super.highlightOffsetsEnums(offsetsEnums);
|
||||
}
|
||||
};
|
||||
|
||||
assertEquals(fieldHighlighter.getField(), fieldName);
|
||||
}
|
||||
|
||||
}
|
Loading…
Reference in New Issue