LUCENE-8286: UnifiedHighlighter: new HighlightFlag.WEIGHT_MATCHES for MatchesIterator API.

Other API changes: New UHComponents, and FieldOffsetStrategy takes a LeafReader not IndexReader now.
Closes #409
This commit is contained in:
David Smiley 2018-08-29 23:29:25 -04:00
parent 18874a6e36
commit b19ae942f1
22 changed files with 949 additions and 252 deletions

View File

@ -186,6 +186,13 @@ API Changes:
* LUCENE-8437: CheckIndex.Status.cantOpenSegments and missingSegmentVersion
have been removed as they were not computed correctly. (Adrien Grand)
* LUCENE-8286: The UnifiedHighlighter has a new HighlightFlag.WEIGHT_MATCHES flag that
will tell this highlighter to use the new MatchesIterator API as the underlying
approach to navigate matching hits for a query. This mode will highlight more
accurately than any other highlighter, and can mark up phrases as one span instead of
word-by-word. The UH's public internal APIs changed a bit in the process.
(David Smiley)
Bug Fixes:
* LUCENE-8445: Tighten condition when two planes are identical to prevent constructing

View File

@ -23,8 +23,6 @@ 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;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.automaton.CharacterRunAutomaton;
/**
* Provides a base class for analysis based offset strategies to extend from.
@ -37,12 +35,12 @@ public abstract class AnalysisOffsetStrategy extends FieldOffsetStrategy {
protected final Analyzer analyzer;
public AnalysisOffsetStrategy(String field, BytesRef[] queryTerms, PhraseHelper phraseHelper, CharacterRunAutomaton[] automata, Analyzer analyzer) {
super(field, queryTerms, phraseHelper, automata);
public AnalysisOffsetStrategy(UHComponents components, Analyzer analyzer) {
super(components);
this.analyzer = analyzer;
if (analyzer.getOffsetGap(field) != 1) { // note: 1 is the default. It is RARELY changed.
if (analyzer.getOffsetGap(getField()) != 1) { // note: 1 is the default. It is RARELY changed.
throw new IllegalArgumentException(
"offset gap of the provided analyzer should be 1 (field " + field + ")");
"offset gap of the provided analyzer should be 1 (field " + getField() + ")");
}
}
@ -55,12 +53,12 @@ public abstract class AnalysisOffsetStrategy extends FieldOffsetStrategy {
// If there is no splitChar in content then we needn't wrap:
int splitCharIdx = content.indexOf(UnifiedHighlighter.MULTIVAL_SEP_CHAR);
if (splitCharIdx == -1) {
return analyzer.tokenStream(field, content);
return analyzer.tokenStream(getField(), content);
}
TokenStream subTokenStream = analyzer.tokenStream(field, content.substring(0, splitCharIdx));
TokenStream subTokenStream = analyzer.tokenStream(getField(), content.substring(0, splitCharIdx));
return new MultiValueTokenStream(subTokenStream, field, analyzer, content, UnifiedHighlighter.MULTIVAL_SEP_CHAR, splitCharIdx);
return new MultiValueTokenStream(subTokenStream, getField(), analyzer, content, UnifiedHighlighter.MULTIVAL_SEP_CHAR, splitCharIdx);
}
/**

View File

@ -24,7 +24,7 @@ import java.util.Comparator;
import java.util.List;
import java.util.PriorityQueue;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.util.BytesRef;
/**
@ -65,8 +65,7 @@ public class FieldHighlighter {
/**
* 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?
public Object highlightFieldForDoc(LeafReader reader, int docId, String content) throws IOException {
// 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

View File

@ -20,11 +20,15 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.lucene.index.IndexReader;
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.search.IndexSearcher;
import org.apache.lucene.search.Matches;
import org.apache.lucene.search.MatchesIterator;
import org.apache.lucene.search.ScoreMode;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CharsRefBuilder;
import org.apache.lucene.util.automaton.CharacterRunAutomaton;
@ -37,20 +41,14 @@ import org.apache.lucene.util.automaton.CharacterRunAutomaton;
*/
public abstract class FieldOffsetStrategy {
protected final String field;
protected final PhraseHelper phraseHelper; // Query: position-sensitive information
protected final BytesRef[] terms; // Query: all terms we extracted (some may be position sensitive)
protected final CharacterRunAutomaton[] automata; // Query: wildcards (i.e. multi-term query), not position sensitive
protected final UHComponents components;
public FieldOffsetStrategy(String field, BytesRef[] queryTerms, PhraseHelper phraseHelper, CharacterRunAutomaton[] automata) {
this.field = field;
this.terms = queryTerms;
this.phraseHelper = phraseHelper;
this.automata = automata;
public FieldOffsetStrategy(UHComponents components) {
this.components = components;
}
public String getField() {
return field;
return components.getField();
}
public abstract UnifiedHighlighter.OffsetSource getOffsetSource();
@ -60,39 +58,97 @@ public abstract class FieldOffsetStrategy {
*
* Callers are expected to close the returned OffsetsEnum when it has been finished with
*/
public abstract OffsetsEnum getOffsetsEnum(IndexReader reader, int docId, String content) throws IOException;
public abstract OffsetsEnum getOffsetsEnum(LeafReader reader, int docId, String content) throws IOException;
protected OffsetsEnum createOffsetsEnumFromReader(LeafReader leafReader, int doc) throws IOException {
final Terms termsIndex = leafReader.terms(field);
final Terms termsIndex = leafReader.terms(getField());
if (termsIndex == null) {
return OffsetsEnum.EMPTY;
}
final List<OffsetsEnum> offsetsEnums = new ArrayList<>(terms.length + automata.length);
final List<OffsetsEnum> offsetsEnums = new ArrayList<>();
// Handle position insensitive terms (a subset of this.terms field):
final BytesRef[] insensitiveTerms;
if (phraseHelper.hasPositionSensitivity()) {
insensitiveTerms = phraseHelper.getAllPositionInsensitiveTerms();
assert insensitiveTerms.length <= terms.length : "insensitive terms should be smaller set of all terms";
} else {
insensitiveTerms = terms;
}
if (insensitiveTerms.length > 0) {
createOffsetsEnumsForTerms(insensitiveTerms, termsIndex, doc, offsetsEnums);
// Handle Weight.matches approach
if (components.getHighlightFlags().contains(UnifiedHighlighter.HighlightFlag.WEIGHT_MATCHES)) {
createOffsetsEnumsWeightMatcher(leafReader, doc, offsetsEnums);
} else { // classic approach
// Handle position insensitive terms (a subset of this.terms field):
final BytesRef[] insensitiveTerms;
final PhraseHelper phraseHelper = components.getPhraseHelper();
final BytesRef[] terms = components.getTerms();
if (phraseHelper.hasPositionSensitivity()) {
insensitiveTerms = phraseHelper.getAllPositionInsensitiveTerms();
assert insensitiveTerms.length <= terms.length : "insensitive terms should be smaller set of all terms";
} else {
insensitiveTerms = terms;
}
if (insensitiveTerms.length > 0) {
createOffsetsEnumsForTerms(insensitiveTerms, termsIndex, doc, offsetsEnums);
}
// Handle spans
if (phraseHelper.hasPositionSensitivity()) {
phraseHelper.createOffsetsEnumsForSpans(leafReader, doc, offsetsEnums);
}
// Handle automata
if (components.getAutomata().length > 0) {
createOffsetsEnumsForAutomata(termsIndex, doc, offsetsEnums);
}
}
// Handle spans
if (phraseHelper.hasPositionSensitivity()) {
phraseHelper.createOffsetsEnumsForSpans(leafReader, doc, offsetsEnums);
switch (offsetsEnums.size()) {
case 0: return OffsetsEnum.EMPTY;
case 1: return offsetsEnums.get(0);
default: return new OffsetsEnum.MultiOffsetsEnum(offsetsEnums);
}
}
protected void createOffsetsEnumsWeightMatcher(LeafReader _leafReader, int docId, List<OffsetsEnum> results) throws IOException {
// remap fieldMatcher/requireFieldMatch fields to the field we are highlighting
LeafReader leafReader = new FilterLeafReader(_leafReader) {
@Override
public Terms terms(String field) throws IOException {
if (components.getFieldMatcher().test(field)) {
return super.terms(components.getField());
} else {
return super.terms(field);
}
}
// So many subclasses do this!
//these ought to be a default or added via some intermediary like "FilterTransientLeafReader" (exception on close).
@Override
public CacheHelper getCoreCacheHelper() {
return null;
}
@Override
public CacheHelper getReaderCacheHelper() {
return null;
}
};
IndexSearcher indexSearcher = new IndexSearcher(leafReader);
indexSearcher.setQueryCache(null);
Matches matches = indexSearcher.rewrite(components.getQuery())
.createWeight(indexSearcher, ScoreMode.COMPLETE_NO_SCORES, 1.0f)
.matches(leafReader.getContext(), docId);
if (matches == null) {
return; // doc doesn't match
}
for (String field : matches) {
if (components.getFieldMatcher().test(field)) {
MatchesIterator iterator = matches.getMatches(field);
if (iterator == null) {
continue;
}
results.add(new OffsetsEnum.OfMatchesIteratorWithSubs(iterator));
}
}
// Handle automata
if (automata.length > 0) {
createOffsetsEnumsForAutomata(termsIndex, doc, offsetsEnums);
}
return new OffsetsEnum.MultiOffsetsEnum(offsetsEnums);
}
protected void createOffsetsEnumsForTerms(BytesRef[] sourceTerms, Terms termsIndex, int doc, List<OffsetsEnum> results) throws IOException {
@ -102,7 +158,7 @@ public abstract class FieldOffsetStrategy {
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");
throw new IllegalArgumentException("field '" + getField() + "' was indexed without offsets, cannot highlight");
}
if (doc == postingsEnum.advance(doc)) { // now it's positioned, although may be exhausted
results.add(new OffsetsEnum.OfPostings(term, postingsEnum));
@ -112,6 +168,7 @@ public abstract class FieldOffsetStrategy {
}
protected void createOffsetsEnumsForAutomata(Terms termsIndex, int doc, List<OffsetsEnum> results) throws IOException {
final CharacterRunAutomaton[] automata = components.getAutomata();
List<List<PostingsEnum>> automataPostings = new ArrayList<>(automata.length);
for (int i = 0; i < automata.length; i++) {
automataPostings.add(new ArrayList<>());

View File

@ -23,18 +23,15 @@ import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.function.Function;
import java.util.function.Predicate;
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.memory.MemoryIndex;
import org.apache.lucene.search.Query;
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;
@ -47,36 +44,32 @@ import org.apache.lucene.util.automaton.CharacterRunAutomaton;
public class MemoryIndexOffsetStrategy extends AnalysisOffsetStrategy {
private final MemoryIndex memoryIndex;
private final LeafReader leafReader;
private final LeafReader memIndexLeafReader;
private final CharacterRunAutomaton preMemIndexFilterAutomaton;
public MemoryIndexOffsetStrategy(String field, Predicate<String> fieldMatcher, BytesRef[] extractedTerms, PhraseHelper phraseHelper,
CharacterRunAutomaton[] automata, Analyzer analyzer,
public MemoryIndexOffsetStrategy(UHComponents components, Analyzer analyzer,
Function<Query, Collection<Query>> multiTermQueryRewrite) {
super(field, extractedTerms, phraseHelper, automata, analyzer);
boolean storePayloads = phraseHelper.hasPositionSensitivity(); // might be needed
super(components, analyzer);
boolean storePayloads = components.getPhraseHelper().hasPositionSensitivity(); // might be needed
memoryIndex = new MemoryIndex(true, storePayloads);//true==store offsets
leafReader = (LeafReader) memoryIndex.createSearcher().getIndexReader(); // appears to be re-usable
memIndexLeafReader = (LeafReader) memoryIndex.createSearcher().getIndexReader(); // appears to be re-usable
// preFilter for MemoryIndex
preMemIndexFilterAutomaton = buildCombinedAutomaton(fieldMatcher, terms, this.automata, phraseHelper, multiTermQueryRewrite);
preMemIndexFilterAutomaton = buildCombinedAutomaton(components, multiTermQueryRewrite);
}
/**
* Build one {@link CharacterRunAutomaton} matching any term the query might match.
*/
private static CharacterRunAutomaton buildCombinedAutomaton(Predicate<String> fieldMatcher,
BytesRef[] terms,
CharacterRunAutomaton[] automata,
PhraseHelper strictPhrases,
private static CharacterRunAutomaton buildCombinedAutomaton(UHComponents components,
Function<Query, Collection<Query>> multiTermQueryRewrite) {
List<CharacterRunAutomaton> allAutomata = new ArrayList<>();
if (terms.length > 0) {
allAutomata.add(new CharacterRunAutomaton(Automata.makeStringUnion(Arrays.asList(terms))));
if (components.getTerms().length > 0) {
allAutomata.add(new CharacterRunAutomaton(Automata.makeStringUnion(Arrays.asList(components.getTerms()))));
}
Collections.addAll(allAutomata, automata);
for (SpanQuery spanQuery : strictPhrases.getSpanQueries()) {
Collections.addAll(allAutomata, components.getAutomata());
for (SpanQuery spanQuery : components.getPhraseHelper().getSpanQueries()) {
Collections.addAll(allAutomata,
MultiTermHighlighting.extractAutomata(spanQuery, fieldMatcher, true, multiTermQueryRewrite));//true==lookInSpan
MultiTermHighlighting.extractAutomata(spanQuery, components.getFieldMatcher(), true, multiTermQueryRewrite));//true==lookInSpan
}
if (allAutomata.size() == 1) {
@ -100,20 +93,28 @@ public class MemoryIndexOffsetStrategy extends AnalysisOffsetStrategy {
}
@Override
public OffsetsEnum getOffsetsEnum(IndexReader reader, int docId, String content) throws IOException {
public OffsetsEnum getOffsetsEnum(LeafReader reader, int docId, String content) throws IOException {
// note: don't need LimitTokenOffsetFilter since content is already truncated to maxLength
TokenStream tokenStream = tokenStream(content);
// Filter the tokenStream to applicable terms
tokenStream = newKeepWordFilter(tokenStream, preMemIndexFilterAutomaton);
memoryIndex.reset();
memoryIndex.addField(field, tokenStream);//note: calls tokenStream.reset() & close()
docId = 0;
memoryIndex.addField(getField(), tokenStream);//note: calls tokenStream.reset() & close()
return createOffsetsEnumFromReader(leafReader, docId);
if (reader == null) {
return createOffsetsEnumFromReader(memIndexLeafReader, 0);
} else {
return createOffsetsEnumFromReader(
new OverlaySingleDocTermsLeafReader(
reader,
memIndexLeafReader,
getField(),
docId),
docId);
}
}
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?

View File

@ -17,8 +17,10 @@
package org.apache.lucene.search.uhighlight;
import java.io.IOException;
import java.util.Collections;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.search.MatchNoDocsQuery;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.automaton.CharacterRunAutomaton;
@ -32,7 +34,7 @@ public class NoOpOffsetStrategy extends FieldOffsetStrategy {
public static final NoOpOffsetStrategy INSTANCE = new NoOpOffsetStrategy();
private NoOpOffsetStrategy() {
super("_ignored_", new BytesRef[0], PhraseHelper.NONE, new CharacterRunAutomaton[0]);
super(new UHComponents("_ignored_", (s) -> false, new MatchNoDocsQuery(), new BytesRef[0], PhraseHelper.NONE, new CharacterRunAutomaton[0], Collections.emptySet()));
}
@Override
@ -41,7 +43,7 @@ public class NoOpOffsetStrategy extends FieldOffsetStrategy {
}
@Override
public OffsetsEnum getOffsetsEnum(IndexReader reader, int docId, String content) throws IOException {
public OffsetsEnum getOffsetsEnum(LeafReader reader, int docId, String content) throws IOException {
return OffsetsEnum.EMPTY;
}

View File

@ -19,12 +19,20 @@ package org.apache.lucene.search.uhighlight;
import java.io.Closeable;
import java.io.IOException;
import java.util.HashMap;
import java.util.List;
import java.util.Objects;
import java.util.PriorityQueue;
import java.util.TreeSet;
import java.util.function.Supplier;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.MatchesIterator;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.ScoreMode;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.IOUtils;
/**
@ -77,8 +85,10 @@ public abstract class OffsetsEnum implements Comparable<OffsetsEnum>, Closeable
public abstract int freq() throws IOException;
/**
* The term at this position; usually always the same.
* The term at this position.
* This BytesRef is safe to continue to refer to, even after we move to the next position.
*
* @see Passage#getMatchTerms()
*/
public abstract BytesRef getTerm() throws IOException;
@ -93,8 +103,14 @@ public abstract class OffsetsEnum implements Comparable<OffsetsEnum>, Closeable
@Override
public String toString() {
final String name = getClass().getSimpleName();
String offset = "";
try {
return name + "(term:" + getTerm().utf8ToString() +")";
offset = ",[" + startOffset() + "-" + endOffset() + "]";
} catch (Exception e) {
//ignore; for debugging only
}
try {
return name + "(term:" + getTerm().utf8ToString() + offset + ")";
} catch (Exception e) {
return name;
}
@ -157,6 +173,201 @@ public abstract class OffsetsEnum implements Comparable<OffsetsEnum>, Closeable
}
}
/** Based on a {@link MatchesIterator} with submatches. */
public static class OfMatchesIteratorWithSubs extends OffsetsEnum {
//Either CachedOE impls (which are the submatches) or OfMatchesIterator impls
private final PriorityQueue<OffsetsEnum> pendingQueue = new PriorityQueue<>();
private final HashMap<Query,BytesRef> queryToTermMap = new HashMap<>();
public OfMatchesIteratorWithSubs(MatchesIterator matchesIterator) {
pendingQueue.add(new OfMatchesIterator(matchesIterator, () -> queryToTerm(matchesIterator.getQuery())));
}
@Override
public boolean nextPosition() throws IOException {
OffsetsEnum formerHeadOE = pendingQueue.poll(); // removes the head
if (formerHeadOE instanceof CachedOE) {
// we're done with the former head. CachedOE's are one use only.
// Look at the new head...
OffsetsEnum newHeadOE = pendingQueue.peek();
if (newHeadOE instanceof OfMatchesIterator) {
// We found the matchesIterator. Requires processing.
nextWhenMatchesIterator((OfMatchesIterator) newHeadOE); // May or may not remove or re-queue itself
} // else new head is a CacheOE or no more. Nothing to do with it.
} else { // formerHeadOE is OfMatchesIterator; advance it
OfMatchesIterator miOE = (OfMatchesIterator) formerHeadOE;
if (miOE.nextPosition()) {
nextWhenMatchesIterator(miOE); // requires processing. May or may not re-enqueue itself
}
}
return pendingQueue.isEmpty() == false;
}
private void nextWhenMatchesIterator(OfMatchesIterator miOE) throws IOException {
boolean isHead = miOE == pendingQueue.peek();
MatchesIterator subMatches = miOE.matchesIterator.getSubMatches();
if (subMatches != null) {
// remove this miOE from the queue, add it's submatches, next() it, then re-enqueue it
if (isHead) {
pendingQueue.poll(); // remove
}
enqueueCachedMatches(subMatches);
if (miOE.nextPosition()) {
pendingQueue.add(miOE);
assert pendingQueue.peek() != miOE; // miOE should follow cached entries
}
} else { // else has no subMatches. It will stay enqueued.
if (!isHead) {
pendingQueue.add(miOE);
} // else it's *already* in pendingQueue
}
}
private boolean enqueueCachedMatches(MatchesIterator thisMI) throws IOException {
if (thisMI == null) {
return false;
} else {
while (thisMI.next()) {
if (false == enqueueCachedMatches(thisMI.getSubMatches())) { // recursion
// if no sub-matches then add ourselves
pendingQueue.add(new CachedOE(queryToTerm(thisMI.getQuery()), thisMI.startOffset(), thisMI.endOffset()));
}
}
return true;
}
}
/** Maps a Query from {@link MatchesIterator#getQuery()} to {@link OffsetsEnum#getTerm()}.
* See {@link Passage#getMatchTerms()}. */
private BytesRef queryToTerm(Query query) {
// compute an approximate BytesRef term of a Query. We cache this since we're likely to see the same query again.
// Our approach is to call extractTerms and visit each term in order, concatenating them with an adjoining space.
// If we don't have any (perhaps due to an MTQ like a wildcard) then we fall back on the toString() of the query.
return queryToTermMap.computeIfAbsent(query, (Query q) -> {
try {
BytesRefBuilder bytesRefBuilder = new BytesRefBuilder();
UnifiedHighlighter.EMPTY_INDEXSEARCHER
.createWeight(UnifiedHighlighter.EMPTY_INDEXSEARCHER.rewrite(q), ScoreMode.COMPLETE_NO_SCORES, 1f)
.extractTerms(new TreeSet<Term>() {
@Override
public boolean add(Term term) {
if (bytesRefBuilder.length() > 0) {
bytesRefBuilder.append((byte) ' ');
}
bytesRefBuilder.append(term.bytes());
return true;
}
});
if (bytesRefBuilder.length() > 0) {
return bytesRefBuilder.get();
}
} catch (IOException e) {//ignore
// go to fallback...
}
// fallback: (likely a MultiTermQuery)
return new BytesRef(q.toString());
});
}
@Override
public int freq() throws IOException {
return pendingQueue.peek().freq();
}
@Override
public BytesRef getTerm() throws IOException {
return pendingQueue.peek().getTerm();
}
@Override
public int startOffset() throws IOException {
return pendingQueue.peek().startOffset();
}
@Override
public int endOffset() throws IOException {
return pendingQueue.peek().endOffset();
}
private static class CachedOE extends OffsetsEnum {
final BytesRef term;
final int startOffset;
final int endOffset;
private CachedOE(BytesRef term, int startOffset, int endOffset) {
this.term = term;
this.startOffset = startOffset;
this.endOffset = endOffset;
}
@Override
public boolean nextPosition() throws IOException {
return false;
}
@Override
public int freq() throws IOException {
return 1; // documented short-coming of MatchesIterator based UnifiedHighlighter
}
@Override
public BytesRef getTerm() throws IOException {
return term;
}
@Override
public int startOffset() throws IOException {
return startOffset;
}
@Override
public int endOffset() throws IOException {
return endOffset;
}
}
}
/** Based on a {@link MatchesIterator}; does not look at submatches. */
public static class OfMatchesIterator extends OffsetsEnum {
private final MatchesIterator matchesIterator;
private final Supplier<BytesRef> termSupplier;
public OfMatchesIterator(MatchesIterator matchesIterator, Supplier<BytesRef> termSupplier) {
this.matchesIterator = matchesIterator;
this.termSupplier = termSupplier;
}
@Override
public boolean nextPosition() throws IOException {
return matchesIterator.next();
}
@Override
public int freq() throws IOException {
return 1; // documented short-coming of MatchesIterator based UnifiedHighlighter
}
@Override
public BytesRef getTerm() throws IOException {
return termSupplier.get();
}
@Override
public int startOffset() throws IOException {
return matchesIterator.startOffset();
}
@Override
public int endOffset() throws IOException {
return matchesIterator.endOffset();
}
}
/**
* Empty enumeration
*/
@ -191,6 +402,7 @@ public abstract class OffsetsEnum implements Comparable<OffsetsEnum>, Closeable
/**
* A view over several OffsetsEnum instances, merging them in-place
*/
//If OffsetsEnum and MatchesIterator ever truly merge then this could go away in lieu of DisjunctionMatchesIterator
public static class MultiOffsetsEnum extends OffsetsEnum {
private final PriorityQueue<OffsetsEnum> queue;

View File

@ -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;
import java.io.IOException;
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;
/**
* Overlays a 2nd LeafReader for the terms of one field, otherwise the primary reader is
* consulted. The 2nd reader is assumed to have one document of 0 and we remap it to a target doc ID.
*
* @lucene.internal
*/
public class OverlaySingleDocTermsLeafReader extends FilterLeafReader {
private final LeafReader in2;
private final String in2Field;
private final int in2TargetDocId;
public OverlaySingleDocTermsLeafReader(LeafReader in, LeafReader in2, String in2Field, int in2TargetDocId) {
super(in);
this.in2 = in2;
this.in2Field = in2Field;
this.in2TargetDocId = in2TargetDocId;
assert in2.maxDoc() == 1;
}
@Override
public Terms terms(String field) throws IOException {
if (!in2Field.equals(field)) {
return in.terms(field);
}
// Shifts leafReader in2 with only doc ID 0 to a target doc ID
final Terms terms = in2.terms(field);
if (terms == null) {
return null;
}
if (in2TargetDocId == 0) { // no doc ID remapping to do
return terms;
}
return new FilterTerms(terms) {
@Override
public TermsEnum iterator() throws IOException {
return filterTermsEnum(super.iterator());
}
@Override
public TermsEnum intersect(CompiledAutomaton compiled, BytesRef startTerm) throws IOException {
return filterTermsEnum(super.intersect(compiled, startTerm));
}
private TermsEnum filterTermsEnum(TermsEnum termsEnum) throws IOException {
return new FilterTermsEnum(termsEnum) {
@Override
public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
//TODO 'reuse' will always fail to reuse unless we unwrap it
return new FilterPostingsEnum(super.postings(reuse, flags)) {
@Override
public int nextDoc() throws IOException {
final int doc = super.nextDoc();
return doc == 0 ? in2TargetDocId : doc;
}
@Override
public int advance(int target) throws IOException {
return slowAdvance(target);
}
@Override
public int docID() {
final int doc = super.docID();
return doc == 0 ? in2TargetDocId : doc;
}
};
}
};
}
};
}
@Override
public CacheHelper getCoreCacheHelper() {
return null;
}
@Override
public CacheHelper getReaderCacheHelper() {
return null;
}
}

View File

@ -157,7 +157,16 @@ public class Passage {
}
/**
* BytesRef (term text) of the matches, corresponding with {@link #getMatchStarts()}.
* BytesRef (term text) of the matches, corresponding with {@link #getMatchStarts()}. The primary purpose of this
* method is to expose the number of unique terms per passage for use in passage scoring.
* The actual term byte content is not well defined by this highlighter, and thus use of it is more subject to
* change.
* <p>
* The term might be simply the analyzed term at this position.
* Depending on the highlighter's configuration, the match term may be a phrase (instead of a word), and in such
* a case might be a series of space-separated analyzed terms.
* If the match is from a {@link org.apache.lucene.search.MultiTermQuery} then the match term may be the toString() of
* that query.
* <p>
* Only {@link #getNumMatches()} are valid.
*/

View File

@ -17,15 +17,9 @@
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
@ -35,23 +29,13 @@ import org.apache.lucene.util.automaton.CharacterRunAutomaton;
*/
public class PostingsOffsetStrategy extends FieldOffsetStrategy {
public PostingsOffsetStrategy(String field, BytesRef[] queryTerms, PhraseHelper phraseHelper, CharacterRunAutomaton[] automata) {
super(field, queryTerms, phraseHelper, automata);
public PostingsOffsetStrategy(UHComponents components) {
super(components);
}
@Override
public OffsetsEnum getOffsetsEnum(IndexReader reader, int docId, String content) throws IOException {
final 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 createOffsetsEnumFromReader(leafReader, docId);
public OffsetsEnum getOffsetsEnum(LeafReader reader, int docId, String content) throws IOException {
return createOffsetsEnumFromReader(reader, docId);
}

View File

@ -17,15 +17,9 @@
package org.apache.lucene.search.uhighlight;
import java.io.IOException;
import java.util.List;
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.
@ -34,27 +28,17 @@ import org.apache.lucene.util.automaton.CharacterRunAutomaton;
*/
public class PostingsWithTermVectorsOffsetStrategy extends FieldOffsetStrategy {
public PostingsWithTermVectorsOffsetStrategy(String field, BytesRef[] queryTerms, PhraseHelper phraseHelper, CharacterRunAutomaton[] automata) {
super(field, queryTerms, phraseHelper, automata);
public PostingsWithTermVectorsOffsetStrategy(UHComponents components) {
super(components);
}
@Override
public OffsetsEnum getOffsetsEnum(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);
public OffsetsEnum getOffsetsEnum(LeafReader leafReader, int docId, String content) throws IOException {
Terms docTerms = leafReader.getTermVector(docId, getField());
if (docTerms == null) {
return OffsetsEnum.EMPTY;
}
leafReader = new TermVectorFilteredLeafReader(leafReader, docTerms);
leafReader = new TermVectorFilteredLeafReader(leafReader, docTerms, getField());
return createOffsetsEnumFromReader(leafReader, docId);
}

View File

@ -37,21 +37,26 @@ final class TermVectorFilteredLeafReader extends FilterLeafReader {
// NOTE: super ("in") is baseLeafReader
private final Terms filterTerms;
private final String fieldFilter;
/**
* <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.
* @param fieldFilter the field to do this on
*/
TermVectorFilteredLeafReader(LeafReader baseLeafReader, Terms filterTerms) {
TermVectorFilteredLeafReader(LeafReader baseLeafReader, Terms filterTerms, String fieldFilter) {
super(baseLeafReader);
this.filterTerms = filterTerms;
this.fieldFilter = fieldFilter;
}
@Override
public Terms terms(String field) throws IOException {
if (!field.equals(fieldFilter)) {
return super.terms(field); // proceed like normal for fields we're not interested in
}
Terms terms = in.terms(field);
return terms==null ? null : new TermsFilteredTerms(terms, filterTerms);
}
@ -106,7 +111,7 @@ final class TermVectorFilteredLeafReader extends FilterLeafReader {
boolean termInBothTermsEnum = baseTermsEnum.seekExact(currentTerm);
if (!termInBothTermsEnum) {
throw new IllegalStateException("Term vector term " + currentTerm + " does not appear in full index.");
throw new IllegalStateException("Term vector term '" + currentTerm.utf8ToString() + "' does not appear in full index.");
}
}

View File

@ -18,12 +18,9 @@ package org.apache.lucene.search.uhighlight;
import java.io.IOException;
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.
@ -32,8 +29,8 @@ import org.apache.lucene.util.automaton.CharacterRunAutomaton;
*/
public class TermVectorOffsetStrategy extends FieldOffsetStrategy {
public TermVectorOffsetStrategy(String field, BytesRef[] queryTerms, PhraseHelper phraseHelper, CharacterRunAutomaton[] automata) {
super(field, queryTerms, phraseHelper, automata);
public TermVectorOffsetStrategy(UHComponents components) {
super(components);
}
@Override
@ -42,16 +39,20 @@ public class TermVectorOffsetStrategy extends FieldOffsetStrategy {
}
@Override
public OffsetsEnum getOffsetsEnum(IndexReader reader, int docId, String content) throws IOException {
Terms tvTerms = reader.getTermVector(docId, field);
public OffsetsEnum getOffsetsEnum(LeafReader reader, int docId, String content) throws IOException {
Terms tvTerms = reader.getTermVector(docId, getField());
if (tvTerms == null) {
return OffsetsEnum.EMPTY;
}
LeafReader leafReader = new TermVectorLeafReader(field, tvTerms);
docId = 0;
return createOffsetsEnumFromReader(leafReader, docId);
LeafReader singleDocReader = new TermVectorLeafReader(getField(), tvTerms);
return createOffsetsEnumFromReader(
new OverlaySingleDocTermsLeafReader(
reader,
singleDocReader,
getField(),
docId),
docId);
}
}

View File

@ -22,7 +22,7 @@ import org.apache.lucene.analysis.Analyzer;
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.IndexReader;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.automaton.Automata;
import org.apache.lucene.util.automaton.CharacterRunAutomaton;
@ -36,9 +36,17 @@ public class TokenStreamOffsetStrategy extends AnalysisOffsetStrategy {
private static final BytesRef[] ZERO_LEN_BYTES_REF_ARRAY = new BytesRef[0];
public TokenStreamOffsetStrategy(String field, BytesRef[] terms, PhraseHelper phraseHelper, CharacterRunAutomaton[] automata, Analyzer indexAnalyzer) {
super(field, ZERO_LEN_BYTES_REF_ARRAY, phraseHelper, convertTermsToAutomata(terms, automata), indexAnalyzer);
assert phraseHelper.hasPositionSensitivity() == false;
public TokenStreamOffsetStrategy(UHComponents components, Analyzer indexAnalyzer) {
super(new UHComponents(
components.getField(),
components.getFieldMatcher(),
components.getQuery(),
ZERO_LEN_BYTES_REF_ARRAY,
components.getPhraseHelper(),
convertTermsToAutomata(components.getTerms(), components.getAutomata()),
components.getHighlightFlags()),
indexAnalyzer);
assert components.getPhraseHelper().hasPositionSensitivity() == false;
}
private static CharacterRunAutomaton[] convertTermsToAutomata(BytesRef[] terms, CharacterRunAutomaton[] automata) {
@ -58,8 +66,8 @@ public class TokenStreamOffsetStrategy extends AnalysisOffsetStrategy {
}
@Override
public OffsetsEnum getOffsetsEnum(IndexReader reader, int docId, String content) throws IOException {
return new TokenStreamOffsetsEnum(tokenStream(content), automata);
public OffsetsEnum getOffsetsEnum(LeafReader reader, int docId, String content) throws IOException {
return new TokenStreamOffsetsEnum(tokenStream(content), components.getAutomata());
}
private static class TokenStreamOffsetsEnum extends OffsetsEnum {

View File

@ -0,0 +1,80 @@
/*
* 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.Set;
import java.util.function.Predicate;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.automaton.CharacterRunAutomaton;
/**
* A parameter object to hold the components a {@link FieldOffsetStrategy} needs.
*
* @lucene.internal
*/
public class UHComponents {
private final String field;
private final Predicate<String> fieldMatcher;
private final Query query;
private final BytesRef[] terms; // Query: all terms we extracted (some may be position sensitive)
private final PhraseHelper phraseHelper; // Query: position-sensitive information
private final CharacterRunAutomaton[] automata; // Query: wildcards (i.e. multi-term query), not position sensitive
private final Set<UnifiedHighlighter.HighlightFlag> highlightFlags;
public UHComponents(String field, Predicate<String> fieldMatcher, Query query,
BytesRef[] terms, PhraseHelper phraseHelper, CharacterRunAutomaton[] automata,
Set<UnifiedHighlighter.HighlightFlag> highlightFlags) {
this.field = field;
this.fieldMatcher = fieldMatcher;
this.query = query;
this.terms = terms;
this.phraseHelper = phraseHelper;
this.automata = automata;
this.highlightFlags = highlightFlags;
}
public String getField() {
return field;
}
public Predicate<String> getFieldMatcher() {
return fieldMatcher;
}
public Query getQuery() {
return query;
}
public BytesRef[] getTerms() {
return terms;
}
public PhraseHelper getPhraseHelper() {
return phraseHelper;
}
public CharacterRunAutomaton[] getAutomata() {
return automata;
}
public Set<UnifiedHighlighter.HighlightFlag> getHighlightFlags() {
return highlightFlags;
}
}

View File

@ -48,6 +48,7 @@ import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.MultiFields;
import org.apache.lucene.index.MultiReader;
import org.apache.lucene.index.ReaderUtil;
import org.apache.lucene.index.StoredFieldVisitor;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.DocIdSetIterator;
@ -623,11 +624,20 @@ public class UnifiedHighlighter {
&& indexReaderWithTermVecCache != null)
? indexReaderWithTermVecCache
: searcher.getIndexReader();
final LeafReader leafReader;
if (indexReader instanceof LeafReader) {
leafReader = (LeafReader) indexReader;
} else {
List<LeafReaderContext> leaves = indexReader.leaves();
LeafReaderContext leafReaderContext = leaves.get(ReaderUtil.subIndex(docId, leaves));
leafReader = leafReaderContext.reader();
docId -= leafReaderContext.docBase; // adjust 'doc' to be within this leaf reader
}
int docInIndex = docInIndexes[docIdx];//original input order
assert resultByDocIn[docInIndex] == null;
resultByDocIn[docInIndex] =
fieldHighlighter
.highlightFieldForDoc(indexReader, docId, content.toString());
.highlightFieldForDoc(leafReader, docId, content.toString());
}
}
@ -742,13 +752,15 @@ public class UnifiedHighlighter {
}
protected FieldHighlighter getFieldHighlighter(String field, Query query, Set<Term> allTerms, int maxPassages) {
BytesRef[] terms = filterExtractedTerms(getFieldMatcher(field), allTerms);
Predicate<String> fieldMatcher = getFieldMatcher(field);
BytesRef[] terms = filterExtractedTerms(fieldMatcher, allTerms);
Set<HighlightFlag> highlightFlags = getFlags(field);
PhraseHelper phraseHelper = getPhraseHelper(field, query, highlightFlags);
CharacterRunAutomaton[] automata = getAutomata(field, query, highlightFlags);
OffsetSource offsetSource = getOptimizedOffsetSource(field, terms, phraseHelper, automata);
UHComponents components = new UHComponents(field, fieldMatcher, query, terms, phraseHelper, automata, highlightFlags);
return new FieldHighlighter(field,
getOffsetStrategy(offsetSource, field, terms, phraseHelper, automata, highlightFlags),
getOffsetStrategy(offsetSource, components),
new SplittingBreakIterator(getBreakIterator(field), UnifiedHighlighter.MULTIVAL_SEP_CHAR),
getScorer(field),
maxPassages,
@ -782,16 +794,30 @@ public class UnifiedHighlighter {
}
protected PhraseHelper getPhraseHelper(String field, Query query, Set<HighlightFlag> highlightFlags) {
boolean useWeightMatchesIter = highlightFlags.contains(HighlightFlag.WEIGHT_MATCHES);
if (useWeightMatchesIter) {
return PhraseHelper.NONE; // will be handled by Weight.matches which always considers phrases
}
boolean highlightPhrasesStrictly = highlightFlags.contains(HighlightFlag.PHRASES);
boolean handleMultiTermQuery = highlightFlags.contains(HighlightFlag.MULTI_TERM_QUERY);
return highlightPhrasesStrictly ?
new PhraseHelper(query, field, getFieldMatcher(field),
this::requiresRewrite, this::preSpanQueryRewrite, !handleMultiTermQuery) : PhraseHelper.NONE;
this::requiresRewrite,
this::preSpanQueryRewrite,
!handleMultiTermQuery
)
: PhraseHelper.NONE;
}
protected CharacterRunAutomaton[] getAutomata(String field, Query query, Set<HighlightFlag> highlightFlags) {
// do we "eagerly" look in span queries for automata here, or do we not and let PhraseHelper handle those?
// if don't highlight phrases strictly,
final boolean lookInSpan =
!highlightFlags.contains(HighlightFlag.PHRASES) // no PhraseHelper
|| highlightFlags.contains(HighlightFlag.WEIGHT_MATCHES); // Weight.Matches will find all
return highlightFlags.contains(HighlightFlag.MULTI_TERM_QUERY)
? MultiTermHighlighting.extractAutomata(query, getFieldMatcher(field), !highlightFlags.contains(HighlightFlag.PHRASES), this::preMultiTermQueryRewrite)
? MultiTermHighlighting.extractAutomata(query, getFieldMatcher(field), lookInSpan, this::preMultiTermQueryRewrite)
: ZERO_LEN_AUTOMATA_ARRAY;
}
@ -829,27 +855,25 @@ public class UnifiedHighlighter {
return offsetSource;
}
protected FieldOffsetStrategy getOffsetStrategy(OffsetSource offsetSource, String field, BytesRef[] terms,
PhraseHelper phraseHelper, CharacterRunAutomaton[] automata,
Set<HighlightFlag> highlightFlags) {
protected FieldOffsetStrategy getOffsetStrategy(OffsetSource offsetSource, UHComponents components) {
switch (offsetSource) {
case ANALYSIS:
if (!phraseHelper.hasPositionSensitivity() &&
!highlightFlags.contains(HighlightFlag.PASSAGE_RELEVANCY_OVER_SPEED)) {
if (!components.getPhraseHelper().hasPositionSensitivity() &&
!components.getHighlightFlags().contains(HighlightFlag.PASSAGE_RELEVANCY_OVER_SPEED) &&
!components.getHighlightFlags().contains(HighlightFlag.WEIGHT_MATCHES)) {
//skip using a memory index since it's pure term filtering
return new TokenStreamOffsetStrategy(field, terms, phraseHelper, automata, getIndexAnalyzer());
return new TokenStreamOffsetStrategy(components, getIndexAnalyzer());
} else {
return new MemoryIndexOffsetStrategy(field, getFieldMatcher(field), terms, phraseHelper, automata, getIndexAnalyzer(),
this::preMultiTermQueryRewrite);
return new MemoryIndexOffsetStrategy(components, getIndexAnalyzer(), this::preMultiTermQueryRewrite);
}
case NONE_NEEDED:
return NoOpOffsetStrategy.INSTANCE;
case TERM_VECTORS:
return new TermVectorOffsetStrategy(field, terms, phraseHelper, automata);
return new TermVectorOffsetStrategy(components);
case POSTINGS:
return new PostingsOffsetStrategy(field, terms, phraseHelper, automata);
return new PostingsOffsetStrategy(components);
case POSTINGS_WITH_TERM_VECTORS:
return new PostingsWithTermVectorsOffsetStrategy(field, terms, phraseHelper, automata);
return new PostingsWithTermVectorsOffsetStrategy(components);
default:
throw new IllegalArgumentException("Unrecognized offset source " + offsetSource);
}
@ -1088,10 +1112,23 @@ public class UnifiedHighlighter {
* Flags for controlling highlighting behavior.
*/
public enum HighlightFlag {
/** @see UnifiedHighlighter#setHighlightPhrasesStrictly(boolean) */
PHRASES,
/** @see UnifiedHighlighter#setHandleMultiTermQuery(boolean) */
MULTI_TERM_QUERY,
PASSAGE_RELEVANCY_OVER_SPEED
// TODO: ignoreQueryFields
/** Passage relevancy is more important than speed. True by default. */
PASSAGE_RELEVANCY_OVER_SPEED,
/**
* Internally use the {@link Weight#matches(LeafReaderContext, int)} API for highlighting.
* It's more accurate to the query, though might not calculate passage relevancy as well.
* Use of this flag requires {@link #MULTI_TERM_QUERY} and {@link #PHRASES}.
* {@link #PASSAGE_RELEVANCY_OVER_SPEED} will be ignored. False by default.
*/
WEIGHT_MATCHES
// TODO: useQueryBoosts
}
}

View File

@ -86,33 +86,39 @@ public class TestUnifiedHighlighter extends LuceneTestCase {
}
static UnifiedHighlighter randomUnifiedHighlighter(IndexSearcher searcher, Analyzer indexAnalyzer) {
return randomUnifiedHighlighter(searcher, indexAnalyzer, EnumSet.noneOf(HighlightFlag.class));
return randomUnifiedHighlighter(searcher, indexAnalyzer, EnumSet.noneOf(HighlightFlag.class), null);
}
static UnifiedHighlighter randomUnifiedHighlighter(IndexSearcher searcher, Analyzer indexAnalyzer,
EnumSet<HighlightFlag> mandatoryFlags) {
if (random().nextBoolean()) {
return new UnifiedHighlighter(searcher, indexAnalyzer);
} else {
final UnifiedHighlighter uh = new UnifiedHighlighter(searcher, indexAnalyzer) {
@Override
protected Set<HighlightFlag> getFlags(String field) {
final EnumSet<HighlightFlag> result = EnumSet.copyOf(mandatoryFlags);
int r = random().nextInt();
for (HighlightFlag highlightFlag : HighlightFlag.values()) {
if (((1 << highlightFlag.ordinal()) & r) == 0) {
result.add(highlightFlag);
}
}
return result;
EnumSet<HighlightFlag> mandatoryFlags, Boolean requireFieldMatch) {
final UnifiedHighlighter uh = new UnifiedHighlighter(searcher, indexAnalyzer) {
Set<HighlightFlag> flags; // consistently random set of flags for this test run
@Override
protected Set<HighlightFlag> getFlags(String field) {
if (flags != null) {
return flags;
}
};
uh.setCacheFieldValCharsThreshold(random().nextInt(100));
if (random().nextBoolean()) {
uh.setFieldMatcher(f -> true); // requireFieldMatch==false
final EnumSet<HighlightFlag> result = EnumSet.copyOf(mandatoryFlags);
int r = random().nextInt();
for (HighlightFlag highlightFlag : HighlightFlag.values()) {
if (((1 << highlightFlag.ordinal()) & r) == 0) {
result.add(highlightFlag);
}
}
if (result.contains(HighlightFlag.WEIGHT_MATCHES)) {
// these two are required for WEIGHT_MATCHES
result.add(HighlightFlag.MULTI_TERM_QUERY);
result.add(HighlightFlag.PHRASES);
}
return flags = result;
}
return uh;
};
uh.setCacheFieldValCharsThreshold(random().nextInt(100));
if (requireFieldMatch == Boolean.FALSE || (requireFieldMatch == null && random().nextBoolean())) {
uh.setFieldMatcher(f -> true); // requireFieldMatch==false
}
return uh;
}
//
@ -420,7 +426,11 @@ public class TestUnifiedHighlighter extends LuceneTestCase {
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>"));
if (highlighter.getFlags("body").containsAll(EnumSet.of(HighlightFlag.WEIGHT_MATCHES, HighlightFlag.PHRASES))) {
assertTrue(snippets[0], snippets[0].contains("<b>Buddhist origins</b>"));
} else {
assertTrue(snippets[0], snippets[0].contains("<b>Buddhist</b> <b>origins</b>"));
}
ir.close();
}
@ -1113,7 +1123,7 @@ public class TestUnifiedHighlighter extends LuceneTestCase {
return (qf) -> true;
}
};
UnifiedHighlighter highlighterFieldMatch = randomUnifiedHighlighter(searcher, indexAnalyzer, EnumSet.of(HighlightFlag.MULTI_TERM_QUERY));
UnifiedHighlighter highlighterFieldMatch = randomUnifiedHighlighter(searcher, indexAnalyzer, EnumSet.of(HighlightFlag.MULTI_TERM_QUERY), null);
highlighterFieldMatch.setFieldMatcher(null);//default
BooleanQuery.Builder queryBuilder =
new BooleanQuery.Builder()
@ -1187,6 +1197,23 @@ public class TestUnifiedHighlighter extends LuceneTestCase {
ir.close();
}
public void testMatchesSlopBug() throws IOException {
IndexReader ir = indexSomeFields();
IndexSearcher searcher = newSearcher(ir);
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
Query query = new PhraseQuery(2, "title", "this", "is", "the", "field");
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
assertEquals(1, topDocs.totalHits.value);
String[] snippets = highlighter.highlight("title", query, topDocs, 10);
assertEquals(1, snippets.length);
if (highlighter.getFlags("title").contains(HighlightFlag.WEIGHT_MATCHES)) {
assertEquals("<b>This is the title field</b>.", snippets[0]);
} else {
assertEquals("<b>This</b> <b>is</b> <b>the</b> title <b>field</b>.", snippets[0]);
}
ir.close();
}
public void testFieldMatcherPhraseQuery() throws Exception {
IndexReader ir = indexSomeFields();
IndexSearcher searcher = newSearcher(ir);
@ -1197,7 +1224,7 @@ public class TestUnifiedHighlighter extends LuceneTestCase {
return (qf) -> true;
}
};
UnifiedHighlighter highlighterFieldMatch = randomUnifiedHighlighter(searcher, indexAnalyzer, EnumSet.of(HighlightFlag.PHRASES));
UnifiedHighlighter highlighterFieldMatch = randomUnifiedHighlighter(searcher, indexAnalyzer, EnumSet.of(HighlightFlag.PHRASES, HighlightFlag.MULTI_TERM_QUERY), null);
highlighterFieldMatch.setFieldMatcher(null);//default
BooleanQuery.Builder queryBuilder =
new BooleanQuery.Builder()
@ -1214,16 +1241,28 @@ public class TestUnifiedHighlighter extends LuceneTestCase {
assertEquals(1, topDocs.totalHits.value);
String[] snippets = highlighterNoFieldMatch.highlight("title", query, topDocs, 10);
assertEquals(1, snippets.length);
assertEquals("<b>This</b> <b>is</b> <b>the</b> <b>title</b> <b>field</b>.", snippets[0]);
if (highlighterNoFieldMatch.getFlags("title").contains(HighlightFlag.WEIGHT_MATCHES)) {
assertEquals("<b>This is the title field</b>.", snippets[0]);
} else {
assertEquals("<b>This</b> <b>is</b> <b>the</b> <b>title</b> <b>field</b>.", snippets[0]);
}
snippets = highlighterFieldMatch.highlight("title", query, topDocs, 10);
assertEquals(1, snippets.length);
assertEquals("<b>This</b> <b>is</b> <b>the</b> <b>title</b> field.", snippets[0]);
if (highlighterFieldMatch.getFlags("title").contains(HighlightFlag.WEIGHT_MATCHES)) {
assertEquals("<b>This is the title</b> field.", snippets[0]);
} else {
assertEquals("<b>This</b> <b>is</b> <b>the</b> <b>title</b> field.", snippets[0]);
}
highlighterFieldMatch.setFieldMatcher((fq) -> "text".equals(fq));
snippets = highlighterFieldMatch.highlight("title", query, topDocs, 10);
assertEquals(1, snippets.length);
assertEquals("<b>This</b> <b>is</b> the title field.", snippets[0]);
if (highlighterFieldMatch.getFlags("title").contains(HighlightFlag.WEIGHT_MATCHES)) {
assertEquals("<b>This is</b> the title field.", snippets[0]);
} else {
assertEquals("<b>This</b> <b>is</b> the title field.", snippets[0]);
}
highlighterFieldMatch.setFieldMatcher(null);
}
@ -1233,11 +1272,21 @@ public class TestUnifiedHighlighter extends LuceneTestCase {
assertEquals(1, topDocs.totalHits.value);
String[] snippets = highlighterNoFieldMatch.highlight("text", query, topDocs, 10);
assertEquals(1, snippets.length);
assertEquals("<b>This</b> <b>is</b> <b>the</b> <b>text</b> <b>field</b>. <b>You</b> <b>can</b> <b>put</b> some <b>text</b> if you want.", snippets[0]);
if (highlighterNoFieldMatch.getFlags("text").contains(HighlightFlag.WEIGHT_MATCHES)) {
assertEquals("<b>This is the text field</b>. <b>You can put some text</b> if you want.", snippets[0]);
} else {
assertEquals("<b>This</b> <b>is</b> <b>the</b> <b>text</b> <b>field</b>. <b>You</b> <b>can</b> <b>put</b> some <b>text</b> if you want.", snippets[0]);
}
snippets = highlighterFieldMatch.highlight("text", query, topDocs, 10);
assertEquals(1, snippets.length);
assertEquals("<b>This</b> <b>is</b> the <b>text</b> field. <b>You</b> <b>can</b> <b>put</b> some <b>text</b> if you want.", snippets[0]);
if (highlighterFieldMatch.getFlags("text").contains(HighlightFlag.WEIGHT_MATCHES)) {
assertEquals("<b>This is</b> the text field. <b>You can put some text</b> if you want.", snippets[0]);
} else {
// note: odd that the first "text" is highlighted. Apparently WSTE converts PhraseQuery to a SpanNearQuery with
// with inorder=false when non-0 slop. Probably a bug.
assertEquals("<b>This</b> <b>is</b> the <b>text</b> field. <b>You</b> <b>can</b> <b>put</b> some <b>text</b> if you want.", snippets[0]);
}
highlighterFieldMatch.setFieldMatcher((fq) -> "title".equals(fq));
snippets = highlighterFieldMatch.highlight("text", query, topDocs, 10);
@ -1252,19 +1301,60 @@ public class TestUnifiedHighlighter extends LuceneTestCase {
assertEquals(1, topDocs.totalHits.value);
String[] snippets = highlighterNoFieldMatch.highlight("category", query, topDocs, 10);
assertEquals(1, snippets.length);
assertEquals("<b>This</b> <b>is</b> <b>the</b> category <b>field</b>.", snippets[0]);
if (highlighterNoFieldMatch.getFlags("category").contains(HighlightFlag.WEIGHT_MATCHES)) {
assertEquals("<b>This is the category field</b>.", snippets[0]);
} else {
assertEquals("<b>This</b> <b>is</b> <b>the</b> category <b>field</b>.", snippets[0]);
}
snippets = highlighterFieldMatch.highlight("category", query, topDocs, 10);
assertEquals(1, snippets.length);
assertEquals("<b>This</b> <b>is</b> <b>the</b> category <b>field</b>.", snippets[0]);
if (highlighterFieldMatch.getFlags("category").contains(HighlightFlag.WEIGHT_MATCHES)) {
assertEquals("<b>This is the category field</b>.", snippets[0]);
} else {
assertEquals("<b>This</b> <b>is</b> <b>the</b> category <b>field</b>.", snippets[0]);
}
highlighterFieldMatch.setFieldMatcher((fq) -> "text".equals(fq));
snippets = highlighterFieldMatch.highlight("category", query, topDocs, 10);
assertEquals(1, snippets.length);
assertEquals("<b>This</b> <b>is</b> the category field.", snippets[0]);
if (highlighterFieldMatch.getFlags("category").contains(HighlightFlag.WEIGHT_MATCHES)) {
assertEquals("<b>This is</b> the category field.", snippets[0]);
} else {
assertEquals("<b>This</b> <b>is</b> the category field.", snippets[0]);
}
highlighterFieldMatch.setFieldMatcher(null);
}
ir.close();
}
// LUCENE-7909
public void testNestedBooleanQueryAccuracy() throws IOException {
IndexReader ir = indexSomeFields();
IndexSearcher searcher = newSearcher(ir);
UnifiedHighlighter highlighter = randomUnifiedHighlighter(searcher, indexAnalyzer,
EnumSet.of(HighlightFlag.WEIGHT_MATCHES), true);
// This query contains an inner Boolean of two MUST clauses (== "AND"). Only one of them is
// actually in the data, the other is not. We should highlight neither. We can highlight the outer
// SHOULD clauses though.
Query query = new BooleanQuery.Builder()
.add(new PhraseQuery("title", "title", "field"), BooleanClause.Occur.SHOULD)
.add(new BooleanQuery.Builder()
.add(new TermQuery(new Term("category", "category")), BooleanClause.Occur.MUST)
.add(new TermQuery(new Term("category", "nonexistent")), BooleanClause.Occur.MUST)
.build(), BooleanClause.Occur.SHOULD)
.build();
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
String[] snippets = highlighter.highlight("title", query, topDocs);
assertArrayEquals(new String[]{"This is the <b>title field</b>."}, snippets);
// no highlights, not of "category" since "nonexistent" wasn't there
snippets = highlighter.highlight("category", query, topDocs);
assertArrayEquals(new String[]{"This is the category field."}, snippets);
ir.close();
}
}

View File

@ -154,7 +154,7 @@ public class TestUnifiedHighlighterMTQ extends LuceneTestCase {
private UnifiedHighlighter randomUnifiedHighlighter(IndexSearcher searcher, Analyzer indexAnalyzer) {
return TestUnifiedHighlighter.randomUnifiedHighlighter(searcher, indexAnalyzer,
EnumSet.of(HighlightFlag.MULTI_TERM_QUERY));
EnumSet.of(HighlightFlag.MULTI_TERM_QUERY), null);
}
public void testOnePrefix() throws Exception {

View File

@ -17,8 +17,10 @@
package org.apache.lucene.search.uhighlight;
import java.io.IOException;
import java.util.EnumSet;
import java.util.HashSet;
import java.util.Random;
import java.util.Set;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.MockAnalyzer;
@ -274,6 +276,15 @@ public class TestUnifiedHighlighterRanking extends LuceneTestCase {
IndexSearcher searcher = newSearcher(ir);
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer) {
@Override
protected Set<HighlightFlag> getFlags(String field) {
if (random().nextBoolean()) {
return EnumSet.of(HighlightFlag.MULTI_TERM_QUERY, HighlightFlag.PHRASES, HighlightFlag.WEIGHT_MATCHES);
} else {
return super.getFlags(field);
}
}
@Override
protected PassageScorer getScorer(String field) {
return new PassageScorer(1.2f, 0, 87);
@ -314,6 +325,15 @@ public class TestUnifiedHighlighterRanking extends LuceneTestCase {
IndexSearcher searcher = newSearcher(ir);
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer) {
@Override
protected Set<HighlightFlag> getFlags(String field) {
if (random().nextBoolean()) {
return EnumSet.of(HighlightFlag.MULTI_TERM_QUERY, HighlightFlag.PHRASES, HighlightFlag.WEIGHT_MATCHES);
} else {
return super.getFlags(field);
}
}
@Override
protected PassageScorer getScorer(String field) {
return new PassageScorer(0, 0.75f, 87);

View File

@ -17,9 +17,12 @@
package org.apache.lucene.search.uhighlight;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.EnumSet;
import java.util.Locale;
import java.util.Set;
import java.util.concurrent.atomic.AtomicBoolean;
import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
import org.apache.lucene.analysis.MockAnalyzer;
@ -52,6 +55,7 @@ import org.apache.lucene.search.spans.SpanNearQuery;
import org.apache.lucene.search.spans.SpanOrQuery;
import org.apache.lucene.search.spans.SpanQuery;
import org.apache.lucene.search.spans.SpanTermQuery;
import org.apache.lucene.search.uhighlight.UnifiedHighlighter.HighlightFlag;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.LuceneTestCase;
@ -71,6 +75,9 @@ public class TestUnifiedHighlighterStrictPhrases extends LuceneTestCase {
UnifiedHighlighter highlighter;
IndexReader indexReader;
// Is it okay if a match (identified by offset pair) appears multiple times in the passage?
AtomicBoolean dupMatchAllowed = new AtomicBoolean(true);
@ParametersFactory
public static Iterable<Object[]> parameters() {
return UHTestHelper.parametersFactoryList();
@ -104,8 +111,32 @@ public class TestUnifiedHighlighterStrictPhrases extends LuceneTestCase {
private void initReaderSearcherHighlighter() throws IOException {
indexReader = indexWriter.getReader();
searcher = newSearcher(indexReader);
highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
highlighter.setHighlightPhrasesStrictly(true);
highlighter = TestUnifiedHighlighter.randomUnifiedHighlighter(searcher, indexAnalyzer,
EnumSet.of(HighlightFlag.PHRASES, HighlightFlag.MULTI_TERM_QUERY), true);
// intercept the formatter in order to check constraints on the passage.
final PassageFormatter defaultFormatter = highlighter.getFormatter(null);
highlighter.setFormatter(new PassageFormatter() {
@Override
public Object format(Passage[] passages, String content) {
boolean thisDupMatchAllowed = dupMatchAllowed.getAndSet(true);
for (Passage passage : passages) {
String prevPair = "";
for (int i = 0; i < passage.getNumMatches(); i++) {
// pad each to make comparable
String pair = String.format(Locale.ROOT, "%03d-%03d", passage.getMatchStarts()[i], passage.getMatchEnds()[i]);
int cmp = prevPair.compareTo(pair);
if (cmp == 0) {
assertTrue("dup match in passage at offset " + pair, thisDupMatchAllowed);
} else if (cmp > 0) {
fail("bad match order in passage at offset " + pair);
}
prevPair = pair;
}
}
return defaultFormatter.format(passages, content);
}
});
}
private PhraseQuery newPhraseQuery(String field, String phrase) {
@ -137,8 +168,11 @@ public class TestUnifiedHighlighterStrictPhrases extends LuceneTestCase {
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);
if (highlighter.getFlags("body").contains(HighlightFlag.WEIGHT_MATCHES)) {
assertArrayEquals(new String[]{"yin alone, <b>Yin yang</b>, yin gap yang"}, snippets);
} else {
assertArrayEquals(new String[]{"yin alone, <b>Yin</b> <b>yang</b>, yin gap yang"}, snippets);
}
}
public void testWithSameTermQuery() throws IOException {
@ -153,19 +187,13 @@ public class TestUnifiedHighlighterStrictPhrases extends LuceneTestCase {
.build();
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
dupMatchAllowed.set(false); // We don't want duplicates from "Yin" being in TermQuery & PhraseQuery.
String[] snippets = highlighter.highlight("body", query, topDocs);
assertArrayEquals(new String[]{"<b>Yin</b> <b>yang</b>, <b>yin</b> gap yang"}, snippets);
// test the Passage only has 3 matches. We don't want duplicates from "Yin" being in TermQuery & PhraseQuery.
highlighter.setFormatter(new PassageFormatter() {
@Override
public Object format(Passage[] passages, String content) {
return Arrays.toString(passages);
}
});
assertArrayEquals(new String[]{"[Passage[0-22]{yin[0-3],yang[4-8],yin[10-13]}score=2.4964213]"},
highlighter.highlight("body", query, topDocs));
if (highlighter.getFlags("body").contains(HighlightFlag.WEIGHT_MATCHES)) {
assertArrayEquals(new String[]{"<b>Yin yang</b>, <b>yin</b> gap yang"}, snippets);
} else {
assertArrayEquals(new String[]{"<b>Yin</b> <b>yang</b>, <b>yin</b> gap yang"}, snippets);
}
}
public void testPhraseNotInDoc() throws IOException {
@ -197,19 +225,14 @@ public class TestUnifiedHighlighterStrictPhrases extends LuceneTestCase {
.build();
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
dupMatchAllowed.set(false); // We don't want duplicates from both PhraseQuery
String[] snippets = highlighter.highlight("body", query, topDocs);
assertArrayEquals(new String[]{"<b>alpha</b> <b>bravo</b> <b>charlie</b> - charlie bravo alpha"}, snippets);
// test the Passage only has 3 matches. We don't want duplicates from both PhraseQuery
highlighter.setFormatter(new PassageFormatter() {
@Override
public Object format(Passage[] passages, String content) {
return Arrays.toString(passages);
}
});
assertArrayEquals(new String[]{"[Passage[0-41]{alpha[0-5],bravo[6-11],charlie[12-19]}score=3.931102]"},
highlighter.highlight("body", query, topDocs));
if (highlighter.getFlags("body").contains(HighlightFlag.WEIGHT_MATCHES)) {
assertArrayEquals(new String[]{"<b>alpha bravo charlie</b> - charlie bravo alpha"}, snippets);
} else {
assertArrayEquals(new String[]{"<b>alpha</b> <b>bravo</b> <b>charlie</b> - charlie bravo alpha"}, snippets);
}
}
public void testSynonyms() throws IOException {
@ -223,8 +246,11 @@ public class TestUnifiedHighlighterStrictPhrases extends LuceneTestCase {
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);
if (highlighter.getFlags("body").contains(HighlightFlag.WEIGHT_MATCHES)) {
assertArrayEquals(new String[]{"<b>mother father</b> w <b>mom father</b> w dad"}, snippets);
} else {
assertArrayEquals(new String[]{"<b>mother</b> <b>father</b> w <b>mom</b> <b>father</b> w dad"}, snippets);
}
}
/**
@ -252,10 +278,14 @@ public class TestUnifiedHighlighterStrictPhrases extends LuceneTestCase {
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);
if (highlighter.getFlags("body").contains(HighlightFlag.WEIGHT_MATCHES)) {
assertArrayEquals(new String[]{"<b>alpha bravo</b> <b>charlie</b> - charlie bravo <b>alpha</b>"}, snippets);
} else {
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 = new UnifiedHighlighter(searcher, indexAnalyzer);
highlighter.setHandleMultiTermQuery(false);//disable but leave phrase processing enabled
topDocs = searcher.search(query, 10, Sort.INDEXORDER);
@ -288,10 +318,14 @@ public class TestUnifiedHighlighterStrictPhrases extends LuceneTestCase {
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);
if (highlighter.getFlags("body").contains(HighlightFlag.WEIGHT_MATCHES)) {
assertArrayEquals(new String[]{"<b>alpha bravo</b> <b>charlie</b> - charlie bravo alpha"}, snippets);
} else {
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 = new UnifiedHighlighter(searcher, indexAnalyzer);
highlighter.setHandleMultiTermQuery(false);//disable but leave phrase processing enabled
topDocs = searcher.search(query, 10, Sort.INDEXORDER);
@ -325,15 +359,20 @@ public class TestUnifiedHighlighterStrictPhrases extends LuceneTestCase {
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);
if (highlighter.getFlags("body").contains(HighlightFlag.WEIGHT_MATCHES)) {
assertArrayEquals(new String[]{"<b>alpha bravo</b> <b>charlie</b> - charlie bravo <b>alpha</b>"}, snippets);
} else {
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 = new UnifiedHighlighter(searcher, indexAnalyzer);
highlighter.setHandleMultiTermQuery(false);//disable but leave phrase processing enabled
topDocs = searcher.search(query, 10, Sort.INDEXORDER);
snippets = highlighter.highlight("body", query, topDocs);
//note: without MTQ, the WEIGHT_MATCHES is disabled which affects the snippet boundaries
assertArrayEquals(new String[]{"<b>alpha</b> <b>bravo</b> <b>charlie</b> - charlie bravo alpha"},
snippets);
}
@ -351,9 +390,11 @@ public class TestUnifiedHighlighterStrictPhrases extends LuceneTestCase {
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);
if (highlighter.getFlags("body").contains(HighlightFlag.WEIGHT_MATCHES)) {
assertArrayEquals(new String[]{"<b>one bravo</b> three... <b>four bravo</b> six"}, snippets);
} else {
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;
@ -381,7 +422,11 @@ public class TestUnifiedHighlighterStrictPhrases extends LuceneTestCase {
topDocs = searcher.search(query, 10);
assertEquals(1, topDocs.totalHits.value);
snippets = highlighter.highlight("body", query, topDocs, 2);
assertEquals("one <b>bravo</b> <b>three</b>... <b>four</b> <b>bravo</b> six", snippets[0]);
if (highlighter.getFlags("body").contains(HighlightFlag.WEIGHT_MATCHES)) {
assertEquals("one <b>bravo</b> <b>three</b>... four <b>bravo</b> six", snippets[0]);
} else {
assertEquals("one <b>bravo</b> <b>three</b>... <b>four</b> <b>bravo</b> six", snippets[0]);
}
}
public void testMaxLen() throws IOException {
@ -390,16 +435,21 @@ public class TestUnifiedHighlighterStrictPhrases extends LuceneTestCase {
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", "alpha bravo"), BooleanClause.Occur.SHOULD)
.add(newPhraseQuery("body", "gap alpha"), BooleanClause.Occur.SHOULD)
.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);
final boolean weightMatches = highlighter.getFlags("body").contains(HighlightFlag.WEIGHT_MATCHES);
if (fieldType == UHTestHelper.reanalysisType || weightMatches) {
if (weightMatches) {
assertArrayEquals(new String[]{"<b>alpha bravo</b> charlie -"}, snippets);
} else {
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);
}
@ -420,6 +470,7 @@ public class TestUnifiedHighlighterStrictPhrases extends LuceneTestCase {
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
String[] snippets = highlighter.highlight("body", query, topDocs);
// spans' MatchesIterator exposes each underlying term; thus doesn't enclose intermediate "of"
assertArrayEquals(new String[]{"freezing cold stuff like stuff <b>freedom</b> of <b>speech</b>"}, snippets);
}
@ -436,6 +487,13 @@ public class TestUnifiedHighlighterStrictPhrases extends LuceneTestCase {
initReaderSearcherHighlighter();
highlighter = new UnifiedHighlighter(searcher, indexAnalyzer) {
@Override
protected Set<HighlightFlag> getFlags(String field) {
final Set<HighlightFlag> flags = super.getFlags(field);
flags.remove(HighlightFlag.WEIGHT_MATCHES);//unsupported
return flags;
}
@Override
protected Collection<Query> preSpanQueryRewrite(Query query) {
if (query instanceof MyQuery) {

View File

@ -16,13 +16,31 @@
*/
package org.apache.lucene.search.uhighlight;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.BitSet;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Set;
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.index.CheckIndex;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.FilterDirectoryReader;
import org.apache.lucene.index.FilterLeafReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.ParallelLeafReader;
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;
@ -35,13 +53,6 @@ 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>
@ -182,7 +193,12 @@ public class TestUnifiedHighlighterTermVec extends LuceneTestCase {
iw.close();
IndexSearcher searcher = newSearcher(ir);
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer) {
@Override
protected Set<HighlightFlag> getFlags(String field) {
return Collections.emptySet();//no WEIGHT_MATCHES
}
};
TermQuery query = new TermQuery(new Term("body", "vectors"));
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
try {

View File

@ -19,17 +19,19 @@ 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.Set;
import java.util.function.Predicate;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.spans.SpanQuery;
import org.apache.lucene.search.uhighlight.FieldHighlighter;
@ -40,6 +42,7 @@ 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.SplittingBreakIterator;
import org.apache.lucene.search.uhighlight.UHComponents;
import org.apache.lucene.search.uhighlight.UnifiedHighlighter;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
@ -58,17 +61,18 @@ public class TestUnifiedHighlighterExtensibility extends LuceneTestCase {
@Test
public void testFieldOffsetStrategyExtensibility() {
final UnifiedHighlighter.OffsetSource offsetSource = UnifiedHighlighter.OffsetSource.NONE_NEEDED;
FieldOffsetStrategy strategy = new FieldOffsetStrategy("field",
new BytesRef[0],
FieldOffsetStrategy strategy = new FieldOffsetStrategy(new UHComponents("field",
(s) -> false,
new MatchAllDocsQuery(), new BytesRef[0],
PhraseHelper.NONE,
new CharacterRunAutomaton[0]) {
new CharacterRunAutomaton[0], Collections.emptySet())) {
@Override
public UnifiedHighlighter.OffsetSource getOffsetSource() {
return offsetSource;
}
@Override
public OffsetsEnum getOffsetsEnum(IndexReader reader, int docId, String content) throws IOException {
public OffsetsEnum getOffsetsEnum(LeafReader reader, int docId, String content) throws IOException {
return OffsetsEnum.EMPTY;
}
@ -148,13 +152,25 @@ public class TestUnifiedHighlighterExtensibility extends LuceneTestCase {
@Override
protected FieldHighlighter getFieldHighlighter(String field, Query query, Set<Term> allTerms, int maxPassages) {
// THIS IS A COPY of the superclass impl; but use CustomFieldHighlighter
BytesRef[] terms = filterExtractedTerms(getFieldMatcher(field), allTerms);
Predicate<String> fieldMatcher = getFieldMatcher(field);
BytesRef[] terms = filterExtractedTerms(fieldMatcher, allTerms);
Set<HighlightFlag> highlightFlags = getFlags(field);
PhraseHelper phraseHelper = getPhraseHelper(field, query, highlightFlags);
CharacterRunAutomaton[] automata = getAutomata(field, query, highlightFlags);
OffsetSource offsetSource = getOptimizedOffsetSource(field, terms, phraseHelper, automata);
UHComponents components = new UHComponents(field, fieldMatcher, query, terms, phraseHelper, automata, highlightFlags);
// test all is accessible
components.getAutomata();
components.getPhraseHelper();
components.getTerms();
components.getField();
components.getHighlightFlags();
components.getQuery();
components.getFieldMatcher();
return new CustomFieldHighlighter(field,
getOffsetStrategy(offsetSource, field, terms, phraseHelper, automata, highlightFlags),
getOffsetStrategy(offsetSource, components),
new SplittingBreakIterator(getBreakIterator(field), UnifiedHighlighter.MULTIVAL_SEP_CHAR),
getScorer(field),
maxPassages,
@ -163,8 +179,8 @@ public class TestUnifiedHighlighterExtensibility extends LuceneTestCase {
}
@Override
protected FieldOffsetStrategy getOffsetStrategy(OffsetSource offsetSource, String field, BytesRef[] terms, PhraseHelper phraseHelper, CharacterRunAutomaton[] automata, Set<HighlightFlag> highlightFlags) {
return super.getOffsetStrategy(offsetSource, field, terms, phraseHelper, automata, highlightFlags);
protected FieldOffsetStrategy getOffsetStrategy(OffsetSource offsetSource, UHComponents components) {
return super.getOffsetStrategy(offsetSource, components);
}
@Override
@ -207,7 +223,7 @@ public class TestUnifiedHighlighterExtensibility extends LuceneTestCase {
}
@Override
public Object highlightFieldForDoc(IndexReader reader, int docId, String content) throws IOException {
public Object highlightFieldForDoc(LeafReader reader, int docId, String content) throws IOException {
return super.highlightFieldForDoc(reader, docId, content);
}