From 9cb68585d2ad5b7cdd8f203678a913d8578547f6 Mon Sep 17 00:00:00 2001 From: Yizuo Tian Date: Mon, 24 Oct 2022 23:29:12 -0700 Subject: [PATCH] Resolves #1862: Lucene search with highlighting the terms --- docs/ReleaseNotes.md | 2 +- .../apple/foundationdb/record/IndexEntry.java | 13 ++ .../foundationdb/FDBRecordStoreBase.java | 2 +- .../foundationdb/FDBStoredRecord.java | 19 ++ .../LuceneAutoCompleteResultCursor.java | 117 ++++++++-- .../lucene/LuceneDocumentFromRecord.java | 39 ++++ .../LuceneFunctionKeyExpressionFactory.java | 3 +- .../record/lucene/LuceneFunctionNames.java | 2 + ...ceneIndexKeyValueToPartialRecordUtils.java | 110 ++++++++++ .../record/lucene/LuceneIndexMaintainer.java | 3 +- .../record/lucene/LucenePlanner.java | 30 ++- .../record/lucene/LuceneQueryComponent.java | 15 ++ .../record/lucene/LuceneRecordCursor.java | 97 +++++++- .../record/lucene/LuceneScanQuery.java | 14 +- .../lucene/LuceneScanQueryParameters.java | 68 +++++- .../record/lucene/FDBLuceneQueryTest.java | 118 ++++++++-- .../LuceneAutoCompleteResultCursorTest.java | 3 +- .../lucene/LuceneDocumentFromRecordTest.java | 207 ++++++++++++------ .../record/lucene/LuceneIndexTest.java | 8 +- 19 files changed, 740 insertions(+), 130 deletions(-) diff --git a/docs/ReleaseNotes.md b/docs/ReleaseNotes.md index 2c9cc386d0..167f7bd57a 100644 --- a/docs/ReleaseNotes.md +++ b/docs/ReleaseNotes.md @@ -25,7 +25,7 @@ The Guava dependency version has been updated to 31.1. Projects may need to chec * **Performance** Improvement 3 [(Issue #NNN)](https://github.com/FoundationDB/fdb-record-layer/issues/NNN) * **Performance** Improvement 4 [(Issue #NNN)](https://github.com/FoundationDB/fdb-record-layer/issues/NNN) * **Performance** Improvement 5 [(Issue #NNN)](https://github.com/FoundationDB/fdb-record-layer/issues/NNN) -* **Feature** Feature 1 [(Issue #NNN)](https://github.com/FoundationDB/fdb-record-layer/issues/NNN) +* **Feature** Lucene search with highlighting the terms [(Issue #1862)](https://github.com/FoundationDB/fdb-record-layer/issues/1862) * **Feature** Feature 2 [(Issue #NNN)](https://github.com/FoundationDB/fdb-record-layer/issues/NNN) * **Feature** Feature 3 [(Issue #NNN)](https://github.com/FoundationDB/fdb-record-layer/issues/NNN) * **Feature** Feature 4 [(Issue #NNN)](https://github.com/FoundationDB/fdb-record-layer/issues/NNN) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/IndexEntry.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/IndexEntry.java index dc80725b4e..4e91007dc2 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/IndexEntry.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/IndexEntry.java @@ -24,8 +24,10 @@ import com.apple.foundationdb.record.metadata.Index; import com.apple.foundationdb.record.metadata.Key; import com.apple.foundationdb.record.metadata.Key.Evaluated.NullStandin; +import com.apple.foundationdb.record.provider.foundationdb.FDBStoredRecord; import com.apple.foundationdb.tuple.Tuple; import com.apple.foundationdb.tuple.TupleHelpers; +import com.google.protobuf.Message; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -228,6 +230,17 @@ public IndexEntry subKey(int startIdx, int endIdx) { return subKey; } + /** + * Rewrite the fetched stored record if needed. The default behavior is to keep the original fetched record. + * @param record the fetched record to rewrite + * @param type used to represent stored records + * @return the rewritten record + */ + @Nonnull + public FDBStoredRecord rewriteStoredRecord(@Nonnull FDBStoredRecord record) { + return record; + } + private void checkIfNullTypeAvailable() { // This indicates that the key/value was created from a tuple (i.e. likely values were read from an // index entry in the database) and, therefore, we don't know what type of null it was when it was diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBRecordStoreBase.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBRecordStoreBase.java index feb7796263..43b418d855 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBRecordStoreBase.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBRecordStoreBase.java @@ -1311,7 +1311,7 @@ default CompletableFuture> loadIndexEntryRecord(@Nonnull fin throw new RecordCoreException("Unexpected index orphan behavior: " + orphanBehavior); } } - return new FDBIndexedRecord<>(entry, rec); + return new FDBIndexedRecord<>(entry, entry.rewriteStoredRecord(rec)); }); } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBStoredRecord.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBStoredRecord.java index af21340d17..b9c61523e6 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBStoredRecord.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBStoredRecord.java @@ -150,6 +150,25 @@ public static FDBStoredRecordBuilder newBuilder(@Nonnull return new FDBStoredRecordBuilder<>(protoRecord); } + /** + * Get a builder with the parameters of a given {@link FDBStoredRecord}. + * @param record given record + * @param type used to represent stored records + * @return a new builder + */ + @Nonnull + public static FDBStoredRecordBuilder newBuilder(@Nonnull FDBStoredRecord record) { + return new FDBStoredRecordBuilder<>(record.getRecord()) + .setPrimaryKey(record.getPrimaryKey()) + .setRecordType(record.getRecordType()) + .setKeyCount(record.getKeyCount()) + .setKeySize(record.getKeySize()) + .setValueSize(record.getValueSize()) + .setSplit(record.isSplit()) + .setVersion(record.getVersion()) + .setVersionedInline(record.isVersionedInline()); + } + /** * Copy this record with a different version. * @param recordVersion new version diff --git a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneAutoCompleteResultCursor.java b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneAutoCompleteResultCursor.java index 317e1d635d..0e532e31a4 100644 --- a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneAutoCompleteResultCursor.java +++ b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneAutoCompleteResultCursor.java @@ -72,12 +72,14 @@ 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; @@ -92,6 +94,11 @@ */ public class LuceneAutoCompleteResultCursor implements BaseCursor { 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 @@ -192,22 +199,43 @@ private void performLookup() throws IOException { @SuppressWarnings("squid:S3776") // Cognitive complexity is too high. Candidate for later refactoring @Nullable @VisibleForTesting - static String searchAllMaybeHighlight(Analyzer queryAnalyzer, String text, Set matchedTokens, @Nullable String prefixToken, boolean highlight) { - try (TokenStream ts = queryAnalyzer.tokenStream("text", new StringReader(text))) { + static String searchAllMaybeHighlight(@Nonnull String fieldName, @Nonnull Analyzer queryAnalyzer, @Nonnull String text, + @Nonnull Set 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 = highlight ? new StringBuilder() : null; + StringBuilder sb = luceneQueryHighlightParameters.isHighlight() ? new StringBuilder() : null; int upto = 0; Set matchedInText = new HashSet<>(); boolean matchedPrefix = false; + ArrayDeque pres = new ArrayDeque<>(); + ArrayDeque 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 (highlight) { - addNonMatch(sb, text.substring(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) { @@ -216,31 +244,66 @@ static String searchAllMaybeHighlight(Analyzer queryAnalyzer, String text, Set 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); + 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 (highlight) { - addPrefixMatch(sb, text.substring(startOffset, endOffset), 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 ((prefixToken != null && !matchedPrefix) || (matchedInText.size() < matchedTokens.size())) { + 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 (highlight) { + if (luceneQueryHighlightParameters.isHighlight()) { int endOffset = offsetAtt.endOffset(); - if (upto < 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 { @@ -252,6 +315,15 @@ static String searchAllMaybeHighlight(Analyzer queryAnalyzer, String text, Set= 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. @@ -266,11 +338,13 @@ private static void addNonMatch(StringBuilder sb, String text) { * 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) { - sb.append(""); + private static void addWholeMatch(StringBuilder sb, String surface, String leftTag, String rightTag) { + sb.append(leftTag); sb.append(surface); - sb.append(""); + sb.append(rightTag); } /** Called while highlighting a single result, to append a @@ -280,17 +354,19 @@ private static void addWholeMatch(StringBuilder sb, String surface) { * (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) { + 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); + addWholeMatch(sb, surface, leftTag, rightTag); return; } - sb.append(""); + sb.append(leftTag); sb.append(surface.substring(0, prefixToken.length())); - sb.append(""); + sb.append(rightTag); sb.append(surface.substring(prefixToken.length())); } @@ -532,7 +608,8 @@ private RecordCursor findIndexEntriesInRecord(ScoreDocAndRecord scor // matched terms return null; } - String match = searchAllMaybeHighlight(queryAnalyzer, text, queryTokens, prefixToken, highlight); + String match = searchAllMaybeHighlight(documentField.getFieldName(), queryAnalyzer, text, queryTokens, prefixToken, true, + new LuceneScanQueryParameters.LuceneQueryHighlightParameters(highlight)); if (match == null) { // Text not found in this field return null; diff --git a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneDocumentFromRecord.java b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneDocumentFromRecord.java index 97b71da39b..1014412d2e 100644 --- a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneDocumentFromRecord.java +++ b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneDocumentFromRecord.java @@ -38,9 +38,12 @@ 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. @@ -131,6 +134,42 @@ public static List 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 void highlightTermsInMessage(@Nonnull KeyExpression expression, @Nonnull Message.Builder builder, @Nonnull Map> termMap, + @Nonnull LuceneAnalyzerCombinationProvider analyzerSelector, + @Nonnull LuceneScanQueryParameters.LuceneQueryHighlightParameters luceneQueryHighlightParameters) { + LuceneIndexKeyValueToPartialRecordUtils.RecordRebuildSource 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 terms = new HashSet<>(); + terms.addAll(termMap.getOrDefault(fieldName, Collections.emptySet())); + terms.addAll(termMap.getOrDefault("", Collections.emptySet())); + if (terms.isEmpty()) { + return; + } + for (Map.Entry 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 implements LuceneIndexExpressions.RecordSource> { @Nonnull private final FDBRecord rec; diff --git a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneFunctionKeyExpressionFactory.java b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneFunctionKeyExpressionFactory.java index 6da1f9af51..5636085b7a 100644 --- a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneFunctionKeyExpressionFactory.java +++ b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneFunctionKeyExpressionFactory.java @@ -48,7 +48,8 @@ public List getBuilders() { new FunctionKeyExpression.BiFunctionBuilder(LuceneFunctionNames.LUCENE_FULL_TEXT_FIELD_WITH_TERM_VECTOR_POSITIONS, LuceneFunctionKeyExpression.LuceneFieldConfig::new), new FunctionKeyExpression.BiFunctionBuilder(LuceneFunctionNames.LUCENE_AUTO_COMPLETE_FIELD_INDEX_OPTIONS, LuceneFunctionKeyExpression.LuceneFieldConfig::new), new FunctionKeyExpression.BiFunctionBuilder(LuceneFunctionNames.LUCENE_SORT_BY_RELEVANCE, LuceneFunctionKeyExpression.LuceneSortBy::new), - new FunctionKeyExpression.BiFunctionBuilder(LuceneFunctionNames.LUCENE_SORT_BY_DOCUMENT_NUMBER, LuceneFunctionKeyExpression.LuceneSortBy::new) + new FunctionKeyExpression.BiFunctionBuilder(LuceneFunctionNames.LUCENE_SORT_BY_DOCUMENT_NUMBER, LuceneFunctionKeyExpression.LuceneSortBy::new), + new FunctionKeyExpression.BiFunctionBuilder(LuceneFunctionNames.LUCENE_HIGHLIGHT_TAG, LuceneFunctionKeyExpression.LuceneFieldConfig::new) ); } } diff --git a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneFunctionNames.java b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneFunctionNames.java index 2eee452ae8..878b939b37 100644 --- a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneFunctionNames.java +++ b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneFunctionNames.java @@ -38,6 +38,8 @@ public class LuceneFunctionNames { public static final String LUCENE_SORT_BY_RELEVANCE = "lucene_sort_by_relevance"; public static final String LUCENE_SORT_BY_DOCUMENT_NUMBER = "lucene_sort_by_document_number"; + public static final String LUCENE_HIGHLIGHT_TAG = "lucene_highlight_tag"; + private LuceneFunctionNames() { } diff --git a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneIndexKeyValueToPartialRecordUtils.java b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneIndexKeyValueToPartialRecordUtils.java index 63c1ada4c2..20e512b6e8 100644 --- a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneIndexKeyValueToPartialRecordUtils.java +++ b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneIndexKeyValueToPartialRecordUtils.java @@ -23,6 +23,7 @@ import com.apple.foundationdb.record.IndexEntry; import com.apple.foundationdb.record.RecordCoreException; import com.apple.foundationdb.record.logging.LogMessageKeys; +import com.apple.foundationdb.record.metadata.Key; import com.apple.foundationdb.record.metadata.expressions.FieldKeyExpression; import com.apple.foundationdb.record.metadata.expressions.GroupingKeyExpression; import com.apple.foundationdb.record.metadata.expressions.KeyExpression; @@ -200,6 +201,115 @@ private static Pair, List> getOriginalAndMappedFieldElement return Pair.of(fixedFieldNames, dynamicFieldNames); } + static class RecordRebuildSource implements LuceneIndexExpressions.RecordSource> { + @Nullable + public final RecordRebuildSource parent; + @Nonnull + public final Descriptors.Descriptor descriptor; + @Nullable + public final Descriptors.FieldDescriptor fieldDescriptor; + @Nonnull + public final Message.Builder builder; + public final Message message; + public final int indexIfRepeated; + + RecordRebuildSource(@Nullable RecordRebuildSource parent, @Nonnull Descriptors.Descriptor descriptor, @Nonnull Message.Builder builder, @Nonnull Message message) { + //this.rec = rec; + this.parent = parent; + this.descriptor = descriptor; + this.fieldDescriptor = null; + this.builder = builder; + this.message = message; + this.indexIfRepeated = 0; + } + + RecordRebuildSource(@Nullable RecordRebuildSource parent, @Nonnull Descriptors.FieldDescriptor fieldDescriptor, @Nonnull Message.Builder builder, @Nonnull Message message, int indexIfRepeated) { + //this.rec = rec; + this.parent = parent; + this.descriptor = fieldDescriptor.getMessageType(); + this.fieldDescriptor = fieldDescriptor; + this.builder = builder; + this.message = message; + this.indexIfRepeated = indexIfRepeated; + } + + @Override + public Descriptors.Descriptor getDescriptor() { + return descriptor; + } + + @Override + public Iterable> getChildren(@Nonnull FieldKeyExpression parentExpression) { + final String parentField = parentExpression.getFieldName(); + final Descriptors.FieldDescriptor parentFieldDescriptor = descriptor.findFieldByName(parentField); + + final List> children = new ArrayList<>(); + int index = 0; + for (Key.Evaluated evaluated : parentExpression.evaluateMessage(null, message)) { + final Message submessage = (Message)evaluated.toList().get(0); + if (submessage != null) { + if (parentFieldDescriptor.isRepeated()) { + children.add(new RecordRebuildSource(this, parentFieldDescriptor, + builder.newBuilderForField(parentFieldDescriptor), + submessage, index++)); + } else { + children.add(new RecordRebuildSource(this, parentFieldDescriptor, + builder.getFieldBuilder(parentFieldDescriptor), + submessage, index)); + } + } + } + return children; + } + + @Override + public Iterable getValues(@Nonnull FieldKeyExpression fieldExpression) { + final List values = new ArrayList<>(); + for (Key.Evaluated evaluated : fieldExpression.evaluateMessage(null, message)) { + Object value = evaluated.getObject(0); + if (value != null) { + values.add(value); + } + } + return values; + } + + @SuppressWarnings("java:S3776") + public void buildMessage(@Nullable Object value, Descriptors.FieldDescriptor subFieldDescriptor, @Nullable String customizedKey, @Nullable String mappedKeyField, boolean forLuceneField, int index) { + final Descriptors.FieldDescriptor mappedKeyFieldDescriptor = mappedKeyField == null ? null : descriptor.findFieldByName(mappedKeyField); + if (mappedKeyFieldDescriptor != null) { + if (customizedKey == null) { + return; + } + builder.setField(mappedKeyFieldDescriptor, customizedKey); + } + + if (value == null) { + return; + } + if (subFieldDescriptor.isRepeated()) { + if (subFieldDescriptor.getJavaType().equals(Descriptors.FieldDescriptor.JavaType.MESSAGE)) { + Message.Builder subBuilder = builder.newBuilderForField(subFieldDescriptor); + subBuilder.mergeFrom((Message) builder.getRepeatedField(subFieldDescriptor, index)).mergeFrom((Message) value); + builder.setRepeatedField(subFieldDescriptor, index, subBuilder.build()); + } else { + builder.setRepeatedField(subFieldDescriptor, index, value); + } + + } else { + int count = builder.getAllFields().size(); + if (message != null && count == 0) { + builder.mergeFrom(message); + } + builder.setField(subFieldDescriptor, value); + } + + if (parent != null) { + parent.buildMessage(builder.build(), this.fieldDescriptor, mappedKeyFieldDescriptor == null ? customizedKey : null, mappedKeyFieldDescriptor == null ? mappedKeyField : null, forLuceneField, indexIfRepeated); + } + } + } + /** * A {@link com.apple.foundationdb.record.lucene.LuceneIndexExpressions.RecordSource} implementation to build the partial record message. */ diff --git a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneIndexMaintainer.java b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneIndexMaintainer.java index 510bdf2f20..60ecfb10a8 100644 --- a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneIndexMaintainer.java +++ b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneIndexMaintainer.java @@ -133,7 +133,8 @@ public RecordCursor scan(@Nonnull final IndexScanBounds scanBounds, return new LuceneRecordCursor(executor, state.context.getPropertyStorage().getPropertyValue(LuceneRecordContextProperties.LUCENE_EXECUTOR_SERVICE), state.context.getPropertyStorage().getPropertyValue(LuceneRecordContextProperties.LUCENE_INDEX_CURSOR_PAGE_SIZE), scanProperties, state, scanQuery.getQuery(), scanQuery.getSort(), continuation, - scanQuery.getGroupKey(), scanQuery.getStoredFields(), scanQuery.getStoredFieldTypes()); + scanQuery.getGroupKey(), scanQuery.getLuceneQueryHighlightParameters(), + scanQuery.getStoredFields(), scanQuery.getStoredFieldTypes(), indexAnalyzerSelector); } if (scanType.equals(LuceneScanTypes.BY_LUCENE_AUTO_COMPLETE)) { diff --git a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LucenePlanner.java b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LucenePlanner.java index 2c1e616c8d..c7d6ba2a56 100644 --- a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LucenePlanner.java +++ b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LucenePlanner.java @@ -120,8 +120,9 @@ private ScoredPlan planLucene(@Nonnull CandidateScan candidateScan, LucenePlanState state = new LucenePlanState(index, groupingComparisons, filter); getFieldDerivations(state); + QueryComponent queryComponent = state.groupingComparisons.isEmpty() ? state.filter : filterMask.getUnsatisfiedFilter(); // Special scans like auto-complete cannot be combined with regular queries. - LuceneScanParameters scanParameters = getSpecialScan(state, filterMask); + LuceneScanParameters scanParameters = getSpecialScan(state, filterMask, queryComponent); if (scanParameters == null) { // Scan by means of normal Lucene search API. LuceneQueryClause query = getQueryForFilter(state, filter, new ArrayList<>(), filterMask); @@ -133,7 +134,7 @@ private ScoredPlan planLucene(@Nonnull CandidateScan candidateScan, } getStoredFields(state); scanParameters = new LuceneScanQueryParameters(groupingComparisons, query, - state.sort, state.storedFields, state.storedFieldTypes); + state.sort, state.storedFields, state.storedFieldTypes, getHighlightParameters(queryComponent)); } // Wrap in plan. @@ -148,6 +149,28 @@ private ScoredPlan planLucene(@Nonnull CandidateScan candidateScan, state.repeated, null); } + private static LuceneScanQueryParameters.LuceneQueryHighlightParameters getHighlightParameters(@Nonnull QueryComponent queryComponent) { + if (queryComponent instanceof LuceneQueryComponent) { + LuceneQueryComponent luceneQueryComponent = (LuceneQueryComponent) queryComponent; + return luceneQueryComponent.getLuceneQueryHighlightParameters(); + } else if (queryComponent instanceof AndOrComponent) { + for (QueryComponent child : ((AndOrComponent) queryComponent).getChildren()) { + LuceneScanQueryParameters.LuceneQueryHighlightParameters parameters = getHighlightParameters(child); + if (parameters.isHighlight()) { + return parameters; + } + } + } else if (queryComponent instanceof AndComponent) { + for (QueryComponent child : ((AndComponent) queryComponent).getChildren()) { + LuceneScanQueryParameters.LuceneQueryHighlightParameters parameters = getHighlightParameters(child); + if (parameters.isHighlight()) { + return parameters; + } + } + } + return new LuceneScanQueryParameters.LuceneQueryHighlightParameters(false); + } + static class LucenePlanState { @Nonnull final Index index; @@ -178,8 +201,7 @@ static class LucenePlanState { @Nullable @SuppressWarnings("PMD.CompareObjectsWithEquals") - private LuceneScanParameters getSpecialScan(@Nonnull LucenePlanState state, @Nonnull FilterSatisfiedMask filterMask) { - QueryComponent queryComponent = state.groupingComparisons.isEmpty() ? state.filter : filterMask.getUnsatisfiedFilter(); + private LuceneScanParameters getSpecialScan(@Nonnull LucenePlanState state, @Nonnull FilterSatisfiedMask filterMask, @Nonnull QueryComponent queryComponent) { if (queryComponent instanceof LuceneQueryComponent) { LuceneQueryComponent luceneQueryComponent = (LuceneQueryComponent)queryComponent; for (String field : luceneQueryComponent.getFields()) { diff --git a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneQueryComponent.java b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneQueryComponent.java index be380923de..6af4b3ad6d 100644 --- a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneQueryComponent.java +++ b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneQueryComponent.java @@ -54,6 +54,7 @@ public class LuceneQueryComponent implements QueryComponent, ComponentWithNoChil */ public enum Type { QUERY, + QUERY_HIGHLIGHT, AUTO_COMPLETE_HIGHLIGHT, AUTO_COMPLETE, SPELL_CHECK, @@ -68,6 +69,9 @@ public enum Type { @Nonnull private final List fields; + @Nonnull + private LuceneScanQueryParameters.LuceneQueryHighlightParameters luceneQueryHighlightParameters; + //MultiFieldSearch determines whether MultiFieldQueryParser or QueryParserBase is used. // QueryParserBase expects the query to contain the fields to be run against and takes a default field // which in our use case is the primary key field. @@ -86,11 +90,17 @@ public LuceneQueryComponent(String query, List fields, boolean multiFiel } public LuceneQueryComponent(Type type, String query, boolean queryIsParameter, List fields, boolean multiFieldSearch) { + this(type, query, queryIsParameter, fields, multiFieldSearch, new LuceneScanQueryParameters.LuceneQueryHighlightParameters(false)); + } + + public LuceneQueryComponent(Type type, String query, boolean queryIsParameter, List fields, boolean multiFieldSearch, + @Nonnull LuceneScanQueryParameters.LuceneQueryHighlightParameters luceneQueryHighlightParameters) { this.type = type; this.query = query; this.queryIsParameter = queryIsParameter; this.fields = fields; this.multiFieldSearch = multiFieldSearch; + this.luceneQueryHighlightParameters = luceneQueryHighlightParameters; } @Nonnull @@ -136,6 +146,11 @@ public boolean isMultiFieldSearch() { return multiFieldSearch; } + @Nonnull + public LuceneScanQueryParameters.LuceneQueryHighlightParameters getLuceneQueryHighlightParameters() { + return luceneQueryHighlightParameters; + } + @Override public boolean equals(final Object o) { if (this == o) { diff --git a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneRecordCursor.java b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneRecordCursor.java index b8b0ce754b..50a8dd5d93 100644 --- a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneRecordCursor.java +++ b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneRecordCursor.java @@ -38,18 +38,28 @@ import com.apple.foundationdb.record.metadata.Index; import com.apple.foundationdb.record.metadata.expressions.KeyExpression; import com.apple.foundationdb.record.provider.foundationdb.FDBStoreTimer; +import com.apple.foundationdb.record.provider.foundationdb.FDBStoredRecord; import com.apple.foundationdb.record.provider.foundationdb.IndexMaintainerState; import com.apple.foundationdb.tuple.Tuple; import com.apple.foundationdb.tuple.TupleHelpers; import com.google.common.collect.Lists; +import com.google.protobuf.Message; import org.apache.lucene.document.Document; import org.apache.lucene.index.IndexNotFoundException; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexableField; +import org.apache.lucene.index.Term; +import org.apache.lucene.search.BooleanClause; +import org.apache.lucene.search.BooleanQuery; +import org.apache.lucene.search.BoostQuery; import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.MultiPhraseQuery; +import org.apache.lucene.search.PhraseQuery; import org.apache.lucene.search.Query; import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.Sort; +import org.apache.lucene.search.SynonymQuery; +import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TopDocs; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.IOUtils; @@ -60,9 +70,14 @@ import javax.annotation.Nullable; import java.io.IOException; import java.util.Arrays; +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.Optional; +import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; import java.util.concurrent.ExecutorService; @@ -114,6 +129,11 @@ class LuceneRecordCursor implements BaseCursor { @Nullable private final List storedFieldTypes; + @Nonnull + private final LuceneScanQueryParameters.LuceneQueryHighlightParameters luceneQueryHighlightParameters; + @Nonnull + private final LuceneAnalyzerCombinationProvider analyzerSelector; + //TODO: once we fix the available fields logic for lucene to take into account which fields are // stored there should be no need to pass in a list of fields, or we could only pass in the store field values. @SuppressWarnings("squid:S107") @@ -126,8 +146,10 @@ class LuceneRecordCursor implements BaseCursor { @Nullable Sort sort, byte[] continuation, @Nullable Tuple groupingKey, + @Nonnull LuceneScanQueryParameters.LuceneQueryHighlightParameters luceneQueryHighlightParameters, @Nullable final List storedFields, - @Nullable final List storedFieldTypes) { + @Nullable final List storedFieldTypes, + @Nonnull LuceneAnalyzerCombinationProvider analyzerSelector) { this.state = state; this.executor = executor; this.pageSize = pageSize; @@ -151,6 +173,8 @@ class LuceneRecordCursor implements BaseCursor { } this.fields = state.index.getRootExpression().normalizeKeyForPositions(); this.groupingKey = groupingKey; + this.luceneQueryHighlightParameters = luceneQueryHighlightParameters; + this.analyzerSelector = analyzerSelector; } @Nonnull @@ -351,23 +375,90 @@ private CompletableFuture buildIndexEntryFromScoreDocAsync(@ tuple = Tuple.fromList(fieldValues).addAll(setPrimaryKey); } - return new ScoreDocIndexEntry(scoreDoc, state.index, tuple); + return new ScoreDocIndexEntry(scoreDoc, state.index, tuple, luceneQueryHighlightParameters, query, analyzerSelector); } catch (Exception e) { throw new RecordCoreException("Failed to get document", "currentPosition", currentPosition, "exception", e); } }, executor); } + // Parse the Lucene query to get all the mapping from field to terms + private static void getTerms(Query query, Map> map) { + if (query instanceof BooleanQuery) { + BooleanQuery booleanQuery = (BooleanQuery) query; + for (BooleanClause clause : booleanQuery.clauses()) { + getTerms(clause.getQuery(), map); + } + } else if (query instanceof TermQuery) { + TermQuery termQuery = (TermQuery) query; + Term term = termQuery.getTerm(); + map.putIfAbsent(term.field(), new HashSet<>()); + map.get(term.field()).add(term.text().toLowerCase(Locale.ROOT)); + } else if (query instanceof PhraseQuery) { + PhraseQuery phraseQuery = (PhraseQuery) query; + for (Term term : phraseQuery.getTerms()) { + map.putIfAbsent(term.field(), new HashSet<>()); + map.get(term.field()).add(term.text().toLowerCase(Locale.ROOT)); + } + } else if (query instanceof MultiPhraseQuery) { + MultiPhraseQuery multiPhraseQuery = (MultiPhraseQuery) query; + for (Term[] termArray : multiPhraseQuery.getTermArrays()) { + for (Term term : termArray) { + map.putIfAbsent(term.field(), new HashSet<>()); + map.get(term.field()).add(term.text().toLowerCase(Locale.ROOT)); + } + } + } else if (query instanceof BoostQuery) { + BoostQuery boostQuery = (BoostQuery) query; + getTerms(boostQuery.getQuery(), map); + } else if (query instanceof SynonymQuery) { + SynonymQuery synonymQuery = (SynonymQuery) query; + for (Term term : synonymQuery.getTerms()) { + map.putIfAbsent(term.field(), new HashSet<>()); + map.get(term.field()).add(term.text().toLowerCase(Locale.ROOT)); + } + } else { + throw new RecordCoreException("This lucene query is not supported for highlighting"); + } + } + protected static final class ScoreDocIndexEntry extends IndexEntry { private final ScoreDoc scoreDoc; + private final Map> termMap; + private final LuceneAnalyzerCombinationProvider analyzerSelector; + private final LuceneScanQueryParameters.LuceneQueryHighlightParameters luceneQueryHighlightParameters; + private final KeyExpression indexKey; + public ScoreDoc getScoreDoc() { return scoreDoc; } - private ScoreDocIndexEntry(@Nonnull ScoreDoc scoreDoc, @Nonnull Index index, @Nonnull Tuple key) { + private ScoreDocIndexEntry(@Nonnull ScoreDoc scoreDoc, @Nonnull Index index, @Nonnull Tuple key, + @Nonnull LuceneScanQueryParameters.LuceneQueryHighlightParameters luceneQueryHighlightParameters, @Nonnull Query query, + @Nonnull LuceneAnalyzerCombinationProvider analyzerSelector) { super(index, key, TupleHelpers.EMPTY); this.scoreDoc = scoreDoc; + this.luceneQueryHighlightParameters = luceneQueryHighlightParameters; + this.termMap = new HashMap<>(); + this.analyzerSelector = analyzerSelector; + this.indexKey = index.getRootExpression(); + if (luceneQueryHighlightParameters.isHighlight()) { + getTerms(query, this.termMap); + } + } + + @SuppressWarnings("unchecked") + @Override + @Nonnull + public FDBStoredRecord rewriteStoredRecord(@Nonnull FDBStoredRecord record) { + if (!luceneQueryHighlightParameters.isHighlight()) { + return super.rewriteStoredRecord(record); + } + M message = record.getRecord(); + M.Builder builder = message.toBuilder(); + LuceneDocumentFromRecord.highlightTermsInMessage(indexKey, builder, termMap, analyzerSelector, luceneQueryHighlightParameters); + return FDBStoredRecord.newBuilder(record).setRecord((M) builder.build()).build(); } @Override diff --git a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneScanQuery.java b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneScanQuery.java index 7b4faaf8df..ef541835a0 100644 --- a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneScanQuery.java +++ b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneScanQuery.java @@ -44,14 +44,19 @@ public class LuceneScanQuery extends LuceneScanBounds { @Nullable private final List storedFieldTypes; + @Nonnull + private final LuceneScanQueryParameters.LuceneQueryHighlightParameters luceneQueryHighlightParameters; + public LuceneScanQuery(@Nonnull IndexScanType scanType, @Nonnull Tuple groupKey, - @Nonnull Query query, @Nullable Sort sort, - @Nullable List storedFields, @Nullable List storedFieldTypes) { + @Nonnull Query query, @Nullable Sort sort, @Nullable List storedFields, + @Nullable List storedFieldTypes, + @Nonnull LuceneScanQueryParameters.LuceneQueryHighlightParameters luceneQueryHighlightParameters) { super(scanType, groupKey); this.query = query; this.sort = sort; this.storedFields = storedFields; this.storedFieldTypes = storedFieldTypes; + this.luceneQueryHighlightParameters = luceneQueryHighlightParameters; } @Nonnull @@ -74,6 +79,11 @@ public List getStoredFieldTypes() { return storedFieldTypes; } + @Nonnull + public LuceneScanQueryParameters.LuceneQueryHighlightParameters getLuceneQueryHighlightParameters() { + return luceneQueryHighlightParameters; + } + @Override public String toString() { return super.toString() + " " + query; diff --git a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneScanQueryParameters.java b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneScanQueryParameters.java index 13d3bcf0f7..018f277ad6 100644 --- a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneScanQueryParameters.java +++ b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneScanQueryParameters.java @@ -60,18 +60,22 @@ public class LuceneScanQueryParameters extends LuceneScanParameters { @Nullable final List storedFieldTypes; + final LuceneQueryHighlightParameters luceneQueryHighlightParameters; + public LuceneScanQueryParameters(@Nonnull ScanComparisons groupComparisons, @Nonnull LuceneQueryClause query) { - this(groupComparisons, query, null, null, null); + this(groupComparisons, query, null, null, null, new LuceneQueryHighlightParameters(false)); } public LuceneScanQueryParameters(@Nonnull ScanComparisons groupComparisons, @Nonnull LuceneQueryClause query, @Nullable Sort sort, - @Nullable List storedFields, @Nullable List storedFieldTypes) { + @Nullable List storedFields, @Nullable List storedFieldTypes, + @Nonnull LuceneQueryHighlightParameters luceneQueryHighlightParameters) { super(LuceneScanTypes.BY_LUCENE, groupComparisons); this.query = query; this.sort = sort; this.storedFields = storedFields; this.storedFieldTypes = storedFieldTypes; + this.luceneQueryHighlightParameters = luceneQueryHighlightParameters; } @Nonnull @@ -103,7 +107,7 @@ public int planHash(@Nonnull PlanHashKind hashKind) { @Override public LuceneScanQuery bind(@Nonnull FDBRecordStoreBase store, @Nonnull Index index, @Nonnull EvaluationContext context) { return new LuceneScanQuery(scanType, getGroupKey(store, context), query.bind(store, index, context), - sort, storedFields, storedFieldTypes); + sort, storedFields, storedFieldTypes, luceneQueryHighlightParameters); } @Nonnull @@ -192,4 +196,62 @@ public boolean equals(final Object o) { public int hashCode() { return semanticHashCode(); } + + + /** + * The parameters for highlighting matching terms of a Lucene search. + */ + public static class LuceneQueryHighlightParameters { + private static final String DEFAULT_LEFT_TAG = ""; + private static final String DEFAULT_RIGHT_TAG = ""; + private static final int DEFAULT_SNIPPETS_SIZE = 20; + private final boolean highlight; + @Nonnull + private final String leftTag; + @Nonnull + private final String rightTag; + + private final boolean cutSnippets; + private final int snippedSize; + + public LuceneQueryHighlightParameters(boolean highlight) { + this(highlight, false); + } + + public LuceneQueryHighlightParameters(boolean highlight, boolean cutSnippets) { + this(highlight, DEFAULT_LEFT_TAG, DEFAULT_RIGHT_TAG, cutSnippets, DEFAULT_SNIPPETS_SIZE); + } + + public LuceneQueryHighlightParameters(boolean highlight, @Nonnull String leftTag, @Nonnull String rightTag, + boolean cutSnippets, int snippedSize) { + this.highlight = highlight; + this.leftTag = leftTag; + this.rightTag = rightTag; + this.cutSnippets = cutSnippets; + this.snippedSize = snippedSize; + } + + public boolean isHighlight() { + return highlight; + } + + @Nonnull + public String getLeftTag() { + return leftTag; + } + + @Nonnull + public String getRightTag() { + return rightTag; + } + + @Nonnull + public boolean isCutSnippets() { + return cutSnippets; + } + + public int getSnippedSize() { + return snippedSize; + } + } } diff --git a/fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/FDBLuceneQueryTest.java b/fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/FDBLuceneQueryTest.java index 80c568d8aa..5051976f80 100644 --- a/fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/FDBLuceneQueryTest.java +++ b/fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/FDBLuceneQueryTest.java @@ -38,8 +38,10 @@ import com.apple.foundationdb.record.provider.common.text.AllSuffixesTextTokenizer; import com.apple.foundationdb.record.provider.common.text.TextSamples; import com.apple.foundationdb.record.provider.foundationdb.FDBQueriedRecord; +import com.apple.foundationdb.record.provider.foundationdb.FDBRecord; import com.apple.foundationdb.record.provider.foundationdb.FDBRecordContext; import com.apple.foundationdb.record.provider.foundationdb.FDBRecordContextConfig; +import com.apple.foundationdb.record.provider.foundationdb.FDBStoredRecord; import com.apple.foundationdb.record.provider.foundationdb.indexes.TextIndexTestUtils; import com.apple.foundationdb.record.provider.foundationdb.properties.RecordLayerPropertyStorage; import com.apple.foundationdb.record.provider.foundationdb.query.FDBRecordStoreQueryTestBase; @@ -61,7 +63,11 @@ import org.hamcrest.Matchers; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtensionContext; import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.ArgumentsProvider; +import org.junit.jupiter.params.provider.ArgumentsSource; import org.junit.jupiter.params.provider.MethodSource; import javax.annotation.Nonnull; @@ -113,6 +119,7 @@ import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.hasToString; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; /** * Sophisticated queries involving full text predicates. @@ -120,6 +127,16 @@ @Tag(Tags.RequiresFDB) public class FDBLuceneQueryTest extends FDBRecordStoreQueryTestBase { + static class BooleanArgumentsProvider implements ArgumentsProvider { + @Override + public Stream provideArguments(final ExtensionContext context) { + return Stream.of(Arguments.of(false, false), + Arguments.of(true, false), + Arguments.of(false, true), + Arguments.of(true, true)); + } + } + private static final List DOCUMENTS = TextIndexTestUtils.toSimpleDocuments(Arrays.asList( TextSamples.ANGSTROM, TextSamples.AETHELRED, @@ -335,6 +352,39 @@ private void assertTermIndexedOrNot(String term, boolean indexedExpected, boolea } } + @Test + void luceneQueryCustomizedHighlighting() throws Exception { + try (FDBRecordContext context = openContext()) { + openRecordStore(context); + final String text = "record record record record record record " + + "layer " + + "record record record record record record record record record record " + + "layer " + + "record record " + + "layer " + + "record record record record record record record record"; + TestRecordsTextProto.SimpleDocument simpleDocument = TestRecordsTextProto.SimpleDocument.newBuilder().setDocId(0).setGroup(0).setText(text).build(); + recordStore.saveRecord(simpleDocument); + + final QueryComponent filter = new LuceneQueryComponent(LuceneQueryComponent.Type.QUERY_HIGHLIGHT, + "layer", false, Lists.newArrayList(), true, + new LuceneScanQueryParameters.LuceneQueryHighlightParameters(true, "", "", true, 6)); + RecordQuery query = RecordQuery.newBuilder() + .setRecordType(TextIndexTestUtils.SIMPLE_DOC) + .setFilter(filter) + .build(); + + RecordQueryPlan plan = planner.plan(query); + + List> queriedRecordList = recordStore.executeQuery(plan).asList().get(); + + Set texts = queriedRecordList.stream().map(FDBQueriedRecord::getStoredRecord).map(FDBStoredRecord::getRecord).map(m -> (String) m.getField(m.getDescriptorForType().findFieldByName("text"))).collect(Collectors.toSet()); + assertEquals(1, texts.size()); + assertEquals("...record record record layer record record record ...record record record layer record record layer record record record record record record ...", + texts.iterator().next()); + } + } + @ParameterizedTest(name = "testSynonym[shouldDeferFetch={0}]") @BooleanSource void testSynonym(boolean shouldDeferFetch) throws Exception { @@ -392,13 +442,15 @@ void testNgramEdgesOnly(boolean shouldDeferFetch) throws Exception { assertTermIndexedOrNot("rning", false, shouldDeferFetch); } - @ParameterizedTest(name = "simpleLuceneScans[shouldDeferFetch={0}]") - @BooleanSource - void simpleLuceneScans(boolean shouldDeferFetch) throws Exception { + @ParameterizedTest + @ArgumentsSource(BooleanArgumentsProvider.class) + void simpleLuceneScans(boolean shouldDeferFetch, boolean withHighlight) throws Exception { initializeFlat(); try (FDBRecordContext context = openContext()) { openRecordStore(context); - final QueryComponent filter1 = new LuceneQueryComponent("civil blood makes civil hands unclean", Lists.newArrayList()); + final QueryComponent filter1 = new LuceneQueryComponent(withHighlight ? LuceneQueryComponent.Type.QUERY_HIGHLIGHT : LuceneQueryComponent.Type.QUERY, + "civil blood makes civil hands unclean", false, Lists.newArrayList(), true, + new LuceneScanQueryParameters.LuceneQueryHighlightParameters(withHighlight, true)); // Query for full records RecordQuery query = RecordQuery.newBuilder() .setRecordType(TextIndexTestUtils.SIMPLE_DOC) @@ -410,10 +462,12 @@ void simpleLuceneScans(boolean shouldDeferFetch) throws Exception { scanParams(query(hasToString("MULTI civil blood makes civil hands unclean"))))); RecordQueryPlan plan = planner.plan(query); assertThat(plan, matcher); - RecordCursor> fdbQueriedRecordRecordCursor = recordStore.executeQuery(plan); - RecordCursor map = fdbQueriedRecordRecordCursor.map(FDBQueriedRecord::getPrimaryKey); - List primaryKeys = map.map(t -> t.getLong(0)).asList().get(); + List> queriedRecordList = recordStore.executeQuery(plan).asList().get(); + Set primaryKeys = queriedRecordList.stream().map(FDBQueriedRecord::getPrimaryKey).map(t -> t.getLong(0)).collect(Collectors.toSet()); assertEquals(Set.of(2L, 4L), Set.copyOf(primaryKeys)); + + Set texts = queriedRecordList.stream().map(FDBQueriedRecord::getStoredRecord).map(FDBStoredRecord::getRecord).map(m -> (String) m.getField(m.getDescriptorForType().findFieldByName("text"))).collect(Collectors.toSet()); + texts.forEach(t -> assertTrue(t.contains(withHighlight ? "civil blood makes civil hands unclean" : "civil blood makes civil hands unclean"))); } } @@ -546,14 +600,16 @@ void delayFetchOnAndOfLuceneAndFieldFilter(boolean shouldDeferFetch) throws Exce } } - @ParameterizedTest(name = "delayFetchOnOrOfLuceneFiltersGivesUnion[shouldDeferFetch={0}]") - @BooleanSource - void delayFetchOnOrOfLuceneFiltersGivesUnion(boolean shouldDeferFetch) throws Exception { + @ParameterizedTest + @ArgumentsSource(BooleanArgumentsProvider.class) + void delayFetchOnOrOfLuceneFiltersGivesUnion(boolean shouldDeferFetch, boolean withHighlight) throws Exception { initializeFlat(); try (FDBRecordContext context = openContext()) { openRecordStore(context); - final QueryComponent filter1 = new LuceneQueryComponent("(\"civil blood makes civil hands unclean\")", Lists.newArrayList("text"), true); - final QueryComponent filter2 = new LuceneQueryComponent("(\"was king from 966 to 1016\")", Lists.newArrayList()); + final QueryComponent filter1 = new LuceneQueryComponent(withHighlight ? LuceneQueryComponent.Type.QUERY_HIGHLIGHT : LuceneQueryComponent.Type.QUERY, "(\"civil blood makes civil hands unclean\")", false, Lists.newArrayList("text"), true, + new LuceneScanQueryParameters.LuceneQueryHighlightParameters(withHighlight, true)); + final QueryComponent filter2 = new LuceneQueryComponent(withHighlight ? LuceneQueryComponent.Type.QUERY_HIGHLIGHT : LuceneQueryComponent.Type.QUERY, "(\"was king from 966 to 1016\")", false, Lists.newArrayList(), true, + new LuceneScanQueryParameters.LuceneQueryHighlightParameters(withHighlight, true)); // Query for full records RecordQuery query = RecordQuery.newBuilder() .setRecordType(TextIndexTestUtils.SIMPLE_DOC) @@ -578,24 +634,36 @@ void delayFetchOnOrOfLuceneFiltersGivesUnion(boolean shouldDeferFetch) throws Ex scanParams(query(hasToString("MULTI (\"was king from 966 to 1016\")"))))))))); } assertThat(plan, matcher); - List primaryKeys = recordStore.executeQuery(plan).map(FDBQueriedRecord::getPrimaryKey).map(t -> t.getLong(0)).asList().get(); - assertEquals(Set.of(1L, 2L, 4L), Set.copyOf(primaryKeys)); + List> queriedRecordList = recordStore.executeQuery(plan).asList().get(); + Set primaryKeys = queriedRecordList.stream().map(FDBRecord::getPrimaryKey).map(t -> t.getLong(0)).collect(Collectors.toSet()); + assertEquals(Set.of(1L, 2L, 4L), primaryKeys); if (shouldDeferFetch) { assertLoadRecord(5, context); } else { assertLoadRecord(6, context); } + + Set texts = queriedRecordList.stream().map(FDBQueriedRecord::getStoredRecord).map(FDBStoredRecord::getRecord).map(m -> (String) m.getField(m.getDescriptorForType().findFieldByName("text"))).collect(Collectors.toSet()); + for (String text : texts) { + boolean match1 = text.contains(withHighlight ? "was king from 966 to 1016" : "was king from 966 to 1016"); + boolean match2 = text.contains(withHighlight ? "civil blood makes civil hands unclean" : "civil blood makes civil hands unclean"); + assertTrue(match1 || match2); + } } } - @ParameterizedTest(name = "delayFetchOnAndOfLuceneFilters[shouldDeferFetch={0}]") - @BooleanSource - void delayFetchOnAndOfLuceneFilters(boolean shouldDeferFetch) throws Exception { + @ParameterizedTest + @ArgumentsSource(BooleanArgumentsProvider.class) + void delayFetchOnAndOfLuceneFilters(boolean shouldDeferFetch, boolean withHighlight) throws Exception { initializeFlat(); try (FDBRecordContext context = openContext()) { openRecordStore(context); - final QueryComponent filter1 = new LuceneQueryComponent("\"the continuance\"", Lists.newArrayList()); - final QueryComponent filter2 = new LuceneQueryComponent("grudge", Lists.newArrayList()); + final QueryComponent filter1 = new LuceneQueryComponent(withHighlight ? LuceneQueryComponent.Type.QUERY_HIGHLIGHT : LuceneQueryComponent.Type.QUERY, + "\"the continuance\"", false, Lists.newArrayList(), true, + new LuceneScanQueryParameters.LuceneQueryHighlightParameters(withHighlight, true)); + final QueryComponent filter2 = new LuceneQueryComponent(withHighlight ? LuceneQueryComponent.Type.QUERY_HIGHLIGHT : LuceneQueryComponent.Type.QUERY, + "grudge", false, Lists.newArrayList(), true, + new LuceneScanQueryParameters.LuceneQueryHighlightParameters(withHighlight, true)); // Query for full records RecordQuery query = RecordQuery.newBuilder() .setRecordType(TextIndexTestUtils.SIMPLE_DOC) @@ -607,13 +675,21 @@ void delayFetchOnAndOfLuceneFilters(boolean shouldDeferFetch) throws Exception { indexName(SIMPLE_TEXT_SUFFIXES.getName()), scanParams(query(hasToString("MULTI \"the continuance\" AND MULTI grudge"))))); assertThat(plan, matcher); - List primaryKeys = recordStore.executeQuery(plan).map(FDBQueriedRecord::getPrimaryKey).map(t -> t.getLong(0)).asList().get(); - assertEquals(Set.of(4L), Set.copyOf(primaryKeys)); + List> queriedRecordList = recordStore.executeQuery(plan).asList().get(); + Set primaryKeys = queriedRecordList.stream().map(FDBQueriedRecord::getPrimaryKey).map(t -> t.getLong(0)).collect(Collectors.toSet()); + assertEquals(Set.of(4L), primaryKeys); if (shouldDeferFetch) { assertLoadRecord(3, context); } else { assertLoadRecord(4, context); } + + Set texts = queriedRecordList.stream().map(FDBQueriedRecord::getStoredRecord).map(FDBStoredRecord::getRecord).map(m -> (String) m.getField(m.getDescriptorForType().findFieldByName("text"))).collect(Collectors.toSet()); + for (String text : texts) { + boolean match1 = text.contains(withHighlight ? "the continuance" : "the continuance"); + boolean match2 = text.contains(withHighlight ? "grudge" : "grudge"); + assertTrue(match1 || match2); + } } } diff --git a/fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/LuceneAutoCompleteResultCursorTest.java b/fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/LuceneAutoCompleteResultCursorTest.java index 41bd16331d..0907090d7f 100644 --- a/fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/LuceneAutoCompleteResultCursorTest.java +++ b/fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/LuceneAutoCompleteResultCursorTest.java @@ -126,7 +126,8 @@ private static void assertSearchMatches(String queryString, List expecte assertEquals(expectedPrefixToken, prefixToken); Set queryTokenSet = new HashSet<>(tokens); - @Nullable String match = LuceneAutoCompleteResultCursor.searchAllMaybeHighlight(analyzer, text, queryTokenSet, prefixToken, highlight); + @Nullable String match = LuceneAutoCompleteResultCursor.searchAllMaybeHighlight("text", analyzer, text, queryTokenSet, prefixToken, true, + new LuceneScanQueryParameters.LuceneQueryHighlightParameters(highlight)); assertEquals(expectedMatch, match); } } diff --git a/fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/LuceneDocumentFromRecordTest.java b/fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/LuceneDocumentFromRecordTest.java index 94a1acfaf2..05f98316e7 100644 --- a/fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/LuceneDocumentFromRecordTest.java +++ b/fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/LuceneDocumentFromRecordTest.java @@ -34,9 +34,9 @@ import javax.annotation.Nonnull; import javax.annotation.Nullable; - import java.util.Collections; import java.util.Map; +import java.util.Set; import static com.apple.foundationdb.record.metadata.Key.Expressions.concat; import static com.apple.foundationdb.record.metadata.Key.Expressions.field; @@ -50,25 +50,34 @@ */ class LuceneDocumentFromRecordTest { + private LuceneAnalyzerCombinationProvider analyzerProvider = new LuceneAnalyzerCombinationProvider(t -> LuceneAnalyzerWrapper.getStandardAnalyzerWrapper(), + t -> LuceneAnalyzerWrapper.getStandardAnalyzerWrapper(), + null, null); + @Test void simple() { - TestRecordsTextProto.SimpleDocument message = TestRecordsTextProto.SimpleDocument.newBuilder() + TestRecordsTextProto.SimpleDocument.Builder builder = TestRecordsTextProto.SimpleDocument.newBuilder() .setDocId(1) - .setText("some text") - .build(); + .setText("some text"); + TestRecordsTextProto.SimpleDocument message = builder.build(); FDBRecord record = unstoredRecord(message); KeyExpression index = function(LuceneFunctionNames.LUCENE_TEXT, field("text")); assertEquals(ImmutableMap.of(Tuple.from(), ImmutableList.of(textField("text", "some text"))), LuceneDocumentFromRecord.getRecordFields(index, record)); + // Highlight "some" for text field + LuceneDocumentFromRecord.highlightTermsInMessage(index, builder, Map.of("text", Set.of("some")), analyzerProvider, + new LuceneScanQueryParameters.LuceneQueryHighlightParameters(true)); + assertEquals("some text", builder.build().getText()); + KeyExpression primaryKey = field("doc_id"); // Build the partial record message for suggestion Descriptors.Descriptor recordDescriptor = message.getDescriptorForType(); - TestRecordsTextProto.SimpleDocument.Builder builder = TestRecordsTextProto.SimpleDocument.newBuilder(); - LuceneIndexKeyValueToPartialRecordUtils.buildPartialRecord(index, recordDescriptor, builder, "text", "suggestion"); - LuceneIndexKeyValueToPartialRecordUtils.populatePrimaryKey(primaryKey, recordDescriptor, builder, Tuple.from(1)); - TestRecordsTextProto.SimpleDocument partialMsg = builder.build(); + TestRecordsTextProto.SimpleDocument.Builder newBuilder = TestRecordsTextProto.SimpleDocument.newBuilder(); + LuceneIndexKeyValueToPartialRecordUtils.buildPartialRecord(index, recordDescriptor, newBuilder, "text", "suggestion"); + LuceneIndexKeyValueToPartialRecordUtils.populatePrimaryKey(primaryKey, recordDescriptor, newBuilder, Tuple.from(1)); + TestRecordsTextProto.SimpleDocument partialMsg = newBuilder.build(); // The suggestion is supposed to show up in text field assertEquals("suggestion", partialMsg.getText()); @@ -79,24 +88,29 @@ void simple() { @Test void group() { - TestRecordsTextProto.SimpleDocument message = TestRecordsTextProto.SimpleDocument.newBuilder() + TestRecordsTextProto.SimpleDocument.Builder builder = TestRecordsTextProto.SimpleDocument.newBuilder() .setDocId(2) .setText("more text") - .setGroup(2) - .build(); + .setGroup(2); + TestRecordsTextProto.SimpleDocument message = builder.build(); FDBRecord record = unstoredRecord(message); KeyExpression index = function(LuceneFunctionNames.LUCENE_TEXT, field("text")).groupBy(field("group")); assertEquals(ImmutableMap.of(Tuple.from(2), ImmutableList.of(textField("text", "more text"))), LuceneDocumentFromRecord.getRecordFields(index, record)); + // Highlight "text" for text field + LuceneDocumentFromRecord.highlightTermsInMessage(index, builder, Map.of("text", Set.of("text")), analyzerProvider, + new LuceneScanQueryParameters.LuceneQueryHighlightParameters(true)); + assertEquals("more text", builder.build().getText()); + KeyExpression primaryKey = concat(field("group"), field("doc_id")); // Build the partial record message for suggestion Descriptors.Descriptor recordDescriptor = message.getDescriptorForType(); - TestRecordsTextProto.SimpleDocument.Builder builder = TestRecordsTextProto.SimpleDocument.newBuilder(); - LuceneIndexKeyValueToPartialRecordUtils.buildPartialRecord(index, recordDescriptor, builder, "text", "suggestion", Tuple.from(2)); - LuceneIndexKeyValueToPartialRecordUtils.populatePrimaryKey(primaryKey, recordDescriptor, builder, Tuple.from(2, 2)); - TestRecordsTextProto.SimpleDocument partialMsg = builder.build(); + TestRecordsTextProto.SimpleDocument.Builder newBuilder = TestRecordsTextProto.SimpleDocument.newBuilder(); + LuceneIndexKeyValueToPartialRecordUtils.buildPartialRecord(index, recordDescriptor, newBuilder, "text", "suggestion", Tuple.from(2)); + LuceneIndexKeyValueToPartialRecordUtils.populatePrimaryKey(primaryKey, recordDescriptor, newBuilder, Tuple.from(2, 2)); + TestRecordsTextProto.SimpleDocument partialMsg = newBuilder.build(); // The suggestion is supposed to show up in text field assertEquals("suggestion", partialMsg.getText()); @@ -110,11 +124,11 @@ void group() { @Test void multi() { - TestRecordsTextProto.MultiDocument message = TestRecordsTextProto.MultiDocument.newBuilder() + TestRecordsTextProto.MultiDocument.Builder builder = TestRecordsTextProto.MultiDocument.newBuilder() .setDocId(3) .addText("some text") - .addText("other text") - .build(); + .addText("other text"); + TestRecordsTextProto.MultiDocument message = builder.build(); FDBRecord record = unstoredRecord(message); KeyExpression index = function(LuceneFunctionNames.LUCENE_TEXT, field("text", KeyExpression.FanType.FanOut)); assertEquals(ImmutableMap.of(Tuple.from(), ImmutableList.of( @@ -122,14 +136,20 @@ void multi() { textField("text", "other text"))), LuceneDocumentFromRecord.getRecordFields(index, record)); + // Highlight "text" for text field + LuceneDocumentFromRecord.highlightTermsInMessage(index, builder, Map.of("text", Set.of("text")), analyzerProvider, + new LuceneScanQueryParameters.LuceneQueryHighlightParameters(true)); + assertEquals("some text", builder.build().getText(0)); + assertEquals("other text", builder.build().getText(1)); + KeyExpression primaryKey = field("doc_id"); // Build the partial record message for suggestion Descriptors.Descriptor recordDescriptor = message.getDescriptorForType(); - TestRecordsTextProto.MultiDocument.Builder builder = TestRecordsTextProto.MultiDocument.newBuilder(); - LuceneIndexKeyValueToPartialRecordUtils.buildPartialRecord(index, recordDescriptor, builder, "text", "suggestion", Tuple.from(2)); - LuceneIndexKeyValueToPartialRecordUtils.populatePrimaryKey(primaryKey, recordDescriptor, builder, Tuple.from(3)); - TestRecordsTextProto.MultiDocument partialMsg = builder.build(); + TestRecordsTextProto.MultiDocument.Builder newBuilder = TestRecordsTextProto.MultiDocument.newBuilder(); + LuceneIndexKeyValueToPartialRecordUtils.buildPartialRecord(index, recordDescriptor, newBuilder, "text", "suggestion", Tuple.from(2)); + LuceneIndexKeyValueToPartialRecordUtils.populatePrimaryKey(primaryKey, recordDescriptor, newBuilder, Tuple.from(3)); + TestRecordsTextProto.MultiDocument partialMsg = newBuilder.build(); // The suggestion is supposed to show up in text field assertEquals(1, partialMsg.getTextCount()); @@ -141,7 +161,7 @@ void multi() { @Test void biGroup() { - TestRecordsTextProto.ComplexDocument message = TestRecordsTextProto.ComplexDocument.newBuilder() + TestRecordsTextProto.ComplexDocument.Builder builder = TestRecordsTextProto.ComplexDocument.newBuilder() .setHeader(TestRecordsTextProto.ComplexDocument.Header.newBuilder().setHeaderId(4)) .setGroup(10) .setDocId(4) @@ -149,8 +169,8 @@ void biGroup() { .addTag("tag1") .addTag("tag2") .setText2("second text") - .setScore(100) - .build(); + .setScore(100); + TestRecordsTextProto.ComplexDocument message = builder.build(); FDBRecord record = unstoredRecord(message); KeyExpression index = concat( function(LuceneFunctionNames.LUCENE_TEXT, field("text")), @@ -162,14 +182,20 @@ void biGroup() { Tuple.from(10, "tag2"), ImmutableList.of(textField("text", "first text"), textField("text2", "second text"), intField("score", 100))), LuceneDocumentFromRecord.getRecordFields(index, record)); + // Highlight "text" for text field + LuceneDocumentFromRecord.highlightTermsInMessage(index, builder, Map.of("text2", Set.of("text")), analyzerProvider, + new LuceneScanQueryParameters.LuceneQueryHighlightParameters(true)); + assertEquals("first text", builder.build().getText()); + assertEquals("second text", builder.build().getText2()); + KeyExpression primaryKey = concat(field("group"), field("header").nest("header_id"), field("doc_id")); // Build the partial record message for suggestion Descriptors.Descriptor recordDescriptor = message.getDescriptorForType(); - TestRecordsTextProto.ComplexDocument.Builder builder = TestRecordsTextProto.ComplexDocument.newBuilder(); - LuceneIndexKeyValueToPartialRecordUtils.buildPartialRecord(index, recordDescriptor, builder, "text", "suggestion", Tuple.from(10, "tag1")); - LuceneIndexKeyValueToPartialRecordUtils.populatePrimaryKey(primaryKey, recordDescriptor, builder, Tuple.from(10, 4, 4)); - TestRecordsTextProto.ComplexDocument partialMsg = builder.build(); + TestRecordsTextProto.ComplexDocument.Builder newBuilder = TestRecordsTextProto.ComplexDocument.newBuilder(); + LuceneIndexKeyValueToPartialRecordUtils.buildPartialRecord(index, recordDescriptor, newBuilder, "text", "suggestion", Tuple.from(10, "tag1")); + LuceneIndexKeyValueToPartialRecordUtils.populatePrimaryKey(primaryKey, recordDescriptor, newBuilder, Tuple.from(10, 4, 4)); + TestRecordsTextProto.ComplexDocument partialMsg = newBuilder.build(); // The suggestion is supposed to show up in text field assertEquals("suggestion", partialMsg.getText()); @@ -187,12 +213,12 @@ void biGroup() { @Test void uncorrelatedMap() { - TestRecordsTextProto.MapDocument message = TestRecordsTextProto.MapDocument.newBuilder() + TestRecordsTextProto.MapDocument.Builder builder = TestRecordsTextProto.MapDocument.newBuilder() .setGroup(10) .setDocId(5) .addEntry(TestRecordsTextProto.MapDocument.Entry.newBuilder().setKey("k1").setValue("v1")) - .addEntry(TestRecordsTextProto.MapDocument.Entry.newBuilder().setKey("k2").setValue("v2")) - .build(); + .addEntry(TestRecordsTextProto.MapDocument.Entry.newBuilder().setKey("k2").setValue("v2")); + TestRecordsTextProto.MapDocument message = builder.build(); FDBRecord record = unstoredRecord(message); KeyExpression index = field("entry", KeyExpression.FanType.FanOut).nest(concat(field("key"), function(LuceneFunctionNames.LUCENE_TEXT, field("value")))); assertEquals(ImmutableMap.of(Tuple.from(), ImmutableList.of( @@ -202,14 +228,20 @@ void uncorrelatedMap() { textField("entry_value", "v2"))), LuceneDocumentFromRecord.getRecordFields(index, record)); + // Highlight "v2" for entry_value field + LuceneDocumentFromRecord.highlightTermsInMessage(index, builder, Map.of("entry_value", Set.of("v2")), analyzerProvider, + new LuceneScanQueryParameters.LuceneQueryHighlightParameters(true)); + assertEquals("v1", builder.build().getEntry(0).getValue()); + assertEquals("v2", builder.build().getEntry(1).getValue()); + KeyExpression primaryKey = concat(field("group"), field("doc_id")); // Build the partial record message for suggestion Descriptors.Descriptor recordDescriptor = message.getDescriptorForType(); - TestRecordsTextProto.MapDocument.Builder builder = TestRecordsTextProto.MapDocument.newBuilder(); - LuceneIndexKeyValueToPartialRecordUtils.buildPartialRecord(index, recordDescriptor, builder, "entry_value", "suggestion"); - LuceneIndexKeyValueToPartialRecordUtils.populatePrimaryKey(primaryKey, recordDescriptor, builder, Tuple.from(10, 5)); - TestRecordsTextProto.MapDocument partialMsg = builder.build(); + TestRecordsTextProto.MapDocument.Builder newBuilder = TestRecordsTextProto.MapDocument.newBuilder(); + LuceneIndexKeyValueToPartialRecordUtils.buildPartialRecord(index, recordDescriptor, newBuilder, "entry_value", "suggestion"); + LuceneIndexKeyValueToPartialRecordUtils.populatePrimaryKey(primaryKey, recordDescriptor, newBuilder, Tuple.from(10, 5)); + TestRecordsTextProto.MapDocument partialMsg = newBuilder.build(); assertEquals(1, partialMsg.getEntryCount()); TestRecordsTextProto.MapDocument.Entry entry = partialMsg.getEntry(0); @@ -223,12 +255,12 @@ void uncorrelatedMap() { @Test void map() { - TestRecordsTextProto.MapDocument message = TestRecordsTextProto.MapDocument.newBuilder() + TestRecordsTextProto.MapDocument.Builder builder = TestRecordsTextProto.MapDocument.newBuilder() .setGroup(10) .setDocId(5) .addEntry(TestRecordsTextProto.MapDocument.Entry.newBuilder().setKey("k1").setValue("v1")) - .addEntry(TestRecordsTextProto.MapDocument.Entry.newBuilder().setKey("k2").setValue("v2")) - .build(); + .addEntry(TestRecordsTextProto.MapDocument.Entry.newBuilder().setKey("k2").setValue("v2")); + TestRecordsTextProto.MapDocument message = builder.build(); FDBRecord record = unstoredRecord(message); KeyExpression index = function(LuceneFunctionNames.LUCENE_FIELD_NAME, concat( field("entry", KeyExpression.FanType.FanOut).nest(function(LuceneFunctionNames.LUCENE_FIELD_NAME, concat(function(LuceneFunctionNames.LUCENE_TEXT, field("value")), field("key")))), @@ -238,14 +270,20 @@ void map() { textField("k2", "v2"))), LuceneDocumentFromRecord.getRecordFields(index, record)); + // Highlight "v2" for k2 field + LuceneDocumentFromRecord.highlightTermsInMessage(index, builder, Map.of("k2", Set.of("v2")), analyzerProvider, + new LuceneScanQueryParameters.LuceneQueryHighlightParameters(true)); + assertEquals("v1", builder.build().getEntry(0).getValue()); + assertEquals("v2", builder.build().getEntry(1).getValue()); + KeyExpression primaryKey = concat(field("group"), field("doc_id")); // Build the partial record message for suggestion Descriptors.Descriptor recordDescriptor = message.getDescriptorForType(); - TestRecordsTextProto.MapDocument.Builder builder = TestRecordsTextProto.MapDocument.newBuilder(); - LuceneIndexKeyValueToPartialRecordUtils.buildPartialRecord(index, recordDescriptor, builder, "k1", "suggestion"); - LuceneIndexKeyValueToPartialRecordUtils.populatePrimaryKey(primaryKey, recordDescriptor, builder, Tuple.from(10, 5)); - TestRecordsTextProto.MapDocument partialMsg = builder.build(); + TestRecordsTextProto.MapDocument.Builder newBuilder = TestRecordsTextProto.MapDocument.newBuilder(); + LuceneIndexKeyValueToPartialRecordUtils.buildPartialRecord(index, recordDescriptor, newBuilder, "k1", "suggestion"); + LuceneIndexKeyValueToPartialRecordUtils.populatePrimaryKey(primaryKey, recordDescriptor, newBuilder, Tuple.from(10, 5)); + TestRecordsTextProto.MapDocument partialMsg = newBuilder.build(); assertEquals(1, partialMsg.getEntryCount()); TestRecordsTextProto.MapDocument.Entry entry = partialMsg.getEntry(0); @@ -261,12 +299,12 @@ void map() { @Test void groupedMap() { - TestRecordsTextProto.MapDocument message = TestRecordsTextProto.MapDocument.newBuilder() + TestRecordsTextProto.MapDocument.Builder builder = TestRecordsTextProto.MapDocument.newBuilder() .setDocId(6) .addEntry(TestRecordsTextProto.MapDocument.Entry.newBuilder().setKey("k1").setValue("v10")) .addEntry(TestRecordsTextProto.MapDocument.Entry.newBuilder().setKey("k2").setValue("v20")) - .setGroup(20) - .build(); + .setGroup(20); + TestRecordsTextProto.MapDocument message = builder.build(); FDBRecord record = unstoredRecord(message); KeyExpression index = function(LuceneFunctionNames.LUCENE_FIELD_NAME, concat( field("entry", KeyExpression.FanType.FanOut).nest(function(LuceneFunctionNames.LUCENE_FIELD_NAME, concat(function(LuceneFunctionNames.LUCENE_TEXT, field("value")), field("key")))), @@ -277,14 +315,20 @@ void groupedMap() { textField("k2", "v20"))), LuceneDocumentFromRecord.getRecordFields(index, record)); + // Highlight "v20" for k2 field + LuceneDocumentFromRecord.highlightTermsInMessage(index, builder, Map.of("k2", Set.of("v20")), analyzerProvider, + new LuceneScanQueryParameters.LuceneQueryHighlightParameters(true)); + assertEquals("v10", builder.build().getEntry(0).getValue()); + assertEquals("v20", builder.build().getEntry(1).getValue()); + KeyExpression primaryKey = concat(field("group"), field("doc_id")); // Build the partial record message for suggestion Descriptors.Descriptor recordDescriptor = message.getDescriptorForType(); - TestRecordsTextProto.MapDocument.Builder builder = TestRecordsTextProto.MapDocument.newBuilder(); - LuceneIndexKeyValueToPartialRecordUtils.buildPartialRecord(index, recordDescriptor, builder, "k1", "suggestion", Tuple.from(20)); - LuceneIndexKeyValueToPartialRecordUtils.populatePrimaryKey(primaryKey, recordDescriptor, builder, Tuple.from(20, 6)); - TestRecordsTextProto.MapDocument partialMsg = builder.build(); + TestRecordsTextProto.MapDocument.Builder newBuilder = TestRecordsTextProto.MapDocument.newBuilder(); + LuceneIndexKeyValueToPartialRecordUtils.buildPartialRecord(index, recordDescriptor, newBuilder, "k1", "suggestion", Tuple.from(20)); + LuceneIndexKeyValueToPartialRecordUtils.populatePrimaryKey(primaryKey, recordDescriptor, newBuilder, Tuple.from(20, 6)); + TestRecordsTextProto.MapDocument partialMsg = newBuilder.build(); assertEquals(1, partialMsg.getEntryCount()); TestRecordsTextProto.MapDocument.Entry entry = partialMsg.getEntry(0); @@ -303,12 +347,12 @@ void groupedMap() { @Test void groupingMap() { - TestRecordsTextProto.MapDocument message = TestRecordsTextProto.MapDocument.newBuilder() + TestRecordsTextProto.MapDocument.Builder builder = TestRecordsTextProto.MapDocument.newBuilder() .setDocId(7) .addEntry(TestRecordsTextProto.MapDocument.Entry.newBuilder().setKey("r1").setValue("val").setSecondValue("2val").setThirdValue("3val")) .addEntry(TestRecordsTextProto.MapDocument.Entry.newBuilder().setKey("r2").setValue("nval").setSecondValue("2nval").setThirdValue("3nval")) - .setGroup(30) - .build(); + .setGroup(30); + TestRecordsTextProto.MapDocument message = builder.build(); FDBRecord record = unstoredRecord(message); KeyExpression index = new GroupingKeyExpression(concat(field("group"), field("entry", KeyExpression.FanType.FanOut) @@ -321,14 +365,22 @@ void groupingMap() { Tuple.from(30, "r2"), ImmutableList.of(textField("entry_value", "nval"), textField("entry_second_value", "2nval"), textField("entry_third_value", "3nval"))), LuceneDocumentFromRecord.getRecordFields(index, record)); + // Highlight "2val" for entry_second_value field + LuceneDocumentFromRecord.highlightTermsInMessage(index, builder, Map.of("entry_second_value", Set.of("2val")), analyzerProvider, + new LuceneScanQueryParameters.LuceneQueryHighlightParameters(true)); + assertEquals("val", builder.build().getEntry(0).getValue()); + assertEquals("2val", builder.build().getEntry(0).getSecondValue()); + assertEquals("nval", builder.build().getEntry(1).getValue()); + assertEquals("2nval", builder.build().getEntry(1).getSecondValue()); + KeyExpression primaryKey = concat(field("group"), field("doc_id")); // Build the partial record message for suggestion Descriptors.Descriptor recordDescriptor = message.getDescriptorForType(); - TestRecordsTextProto.MapDocument.Builder builder = TestRecordsTextProto.MapDocument.newBuilder(); - LuceneIndexKeyValueToPartialRecordUtils.buildPartialRecord(index, recordDescriptor, builder, "entry_value", "suggestion", Tuple.from(30, "r1")); - LuceneIndexKeyValueToPartialRecordUtils.populatePrimaryKey(primaryKey, recordDescriptor, builder, Tuple.from(30, 7)); - TestRecordsTextProto.MapDocument partialMsg = builder.build(); + TestRecordsTextProto.MapDocument.Builder newBuilder = TestRecordsTextProto.MapDocument.newBuilder(); + LuceneIndexKeyValueToPartialRecordUtils.buildPartialRecord(index, recordDescriptor, newBuilder, "entry_value", "suggestion", Tuple.from(30, "r1")); + LuceneIndexKeyValueToPartialRecordUtils.populatePrimaryKey(primaryKey, recordDescriptor, newBuilder, Tuple.from(30, 7)); + TestRecordsTextProto.MapDocument partialMsg = newBuilder.build(); assertEquals(1, partialMsg.getEntryCount()); TestRecordsTextProto.MapDocument.Entry entry = partialMsg.getEntry(0); @@ -349,13 +401,13 @@ void groupingMap() { @Test void groupingMapWithExtra() { - TestRecordsTextProto.MapDocument message = TestRecordsTextProto.MapDocument.newBuilder() + TestRecordsTextProto.MapDocument.Builder builder = TestRecordsTextProto.MapDocument.newBuilder() .setDocId(8) .addEntry(TestRecordsTextProto.MapDocument.Entry.newBuilder().setKey("en").setValue("first").setSecondValue("second")) .addEntry(TestRecordsTextProto.MapDocument.Entry.newBuilder().setKey("de").setValue("erste").setSecondValue("zweite")) .setGroup(40) - .setText2("extra") - .build(); + .setText2("extra"); + TestRecordsTextProto.MapDocument message = builder.build(); FDBRecord record = unstoredRecord(message); KeyExpression index = new GroupingKeyExpression(concat( field("group"), @@ -366,14 +418,22 @@ void groupingMapWithExtra() { Tuple.from(40, "de"), ImmutableList.of(textField("entry_value", "erste"), textField("entry_second_value", "zweite"), textField("text2", "extra"))), LuceneDocumentFromRecord.getRecordFields(index, record)); + // Highlight "second" for entry_second_value field + LuceneDocumentFromRecord.highlightTermsInMessage(index, builder, Map.of("entry_second_value", Set.of("second")), analyzerProvider, + new LuceneScanQueryParameters.LuceneQueryHighlightParameters(true)); + assertEquals("first", builder.build().getEntry(0).getValue()); + assertEquals("second", builder.build().getEntry(0).getSecondValue()); + assertEquals("erste", builder.build().getEntry(1).getValue()); + assertEquals("zweite", builder.build().getEntry(1).getSecondValue()); + KeyExpression primaryKey = concat(field("group"), field("doc_id")); // Build the partial record message for suggestion Descriptors.Descriptor recordDescriptor = message.getDescriptorForType(); - TestRecordsTextProto.MapDocument.Builder builder = TestRecordsTextProto.MapDocument.newBuilder(); - LuceneIndexKeyValueToPartialRecordUtils.buildPartialRecord(index, recordDescriptor, builder, "entry_second_value", "suggestion", Tuple.from(40, "en")); - LuceneIndexKeyValueToPartialRecordUtils.populatePrimaryKey(primaryKey, recordDescriptor, builder, Tuple.from(40, 8)); - TestRecordsTextProto.MapDocument partialMsg = builder.build(); + TestRecordsTextProto.MapDocument.Builder newBuilder = TestRecordsTextProto.MapDocument.newBuilder(); + LuceneIndexKeyValueToPartialRecordUtils.buildPartialRecord(index, recordDescriptor, newBuilder, "entry_second_value", "suggestion", Tuple.from(40, "en")); + LuceneIndexKeyValueToPartialRecordUtils.populatePrimaryKey(primaryKey, recordDescriptor, newBuilder, Tuple.from(40, 8)); + TestRecordsTextProto.MapDocument partialMsg = newBuilder.build(); assertEquals(1, partialMsg.getEntryCount()); TestRecordsTextProto.MapDocument.Entry entry = partialMsg.getEntry(0); @@ -394,11 +454,11 @@ void groupingMapWithExtra() { @Test void mapWithSubMessage() { - TestRecordsTextProto.NestedMapDocument message = TestRecordsTextProto.NestedMapDocument.newBuilder() + TestRecordsTextProto.NestedMapDocument.Builder builder = TestRecordsTextProto.NestedMapDocument.newBuilder() .setDocId(5) .setGroup(50) - .addEntry(TestRecordsTextProto.NestedMapDocument.Entry.newBuilder().setKey("k1").setSubEntry(TestRecordsTextProto.NestedMapDocument.SubEntry.newBuilder().setValue("testValue").build()).build()) - .build(); + .addEntry(TestRecordsTextProto.NestedMapDocument.Entry.newBuilder().setKey("k1").setSubEntry(TestRecordsTextProto.NestedMapDocument.SubEntry.newBuilder().setValue("testValue").build()).build()); + TestRecordsTextProto.NestedMapDocument message = builder.build(); KeyExpression index = field("entry", KeyExpression.FanType.FanOut) .nest(function(LuceneFunctionNames.LUCENE_FIELD_NAME, concat(field("sub_entry").nest(concat(function(LuceneFunctionNames.LUCENE_TEXT, field("value")), function(LuceneFunctionNames.LUCENE_TEXT, field("second_value")))), @@ -409,14 +469,19 @@ void mapWithSubMessage() { textField("entry_k1_value", "testValue"))), LuceneDocumentFromRecord.getRecordFields(index, record)); + // Highlight "testValue" for entry_k1_value field + LuceneDocumentFromRecord.highlightTermsInMessage(index, builder, Map.of("entry_k1_value", Set.of("testvalue")), analyzerProvider, + new LuceneScanQueryParameters.LuceneQueryHighlightParameters(true)); + assertEquals("testValue", builder.build().getEntry(0).getSubEntry().getValue()); + KeyExpression primaryKey = concat(field("group"), field("doc_id")); // Build the partial record message for suggestion Descriptors.Descriptor recordDescriptor = message.getDescriptorForType(); - TestRecordsTextProto.NestedMapDocument.Builder builder = TestRecordsTextProto.NestedMapDocument.newBuilder(); - LuceneIndexKeyValueToPartialRecordUtils.buildPartialRecord(index, recordDescriptor, builder, "entry_k1_value", "suggestion"); - LuceneIndexKeyValueToPartialRecordUtils.populatePrimaryKey(primaryKey, recordDescriptor, builder, Tuple.from(50, 5)); - TestRecordsTextProto.NestedMapDocument partialMsg = builder.build(); + TestRecordsTextProto.NestedMapDocument.Builder newBuilder = TestRecordsTextProto.NestedMapDocument.newBuilder(); + LuceneIndexKeyValueToPartialRecordUtils.buildPartialRecord(index, recordDescriptor, newBuilder, "entry_k1_value", "suggestion"); + LuceneIndexKeyValueToPartialRecordUtils.populatePrimaryKey(primaryKey, recordDescriptor, newBuilder, Tuple.from(50, 5)); + TestRecordsTextProto.NestedMapDocument partialMsg = newBuilder.build(); assertEquals(1, partialMsg.getEntryCount()); TestRecordsTextProto.NestedMapDocument.Entry entry = partialMsg.getEntry(0); diff --git a/fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/LuceneIndexTest.java b/fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/LuceneIndexTest.java index 5ff5fdcd79..098d76f749 100644 --- a/fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/LuceneIndexTest.java +++ b/fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/LuceneIndexTest.java @@ -340,9 +340,15 @@ protected FDBRecordContextConfig.Builder contextConfig(@Nonnull final RecordLaye } private LuceneScanBounds fullTextSearch(Index index, String search) { + return fullTextSearch(index, search, false); + } + + private LuceneScanBounds fullTextSearch(Index index, String search, boolean highlight) { LuceneScanParameters scan = new LuceneScanQueryParameters( ScanComparisons.EMPTY, - new LuceneQueryMultiFieldSearchClause(search, false)); + new LuceneQueryMultiFieldSearchClause(search, false), + null, null, null, + new LuceneScanQueryParameters.LuceneQueryHighlightParameters(false)); return scan.bind(recordStore, index, EvaluationContext.EMPTY); }