Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Extend the Lucene highlighting API to support getting positions rather than interpolating markup #1947

Merged
merged 3 commits into from Dec 17, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Expand Up @@ -72,14 +72,12 @@
import javax.annotation.Nullable;
import java.io.IOException;
import java.io.StringReader;
import java.util.ArrayDeque;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
Expand All @@ -95,10 +93,6 @@
public class LuceneAutoCompleteResultCursor implements BaseCursor<IndexEntry> {
private static final Logger LOGGER = LoggerFactory.getLogger(LuceneAutoCompleteResultCursor.class);

private static final int tokenCountBeforeHighlighted = 3;
private static final int tokenCountAfterHighlighted = 3;
private static final String highlightedTextConnector = "... ";

@Nonnull
private final Executor executor;
@Nonnull
Expand Down Expand Up @@ -196,184 +190,6 @@ private void performLookup() throws IOException {
}
}

@SuppressWarnings("squid:S3776") // Cognitive complexity is too high. Candidate for later refactoring
@Nullable
@VisibleForTesting
static String searchAllMaybeHighlight(@Nonnull String fieldName, @Nonnull Analyzer queryAnalyzer, @Nonnull String text,
@Nonnull Set<String> matchedTokens, @Nullable String prefixToken,
boolean allMatchingRequired,
@Nonnull LuceneScanQueryParameters.LuceneQueryHighlightParameters luceneQueryHighlightParameters) {
try (TokenStream ts = queryAnalyzer.tokenStream(fieldName, new StringReader(text))) {
CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
OffsetAttribute offsetAtt = ts.addAttribute(OffsetAttribute.class);
ts.reset();
StringBuilder sb = luceneQueryHighlightParameters.isHighlight() ? new StringBuilder() : null;
int upto = 0;
Set<String> matchedInText = new HashSet<>();
boolean matchedPrefix = false;
ArrayDeque<String> pres = new ArrayDeque<>();
ArrayDeque<String> ends = new ArrayDeque<>();
int lastMatchPos = -tokenCountAfterHighlighted - 1;
int currentPos = 0;
while (ts.incrementToken()) {
String token = termAtt.toString();
int startOffset = offsetAtt.startOffset();
int endOffset = offsetAtt.endOffset();
if (upto < startOffset) {
if (luceneQueryHighlightParameters.isHighlight()) {
if (luceneQueryHighlightParameters.isCutSnippets()) {
if (currentPos - lastMatchPos <= tokenCountAfterHighlighted + 1) {
addNonMatch(sb, text.substring(upto, startOffset));
} else {
pres.add(text.substring(upto, startOffset));
if (pres.size() > tokenCountBeforeHighlighted) {
pres.poll();
}
if (ends.size() < luceneQueryHighlightParameters.getSnippedSize() - tokenCountAfterHighlighted) {
ends.add(text.substring(upto, startOffset));
}
}
} else {
addNonMatch(sb, text.substring(upto, startOffset));
}
}
upto = startOffset;
} else if (upto > startOffset) {
continue;
}

if (matchedTokens.contains(token)) {
// Token matches.
if (luceneQueryHighlightParameters.isHighlight()) {
if (luceneQueryHighlightParameters.isCutSnippets() && currentPos - lastMatchPos > tokenCountBeforeHighlighted + tokenCountAfterHighlighted + 1) {
addNonMatch(sb, highlightedTextConnector);
}
while (!pres.isEmpty()) {
addNonMatch(sb, pres.poll());
}
ends.clear();
int start = startOffset;
while (start < endOffset) {
int index = text.toLowerCase(Locale.ROOT).indexOf(token, start);
if (index < 0 || index >= endOffset) {
addNonMatch(sb, text.substring(start, endOffset));
break;
}
int actualStartOffset = index;
int actualEndOffset = index + token.length();
addNonMatch(sb, text.substring(start, index));
String substring = text.substring(actualStartOffset, actualEndOffset);
if (substring.equalsIgnoreCase(token) && !tokenAlreadyHighlighted(text, actualStartOffset, actualEndOffset,
luceneQueryHighlightParameters.getLeftTag(), luceneQueryHighlightParameters.getRightTag())) {
addWholeMatch(sb, substring,
luceneQueryHighlightParameters.getLeftTag(), luceneQueryHighlightParameters.getRightTag());
} else {
addNonMatch(sb, substring);
}
start = actualEndOffset;
}
}
upto = endOffset;
matchedInText.add(token);
lastMatchPos = currentPos;
} else if (prefixToken != null && token.startsWith(prefixToken)) {
if (luceneQueryHighlightParameters.isHighlight()) {
if (!tokenAlreadyHighlighted(text, startOffset, endOffset,
luceneQueryHighlightParameters.getLeftTag(), luceneQueryHighlightParameters.getRightTag())) {
addPrefixMatch(sb, text.substring(startOffset, endOffset), prefixToken,
luceneQueryHighlightParameters.getLeftTag(), luceneQueryHighlightParameters.getRightTag());
} else {
addNonMatch(sb, text.substring(startOffset, endOffset));
}
}
upto = endOffset;
matchedPrefix = true;
}
currentPos++;
}
ts.end();

if (allMatchingRequired && ((prefixToken != null && !matchedPrefix) || (matchedInText.size() < matchedTokens.size()))) {
// Query text not actually found in document text. Return null
return null;
}

// Text was found. Return text (highlighted or not)
if (luceneQueryHighlightParameters.isHighlight()) {
int endOffset = offsetAtt.endOffset();
if (upto < endOffset && !luceneQueryHighlightParameters.isCutSnippets()) {
addNonMatch(sb, text.substring(upto));
} else if (luceneQueryHighlightParameters.isCutSnippets()) {
while (!ends.isEmpty()) {
addNonMatch(sb, ends.poll());
}
addNonMatch(sb, highlightedTextConnector);
}
return sb.toString();
} else {
return text;
}

} catch (IOException e) {
return null;
}
}

// Check this before highlighting tokens, so the highlighting is idempotent
private static boolean tokenAlreadyHighlighted(@Nonnull String text, int startOffset, int endOffset,
@Nonnull String leftTag, @Nonnull String rightTag) {
return startOffset - leftTag.length() >= 0
&& endOffset + rightTag.length() > text.length()
&& text.startsWith(leftTag, startOffset - 3)
&& text.startsWith(rightTag, endOffset);
}

/** Called while highlighting a single result, to append a
* non-matching chunk of text from the suggestion to the
* provided fragments list.
* @param sb The {@code StringBuilder} to append to
* @param text The text chunk to add
*/
private static void addNonMatch(StringBuilder sb, String text) {
sb.append(text);
}

/** Called while highlighting a single result, to append
* the whole matched token to the provided fragments list.
* @param sb The {@code StringBuilder} to append to
* @param surface The surface form (original) text
* @param leftTag the tag to add left to the surface
* @param rightTag the tag to add right to the surface
*/
private static void addWholeMatch(StringBuilder sb, String surface, String leftTag, String rightTag) {
sb.append(leftTag);
sb.append(surface);
sb.append(rightTag);
}

/** Called while highlighting a single result, to append a
* matched prefix token, to the provided fragments list.
* @param sb The {@code StringBuilder} to append to
* @param surface The fragment of the surface form
* (indexed during build, corresponding to
* this match
* @param prefixToken The prefix of the token that matched
* @param leftTag the tag to add left to the surface
* @param rightTag the tag to add right to the surface
*/
private static void addPrefixMatch(StringBuilder sb, String surface, String prefixToken, String leftTag, String rightTag) {
// TODO: apps can try to invert their analysis logic
// here, e.g. downcase the two before checking prefix:
if (prefixToken.length() >= surface.length()) {
addWholeMatch(sb, surface, leftTag, rightTag);
return;
}
sb.append(leftTag);
sb.append(surface.substring(0, prefixToken.length()));
sb.append(rightTag);
sb.append(surface.substring(prefixToken.length()));
}

@SuppressWarnings("PMD.CloseResource")
public RecordCursor<IndexEntry> lookup() throws IOException {
// Determine the tokens from the query key
Expand Down Expand Up @@ -612,8 +428,8 @@ private RecordCursor<IndexEntry> findIndexEntriesInRecord(ScoreDocAndRecord scor
// matched terms
return null;
}
String match = searchAllMaybeHighlight(documentField.getFieldName(), queryAnalyzer, text, queryTokens, prefixToken, true,
new LuceneScanQueryParameters.LuceneQueryHighlightParameters(highlight));
String match = LuceneHighlighting.searchAllMaybeHighlight(documentField.getFieldName(), queryAnalyzer, text, queryTokens, prefixToken, true,
new LuceneScanQueryParameters.LuceneQueryHighlightParameters(highlight), null);
if (match == null) {
// Text not found in this field
return null;
Expand Down
Expand Up @@ -38,12 +38,9 @@
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Objects;
import java.util.Set;

/**
* Helper class for converting {@link FDBRecord}s to Lucene documents.
Expand Down Expand Up @@ -134,42 +131,6 @@ public static <M extends Message> List<DocumentField> getFields(@Nonnull KeyExpr
return fields.getFields();
}

// Modify the Lucene fields of a record message with highlighting the terms from the given termMap
@Nonnull
public static <M extends Message> void highlightTermsInMessage(@Nonnull KeyExpression expression, @Nonnull Message.Builder builder, @Nonnull Map<String, Set<String>> termMap,
@Nonnull LuceneAnalyzerCombinationProvider analyzerSelector,
@Nonnull LuceneScanQueryParameters.LuceneQueryHighlightParameters luceneQueryHighlightParameters) {
LuceneIndexKeyValueToPartialRecordUtils.RecordRebuildSource<M> recordRebuildSource = new LuceneIndexKeyValueToPartialRecordUtils.RecordRebuildSource<>(null, builder.getDescriptorForType(), builder, builder.build());

LuceneIndexExpressions.getFields(expression, recordRebuildSource,
(source, fieldName, value, type, stored, sorted, overriddenKeyRanges, groupingKeyIndex, keyIndex, fieldConfigsIgnored) -> {
Set<String> terms = new HashSet<>();
terms.addAll(termMap.getOrDefault(fieldName, Collections.emptySet()));
terms.addAll(termMap.getOrDefault("", Collections.emptySet()));
if (terms.isEmpty()) {
return;
}
for (Map.Entry<Descriptors.FieldDescriptor, Object> entry : source.message.getAllFields().entrySet()) {
Object entryValue = entry.getValue();
if (entryValue instanceof String && entryValue.equals(value)
&& terms.stream().filter(t -> ((String) entryValue).toLowerCase(Locale.ROOT).contains(t.toLowerCase(Locale.ROOT))).findAny().isPresent()) {
String highlightedText = LuceneAutoCompleteResultCursor.searchAllMaybeHighlight(fieldName, analyzerSelector.provideIndexAnalyzer((String) entryValue).getAnalyzer(), (String) entryValue, termMap.get(fieldName), null, false, luceneQueryHighlightParameters);
source.buildMessage(highlightedText, entry.getKey(), null, null, true, 0);
} else if (entryValue instanceof List) {
int index = 0;
for (Object entryValueElement : ((List) entryValue)) {
if (entryValueElement instanceof String && entryValueElement.equals(value)
&& terms.stream().filter(t -> ((String) entryValueElement).toLowerCase(Locale.ROOT).contains(t.toLowerCase(Locale.ROOT))).findAny().isPresent()) {
String highlightedText = LuceneAutoCompleteResultCursor.searchAllMaybeHighlight(fieldName, analyzerSelector.provideIndexAnalyzer((String) entryValueElement).getAnalyzer(), (String) entryValueElement, termMap.get(fieldName), null, false, luceneQueryHighlightParameters);
source.buildMessage(highlightedText, entry.getKey(), null, null, true, index);
}
index++;
}
}
}
}, null);
}

protected static class FDBRecordSource<M extends Message> implements LuceneIndexExpressions.RecordSource<FDBRecordSource<M>> {
@Nonnull
private final FDBRecord<M> rec;
Expand All @@ -181,6 +142,11 @@ public FDBRecordSource(@Nonnull final FDBRecord<M> rec, @Nonnull final Message m
this.message = message;
}

@Nonnull
public Message getMessage() {
return message;
}

@Override
public Descriptors.Descriptor getDescriptor() {
return message.getDescriptorForType();
Expand Down
Expand Up @@ -22,15 +22,11 @@

import com.apple.foundationdb.record.EvaluationContext;
import com.apple.foundationdb.record.ExecuteProperties;
import com.apple.foundationdb.record.IndexEntry;
import com.apple.foundationdb.record.ObjectPlanHash;
import com.apple.foundationdb.record.PlanHashable;
import com.apple.foundationdb.record.RecordCursor;
import com.apple.foundationdb.record.provider.common.StoreTimer;
import com.apple.foundationdb.record.provider.foundationdb.FDBIndexedRecord;
import com.apple.foundationdb.record.provider.foundationdb.FDBQueriedRecord;
import com.apple.foundationdb.record.provider.foundationdb.FDBRecordStoreBase;
import com.apple.foundationdb.record.provider.foundationdb.FDBStoredRecord;
import com.apple.foundationdb.record.query.plan.cascades.AliasMap;
import com.apple.foundationdb.record.query.plan.cascades.CorrelationIdentifier;
import com.apple.foundationdb.record.query.plan.cascades.GroupExpressionRef;
Expand Down Expand Up @@ -83,29 +79,7 @@ public <M extends Message> RecordCursor<QueryResult> executePlan(@Nonnull final
@Nonnull final ExecuteProperties executeProperties) {
final RecordCursor<QueryResult> results = getInnerPlan().executePlan(store, context, continuation, executeProperties);

return results .map(result -> QueryResult.fromQueriedRecord(highlightTermsInRecord(result.getQueriedRecord())));
}

@Nullable
@SuppressWarnings("unchecked")
private <M extends Message> FDBQueriedRecord<M> highlightTermsInRecord(@Nullable FDBQueriedRecord<M> queriedRecord) {
if (queriedRecord == null) {
return queriedRecord;
}
IndexEntry indexEntry = queriedRecord.getIndexEntry();
if (!(indexEntry instanceof LuceneRecordCursor.ScoreDocIndexEntry)) {
return queriedRecord;
}
LuceneRecordCursor.ScoreDocIndexEntry docIndexEntry = (LuceneRecordCursor.ScoreDocIndexEntry)indexEntry;
if (!docIndexEntry.getLuceneQueryHighlightParameters().isHighlight()) {
return queriedRecord;
}
M message = queriedRecord.getRecord();
M.Builder builder = message.toBuilder();
LuceneDocumentFromRecord.highlightTermsInMessage(docIndexEntry.getIndexKey(), builder,
docIndexEntry.getTermMap(), docIndexEntry.getAnalyzerSelector(), docIndexEntry.getLuceneQueryHighlightParameters());
FDBStoredRecord<M> storedRecord = queriedRecord.getStoredRecord().asBuilder().setRecord((M) builder.build()).build();
return FDBQueriedRecord.indexed(new FDBIndexedRecord<>(indexEntry, storedRecord));
return results .map(result -> QueryResult.fromQueriedRecord(LuceneHighlighting.highlightTermsInRecord(result.getQueriedRecord())));
}

@Override
Expand Down