LUCENE-8145: FieldOffsetStrategy.getOffsetEnum() now returns a single MultiOffsetsEnum

Closes #317
This commit is contained in:
Alan Woodward 2018-01-30 17:50:53 +00:00
parent 98a0b83714
commit 9422609a5d
15 changed files with 256 additions and 375 deletions

View File

@ -98,6 +98,10 @@ API Changes
* LUCENE-8104: Remove facets module compile-time dependency on queries
(Alan Woodward)
* LUCENE-8145: UnifiedHighlighter now uses a unitary OffsetsEnum rather
than a list of enums (Alan Woodward, David Smiley, Jim Ferenczi, Timothy
Rodriguez)
New Features
* LUCENE-2899: Add new module analysis/opennlp, with analysis components

View File

@ -1,145 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.search.uhighlight;
import java.io.IOException;
import java.util.List;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.PriorityQueue;
/**
* Provides a view over several underlying PostingsEnums for the iteration of offsets on the current document only.
* It's not general purpose; the position returned is always -1 and it doesn't iterate the documents.
*/
final class CompositeOffsetsPostingsEnum extends PostingsEnum {
private final int docId;
private final int freq;
private final PriorityQueue<BoundsCheckingPostingsEnum> queue;
private boolean firstPositionConsumed = false;
/**
* This class is used to ensure we don't over iterate the underlying
* postings enum by keeping track of the position relative to the
* frequency.
* Ideally this would've been an implementation of a PostingsEnum
* but it would have to delegate most methods and it seemed easier
* to just wrap the tweaked method.
*/
private static final class BoundsCheckingPostingsEnum {
private final PostingsEnum postingsEnum;
private int remainingPositions;
BoundsCheckingPostingsEnum(PostingsEnum postingsEnum) throws IOException {
this.postingsEnum = postingsEnum;
this.remainingPositions = postingsEnum.freq();
nextPosition();
}
/** Advances to the next position and returns true, or returns false if it can't. */
private boolean nextPosition() throws IOException {
if (remainingPositions-- > 0) {
postingsEnum.nextPosition(); // ignore the actual position; we don't care.
return true;
} else {
return false;
}
}
}
/** The provided {@link PostingsEnum}s must all be positioned to the same document, and must have offsets. */
CompositeOffsetsPostingsEnum(List<PostingsEnum> postingsEnums) throws IOException {
queue = new PriorityQueue<BoundsCheckingPostingsEnum>(postingsEnums.size()) {
@Override
protected boolean lessThan(BoundsCheckingPostingsEnum a, BoundsCheckingPostingsEnum b) {
try {
return a.postingsEnum.startOffset() < b.postingsEnum.startOffset();
} catch (IOException e) {
throw new RuntimeException(e);
}
}
};
int freqAdd = 0;
for (PostingsEnum postingsEnum : postingsEnums) {
queue.add(new BoundsCheckingPostingsEnum(postingsEnum));
freqAdd += postingsEnum.freq();
}
freq = freqAdd;
this.docId = queue.top().postingsEnum.docID();
}
@Override
public int freq() throws IOException {
return freq;
}
/** Advances to the next position. Always returns -1; the caller is assumed not to care for the highlighter. */
@Override
public int nextPosition() throws IOException {
if (!firstPositionConsumed) {
firstPositionConsumed = true;
} else if (queue.size() == 0) {
throw new IllegalStateException("nextPosition called too many times");
} else if (queue.top().nextPosition()) { // advance head
queue.updateTop(); //the new position may be behind another postingsEnum in the queue
} else {
queue.pop(); //this postingsEnum is consumed; get rid of it. Another will take it's place.
}
assert queue.size() > 0;
return -1;
}
@Override
public int startOffset() throws IOException {
return queue.top().postingsEnum.startOffset();
}
@Override
public int endOffset() throws IOException {
return queue.top().postingsEnum.endOffset();
}
@Override
public BytesRef getPayload() throws IOException {
return queue.top().postingsEnum.getPayload();
}
@Override
public int docID() {
return docId;
}
@Override
public int nextDoc() throws IOException {
return NO_MORE_DOCS;
}
@Override
public int advance(int target) throws IOException {
return NO_MORE_DOCS;
}
@Override
public long cost() {
return 1L; //at most 1 doc is returned
}
}

View File

@ -25,9 +25,7 @@ import java.util.List;
import java.util.PriorityQueue;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
/**
* Internal highlighter abstraction that operates on a per field basis.
@ -76,27 +74,22 @@ public class FieldHighlighter {
breakIterator.setText(content);
List<OffsetsEnum> offsetsEnums = fieldOffsetStrategy.getOffsetsEnums(reader, docId, content);
try (OffsetsEnum offsetsEnums = fieldOffsetStrategy.getOffsetsEnum(reader, docId, content)) {
Passage[] passages;
try {
// Highlight the offsetsEnum list against the content to produce Passages.
passages = highlightOffsetsEnums(offsetsEnums);// and breakIterator & scorer
} finally {
// Ensure closeable resources get closed
IOUtils.close(offsetsEnums);
}
Passage[] passages = highlightOffsetsEnums(offsetsEnums);// and breakIterator & scorer
// Format the resulting Passages.
if (passages.length == 0) {
// no passages were returned, so ask for a default summary
passages = getSummaryPassagesNoHighlight(maxNoHighlightPassages == -1 ? maxPassages : maxNoHighlightPassages);
}
// Format the resulting Passages.
if (passages.length == 0) {
// no passages were returned, so ask for a default summary
passages = getSummaryPassagesNoHighlight(maxNoHighlightPassages == -1 ? maxPassages : maxNoHighlightPassages);
}
if (passages.length > 0) {
return passageFormatter.format(passages, content);
} else {
return null;
if (passages.length > 0) {
return passageFormatter.format(passages, content);
} else {
return null;
}
}
}
@ -118,7 +111,6 @@ public class FieldHighlighter {
break;
}
Passage passage = new Passage();
passage.setScore(Float.NaN);
passage.setStartOffset(pos);
passage.setEndOffset(next);
passages.add(passage);
@ -131,21 +123,14 @@ public class FieldHighlighter {
// algorithm: treat sentence snippets as miniature documents
// we can intersect these with the postings lists via BreakIterator.preceding(offset),s
// score each sentence as norm(sentenceStartOffset) * sum(weight * tf(freq))
protected Passage[] highlightOffsetsEnums(List<OffsetsEnum> offsetsEnums)
protected Passage[] highlightOffsetsEnums(OffsetsEnum off)
throws IOException {
PassageScorer scorer = passageScorer;
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<OffsetsEnum> offsetsEnumQueue = new PriorityQueue<>(offsetsEnums.size() + 1);
for (OffsetsEnum off : offsetsEnums) {
off.setWeight(scorer.weight(contentLength, off.freq()));
if (off.nextPosition()) {// go to first position
offsetsEnumQueue.add(off);
}
final int contentLength = this.breakIterator.getText().getEndIndex();
if (off.nextPosition() == false) {
return new Passage[0];
}
offsetsEnumQueue.add(new OffsetsEnum.OfPostings(new BytesRef(), EMPTY)); // a sentinel for termination
PriorityQueue<Passage> passageQueue = new PriorityQueue<>(Math.min(64, maxPassages + 1), (left, right) -> {
if (left.getScore() < right.getScore()) {
@ -158,121 +143,58 @@ public class FieldHighlighter {
});
Passage passage = new Passage(); // the current passage in-progress. Will either get reset or added to queue.
OffsetsEnum off;
while ((off = offsetsEnumQueue.poll()) != null) {
do {
int start = off.startOffset();
if (start == -1) {
throw new IllegalArgumentException("field '" + field + "' was indexed without offsets, cannot highlight");
}
int end = off.endOffset();
// LUCENE-5166: this hit would span the content limit... however more valid
// hits may exist (they are sorted by start). so we pretend like we never
// saw this term, it won't cause a passage to be added to passageQueue or anything.
assert EMPTY.startOffset() == Integer.MAX_VALUE;
if (start < contentLength && end > contentLength) {
continue;
}
// See if this term should be part of a new passage.
if (start >= passage.getEndOffset()) {
if (passage.getStartOffset() >= 0) { // true if this passage has terms; otherwise couldn't find any (yet)
// finalize passage
passage.setScore(passage.getScore() * scorer.norm(passage.getStartOffset()));
// new sentence: first add 'passage' to queue
if (passageQueue.size() == maxPassages && passage.getScore() < passageQueue.peek().getScore()) {
passage.reset(); // can't compete, just reset it
} else {
passageQueue.offer(passage);
if (passageQueue.size() > maxPassages) {
passage = passageQueue.poll();
passage.reset();
} else {
passage = new Passage();
}
}
}
passage = maybeAddPassage(passageQueue, passageScorer, passage, contentLength);
// if we exceed limit, we are done
if (start >= contentLength) {
break;
}
// advance breakIterator
passage.setStartOffset(Math.max(breakIterator.preceding(start + 1), 0));
passage.setEndOffset(Math.min(breakIterator.following(start), contentLength));
passage.setStartOffset(Math.max(this.breakIterator.preceding(start + 1), 0));
passage.setEndOffset(Math.min(this.breakIterator.following(start), contentLength));
}
// Add this term to the passage.
int tf = 0;
while (true) {
tf++;
BytesRef term = off.getTerm();// a reference; safe to refer to
assert term != null;
passage.addMatch(start, end, term);
// see if there are multiple occurrences of this term in this passage. If so, add them.
if (!off.nextPosition()) {
break; // No more in the entire text. Already removed from pq; move on
}
start = off.startOffset();
end = off.endOffset();
if (start >= passage.getEndOffset() || end > contentLength) { // it's beyond this passage
offsetsEnumQueue.offer(off);
break;
}
}
passage.setScore(passage.getScore() + off.getWeight() * scorer.tf(tf, passage.getEndOffset() - passage.getStartOffset()));
}
BytesRef term = off.getTerm();// a reference; safe to refer to
assert term != null;
passage.addMatch(start, end, term, off.freq());
} while (off.nextPosition());
maybeAddPassage(passageQueue, passageScorer, passage, contentLength);
Passage[] passages = passageQueue.toArray(new Passage[passageQueue.size()]);
for (Passage p : passages) {
p.sort();
}
// sort in ascending order
Arrays.sort(passages, Comparator.comparingInt(Passage::getStartOffset));
return passages;
}
protected static final PostingsEnum EMPTY = new PostingsEnum() {
@Override
public int nextPosition() throws IOException {
return 0;
private Passage maybeAddPassage(PriorityQueue<Passage> passageQueue, PassageScorer scorer, Passage passage, int contentLength) {
if (passage.getStartOffset() == -1) {
// empty passage, we can ignore it
return passage;
}
@Override
public int startOffset() throws IOException {
return Integer.MAX_VALUE;
passage.setScore(scorer.score(passage, contentLength));
// new sentence: first add 'passage' to queue
if (passageQueue.size() == maxPassages && passage.getScore() < passageQueue.peek().getScore()) {
passage.reset(); // can't compete, just reset it
} else {
passageQueue.offer(passage);
if (passageQueue.size() > maxPassages) {
passage = passageQueue.poll();
passage.reset();
} else {
passage = new Passage();
}
}
return passage;
}
@Override
public int endOffset() throws IOException {
return Integer.MAX_VALUE;
}
@Override
public BytesRef getPayload() throws IOException {
return null;
}
@Override
public int freq() throws IOException {
return 0;
}
@Override
public int docID() {
return NO_MORE_DOCS;
}
@Override
public int nextDoc() throws IOException {
return NO_MORE_DOCS;
}
@Override
public int advance(int target) throws IOException {
return NO_MORE_DOCS;
}
@Override
public long cost() {
return 0;
}
};
}

View File

@ -18,7 +18,6 @@ package org.apache.lucene.search.uhighlight;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import org.apache.lucene.index.IndexReader;
@ -58,14 +57,15 @@ public abstract class FieldOffsetStrategy {
/**
* The primary method -- return offsets for highlightable words in the specified document.
* IMPORTANT: remember to close them all.
*
* Callers are expected to close the returned OffsetsEnum when it has been finished with
*/
public abstract List<OffsetsEnum> getOffsetsEnums(IndexReader reader, int docId, String content) throws IOException;
public abstract OffsetsEnum getOffsetsEnum(IndexReader reader, int docId, String content) throws IOException;
protected List<OffsetsEnum> createOffsetsEnumsFromReader(LeafReader leafReader, int doc) throws IOException {
protected OffsetsEnum createOffsetsEnumFromReader(LeafReader leafReader, int doc) throws IOException {
final Terms termsIndex = leafReader.terms(field);
if (termsIndex == null) {
return Collections.emptyList();
return OffsetsEnum.EMPTY;
}
final List<OffsetsEnum> offsetsEnums = new ArrayList<>(terms.length + automata.length);
@ -92,7 +92,7 @@ public abstract class FieldOffsetStrategy {
createOffsetsEnumsForAutomata(termsIndex, doc, offsetsEnums);
}
return offsetsEnums;
return new OffsetsEnum.MultiOffsetsEnum(offsetsEnums);
}
protected void createOffsetsEnumsForTerms(BytesRef[] sourceTerms, Terms termsIndex, int doc, List<OffsetsEnum> results) throws IOException {
@ -137,14 +137,17 @@ public abstract class FieldOffsetStrategy {
for (int i = 0; i < automata.length; i++) {
CharacterRunAutomaton automaton = automata[i];
List<PostingsEnum> postingsEnums = automataPostings.get(i);
int size = postingsEnums.size();
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
results.add(new OffsetsEnum.OfPostings(wildcardTerm, postingsEnums.get(0)));
} else {
results.add(new OffsetsEnum.OfPostings(wildcardTerm, new CompositeOffsetsPostingsEnum(postingsEnums)));
}
if (postingsEnums.isEmpty()) {
continue;
}
// Build one OffsetsEnum exposing the automata.toString as the term, and the sum of freq
BytesRef wildcardTerm = new BytesRef(automaton.toString());
int sumFreq = 0;
for (PostingsEnum postingsEnum : postingsEnums) {
sumFreq += postingsEnum.freq();
}
for (PostingsEnum postingsEnum : postingsEnums) {
results.add(new OffsetsEnum.OfPostings(wildcardTerm, sumFreq, postingsEnum));
}
}

View File

@ -100,7 +100,7 @@ public class MemoryIndexOffsetStrategy extends AnalysisOffsetStrategy {
}
@Override
public List<OffsetsEnum> getOffsetsEnums(IndexReader reader, int docId, String content) throws IOException {
public OffsetsEnum getOffsetsEnum(IndexReader reader, int docId, String content) throws IOException {
// note: don't need LimitTokenOffsetFilter since content is already truncated to maxLength
TokenStream tokenStream = tokenStream(content);
@ -110,7 +110,7 @@ public class MemoryIndexOffsetStrategy extends AnalysisOffsetStrategy {
memoryIndex.addField(field, tokenStream);//note: calls tokenStream.reset() & close()
docId = 0;
return createOffsetsEnumsFromReader(leafReader, docId);
return createOffsetsEnumFromReader(leafReader, docId);
}

View File

@ -17,8 +17,6 @@
package org.apache.lucene.search.uhighlight;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.util.BytesRef;
@ -43,8 +41,8 @@ public class NoOpOffsetStrategy extends FieldOffsetStrategy {
}
@Override
public List<OffsetsEnum> getOffsetsEnums(IndexReader reader, int docId, String content) throws IOException {
return Collections.emptyList();
public OffsetsEnum getOffsetsEnum(IndexReader reader, int docId, String content) throws IOException {
return OffsetsEnum.EMPTY;
}
}

View File

@ -21,21 +21,21 @@ import java.io.Closeable;
import java.io.IOException;
import java.util.List;
import java.util.Objects;
import java.util.PriorityQueue;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
/**
* 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.
* {@link FieldHighlighter#highlightOffsetsEnums(OffsetsEnum)} based on the start offset.
*
* @lucene.internal
*/
public abstract class OffsetsEnum implements Comparable<OffsetsEnum>, Closeable {
private float weight; // set once in highlightOffsetsEnums
// 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
@ -82,14 +82,6 @@ public abstract class OffsetsEnum implements Comparable<OffsetsEnum>, Closeable
public abstract int endOffset() throws IOException;
public float getWeight() {
return weight;
}
public void setWeight(float weight) {
this.weight = weight;
}
@Override
public void close() throws IOException {
}
@ -110,12 +102,19 @@ public abstract class OffsetsEnum implements Comparable<OffsetsEnum>, Closeable
public static class OfPostings extends OffsetsEnum {
private final BytesRef term;
private final PostingsEnum postingsEnum; // with offsets
private final int freq;
private int posCounter = 0; // the occurrence counter of this term within the text being highlighted.
private int posCounter = -1;
public OfPostings(BytesRef term, PostingsEnum postingsEnum) throws IOException {
public OfPostings(BytesRef term, int freq, PostingsEnum postingsEnum) throws IOException {
this.term = Objects.requireNonNull(term);
this.postingsEnum = Objects.requireNonNull(postingsEnum);
this.freq = freq;
this.posCounter = this.postingsEnum.freq();
}
public OfPostings(BytesRef term, PostingsEnum postingsEnum) throws IOException {
this(term, postingsEnum.freq(), postingsEnum);
}
public PostingsEnum getPostingsEnum() {
@ -124,8 +123,8 @@ public abstract class OffsetsEnum implements Comparable<OffsetsEnum>, Closeable
@Override
public boolean nextPosition() throws IOException {
if (posCounter < postingsEnum.freq()) {
posCounter++;
if (posCounter > 0) {
posCounter--;
postingsEnum.nextPosition(); // note: we don't need to save the position
return true;
} else {
@ -133,11 +132,6 @@ public abstract class OffsetsEnum implements Comparable<OffsetsEnum>, Closeable
}
}
@Override
public int freq() throws IOException {
return postingsEnum.freq();
}
@Override
public BytesRef getTerm() throws IOException {
return term;
@ -153,5 +147,104 @@ public abstract class OffsetsEnum implements Comparable<OffsetsEnum>, Closeable
return postingsEnum.endOffset();
}
@Override
public int freq() throws IOException {
return freq;
}
}
/**
* Empty enumeration
*/
public static final OffsetsEnum EMPTY = new OffsetsEnum() {
@Override
public boolean nextPosition() throws IOException {
return false;
}
@Override
public BytesRef getTerm() throws IOException {
throw new UnsupportedOperationException();
}
@Override
public int startOffset() throws IOException {
throw new UnsupportedOperationException();
}
@Override
public int endOffset() throws IOException {
throw new UnsupportedOperationException();
}
@Override
public int freq() throws IOException {
return 0;
}
};
/**
* A view over several OffsetsEnum instances, merging them in-place
*/
public static class MultiOffsetsEnum extends OffsetsEnum {
private final PriorityQueue<OffsetsEnum> queue;
private boolean started = false;
public MultiOffsetsEnum(List<OffsetsEnum> inner) throws IOException {
this.queue = new PriorityQueue<>();
for (OffsetsEnum oe : inner) {
if (oe.nextPosition())
this.queue.add(oe);
}
}
@Override
public boolean nextPosition() throws IOException {
if (started == false) {
started = true;
return this.queue.size() > 0;
}
if (this.queue.size() > 0) {
OffsetsEnum top = this.queue.poll();
if (top.nextPosition()) {
this.queue.add(top);
return true;
}
else {
top.close();
}
return this.queue.size() > 0;
}
return false;
}
@Override
public BytesRef getTerm() throws IOException {
return this.queue.peek().getTerm();
}
@Override
public int startOffset() throws IOException {
return this.queue.peek().startOffset();
}
@Override
public int endOffset() throws IOException {
return this.queue.peek().endOffset();
}
@Override
public int freq() throws IOException {
return this.queue.peek().freq();
}
@Override
public void close() throws IOException {
// most child enums will have been closed in .nextPosition()
// here all remaining non-exhausted enums are closed
IOUtils.close(queue);
}
}
}

View File

@ -19,7 +19,6 @@ package org.apache.lucene.search.uhighlight;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.InPlaceMergeSorter;
import org.apache.lucene.util.RamUsageEstimator;
/**
@ -38,59 +37,35 @@ public class Passage {
private int[] matchStarts = new int[8];
private int[] matchEnds = new int[8];
private BytesRef[] matchTerms = new BytesRef[8];
private int[] matchTermFreqInDoc = new int[8];
private int numMatches = 0;
/** @lucene.internal */
public void addMatch(int startOffset, int endOffset, BytesRef term) {
public void addMatch(int startOffset, int endOffset, BytesRef term, int termFreqInDoc) {
assert startOffset >= this.startOffset && startOffset <= this.endOffset;
if (numMatches == matchStarts.length) {
int newLength = ArrayUtil.oversize(numMatches + 1, RamUsageEstimator.NUM_BYTES_OBJECT_REF);
int newMatchStarts[] = new int[newLength];
int newMatchEnds[] = new int[newLength];
int newMatchTermFreqInDoc[] = new int[newLength];
BytesRef newMatchTerms[] = new BytesRef[newLength];
System.arraycopy(matchStarts, 0, newMatchStarts, 0, numMatches);
System.arraycopy(matchEnds, 0, newMatchEnds, 0, numMatches);
System.arraycopy(matchTerms, 0, newMatchTerms, 0, numMatches);
System.arraycopy(matchTermFreqInDoc, 0, newMatchTermFreqInDoc, 0, numMatches);
matchStarts = newMatchStarts;
matchEnds = newMatchEnds;
matchTerms = newMatchTerms;
matchTermFreqInDoc = newMatchTermFreqInDoc;
}
assert matchStarts.length == matchEnds.length && matchEnds.length == matchTerms.length;
matchStarts[numMatches] = startOffset;
matchEnds[numMatches] = endOffset;
matchTerms[numMatches] = term;
matchTermFreqInDoc[numMatches] = termFreqInDoc;
numMatches++;
}
/** @lucene.internal */
public void sort() {
final int starts[] = matchStarts;
final int ends[] = matchEnds;
final BytesRef terms[] = matchTerms;
new InPlaceMergeSorter() {
@Override
protected void swap(int i, int j) {
int temp = starts[i];
starts[i] = starts[j];
starts[j] = temp;
temp = ends[i];
ends[i] = ends[j];
ends[j] = temp;
BytesRef tempTerm = terms[i];
terms[i] = terms[j];
terms[j] = tempTerm;
}
@Override
protected int compare(int i, int j) {
return Integer.compare(starts[i], starts[j]);
}
}.sort(0, numMatches);
}
/** @lucene.internal */
public void reset() {
startOffset = endOffset = -1;
@ -136,6 +111,10 @@ public class Passage {
return endOffset;
}
public int getLength() {
return endOffset - startOffset;
}
/**
* Passage's score.
*/
@ -143,6 +122,10 @@ public class Passage {
return score;
}
public void setScore(float score) {
this.score = score;
}
/**
* Number of term matches available in
* {@link #getMatchStarts}, {@link #getMatchEnds},
@ -182,6 +165,10 @@ public class Passage {
return matchTerms;
}
public int[] getMatchTermFreqsInDoc() {
return matchTermFreqInDoc;
}
/** @lucene.internal */
public void setStartOffset(int startOffset) {
this.startOffset = startOffset;
@ -193,8 +180,4 @@ public class Passage {
this.endOffset = endOffset;
}
/** @lucene.internal */
public void setScore(float score) {
this.score = score;
}
}

View File

@ -16,6 +16,10 @@
*/
package org.apache.lucene.search.uhighlight;
import java.util.Arrays;
import org.apache.lucene.util.BytesRefHash;
/**
* Ranks passages found by {@link UnifiedHighlighter}.
* <p>
@ -110,4 +114,30 @@ public class PassageScorer {
public float norm(int passageStart) {
return 1 + 1 / (float) Math.log(pivot + passageStart);
}
public float score(Passage passage, int contentLength) {
float score = 0;
BytesRefHash termsHash = new BytesRefHash();
int hitCount = passage.getNumMatches();
int[] termFreqsInPassage = new int[hitCount]; // maximum size
int[] termFreqsInDoc = new int[hitCount];
Arrays.fill(termFreqsInPassage, 0);
for (int i = 0; i < passage.getNumMatches(); i++) {
int termIndex = termsHash.add(passage.getMatchTerms()[i]);
if (termIndex < 0) {
termIndex = -(termIndex + 1);
}
else {
termFreqsInDoc[termIndex] = passage.getMatchTermFreqsInDoc()[i];
}
termFreqsInPassage[termIndex]++;
}
for (int i = 0; i < termsHash.size(); i++) {
score += tf(termFreqsInPassage[i], passage.getLength()) * weight(contentLength, termFreqsInDoc[i]);
}
score *= norm(passage.getStartOffset());
return score;
}
}

View File

@ -40,7 +40,7 @@ public class PostingsOffsetStrategy extends FieldOffsetStrategy {
}
@Override
public List<OffsetsEnum> getOffsetsEnums(IndexReader reader, int docId, String content) throws IOException {
public OffsetsEnum getOffsetsEnum(IndexReader reader, int docId, String content) throws IOException {
final LeafReader leafReader;
if (reader instanceof LeafReader) {
leafReader = (LeafReader) reader;
@ -51,7 +51,7 @@ public class PostingsOffsetStrategy extends FieldOffsetStrategy {
docId -= leafReaderContext.docBase; // adjust 'doc' to be within this leaf reader
}
return createOffsetsEnumsFromReader(leafReader, docId);
return createOffsetsEnumFromReader(leafReader, docId);
}

View File

@ -17,7 +17,6 @@
package org.apache.lucene.search.uhighlight;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
import org.apache.lucene.index.IndexReader;
@ -40,7 +39,7 @@ public class PostingsWithTermVectorsOffsetStrategy extends FieldOffsetStrategy {
}
@Override
public List<OffsetsEnum> getOffsetsEnums(IndexReader reader, int docId, String content) throws IOException {
public OffsetsEnum getOffsetsEnum(IndexReader reader, int docId, String content) throws IOException {
LeafReader leafReader;
if (reader instanceof LeafReader) {
leafReader = (LeafReader) reader;
@ -53,11 +52,11 @@ public class PostingsWithTermVectorsOffsetStrategy extends FieldOffsetStrategy {
Terms docTerms = leafReader.getTermVector(docId, field);
if (docTerms == null) {
return Collections.emptyList();
return OffsetsEnum.EMPTY;
}
leafReader = new TermVectorFilteredLeafReader(leafReader, docTerms);
return createOffsetsEnumsFromReader(leafReader, docId);
return createOffsetsEnumFromReader(leafReader, docId);
}
@Override

View File

@ -17,8 +17,6 @@
package org.apache.lucene.search.uhighlight;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReader;
@ -44,16 +42,16 @@ public class TermVectorOffsetStrategy extends FieldOffsetStrategy {
}
@Override
public List<OffsetsEnum> getOffsetsEnums(IndexReader reader, int docId, String content) throws IOException {
public OffsetsEnum getOffsetsEnum(IndexReader reader, int docId, String content) throws IOException {
Terms tvTerms = reader.getTermVector(docId, field);
if (tvTerms == null) {
return Collections.emptyList();
return OffsetsEnum.EMPTY;
}
LeafReader leafReader = new TermVectorLeafReader(field, tvTerms);
docId = 0;
return createOffsetsEnumsFromReader(leafReader, docId);
return createOffsetsEnumFromReader(leafReader, docId);
}
}

View File

@ -17,8 +17,6 @@
package org.apache.lucene.search.uhighlight;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.TokenStream;
@ -60,8 +58,8 @@ public class TokenStreamOffsetStrategy extends AnalysisOffsetStrategy {
}
@Override
public List<OffsetsEnum> getOffsetsEnums(IndexReader reader, int docId, String content) throws IOException {
return Collections.singletonList(new TokenStreamOffsetsEnum(tokenStream(content), automata));
public OffsetsEnum getOffsetsEnum(IndexReader reader, int docId, String content) throws IOException {
return new TokenStreamOffsetsEnum(tokenStream(content), automata);
}
private static class TokenStreamOffsetsEnum extends OffsetsEnum {
@ -106,6 +104,7 @@ public class TokenStreamOffsetStrategy extends AnalysisOffsetStrategy {
return Integer.MAX_VALUE; // lie
}
@Override
public int startOffset() throws IOException {
return offsetAtt.startOffset();

View File

@ -16,6 +16,10 @@
*/
package org.apache.lucene.search.uhighlight;
import java.io.IOException;
import java.util.HashSet;
import java.util.Random;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.analysis.MockTokenizer;
@ -39,10 +43,6 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
import java.io.IOException;
import java.util.HashSet;
import java.util.Random;
public class TestUnifiedHighlighterRanking extends LuceneTestCase {
Analyzer indexAnalyzer;

View File

@ -19,7 +19,6 @@ 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;
@ -69,13 +68,13 @@ public class TestUnifiedHighlighterExtensibility extends LuceneTestCase {
}
@Override
public List<OffsetsEnum> getOffsetsEnums(IndexReader reader, int docId, String content) throws IOException {
return Collections.emptyList();
public OffsetsEnum getOffsetsEnum(IndexReader reader, int docId, String content) throws IOException {
return OffsetsEnum.EMPTY;
}
@Override
protected List<OffsetsEnum> createOffsetsEnumsFromReader(LeafReader leafReader, int doc) throws IOException {
return super.createOffsetsEnumsFromReader(leafReader, doc);
protected OffsetsEnum createOffsetsEnumFromReader(LeafReader leafReader, int doc) throws IOException {
return super.createOffsetsEnumFromReader(leafReader, doc);
}
};
@ -193,7 +192,7 @@ public class TestUnifiedHighlighterExtensibility extends LuceneTestCase {
final String fieldName = "fieldName";
FieldHighlighter fieldHighlighter = new FieldHighlighter(fieldName, null, null, null, 1, 1, null) {
@Override
protected Passage[] highlightOffsetsEnums(List<OffsetsEnum> offsetsEnums) throws IOException {
protected Passage[] highlightOffsetsEnums(OffsetsEnum offsetsEnums) throws IOException {
return super.highlightOffsetsEnums(offsetsEnums);
}
};
@ -213,31 +212,29 @@ public class TestUnifiedHighlighterExtensibility extends LuceneTestCase {
}
@Override
protected Passage[] highlightOffsetsEnums(List<OffsetsEnum> offsetsEnums) throws IOException {
protected Passage[] highlightOffsetsEnums(OffsetsEnum offsetsEnums) throws IOException {
// TEST OffsetsEnums & Passage visibility
// this code never runs; just for compilation
Passage p;
try (OffsetsEnum oe = new OffsetsEnum.OfPostings(null, EMPTY)) {
try (OffsetsEnum oe = new OffsetsEnum.OfPostings(null, null)) {
oe.getTerm();
oe.freq();
oe.nextPosition();
oe.startOffset();
oe.endOffset();
oe.getWeight();
oe.setWeight(2f);
oe.freq();
}
p = new Passage();
p.setStartOffset(0);
p.setEndOffset(9);
p.setScore(1f);
p.addMatch(1, 2, new BytesRef());
p.addMatch(1, 2, new BytesRef(), 1);
p.reset();
p.sort();
p.setScore(1);
//... getters are all exposed; custom PassageFormatter impls uses them
return super.highlightOffsetsEnums(offsetsEnums);
}
}
}