Added third highlighter type based on lucene postings highlighter
Requires field index_options set to "offsets" in order to store positions and offsets in the postings list. Considerably faster than the plain highlighter since it doesn't require to reanalyze the text to be highlighted: the larger the documents the better the performance gain should be. Requires less disk space than term_vectors, needed for the fast_vector_highlighter. Breaks the text into sentences and highlights them. Uses a BreakIterator to find sentences in the text. Plays really well with natural text, not quite the same if the text contains html markup for instance. Treats the document as the whole corpus, and scores individual sentences as if they were documents in this corpus, using the BM25 algorithm. Uses forked version of lucene postings highlighter to support: - per value discrete highlighting for fields that have multiple values, needed when number_of_fragments=0 since we want to return a snippet per value - manually passing in query terms to avoid calling extract terms multiple times, since we use a different highlighter instance per doc/field, but the query is always the same The lucene postings highlighter api is quite different compared to the existing highlighters api, the main difference being that it allows to highlight multiple fields in multiple docs with a single call, ensuring sequential IO. The way it is introduced in elasticsearch in this first round is a compromise trying not to change the current highlight api, which works per document, per field. The main disadvantage is that we lose the sequential IO, but we can always refactor the highlight api to work with multiple documents. Supports pre_tag, post_tag, number_of_fragments (0 highlights the whole field), require_field_match, no_match_size, order by score and html encoding. Closes #3704
This commit is contained in:
parent
08b4ca66c3
commit
48ac9747a8
|
@ -2,8 +2,9 @@
|
|||
=== Highlighting
|
||||
|
||||
Allows to highlight search results on one or more fields. The
|
||||
implementation uses either the lucene `fast-vector-highlighter` or
|
||||
`highlighter`. The search request body:
|
||||
implementation uses either the lucene `highlighter`, `fast-vector-highlighter`
|
||||
or `postings-highlighter`. The following is an example of the search request
|
||||
body:
|
||||
|
||||
[source,js]
|
||||
--------------------------------------------------
|
||||
|
@ -24,16 +25,54 @@ fragments).
|
|||
|
||||
In order to perform highlighting, the actual content of the field is
|
||||
required. If the field in question is stored (has `store` set to `yes`
|
||||
in the mapping), it will be used, otherwise, the actual `_source` will
|
||||
in the mapping) it will be used, otherwise, the actual `_source` will
|
||||
be loaded and the relevant field will be extracted from it.
|
||||
|
||||
The field name supports wildcard notation. For example, using `comment_*`
|
||||
will cause all fields that match the expression to be highlighted.
|
||||
|
||||
==== Postings highlighter
|
||||
|
||||
If `index_options` is set to `offsets` in the mapping the postings highlighter
|
||||
will be used instead of the plain highlighter. The postings highlighter:
|
||||
|
||||
* Is faster since it doesn't require to reanalyze the text to be highlighted:
|
||||
the larger the documents the better the performance gain should be
|
||||
* Requires less disk space than term_vectors, needed for the fast vector
|
||||
highlighter
|
||||
* Breaks the text into sentences and highlights them. Plays really well with
|
||||
natural languages, not as well with fields containing for instance html markup
|
||||
* Treats the document as the whole corpus, and scores individual sentences as
|
||||
if they were documents in this corpus, using the BM25 algorithm
|
||||
|
||||
Here is an example of setting the `content` field to allow for
|
||||
highlighting using the postings highlighter on it:
|
||||
|
||||
[source,js]
|
||||
--------------------------------------------------
|
||||
{
|
||||
"type_name" : {
|
||||
"content" : {"index_options" : "offsets"}
|
||||
}
|
||||
}
|
||||
--------------------------------------------------
|
||||
|
||||
Note that the postings highlighter is meant to perform simple query terms
|
||||
highlighting, regardless of their positions. That means that when used for
|
||||
instance in combination with a phrase query, it will highlight all the terms
|
||||
that the query is composed of, regardless of whether they are actually part of
|
||||
a query match, effectively ignoring their positions.
|
||||
|
||||
|
||||
==== Fast vector highlighter
|
||||
|
||||
If `term_vector` information is provided by setting `term_vector` to
|
||||
`with_positions_offsets` in the mapping then the fast vector
|
||||
highlighter will be used instead of the plain highlighter. The fast vector highlighter:
|
||||
`with_positions_offsets` in the mapping then the fast vector highlighter
|
||||
will be used instead of the plain highlighter. The fast vector highlighter:
|
||||
|
||||
* Is faster especially for large fields (> `1MB`)
|
||||
* Can be customized with `boundary_chars`, `boundary_max_scan`, and
|
||||
`fragment_offset` (see below)
|
||||
`fragment_offset` (see <<boundary-characters,below>>)
|
||||
* Requires setting `term_vector` to `with_positions_offsets` which
|
||||
increases the size of the index
|
||||
|
||||
|
@ -50,9 +89,25 @@ the index to be bigger):
|
|||
}
|
||||
--------------------------------------------------
|
||||
|
||||
The field name supports wildcard notation, for example,
|
||||
using `comment_*` which will cause all fields that match the expression
|
||||
to be highlighted.
|
||||
==== Force highlighter type
|
||||
|
||||
The `type` field allows to force a specific highlighter type. This is useful
|
||||
for instance when needing to use the plain highlighter on a field that has
|
||||
`term_vectors` enabled. The allowed values are: `plain`, `postings` and `fvh`.
|
||||
The following is an example that forces the use of the plain highlighter:
|
||||
|
||||
[source,js]
|
||||
--------------------------------------------------
|
||||
{
|
||||
"query" : {...},
|
||||
"highlight" : {
|
||||
"fields" : {
|
||||
"content" : { "type" : "plain"}
|
||||
}
|
||||
}
|
||||
}
|
||||
--------------------------------------------------
|
||||
|
||||
|
||||
[[tags]]
|
||||
==== Highlighting Tags
|
||||
|
@ -61,6 +116,23 @@ By default, the highlighting will wrap highlighted text in `<em>` and
|
|||
`</em>`. This can be controlled by setting `pre_tags` and `post_tags`,
|
||||
for example:
|
||||
|
||||
[source,js]
|
||||
--------------------------------------------------
|
||||
{
|
||||
"query" : {...},
|
||||
"highlight" : {
|
||||
"pre_tags" : ["<tag1>"],
|
||||
"post_tags" : ["</tag1>"],
|
||||
"fields" : {
|
||||
"_all" : {}
|
||||
}
|
||||
}
|
||||
}
|
||||
--------------------------------------------------
|
||||
|
||||
Using the fast vector highlighter there can be more tags, and the "importance"
|
||||
is ordered.
|
||||
|
||||
[source,js]
|
||||
--------------------------------------------------
|
||||
{
|
||||
|
@ -75,9 +147,8 @@ for example:
|
|||
}
|
||||
--------------------------------------------------
|
||||
|
||||
There can be a single tag or more, and the "importance" is ordered.
|
||||
There are also built in "tag" schemas, with currently a single schema
|
||||
called `styled` with `pre_tags` of:
|
||||
called `styled` with the following `pre_tags`:
|
||||
|
||||
[source,js]
|
||||
--------------------------------------------------
|
||||
|
@ -87,7 +158,7 @@ called `styled` with `pre_tags` of:
|
|||
<em class="hlt10">
|
||||
--------------------------------------------------
|
||||
|
||||
And post tag of `</em>`. If you think of more nice to have built in tag
|
||||
and `</em>` as `post_tags`. If you think of more nice to have built in tag
|
||||
schemas, just send an email to the mailing list or open an issue. Here
|
||||
is an example of switching tag schemas:
|
||||
|
||||
|
@ -104,6 +175,9 @@ is an example of switching tag schemas:
|
|||
}
|
||||
--------------------------------------------------
|
||||
|
||||
|
||||
==== Encoder
|
||||
|
||||
An `encoder` parameter can be used to define how highlighted text will
|
||||
be encoded. It can be either `default` (no encoding) or `html` (will
|
||||
escape html, if you use html highlighting tags).
|
||||
|
@ -112,7 +186,8 @@ escape html, if you use html highlighting tags).
|
|||
|
||||
Each field highlighted can control the size of the highlighted fragment
|
||||
in characters (defaults to `100`), and the maximum number of fragments
|
||||
to return (defaults to `5`). For example:
|
||||
to return (defaults to `5`).
|
||||
For example:
|
||||
|
||||
[source,js]
|
||||
--------------------------------------------------
|
||||
|
@ -126,8 +201,11 @@ to return (defaults to `5`). For example:
|
|||
}
|
||||
--------------------------------------------------
|
||||
|
||||
On top of this it is possible to specify that highlighted fragments are
|
||||
order by score:
|
||||
The `fragment_size` is ignored when using the postings highlighter, as it
|
||||
outputs sentences regardless of their length.
|
||||
|
||||
On top of this it is possible to specify that highlighted fragments need
|
||||
to be sorted by score:
|
||||
|
||||
[source,js]
|
||||
--------------------------------------------------
|
||||
|
@ -168,7 +246,10 @@ In the case where there is no matching fragment to highlight, the default is
|
|||
to not return anything. Instead, we can return a snippet of text from the
|
||||
beginning of the field by setting `no_match_size` (default `0`) to the length
|
||||
of the text that you want returned. The actual length may be shorter than
|
||||
specified as it tries to break on a word boundary.
|
||||
specified as it tries to break on a word boundary. When using the postings
|
||||
highlighter it is not possible to control the actual size of the snippet,
|
||||
therefore the first sentence gets returned whenever `no_match_size` is
|
||||
greater than `0`.
|
||||
|
||||
[source,js]
|
||||
--------------------------------------------------
|
||||
|
@ -256,9 +337,11 @@ query and the rescore query in `highlight_query`.
|
|||
}
|
||||
--------------------------------------------------
|
||||
|
||||
Note the score of text fragment in this case is calculated by Lucene
|
||||
highlighting framework. For implementation details you can check
|
||||
`ScoreOrderFragmentsBuilder.java` class.
|
||||
Note that the score of text fragment in this case is calculated by the Lucene
|
||||
highlighting framework. For implementation details you can check the
|
||||
`ScoreOrderFragmentsBuilder.java` class. On the other hand when using the
|
||||
postings highlighter the fragments are scored using, as mentioned above,
|
||||
the BM25 algorithm.
|
||||
|
||||
[[highlighting-settings]]
|
||||
==== Global Settings
|
||||
|
@ -295,7 +378,7 @@ matches specifically on them.
|
|||
[[boundary-characters]]
|
||||
==== Boundary Characters
|
||||
|
||||
When highlighting a field that is mapped with term vectors,
|
||||
When highlighting a field using the fast vector highlighter,
|
||||
`boundary_chars` can be configured to define what constitutes a boundary
|
||||
for highlighting. It's a single string with each boundary character
|
||||
defined in it. It defaults to `.,!? \t\n`.
|
||||
|
|
|
@ -0,0 +1,78 @@
|
|||
/*
|
||||
* Licensed to ElasticSearch and Shay Banon under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. ElasticSearch 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.postingshighlight;
|
||||
|
||||
import org.apache.lucene.search.highlight.Encoder;
|
||||
import org.elasticsearch.search.highlight.HighlightUtils;
|
||||
|
||||
/**
|
||||
Custom passage formatter that allows us to:
|
||||
1) extract different snippets (instead of a single big string) together with their scores ({@link Snippet})
|
||||
2) use the {@link Encoder} implementations that are already used with the other highlighters
|
||||
*/
|
||||
public class CustomPassageFormatter extends XPassageFormatter {
|
||||
|
||||
private final String preTag;
|
||||
private final String postTag;
|
||||
private final Encoder encoder;
|
||||
|
||||
public CustomPassageFormatter(String preTag, String postTag, Encoder encoder) {
|
||||
this.preTag = preTag;
|
||||
this.postTag = postTag;
|
||||
this.encoder = encoder;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Snippet[] format(Passage[] passages, String content) {
|
||||
Snippet[] snippets = new Snippet[passages.length];
|
||||
int pos;
|
||||
for (int j = 0; j < passages.length; j++) {
|
||||
Passage passage = passages[j];
|
||||
StringBuilder sb = new StringBuilder();
|
||||
pos = passage.startOffset;
|
||||
for (int i = 0; i < passage.numMatches; i++) {
|
||||
int start = passage.matchStarts[i];
|
||||
int end = passage.matchEnds[i];
|
||||
// its possible to have overlapping terms
|
||||
if (start > pos) {
|
||||
append(sb, content, pos, start);
|
||||
}
|
||||
if (end > pos) {
|
||||
sb.append(preTag);
|
||||
append(sb, content, Math.max(pos, start), end);
|
||||
sb.append(postTag);
|
||||
pos = end;
|
||||
}
|
||||
}
|
||||
// its possible a "term" from the analyzer could span a sentence boundary.
|
||||
append(sb, content, pos, Math.max(pos, passage.endOffset));
|
||||
//we remove the paragraph separator if present at the end of the snippet (we used it as separator between values)
|
||||
if (sb.charAt(sb.length() - 1) == HighlightUtils.PARAGRAPH_SEPARATOR) {
|
||||
sb.deleteCharAt(sb.length() - 1);
|
||||
}
|
||||
//and we trim the snippets too
|
||||
snippets[j] = new Snippet(sb.toString().trim(), passage.score, passage.numMatches > 0);
|
||||
}
|
||||
return snippets;
|
||||
}
|
||||
|
||||
protected void append(StringBuilder dest, String content, int start, int end) {
|
||||
dest.append(encoder.encodeText(content.substring(start, end)));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,187 @@
|
|||
/*
|
||||
* Licensed to ElasticSearch and Shay Banon under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. ElasticSearch 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.postingshighlight;
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexReaderContext;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.search.highlight.HighlightUtils;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.text.BreakIterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Subclass of the {@link XPostingsHighlighter} that works for a single field in a single document.
|
||||
* It receives the field values as input and it performs discrete highlighting on each single value
|
||||
* calling the highlightDoc method multiple times.
|
||||
* It allows to pass in the query terms to avoid calling extract terms multiple times.
|
||||
*
|
||||
* The use that we make of the postings highlighter is not optimal. It would be much better to
|
||||
* highlight multiple docs in a single call, as we actually lose its sequential IO. But that would require:
|
||||
* 1) to make our fork more complex and harder to maintain to perform discrete highlighting (needed to return
|
||||
* a different snippet per value when number_of_fragments=0 and the field has multiple values)
|
||||
* 2) refactoring of the elasticsearch highlight api which currently works per hit
|
||||
*
|
||||
*/
|
||||
public final class CustomPostingsHighlighter extends XPostingsHighlighter {
|
||||
|
||||
private static final Snippet[] EMPTY_SNIPPET = new Snippet[0];
|
||||
private static final Passage[] EMPTY_PASSAGE = new Passage[0];
|
||||
|
||||
private final CustomPassageFormatter passageFormatter;
|
||||
private final int noMatchSize;
|
||||
private final int totalContentLength;
|
||||
private final String[] fieldValues;
|
||||
private final int[] fieldValuesOffsets;
|
||||
private int currentValueIndex = 0;
|
||||
|
||||
private BreakIterator breakIterator;
|
||||
|
||||
public CustomPostingsHighlighter(CustomPassageFormatter passageFormatter, List<Object> fieldValues, boolean mergeValues, int maxLength, int noMatchSize) {
|
||||
super(maxLength);
|
||||
this.passageFormatter = passageFormatter;
|
||||
this.noMatchSize = noMatchSize;
|
||||
|
||||
if (mergeValues) {
|
||||
String rawValue = Strings.collectionToDelimitedString(fieldValues, String.valueOf(getMultiValuedSeparator("")));
|
||||
String fieldValue = rawValue.substring(0, Math.min(rawValue.length(), maxLength));
|
||||
this.fieldValues = new String[]{fieldValue};
|
||||
this.fieldValuesOffsets = new int[]{0};
|
||||
this.totalContentLength = fieldValue.length();
|
||||
} else {
|
||||
this.fieldValues = new String[fieldValues.size()];
|
||||
this.fieldValuesOffsets = new int[fieldValues.size()];
|
||||
int contentLength = 0;
|
||||
int offset = 0;
|
||||
int previousLength = -1;
|
||||
for (int i = 0; i < fieldValues.size(); i++) {
|
||||
String rawValue = fieldValues.get(i).toString();
|
||||
String fieldValue = rawValue.substring(0, Math.min(rawValue.length(), maxLength));
|
||||
this.fieldValues[i] = fieldValue;
|
||||
contentLength += fieldValue.length();
|
||||
offset += previousLength + 1;
|
||||
this.fieldValuesOffsets[i] = offset;
|
||||
previousLength = fieldValue.length();
|
||||
}
|
||||
this.totalContentLength = contentLength;
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
Our own api to highlight a single document field, passing in the query terms, and get back our own Snippet object
|
||||
*/
|
||||
public Snippet[] highlightDoc(String field, BytesRef[] terms, IndexSearcher searcher, int docId, int maxPassages) throws IOException {
|
||||
IndexReader reader = searcher.getIndexReader();
|
||||
IndexReaderContext readerContext = reader.getContext();
|
||||
List<AtomicReaderContext> leaves = readerContext.leaves();
|
||||
|
||||
String[] contents = new String[]{loadCurrentFieldValue()};
|
||||
Map<Integer, Object> snippetsMap = highlightField(field, contents, getBreakIterator(field), terms, new int[]{docId}, leaves, maxPassages);
|
||||
|
||||
//increment the current value index so that next time we'll highlight the next value if available
|
||||
currentValueIndex++;
|
||||
|
||||
Object snippetObject = snippetsMap.get(docId);
|
||||
if (snippetObject != null && snippetObject instanceof Snippet[]) {
|
||||
return (Snippet[]) snippetObject;
|
||||
}
|
||||
return EMPTY_SNIPPET;
|
||||
}
|
||||
|
||||
/*
|
||||
Method provided through our own fork: allows to do proper scoring when doing per value discrete highlighting.
|
||||
Used to provide the total length of the field (all values) for proper scoring.
|
||||
*/
|
||||
@Override
|
||||
protected int getContentLength(String field, int docId) {
|
||||
return totalContentLength;
|
||||
}
|
||||
|
||||
/*
|
||||
Method provided through our own fork: allows to perform proper per value discrete highlighting.
|
||||
Used to provide the offset for the current value.
|
||||
*/
|
||||
@Override
|
||||
protected int getOffsetForCurrentValue(String field, int docId) {
|
||||
if (currentValueIndex < fieldValuesOffsets.length) {
|
||||
return fieldValuesOffsets[currentValueIndex];
|
||||
}
|
||||
throw new IllegalArgumentException("No more values offsets to return");
|
||||
}
|
||||
|
||||
public void setBreakIterator(BreakIterator breakIterator) {
|
||||
this.breakIterator = breakIterator;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XPassageFormatter getFormatter(String field) {
|
||||
return passageFormatter;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected BreakIterator getBreakIterator(String field) {
|
||||
if (breakIterator == null) {
|
||||
return super.getBreakIterator(field);
|
||||
}
|
||||
return breakIterator;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected char getMultiValuedSeparator(String field) {
|
||||
//U+2029 PARAGRAPH SEPARATOR (PS): each value holds a discrete passage for highlighting
|
||||
return HighlightUtils.PARAGRAPH_SEPARATOR;
|
||||
}
|
||||
|
||||
/*
|
||||
By default the postings highlighter returns non highlighted snippet when there are no matches.
|
||||
We want to return no snippets by default, unless no_match_size is greater than 0
|
||||
*/
|
||||
@Override
|
||||
protected Passage[] getEmptyHighlight(String fieldName, BreakIterator bi, int maxPassages) {
|
||||
if (noMatchSize > 0) {
|
||||
//we want to return the first sentence of the first snippet only
|
||||
return super.getEmptyHighlight(fieldName, bi, 1);
|
||||
}
|
||||
return EMPTY_PASSAGE;
|
||||
}
|
||||
|
||||
/*
|
||||
Not needed since we call our own loadCurrentFieldValue explicitly, but we override it anyway for consistency.
|
||||
*/
|
||||
@Override
|
||||
protected String[][] loadFieldValues(IndexSearcher searcher, String[] fields, int[] docids, int maxLength) throws IOException {
|
||||
return new String[][]{new String[]{loadCurrentFieldValue()}};
|
||||
}
|
||||
|
||||
/*
|
||||
Our own method that returns the field values, which relies on the content that was provided when creating the highlighter.
|
||||
Supports per value discrete highlighting calling the highlightDoc method multiple times, one per value.
|
||||
*/
|
||||
protected String loadCurrentFieldValue() {
|
||||
if (currentValueIndex < fieldValues.length) {
|
||||
return fieldValues[currentValueIndex];
|
||||
}
|
||||
throw new IllegalArgumentException("No more values to return");
|
||||
}
|
||||
}
|
|
@ -0,0 +1,50 @@
|
|||
/*
|
||||
* Licensed to ElasticSearch and Shay Banon under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. ElasticSearch 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.postingshighlight;
|
||||
|
||||
/**
|
||||
* Represents a scored highlighted snippet.
|
||||
* It's our own arbitrary object that we get back from the postings highlighter when highlighting a document.
|
||||
* Every snippet contains its formatted text and its score.
|
||||
* The score is needed since we highlight every single value separately and we might want to return snippets sorted by score.
|
||||
*/
|
||||
public class Snippet {
|
||||
|
||||
private final String text;
|
||||
private final float score;
|
||||
private final boolean isHighlighted;
|
||||
|
||||
public Snippet(String text, float score, boolean isHighlighted) {
|
||||
this.text = text;
|
||||
this.score = score;
|
||||
this.isHighlighted = isHighlighted;
|
||||
}
|
||||
|
||||
public String getText() {
|
||||
return text;
|
||||
}
|
||||
|
||||
public float getScore() {
|
||||
return score;
|
||||
}
|
||||
|
||||
public boolean isHighlighted() {
|
||||
return isHighlighted;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,147 @@
|
|||
package org.apache.lucene.search.postingshighlight;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.elasticsearch.Version;
|
||||
|
||||
/**
|
||||
* Creates a formatted snippet from the top passages.
|
||||
* <p>
|
||||
* The default implementation marks the query terms as bold, and places
|
||||
* ellipses between unconnected passages.
|
||||
*/
|
||||
//LUCENE MONITOR - REMOVE ME WHEN LUCENE 4.6 IS OUT
|
||||
//Applied LUCENE-4906 to be able to return arbitrary objects
|
||||
public class XDefaultPassageFormatter extends XPassageFormatter {
|
||||
|
||||
static {
|
||||
assert Version.CURRENT.luceneVersion.compareTo(org.apache.lucene.util.Version.LUCENE_45) == 0 : "Remove XDefaultPassageFormatter once 4.6 is out";
|
||||
}
|
||||
|
||||
/** text that will appear before highlighted terms */
|
||||
protected final String preTag;
|
||||
/** text that will appear after highlighted terms */
|
||||
protected final String postTag;
|
||||
/** text that will appear between two unconnected passages */
|
||||
protected final String ellipsis;
|
||||
/** true if we should escape for html */
|
||||
protected final boolean escape;
|
||||
|
||||
/**
|
||||
* Creates a new DefaultPassageFormatter with the default tags.
|
||||
*/
|
||||
public XDefaultPassageFormatter() {
|
||||
this("<b>", "</b>", "... ", false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new DefaultPassageFormatter with custom tags.
|
||||
* @param preTag text which should appear before a highlighted term.
|
||||
* @param postTag text which should appear after a highlighted term.
|
||||
* @param ellipsis text which should be used to connect two unconnected passages.
|
||||
* @param escape true if text should be html-escaped
|
||||
*/
|
||||
public XDefaultPassageFormatter(String preTag, String postTag, String ellipsis, boolean escape) {
|
||||
if (preTag == null || postTag == null || ellipsis == null) {
|
||||
throw new NullPointerException();
|
||||
}
|
||||
this.preTag = preTag;
|
||||
this.postTag = postTag;
|
||||
this.ellipsis = ellipsis;
|
||||
this.escape = escape;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String format(Passage passages[], String content) {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
int pos = 0;
|
||||
for (Passage passage : passages) {
|
||||
// don't add ellipsis if its the first one, or if its connected.
|
||||
if (passage.startOffset > pos && pos > 0) {
|
||||
sb.append(ellipsis);
|
||||
}
|
||||
pos = passage.startOffset;
|
||||
for (int i = 0; i < passage.numMatches; i++) {
|
||||
int start = passage.matchStarts[i];
|
||||
int end = passage.matchEnds[i];
|
||||
// its possible to have overlapping terms
|
||||
if (start > pos) {
|
||||
append(sb, content, pos, start);
|
||||
}
|
||||
if (end > pos) {
|
||||
sb.append(preTag);
|
||||
append(sb, content, Math.max(pos, start), end);
|
||||
sb.append(postTag);
|
||||
pos = end;
|
||||
}
|
||||
}
|
||||
// its possible a "term" from the analyzer could span a sentence boundary.
|
||||
append(sb, content, pos, Math.max(pos, passage.endOffset));
|
||||
pos = passage.endOffset;
|
||||
}
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Appends original text to the response.
|
||||
* @param dest resulting text, possibly transformed or encoded
|
||||
* @param content original text content
|
||||
* @param start index of the first character in content
|
||||
* @param end index of the character following the last character in content
|
||||
*/
|
||||
protected void append(StringBuilder dest, String content, int start, int end) {
|
||||
if (escape) {
|
||||
// note: these are the rules from owasp.org
|
||||
for (int i = start; i < end; i++) {
|
||||
char ch = content.charAt(i);
|
||||
switch(ch) {
|
||||
case '&':
|
||||
dest.append("&");
|
||||
break;
|
||||
case '<':
|
||||
dest.append("<");
|
||||
break;
|
||||
case '>':
|
||||
dest.append(">");
|
||||
break;
|
||||
case '"':
|
||||
dest.append(""");
|
||||
break;
|
||||
case '\'':
|
||||
dest.append("'");
|
||||
break;
|
||||
case '/':
|
||||
dest.append("/");
|
||||
break;
|
||||
default:
|
||||
if (ch >= 0x30 && ch <= 0x39 || ch >= 0x41 && ch <= 0x5A || ch >= 0x61 && ch <= 0x7A) {
|
||||
dest.append(ch);
|
||||
} else if (ch < 0xff) {
|
||||
dest.append("&#");
|
||||
dest.append((int)ch);
|
||||
dest.append(";");
|
||||
} else {
|
||||
dest.append(ch);
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
dest.append(content, start, end);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,46 @@
|
|||
package org.apache.lucene.search.postingshighlight;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.elasticsearch.Version;
|
||||
|
||||
/**
|
||||
* Creates a formatted snippet from the top passages.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
//LUCENE MONITOR - REMOVE ME WHEN LUCENE 4.6 IS OUT
|
||||
//Applied LUCENE-4906 to be able to return arbitrary objects
|
||||
public abstract class XPassageFormatter {
|
||||
|
||||
static {
|
||||
assert Version.CURRENT.luceneVersion.compareTo(org.apache.lucene.util.Version.LUCENE_45) == 0 : "Remove XPassageFormatter once 4.6 is out";
|
||||
}
|
||||
|
||||
/**
|
||||
* Formats the top <code>passages</code> from <code>content</code>
|
||||
* into a human-readable text snippet.
|
||||
*
|
||||
* @param passages top-N passages for the field. Note these are sorted in
|
||||
* the order that they appear in the document for convenience.
|
||||
* @param content content for the field.
|
||||
* @return formatted highlight
|
||||
*/
|
||||
public abstract Object format(Passage passages[], String content);
|
||||
|
||||
}
|
|
@ -0,0 +1,777 @@
|
|||
/*
|
||||
* Licensed to ElasticSearch and Shay Banon under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. ElasticSearch 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.postingshighlight;
|
||||
|
||||
import org.apache.lucene.index.*;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.ScoreDoc;
|
||||
import org.apache.lucene.search.TopDocs;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.InPlaceMergeSorter;
|
||||
import org.apache.lucene.util.UnicodeUtil;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.text.BreakIterator;
|
||||
import java.util.*;
|
||||
|
||||
/*
|
||||
FORKED from Lucene 4.5 to be able to:
|
||||
1) support discrete highlighting for multiple values, so that we can return a different snippet per value when highlighting the whole text
|
||||
2) call the highlightField method directly from subclasses and provide the terms by ourselves
|
||||
3) Applied LUCENE-4906 to allow PassageFormatter to return arbitrary objects (LUCENE 4.6)
|
||||
|
||||
All our changes start with //BEGIN EDIT
|
||||
*/
|
||||
public class XPostingsHighlighter {
|
||||
|
||||
//BEGIN EDIT added method to override offset for current value (default 0)
|
||||
//we need this to perform discrete highlighting per field
|
||||
protected int getOffsetForCurrentValue(String field, int docId) {
|
||||
return 0;
|
||||
}
|
||||
//END EDIT
|
||||
|
||||
//BEGIN EDIT
|
||||
//we need this to fix scoring when highlighting every single value separately, since the score depends on the total length of the field (all values rather than only the current one)
|
||||
protected int getContentLength(String field, int docId) {
|
||||
return -1;
|
||||
}
|
||||
//END EDIT
|
||||
|
||||
|
||||
// TODO: maybe allow re-analysis for tiny fields? currently we require offsets,
|
||||
// but if the analyzer is really fast and the field is tiny, this might really be
|
||||
// unnecessary.
|
||||
|
||||
/** for rewriting: we don't want slow processing from MTQs */
|
||||
private static final IndexReader EMPTY_INDEXREADER = new MultiReader();
|
||||
|
||||
/** Default maximum content size to process. Typically snippets
|
||||
* closer to the beginning of the document better summarize its content */
|
||||
public static final int DEFAULT_MAX_LENGTH = 10000;
|
||||
|
||||
private final int maxLength;
|
||||
|
||||
/** Set the first time {@link #getFormatter} is called,
|
||||
* and then reused. */
|
||||
private XPassageFormatter defaultFormatter;
|
||||
|
||||
/** Set the first time {@link #getScorer} is called,
|
||||
* and then reused. */
|
||||
private PassageScorer defaultScorer;
|
||||
|
||||
/**
|
||||
* Creates a new highlighter with default parameters.
|
||||
*/
|
||||
public XPostingsHighlighter() {
|
||||
this(DEFAULT_MAX_LENGTH);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new highlighter, specifying maximum content length.
|
||||
* @param maxLength maximum content size to process.
|
||||
* @throws IllegalArgumentException if <code>maxLength</code> is negative or <code>Integer.MAX_VALUE</code>
|
||||
*/
|
||||
public XPostingsHighlighter(int maxLength) {
|
||||
if (maxLength < 0 || maxLength == Integer.MAX_VALUE) {
|
||||
// two reasons: no overflow problems in BreakIterator.preceding(offset+1),
|
||||
// our sentinel in the offsets queue uses this value to terminate.
|
||||
throw new IllegalArgumentException("maxLength must be < Integer.MAX_VALUE");
|
||||
}
|
||||
this.maxLength = maxLength;
|
||||
}
|
||||
|
||||
/** Returns the {@link java.text.BreakIterator} to use for
|
||||
* dividing text into passages. This returns
|
||||
* {@link java.text.BreakIterator#getSentenceInstance(java.util.Locale)} by default;
|
||||
* subclasses can override to customize. */
|
||||
protected BreakIterator getBreakIterator(String field) {
|
||||
return BreakIterator.getSentenceInstance(Locale.ROOT);
|
||||
}
|
||||
|
||||
/** Returns the {@link PassageFormatter} to use for
|
||||
* formatting passages into highlighted snippets. This
|
||||
* returns a new {@code PassageFormatter} by default;
|
||||
* subclasses can override to customize. */
|
||||
protected XPassageFormatter getFormatter(String field) {
|
||||
if (defaultFormatter == null) {
|
||||
defaultFormatter = new XDefaultPassageFormatter();
|
||||
}
|
||||
return defaultFormatter;
|
||||
}
|
||||
|
||||
/** Returns the {@link PassageScorer} to use for
|
||||
* ranking passages. This
|
||||
* returns a new {@code PassageScorer} by default;
|
||||
* subclasses can override to customize. */
|
||||
protected PassageScorer getScorer(String field) {
|
||||
if (defaultScorer == null) {
|
||||
defaultScorer = new PassageScorer();
|
||||
}
|
||||
return defaultScorer;
|
||||
}
|
||||
|
||||
/**
|
||||
* Highlights the top passages from a single field.
|
||||
*
|
||||
* @param field field name to highlight.
|
||||
* Must have a stored string value and also be indexed with offsets.
|
||||
* @param query query to highlight.
|
||||
* @param searcher searcher that was previously used to execute the query.
|
||||
* @param topDocs TopDocs containing the summary result documents to highlight.
|
||||
* @return Array of formatted snippets corresponding to the documents in <code>topDocs</code>.
|
||||
* If no highlights were found for a document, the
|
||||
* first sentence for the field will be returned.
|
||||
* @throws java.io.IOException if an I/O error occurred during processing
|
||||
* @throws IllegalArgumentException if <code>field</code> was indexed without
|
||||
* {@link org.apache.lucene.index.FieldInfo.IndexOptions#DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS}
|
||||
*/
|
||||
public String[] highlight(String field, Query query, IndexSearcher searcher, TopDocs topDocs) throws IOException {
|
||||
return highlight(field, query, searcher, topDocs, 1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Highlights the top-N passages from a single field.
|
||||
*
|
||||
* @param field field name to highlight.
|
||||
* Must have a stored string value and also be indexed with offsets.
|
||||
* @param query query to highlight.
|
||||
* @param searcher searcher that was previously used to execute the query.
|
||||
* @param topDocs TopDocs containing the summary result documents to highlight.
|
||||
* @param maxPassages The maximum number of top-N ranked passages used to
|
||||
* form the highlighted snippets.
|
||||
* @return Array of formatted snippets corresponding to the documents in <code>topDocs</code>.
|
||||
* If no highlights were found for a document, the
|
||||
* first {@code maxPassages} sentences from the
|
||||
* field will be returned.
|
||||
* @throws IOException if an I/O error occurred during processing
|
||||
* @throws IllegalArgumentException if <code>field</code> was indexed without
|
||||
* {@link org.apache.lucene.index.FieldInfo.IndexOptions#DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS}
|
||||
*/
|
||||
public String[] highlight(String field, Query query, IndexSearcher searcher, TopDocs topDocs, int maxPassages) throws IOException {
|
||||
Map<String,String[]> res = highlightFields(new String[] { field }, query, searcher, topDocs, new int[] { maxPassages });
|
||||
return res.get(field);
|
||||
}
|
||||
|
||||
/**
|
||||
* Highlights the top passages from multiple fields.
|
||||
* <p>
|
||||
* Conceptually, this behaves as a more efficient form of:
|
||||
* <pre class="prettyprint">
|
||||
* Map m = new HashMap();
|
||||
* for (String field : fields) {
|
||||
* m.put(field, highlight(field, query, searcher, topDocs));
|
||||
* }
|
||||
* return m;
|
||||
* </pre>
|
||||
*
|
||||
* @param fields field names to highlight.
|
||||
* Must have a stored string value and also be indexed with offsets.
|
||||
* @param query query to highlight.
|
||||
* @param searcher searcher that was previously used to execute the query.
|
||||
* @param topDocs TopDocs containing the summary result documents to highlight.
|
||||
* @return Map keyed on field name, containing the array of formatted snippets
|
||||
* corresponding to the documents in <code>topDocs</code>.
|
||||
* If no highlights were found for a document, the
|
||||
* first sentence from the field will be returned.
|
||||
* @throws IOException if an I/O error occurred during processing
|
||||
* @throws IllegalArgumentException if <code>field</code> was indexed without
|
||||
* {@link org.apache.lucene.index.FieldInfo.IndexOptions#DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS}
|
||||
*/
|
||||
public Map<String,String[]> highlightFields(String fields[], Query query, IndexSearcher searcher, TopDocs topDocs) throws IOException {
|
||||
int maxPassages[] = new int[fields.length];
|
||||
Arrays.fill(maxPassages, 1);
|
||||
return highlightFields(fields, query, searcher, topDocs, maxPassages);
|
||||
}
|
||||
|
||||
/**
|
||||
* Highlights the top-N passages from multiple fields.
|
||||
* <p>
|
||||
* Conceptually, this behaves as a more efficient form of:
|
||||
* <pre class="prettyprint">
|
||||
* Map m = new HashMap();
|
||||
* for (String field : fields) {
|
||||
* m.put(field, highlight(field, query, searcher, topDocs, maxPassages));
|
||||
* }
|
||||
* return m;
|
||||
* </pre>
|
||||
*
|
||||
* @param fields field names to highlight.
|
||||
* Must have a stored string value and also be indexed with offsets.
|
||||
* @param query query to highlight.
|
||||
* @param searcher searcher that was previously used to execute the query.
|
||||
* @param topDocs TopDocs containing the summary result documents to highlight.
|
||||
* @param maxPassages The maximum number of top-N ranked passages per-field used to
|
||||
* form the highlighted snippets.
|
||||
* @return Map keyed on field name, containing the array of formatted snippets
|
||||
* corresponding to the documents in <code>topDocs</code>.
|
||||
* If no highlights were found for a document, the
|
||||
* first {@code maxPassages} sentences from the
|
||||
* field will be returned.
|
||||
* @throws IOException if an I/O error occurred during processing
|
||||
* @throws IllegalArgumentException if <code>field</code> was indexed without
|
||||
* {@link org.apache.lucene.index.FieldInfo.IndexOptions#DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS}
|
||||
*/
|
||||
public Map<String,String[]> highlightFields(String fields[], Query query, IndexSearcher searcher, TopDocs topDocs, int maxPassages[]) throws IOException {
|
||||
final ScoreDoc scoreDocs[] = topDocs.scoreDocs;
|
||||
int docids[] = new int[scoreDocs.length];
|
||||
for (int i = 0; i < docids.length; i++) {
|
||||
docids[i] = scoreDocs[i].doc;
|
||||
}
|
||||
|
||||
return highlightFields(fields, query, searcher, docids, maxPassages);
|
||||
}
|
||||
|
||||
/**
|
||||
* Highlights the top-N passages from multiple fields,
|
||||
* for the provided int[] docids.
|
||||
*
|
||||
* @param fieldsIn field names to highlight.
|
||||
* Must have a stored string value and also be indexed with offsets.
|
||||
* @param query query to highlight.
|
||||
* @param searcher searcher that was previously used to execute the query.
|
||||
* @param docidsIn containing the document IDs to highlight.
|
||||
* @param maxPassagesIn The maximum number of top-N ranked passages per-field used to
|
||||
* form the highlighted snippets.
|
||||
* @return Map keyed on field name, containing the array of formatted snippets
|
||||
* corresponding to the documents in <code>topDocs</code>.
|
||||
* If no highlights were found for a document, the
|
||||
* first {@code maxPassages} from the field will
|
||||
* be returned.
|
||||
* @throws IOException if an I/O error occurred during processing
|
||||
* @throws IllegalArgumentException if <code>field</code> was indexed without
|
||||
* {@link org.apache.lucene.index.FieldInfo.IndexOptions#DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS}
|
||||
*/
|
||||
public Map<String,String[]> highlightFields(String fieldsIn[], Query query, IndexSearcher searcher, int[] docidsIn, int maxPassagesIn[]) throws IOException {
|
||||
Map<String,String[]> snippets = new HashMap<String,String[]>();
|
||||
for(Map.Entry<String,Object[]> ent : highlightFieldsAsObjects(fieldsIn, query, searcher, docidsIn, maxPassagesIn).entrySet()) {
|
||||
Object[] snippetObjects = ent.getValue();
|
||||
String[] snippetStrings = new String[snippetObjects.length];
|
||||
snippets.put(ent.getKey(), snippetStrings);
|
||||
for(int i=0;i<snippetObjects.length;i++) {
|
||||
Object snippet = snippetObjects[i];
|
||||
if (snippet != null) {
|
||||
snippetStrings[i] = snippet.toString();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return snippets;
|
||||
}
|
||||
|
||||
public Map<String,Object[]> highlightFieldsAsObjects(String fieldsIn[], Query query, IndexSearcher searcher, int[] docidsIn, int maxPassagesIn[]) throws IOException {
|
||||
if (fieldsIn.length < 1) {
|
||||
throw new IllegalArgumentException("fieldsIn must not be empty");
|
||||
}
|
||||
if (fieldsIn.length != maxPassagesIn.length) {
|
||||
throw new IllegalArgumentException("invalid number of maxPassagesIn");
|
||||
}
|
||||
final IndexReader reader = searcher.getIndexReader();
|
||||
query = rewrite(query);
|
||||
SortedSet<Term> queryTerms = new TreeSet<Term>();
|
||||
query.extractTerms(queryTerms);
|
||||
|
||||
IndexReaderContext readerContext = reader.getContext();
|
||||
List<AtomicReaderContext> leaves = readerContext.leaves();
|
||||
|
||||
// Make our own copies because we sort in-place:
|
||||
int[] docids = new int[docidsIn.length];
|
||||
System.arraycopy(docidsIn, 0, docids, 0, docidsIn.length);
|
||||
final String fields[] = new String[fieldsIn.length];
|
||||
System.arraycopy(fieldsIn, 0, fields, 0, fieldsIn.length);
|
||||
final int maxPassages[] = new int[maxPassagesIn.length];
|
||||
System.arraycopy(maxPassagesIn, 0, maxPassages, 0, maxPassagesIn.length);
|
||||
|
||||
// sort for sequential io
|
||||
Arrays.sort(docids);
|
||||
new InPlaceMergeSorter() {
|
||||
|
||||
@Override
|
||||
protected void swap(int i, int j) {
|
||||
String tmp = fields[i];
|
||||
fields[i] = fields[j];
|
||||
fields[j] = tmp;
|
||||
int tmp2 = maxPassages[i];
|
||||
maxPassages[i] = maxPassages[j];
|
||||
maxPassages[j] = tmp2;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int compare(int i, int j) {
|
||||
return fields[i].compareTo(fields[j]);
|
||||
}
|
||||
|
||||
}.sort(0, fields.length);
|
||||
|
||||
// pull stored data:
|
||||
String[][] contents = loadFieldValues(searcher, fields, docids, maxLength);
|
||||
|
||||
Map<String,Object[]> highlights = new HashMap<String,Object[]>();
|
||||
for (int i = 0; i < fields.length; i++) {
|
||||
String field = fields[i];
|
||||
int numPassages = maxPassages[i];
|
||||
|
||||
Term floor = new Term(field, "");
|
||||
Term ceiling = new Term(field, UnicodeUtil.BIG_TERM);
|
||||
SortedSet<Term> fieldTerms = queryTerms.subSet(floor, ceiling);
|
||||
// TODO: should we have some reasonable defaults for term pruning? (e.g. stopwords)
|
||||
|
||||
// Strip off the redundant field:
|
||||
BytesRef terms[] = new BytesRef[fieldTerms.size()];
|
||||
int termUpto = 0;
|
||||
for(Term term : fieldTerms) {
|
||||
terms[termUpto++] = term.bytes();
|
||||
}
|
||||
Map<Integer,Object> fieldHighlights = highlightField(field, contents[i], getBreakIterator(field), terms, docids, leaves, numPassages);
|
||||
|
||||
Object[] result = new Object[docids.length];
|
||||
for (int j = 0; j < docidsIn.length; j++) {
|
||||
result[j] = fieldHighlights.get(docidsIn[j]);
|
||||
}
|
||||
highlights.put(field, result);
|
||||
}
|
||||
return highlights;
|
||||
}
|
||||
|
||||
/** Loads the String values for each field X docID to be
|
||||
* highlighted. By default this loads from stored
|
||||
* fields, but a subclass can change the source. This
|
||||
* method should allocate the String[fields.length][docids.length]
|
||||
* and fill all values. The returned Strings must be
|
||||
* identical to what was indexed. */
|
||||
protected String[][] loadFieldValues(IndexSearcher searcher, String[] fields, int[] docids, int maxLength) throws IOException {
|
||||
String contents[][] = new String[fields.length][docids.length];
|
||||
char valueSeparators[] = new char[fields.length];
|
||||
for (int i = 0; i < fields.length; i++) {
|
||||
valueSeparators[i] = getMultiValuedSeparator(fields[i]);
|
||||
}
|
||||
LimitedStoredFieldVisitor visitor = new LimitedStoredFieldVisitor(fields, valueSeparators, maxLength);
|
||||
for (int i = 0; i < docids.length; i++) {
|
||||
searcher.doc(docids[i], visitor);
|
||||
for (int j = 0; j < fields.length; j++) {
|
||||
contents[j][i] = visitor.getValue(j);
|
||||
}
|
||||
visitor.reset();
|
||||
}
|
||||
return contents;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the logical separator between values for multi-valued fields.
|
||||
* The default value is a space character, which means passages can span across values,
|
||||
* but a subclass can override, for example with {@code U+2029 PARAGRAPH SEPARATOR (PS)}
|
||||
* if each value holds a discrete passage for highlighting.
|
||||
*/
|
||||
protected char getMultiValuedSeparator(String field) {
|
||||
return ' ';
|
||||
}
|
||||
|
||||
//BEGIN EDIT: made protected so that we can call from our subclass and pass in the terms by ourselves
|
||||
protected Map<Integer,Object> highlightField(String field, String contents[], BreakIterator bi, BytesRef terms[], int[] docids, List<AtomicReaderContext> leaves, int maxPassages) throws IOException {
|
||||
//private Map<Integer,Object> highlightField(String field, String contents[], BreakIterator bi, BytesRef terms[], int[] docids, List<AtomicReaderContext > leaves, int maxPassages) throws IOException {
|
||||
//END EDIT
|
||||
|
||||
Map<Integer,Object> highlights = new HashMap<Integer,Object>();
|
||||
|
||||
// reuse in the real sense... for docs in same segment we just advance our old enum
|
||||
DocsAndPositionsEnum postings[] = null;
|
||||
TermsEnum termsEnum = null;
|
||||
int lastLeaf = -1;
|
||||
|
||||
XPassageFormatter fieldFormatter = getFormatter(field);
|
||||
if (fieldFormatter == null) {
|
||||
throw new NullPointerException("PassageFormatter cannot be null");
|
||||
}
|
||||
|
||||
for (int i = 0; i < docids.length; i++) {
|
||||
String content = contents[i];
|
||||
if (content.length() == 0) {
|
||||
continue; // nothing to do
|
||||
}
|
||||
bi.setText(content);
|
||||
int doc = docids[i];
|
||||
int leaf = ReaderUtil.subIndex(doc, leaves);
|
||||
AtomicReaderContext subContext = leaves.get(leaf);
|
||||
AtomicReader r = subContext.reader();
|
||||
Terms t = r.terms(field);
|
||||
if (t == null) {
|
||||
continue; // nothing to do
|
||||
}
|
||||
if (leaf != lastLeaf) {
|
||||
termsEnum = t.iterator(null);
|
||||
postings = new DocsAndPositionsEnum[terms.length];
|
||||
}
|
||||
Passage passages[] = highlightDoc(field, terms, content.length(), bi, doc - subContext.docBase, termsEnum, postings, maxPassages);
|
||||
if (passages.length == 0) {
|
||||
passages = getEmptyHighlight(field, bi, maxPassages);
|
||||
}
|
||||
if (passages.length > 0) {
|
||||
// otherwise a null snippet (eg if field is missing
|
||||
// entirely from the doc)
|
||||
highlights.put(doc, fieldFormatter.format(passages, content));
|
||||
}
|
||||
lastLeaf = leaf;
|
||||
}
|
||||
|
||||
return highlights;
|
||||
}
|
||||
|
||||
// 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))
|
||||
private Passage[] highlightDoc(String field, BytesRef terms[], int contentLength, BreakIterator bi, int doc,
|
||||
TermsEnum termsEnum, DocsAndPositionsEnum[] postings, int n) throws IOException {
|
||||
|
||||
//BEGIN EDIT added call to method that returns the offset for the current value (discrete highlighting)
|
||||
int valueOffset = getOffsetForCurrentValue(field, doc);
|
||||
//END EDIT
|
||||
|
||||
PassageScorer scorer = getScorer(field);
|
||||
if (scorer == null) {
|
||||
throw new NullPointerException("PassageScorer cannot be null");
|
||||
}
|
||||
|
||||
|
||||
//BEGIN EDIT discrete highlighting
|
||||
// the scoring needs to be based on the length of the whole field (all values rather than only the current one)
|
||||
int totalContentLength = getContentLength(field, doc);
|
||||
if (totalContentLength == -1) {
|
||||
totalContentLength = contentLength;
|
||||
}
|
||||
//END EDIT
|
||||
|
||||
|
||||
PriorityQueue<OffsetsEnum> pq = new PriorityQueue<OffsetsEnum>();
|
||||
float weights[] = new float[terms.length];
|
||||
// initialize postings
|
||||
for (int i = 0; i < terms.length; i++) {
|
||||
DocsAndPositionsEnum de = postings[i];
|
||||
int pDoc;
|
||||
if (de == EMPTY) {
|
||||
continue;
|
||||
} else if (de == null) {
|
||||
postings[i] = EMPTY; // initially
|
||||
if (!termsEnum.seekExact(terms[i])) {
|
||||
continue; // term not found
|
||||
}
|
||||
de = postings[i] = termsEnum.docsAndPositions(null, null, DocsAndPositionsEnum.FLAG_OFFSETS);
|
||||
if (de == null) {
|
||||
// no positions available
|
||||
throw new IllegalArgumentException("field '" + field + "' was indexed without offsets, cannot highlight");
|
||||
}
|
||||
pDoc = de.advance(doc);
|
||||
} else {
|
||||
pDoc = de.docID();
|
||||
if (pDoc < doc) {
|
||||
pDoc = de.advance(doc);
|
||||
}
|
||||
}
|
||||
|
||||
if (doc == pDoc) {
|
||||
//BEGIN EDIT we take into account the length of the whole field (all values) to properly score the snippets
|
||||
weights[i] = scorer.weight(totalContentLength, de.freq());
|
||||
//weights[i] = scorer.weight(contentLength, de.freq());
|
||||
//END EDIT
|
||||
de.nextPosition();
|
||||
pq.add(new OffsetsEnum(de, i));
|
||||
}
|
||||
}
|
||||
|
||||
pq.add(new OffsetsEnum(EMPTY, Integer.MAX_VALUE)); // a sentinel for termination
|
||||
|
||||
PriorityQueue<Passage> passageQueue = new PriorityQueue<Passage>(n, new Comparator<Passage>() {
|
||||
@Override
|
||||
public int compare(Passage left, Passage right) {
|
||||
if (left.score < right.score) {
|
||||
return -1;
|
||||
} else if (left.score > right.score) {
|
||||
return 1;
|
||||
} else {
|
||||
return left.startOffset - right.startOffset;
|
||||
}
|
||||
}
|
||||
});
|
||||
Passage current = new Passage();
|
||||
|
||||
OffsetsEnum off;
|
||||
while ((off = pq.poll()) != null) {
|
||||
final DocsAndPositionsEnum dp = off.dp;
|
||||
|
||||
int start = dp.startOffset();
|
||||
if (start == -1) {
|
||||
throw new IllegalArgumentException("field '" + field + "' was indexed without offsets, cannot highlight");
|
||||
}
|
||||
int end = dp.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;
|
||||
}
|
||||
|
||||
|
||||
//BEGIN EDIT support for discrete highlighting (added block code)
|
||||
//switch to the first match in the current value if there is one
|
||||
boolean seenEnough = false;
|
||||
while (start < valueOffset) {
|
||||
if (off.pos == dp.freq()) {
|
||||
seenEnough = true;
|
||||
break;
|
||||
} else {
|
||||
off.pos++;
|
||||
dp.nextPosition();
|
||||
start = dp.startOffset();
|
||||
end = dp.endOffset();
|
||||
}
|
||||
}
|
||||
|
||||
//continue with next term if we've already seen the current one all the times it appears
|
||||
//that means that the current value doesn't hold matches for the current term
|
||||
if (seenEnough) {
|
||||
continue;
|
||||
}
|
||||
|
||||
//we now subtract the offset of the current value to both start and end
|
||||
start -= valueOffset;
|
||||
end -= valueOffset;
|
||||
//END EDIT
|
||||
|
||||
|
||||
if (start >= current.endOffset) {
|
||||
if (current.startOffset >= 0) {
|
||||
// finalize current
|
||||
//BEGIN EDIT we take into account the value offset when scoring the snippet based on its position
|
||||
current.score *= scorer.norm(current.startOffset + valueOffset);
|
||||
//current.score *= scorer.norm(current.startOffset);
|
||||
//END EDIT
|
||||
// new sentence: first add 'current' to queue
|
||||
if (passageQueue.size() == n && current.score < passageQueue.peek().score) {
|
||||
current.reset(); // can't compete, just reset it
|
||||
} else {
|
||||
passageQueue.offer(current);
|
||||
if (passageQueue.size() > n) {
|
||||
current = passageQueue.poll();
|
||||
current.reset();
|
||||
} else {
|
||||
current = new Passage();
|
||||
}
|
||||
}
|
||||
}
|
||||
// if we exceed limit, we are done
|
||||
if (start >= contentLength) {
|
||||
Passage passages[] = new Passage[passageQueue.size()];
|
||||
passageQueue.toArray(passages);
|
||||
for (Passage p : passages) {
|
||||
p.sort();
|
||||
}
|
||||
// sort in ascending order
|
||||
Arrays.sort(passages, new Comparator<Passage>() {
|
||||
@Override
|
||||
public int compare(Passage left, Passage right) {
|
||||
return left.startOffset - right.startOffset;
|
||||
}
|
||||
});
|
||||
return passages;
|
||||
}
|
||||
// advance breakiterator
|
||||
assert BreakIterator.DONE < 0;
|
||||
current.startOffset = Math.max(bi.preceding(start+1), 0);
|
||||
current.endOffset = Math.min(bi.next(), contentLength);
|
||||
}
|
||||
int tf = 0;
|
||||
while (true) {
|
||||
tf++;
|
||||
current.addMatch(start, end, terms[off.id]);
|
||||
if (off.pos == dp.freq()) {
|
||||
break; // removed from pq
|
||||
} else {
|
||||
off.pos++;
|
||||
dp.nextPosition();
|
||||
//BEGIN EDIT support for discrete highlighting
|
||||
start = dp.startOffset() - valueOffset;
|
||||
end = dp.endOffset() - valueOffset;
|
||||
//start = dp.startOffset();
|
||||
//end = dp.endOffset();
|
||||
//END EDIT
|
||||
}
|
||||
if (start >= current.endOffset || end > contentLength) {
|
||||
pq.offer(off);
|
||||
break;
|
||||
}
|
||||
}
|
||||
current.score += weights[off.id] * scorer.tf(tf, current.endOffset - current.startOffset);
|
||||
}
|
||||
|
||||
// Dead code but compiler disagrees:
|
||||
assert false;
|
||||
return null;
|
||||
}
|
||||
|
||||
/** Called to summarize a document when no hits were
|
||||
* found. By default this just returns the first
|
||||
* {@code maxPassages} sentences; subclasses can override
|
||||
* to customize. */
|
||||
protected Passage[] getEmptyHighlight(String fieldName, BreakIterator bi, int maxPassages) {
|
||||
// BreakIterator should be un-next'd:
|
||||
List<Passage> passages = new ArrayList<Passage>();
|
||||
int pos = bi.current();
|
||||
assert pos == 0;
|
||||
while (passages.size() < maxPassages) {
|
||||
int next = bi.next();
|
||||
if (next == BreakIterator.DONE) {
|
||||
break;
|
||||
}
|
||||
Passage passage = new Passage();
|
||||
passage.score = Float.NaN;
|
||||
passage.startOffset = pos;
|
||||
passage.endOffset = next;
|
||||
passages.add(passage);
|
||||
pos = next;
|
||||
}
|
||||
|
||||
return passages.toArray(new Passage[passages.size()]);
|
||||
}
|
||||
|
||||
private static class OffsetsEnum implements Comparable<OffsetsEnum> {
|
||||
DocsAndPositionsEnum dp;
|
||||
int pos;
|
||||
int id;
|
||||
|
||||
OffsetsEnum(DocsAndPositionsEnum dp, int id) throws IOException {
|
||||
this.dp = dp;
|
||||
this.id = id;
|
||||
this.pos = 1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(OffsetsEnum other) {
|
||||
try {
|
||||
int off = dp.startOffset();
|
||||
int otherOff = other.dp.startOffset();
|
||||
if (off == otherOff) {
|
||||
return id - other.id;
|
||||
} else {
|
||||
return Long.signum(((long)off) - otherOff);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static final DocsAndPositionsEnum EMPTY = new DocsAndPositionsEnum() {
|
||||
|
||||
@Override
|
||||
public int nextPosition() throws IOException { return 0; }
|
||||
|
||||
@Override
|
||||
public int startOffset() throws IOException { return Integer.MAX_VALUE; }
|
||||
|
||||
@Override
|
||||
public int endOffset() throws IOException { return Integer.MAX_VALUE; }
|
||||
|
||||
@Override
|
||||
public BytesRef getPayload() throws IOException { return null; }
|
||||
|
||||
@Override
|
||||
public int freq() throws IOException { return 0; }
|
||||
|
||||
@Override
|
||||
public int docID() { return NO_MORE_DOCS; }
|
||||
|
||||
@Override
|
||||
public int nextDoc() throws IOException { return NO_MORE_DOCS; }
|
||||
|
||||
@Override
|
||||
public int advance(int target) throws IOException { return NO_MORE_DOCS; }
|
||||
|
||||
@Override
|
||||
public long cost() { return 0; }
|
||||
};
|
||||
|
||||
/**
|
||||
* we rewrite against an empty indexreader: as we don't want things like
|
||||
* rangeQueries that don't summarize the document
|
||||
*/
|
||||
private static Query rewrite(Query original) throws IOException {
|
||||
Query query = original;
|
||||
for (Query rewrittenQuery = query.rewrite(EMPTY_INDEXREADER); rewrittenQuery != query;
|
||||
rewrittenQuery = query.rewrite(EMPTY_INDEXREADER)) {
|
||||
query = rewrittenQuery;
|
||||
}
|
||||
return query;
|
||||
}
|
||||
|
||||
private static class LimitedStoredFieldVisitor extends StoredFieldVisitor {
|
||||
private final String fields[];
|
||||
private final char valueSeparators[];
|
||||
private final int maxLength;
|
||||
private final StringBuilder builders[];
|
||||
private int currentField = -1;
|
||||
|
||||
public LimitedStoredFieldVisitor(String fields[], char valueSeparators[], int maxLength) {
|
||||
assert fields.length == valueSeparators.length;
|
||||
this.fields = fields;
|
||||
this.valueSeparators = valueSeparators;
|
||||
this.maxLength = maxLength;
|
||||
builders = new StringBuilder[fields.length];
|
||||
for (int i = 0; i < builders.length; i++) {
|
||||
builders[i] = new StringBuilder();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stringField(FieldInfo fieldInfo, String value) throws IOException {
|
||||
assert currentField >= 0;
|
||||
StringBuilder builder = builders[currentField];
|
||||
if (builder.length() > 0 && builder.length() < maxLength) {
|
||||
builder.append(valueSeparators[currentField]);
|
||||
}
|
||||
if (builder.length() + value.length() > maxLength) {
|
||||
builder.append(value, 0, maxLength - builder.length());
|
||||
} else {
|
||||
builder.append(value);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Status needsField(FieldInfo fieldInfo) throws IOException {
|
||||
currentField = Arrays.binarySearch(fields, fieldInfo.name);
|
||||
if (currentField < 0) {
|
||||
return Status.NO;
|
||||
} else if (builders[currentField].length() > maxLength) {
|
||||
return fields.length == 1 ? Status.STOP : Status.NO;
|
||||
}
|
||||
return Status.YES;
|
||||
}
|
||||
|
||||
String getValue(int i) {
|
||||
return builders[i].toString();
|
||||
}
|
||||
|
||||
void reset() {
|
||||
currentField = -1;
|
||||
for (int i = 0; i < fields.length; i++) {
|
||||
builders[i].setLength(0);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -19,9 +19,7 @@
|
|||
package org.elasticsearch.search.highlight;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
import org.apache.lucene.search.highlight.DefaultEncoder;
|
||||
import org.apache.lucene.search.highlight.Encoder;
|
||||
import org.apache.lucene.search.highlight.SimpleHTMLEncoder;
|
||||
import org.apache.lucene.search.vectorhighlight.*;
|
||||
import org.apache.lucene.search.vectorhighlight.FieldPhraseList.WeightedPhraseInfo;
|
||||
import org.elasticsearch.ElasticSearchIllegalArgumentException;
|
||||
|
@ -70,7 +68,7 @@ public class FastVectorHighlighter implements Highlighter {
|
|||
throw new ElasticSearchIllegalArgumentException("the field [" + field.field() + "] should be indexed with term vector with position offsets to be used with fast vector highlighter");
|
||||
}
|
||||
|
||||
Encoder encoder = field.encoder().equals("html") ? Encoders.HTML : Encoders.DEFAULT;
|
||||
Encoder encoder = field.encoder().equals("html") ? HighlightUtils.Encoders.HTML : HighlightUtils.Encoders.DEFAULT;
|
||||
|
||||
if (!hitContext.cache().containsKey(CACHE_KEY)) {
|
||||
hitContext.cache().put(CACHE_KEY, new HighlighterEntry());
|
||||
|
@ -185,9 +183,4 @@ public class FastVectorHighlighter implements Highlighter {
|
|||
public FieldQuery fieldMatchFieldQuery;
|
||||
public Map<FieldMapper, MapperHighlightEntry> mappers = Maps.newHashMap();
|
||||
}
|
||||
|
||||
private static class Encoders {
|
||||
public static Encoder DEFAULT = new DefaultEncoder();
|
||||
public static Encoder HTML = new SimpleHTMLEncoder();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -190,7 +190,7 @@ public class HighlightBuilder implements ToXContent {
|
|||
|
||||
/**
|
||||
* Set type of highlighter to use. Supported types
|
||||
* are <tt>highlighter</tt> and <tt>fast-vector-highlighter</tt>.
|
||||
* are <tt>highlighter</tt>, <tt>fast-vector-highlighter</tt> and <tt>postings-highlighter</tt>.
|
||||
*/
|
||||
public HighlightBuilder highlighterType(String highlighterType) {
|
||||
this.highlighterType = highlighterType;
|
||||
|
@ -420,7 +420,7 @@ public class HighlightBuilder implements ToXContent {
|
|||
|
||||
/**
|
||||
* Set type of highlighter to use. Supported types
|
||||
* are <tt>highlighter</tt> and <tt>fast-vector-highlighter</tt>.
|
||||
* are <tt>highlighter</tt>, <tt>fast-vector-highlighter</tt> nad <tt>postings-highlighter</tt>.
|
||||
* This overrides global settings set by {@link HighlightBuilder#highlighterType(String)}.
|
||||
*/
|
||||
public Field highlighterType(String highlighterType) {
|
||||
|
|
|
@ -34,6 +34,7 @@ public class HighlightModule extends AbstractModule {
|
|||
public HighlightModule() {
|
||||
registerHighlighter(FastVectorHighlighter.class);
|
||||
registerHighlighter(PlainHighlighter.class);
|
||||
registerHighlighter(PostingsHighlighter.class);
|
||||
}
|
||||
|
||||
public void registerHighlighter(Class<? extends Highlighter> clazz) {
|
||||
|
|
|
@ -22,6 +22,7 @@ package org.elasticsearch.search.highlight;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.elasticsearch.ElasticSearchException;
|
||||
import org.elasticsearch.ElasticSearchIllegalArgumentException;
|
||||
import org.elasticsearch.common.component.AbstractComponent;
|
||||
|
@ -46,7 +47,7 @@ import static com.google.common.collect.Maps.newHashMap;
|
|||
*/
|
||||
public class HighlightPhase extends AbstractComponent implements FetchSubPhase {
|
||||
|
||||
private Highlighters highlighters;
|
||||
private final Highlighters highlighters;
|
||||
|
||||
@Inject
|
||||
public HighlightPhase(Settings settings, Highlighters highlighters) {
|
||||
|
@ -93,7 +94,13 @@ public class HighlightPhase extends AbstractComponent implements FetchSubPhase {
|
|||
|
||||
if (field.highlighterType() == null) {
|
||||
boolean useFastVectorHighlighter = fieldMapper.fieldType().storeTermVectors() && fieldMapper.fieldType().storeTermVectorOffsets() && fieldMapper.fieldType().storeTermVectorPositions();
|
||||
field.highlighterType(useFastVectorHighlighter ? "fvh" : "plain");
|
||||
if (useFastVectorHighlighter) {
|
||||
field.highlighterType("fvh");
|
||||
} else if (fieldMapper.fieldType().indexOptions() == FieldInfo.IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) {
|
||||
field.highlighterType("postings");
|
||||
} else {
|
||||
field.highlighterType("plain");
|
||||
}
|
||||
}
|
||||
|
||||
Highlighter highlighter = highlighters.get(field.highlighterType());
|
||||
|
|
|
@ -0,0 +1,68 @@
|
|||
/*
|
||||
* Licensed to ElasticSearch and Shay Banon under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. ElasticSearch 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.elasticsearch.search.highlight;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import org.apache.lucene.search.highlight.DefaultEncoder;
|
||||
import org.apache.lucene.search.highlight.Encoder;
|
||||
import org.apache.lucene.search.highlight.SimpleHTMLEncoder;
|
||||
import org.elasticsearch.index.fieldvisitor.CustomFieldsVisitor;
|
||||
import org.elasticsearch.index.mapper.FieldMapper;
|
||||
import org.elasticsearch.search.fetch.FetchSubPhase;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
import org.elasticsearch.search.lookup.SearchLookup;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
public final class HighlightUtils {
|
||||
|
||||
//U+2029 PARAGRAPH SEPARATOR (PS): each value holds a discrete passage for highlighting (postings highlighter)
|
||||
public static final char PARAGRAPH_SEPARATOR = 8233;
|
||||
|
||||
private HighlightUtils() {
|
||||
|
||||
}
|
||||
|
||||
static List<Object> loadFieldValues(FieldMapper<?> mapper, SearchContext searchContext, FetchSubPhase.HitContext hitContext) throws IOException {
|
||||
List<Object> textsToHighlight;
|
||||
if (mapper.fieldType().stored()) {
|
||||
CustomFieldsVisitor fieldVisitor = new CustomFieldsVisitor(ImmutableSet.of(mapper.names().indexName()), false);
|
||||
hitContext.reader().document(hitContext.docId(), fieldVisitor);
|
||||
textsToHighlight = fieldVisitor.fields().get(mapper.names().indexName());
|
||||
if (textsToHighlight == null) {
|
||||
// Can happen if the document doesn't have the field to highlight
|
||||
textsToHighlight = ImmutableList.of();
|
||||
}
|
||||
} else {
|
||||
SearchLookup lookup = searchContext.lookup();
|
||||
lookup.setNextReader(hitContext.readerContext());
|
||||
lookup.setNextDocId(hitContext.docId());
|
||||
textsToHighlight = lookup.source().extractRawValues(mapper.names().sourcePath());
|
||||
}
|
||||
assert textsToHighlight != null;
|
||||
return textsToHighlight;
|
||||
}
|
||||
|
||||
static class Encoders {
|
||||
static Encoder DEFAULT = new DefaultEncoder();
|
||||
static Encoder HTML = new SimpleHTMLEncoder();
|
||||
}
|
||||
}
|
|
@ -28,12 +28,12 @@ import org.elasticsearch.search.internal.SearchContext;
|
|||
*/
|
||||
public class HighlighterContext {
|
||||
|
||||
public String fieldName;
|
||||
public SearchContextHighlight.Field field;
|
||||
public FieldMapper<?> mapper;
|
||||
public SearchContext context;
|
||||
public FetchSubPhase.HitContext hitContext;
|
||||
public Query highlightQuery;
|
||||
public final String fieldName;
|
||||
public final SearchContextHighlight.Field field;
|
||||
public final FieldMapper<?> mapper;
|
||||
public final SearchContext context;
|
||||
public final FetchSubPhase.HitContext hitContext;
|
||||
public final Query highlightQuery;
|
||||
|
||||
public HighlighterContext(String fieldName, SearchContextHighlight.Field field, FieldMapper<?> mapper, SearchContext context,
|
||||
FetchSubPhase.HitContext hitContext, Query highlightQuery) {
|
||||
|
|
|
@ -18,8 +18,6 @@
|
|||
*/
|
||||
package org.elasticsearch.search.highlight;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Maps;
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
|
@ -31,12 +29,10 @@ import org.apache.lucene.util.CollectionUtil;
|
|||
import org.elasticsearch.ElasticSearchIllegalArgumentException;
|
||||
import org.elasticsearch.common.text.StringText;
|
||||
import org.elasticsearch.common.text.Text;
|
||||
import org.elasticsearch.index.fieldvisitor.CustomFieldsVisitor;
|
||||
import org.elasticsearch.index.mapper.FieldMapper;
|
||||
import org.elasticsearch.search.fetch.FetchPhaseExecutionException;
|
||||
import org.elasticsearch.search.fetch.FetchSubPhase;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
import org.elasticsearch.search.lookup.SearchLookup;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
@ -62,7 +58,7 @@ public class PlainHighlighter implements Highlighter {
|
|||
FetchSubPhase.HitContext hitContext = highlighterContext.hitContext;
|
||||
FieldMapper<?> mapper = highlighterContext.mapper;
|
||||
|
||||
Encoder encoder = field.encoder().equals("html") ? Encoders.HTML : Encoders.DEFAULT;
|
||||
Encoder encoder = field.encoder().equals("html") ? HighlightUtils.Encoders.HTML : HighlightUtils.Encoders.DEFAULT;
|
||||
|
||||
if (!hitContext.cache().containsKey(CACHE_KEY)) {
|
||||
Map<FieldMapper<?>, org.apache.lucene.search.highlight.Highlighter> mappers = Maps.newHashMap();
|
||||
|
@ -97,31 +93,14 @@ public class PlainHighlighter implements Highlighter {
|
|||
cache.put(mapper, entry);
|
||||
}
|
||||
|
||||
List<Object> textsToHighlight;
|
||||
if (mapper.fieldType().stored()) {
|
||||
try {
|
||||
CustomFieldsVisitor fieldVisitor = new CustomFieldsVisitor(ImmutableSet.of(mapper.names().indexName()), false);
|
||||
hitContext.reader().document(hitContext.docId(), fieldVisitor);
|
||||
textsToHighlight = fieldVisitor.fields().get(mapper.names().indexName());
|
||||
if (textsToHighlight == null) {
|
||||
// Can happen if the document doesn't have the field to highlight
|
||||
textsToHighlight = ImmutableList.of();
|
||||
}
|
||||
} catch (Exception e) {
|
||||
throw new FetchPhaseExecutionException(context, "Failed to highlight field [" + highlighterContext.fieldName + "]", e);
|
||||
}
|
||||
} else {
|
||||
SearchLookup lookup = context.lookup();
|
||||
lookup.setNextReader(hitContext.readerContext());
|
||||
lookup.setNextDocId(hitContext.docId());
|
||||
textsToHighlight = lookup.source().extractRawValues(mapper.names().sourcePath());
|
||||
}
|
||||
assert textsToHighlight != null;
|
||||
|
||||
// a HACK to make highlighter do highlighting, even though its using the single frag list builder
|
||||
int numberOfFragments = field.numberOfFragments() == 0 ? 1 : field.numberOfFragments();
|
||||
ArrayList<TextFragment> fragsList = new ArrayList<TextFragment>();
|
||||
List<Object> textsToHighlight;
|
||||
|
||||
try {
|
||||
textsToHighlight = HighlightUtils.loadFieldValues(mapper, context, hitContext);
|
||||
|
||||
for (Object textToHighlight : textsToHighlight) {
|
||||
String text = textToHighlight.toString();
|
||||
Analyzer analyzer = context.mapperService().documentMapper(hitContext.hit().type()).mappers().indexAnalyzer();
|
||||
|
@ -185,7 +164,7 @@ public class PlainHighlighter implements Highlighter {
|
|||
return null;
|
||||
}
|
||||
|
||||
private int findGoodEndForNoHighlightExcerpt(int noMatchSize, TokenStream tokenStream) throws IOException {
|
||||
private static int findGoodEndForNoHighlightExcerpt(int noMatchSize, TokenStream tokenStream) throws IOException {
|
||||
try {
|
||||
if (!tokenStream.hasAttribute(OffsetAttribute.class)) {
|
||||
// Can't split on term boundaries without offsets
|
||||
|
@ -211,9 +190,4 @@ public class PlainHighlighter implements Highlighter {
|
|||
tokenStream.close();
|
||||
}
|
||||
}
|
||||
|
||||
private static class Encoders {
|
||||
public static Encoder DEFAULT = new DefaultEncoder();
|
||||
public static Encoder HTML = new SimpleHTMLEncoder();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,237 @@
|
|||
/*
|
||||
* Licensed to ElasticSearch and Shay Banon under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. ElasticSearch 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.elasticsearch.search.highlight;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.MultiReader;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.highlight.Encoder;
|
||||
import org.apache.lucene.search.postingshighlight.CustomPassageFormatter;
|
||||
import org.apache.lucene.search.postingshighlight.CustomPostingsHighlighter;
|
||||
import org.apache.lucene.search.postingshighlight.Snippet;
|
||||
import org.apache.lucene.search.postingshighlight.WholeBreakIterator;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CollectionUtil;
|
||||
import org.apache.lucene.util.UnicodeUtil;
|
||||
import org.elasticsearch.ElasticSearchIllegalArgumentException;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.text.StringText;
|
||||
import org.elasticsearch.index.mapper.FieldMapper;
|
||||
import org.elasticsearch.search.fetch.FetchPhaseExecutionException;
|
||||
import org.elasticsearch.search.fetch.FetchSubPhase;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.text.BreakIterator;
|
||||
import java.util.*;
|
||||
|
||||
public class PostingsHighlighter implements Highlighter {
|
||||
|
||||
private static final String CACHE_KEY = "highlight-postings";
|
||||
|
||||
@Override
|
||||
public String[] names() {
|
||||
return new String[]{"postings", "postings-highlighter"};
|
||||
}
|
||||
|
||||
@Override
|
||||
public HighlightField highlight(HighlighterContext highlighterContext) {
|
||||
|
||||
FieldMapper<?> fieldMapper = highlighterContext.mapper;
|
||||
SearchContextHighlight.Field field = highlighterContext.field;
|
||||
if (fieldMapper.fieldType().indexOptions() != FieldInfo.IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) {
|
||||
throw new ElasticSearchIllegalArgumentException("the field [" + field.field() + "] should be indexed with positions and offsets in the postings list to be used with postings highlighter");
|
||||
}
|
||||
|
||||
SearchContext context = highlighterContext.context;
|
||||
FetchSubPhase.HitContext hitContext = highlighterContext.hitContext;
|
||||
|
||||
if (!hitContext.cache().containsKey(CACHE_KEY)) {
|
||||
Query query;
|
||||
try {
|
||||
query = rewrite(context.query());
|
||||
} catch (IOException e) {
|
||||
throw new FetchPhaseExecutionException(context, "Failed to highlight field [" + highlighterContext.fieldName + "]", e);
|
||||
}
|
||||
SortedSet<Term> queryTerms = extractTerms(query);
|
||||
hitContext.cache().put(CACHE_KEY, new HighlighterEntry(queryTerms));
|
||||
}
|
||||
|
||||
HighlighterEntry highlighterEntry = (HighlighterEntry) hitContext.cache().get(CACHE_KEY);
|
||||
MapperHighlighterEntry mapperHighlighterEntry = highlighterEntry.mappers.get(fieldMapper);
|
||||
|
||||
if (mapperHighlighterEntry == null) {
|
||||
Encoder encoder = field.encoder().equals("html") ? HighlightUtils.Encoders.HTML : HighlightUtils.Encoders.DEFAULT;
|
||||
CustomPassageFormatter passageFormatter = new CustomPassageFormatter(field.preTags()[0], field.postTags()[0], encoder);
|
||||
BytesRef[] filteredQueryTerms = filterTerms(highlighterEntry.queryTerms, highlighterContext.fieldName, field.requireFieldMatch());
|
||||
mapperHighlighterEntry = new MapperHighlighterEntry(passageFormatter, filteredQueryTerms);
|
||||
}
|
||||
|
||||
//we merge back multiple values into a single value using the paragraph separator, unless we have to highlight every single value separately (number_of_fragments=0).
|
||||
boolean mergeValues = field.numberOfFragments() != 0;
|
||||
List<Snippet> snippets = new ArrayList<Snippet>();
|
||||
int numberOfFragments;
|
||||
|
||||
try {
|
||||
//we manually load the field values (from source if needed)
|
||||
List<Object> textsToHighlight = HighlightUtils.loadFieldValues(fieldMapper, context, hitContext);
|
||||
CustomPostingsHighlighter highlighter = new CustomPostingsHighlighter(mapperHighlighterEntry.passageFormatter, textsToHighlight, mergeValues, Integer.MAX_VALUE-1, field.noMatchSize());
|
||||
|
||||
if (field.numberOfFragments() == 0) {
|
||||
highlighter.setBreakIterator(new WholeBreakIterator());
|
||||
numberOfFragments = 1; //1 per value since we highlight per value
|
||||
} else {
|
||||
numberOfFragments = field.numberOfFragments();
|
||||
}
|
||||
|
||||
//we highlight every value separately calling the highlight method multiple times, only if we need to have back a snippet per value (whole value)
|
||||
int values = mergeValues ? 1 : textsToHighlight.size();
|
||||
for (int i = 0; i < values; i++) {
|
||||
Snippet[] fieldSnippets = highlighter.highlightDoc(highlighterContext.fieldName, mapperHighlighterEntry.filteredQueryTerms, new IndexSearcher(hitContext.reader()), hitContext.docId(), numberOfFragments);
|
||||
if (fieldSnippets != null) {
|
||||
for (Snippet fieldSnippet : fieldSnippets) {
|
||||
if (Strings.hasText(fieldSnippet.getText())) {
|
||||
snippets.add(fieldSnippet);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
} catch(IOException e) {
|
||||
throw new FetchPhaseExecutionException(context, "Failed to highlight field [" + highlighterContext.fieldName + "]", e);
|
||||
}
|
||||
|
||||
snippets = filterSnippets(snippets, field.numberOfFragments());
|
||||
|
||||
if (field.scoreOrdered()) {
|
||||
//let's sort the snippets by score if needed
|
||||
CollectionUtil.introSort(snippets, new Comparator<Snippet>() {
|
||||
public int compare(Snippet o1, Snippet o2) {
|
||||
return (int) Math.signum(o2.getScore() - o1.getScore());
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
String[] fragments = new String[snippets.size()];
|
||||
for (int i = 0; i < fragments.length; i++) {
|
||||
fragments[i] = snippets.get(i).getText();
|
||||
}
|
||||
|
||||
if (fragments.length > 0) {
|
||||
return new HighlightField(highlighterContext.fieldName, StringText.convertFromStringArray(fragments));
|
||||
}
|
||||
|
||||
return null;
|
||||
}
|
||||
|
||||
private static final IndexReader EMPTY_INDEXREADER = new MultiReader();
|
||||
|
||||
private static Query rewrite(Query original) throws IOException {
|
||||
Query query = original;
|
||||
for (Query rewrittenQuery = query.rewrite(EMPTY_INDEXREADER); rewrittenQuery != query;
|
||||
rewrittenQuery = query.rewrite(EMPTY_INDEXREADER)) {
|
||||
query = rewrittenQuery;
|
||||
}
|
||||
return query;
|
||||
}
|
||||
|
||||
private static SortedSet<Term> extractTerms(Query query) {
|
||||
SortedSet<Term> queryTerms = new TreeSet<Term>();
|
||||
query.extractTerms(queryTerms);
|
||||
return queryTerms;
|
||||
}
|
||||
|
||||
private static BytesRef[] filterTerms(SortedSet<Term> queryTerms, String field, boolean requireFieldMatch) {
|
||||
SortedSet<Term> fieldTerms;
|
||||
if (requireFieldMatch) {
|
||||
Term floor = new Term(field, "");
|
||||
Term ceiling = new Term(field, UnicodeUtil.BIG_TERM);
|
||||
fieldTerms = queryTerms.subSet(floor, ceiling);
|
||||
} else {
|
||||
fieldTerms = queryTerms;
|
||||
}
|
||||
|
||||
BytesRef terms[] = new BytesRef[fieldTerms.size()];
|
||||
int termUpto = 0;
|
||||
for(Term term : fieldTerms) {
|
||||
terms[termUpto++] = term.bytes();
|
||||
}
|
||||
|
||||
return terms;
|
||||
}
|
||||
|
||||
private static List<Snippet> filterSnippets(List<Snippet> snippets, int numberOfFragments) {
|
||||
|
||||
//We need to filter the snippets as due to no_match_size we could have
|
||||
//either highlighted snippets together non highlighted ones
|
||||
//We don't want to mix those up
|
||||
List<Snippet> filteredSnippets = new ArrayList<Snippet>(snippets.size());
|
||||
for (Snippet snippet : snippets) {
|
||||
if (snippet.isHighlighted()) {
|
||||
filteredSnippets.add(snippet);
|
||||
}
|
||||
}
|
||||
|
||||
//if there's at least one highlighted snippet, we return all the highlighted ones
|
||||
//otherwise we return the first non highlighted one if available
|
||||
if (filteredSnippets.size() == 0) {
|
||||
if (snippets.size() > 0) {
|
||||
Snippet snippet = snippets.get(0);
|
||||
//if we did discrete per value highlighting using whole break iterator (as number_of_fragments was 0)
|
||||
//we need to obtain the first sentence of the first value
|
||||
if (numberOfFragments == 0) {
|
||||
BreakIterator bi = BreakIterator.getSentenceInstance(Locale.ROOT);
|
||||
String text = snippet.getText();
|
||||
bi.setText(text);
|
||||
int next = bi.next();
|
||||
if (next != BreakIterator.DONE) {
|
||||
String newText = text.substring(0, next).trim();
|
||||
snippet = new Snippet(newText, snippet.getScore(), snippet.isHighlighted());
|
||||
}
|
||||
}
|
||||
filteredSnippets.add(snippet);
|
||||
}
|
||||
}
|
||||
|
||||
return filteredSnippets;
|
||||
}
|
||||
|
||||
private static class HighlighterEntry {
|
||||
final SortedSet<Term> queryTerms;
|
||||
Map<FieldMapper<?>, MapperHighlighterEntry> mappers = Maps.newHashMap();
|
||||
|
||||
private HighlighterEntry(SortedSet<Term> queryTerms) {
|
||||
this.queryTerms = queryTerms;
|
||||
}
|
||||
}
|
||||
|
||||
private static class MapperHighlighterEntry {
|
||||
final CustomPassageFormatter passageFormatter;
|
||||
final BytesRef[] filteredQueryTerms;
|
||||
|
||||
private MapperHighlighterEntry(CustomPassageFormatter passageFormatter, BytesRef[] filteredQueryTerms) {
|
||||
this.passageFormatter = passageFormatter;
|
||||
this.filteredQueryTerms = filteredQueryTerms;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,107 @@
|
|||
/*
|
||||
* Licensed to ElasticSearch and Shay Banon under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. ElasticSearch 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.postingshighlight;
|
||||
|
||||
import org.apache.lucene.search.highlight.DefaultEncoder;
|
||||
import org.apache.lucene.search.highlight.SimpleHTMLEncoder;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.junit.Test;
|
||||
|
||||
import static org.hamcrest.CoreMatchers.equalTo;
|
||||
import static org.hamcrest.CoreMatchers.notNullValue;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
|
||||
|
||||
public class CustomPassageFormatterTests {
|
||||
|
||||
@Test
|
||||
public void testSimpleFormat() {
|
||||
String content = "This is a really cool highlighter. Postings highlighter gives nice snippets back. No matches here.";
|
||||
|
||||
CustomPassageFormatter passageFormatter = new CustomPassageFormatter("<em>", "</em>", new DefaultEncoder());
|
||||
|
||||
Passage[] passages = new Passage[3];
|
||||
String match = "highlighter";
|
||||
BytesRef matchBytesRef = new BytesRef(match);
|
||||
|
||||
Passage passage1 = new Passage();
|
||||
int start = content.indexOf(match);
|
||||
int end = start + match.length();
|
||||
passage1.startOffset = 0;
|
||||
passage1.endOffset = end + 2; //lets include the whitespace at the end to make sure we trim it
|
||||
passage1.addMatch(start, end, matchBytesRef);
|
||||
passages[0] = passage1;
|
||||
|
||||
Passage passage2 = new Passage();
|
||||
start = content.lastIndexOf(match);
|
||||
end = start + match.length();
|
||||
passage2.startOffset = passage1.endOffset;
|
||||
passage2.endOffset = end + 26;
|
||||
passage2.addMatch(start, end, matchBytesRef);
|
||||
passages[1] = passage2;
|
||||
|
||||
Passage passage3 = new Passage();
|
||||
passage3.startOffset = passage2.endOffset;
|
||||
passage3.endOffset = content.length();
|
||||
passages[2] = passage3;
|
||||
|
||||
Snippet[] fragments = passageFormatter.format(passages, content);
|
||||
assertThat(fragments, notNullValue());
|
||||
assertThat(fragments.length, equalTo(3));
|
||||
assertThat(fragments[0].getText(), equalTo("This is a really cool <em>highlighter</em>."));
|
||||
assertThat(fragments[0].isHighlighted(), equalTo(true));
|
||||
assertThat(fragments[1].getText(), equalTo("Postings <em>highlighter</em> gives nice snippets back."));
|
||||
assertThat(fragments[1].isHighlighted(), equalTo(true));
|
||||
assertThat(fragments[2].getText(), equalTo("No matches here."));
|
||||
assertThat(fragments[2].isHighlighted(), equalTo(false));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHtmlEncodeFormat() {
|
||||
String content = "<b>This is a really cool highlighter.</b> Postings highlighter gives nice snippets back.";
|
||||
|
||||
CustomPassageFormatter passageFormatter = new CustomPassageFormatter("<em>", "</em>", new SimpleHTMLEncoder());
|
||||
|
||||
Passage[] passages = new Passage[2];
|
||||
String match = "highlighter";
|
||||
BytesRef matchBytesRef = new BytesRef(match);
|
||||
|
||||
Passage passage1 = new Passage();
|
||||
int start = content.indexOf(match);
|
||||
int end = start + match.length();
|
||||
passage1.startOffset = 0;
|
||||
passage1.endOffset = end + 6; //lets include the whitespace at the end to make sure we trim it
|
||||
passage1.addMatch(start, end, matchBytesRef);
|
||||
passages[0] = passage1;
|
||||
|
||||
Passage passage2 = new Passage();
|
||||
start = content.lastIndexOf(match);
|
||||
end = start + match.length();
|
||||
passage2.startOffset = passage1.endOffset;
|
||||
passage2.endOffset = content.length();
|
||||
passage2.addMatch(start, end, matchBytesRef);
|
||||
passages[1] = passage2;
|
||||
|
||||
Snippet[] fragments = passageFormatter.format(passages, content);
|
||||
assertThat(fragments, notNullValue());
|
||||
assertThat(fragments.length, equalTo(2));
|
||||
assertThat(fragments[0].getText(), equalTo("<b>This is a really cool <em>highlighter</em>.</b>"));
|
||||
assertThat(fragments[1].getText(), equalTo("Postings <em>highlighter</em> gives nice snippets back."));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,487 @@
|
|||
/*
|
||||
* Licensed to ElasticSearch and Shay Banon under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. ElasticSearch 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.postingshighlight;
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.FieldType;
|
||||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.index.*;
|
||||
import org.apache.lucene.search.*;
|
||||
import org.apache.lucene.search.highlight.DefaultEncoder;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.UnicodeUtil;
|
||||
import org.elasticsearch.search.highlight.HighlightUtils;
|
||||
import org.elasticsearch.test.ElasticsearchLuceneTestCase;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.*;
|
||||
|
||||
import static org.hamcrest.CoreMatchers.equalTo;
|
||||
import static org.hamcrest.CoreMatchers.notNullValue;
|
||||
|
||||
@LuceneTestCase.SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom", "Lucene3x"})
|
||||
public class CustomPostingsHighlighterTests extends ElasticsearchLuceneTestCase {
|
||||
|
||||
@Test
|
||||
public void testDiscreteHighlightingPerValue() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
|
||||
iwc.setMergePolicy(newLogMergePolicy());
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
|
||||
|
||||
FieldType offsetsType = new FieldType(TextField.TYPE_STORED);
|
||||
offsetsType.setIndexOptions(FieldInfo.IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
|
||||
Field body = new Field("body", "", offsetsType);
|
||||
final String firstValue = "This is a test. Just a test highlighting from postings highlighter.";
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
body.setStringValue(firstValue);
|
||||
|
||||
final String secondValue = "This is the second value to perform highlighting on.";
|
||||
Field body2 = new Field("body", "", offsetsType);
|
||||
doc.add(body2);
|
||||
body2.setStringValue(secondValue);
|
||||
|
||||
final String thirdValue = "This is the third value to test highlighting with postings.";
|
||||
Field body3 = new Field("body", "", offsetsType);
|
||||
doc.add(body3);
|
||||
body3.setStringValue(thirdValue);
|
||||
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
List<Object> fieldValues = new ArrayList<Object>();
|
||||
fieldValues.add(firstValue);
|
||||
fieldValues.add(secondValue);
|
||||
fieldValues.add(thirdValue);
|
||||
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
|
||||
Query query = new TermQuery(new Term("body", "highlighting"));
|
||||
BytesRef[] queryTerms = filterTerms(extractTerms(query), "body", true);
|
||||
|
||||
TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER);
|
||||
assertThat(topDocs.totalHits, equalTo(1));
|
||||
int docId = topDocs.scoreDocs[0].doc;
|
||||
|
||||
//highlighting per value, considering whole values (simulating number_of_fragments=0)
|
||||
CustomPostingsHighlighter highlighter = new CustomPostingsHighlighter(new CustomPassageFormatter("<b>", "</b>", new DefaultEncoder()), fieldValues, false, Integer.MAX_VALUE - 1, 0);
|
||||
highlighter.setBreakIterator(new WholeBreakIterator());
|
||||
|
||||
Snippet[] snippets = highlighter.highlightDoc("body", queryTerms, searcher, docId, 5);
|
||||
assertThat(snippets.length, equalTo(1));
|
||||
assertThat(snippets[0].getText(), equalTo("This is a test. Just a test <b>highlighting</b> from postings highlighter."));
|
||||
|
||||
snippets = highlighter.highlightDoc("body", queryTerms, searcher, docId, 5);
|
||||
assertThat(snippets.length, equalTo(1));
|
||||
assertThat(snippets[0].getText(), equalTo("This is the second value to perform <b>highlighting</b> on."));
|
||||
|
||||
snippets = highlighter.highlightDoc("body", queryTerms, searcher, docId, 5);
|
||||
assertThat(snippets.length, equalTo(1));
|
||||
assertThat(snippets[0].getText(), equalTo("This is the third value to test <b>highlighting</b> with postings."));
|
||||
|
||||
|
||||
//let's try without whole break iterator as well, to prove that highlighting works the same when working per value (not optimized though)
|
||||
highlighter = new CustomPostingsHighlighter(new CustomPassageFormatter("<b>", "</b>", new DefaultEncoder()), fieldValues, false, Integer.MAX_VALUE - 1, 0);
|
||||
|
||||
snippets = highlighter.highlightDoc("body", queryTerms, searcher, docId, 5);
|
||||
assertThat(snippets.length, equalTo(1));
|
||||
assertThat(snippets[0].getText(), equalTo("Just a test <b>highlighting</b> from postings highlighter."));
|
||||
|
||||
snippets = highlighter.highlightDoc("body", queryTerms, searcher, docId, 5);
|
||||
assertThat(snippets.length, equalTo(1));
|
||||
assertThat(snippets[0].getText(), equalTo("This is the second value to perform <b>highlighting</b> on."));
|
||||
|
||||
snippets = highlighter.highlightDoc("body", queryTerms, searcher, docId, 5);
|
||||
assertThat(snippets.length, equalTo(1));
|
||||
assertThat(snippets[0].getText(), equalTo("This is the third value to test <b>highlighting</b> with postings."));
|
||||
|
||||
ir.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
/*
|
||||
Tests that scoring works properly even when using discrete per value highlighting
|
||||
*/
|
||||
@Test
|
||||
public void testDiscreteHighlightingScoring() throws Exception {
|
||||
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
|
||||
iwc.setMergePolicy(newLogMergePolicy());
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
|
||||
|
||||
FieldType offsetsType = new FieldType(TextField.TYPE_STORED);
|
||||
offsetsType.setIndexOptions(FieldInfo.IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
|
||||
|
||||
//good position but only one match
|
||||
final String firstValue = "This is a test. Just a test1 highlighting from postings highlighter.";
|
||||
Field body = new Field("body", "", offsetsType);
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
body.setStringValue(firstValue);
|
||||
|
||||
//two matches, not the best snippet due to its length though
|
||||
final String secondValue = "This is the second highlighting value to perform highlighting on a longer text that gets scored lower.";
|
||||
Field body2 = new Field("body", "", offsetsType);
|
||||
doc.add(body2);
|
||||
body2.setStringValue(secondValue);
|
||||
|
||||
//two matches and short, will be scored highest
|
||||
final String thirdValue = "This is highlighting the third short highlighting value.";
|
||||
Field body3 = new Field("body", "", offsetsType);
|
||||
doc.add(body3);
|
||||
body3.setStringValue(thirdValue);
|
||||
|
||||
//one match, same as first but at the end, will be scored lower due to its position
|
||||
final String fourthValue = "Just a test4 highlighting from postings highlighter.";
|
||||
Field body4 = new Field("body", "", offsetsType);
|
||||
doc.add(body4);
|
||||
body4.setStringValue(fourthValue);
|
||||
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
|
||||
String firstHlValue = "Just a test1 <b>highlighting</b> from postings highlighter.";
|
||||
String secondHlValue = "This is the second <b>highlighting</b> value to perform <b>highlighting</b> on a longer text that gets scored lower.";
|
||||
String thirdHlValue = "This is <b>highlighting</b> the third short <b>highlighting</b> value.";
|
||||
String fourthHlValue = "Just a test4 <b>highlighting</b> from postings highlighter.";
|
||||
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
Query query = new TermQuery(new Term("body", "highlighting"));
|
||||
BytesRef[] queryTerms = filterTerms(extractTerms(query), "body", true);
|
||||
|
||||
TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER);
|
||||
assertThat(topDocs.totalHits, equalTo(1));
|
||||
|
||||
int docId = topDocs.scoreDocs[0].doc;
|
||||
|
||||
List<Object> fieldValues = new ArrayList<Object>();
|
||||
fieldValues.add(firstValue);
|
||||
fieldValues.add(secondValue);
|
||||
fieldValues.add(thirdValue);
|
||||
fieldValues.add(fourthValue);
|
||||
|
||||
boolean mergeValues = true;
|
||||
CustomPostingsHighlighter highlighter = new CustomPostingsHighlighter(new CustomPassageFormatter("<b>", "</b>", new DefaultEncoder()), fieldValues, mergeValues, Integer.MAX_VALUE-1, 0);
|
||||
Snippet[] snippets = highlighter.highlightDoc("body", queryTerms, searcher, docId, 5);
|
||||
|
||||
assertThat(snippets.length, equalTo(4));
|
||||
|
||||
assertThat(snippets[0].getText(), equalTo(firstHlValue));
|
||||
assertThat(snippets[1].getText(), equalTo(secondHlValue));
|
||||
assertThat(snippets[2].getText(), equalTo(thirdHlValue));
|
||||
assertThat(snippets[3].getText(), equalTo(fourthHlValue));
|
||||
|
||||
|
||||
//Let's highlight each separate value and check how the snippets are scored
|
||||
mergeValues = false;
|
||||
highlighter = new CustomPostingsHighlighter(new CustomPassageFormatter("<b>", "</b>", new DefaultEncoder()), fieldValues, mergeValues, Integer.MAX_VALUE-1, 0);
|
||||
List<Snippet> snippets2 = new ArrayList<Snippet>();
|
||||
for (int i = 0; i < fieldValues.size(); i++) {
|
||||
snippets2.addAll(Arrays.asList(highlighter.highlightDoc("body", queryTerms, searcher, docId, 5)));
|
||||
}
|
||||
|
||||
assertThat(snippets2.size(), equalTo(4));
|
||||
assertThat(snippets2.get(0).getText(), equalTo(firstHlValue));
|
||||
assertThat(snippets2.get(1).getText(), equalTo(secondHlValue));
|
||||
assertThat(snippets2.get(2).getText(), equalTo(thirdHlValue));
|
||||
assertThat(snippets2.get(3).getText(), equalTo(fourthHlValue));
|
||||
|
||||
Comparator <Snippet> comparator = new Comparator<Snippet>() {
|
||||
@Override
|
||||
public int compare(Snippet o1, Snippet o2) {
|
||||
return (int)Math.signum(o1.getScore() - o2.getScore());
|
||||
}
|
||||
};
|
||||
|
||||
//sorting both groups of snippets
|
||||
Arrays.sort(snippets, comparator);
|
||||
Collections.sort(snippets2, comparator);
|
||||
|
||||
//checking that the snippets are in the same order, regardless of whether we used per value discrete highlighting or not
|
||||
//we can't compare the scores directly since they are slightly different due to the multiValued separator added when merging values together
|
||||
//That causes slightly different lengths and start offsets, thus a slightly different score.
|
||||
//Anyways, that's not an issue. What's important is that the score is computed the same way, so that the produced order is always the same.
|
||||
for (int i = 0; i < snippets.length; i++) {
|
||||
assertThat(snippets[i].getText(), equalTo(snippets2.get(i).getText()));
|
||||
}
|
||||
|
||||
ir.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
/*
|
||||
Tests that we produce the same snippets and scores when manually merging values in our own custom highlighter rather than using the built-in code
|
||||
*/
|
||||
@Test
|
||||
public void testMergeValuesScoring() throws Exception {
|
||||
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
|
||||
iwc.setMergePolicy(newLogMergePolicy());
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
|
||||
|
||||
FieldType offsetsType = new FieldType(TextField.TYPE_STORED);
|
||||
offsetsType.setIndexOptions(FieldInfo.IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
|
||||
|
||||
//good position but only one match
|
||||
final String firstValue = "This is a test. Just a test1 highlighting from postings highlighter.";
|
||||
Field body = new Field("body", "", offsetsType);
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
body.setStringValue(firstValue);
|
||||
|
||||
//two matches, not the best snippet due to its length though
|
||||
final String secondValue = "This is the second highlighting value to perform highlighting on a longer text that gets scored lower.";
|
||||
Field body2 = new Field("body", "", offsetsType);
|
||||
doc.add(body2);
|
||||
body2.setStringValue(secondValue);
|
||||
|
||||
//two matches and short, will be scored highest
|
||||
final String thirdValue = "This is highlighting the third short highlighting value.";
|
||||
Field body3 = new Field("body", "", offsetsType);
|
||||
doc.add(body3);
|
||||
body3.setStringValue(thirdValue);
|
||||
|
||||
//one match, same as first but at the end, will be scored lower due to its position
|
||||
final String fourthValue = "Just a test4 highlighting from postings highlighter.";
|
||||
Field body4 = new Field("body", "", offsetsType);
|
||||
doc.add(body4);
|
||||
body4.setStringValue(fourthValue);
|
||||
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
|
||||
String firstHlValue = "Just a test1 <b>highlighting</b> from postings highlighter.";
|
||||
String secondHlValue = "This is the second <b>highlighting</b> value to perform <b>highlighting</b> on a longer text that gets scored lower.";
|
||||
String thirdHlValue = "This is <b>highlighting</b> the third short <b>highlighting</b> value.";
|
||||
String fourthHlValue = "Just a test4 <b>highlighting</b> from postings highlighter.";
|
||||
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
Query query = new TermQuery(new Term("body", "highlighting"));
|
||||
BytesRef[] queryTerms = filterTerms(extractTerms(query), "body", true);
|
||||
|
||||
TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER);
|
||||
assertThat(topDocs.totalHits, equalTo(1));
|
||||
|
||||
int docId = topDocs.scoreDocs[0].doc;
|
||||
|
||||
List<Object> fieldValues = new ArrayList<Object>();
|
||||
fieldValues.add(firstValue);
|
||||
fieldValues.add(secondValue);
|
||||
fieldValues.add(thirdValue);
|
||||
fieldValues.add(fourthValue);
|
||||
|
||||
boolean mergeValues = true;
|
||||
CustomPostingsHighlighter highlighter = new CustomPostingsHighlighter(new CustomPassageFormatter("<b>", "</b>", new DefaultEncoder()), fieldValues, mergeValues, Integer.MAX_VALUE-1, 0);
|
||||
Snippet[] snippets = highlighter.highlightDoc("body", queryTerms, searcher, docId, 5);
|
||||
|
||||
assertThat(snippets.length, equalTo(4));
|
||||
|
||||
assertThat(snippets[0].getText(), equalTo(firstHlValue));
|
||||
assertThat(snippets[1].getText(), equalTo(secondHlValue));
|
||||
assertThat(snippets[2].getText(), equalTo(thirdHlValue));
|
||||
assertThat(snippets[3].getText(), equalTo(fourthHlValue));
|
||||
|
||||
|
||||
//testing now our fork / normal postings highlighter, which merges multiple values together using the paragraph separator
|
||||
XPostingsHighlighter highlighter2 = new XPostingsHighlighter(Integer.MAX_VALUE - 1) {
|
||||
@Override
|
||||
protected char getMultiValuedSeparator(String field) {
|
||||
return HighlightUtils.PARAGRAPH_SEPARATOR;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected XPassageFormatter getFormatter(String field) {
|
||||
return new CustomPassageFormatter("<b>", "</b>", new DefaultEncoder());
|
||||
}
|
||||
};
|
||||
|
||||
Map<String, Object[]> highlightMap = highlighter2.highlightFieldsAsObjects(new String[]{"body"}, query, searcher, new int[]{docId}, new int[]{5});
|
||||
Object[] objects = highlightMap.get("body");
|
||||
assertThat(objects, notNullValue());
|
||||
assertThat(objects.length, equalTo(1));
|
||||
Snippet[] normalSnippets = (Snippet[])objects[0];
|
||||
|
||||
assertThat(normalSnippets.length, equalTo(4));
|
||||
|
||||
assertThat(normalSnippets[0].getText(), equalTo(firstHlValue));
|
||||
assertThat(normalSnippets[1].getText(), equalTo(secondHlValue));
|
||||
assertThat(normalSnippets[2].getText(), equalTo(thirdHlValue));
|
||||
assertThat(normalSnippets[3].getText(), equalTo(fourthHlValue));
|
||||
|
||||
|
||||
for (int i = 0; i < normalSnippets.length; i++) {
|
||||
Snippet normalSnippet = snippets[0];
|
||||
Snippet customSnippet = normalSnippets[0];
|
||||
assertThat(customSnippet.getText(), equalTo(normalSnippet.getText()));
|
||||
assertThat(customSnippet.getScore(), equalTo(normalSnippet.getScore()));
|
||||
}
|
||||
|
||||
ir.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRequireFieldMatch() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
|
||||
iwc.setMergePolicy(newLogMergePolicy());
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
|
||||
|
||||
FieldType offsetsType = new FieldType(TextField.TYPE_STORED);
|
||||
offsetsType.setIndexOptions(FieldInfo.IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
|
||||
Field body = new Field("body", "", offsetsType);
|
||||
Field none = new Field("none", "", offsetsType);
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
doc.add(none);
|
||||
|
||||
String firstValue = "This is a test. Just a test highlighting from postings. Feel free to ignore.";
|
||||
body.setStringValue(firstValue);
|
||||
none.setStringValue(firstValue);
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
Query query = new TermQuery(new Term("none", "highlighting"));
|
||||
SortedSet<Term> queryTerms = extractTerms(query);
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER);
|
||||
assertThat(topDocs.totalHits, equalTo(1));
|
||||
int docId = topDocs.scoreDocs[0].doc;
|
||||
|
||||
List<Object> values = new ArrayList<Object>();
|
||||
values.add(firstValue);
|
||||
|
||||
CustomPassageFormatter passageFormatter = new CustomPassageFormatter("<b>", "</b>", new DefaultEncoder());
|
||||
CustomPostingsHighlighter highlighter = new CustomPostingsHighlighter(passageFormatter, values, true, Integer.MAX_VALUE - 1, 0);
|
||||
|
||||
//no snippets with simulated require field match (we filter the terms ourselves)
|
||||
boolean requireFieldMatch = true;
|
||||
BytesRef[] filteredQueryTerms = filterTerms(queryTerms, "body", requireFieldMatch);
|
||||
Snippet[] snippets = highlighter.highlightDoc("body", filteredQueryTerms, searcher, docId, 5);
|
||||
assertThat(snippets.length, equalTo(0));
|
||||
|
||||
|
||||
highlighter = new CustomPostingsHighlighter(passageFormatter, values, true, Integer.MAX_VALUE - 1, 0);
|
||||
//one snippet without require field match, just passing in the query terms with no filtering on our side
|
||||
requireFieldMatch = false;
|
||||
filteredQueryTerms = filterTerms(queryTerms, "body", requireFieldMatch);
|
||||
snippets = highlighter.highlightDoc("body", filteredQueryTerms, searcher, docId, 5);
|
||||
assertThat(snippets.length, equalTo(1));
|
||||
assertThat(snippets[0].getText(), equalTo("Just a test <b>highlighting</b> from postings."));
|
||||
|
||||
ir.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNoMatchSize() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
|
||||
iwc.setMergePolicy(newLogMergePolicy());
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
|
||||
|
||||
FieldType offsetsType = new FieldType(TextField.TYPE_STORED);
|
||||
offsetsType.setIndexOptions(FieldInfo.IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
|
||||
Field body = new Field("body", "", offsetsType);
|
||||
Field none = new Field("none", "", offsetsType);
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
doc.add(none);
|
||||
|
||||
String firstValue = "This is a test. Just a test highlighting from postings. Feel free to ignore.";
|
||||
body.setStringValue(firstValue);
|
||||
none.setStringValue(firstValue);
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
Query query = new TermQuery(new Term("none", "highlighting"));
|
||||
SortedSet<Term> queryTerms = extractTerms(query);
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER);
|
||||
assertThat(topDocs.totalHits, equalTo(1));
|
||||
int docId = topDocs.scoreDocs[0].doc;
|
||||
|
||||
List<Object> values = new ArrayList<Object>();
|
||||
values.add(firstValue);
|
||||
|
||||
BytesRef[] filteredQueryTerms = filterTerms(queryTerms, "body", true);
|
||||
CustomPassageFormatter passageFormatter = new CustomPassageFormatter("<b>", "</b>", new DefaultEncoder());
|
||||
|
||||
CustomPostingsHighlighter highlighter = new CustomPostingsHighlighter(passageFormatter, values, true, Integer.MAX_VALUE - 1, 0);
|
||||
Snippet[] snippets = highlighter.highlightDoc("body", filteredQueryTerms, searcher, docId, 5);
|
||||
assertThat(snippets.length, equalTo(0));
|
||||
|
||||
highlighter = new CustomPostingsHighlighter(passageFormatter, values, true, Integer.MAX_VALUE - 1, atLeast(1));
|
||||
snippets = highlighter.highlightDoc("body", filteredQueryTerms, searcher, docId, 5);
|
||||
assertThat(snippets.length, equalTo(1));
|
||||
assertThat(snippets[0].getText(), equalTo("This is a test."));
|
||||
|
||||
ir.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
private static SortedSet<Term> extractTerms(Query query) {
|
||||
SortedSet<Term> queryTerms = new TreeSet<Term>();
|
||||
query.extractTerms(queryTerms);
|
||||
return queryTerms;
|
||||
}
|
||||
|
||||
private static BytesRef[] filterTerms(SortedSet<Term> queryTerms, String field, boolean requireFieldMatch) {
|
||||
SortedSet<Term> fieldTerms;
|
||||
if (requireFieldMatch) {
|
||||
Term floor = new Term(field, "");
|
||||
Term ceiling = new Term(field, UnicodeUtil.BIG_TERM);
|
||||
fieldTerms = queryTerms.subSet(floor, ceiling);
|
||||
} else {
|
||||
fieldTerms = queryTerms;
|
||||
}
|
||||
|
||||
BytesRef terms[] = new BytesRef[fieldTerms.size()];
|
||||
int termUpto = 0;
|
||||
for(Term term : fieldTerms) {
|
||||
terms[termUpto++] = term.bytes();
|
||||
}
|
||||
|
||||
return terms;
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -25,24 +25,29 @@ import org.elasticsearch.action.search.SearchPhaseExecutionException;
|
|||
import org.elasticsearch.action.search.SearchRequestBuilder;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.SearchType;
|
||||
import org.elasticsearch.action.search.ShardSearchFailure;
|
||||
import org.elasticsearch.common.Priority;
|
||||
import org.elasticsearch.common.settings.ImmutableSettings;
|
||||
import org.elasticsearch.common.settings.ImmutableSettings.Builder;
|
||||
import org.elasticsearch.common.text.Text;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||
import org.elasticsearch.index.query.*;
|
||||
import org.elasticsearch.index.query.FilterBuilders;
|
||||
import org.elasticsearch.index.query.IdsQueryBuilder;
|
||||
import org.elasticsearch.index.query.MatchQueryBuilder;
|
||||
import org.elasticsearch.index.query.MatchQueryBuilder.Operator;
|
||||
import org.elasticsearch.index.query.MatchQueryBuilder.Type;
|
||||
import org.elasticsearch.index.query.QueryBuilders;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
import org.elasticsearch.search.SearchHit;
|
||||
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
||||
import org.elasticsearch.test.AbstractIntegrationTest;
|
||||
import org.elasticsearch.test.hamcrest.ElasticsearchAssertions;
|
||||
import org.hamcrest.Matcher;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.elasticsearch.action.search.SearchType.QUERY_THEN_FETCH;
|
||||
import static org.elasticsearch.client.Requests.searchRequest;
|
||||
|
@ -177,7 +182,8 @@ public class HighlighterSearchTests extends AbstractIntegrationTest {
|
|||
endObject().
|
||||
endObject().
|
||||
endObject();
|
||||
ElasticsearchAssertions.assertAcked(prepareCreate("test").addMapping("test", builder).setSettings(
|
||||
|
||||
assertAcked(prepareCreate("test").addMapping("test", builder).setSettings(
|
||||
ImmutableSettings.settingsBuilder().put("index.number_of_shards", 1)
|
||||
.put("index.number_of_replicas", 0)
|
||||
.put("analysis.filter.wordDelimiter.type", "word_delimiter")
|
||||
|
@ -190,6 +196,7 @@ public class HighlighterSearchTests extends AbstractIntegrationTest {
|
|||
.put("analysis.analyzer.custom_analyzer.tokenizer", "whitespace")
|
||||
.putArray("analysis.analyzer.custom_analyzer.filter", "lowercase", "wordDelimiter"))
|
||||
);
|
||||
|
||||
ensureGreen();
|
||||
client().prepareIndex("test", "test", "1")
|
||||
.setSource(XContentFactory.jsonBuilder()
|
||||
|
@ -520,6 +527,80 @@ public class HighlighterSearchTests extends AbstractIntegrationTest {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSourceLookupHighlightingUsingPostingsHighlighter() throws Exception {
|
||||
client().admin().indices().prepareCreate("test").setSettings(ImmutableSettings.settingsBuilder().put("index.number_of_shards", 2))
|
||||
.addMapping("type1", jsonBuilder().startObject().startObject("type1").startObject("properties")
|
||||
// we don't store title, now lets see if it works...
|
||||
.startObject("title").field("type", "string").field("store", "no").field("index_options", "offsets").endObject()
|
||||
.startObject("attachments").startObject("properties").startObject("body").field("type", "string").field("store", "no").field("index_options", "offsets").endObject().endObject().endObject()
|
||||
.endObject().endObject().endObject())
|
||||
.execute().actionGet();
|
||||
client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForYellowStatus().execute().actionGet();
|
||||
|
||||
for (int i = 0; i < 5; i++) {
|
||||
client().prepareIndex("test", "type1", Integer.toString(i))
|
||||
.setSource(XContentFactory.jsonBuilder().startObject()
|
||||
.array("title", "This is a test on the highlighting bug present in elasticsearch. Hopefully it works.",
|
||||
"This is the second bug to perform highlighting on.")
|
||||
.startArray("attachments").startObject().field("body", "attachment for this test").endObject().startObject().field("body", "attachment 2").endObject().endArray()
|
||||
.endObject())
|
||||
.setRefresh(true).execute().actionGet();
|
||||
}
|
||||
|
||||
SearchResponse search = client().prepareSearch()
|
||||
.setQuery(matchQuery("title", "bug"))
|
||||
//asking for the whole field to be highlighted
|
||||
.addHighlightedField("title", -1, 0)
|
||||
.execute().actionGet();
|
||||
|
||||
assertNoFailures(search);
|
||||
|
||||
assertThat(search.getHits().totalHits(), equalTo(5l));
|
||||
assertThat(search.getHits().hits().length, equalTo(5));
|
||||
|
||||
for (SearchHit hit : search.getHits()) {
|
||||
Text[] fragments = hit.highlightFields().get("title").fragments();
|
||||
assertThat(fragments.length, equalTo(2));
|
||||
assertThat(fragments[0].string(), equalTo("This is a test on the highlighting <em>bug</em> present in elasticsearch. Hopefully it works."));
|
||||
assertThat(fragments[1].string(), equalTo("This is the second <em>bug</em> to perform highlighting on."));
|
||||
}
|
||||
|
||||
search = client().prepareSearch()
|
||||
.setQuery(matchQuery("title", "bug"))
|
||||
//sentences will be generated out of each value
|
||||
.addHighlightedField("title")
|
||||
.execute().actionGet();
|
||||
|
||||
assertNoFailures(search);
|
||||
|
||||
assertThat(search.getHits().totalHits(), equalTo(5l));
|
||||
assertThat(search.getHits().hits().length, equalTo(5));
|
||||
|
||||
for (SearchHit hit : search.getHits()) {
|
||||
Text[] fragments = hit.highlightFields().get("title").fragments();
|
||||
assertThat(fragments.length, equalTo(2));
|
||||
assertThat(fragments[0].string(), equalTo("This is a test on the highlighting <em>bug</em> present in elasticsearch."));
|
||||
assertThat(fragments[1].string(), equalTo("This is the second <em>bug</em> to perform highlighting on."));
|
||||
}
|
||||
|
||||
search = client().prepareSearch()
|
||||
.setQuery(matchQuery("attachments.body", "attachment"))
|
||||
.addHighlightedField("attachments.body", -1, 2)
|
||||
.execute().actionGet();
|
||||
|
||||
assertNoFailures(search);
|
||||
|
||||
assertThat(search.getHits().totalHits(), equalTo(5l));
|
||||
assertThat(search.getHits().hits().length, equalTo(5));
|
||||
|
||||
for (SearchHit hit : search.getHits()) {
|
||||
//shorter fragments are scored higher
|
||||
assertThat(hit.highlightFields().get("attachments.body").fragments()[0].string(), equalTo("<em>attachment</em> for this test"));
|
||||
assertThat(hit.highlightFields().get("attachments.body").fragments()[1].string(), equalTo("<em>attachment</em> 2"));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHighlightIssue1994() throws Exception {
|
||||
client().admin().indices().prepareCreate("test").setSettings(ImmutableSettings.settingsBuilder().put("index.number_of_shards", 2))
|
||||
|
@ -1662,11 +1743,11 @@ public class HighlighterSearchTests extends AbstractIntegrationTest {
|
|||
public void testHighlightNoMatchSize() throws IOException {
|
||||
|
||||
prepareCreate("test")
|
||||
.addMapping("type1", "text", "type=string," + randomStoreField() + "term_vector=with_positions_offsets")
|
||||
.addMapping("type1", "text", "type=string," + randomStoreField() + "term_vector=with_positions_offsets,index_options=offsets")
|
||||
.get();
|
||||
ensureGreen();
|
||||
|
||||
String text = "I am pretty long so some of me should get cut off";
|
||||
String text = "I am pretty long so some of me should get cut off. Second sentence";
|
||||
index("test", "type1", "1", "text", text);
|
||||
refresh();
|
||||
|
||||
|
@ -1682,6 +1763,10 @@ public class HighlighterSearchTests extends AbstractIntegrationTest {
|
|||
response = client().prepareSearch("test").addHighlightedField(field).get();
|
||||
assertNotHighlighted(response, 0, "text");
|
||||
|
||||
field.highlighterType("postings");
|
||||
response = client().prepareSearch("test").addHighlightedField(field).get();
|
||||
assertNotHighlighted(response, 0, "text");
|
||||
|
||||
// When noMatchSize is set to 0 you also shouldn't get any
|
||||
field.highlighterType("plain").noMatchSize(0);
|
||||
response = client().prepareSearch("test").addHighlightedField(field).get();
|
||||
|
@ -1691,55 +1776,88 @@ public class HighlighterSearchTests extends AbstractIntegrationTest {
|
|||
response = client().prepareSearch("test").addHighlightedField(field).get();
|
||||
assertNotHighlighted(response, 0, "text");
|
||||
|
||||
field.highlighterType("postings");
|
||||
response = client().prepareSearch("test").addHighlightedField(field).get();
|
||||
assertNotHighlighted(response, 0, "text");
|
||||
|
||||
// When noMatchSize is between 0 and the size of the string
|
||||
field.highlighterType("plain").noMatchSize(21);
|
||||
response = client().prepareSearch("test").addHighlightedField(field).get();
|
||||
assertHighlight(response, 0, "text", 0, equalTo("I am pretty long so"));
|
||||
assertHighlight(response, 0, "text", 0, 1, equalTo("I am pretty long so"));
|
||||
|
||||
// The FVH also works but the fragment is longer than the plain highlighter because of boundary_max_scan
|
||||
field.highlighterType("fvh");
|
||||
response = client().prepareSearch("test").addHighlightedField(field).get();
|
||||
assertHighlight(response, 0, "text", 0, equalTo("I am pretty long so some"));
|
||||
assertHighlight(response, 0, "text", 0, 1, equalTo("I am pretty long so some"));
|
||||
|
||||
// Postings hl also works but the fragment is the whole first sentence (size ignored)
|
||||
field.highlighterType("postings");
|
||||
response = client().prepareSearch("test").addHighlightedField(field).get();
|
||||
assertHighlight(response, 0, "text", 0, 1, equalTo("I am pretty long so some of me should get cut off."));
|
||||
|
||||
// We can also ask for a fragment longer than the input string and get the whole string
|
||||
field.highlighterType("plain").noMatchSize(text.length() * 2);
|
||||
response = client().prepareSearch("test").addHighlightedField(field).get();
|
||||
assertHighlight(response, 0, "text", 0, equalTo(text));
|
||||
assertHighlight(response, 0, "text", 0, 1, equalTo(text));
|
||||
|
||||
// Same for the fvh
|
||||
field.highlighterType("fvh");
|
||||
response = client().prepareSearch("test").addHighlightedField(field).get();
|
||||
assertHighlight(response, 0, "text", 0, equalTo(text));
|
||||
assertHighlight(response, 0, "text", 0, 1, equalTo(text));
|
||||
|
||||
//no difference using postings hl as the noMatchSize is ignored (just needs to be greater than 0)
|
||||
field.highlighterType("postings");
|
||||
response = client().prepareSearch("test").addHighlightedField(field).get();
|
||||
assertHighlight(response, 0, "text", 0, 1, equalTo("I am pretty long so some of me should get cut off."));
|
||||
|
||||
// We can also ask for a fragment exactly the size of the input field and get the whole field
|
||||
field.highlighterType("plain").noMatchSize(text.length());
|
||||
response = client().prepareSearch("test").addHighlightedField(field).get();
|
||||
assertHighlight(response, 0, "text", 0, equalTo(text));
|
||||
assertHighlight(response, 0, "text", 0, 1, equalTo(text));
|
||||
|
||||
// Same for the fvh
|
||||
field.highlighterType("fvh");
|
||||
response = client().prepareSearch("test").addHighlightedField(field).get();
|
||||
assertHighlight(response, 0, "text", 0, equalTo(text));
|
||||
assertHighlight(response, 0, "text", 0, 1, equalTo(text));
|
||||
|
||||
//no difference using postings hl as the noMatchSize is ignored (just needs to be greater than 0)
|
||||
field.highlighterType("postings");
|
||||
response = client().prepareSearch("test").addHighlightedField(field).get();
|
||||
assertHighlight(response, 0, "text", 0, 1, equalTo("I am pretty long so some of me should get cut off."));
|
||||
|
||||
// You can set noMatchSize globally in the highlighter as well
|
||||
field.highlighterType("plain").noMatchSize(null);
|
||||
response = client().prepareSearch("test").setHighlighterNoMatchSize(21).addHighlightedField(field).get();
|
||||
assertHighlight(response, 0, "text", 0, equalTo("I am pretty long so"));
|
||||
assertHighlight(response, 0, "text", 0, 1, equalTo("I am pretty long so"));
|
||||
|
||||
// Same for the fvh
|
||||
field.highlighterType("fvh");
|
||||
response = client().prepareSearch("test").setHighlighterNoMatchSize(21).addHighlightedField(field).get();
|
||||
assertHighlight(response, 0, "text", 0, equalTo("I am pretty long so some"));
|
||||
assertHighlight(response, 0, "text", 0, 1, equalTo("I am pretty long so some"));
|
||||
|
||||
field.highlighterType("postings");
|
||||
response = client().prepareSearch("test").setHighlighterNoMatchSize(21).addHighlightedField(field).get();
|
||||
assertHighlight(response, 0, "text", 0, 1, equalTo("I am pretty long so some of me should get cut off."));
|
||||
|
||||
// We don't break if noMatchSize is less than zero though
|
||||
field.highlighterType("plain").noMatchSize(randomIntBetween(Integer.MIN_VALUE, -1));
|
||||
response = client().prepareSearch("test").addHighlightedField(field).get();
|
||||
assertNotHighlighted(response, 0, "text");
|
||||
|
||||
field.highlighterType("fvh");
|
||||
response = client().prepareSearch("test").addHighlightedField(field).get();
|
||||
assertNotHighlighted(response, 0, "text");
|
||||
|
||||
field.highlighterType("postings");
|
||||
response = client().prepareSearch("test").addHighlightedField(field).get();
|
||||
assertNotHighlighted(response, 0, "text");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHighlightNoMatchSizeWithMultivaluedFields() throws IOException {
|
||||
prepareCreate("test")
|
||||
.addMapping("type1", "text", "type=string," + randomStoreField() + "term_vector=with_positions_offsets")
|
||||
.addMapping("type1", "text", "type=string," + randomStoreField() + "term_vector=with_positions_offsets,index_options=offsets")
|
||||
.get();
|
||||
ensureGreen();
|
||||
|
||||
String text1 = "I am pretty long so some of me should get cut off";
|
||||
String text1 = "I am pretty long so some of me should get cut off. We'll see how that goes.";
|
||||
String text2 = "I am short";
|
||||
index("test", "type1", "1", "text", new String[] {text1, text2});
|
||||
refresh();
|
||||
|
@ -1751,56 +1869,85 @@ public class HighlighterSearchTests extends AbstractIntegrationTest {
|
|||
.highlighterType("plain")
|
||||
.noMatchSize(21);
|
||||
SearchResponse response = client().prepareSearch("test").addHighlightedField(field).get();
|
||||
assertHighlight(response, 0, "text", 0, equalTo("I am pretty long so"));
|
||||
assertHighlight(response, 0, "text", 0, 1, equalTo("I am pretty long so"));
|
||||
|
||||
// And the fvh should work as well
|
||||
field.highlighterType("fvh");
|
||||
response = client().prepareSearch("test").addHighlightedField(field).get();
|
||||
assertHighlight(response, 0, "text", 0, equalTo("I am pretty long so some"));
|
||||
assertHighlight(response, 0, "text", 0, 1, equalTo("I am pretty long so some"));
|
||||
|
||||
// Postings hl also works but the fragment is the whole first sentence (size ignored)
|
||||
field.highlighterType("postings");
|
||||
response = client().prepareSearch("test").addHighlightedField(field).get();
|
||||
assertHighlight(response, 0, "text", 0, 1, equalTo("I am pretty long so some of me should get cut off."));
|
||||
|
||||
// And noMatchSize returns nothing when the first entry is empty string!
|
||||
index("test", "type1", "2", "text", new String[] {"", text2});
|
||||
refresh();
|
||||
|
||||
IdsQueryBuilder idsQueryBuilder = QueryBuilders.idsQuery("type1").addIds("2");
|
||||
field.highlighterType("plain");
|
||||
response = client().prepareSearch("test")
|
||||
.setQuery(QueryBuilders.idsQuery("type1").addIds("2"))
|
||||
.setQuery(idsQueryBuilder)
|
||||
.addHighlightedField(field).get();
|
||||
assertNotHighlighted(response, 0, "text");
|
||||
|
||||
// And the fvh should do the same
|
||||
field.highlighterType("fvh");
|
||||
response = client().prepareSearch("test").addHighlightedField(field).get();
|
||||
response = client().prepareSearch("test")
|
||||
.setQuery(idsQueryBuilder)
|
||||
.addHighlightedField(field).get();
|
||||
assertNotHighlighted(response, 0, "text");
|
||||
|
||||
field.highlighterType("postings");
|
||||
response = client().prepareSearch("test")
|
||||
.setQuery(idsQueryBuilder)
|
||||
.addHighlightedField(field).get();
|
||||
assertNotHighlighted(response, 0, "text");
|
||||
|
||||
// But if the field was actually empty then you should get no highlighting field
|
||||
index("test", "type1", "3", "text", new String[] {});
|
||||
refresh();
|
||||
idsQueryBuilder = QueryBuilders.idsQuery("type1").addIds("3");
|
||||
field.highlighterType("plain");
|
||||
response = client().prepareSearch("test")
|
||||
.setQuery(QueryBuilders.idsQuery("type1").addIds("3"))
|
||||
.setQuery(idsQueryBuilder)
|
||||
.addHighlightedField(field).get();
|
||||
assertNotHighlighted(response, 0, "text");
|
||||
|
||||
// And the fvh should do the same
|
||||
field.highlighterType("fvh");
|
||||
response = client().prepareSearch("test").addHighlightedField(field).get();
|
||||
response = client().prepareSearch("test")
|
||||
.setQuery(idsQueryBuilder)
|
||||
.addHighlightedField(field).get();
|
||||
assertNotHighlighted(response, 0, "text");
|
||||
|
||||
field.highlighterType("postings");
|
||||
response = client().prepareSearch("test")
|
||||
.setQuery(idsQueryBuilder)
|
||||
.addHighlightedField(field).get();
|
||||
assertNotHighlighted(response, 0, "text");
|
||||
|
||||
// Same for if the field doesn't even exist on the document
|
||||
index("test", "type1", "4");
|
||||
refresh();
|
||||
|
||||
idsQueryBuilder = QueryBuilders.idsQuery("type1").addIds("4");
|
||||
field.highlighterType("plain");
|
||||
response = client().prepareSearch("test")
|
||||
.setQuery(QueryBuilders.idsQuery("type1").addIds("4"))
|
||||
.setQuery(idsQueryBuilder)
|
||||
.addHighlightedField(field).get();
|
||||
assertNotHighlighted(response, 0, "text");
|
||||
|
||||
// And the fvh should do the same
|
||||
field.highlighterType("fvh");
|
||||
response = client().prepareSearch("test").addHighlightedField(field).get();
|
||||
response = client().prepareSearch("test")
|
||||
.setQuery(idsQueryBuilder)
|
||||
.addHighlightedField(field).get();
|
||||
assertNotHighlighted(response, 0, "text");
|
||||
|
||||
field.highlighterType("fvh");
|
||||
response = client().prepareSearch("test")
|
||||
.setQuery(idsQueryBuilder)
|
||||
.addHighlightedField(field).get();
|
||||
assertNotHighlighted(response, 0, "postings");
|
||||
|
||||
// Again same if the field isn't mapped
|
||||
field = new HighlightBuilder.Field("unmapped")
|
||||
.highlighterType("plain")
|
||||
|
@ -1808,9 +1955,573 @@ public class HighlighterSearchTests extends AbstractIntegrationTest {
|
|||
response = client().prepareSearch("test").addHighlightedField(field).get();
|
||||
assertNotHighlighted(response, 0, "text");
|
||||
|
||||
// And the fvh should work as well
|
||||
field.highlighterType("fvh");
|
||||
response = client().prepareSearch("test").addHighlightedField(field).get();
|
||||
assertNotHighlighted(response, 0, "text");
|
||||
|
||||
field.highlighterType("postings");
|
||||
response = client().prepareSearch("test").addHighlightedField(field).get();
|
||||
assertNotHighlighted(response, 0, "text");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHighlightNoMatchSizeNumberOfFragments() throws IOException {
|
||||
prepareCreate("test")
|
||||
.addMapping("type1", "text", "type=string," + randomStoreField() + "term_vector=with_positions_offsets,index_options=offsets")
|
||||
.get();
|
||||
ensureGreen();
|
||||
|
||||
String text1 = "This is the first sentence. This is the second sentence.";
|
||||
String text2 = "This is the third sentence. This is the fourth sentence.";
|
||||
String text3 = "This is the fifth sentence";
|
||||
index("test", "type1", "1", "text", new String[] {text1, text2, text3});
|
||||
refresh();
|
||||
|
||||
// The no match fragment should come from the first value of a multi-valued field
|
||||
HighlightBuilder.Field field = new HighlightBuilder.Field("text")
|
||||
.fragmentSize(1)
|
||||
.numOfFragments(0)
|
||||
.highlighterType("plain")
|
||||
.noMatchSize(20);
|
||||
SearchResponse response = client().prepareSearch("test").addHighlightedField(field).get();
|
||||
assertHighlight(response, 0, "text", 0, 1, equalTo("This is the first"));
|
||||
|
||||
field.highlighterType("fvh");
|
||||
response = client().prepareSearch("test").addHighlightedField(field).get();
|
||||
assertHighlight(response, 0, "text", 0, 1, equalTo("This is the first sentence"));
|
||||
|
||||
// Postings hl also works but the fragment is the whole first sentence (size ignored)
|
||||
field.highlighterType("postings");
|
||||
response = client().prepareSearch("test").addHighlightedField(field).get();
|
||||
assertHighlight(response, 0, "text", 0, 1, equalTo("This is the first sentence."));
|
||||
|
||||
//if there's a match we only return the values with matches (whole value as number_of_fragments == 0)
|
||||
MatchQueryBuilder queryBuilder = QueryBuilders.matchQuery("text", "third fifth");
|
||||
field.highlighterType("plain");
|
||||
response = client().prepareSearch("test").setQuery(queryBuilder).addHighlightedField(field).get();
|
||||
assertHighlight(response, 0, "text", 0, 2, equalTo("This is the <em>third</em> sentence. This is the fourth sentence."));
|
||||
assertHighlight(response, 0, "text", 1, 2, equalTo("This is the <em>fifth</em> sentence"));
|
||||
|
||||
field.highlighterType("fvh");
|
||||
response = client().prepareSearch("test").setQuery(queryBuilder).addHighlightedField(field).get();
|
||||
assertHighlight(response, 0, "text", 0, 2, equalTo("This is the <em>third</em> sentence. This is the fourth sentence."));
|
||||
assertHighlight(response, 0, "text", 1, 2, equalTo("This is the <em>fifth</em> sentence"));
|
||||
|
||||
field.highlighterType("postings");
|
||||
response = client().prepareSearch("test").setQuery(queryBuilder).addHighlightedField(field).get();
|
||||
assertHighlight(response, 0, "text", 0, 2, equalTo("This is the <em>third</em> sentence. This is the fourth sentence."));
|
||||
assertHighlight(response, 0, "text", 1, 2, equalTo("This is the <em>fifth</em> sentence"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPostingsHighlighter() throws Exception {
|
||||
client().admin().indices().prepareCreate("test").addMapping("type1", type1PostingsffsetsMapping()).get();
|
||||
ensureGreen();
|
||||
|
||||
client().prepareIndex("test", "type1")
|
||||
.setSource("field1", "this is a test", "field2", "The quick brown fox jumps over the lazy dog").setRefresh(true).get();
|
||||
|
||||
logger.info("--> highlighting and searching on field1");
|
||||
SearchSourceBuilder source = searchSource()
|
||||
.query(termQuery("field1", "test"))
|
||||
.highlight(highlight().field("field1").preTags("<xxx>").postTags("</xxx>"));
|
||||
|
||||
SearchResponse searchResponse = client().search(searchRequest("test").source(source)).actionGet();
|
||||
assertHitCount(searchResponse, 1l);
|
||||
|
||||
assertThat(searchResponse.getHits().getAt(0).highlightFields().get("field1").fragments()[0].string(), equalTo("this is a <xxx>test</xxx>"));
|
||||
|
||||
logger.info("--> searching on _all, highlighting on field1");
|
||||
source = searchSource()
|
||||
.query(termQuery("_all", "test"))
|
||||
.highlight(highlight().field("field1").preTags("<xxx>").postTags("</xxx>"));
|
||||
|
||||
searchResponse = client().search(searchRequest("test").source(source)).actionGet();
|
||||
assertHitCount(searchResponse, 1l);
|
||||
|
||||
assertThat(searchResponse.getHits().getAt(0).highlightFields().get("field1").fragments()[0].string(), equalTo("this is a <xxx>test</xxx>"));
|
||||
|
||||
logger.info("--> searching on _all, highlighting on field2");
|
||||
source = searchSource()
|
||||
.query(termQuery("_all", "quick"))
|
||||
.highlight(highlight().field("field2").order("score").preTags("<xxx>").postTags("</xxx>"));
|
||||
|
||||
searchResponse = client().search(searchRequest("test").source(source)).actionGet();
|
||||
assertHitCount(searchResponse, 1l);
|
||||
|
||||
assertThat(searchResponse.getHits().getAt(0).highlightFields().get("field2").fragments()[0].string(), equalTo("The <xxx>quick</xxx> brown fox jumps over the lazy dog"));
|
||||
|
||||
logger.info("--> searching on _all, highlighting on field2");
|
||||
source = searchSource()
|
||||
.query(prefixQuery("_all", "qui"))
|
||||
.highlight(highlight().field("field2").preTags("<xxx>").postTags("</xxx>"));
|
||||
|
||||
searchResponse = client().search(searchRequest("test").source(source)).actionGet();
|
||||
assertHitCount(searchResponse, 1l);
|
||||
//no snippets produced for prefix query, not supported by postings highlighter
|
||||
assertThat(searchResponse.getHits().getAt(0).highlightFields().size(), equalTo(0));
|
||||
|
||||
//lets fall back to the standard highlighter then, what people would do with unsupported queries
|
||||
logger.info("--> searching on _all, highlighting on field2, falling back to the plain highlighter");
|
||||
source = searchSource()
|
||||
.query(prefixQuery("_all", "qui"))
|
||||
.highlight(highlight().field("field2").preTags("<xxx>").postTags("</xxx>").highlighterType("highlighter"));
|
||||
|
||||
searchResponse = client().search(searchRequest("test").source(source)).actionGet();
|
||||
assertHitCount(searchResponse, 1l);
|
||||
|
||||
assertThat(searchResponse.getHits().getAt(0).highlightFields().get("field2").fragments()[0].string(), equalTo("The <xxx>quick</xxx> brown fox jumps over the lazy dog"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPostingsHighlighterMultipleFields() throws Exception {
|
||||
client().admin().indices().prepareCreate("test").addMapping("type1", type1PostingsffsetsMapping()).get();
|
||||
ensureGreen();
|
||||
|
||||
client().prepareIndex("test", "type1")
|
||||
.setSource("field1", "this is a test1", "field2", "this is a test2", "field3", "this is a test3").setRefresh(true).get();
|
||||
|
||||
logger.info("--> highlighting and searching on field1");
|
||||
SearchSourceBuilder source = searchSource()
|
||||
.query(boolQuery()
|
||||
.should(termQuery("field1", "test1"))
|
||||
.should(termQuery("field2", "test2"))
|
||||
.should(termQuery("field3", "test3")))
|
||||
.highlight(highlight().preTags("<xxx>").postTags("</xxx>").requireFieldMatch(false)
|
||||
.field("field1").field("field2").field(new HighlightBuilder.Field("field3").preTags("<x3>").postTags("</x3>")));
|
||||
|
||||
SearchResponse searchResponse = client().search(searchRequest("test").source(source)).actionGet();
|
||||
assertHitCount(searchResponse, 1l);
|
||||
|
||||
assertThat(searchResponse.getHits().getAt(0).highlightFields().get("field1").fragments()[0].string(), equalTo("this is a <xxx>test1</xxx>"));
|
||||
assertThat(searchResponse.getHits().getAt(0).highlightFields().get("field2").fragments()[0].string(), equalTo("this is a <xxx>test2</xxx>"));
|
||||
assertThat(searchResponse.getHits().getAt(0).highlightFields().get("field3").fragments()[0].string(), equalTo("this is a <x3>test3</x3>"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPostingsHighlighterNumberOfFragments() throws Exception {
|
||||
client().admin().indices().prepareCreate("test").addMapping("type1", type1PostingsffsetsMapping()).get();
|
||||
ensureGreen();
|
||||
|
||||
client().prepareIndex("test", "type1", "1")
|
||||
.setSource("field1", "The quick brown fox jumps over the lazy dog. The lazy red fox jumps over the quick dog. The quick brown dog jumps over the lazy fox.",
|
||||
"field2", "The quick brown fox jumps over the lazy dog. The lazy red fox jumps over the quick dog. The quick brown dog jumps over the lazy fox.")
|
||||
.setRefresh(true).get();
|
||||
|
||||
logger.info("--> highlighting and searching on field1");
|
||||
SearchSourceBuilder source = searchSource()
|
||||
.query(termQuery("field1", "fox"))
|
||||
.highlight(highlight()
|
||||
.field(new HighlightBuilder.Field("field1").numOfFragments(5).preTags("<field1>").postTags("</field1>"))
|
||||
.field(new HighlightBuilder.Field("field2").numOfFragments(2).preTags("<field2>").postTags("</field2>")));
|
||||
|
||||
SearchResponse searchResponse = client().search(searchRequest("test").source(source)).actionGet();
|
||||
assertHitCount(searchResponse, 1l);
|
||||
|
||||
|
||||
Map<String,HighlightField> highlightFieldMap = searchResponse.getHits().getAt(0).highlightFields();
|
||||
assertThat(highlightFieldMap.size(), equalTo(2));
|
||||
HighlightField field1 = highlightFieldMap.get("field1");
|
||||
assertThat(field1.fragments().length, equalTo(3));
|
||||
assertThat(field1.fragments()[0].string(), equalTo("The quick brown <field1>fox</field1> jumps over the lazy dog."));
|
||||
assertThat(field1.fragments()[1].string(), equalTo("The lazy red <field1>fox</field1> jumps over the quick dog."));
|
||||
assertThat(field1.fragments()[2].string(), equalTo("The quick brown dog jumps over the lazy <field1>fox</field1>."));
|
||||
|
||||
HighlightField field2 = highlightFieldMap.get("field2");
|
||||
assertThat(field2.fragments().length, equalTo(2));
|
||||
assertThat(field2.fragments()[0].string(), equalTo("The quick brown <field2>fox</field2> jumps over the lazy dog."));
|
||||
assertThat(field2.fragments()[1].string(), equalTo("The lazy red <field2>fox</field2> jumps over the quick dog."));
|
||||
|
||||
|
||||
client().prepareIndex("test", "type1", "2")
|
||||
.setSource("field1", new String[]{"The quick brown fox jumps over the lazy dog. Second sentence not finished", "The lazy red fox jumps over the quick dog.", "The quick brown dog jumps over the lazy fox."})
|
||||
.setRefresh(true).get();
|
||||
|
||||
source = searchSource()
|
||||
.query(termQuery("field1", "fox"))
|
||||
.highlight(highlight()
|
||||
.field(new HighlightBuilder.Field("field1").numOfFragments(0).preTags("<field1>").postTags("</field1>")));
|
||||
|
||||
searchResponse = client().search(searchRequest("test").source(source)).actionGet();
|
||||
assertHitCount(searchResponse, 2l);
|
||||
|
||||
for (SearchHit searchHit : searchResponse.getHits()) {
|
||||
highlightFieldMap = searchHit.highlightFields();
|
||||
assertThat(highlightFieldMap.size(), equalTo(1));
|
||||
field1 = highlightFieldMap.get("field1");
|
||||
assertThat(field1, notNullValue());
|
||||
if ("1".equals(searchHit.id())) {
|
||||
assertThat(field1.fragments().length, equalTo(1));
|
||||
assertThat(field1.fragments()[0].string(), equalTo("The quick brown <field1>fox</field1> jumps over the lazy dog. The lazy red <field1>fox</field1> jumps over the quick dog. The quick brown dog jumps over the lazy <field1>fox</field1>."));
|
||||
} else if ("2".equals(searchHit.id())) {
|
||||
assertThat(field1.fragments().length, equalTo(3));
|
||||
assertThat(field1.fragments()[0].string(), equalTo("The quick brown <field1>fox</field1> jumps over the lazy dog. Second sentence not finished"));
|
||||
assertThat(field1.fragments()[1].string(), equalTo("The lazy red <field1>fox</field1> jumps over the quick dog."));
|
||||
assertThat(field1.fragments()[2].string(), equalTo("The quick brown dog jumps over the lazy <field1>fox</field1>."));
|
||||
} else {
|
||||
fail("Only hits with id 1 and 2 are returned");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPostingsHighlighterRequireFieldMatch() throws Exception {
|
||||
client().admin().indices().prepareCreate("test").addMapping("type1", type1PostingsffsetsMapping()).get();
|
||||
ensureGreen();
|
||||
|
||||
client().prepareIndex("test", "type1")
|
||||
.setSource("field1", "The quick brown fox jumps over the lazy dog. The lazy red fox jumps over the quick dog. The quick brown dog jumps over the lazy fox.",
|
||||
"field2", "The quick brown fox jumps over the lazy dog. The lazy red fox jumps over the quick dog. The quick brown dog jumps over the lazy fox.")
|
||||
.setRefresh(true).get();
|
||||
|
||||
logger.info("--> highlighting and searching on field1");
|
||||
SearchSourceBuilder source = searchSource()
|
||||
.query(termQuery("field1", "fox"))
|
||||
.highlight(highlight()
|
||||
.field(new HighlightBuilder.Field("field1").requireFieldMatch(true).preTags("<field1>").postTags("</field1>"))
|
||||
.field(new HighlightBuilder.Field("field2").requireFieldMatch(true).preTags("<field2>").postTags("</field2>")));
|
||||
|
||||
SearchResponse searchResponse = client().search(searchRequest("test").source(source)).actionGet();
|
||||
assertHitCount(searchResponse, 1l);
|
||||
|
||||
//field2 is not returned highlighted because of the require field match option set to true
|
||||
Map<String,HighlightField> highlightFieldMap = searchResponse.getHits().getAt(0).highlightFields();
|
||||
assertThat(highlightFieldMap.size(), equalTo(1));
|
||||
HighlightField field1 = highlightFieldMap.get("field1");
|
||||
assertThat(field1.fragments().length, equalTo(3));
|
||||
assertThat(field1.fragments()[0].string(), equalTo("The quick brown <field1>fox</field1> jumps over the lazy dog."));
|
||||
assertThat(field1.fragments()[1].string(), equalTo("The lazy red <field1>fox</field1> jumps over the quick dog."));
|
||||
assertThat(field1.fragments()[2].string(), equalTo("The quick brown dog jumps over the lazy <field1>fox</field1>."));
|
||||
|
||||
|
||||
logger.info("--> highlighting and searching on field1 and field2 - require field match set to false");
|
||||
source = searchSource()
|
||||
.query(termQuery("field1", "fox"))
|
||||
.highlight(highlight()
|
||||
.field(new HighlightBuilder.Field("field1").requireFieldMatch(false).preTags("<field1>").postTags("</field1>"))
|
||||
.field(new HighlightBuilder.Field("field2").requireFieldMatch(false).preTags("<field2>").postTags("</field2>")));
|
||||
|
||||
searchResponse = client().search(searchRequest("test").source(source)).actionGet();
|
||||
assertHitCount(searchResponse, 1l);
|
||||
|
||||
//field2 is now returned highlighted thanks to the multi_match query on both fields
|
||||
highlightFieldMap = searchResponse.getHits().getAt(0).highlightFields();
|
||||
assertThat(highlightFieldMap.size(), equalTo(2));
|
||||
field1 = highlightFieldMap.get("field1");
|
||||
assertThat(field1.fragments().length, equalTo(3));
|
||||
assertThat(field1.fragments()[0].string(), equalTo("The quick brown <field1>fox</field1> jumps over the lazy dog."));
|
||||
assertThat(field1.fragments()[1].string(), equalTo("The lazy red <field1>fox</field1> jumps over the quick dog."));
|
||||
assertThat(field1.fragments()[2].string(), equalTo("The quick brown dog jumps over the lazy <field1>fox</field1>."));
|
||||
|
||||
HighlightField field2 = highlightFieldMap.get("field2");
|
||||
assertThat(field2.fragments().length, equalTo(3));
|
||||
assertThat(field2.fragments()[0].string(), equalTo("The quick brown <field2>fox</field2> jumps over the lazy dog."));
|
||||
assertThat(field2.fragments()[1].string(), equalTo("The lazy red <field2>fox</field2> jumps over the quick dog."));
|
||||
assertThat(field2.fragments()[2].string(), equalTo("The quick brown dog jumps over the lazy <field2>fox</field2>."));
|
||||
|
||||
|
||||
logger.info("--> highlighting and searching on field1 and field2 via multi_match query");
|
||||
source = searchSource()
|
||||
.query(multiMatchQuery("fox", "field1", "field2"))
|
||||
.highlight(highlight()
|
||||
.field(new HighlightBuilder.Field("field1").requireFieldMatch(true).preTags("<field1>").postTags("</field1>"))
|
||||
.field(new HighlightBuilder.Field("field2").requireFieldMatch(true).preTags("<field2>").postTags("</field2>")));
|
||||
|
||||
searchResponse = client().search(searchRequest("test").source(source)).actionGet();
|
||||
assertHitCount(searchResponse, 1l);
|
||||
|
||||
//field2 is now returned highlighted thanks to the multi_match query on both fields
|
||||
highlightFieldMap = searchResponse.getHits().getAt(0).highlightFields();
|
||||
assertThat(highlightFieldMap.size(), equalTo(2));
|
||||
field1 = highlightFieldMap.get("field1");
|
||||
assertThat(field1.fragments().length, equalTo(3));
|
||||
assertThat(field1.fragments()[0].string(), equalTo("The quick brown <field1>fox</field1> jumps over the lazy dog."));
|
||||
assertThat(field1.fragments()[1].string(), equalTo("The lazy red <field1>fox</field1> jumps over the quick dog."));
|
||||
assertThat(field1.fragments()[2].string(), equalTo("The quick brown dog jumps over the lazy <field1>fox</field1>."));
|
||||
|
||||
field2 = highlightFieldMap.get("field2");
|
||||
assertThat(field2.fragments().length, equalTo(3));
|
||||
assertThat(field2.fragments()[0].string(), equalTo("The quick brown <field2>fox</field2> jumps over the lazy dog."));
|
||||
assertThat(field2.fragments()[1].string(), equalTo("The lazy red <field2>fox</field2> jumps over the quick dog."));
|
||||
assertThat(field2.fragments()[2].string(), equalTo("The quick brown dog jumps over the lazy <field2>fox</field2>."));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPostingsHighlighterOrderByScore() throws Exception {
|
||||
client().admin().indices().prepareCreate("test").addMapping("type1", type1PostingsffsetsMapping()).get();
|
||||
ensureGreen();
|
||||
|
||||
client().prepareIndex("test", "type1")
|
||||
.setSource("field1", new String[]{"This sentence contains one match, not that short. This sentence contains two sentence matches. This one contains no matches.",
|
||||
"This is the second value's first sentence. This one contains no matches. This sentence contains three sentence occurrences (sentence).",
|
||||
"One sentence match here and scored lower since the text is quite long, not that appealing. This one contains no matches."})
|
||||
.setRefresh(true).get();
|
||||
|
||||
logger.info("--> highlighting and searching on field1");
|
||||
SearchSourceBuilder source = searchSource()
|
||||
.query(termQuery("field1", "sentence"))
|
||||
.highlight(highlight().field("field1").order("score"));
|
||||
|
||||
SearchResponse searchResponse = client().search(searchRequest("test").source(source)).actionGet();
|
||||
assertHitCount(searchResponse, 1l);
|
||||
|
||||
Map<String,HighlightField> highlightFieldMap = searchResponse.getHits().getAt(0).highlightFields();
|
||||
assertThat(highlightFieldMap.size(), equalTo(1));
|
||||
HighlightField field1 = highlightFieldMap.get("field1");
|
||||
assertThat(field1.fragments().length, equalTo(5));
|
||||
assertThat(field1.fragments()[0].string(), equalTo("This <em>sentence</em> contains three <em>sentence</em> occurrences (<em>sentence</em>)."));
|
||||
assertThat(field1.fragments()[1].string(), equalTo("This <em>sentence</em> contains two <em>sentence</em> matches."));
|
||||
assertThat(field1.fragments()[2].string(), equalTo("This is the second value's first <em>sentence</em>."));
|
||||
assertThat(field1.fragments()[3].string(), equalTo("This <em>sentence</em> contains one match, not that short."));
|
||||
assertThat(field1.fragments()[4].string(), equalTo("One <em>sentence</em> match here and scored lower since the text is quite long, not that appealing."));
|
||||
|
||||
//lets use now number_of_fragments = 0, so that we highlight per value without breaking them into snippets, but we sort the values by score
|
||||
source = searchSource()
|
||||
.query(termQuery("field1", "sentence"))
|
||||
.highlight(highlight().field("field1", -1, 0).order("score"));
|
||||
|
||||
searchResponse = client().search(searchRequest("test").source(source)).actionGet();
|
||||
assertHitCount(searchResponse, 1l);
|
||||
|
||||
highlightFieldMap = searchResponse.getHits().getAt(0).highlightFields();
|
||||
assertThat(highlightFieldMap.size(), equalTo(1));
|
||||
field1 = highlightFieldMap.get("field1");
|
||||
assertThat(field1.fragments().length, equalTo(3));
|
||||
assertThat(field1.fragments()[0].string(), equalTo("This is the second value's first <em>sentence</em>. This one contains no matches. This <em>sentence</em> contains three <em>sentence</em> occurrences (<em>sentence</em>)."));
|
||||
assertThat(field1.fragments()[1].string(), equalTo("This <em>sentence</em> contains one match, not that short. This <em>sentence</em> contains two <em>sentence</em> matches. This one contains no matches."));
|
||||
assertThat(field1.fragments()[2].string(), equalTo("One <em>sentence</em> match here and scored lower since the text is quite long, not that appealing. This one contains no matches."));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPostingsHighlighterEscapeHtml() throws Exception {
|
||||
client().admin().indices().prepareCreate("test").setSettings(ImmutableSettings.settingsBuilder().put("index.number_of_shards", 2))
|
||||
.addMapping("type1", jsonBuilder().startObject().startObject("type1").startObject("properties")
|
||||
.startObject("title").field("type", "string").field("store", "yes").field("index_options", "offsets").endObject()
|
||||
.endObject().endObject().endObject())
|
||||
.get();
|
||||
ensureYellow();
|
||||
for (int i = 0; i < 5; i++) {
|
||||
client().prepareIndex("test", "type1", Integer.toString(i))
|
||||
.setSource("title", "This is a html escaping highlighting test for *&? elasticsearch").setRefresh(true).execute().actionGet();
|
||||
}
|
||||
|
||||
SearchResponse searchResponse = client().prepareSearch()
|
||||
.setQuery(matchQuery("title", "test"))
|
||||
.setHighlighterEncoder("html")
|
||||
.addHighlightedField("title").get();
|
||||
|
||||
assertHitCount(searchResponse, 5l);
|
||||
assertThat(searchResponse.getHits().hits().length, equalTo(5));
|
||||
|
||||
for (SearchHit hit : searchResponse.getHits()) {
|
||||
assertThat(hit.highlightFields().get("title").fragments()[0].string(), equalTo("This is a html escaping highlighting <em>test</em> for *&?"));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPostingsHighlighterMultiMapperWithStore() throws Exception {
|
||||
client().admin().indices().prepareCreate("test").setSettings(ImmutableSettings.settingsBuilder().put("index.number_of_shards", 2))
|
||||
.addMapping("type1", jsonBuilder().startObject().startObject("type1")
|
||||
//just to make sure that we hit the stored fields rather than the _source
|
||||
.startObject("_source").field("enabled", false).endObject()
|
||||
.startObject("properties")
|
||||
.startObject("title").field("type", "multi_field").startObject("fields")
|
||||
.startObject("title").field("type", "string").field("store", "yes").field("index_options", "offsets").endObject()
|
||||
.startObject("key").field("type", "string").field("store", "yes").field("index_options", "offsets").field("analyzer", "whitespace").endObject()
|
||||
.endObject().endObject()
|
||||
.endObject().endObject().endObject())
|
||||
.execute().actionGet();
|
||||
ensureGreen();
|
||||
client().prepareIndex("test", "type1", "1").setSource("title", "this is a test . Second sentence.").get();
|
||||
refresh();
|
||||
// simple search on body with standard analyzer with a simple field query
|
||||
SearchResponse searchResponse = client().prepareSearch()
|
||||
//lets make sure we analyze the query and we highlight the resulting terms
|
||||
.setQuery(matchQuery("title", "This is a Test"))
|
||||
.addHighlightedField("title").get();
|
||||
|
||||
assertHitCount(searchResponse, 1l);
|
||||
SearchHit hit = searchResponse.getHits().getAt(0);
|
||||
assertThat(hit.source(), nullValue());
|
||||
|
||||
//stopwords are not highlighted since not indexed
|
||||
assertThat(hit.highlightFields().get("title").fragments()[0].string(), equalTo("this is a <em>test</em> ."));
|
||||
|
||||
// search on title.key and highlight on title
|
||||
searchResponse = client().prepareSearch()
|
||||
.setQuery(matchQuery("title.key", "this is a test"))
|
||||
.addHighlightedField("title.key").get();
|
||||
assertHitCount(searchResponse, 1l);
|
||||
|
||||
hit = searchResponse.getHits().getAt(0);
|
||||
//stopwords are now highlighted since we used only whitespace analyzer here
|
||||
assertThat(hit.highlightFields().get("title.key").fragments()[0].string(), equalTo("<em>this</em> <em>is</em> <em>a</em> <em>test</em> ."));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPostingsHighlighterMultiMapperFromSource() throws Exception {
|
||||
client().admin().indices().prepareCreate("test").setSettings(ImmutableSettings.settingsBuilder().put("index.number_of_shards", 2))
|
||||
.addMapping("type1", jsonBuilder().startObject().startObject("type1").startObject("properties")
|
||||
.startObject("title").field("type", "multi_field").startObject("fields")
|
||||
.startObject("title").field("type", "string").field("store", "no").field("index_options", "offsets").endObject()
|
||||
.startObject("key").field("type", "string").field("store", "no").field("index_options", "offsets").field("analyzer", "whitespace").endObject()
|
||||
.endObject().endObject()
|
||||
.endObject().endObject().endObject())
|
||||
.get();
|
||||
ensureGreen();
|
||||
|
||||
client().prepareIndex("test", "type1", "1").setSource("title", "this is a test").get();
|
||||
refresh();
|
||||
|
||||
// simple search on body with standard analyzer with a simple field query
|
||||
SearchResponse searchResponse = client().prepareSearch()
|
||||
.setQuery(matchQuery("title", "this is a test"))
|
||||
.addHighlightedField("title")
|
||||
.execute().actionGet();
|
||||
|
||||
assertHitCount(searchResponse, 1l);
|
||||
|
||||
SearchHit hit = searchResponse.getHits().getAt(0);
|
||||
assertThat(hit.highlightFields().get("title").fragments()[0].string(), equalTo("this is a <em>test</em>"));
|
||||
|
||||
// search on title.key and highlight on title.key
|
||||
searchResponse = client().prepareSearch()
|
||||
.setQuery(matchQuery("title.key", "this is a test"))
|
||||
.addHighlightedField("title.key")
|
||||
.get();
|
||||
assertHitCount(searchResponse, 1l);
|
||||
|
||||
hit = searchResponse.getHits().getAt(0);
|
||||
assertThat(hit.highlightFields().get("title.key").fragments()[0].string(), equalTo("<em>this</em> <em>is</em> <em>a</em> <em>test</em>"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPostingsHighlighterShouldFailIfNoOffsets() throws Exception {
|
||||
client().admin().indices().prepareCreate("test").setSettings(ImmutableSettings.settingsBuilder().put("index.number_of_shards", 2))
|
||||
.addMapping("type1", jsonBuilder().startObject().startObject("type1").startObject("properties")
|
||||
.startObject("title").field("type", "string").field("store", "yes").field("index_options", "docs").endObject()
|
||||
.endObject().endObject().endObject())
|
||||
.get();
|
||||
ensureGreen();
|
||||
|
||||
for (int i = 0; i < 5; i++) {
|
||||
client().prepareIndex("test", "type1", Integer.toString(i))
|
||||
.setSource("title", "This is a test for the postings highlighter").setRefresh(true).get();
|
||||
}
|
||||
refresh();
|
||||
|
||||
SearchResponse search = client().prepareSearch()
|
||||
.setQuery(matchQuery("title", "this is a test"))
|
||||
.addHighlightedField("title")
|
||||
.get();
|
||||
assertNoFailures(search);
|
||||
|
||||
search = client().prepareSearch()
|
||||
.setQuery(matchQuery("title", "this is a test"))
|
||||
.addHighlightedField("title")
|
||||
.setHighlighterType("postings-highlighter")
|
||||
.get();
|
||||
assertThat(search.getFailedShards(), equalTo(2));
|
||||
for (ShardSearchFailure shardSearchFailure : search.getShardFailures()) {
|
||||
assertThat(shardSearchFailure.reason(), containsString("the field [title] should be indexed with positions and offsets in the postings list to be used with postings highlighter"));
|
||||
}
|
||||
|
||||
search = client().prepareSearch()
|
||||
.setQuery(matchQuery("title", "this is a test"))
|
||||
.addHighlightedField("title")
|
||||
.setHighlighterType("postings")
|
||||
.get();
|
||||
|
||||
assertThat(search.getFailedShards(), equalTo(2));
|
||||
for (ShardSearchFailure shardSearchFailure : search.getShardFailures()) {
|
||||
assertThat(shardSearchFailure.reason(), containsString("the field [title] should be indexed with positions and offsets in the postings list to be used with postings highlighter"));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPostingsHighlighterBoostingQuery() throws ElasticSearchException, IOException {
|
||||
client().admin().indices().prepareCreate("test").addMapping("type1", type1PostingsffsetsMapping()).get();
|
||||
ensureGreen();
|
||||
client().prepareIndex("test", "type1").setSource("field1", "this is a test", "field2", "The quick brown fox jumps over the lazy dog! Second sentence.")
|
||||
.get();
|
||||
refresh();
|
||||
|
||||
logger.info("--> highlighting and searching on field1");
|
||||
SearchSourceBuilder source = searchSource()
|
||||
.query(boostingQuery().positive(termQuery("field2", "brown")).negative(termQuery("field2", "foobar")).negativeBoost(0.5f))
|
||||
.highlight(highlight().field("field2").preTags("<x>").postTags("</x>"));
|
||||
|
||||
SearchResponse searchResponse = client().search(searchRequest("test").source(source)).actionGet();
|
||||
assertHitCount(searchResponse, 1l);
|
||||
|
||||
assertThat(searchResponse.getHits().getAt(0).highlightFields().get("field2").fragments()[0].string(),
|
||||
equalTo("The quick <x>brown</x> fox jumps over the lazy dog!"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPostingsHighlighterCommonTermsQuery() throws ElasticSearchException, IOException {
|
||||
client().admin().indices().prepareCreate("test").addMapping("type1", type1PostingsffsetsMapping()).get();
|
||||
ensureGreen();
|
||||
|
||||
client().prepareIndex("test", "type1").setSource("field1", "this is a test", "field2", "The quick brown fox jumps over the lazy dog! Second sentence.").get();
|
||||
refresh();
|
||||
logger.info("--> highlighting and searching on field1");
|
||||
SearchSourceBuilder source = searchSource().query(commonTerms("field2", "quick brown").cutoffFrequency(100))
|
||||
.highlight(highlight().field("field2").preTags("<x>").postTags("</x>"));
|
||||
|
||||
SearchResponse searchResponse = client().search(searchRequest("test").source(source)).actionGet();
|
||||
assertHitCount(searchResponse, 1l);
|
||||
|
||||
assertThat(searchResponse.getHits().getAt(0).highlightFields().get("field2").fragments()[0].string(),
|
||||
equalTo("The <x>quick</x> <x>brown</x> fox jumps over the lazy dog!"));
|
||||
}
|
||||
|
||||
public XContentBuilder type1PostingsffsetsMapping() throws IOException {
|
||||
return XContentFactory.jsonBuilder().startObject().startObject("type1")
|
||||
.startObject("_all").field("store", "yes").field("index_options", "offsets").endObject()
|
||||
.startObject("properties")
|
||||
.startObject("field1").field("type", "string").field("index_options", "offsets").endObject()
|
||||
.startObject("field2").field("type", "string").field("index_options", "offsets").endObject()
|
||||
.endObject()
|
||||
.endObject().endObject();
|
||||
}
|
||||
|
||||
@Test
|
||||
@Slow
|
||||
public void testPostingsHighlighterManyDocs() throws Exception {
|
||||
client().admin().indices().prepareCreate("test").addMapping("type1", type1PostingsffsetsMapping()).get();
|
||||
ensureGreen();
|
||||
|
||||
int COUNT = between(20, 100);
|
||||
logger.info("--> indexing docs");
|
||||
for (int i = 0; i < COUNT; i++) {
|
||||
client().prepareIndex("test", "type1", Integer.toString(i)).setSource("field1", "Sentence test " + i + ". Sentence two.").get();
|
||||
}
|
||||
refresh();
|
||||
|
||||
logger.info("--> searching explicitly on field1 and highlighting on it");
|
||||
SearchResponse searchResponse = client().prepareSearch()
|
||||
.setSize(COUNT)
|
||||
.setQuery(termQuery("field1", "test"))
|
||||
.addHighlightedField("field1")
|
||||
.get();
|
||||
assertHitCount(searchResponse, (long)COUNT);
|
||||
assertThat(searchResponse.getHits().hits().length, equalTo(COUNT));
|
||||
for (SearchHit hit : searchResponse.getHits()) {
|
||||
assertThat(hit.highlightFields().get("field1").fragments()[0].string(), equalTo("Sentence <em>test</em> " + hit.id() + "."));
|
||||
}
|
||||
|
||||
logger.info("--> searching explicitly on field1 and highlighting on it, with DFS");
|
||||
searchResponse = client().prepareSearch()
|
||||
.setSearchType(SearchType.DFS_QUERY_THEN_FETCH)
|
||||
.setSize(COUNT)
|
||||
.setQuery(termQuery("field1", "test"))
|
||||
.addHighlightedField("field1")
|
||||
.get();
|
||||
assertHitCount(searchResponse, (long)COUNT);
|
||||
assertThat(searchResponse.getHits().hits().length, equalTo(COUNT));
|
||||
for (SearchHit hit : searchResponse.getHits()) {
|
||||
assertThat(hit.highlightFields().get("field1").fragments()[0].string(), equalTo("Sentence <em>test</em> " + hit.id() + "."));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -187,10 +187,18 @@ public class ElasticsearchAssertions {
|
|||
}
|
||||
|
||||
public static void assertHighlight(SearchResponse resp, int hit, String field, int fragment, Matcher<String> matcher) {
|
||||
assertHighlight(resp, hit, field, fragment, greaterThan(fragment), matcher);
|
||||
}
|
||||
|
||||
public static void assertHighlight(SearchResponse resp, int hit, String field, int fragment, int totalFragments, Matcher<String> matcher) {
|
||||
assertHighlight(resp, hit, field, fragment, equalTo(totalFragments), matcher);
|
||||
}
|
||||
|
||||
private static void assertHighlight(SearchResponse resp, int hit, String field, int fragment, Matcher<Integer> fragmentsMatcher, Matcher<String> matcher) {
|
||||
assertNoFailures(resp);
|
||||
assertThat("not enough hits", resp.getHits().hits().length, greaterThan(hit));
|
||||
assertThat(resp.getHits().hits()[hit].getHighlightFields(), hasKey(field));
|
||||
assertThat(resp.getHits().hits()[hit].getHighlightFields().get(field).fragments().length, greaterThan(fragment));
|
||||
assertThat(resp.getHits().hits()[hit].getHighlightFields().get(field).fragments().length, fragmentsMatcher);
|
||||
assertThat(resp.getHits().hits()[hit].highlightFields().get(field).fragments()[fragment].string(), matcher);
|
||||
assertVersionSerializable(resp);
|
||||
}
|
||||
|
|
File diff suppressed because one or more lines are too long
Loading…
Reference in New Issue