From 352ec01a6ef68bc81fdb84a7f72e81a6698f594c Mon Sep 17 00:00:00 2001 From: David Smiley Date: Wed, 10 Jan 2018 22:49:40 -0500 Subject: [PATCH] LUCENE-8121: UH switch to SpanCollector API. Better accuracy. * Use the filtered freq in position sensitive terms (better scores) * Refactored UH's OffsetsEnum * Improved test randomization in TestUnifiedHighlighter & MTQ --- lucene/CHANGES.txt | 10 + .../search/uhighlight/FieldHighlighter.java | 14 +- .../uhighlight/FieldOffsetStrategy.java | 78 +-- .../lucene/search/uhighlight/OffsetsEnum.java | 132 ++-- .../lucene/search/uhighlight/Passage.java | 18 + .../search/uhighlight/PhraseHelper.java | 598 +++++------------- .../uhighlight/TokenStreamOffsetStrategy.java | 56 +- .../uhighlight/TestUnifiedHighlighter.java | 87 ++- .../uhighlight/TestUnifiedHighlighterMTQ.java | 55 +- .../TestUnifiedHighlighterStrictPhrases.java | 88 +++ .../TestUnifiedHighlighterExtensibility.java | 4 +- 11 files changed, 524 insertions(+), 616 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index c309a92030e..570da98a654 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -119,6 +119,10 @@ Improvements * LUCENE-8094: TermInSetQuery.toString now returns "field:(A B C)" (Mike McCandless) +* LUCENE-8121: UnifiedHighlighter passage relevancy is improved for terms that are + position sensitive (e.g. part of a phrase) by having an accurate freq. + (David Smiley) + Bug Fixes * LUCENE-8077: Fixed bug in how CheckIndex verifies doc-value iterators. @@ -127,6 +131,12 @@ Bug Fixes * SOLR-11758: Fixed FloatDocValues.boolVal to correctly return true for all values != 0.0F (Munendra S N via hossman) +* LUCENE-8121: The UnifiedHighlighter would highlight some terms within some nested + SpanNearQueries at positions where it should not have. It's fixed in the UH by + switching to the SpanCollector API. The original Highlighter still has this + problem (LUCENE-2287, LUCENE-5455, LUCENE-6796). Some public but internal parts of + the UH were refactored. (David Smiley, Steve Davids) + Other * LUCENE-8111: IndexOrDocValuesQuery Javadoc references outdated method name. diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/FieldHighlighter.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/FieldHighlighter.java index cc9f3186304..a0e6d0a9662 100644 --- a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/FieldHighlighter.java +++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/FieldHighlighter.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.text.BreakIterator; import java.util.ArrayList; import java.util.Arrays; +import java.util.Comparator; import java.util.List; import java.util.PriorityQueue; @@ -136,13 +137,15 @@ public class FieldHighlighter { BreakIterator breakIterator = this.breakIterator; final int contentLength = breakIterator.getText().getEndIndex(); + //TODO consider moving this part to an aggregate OffsetsEnum subclass so we have one enum that already has its weight PriorityQueue offsetsEnumQueue = new PriorityQueue<>(offsetsEnums.size() + 1); for (OffsetsEnum off : offsetsEnums) { off.setWeight(scorer.weight(contentLength, off.freq())); - off.nextPosition(); // go to first position - offsetsEnumQueue.add(off); + if (off.nextPosition()) {// go to first position + offsetsEnumQueue.add(off); + } } - offsetsEnumQueue.add(new OffsetsEnum(null, EMPTY)); // a sentinel for termination + offsetsEnumQueue.add(new OffsetsEnum.OfPostings(new BytesRef(), EMPTY)); // a sentinel for termination PriorityQueue passageQueue = new PriorityQueue<>(Math.min(64, maxPassages + 1), (left, right) -> { if (left.getScore() < right.getScore()) { @@ -203,10 +206,9 @@ public class FieldHighlighter { 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()) { + if (!off.nextPosition()) { break; // No more in the entire text. Already removed from pq; move on } - off.nextPosition(); start = off.startOffset(); end = off.endOffset(); if (start >= passage.getEndOffset() || end > contentLength) { // it's beyond this passage @@ -222,7 +224,7 @@ public class FieldHighlighter { p.sort(); } // sort in ascending order - Arrays.sort(passages, (left, right) -> left.getStartOffset() - right.getStartOffset()); + Arrays.sort(passages, Comparator.comparingInt(Passage::getStartOffset)); return passages; } diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/FieldOffsetStrategy.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/FieldOffsetStrategy.java index 155f0a76fb9..faef1062208 100644 --- a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/FieldOffsetStrategy.java +++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/FieldOffsetStrategy.java @@ -20,14 +20,12 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.Map; 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.CharsRefBuilder; import org.apache.lucene.util.automaton.CharacterRunAutomaton; @@ -41,9 +39,9 @@ import org.apache.lucene.util.automaton.CharacterRunAutomaton; public abstract class FieldOffsetStrategy { protected final String field; - protected final PhraseHelper phraseHelper; // Query: position-sensitive information TODO: rename - protected final BytesRef[] terms; // Query: free-standing terms - protected final CharacterRunAutomaton[] automata; // Query: free-standing wildcards (multi-term query) + 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 public FieldOffsetStrategy(String field, BytesRef[] queryTerms, PhraseHelper phraseHelper, CharacterRunAutomaton[] automata) { this.field = field; @@ -70,47 +68,50 @@ public abstract class FieldOffsetStrategy { return Collections.emptyList(); } - // For strict positions, get a Map of term to Spans: - // note: ScriptPhraseHelper.NONE does the right thing for these method calls - final Map strictPhrasesTermToSpans = - phraseHelper.getTermToSpans(leafReader, doc); - // Usually simply wraps terms in a List; but if willRewrite() then can be expanded - final List sourceTerms = - phraseHelper.expandTermsIfRewrite(terms, strictPhrasesTermToSpans); + final List offsetsEnums = new ArrayList<>(terms.length + automata.length); - final List offsetsEnums = new ArrayList<>(sourceTerms.size() + automata.length); + // 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 sourceTerms: - if (!sourceTerms.isEmpty()) { - TermsEnum termsEnum = termsIndex.iterator();//does not return null - for (BytesRef term : sourceTerms) { - if (termsEnum.seekExact(term)) { - 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 - postingsEnum = phraseHelper.filterPostings(term, postingsEnum, strictPhrasesTermToSpans.get(term)); - if (postingsEnum != null) { - offsetsEnums.add(new OffsetsEnum(term, postingsEnum)); - } - } - } - } + // Handle spans + if (phraseHelper.hasPositionSensitivity()) { + phraseHelper.createOffsetsEnumsForSpans(leafReader, doc, offsetsEnums); } // Handle automata if (automata.length > 0) { - offsetsEnums.addAll(createAutomataOffsetsFromTerms(termsIndex, doc)); + createOffsetsEnumsForAutomata(termsIndex, doc, offsetsEnums); } return offsetsEnums; } - protected List createAutomataOffsetsFromTerms(Terms termsIndex, int doc) throws IOException { + protected void createOffsetsEnumsForTerms(BytesRef[] sourceTerms, Terms termsIndex, int doc, List results) throws IOException { + TermsEnum termsEnum = termsIndex.iterator();//does not return null + for (BytesRef term : sourceTerms) { + if (termsEnum.seekExact(term)) { + 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 + results.add(new OffsetsEnum.OfPostings(term, postingsEnum)); + } + } + } + } + + protected void createOffsetsEnumsForAutomata(Terms termsIndex, int doc, List results) throws IOException { List> automataPostings = new ArrayList<>(automata.length); for (int i = 0; i < automata.length; i++) { automataPostings.add(new ArrayList<>()); @@ -118,6 +119,7 @@ public abstract class FieldOffsetStrategy { TermsEnum termsEnum = termsIndex.iterator(); BytesRef term; + CharsRefBuilder refBuilder = new CharsRefBuilder(); while ((term = termsEnum.next()) != null) { for (int i = 0; i < automata.length; i++) { @@ -132,7 +134,6 @@ public abstract class FieldOffsetStrategy { } } - List offsetsEnums = new ArrayList<>(automata.length); //will be at most this long for (int i = 0; i < automata.length; i++) { CharacterRunAutomaton automaton = automata[i]; List postingsEnums = automataPostings.get(i); @@ -140,14 +141,13 @@ public abstract class FieldOffsetStrategy { if (size > 0) { //only add if we have offsets BytesRef wildcardTerm = new BytesRef(automaton.toString()); if (size == 1) { //don't wrap in a composite if there's only one OffsetsEnum - offsetsEnums.add(new OffsetsEnum(wildcardTerm, postingsEnums.get(0))); + results.add(new OffsetsEnum.OfPostings(wildcardTerm, postingsEnums.get(0))); } else { - offsetsEnums.add(new OffsetsEnum(wildcardTerm, new CompositeOffsetsPostingsEnum(postingsEnums))); + results.add(new OffsetsEnum.OfPostings(wildcardTerm, new CompositeOffsetsPostingsEnum(postingsEnums))); } } } - return offsetsEnums; } } diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/OffsetsEnum.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/OffsetsEnum.java index 708f5c33520..f0a46a5d838 100644 --- a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/OffsetsEnum.java +++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/OffsetsEnum.java @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.lucene.search.uhighlight; import java.io.Closeable; @@ -25,25 +26,19 @@ import org.apache.lucene.index.PostingsEnum; import org.apache.lucene.util.BytesRef; /** - * Holds the term ({@link BytesRef}), {@link PostingsEnum}, offset iteration tracking. - * It is advanced with the underlying postings and is placed in a priority queue by + * An enumeration/iterator of a term and its offsets for use by {@link FieldHighlighter}. + * It is advanced and is placed in a priority queue by * {@link FieldHighlighter#highlightOffsetsEnums(List)} based on the start offset. * * @lucene.internal */ -public class OffsetsEnum implements Comparable, Closeable { - private final BytesRef term; - private final PostingsEnum postingsEnum; // with offsets +public abstract class OffsetsEnum implements Comparable, Closeable { private float weight; // set once in highlightOffsetsEnums - private int posCounter = 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 + // note: would be neat to use some Comparator utilities with method + // references but our methods throw IOException @Override public int compareTo(OffsetsEnum other) { try { @@ -51,53 +46,41 @@ public class OffsetsEnum implements Comparable, Closeable { 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) { + final BytesRef thisTerm = this.getTerm(); + final BytesRef otherTerm = other.getTerm(); + if (thisTerm == null || otherTerm == null) { + if (thisTerm == null && otherTerm == null) { return 0; - } else if (this.term == null) { + } else if (thisTerm == null) { return 1; // put "this" (wildcard mtq enum) last } else { return -1; } } - return term.compareTo(other.term); + return thisTerm.compareTo(otherTerm); } catch (IOException e) { throw new RuntimeException(e); } } - /** The term at this position; usually always the same. This term is a reference that is safe to continue to refer to, - * even after we move to next position. */ - public BytesRef getTerm() throws IOException { - // TODO TokenStreamOffsetStrategy could override OffsetsEnum; then remove this hack here - return term != null ? term : postingsEnum.getPayload(); // abusing payload like this is a total hack! - } + /** + * Advances to the next position and returns true, or if can't then returns false. + * Note that the initial state of this class is not positioned. + */ + public abstract boolean nextPosition() throws IOException; - public PostingsEnum getPostingsEnum() { - return postingsEnum; - } + /** An estimate of the number of occurrences of this term/OffsetsEnum. */ + public abstract int freq() throws IOException; - public int freq() throws IOException { - return postingsEnum.freq(); - } + /** + * The term at this position; usually always the same. + * This BytesRef is safe to continue to refer to, even after we move to the next position. + */ + public abstract BytesRef getTerm() throws IOException; - public boolean hasMorePositions() throws IOException { - return posCounter < postingsEnum.freq(); - } + public abstract int startOffset() throws IOException; - public void nextPosition() throws IOException { - assert hasMorePositions(); - posCounter++; - postingsEnum.nextPosition(); - } - - public int startOffset() throws IOException { - return postingsEnum.startOffset(); - } - - public int endOffset() throws IOException { - return postingsEnum.endOffset(); - } + public abstract int endOffset() throws IOException; public float getWeight() { return weight; @@ -109,9 +92,66 @@ public class OffsetsEnum implements Comparable, Closeable { @Override public void close() throws IOException { - // TODO TokenStreamOffsetStrategy could override OffsetsEnum; then this base impl would be no-op. - if (postingsEnum instanceof Closeable) { - ((Closeable) postingsEnum).close(); + } + + @Override + public String toString() { + final String name = getClass().getSimpleName(); + try { + return name + "(term:" + getTerm().utf8ToString() +")"; + } catch (Exception e) { + return name; } } + + /** + * Based on a {@link PostingsEnum} -- the typical/standard OE impl. + */ + public static class OfPostings extends OffsetsEnum { + private final BytesRef term; + private final PostingsEnum postingsEnum; // with offsets + + private int posCounter = 0; // the occurrence counter of this term within the text being highlighted. + + public OfPostings(BytesRef term, PostingsEnum postingsEnum) throws IOException { + this.term = Objects.requireNonNull(term); + this.postingsEnum = Objects.requireNonNull(postingsEnum); + } + + public PostingsEnum getPostingsEnum() { + return postingsEnum; + } + + @Override + public boolean nextPosition() throws IOException { + if (posCounter < postingsEnum.freq()) { + posCounter++; + postingsEnum.nextPosition(); // note: we don't need to save the position + return true; + } else { + return false; + } + } + + @Override + public int freq() throws IOException { + return postingsEnum.freq(); + } + + @Override + public BytesRef getTerm() throws IOException { + return term; + } + + @Override + public int startOffset() throws IOException { + return postingsEnum.startOffset(); + } + + @Override + public int endOffset() throws IOException { + return postingsEnum.endOffset(); + } + + } } diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/Passage.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/Passage.java index 3efb694f9e7..24b1015d104 100644 --- a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/Passage.java +++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/Passage.java @@ -98,6 +98,24 @@ public class Passage { numMatches = 0; } + /** For debugging. ex: Passage[0-22]{yin[0-3],yang[4-8],yin[10-13]}score=2.4964213 */ + @Override + public String toString() { + StringBuilder buf = new StringBuilder(); + buf.append("Passage[").append(startOffset).append('-').append(endOffset).append(']'); + buf.append('{'); + for (int i = 0; i < numMatches; i++) { + if (i != 0) { + buf.append(','); + } + buf.append(matchTerms[i].utf8ToString()); + buf.append('[').append(matchStarts[i] - startOffset).append('-').append(matchEnds[i] - startOffset).append(']'); + } + buf.append('}'); + buf.append("score=").append(score); + return buf.toString(); + } + /** * Start offset of this passage. * diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/PhraseHelper.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/PhraseHelper.java index cfb65708df8..f5205e412e6 100644 --- a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/PhraseHelper.java +++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/PhraseHelper.java @@ -17,82 +17,58 @@ package org.apache.lucene.search.uhighlight; import java.io.IOException; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; -import java.util.Collections; -import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; -import java.util.LinkedHashSet; import java.util.List; import java.util.Map; -import java.util.PriorityQueue; import java.util.Set; import java.util.TreeSet; import java.util.function.Function; import java.util.function.Predicate; -import org.apache.lucene.index.BinaryDocValues; import org.apache.lucene.index.FieldInfos; import org.apache.lucene.index.FilterLeafReader; import org.apache.lucene.index.LeafReader; -import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.NumericDocValues; import org.apache.lucene.index.PostingsEnum; -import org.apache.lucene.index.SortedDocValues; import org.apache.lucene.index.Term; import org.apache.lucene.index.Terms; -import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.index.TermsEnum; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.MultiTermQuery; import org.apache.lucene.search.Query; import org.apache.lucene.search.ScoreMode; +import org.apache.lucene.search.Scorer; import org.apache.lucene.search.TwoPhaseIterator; +import org.apache.lucene.search.Weight; import org.apache.lucene.search.highlight.WeightedSpanTerm; import org.apache.lucene.search.highlight.WeightedSpanTermExtractor; import org.apache.lucene.search.spans.SpanCollector; import org.apache.lucene.search.spans.SpanMultiTermQueryWrapper; import org.apache.lucene.search.spans.SpanQuery; -import org.apache.lucene.search.spans.SpanWeight; +import org.apache.lucene.search.spans.SpanScorer; import org.apache.lucene.search.spans.Spans; import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.PriorityQueue; /** - * Helps the {@link FieldOffsetStrategy} with strict position highlighting (e.g. highlight phrases correctly). + * Helps the {@link FieldOffsetStrategy} with position sensitive queries (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. + * documents. Despite this state, it's immutable after construction. * * @lucene.internal */ +// TODO rename to SpanHighlighting ? public class PhraseHelper { public static final PhraseHelper NONE = new PhraseHelper(new MatchAllDocsQuery(), "_ignored_", (s) -> false, spanQuery -> null, query -> null, true); - //TODO it seems this ought to be a general thing on Spans? - private static final Comparator 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; - private final Set positionInsensitiveTerms; // (TermQuery terms) + private final Set positionInsensitiveTerms; // (TermQuery terms) private final Set spanQueries; private final boolean willRewrite; private final Predicate fieldMatcher; @@ -114,13 +90,27 @@ public class PhraseHelper { this.fieldName = field; this.fieldMatcher = fieldMatcher; // filter terms to those we want - positionInsensitiveTerms = new FieldFilteringTermSet(); + positionInsensitiveTerms = new HashSet<>(); 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 + // When we call Weight.extractTerms, we do it on clauses that are NOT position sensitive. + // We only want the to track a Set of bytes for the Term, not Term class with field part. + Set extractPosInsensitiveTermsTarget = new TreeSet() { + @Override + public boolean add(Term term) { + // don't call super.add; we don't actually use the superclass + if (fieldMatcher.test(term.field())) { + return positionInsensitiveTerms.add(term.bytes()); + } else { + return false; + } + } + }; + // 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) { @@ -155,13 +145,15 @@ public class PhraseHelper { return true; //TODO set to false and provide a hook to customize certain queries. } + // called on Query types that are NOT position sensitive, e.g. TermQuery @Override protected void extractWeightedTerms(Map terms, Query query, float boost) throws IOException { query.createWeight(UnifiedHighlighter.EMPTY_INDEXSEARCHER, ScoreMode.COMPLETE_NO_SCORES, boost) - .extractTerms(positionInsensitiveTerms); + .extractTerms(extractPosInsensitiveTermsTarget); } + // called on SpanQueries. Some other position-sensitive queries like PhraseQuery are converted beforehand @Override protected void extractWeightedSpanTerms(Map terms, SpanQuery spanQuery, float boost) throws IOException { @@ -174,7 +166,6 @@ public class PhraseHelper { } } - // TODO allow users to override the answer to mustRewriteQuery boolean mustRewriteQuery = mustRewriteQuery(spanQuery); if (ignoreQueriesNeedingRewrite && mustRewriteQuery) { return;// ignore this query @@ -194,14 +185,14 @@ public class PhraseHelper { willRewrite = mustRewriteHolder[0]; } - Set getSpanQueries() { + public Set getSpanQueries() { return spanQueries; } /** * If there is no position sensitivity then use of the instance of this class can be ignored. */ - boolean hasPositionSensitivity() { + public boolean hasPositionSensitivity() { return spanQueries.isEmpty() == false; } @@ -210,335 +201,85 @@ public class PhraseHelper { * custom things. When true, the resulting term list will probably be different than what it was known * to be initially. */ - boolean willRewrite() { + public 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 getTermToSpans(LeafReader leafReader, int doc) - throws IOException { - if (spanQueries.isEmpty()) { - return Collections.emptyMap(); - } - final LeafReader filteredReader = new SingleFieldFilterLeafReader(leafReader, fieldName); - // for each SpanQuery, collect the member spans into a map. - Map result = new HashMap<>(); - for (SpanQuery spanQuery : spanQueries) { - getTermToSpans(spanQuery, filteredReader.getContext(), doc, result); - } + /** Returns the terms that are position-insensitive (sorted). */ + public BytesRef[] getAllPositionInsensitiveTerms() { + BytesRef[] result = positionInsensitiveTerms.toArray(new BytesRef[positionInsensitiveTerms.size()]); + Arrays.sort(result); return result; } - // code extracted & refactored from WSTE.extractWeightedSpanTerms() - private void getTermToSpans(SpanQuery spanQuery, LeafReaderContext readerContext, - int doc, Map 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.reader()); + /** Given the internal SpanQueries, produce a number of OffsetsEnum into the {@code results} param. */ + public void createOffsetsEnumsForSpans(LeafReader leafReader, int docId, List results) throws IOException { + leafReader = new SingleFieldWithOffsetsFilterLeafReader(leafReader, fieldName); + //TODO avoid searcher and do what it does to rewrite & get weight? + IndexSearcher searcher = new IndexSearcher(leafReader); searcher.setQueryCache(null); - if (willRewrite) { - spanQuery = (SpanQuery) searcher.rewrite(spanQuery); // searcher.rewrite loops till done - } - - // Get the underlying query terms - TreeSet termSet = new FieldFilteringTermSet(); // sorted so we can loop over results in order shortly... - searcher.createWeight(spanQuery, ScoreMode.COMPLETE_NO_SCORES, 1.0f).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, ScoreMode.COMPLETE_NO_SCORES); - 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 (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 phraseHelper' keySet if present. That can only - * happen if willRewrite() is true. - */ - List expandTermsIfRewrite(BytesRef[] terms, Map strictPhrasesTermToSpans) { - if (willRewrite()) { - Set 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 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. - *

- * 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}. - *

- * This will return null if it's completely filtered out (i.e. effectively has no postings). - */ - 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(); + // for each SpanQuery, grab it's Spans and put it into a PriorityQueue + PriorityQueue spansPriorityQueue = new PriorityQueue(spanQueries.size()) { @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; + protected boolean lessThan(Spans a, Spans b) { + return a.startPosition() <= b.startPosition(); } }; - } - - /** - * Simple TreeSet that filters out Terms not matching the provided predicate on {@code add()}. - */ - private class FieldFilteringTermSet extends TreeSet { - @Override - public boolean add(Term term) { - if (fieldMatcher.test(term.field())) { - if (term.field().equals(fieldName)) { - return super.add(term); - } else { - return super.add(new Term(fieldName, term.bytes())); + for (Query query : spanQueries) { + Weight weight = searcher.createNormalizedWeight(query, ScoreMode.COMPLETE_NO_SCORES); + Scorer scorer = weight.scorer(leafReader.getContext()); + if (scorer == null) { + continue; + } + TwoPhaseIterator twoPhaseIterator = scorer.twoPhaseIterator(); + if (twoPhaseIterator != null) { + if (twoPhaseIterator.approximation().advance(docId) != docId || !twoPhaseIterator.matches()) { + continue; } + } else if (scorer.iterator().advance(docId) != docId) { // preposition, and return doing nothing if find none + continue; + } + + Spans spans = ((SpanScorer) scorer).getSpans(); + assert spans.docID() == docId; + if (spans.nextStartPosition() != Spans.NO_MORE_POSITIONS) { + spansPriorityQueue.add(spans); + } + } + + // Iterate the Spans in the PriorityQueue, collecting as we go. By using a PriorityQueue ordered by position, + // the underlying offsets in our collector will be mostly appended to the end of arrays (efficient). + // note: alternatively it'd interesting if we produced one OffsetsEnum that internally advanced + // this PriorityQueue when nextPosition is called; it would cap what we have to cache for large docs and + // exiting early (due to maxLen) is easy. + // But at least we have an accurate "freq" and it shouldn't be too much data to collect. Even SpanScorer + // navigates the spans fully to compute a good freq (and thus score)! + OffsetSpanCollector spanCollector = new OffsetSpanCollector(); + while (spansPriorityQueue.size() > 0) { + Spans spans = spansPriorityQueue.top(); + //TODO limit to a capped endOffset length somehow so we can break this loop early + spans.collect(spanCollector); + + if (spans.nextStartPosition() == Spans.NO_MORE_POSITIONS) { + spansPriorityQueue.pop(); } else { - return false; + spansPriorityQueue.updateTop(); } } + results.addAll(spanCollector.termToOffsetsEnums.values()); } - /** - * 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 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 - } - } - - //TODO move up; it's currently inbetween other inner classes that are related /** * Needed to support the ability to highlight a query irrespective of the field a query refers to * (aka requireFieldMatch=false). * This reader will just delegate every call to a single field in the wrapped * LeafReader. This way we ensure that all queries going through this reader target the same field. */ - static final class SingleFieldFilterLeafReader extends FilterLeafReader { + private static final class SingleFieldWithOffsetsFilterLeafReader extends FilterLeafReader { final String fieldName; - SingleFieldFilterLeafReader(LeafReader in, String fieldName) { + SingleFieldWithOffsetsFilterLeafReader(LeafReader in, String fieldName) { super(in); this.fieldName = fieldName; } @@ -550,22 +291,18 @@ public class PhraseHelper { @Override public Terms terms(String field) throws IOException { - return super.terms(fieldName); - } - - @Override - public NumericDocValues getNumericDocValues(String field) throws IOException { - return super.getNumericDocValues(fieldName); - } - - @Override - public BinaryDocValues getBinaryDocValues(String field) throws IOException { - return super.getBinaryDocValues(fieldName); - } - - @Override - public SortedDocValues getSortedDocValues(String field) throws IOException { - return super.getSortedDocValues(fieldName); + // ensure the underlying PostingsEnum returns offsets. It's sad we have to do this to use the SpanCollector. + return new FilterTerms(super.terms(fieldName)) { + @Override + public TermsEnum iterator() throws IOException { + return new FilterTermsEnum(in.iterator()) { + @Override + public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException { + return super.postings(reuse, flags | PostingsEnum.OFFSETS); + } + }; + } + }; } @Override @@ -584,99 +321,102 @@ public class PhraseHelper { } } + private class OffsetSpanCollector implements SpanCollector { + Map termToOffsetsEnums = new HashMap<>(); - /** - * 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; + @Override + public void collectLeaf(PostingsEnum postings, int position, Term term) throws IOException { + if (!fieldMatcher.test(term.field())) { + return; } - } - final int docId; - final ArrayList 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())); + SpanCollectedOffsetsEnum offsetsEnum = termToOffsetsEnums.get(term.bytes()); + if (offsetsEnum == null) { + // If it's pos insensitive we handle it outside of PhraseHelper. term.field() is from the Query. + if (positionInsensitiveTerms.contains(term.bytes())) { + return; + } + offsetsEnum = new SpanCollectedOffsetsEnum(term.bytes(), postings.freq()); + termToOffsetsEnums.put(term.bytes(), offsetsEnum); } - assert !cachedSpanList.isEmpty(); // bad Span impl? - } - - /** - * Clone; reset iteration state. - */ - CachedSpans(CachedSpans cloneMe) { - docId = cloneMe.docId; - cachedSpanList = cloneMe.cachedSpanList; + offsetsEnum.add(postings.startOffset(), postings.endOffset()); } @Override - public int nextDoc() throws IOException { - throw new UnsupportedOperationException("Not expected"); + public void reset() { // called when at a new position. We don't care. + } + } + + private static class SpanCollectedOffsetsEnum extends OffsetsEnum { + // TODO perhaps optionally collect (and expose) payloads? + private final BytesRef term; + private final int[] startOffsets; + private final int[] endOffsets; + private int numPairs = 0; + private int enumIdx = -1; + + private SpanCollectedOffsetsEnum(BytesRef term, int postingsFreq) { + this.term = term; + this.startOffsets = new int[postingsFreq]; // hopefully not wasteful? At least we needn't resize it. + this.endOffsets = new int[postingsFreq]; + } + + // called from collector before it's navigated + void add(int startOffset, int endOffset) { + assert enumIdx == -1 : "bad state"; + + // loop backwards since we expect a match at the end or close to it. We expect O(1) not O(N). + int pairIdx = numPairs - 1; + for (; pairIdx >= 0; pairIdx--) { + int iStartOffset = startOffsets[pairIdx]; + int iEndOffset = endOffsets[pairIdx]; + int cmp = Integer.compare(iStartOffset, startOffset); + if (cmp == 0) { + cmp = Integer.compare(iEndOffset, endOffset); + } + if (cmp == 0) { + return; // we already have this offset-pair for this term + } else if (cmp < 0) { + break; //we will insert offsetPair to the right of pairIdx + } + } + // pairIdx is now one position to the left of where we insert the new pair + // shift right any pairs by one to make room + final int shiftLen = numPairs - (pairIdx + 1); + if (shiftLen > 0) { + System.arraycopy(startOffsets, pairIdx + 2, startOffsets, pairIdx + 3, shiftLen); + System.arraycopy(endOffsets, pairIdx + 2, endOffsets, pairIdx + 3, shiftLen); + } + // now we can place the offset pair + startOffsets[pairIdx + 1] = startOffset; + endOffsets[pairIdx + 1] = endOffset; + numPairs++; } @Override - public int advance(int target) throws IOException { - throw new UnsupportedOperationException("Not expected"); + public boolean nextPosition() throws IOException { + return ++enumIdx < numPairs; } @Override - public int docID() { - return docId; + public int freq() throws IOException { + return startOffsets.length; } @Override - public long cost() { - return 1; + public BytesRef getTerm() throws IOException { + return term; } @Override - public int nextStartPosition() throws IOException { - index++; - return startPosition(); + public int startOffset() throws IOException { + return startOffsets[enumIdx]; } @Override - public int startPosition() { - return index < 0 ? - -1 : index >= cachedSpanList.size() ? - NO_MORE_POSITIONS : cachedSpanList.get(index).start; + public int endOffset() throws IOException { + return endOffsets[enumIdx]; } + } - @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 } diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/TokenStreamOffsetStrategy.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/TokenStreamOffsetStrategy.java index 28eb6b1a613..5f47a5daac7 100644 --- a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/TokenStreamOffsetStrategy.java +++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/TokenStreamOffsetStrategy.java @@ -16,7 +16,6 @@ */ package org.apache.lucene.search.uhighlight; -import java.io.Closeable; import java.io.IOException; import java.util.Collections; import java.util.List; @@ -26,7 +25,6 @@ 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.PostingsEnum; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.automaton.Automata; import org.apache.lucene.util.automaton.CharacterRunAutomaton; @@ -63,29 +61,20 @@ public class TokenStreamOffsetStrategy extends AnalysisOffsetStrategy { @Override public List getOffsetsEnums(IndexReader reader, int docId, String content) throws IOException { - TokenStream tokenStream = tokenStream(content); - PostingsEnum mtqPostingsEnum = new TokenStreamPostingsEnum(tokenStream, automata); - mtqPostingsEnum.advance(docId); - return Collections.singletonList(new OffsetsEnum(null, mtqPostingsEnum)); + return Collections.singletonList(new TokenStreamOffsetsEnum(tokenStream(content), automata)); } - // See class javadocs. - // TODO: DWS perhaps instead OffsetsEnum could become abstract and this would be an impl? See TODOs in OffsetsEnum. - private static class TokenStreamPostingsEnum extends PostingsEnum implements Closeable { + private static class TokenStreamOffsetsEnum extends OffsetsEnum { 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 { + TokenStreamOffsetsEnum(TokenStream ts, CharacterRunAutomaton[] matchers) throws IOException { this.stream = ts; this.matchers = matchers; matchDescriptions = new BytesRef[matchers.length]; @@ -95,15 +84,13 @@ public class TokenStreamOffsetStrategy extends AnalysisOffsetStrategy { } @Override - public int nextPosition() throws IOException { + public boolean 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; + return true; } } } @@ -111,8 +98,7 @@ public class TokenStreamOffsetStrategy extends AnalysisOffsetStrategy { close(); } // exhausted - currentStartOffset = currentEndOffset = Integer.MAX_VALUE; - return Integer.MAX_VALUE; + return false; } @Override @@ -122,45 +108,23 @@ public class TokenStreamOffsetStrategy extends AnalysisOffsetStrategy { @Override public int startOffset() throws IOException { - assert currentStartOffset >= 0; - return currentStartOffset; + return offsetAtt.startOffset(); } @Override public int endOffset() throws IOException { - assert currentEndOffset >= 0; - return currentEndOffset; + return offsetAtt.endOffset(); } - // TOTAL HACK; used in OffsetsEnum.getTerm() @Override - public BytesRef getPayload() throws IOException { + public BytesRef getTerm() throws IOException { if (matchDescriptions[currentMatch] == null) { + // these CharRunAutomata are subclassed so that toString() returns the query 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) { diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighter.java b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighter.java index 96ec15501ff..086d7a03cfb 100644 --- a/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighter.java +++ b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighter.java @@ -23,11 +23,14 @@ import java.nio.charset.StandardCharsets; import java.text.BreakIterator; import java.util.Arrays; import java.util.Collections; +import java.util.EnumSet; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.function.Predicate; 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.document.Document; @@ -49,6 +52,7 @@ 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.uhighlight.UnifiedHighlighter.HighlightFlag; import org.apache.lucene.store.Directory; import org.apache.lucene.util.LuceneTestCase; import org.junit.After; @@ -81,6 +85,36 @@ public class TestUnifiedHighlighter extends LuceneTestCase { dir.close(); } + static UnifiedHighlighter randomUnifiedHighlighter(IndexSearcher searcher, Analyzer indexAnalyzer) { + return randomUnifiedHighlighter(searcher, indexAnalyzer, EnumSet.noneOf(HighlightFlag.class)); + } + + static UnifiedHighlighter randomUnifiedHighlighter(IndexSearcher searcher, Analyzer indexAnalyzer, + EnumSet mandatoryFlags) { + if (random().nextBoolean()) { + return new UnifiedHighlighter(searcher, indexAnalyzer); + } else { + final UnifiedHighlighter uh = new UnifiedHighlighter(searcher, indexAnalyzer) { + @Override + protected Set getFlags(String field) { + final EnumSet result = EnumSet.copyOf(mandatoryFlags); + int r = random().nextInt(); + for (HighlightFlag highlightFlag : HighlightFlag.values()) { + if (((1 << highlightFlag.ordinal()) & r) == 0) { + result.add(highlightFlag); + } + } + return result; + } + }; + uh.setCacheFieldValCharsThreshold(random().nextInt(100)); + if (random().nextBoolean()) { + uh.setFieldMatcher(f -> true); // requireFieldMatch==false + } + return uh; + } + } + // // Tests below were ported from the PostingsHighlighter. Possibly augmented. Far below are newer tests. // @@ -101,7 +135,7 @@ public class TestUnifiedHighlighter extends LuceneTestCase { iw.close(); IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(searcher, indexAnalyzer); Query query = new TermQuery(new Term("body", "highlighting")); TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER); assertEquals(2, topDocs.totalHits); @@ -167,7 +201,7 @@ public class TestUnifiedHighlighter extends LuceneTestCase { TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER); assertEquals(1, topDocs.totalHits); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(searcher, indexAnalyzer); highlighter.setMaxLength(maxLength); String snippets[] = highlighter.highlight("body", query, topDocs); @@ -191,7 +225,7 @@ public class TestUnifiedHighlighter extends LuceneTestCase { iw.close(); IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(searcher, indexAnalyzer); Query query = new TermQuery(new Term("body", "test")); TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER); assertEquals(1, topDocs.totalHits); @@ -219,7 +253,7 @@ public class TestUnifiedHighlighter extends LuceneTestCase { iw.close(); IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(searcher, indexAnalyzer); Query query = new TermQuery(new Term("body", "test")); TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER); assertEquals(2, topDocs.totalHits); @@ -248,7 +282,7 @@ public class TestUnifiedHighlighter extends LuceneTestCase { iw.close(); IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(searcher, indexAnalyzer); highlighter.setMaxLength(value.length() * 2 + 1); Query query = new TermQuery(new Term("body", "field")); TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER); @@ -281,7 +315,7 @@ public class TestUnifiedHighlighter extends LuceneTestCase { iw.close(); IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(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) @@ -313,7 +347,7 @@ public class TestUnifiedHighlighter extends LuceneTestCase { iw.close(); IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(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) @@ -345,7 +379,7 @@ public class TestUnifiedHighlighter extends LuceneTestCase { iw.close(); IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(searcher, indexAnalyzer); Query query = new TermQuery(new Term("body", "test")); TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER); assertEquals(2, topDocs.totalHits); @@ -382,7 +416,7 @@ public class TestUnifiedHighlighter extends LuceneTestCase { .build(); TopDocs topDocs = searcher.search(query, 10); assertEquals(1, topDocs.totalHits); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(searcher, indexAnalyzer); highlighter.setHighlightPhrasesStrictly(false); String snippets[] = highlighter.highlight("body", query, topDocs, 2); assertEquals(1, snippets.length); @@ -410,7 +444,7 @@ public class TestUnifiedHighlighter extends LuceneTestCase { .build(); TopDocs topDocs = searcher.search(query, 10); assertEquals(1, topDocs.totalHits); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(searcher, indexAnalyzer); highlighter.setHighlightPhrasesStrictly(false); String snippets[] = highlighter.highlight("body", query, topDocs, 2); assertEquals(1, snippets.length); @@ -438,7 +472,7 @@ public class TestUnifiedHighlighter extends LuceneTestCase { .build(); TopDocs topDocs = searcher.search(query, 10); assertEquals(1, topDocs.totalHits); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(searcher, indexAnalyzer); highlighter.setMaxLength(Integer.MAX_VALUE - 1); String snippets[] = highlighter.highlight("body", query, topDocs, 2); assertEquals(1, snippets.length); @@ -461,7 +495,7 @@ public class TestUnifiedHighlighter extends LuceneTestCase { iw.close(); IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(searcher, indexAnalyzer); Query query = new TermQuery(new Term("body", "test")); TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER); assertEquals(1, topDocs.totalHits); @@ -494,7 +528,7 @@ public class TestUnifiedHighlighter extends LuceneTestCase { TopDocs topDocs = searcher.search(query, 10); assertEquals(1, topDocs.totalHits); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(searcher, indexAnalyzer); highlighter.setMaxLength(Integer.MAX_VALUE - 1); String snippets[] = highlighter.highlight("body", query, topDocs, 2); assertEquals(1, snippets.length); @@ -549,7 +583,7 @@ public class TestUnifiedHighlighter extends LuceneTestCase { iw.close(); IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(searcher, indexAnalyzer); Query query = new TermQuery(new Term("body", "highlighting")); TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER); assertEquals(2, topDocs.totalHits); @@ -623,7 +657,7 @@ public class TestUnifiedHighlighter extends LuceneTestCase { iw.close(); IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(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"); @@ -652,7 +686,7 @@ public class TestUnifiedHighlighter extends LuceneTestCase { iw.close(); IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(searcher, indexAnalyzer); int docID = searcher.search(new TermQuery(new Term("id", "id")), 1).scoreDocs[0].doc; Query query = new TermQuery(new Term("body", "highlighting")); @@ -683,7 +717,7 @@ public class TestUnifiedHighlighter extends LuceneTestCase { iw.close(); IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(searcher, indexAnalyzer); highlighter.setMaxNoHighlightPassages(0);// don't want any default summary Query query = new TermQuery(new Term("body", "highlighting")); int[] docIDs = new int[]{0}; @@ -743,7 +777,7 @@ public class TestUnifiedHighlighter extends LuceneTestCase { iw.close(); IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(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"); @@ -769,7 +803,7 @@ public class TestUnifiedHighlighter extends LuceneTestCase { iw.close(); IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(searcher, indexAnalyzer); int docID = searcher.search(new TermQuery(new Term("id", "id")), 1).scoreDocs[0].doc; Query query = new TermQuery(new Term("body", "highlighting")); @@ -798,7 +832,7 @@ public class TestUnifiedHighlighter extends LuceneTestCase { iw.close(); IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(searcher, indexAnalyzer); int docID = searcher.search(new TermQuery(new Term("id", "id")), 1).scoreDocs[0].doc; Query query = new TermQuery(new Term("body", "highlighting")); @@ -834,7 +868,7 @@ public class TestUnifiedHighlighter extends LuceneTestCase { iw.close(); IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(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); @@ -872,7 +906,7 @@ public class TestUnifiedHighlighter extends LuceneTestCase { iw.close(); IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(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) @@ -995,7 +1029,8 @@ public class TestUnifiedHighlighter extends LuceneTestCase { return (qf) -> true; } }; - UnifiedHighlighter highlighterFieldMatch = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighterFieldMatch = randomUnifiedHighlighter(searcher, indexAnalyzer); + highlighterFieldMatch.setFieldMatcher(null);//default BooleanQuery.Builder queryBuilder = new BooleanQuery.Builder() .add(new TermQuery(new Term("text", "some")), BooleanClause.Occur.SHOULD) @@ -1078,7 +1113,8 @@ public class TestUnifiedHighlighter extends LuceneTestCase { return (qf) -> true; } }; - UnifiedHighlighter highlighterFieldMatch = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighterFieldMatch = randomUnifiedHighlighter(searcher, indexAnalyzer, EnumSet.of(HighlightFlag.MULTI_TERM_QUERY)); + highlighterFieldMatch.setFieldMatcher(null);//default BooleanQuery.Builder queryBuilder = new BooleanQuery.Builder() .add(new FuzzyQuery(new Term("text", "sime"), 1), BooleanClause.Occur.SHOULD) @@ -1161,7 +1197,8 @@ public class TestUnifiedHighlighter extends LuceneTestCase { return (qf) -> true; } }; - UnifiedHighlighter highlighterFieldMatch = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighterFieldMatch = randomUnifiedHighlighter(searcher, indexAnalyzer, EnumSet.of(HighlightFlag.PHRASES)); + highlighterFieldMatch.setFieldMatcher(null);//default BooleanQuery.Builder queryBuilder = new BooleanQuery.Builder() .add(new PhraseQuery("title", "this", "is", "the", "title"), BooleanClause.Occur.SHOULD) diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterMTQ.java b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterMTQ.java index 8791b76e7e7..a9fadc0175f 100644 --- a/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterMTQ.java +++ b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterMTQ.java @@ -20,6 +20,7 @@ package org.apache.lucene.search.uhighlight; import java.io.IOException; import java.util.Arrays; import java.util.Collections; +import java.util.EnumSet; import java.util.List; import java.util.Objects; @@ -65,6 +66,7 @@ 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.spans.SpanWeight; +import org.apache.lucene.search.uhighlight.UnifiedHighlighter.HighlightFlag; import org.apache.lucene.store.BaseDirectoryWrapper; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.LuceneTestCase; @@ -150,6 +152,11 @@ public class TestUnifiedHighlighterMTQ extends LuceneTestCase { ir.close(); } + private UnifiedHighlighter randomUnifiedHighlighter(IndexSearcher searcher, Analyzer indexAnalyzer) { + return TestUnifiedHighlighter.randomUnifiedHighlighter(searcher, indexAnalyzer, + EnumSet.of(HighlightFlag.MULTI_TERM_QUERY)); + } + public void testOnePrefix() throws Exception { RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer); @@ -166,7 +173,7 @@ public class TestUnifiedHighlighterMTQ extends LuceneTestCase { iw.close(); IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(searcher, indexAnalyzer); // wrap in a BoostQuery to also show we see inside it Query query = new BoostQuery(new PrefixQuery(new Term("body", "te")), 2.0f); TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER); @@ -177,6 +184,7 @@ public class TestUnifiedHighlighterMTQ extends LuceneTestCase { assertEquals("Test a one sentence document.", snippets[1]); // wrong field + highlighter.setFieldMatcher(null);//default BooleanQuery bq = new BooleanQuery.Builder() .add(new MatchAllDocsQuery(), BooleanClause.Occur.SHOULD) .add(new PrefixQuery(new Term("bogus", "te")), BooleanClause.Occur.SHOULD) @@ -207,7 +215,7 @@ public class TestUnifiedHighlighterMTQ extends LuceneTestCase { iw.close(); IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(searcher, indexAnalyzer); Query query = new RegexpQuery(new Term("body", "te.*")); TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER); assertEquals(2, topDocs.totalHits); @@ -217,6 +225,7 @@ public class TestUnifiedHighlighterMTQ extends LuceneTestCase { assertEquals("Test a one sentence document.", snippets[1]); // wrong field + highlighter.setFieldMatcher(null);//default BooleanQuery bq = new BooleanQuery.Builder() .add(new MatchAllDocsQuery(), BooleanClause.Occur.SHOULD) .add(new RegexpQuery(new Term("bogus", "te.*")), BooleanClause.Occur.SHOULD) @@ -247,7 +256,7 @@ public class TestUnifiedHighlighterMTQ extends LuceneTestCase { iw.close(); IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(searcher, indexAnalyzer); Query query = new FuzzyQuery(new Term("body", "tets"), 1); TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER); assertEquals(2, topDocs.totalHits); @@ -266,6 +275,7 @@ public class TestUnifiedHighlighterMTQ extends LuceneTestCase { assertEquals("Test a one sentence document.", snippets[1]); // wrong field + highlighter.setFieldMatcher(null);//default BooleanQuery bq = new BooleanQuery.Builder() .add(new MatchAllDocsQuery(), BooleanClause.Occur.SHOULD) .add(new FuzzyQuery(new Term("bogus", "tets"), 1), BooleanClause.Occur.SHOULD) @@ -296,7 +306,7 @@ public class TestUnifiedHighlighterMTQ extends LuceneTestCase { iw.close(); IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(searcher, indexAnalyzer); Query query = TermRangeQuery.newStringRange("body", "ta", "tf", true, true); TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER); assertEquals(2, topDocs.totalHits); @@ -366,6 +376,7 @@ public class TestUnifiedHighlighterMTQ extends LuceneTestCase { assertEquals("Test a one sentence document.", snippets[1]); // wrong field + highlighter.setFieldMatcher(null);//default bq = new BooleanQuery.Builder() .add(new MatchAllDocsQuery(), BooleanClause.Occur.SHOULD) .add(TermRangeQuery.newStringRange("bogus", "ta", "tf", true, true), BooleanClause.Occur.SHOULD) @@ -396,7 +407,7 @@ public class TestUnifiedHighlighterMTQ extends LuceneTestCase { iw.close(); IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(searcher, indexAnalyzer); BooleanQuery query = new BooleanQuery.Builder() .add(new WildcardQuery(new Term("body", "te*")), BooleanClause.Occur.SHOULD) .build(); @@ -438,7 +449,7 @@ public class TestUnifiedHighlighterMTQ extends LuceneTestCase { iw.close(); IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(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) @@ -469,7 +480,7 @@ public class TestUnifiedHighlighterMTQ extends LuceneTestCase { iw.close(); IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(searcher, indexAnalyzer); ConstantScoreQuery query = new ConstantScoreQuery(new WildcardQuery(new Term("body", "te*"))); TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER); assertEquals(2, topDocs.totalHits); @@ -497,7 +508,7 @@ public class TestUnifiedHighlighterMTQ extends LuceneTestCase { iw.close(); IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(searcher, indexAnalyzer); DisjunctionMaxQuery query = new DisjunctionMaxQuery( Collections.singleton(new WildcardQuery(new Term("body", "te*"))), 0); TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER); @@ -526,7 +537,7 @@ public class TestUnifiedHighlighterMTQ extends LuceneTestCase { iw.close(); IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(searcher, indexAnalyzer); // wrap in a SpanBoostQuery to also show we see inside it Query query = new SpanBoostQuery( new SpanMultiTermQueryWrapper<>(new WildcardQuery(new Term("body", "te*"))), 2.0f); @@ -556,7 +567,7 @@ public class TestUnifiedHighlighterMTQ extends LuceneTestCase { iw.close(); IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(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); @@ -585,7 +596,7 @@ public class TestUnifiedHighlighterMTQ extends LuceneTestCase { iw.close(); IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(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); @@ -614,7 +625,7 @@ public class TestUnifiedHighlighterMTQ extends LuceneTestCase { iw.close(); IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(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); @@ -644,7 +655,7 @@ public class TestUnifiedHighlighterMTQ extends LuceneTestCase { iw.close(); IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(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); @@ -675,7 +686,7 @@ public class TestUnifiedHighlighterMTQ extends LuceneTestCase { iw.close(); IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(searcher, indexAnalyzer); // use a variety of common MTQ types BooleanQuery query = new BooleanQuery.Builder() .add(new PrefixQuery(new Term("body", "te")), BooleanClause.Occur.SHOULD) @@ -765,7 +776,7 @@ public class TestUnifiedHighlighterMTQ extends LuceneTestCase { iw.close(); IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(searcher, indexAnalyzer); highlighter.setMaxLength(25);//a little past first sentence BooleanQuery query = new BooleanQuery.Builder() @@ -798,7 +809,7 @@ public class TestUnifiedHighlighterMTQ extends LuceneTestCase { iw.close(); IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(searcher, indexAnalyzer); highlighter.setMaxLength(32);//a little past first sentence BooleanQuery query = new BooleanQuery.Builder() @@ -846,7 +857,7 @@ public class TestUnifiedHighlighterMTQ extends LuceneTestCase { }; IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, buggyAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(searcher, buggyAnalyzer); highlighter.setHandleMultiTermQuery(true); if (rarely()) { highlighter.setMaxLength(25);//a little past first sentence @@ -903,7 +914,7 @@ public class TestUnifiedHighlighterMTQ extends LuceneTestCase { iw.close(); IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(searcher, indexAnalyzer); int docID = searcher.search(new TermQuery(new Term("id", "id")), 1).scoreDocs[0].doc; Query query = new PrefixQuery(new Term("body", "nonexistent")); @@ -934,7 +945,7 @@ public class TestUnifiedHighlighterMTQ extends LuceneTestCase { iw.close(); IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(searcher, indexAnalyzer); Query query = new PrefixQuery(new Term("body", "ab")); TopDocs topDocs = searcher.search(query, 10); @@ -956,7 +967,7 @@ public class TestUnifiedHighlighterMTQ extends LuceneTestCase { iw.close(); IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(searcher, indexAnalyzer); int docID = searcher.search(new TermQuery(new Term("id", "id")), 1).scoreDocs[0].doc; PhraseQuery pq = new PhraseQuery.Builder() @@ -1076,7 +1087,7 @@ public class TestUnifiedHighlighterMTQ extends LuceneTestCase { IndexSearcher searcher = newSearcher(ir); Query query = new PrefixQuery(new Term(field, "я")); TopDocs topDocs = searcher.search(query, 1); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, analyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(searcher, analyzer); String[] snippets = highlighter.highlight(field, query, topDocs); assertEquals("[я]", Arrays.toString(snippets)); ir.close(); @@ -1100,7 +1111,7 @@ public class TestUnifiedHighlighterMTQ extends LuceneTestCase { iw.commit(); try (IndexReader ir = iw.getReader()) { IndexSearcher searcher = newSearcher(ir); - UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, analyzer); + UnifiedHighlighter highlighter = randomUnifiedHighlighter(searcher, analyzer); highlighter.setBreakIterator(WholeBreakIterator::new); // Test PrefixQuery diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterStrictPhrases.java b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterStrictPhrases.java index acc4bd733ca..08820aa543c 100644 --- a/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterStrictPhrases.java +++ b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterStrictPhrases.java @@ -17,6 +17,7 @@ package org.apache.lucene.search.uhighlight; import java.io.IOException; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -38,6 +39,7 @@ import org.apache.lucene.search.MultiPhraseQuery; import org.apache.lucene.search.PhraseQuery; import org.apache.lucene.search.PrefixQuery; import org.apache.lucene.search.Query; +import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.ScoreMode; import org.apache.lucene.search.Sort; import org.apache.lucene.search.TermQuery; @@ -46,6 +48,7 @@ import org.apache.lucene.search.Weight; 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.SpanOrQuery; import org.apache.lucene.search.spans.SpanQuery; import org.apache.lucene.search.spans.SpanTermQuery; import org.apache.lucene.store.Directory; @@ -55,6 +58,7 @@ import org.apache.lucene.util.QueryBuilder; import org.junit.After; import org.junit.Before; +//TODO rename to reflect position sensitivity public class TestUnifiedHighlighterStrictPhrases extends LuceneTestCase { final FieldType fieldType; @@ -151,6 +155,16 @@ public class TestUnifiedHighlighterStrictPhrases extends LuceneTestCase { String[] snippets = highlighter.highlight("body", query, topDocs); assertArrayEquals(new String[]{"Yin yang, yin 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)); } public void testPhraseNotInDoc() throws IOException { @@ -185,6 +199,16 @@ public class TestUnifiedHighlighterStrictPhrases extends LuceneTestCase { String[] snippets = highlighter.highlight("body", query, topDocs); assertArrayEquals(new String[]{"alpha bravo charlie - 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)); } public void testSynonyms() throws IOException { @@ -477,4 +501,68 @@ public class TestUnifiedHighlighterStrictPhrases extends LuceneTestCase { return wrapped.hashCode(); } } + + // Ported from LUCENE-5455 (fixed in LUCENE-8121). Also see LUCENE-2287. + public void testNestedSpanQueryHighlight() throws Exception { + // For a long time, the highlighters used to assume all query terms within the SpanQuery were valid at the Spans' + // position range. This would highlight occurrences of terms that were actually not matched by the query. + // But now using the SpanCollector API we don't make this kind of mistake. + final String FIELD_NAME = "body"; + final String indexedText = "x y z x z x a"; + indexWriter.addDocument(newDoc(indexedText)); + initReaderSearcherHighlighter(); + TopDocs topDocs = new TopDocs(1, new ScoreDoc[]{new ScoreDoc(0, 1f)}, 1f); + + String expected = "x y z x z x a"; + Query q = new SpanNearQuery(new SpanQuery[] { + new SpanNearQuery(new SpanQuery[] { + new SpanTermQuery(new Term(FIELD_NAME, "x")), + new SpanTermQuery(new Term(FIELD_NAME, "y")), + new SpanTermQuery(new Term(FIELD_NAME, "z"))}, 0, true), + new SpanTermQuery(new Term(FIELD_NAME, "a"))}, 10, false); + String observed = highlighter.highlight(FIELD_NAME, q, topDocs)[0]; + if (VERBOSE) System.out.println("Expected: \"" + expected + "\n" + "Observed: \"" + observed); + assertEquals("Nested SpanNear query not properly highlighted.", expected, observed); + + expected = "x y z x z x a"; + q = new SpanNearQuery(new SpanQuery[] { + new SpanOrQuery( + new SpanNearQuery(new SpanQuery[] { + new SpanTermQuery(new Term(FIELD_NAME, "x")), + new SpanTermQuery(new Term(FIELD_NAME, "z"))}, 0, true), + new SpanNearQuery(new SpanQuery[] { + new SpanTermQuery(new Term(FIELD_NAME, "y")), + new SpanTermQuery(new Term(FIELD_NAME, "z"))}, 0, true)), + new SpanOrQuery( + new SpanTermQuery(new Term(FIELD_NAME, "a")), + new SpanTermQuery(new Term(FIELD_NAME, "b")))}, 10, false); + observed = highlighter.highlight(FIELD_NAME, q, topDocs)[0]; + if (VERBOSE) System.out.println("Expected: \"" + expected + "\n" + "Observed: \"" + observed); + assertEquals("Nested SpanNear query within SpanOr not properly highlighted.", expected, observed); + + expected = "x y z x z x a"; + q = new SpanNearQuery(new SpanQuery[] { + new SpanNearQuery(new SpanQuery[] { + new SpanMultiTermQueryWrapper<>(new WildcardQuery(new Term(FIELD_NAME, "*"))), + new SpanTermQuery(new Term(FIELD_NAME, "z"))}, 0, true), + new SpanTermQuery(new Term(FIELD_NAME, "a"))}, 10, false); + observed = highlighter.highlight(FIELD_NAME, q, topDocs)[0]; + if (VERBOSE) System.out.println("Expected: \"" + expected + "\n" + "Observed: \"" + observed); + assertEquals("Nested SpanNear query with wildcard not properly highlighted.", expected, observed); + + expected = "x y z x z x a"; + q = new SpanNearQuery(new SpanQuery[] { + new SpanOrQuery( + new SpanNearQuery(new SpanQuery[] { + new SpanTermQuery(new Term(FIELD_NAME, "x")), + new SpanTermQuery(new Term(FIELD_NAME, "y"))}, 0, true), + new SpanNearQuery(new SpanQuery[] { //No hit span query + new SpanTermQuery(new Term(FIELD_NAME, "z")), + new SpanTermQuery(new Term(FIELD_NAME, "a"))}, 0, true)), + new SpanTermQuery(new Term(FIELD_NAME, "a"))}, 10, false); + observed = highlighter.highlight(FIELD_NAME, q, topDocs)[0]; + if (VERBOSE) System.out.println("Expected: \"" + expected + "\n" + "Observed: \"" + observed); + assertEquals("Nested SpanNear query within SpanOr not properly highlighted.", expected, observed); + } + } diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/visibility/TestUnifiedHighlighterExtensibility.java b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/visibility/TestUnifiedHighlighterExtensibility.java index 738a0b9b4b0..e60b17be766 100644 --- a/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/visibility/TestUnifiedHighlighterExtensibility.java +++ b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/visibility/TestUnifiedHighlighterExtensibility.java @@ -218,11 +218,9 @@ public class TestUnifiedHighlighterExtensibility extends LuceneTestCase { // this code never runs; just for compilation Passage p; - try (OffsetsEnum oe = new OffsetsEnum(null, EMPTY)) { + try (OffsetsEnum oe = new OffsetsEnum.OfPostings(null, EMPTY)) { oe.getTerm(); - oe.getPostingsEnum(); oe.freq(); - oe.hasMorePositions(); oe.nextPosition(); oe.startOffset(); oe.endOffset();