From 606bfb804abb199fe9f202581027c2fb5c62bd7d Mon Sep 17 00:00:00 2001 From: Alec Grieser Date: Tue, 24 May 2022 18:07:51 -0700 Subject: [PATCH] Resolves #1682: Support auto-complete via querying main Lucene directory (#1683) * Current State is rough * Resolves #1682: Support auto-complete via querying main Lucene directory This modifies the auto-complete cursor so that it reads from the main Lucene index and then reconstitutes the original text by reading from the base record data. Co-authored-by: john_leach --- docs/ReleaseNotes.md | 2 +- .../LuceneAutoCompleteResultCursor.java | 501 +++++++++++++-- .../lucene/LuceneCursorContinuation.java | 79 +++ .../record/lucene/LuceneIndexMaintainer.java | 86 +-- .../record/lucene/LuceneIndexOptions.java | 6 + .../lucene/LuceneRecordContextProperties.java | 8 +- .../record/lucene/LuceneRecordCursor.java | 11 +- ...endedInfixSuggesterWithoutTermVectors.java | 571 ------------------ ...OptimizedWrappedBlendedInfixSuggester.java | 142 ----- .../lucene/directory/FDBDirectoryManager.java | 16 - .../lucene/directory/FDBDirectoryWrapper.java | 130 ++-- .../LuceneAutoCompleteResultCursorTest.java | 132 ++++ .../record/lucene/LuceneIndexTest.java | 263 ++++---- 13 files changed, 835 insertions(+), 1112 deletions(-) create mode 100644 fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneCursorContinuation.java delete mode 100644 fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/codec/LuceneOptimizedBlendedInfixSuggesterWithoutTermVectors.java delete mode 100644 fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/codec/LuceneOptimizedWrappedBlendedInfixSuggester.java create mode 100644 fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/LuceneAutoCompleteResultCursorTest.java diff --git a/docs/ReleaseNotes.md b/docs/ReleaseNotes.md index 96e9a33ced..e8d59840f7 100644 --- a/docs/ReleaseNotes.md +++ b/docs/ReleaseNotes.md @@ -26,7 +26,7 @@ This release also updates downstream dependency versions. Most notably, the prot * **Bug fix** Fix 4 [(Issue #NNN)](https://github.com/FoundationDB/fdb-record-layer/issues/NNN) * **Bug fix** Failed no-ops no longer log at `ERROR` [(Issue #1692)](https://github.com/FoundationDB/fdb-record-layer/issues/1692) * **Performance** Improvement 1 [(Issue #NNN)](https://github.com/FoundationDB/fdb-record-layer/issues/NNN) -* **Performance** Improvement 2 [(Issue #NNN)](https://github.com/FoundationDB/fdb-record-layer/issues/NNN) +* **Performance** Lucene auto-complete is now handled by running queries on the main index to allow it to avoid needing a separate directory [(Issue #1682)](https://github.com/FoundationDB/fdb-record-layer/issues/1682) * **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) 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 43136f50ff..e28f7be0fa 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 @@ -20,32 +20,67 @@ package com.apple.foundationdb.record.lucene; -import com.apple.foundationdb.record.ByteArrayContinuation; import com.apple.foundationdb.record.IndexEntry; +import com.apple.foundationdb.record.PipelineOperation; import com.apple.foundationdb.record.RecordCoreArgumentException; import com.apple.foundationdb.record.RecordCoreException; +import com.apple.foundationdb.record.RecordCursor; import com.apple.foundationdb.record.RecordCursorContinuation; import com.apple.foundationdb.record.RecordCursorResult; import com.apple.foundationdb.record.RecordCursorVisitor; import com.apple.foundationdb.record.ScanProperties; import com.apple.foundationdb.record.cursors.BaseCursor; +import com.apple.foundationdb.record.logging.KeyValueLogMessage; import com.apple.foundationdb.record.logging.LogMessageKeys; +import com.apple.foundationdb.record.lucene.directory.FDBDirectoryManager; +import com.apple.foundationdb.record.provider.foundationdb.FDBRecord; import com.apple.foundationdb.record.provider.foundationdb.FDBStoreTimer; import com.apple.foundationdb.record.provider.foundationdb.IndexMaintainerState; +import com.apple.foundationdb.record.provider.foundationdb.SubspaceProvider; import com.apple.foundationdb.tuple.Tuple; -import com.google.protobuf.ByteString; -import org.apache.lucene.search.suggest.Lookup; -import org.apache.lucene.search.suggest.analyzing.AnalyzingInfixSuggester; +import com.apple.foundationdb.tuple.TupleHelpers; +import com.google.common.annotations.VisibleForTesting; +import org.apache.lucene.analysis.Analyzer; +import org.apache.lucene.analysis.TokenStream; +import org.apache.lucene.analysis.tokenattributes.CharTermAttribute; +import org.apache.lucene.analysis.tokenattributes.OffsetAttribute; +import org.apache.lucene.index.IndexNotFoundException; +import org.apache.lucene.index.IndexOptions; +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.IndexSearcher; +import org.apache.lucene.search.PhraseQuery; +import org.apache.lucene.search.PrefixQuery; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.ScoreDoc; +import org.apache.lucene.search.TermQuery; +import org.apache.lucene.search.TopDocs; +import org.apache.lucene.search.spans.FieldMaskingSpanQuery; +import org.apache.lucene.search.spans.SpanMultiTermQueryWrapper; +import org.apache.lucene.search.spans.SpanNearQuery; +import org.apache.lucene.search.spans.SpanQuery; +import org.apache.lucene.search.spans.SpanTermQuery; +import org.apache.lucene.util.BytesRef; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.io.IOException; -import java.util.Collections; +import java.io.StringReader; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashSet; import java.util.List; +import java.util.Objects; +import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; +import java.util.function.Function; /** * This class is a Record Cursor implementation for Lucene auto complete suggestion lookup. @@ -62,89 +97,60 @@ public class LuceneAutoCompleteResultCursor implements BaseCursor { private final String query; @Nullable private final FDBStoreTimer timer; - private int limit; + private final int limit; + private final int skip; @Nullable - private List lookupResults = null; - private int currentPosition; + private RecordCursor lookupResults = null; @Nullable private final Tuple groupingKey; - @Nonnull - private final AnalyzingInfixSuggester suggester; private final boolean highlight; + private final Analyzer queryAnalyzer; - public LuceneAutoCompleteResultCursor(@Nonnull AnalyzingInfixSuggester suggester, @Nonnull String query, + public LuceneAutoCompleteResultCursor(@Nonnull String query, @Nonnull Executor executor, @Nonnull ScanProperties scanProperties, - @Nonnull IndexMaintainerState state, @Nullable Tuple groupingKey, boolean highlight) { + @Nonnull Analyzer queryAnalyzer, @Nonnull IndexMaintainerState state, + @Nullable Tuple groupingKey, boolean highlight) { if (query.isEmpty()) { throw new RecordCoreArgumentException("Invalid query for auto-complete search") .addLogInfo(LogMessageKeys.QUERY, query) .addLogInfo(LogMessageKeys.INDEX_NAME, state.index.getName()); } - this.suggester = suggester; this.query = query; this.executor = executor; - this.state = state; this.limit = Math.min(scanProperties.getExecuteProperties().getReturnedRowLimitOrMax(), state.context.getPropertyStorage().getPropertyValue(LuceneRecordContextProperties.LUCENE_AUTO_COMPLETE_SEARCH_LIMITATION)); + this.skip = scanProperties.getExecuteProperties().getSkip(); this.timer = state.context.getTimer(); - this.currentPosition = 0; - if (scanProperties.getExecuteProperties().getSkip() > 0) { - this.currentPosition += scanProperties.getExecuteProperties().getSkip(); - } - this.groupingKey = groupingKey; this.highlight = highlight; + this.state = state; + this.groupingKey = groupingKey; + this.queryAnalyzer = queryAnalyzer; + } + + private synchronized IndexReader getIndexReader() throws IOException { + return FDBDirectoryManager.getManager(state).getIndexReader(groupingKey); } - @SuppressWarnings("cast") @Nonnull @Override public CompletableFuture> onNext() { - CompletableFuture lookupResult = CompletableFuture.supplyAsync(() -> { - if (lookupResults == null) { + if (lookupResults == null) { + return CompletableFuture.>>supplyAsync(() -> { try { performLookup(); + } catch (IndexNotFoundException indexNotFoundException) { + // Trying to open an empty directory results in an IndexNotFoundException, + // but this should be interpreted as there not being any data to read + return CompletableFuture.completedFuture(RecordCursorResult.exhausted()); } catch (IOException ioException) { throw new RecordCoreException("Exception to lookup the auto complete suggestions", ioException) .addLogInfo(LogMessageKeys.QUERY, query); } - } - return currentPosition < lookupResults.size() ? lookupResults.get(currentPosition) : null; - }, executor); - - return lookupResult.thenApply(r -> { - if (r == null) { - return RecordCursorResult.exhausted(); - } else { - final String suggestion = highlight ? (String) r.highlightKey : (String) r.key; - - if (r.payload == null) { - throw new RecordCoreException("Empty payload of lookup result for lucene auto complete suggestion") - .addLogInfo(LogMessageKeys.QUERY, query) - .addLogInfo(LogMessageKeys.RESULT, suggestion); - } - - Tuple key = Tuple.fromBytes(r.payload.bytes).add(suggestion); - if (groupingKey != null) { - key = groupingKey.addAll(key); - } - IndexEntry indexEntry = new IndexEntry(state.index, key, Tuple.from(r.value)); - - if (LOGGER.isTraceEnabled()) { - LOGGER.trace("Suggestion read as an index entry={}", indexEntry); - } - - return RecordCursorResult.withNextValue(indexEntry, continuationHelper(lookupResults.get(currentPosition++))); - } - }); - } - - @Nonnull - private static RecordCursorContinuation continuationHelper(@Nonnull Lookup.LookupResult lookupResult) { - LuceneContinuationProto.LuceneAutoCompleteIndexContinuation.Builder continuationBuilder = LuceneContinuationProto.LuceneAutoCompleteIndexContinuation.newBuilder().setKey((String) lookupResult.key); - continuationBuilder.setValue(lookupResult.value); - continuationBuilder.setPayload(ByteString.copyFrom(lookupResult.payload.bytes)); - return ByteArrayContinuation.fromNullable(continuationBuilder.build().toByteArray()); + return lookupResults.onNext(); + }).thenCompose(Function.identity()); + } + return lookupResults.onNext(); } @Override @@ -169,12 +175,381 @@ private void performLookup() throws IOException { return; } long startTime = System.nanoTime(); - lookupResults = suggester.getCount() > 0 - ? suggester.lookup(query, Collections.emptySet(), limit, true, highlight) - : Collections.emptyList(); + + lookupResults = lookup().skip(skip); if (timer != null) { timer.recordSinceNanoTime(LuceneEvents.Events.LUCENE_AUTO_COMPLETE_SUGGESTIONS_SCAN, startTime); - timer.increment(LuceneEvents.Counts.LUCENE_SCAN_MATCHED_AUTO_COMPLETE_SUGGESTIONS, lookupResults.size()); } } + + @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))) { + CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class); + OffsetAttribute offsetAtt = ts.addAttribute(OffsetAttribute.class); + ts.reset(); + StringBuilder sb = highlight ? new StringBuilder() : null; + int upto = 0; + Set matchedInText = new HashSet<>(); + boolean matchedPrefix = false; + 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)); + } + upto = startOffset; + } else if (upto > startOffset) { + continue; + } + + if (matchedTokens.contains(token)) { + // Token matches. + if (highlight) { + addWholeMatch(sb, text.substring(startOffset, endOffset)); + } + upto = endOffset; + matchedInText.add(token); + } else if (prefixToken != null && token.startsWith(prefixToken)) { + if (highlight) { + addPrefixMatch(sb, text.substring(startOffset, endOffset), prefixToken); + } + upto = endOffset; + matchedPrefix = true; + } + } + ts.end(); + + if ((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) { + int endOffset = offsetAtt.endOffset(); + if (upto < endOffset) { + addNonMatch(sb, text.substring(upto)); + } + return sb.toString(); + } else { + return text; + } + + } catch (IOException e) { + return null; + } + } + + /** 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 + */ + private static void addWholeMatch(StringBuilder sb, String surface) { + sb.append(""); + sb.append(surface); + sb.append(""); + } + + /** 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 + */ + private static void addPrefixMatch(StringBuilder sb, String surface, String prefixToken) { + // 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); + return; + } + sb.append(""); + sb.append(surface.substring(0, prefixToken.length())); + sb.append(""); + sb.append(surface.substring(prefixToken.length())); + } + + public RecordCursor lookup() throws IOException { + // Determine the tokens from the query key + final boolean phraseQueryNeeded = query.startsWith("\"") && query.endsWith("\""); + final String searchKey = phraseQueryNeeded ? query.substring(1, query.length() - 1) : query; + List tokens = new ArrayList<>(); + final String prefixToken = getQueryTokens(queryAnalyzer, searchKey, tokens); + + IndexReader indexReader = getIndexReader(); + Set fieldNames = getAllIndexedFieldNames(indexReader); + + final Set tokenSet = new HashSet<>(tokens); + // Note: phrase matching needs to know token order, so we pass the *list* of tokens to the phrase + // matching query, but if we don't need the phrase query, we pass the *set* because order doesn't + // matter and we want to remove duplicates + Query finalQuery = phraseQueryNeeded + ? buildQueryForPhraseMatching(fieldNames, tokens, prefixToken) + : buildQueryForTermsMatching(fieldNames, tokenSet, prefixToken); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(logMessage("query for auto-complete") + .addKeyAndValue(LogMessageKeys.QUERY, query.replace("\"", "\\\"")) + .addKeyAndValue("lucene_query", finalQuery) + .toString()); + } + + IndexSearcher searcher = new IndexSearcher(indexReader, executor); + TopDocs topDocs = searcher.search(finalQuery, limit); + if (timer != null) { + timer.increment(LuceneEvents.Counts.LUCENE_SCAN_MATCHED_AUTO_COMPLETE_SUGGESTIONS, topDocs.scoreDocs.length); + } + return createResults(searcher, topDocs, tokenSet, prefixToken); + } + + private Set getAllIndexedFieldNames(IndexReader indexReader) { + Set fieldNames = new HashSet<>(); + indexReader.leaves().forEach(leaf -> leaf.reader().getFieldInfos().forEach(fieldInfo -> { + // Exclude any field where the field's IndexOptions indicate that the field data is not indexed + if (fieldInfo.getIndexOptions() != IndexOptions.NONE) { + fieldNames.add(fieldInfo.name); + } + })); + return fieldNames; + } + + /** + * Extract the query tokens from a string. All of the tokens (except the last one) will be added to the + * {@code tokens} list. The last token is special. If the there is no whitespace following that token, + * this indicates that this is an incomplete prefix of a token that will be completed by the query. + * If there is whitespace following that token, then it is assumed that token is complete and is added + * to the {@code tokens} list. The final token will be returned by this method if and only if we are in + * the former case. + * + * @param searchKey the phrase to find completions of + * @param tokens the list to insert all complete tokens extracted from the query phrase + * @return the final token if it needs to be added as a "prefix" component to the final query + * @throws IOException from the analyzer attempting to tokenize the query + */ + @Nullable + @VisibleForTesting + static String getQueryTokens(Analyzer queryAnalyzer, String searchKey, @Nonnull List tokens) throws IOException { + String prefixToken = null; + try (TokenStream ts = queryAnalyzer.tokenStream("", new StringReader(searchKey))) { + ts.reset(); + final CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class); + final OffsetAttribute offsetAtt = ts.addAttribute(OffsetAttribute.class); + String lastToken = null; + int maxEndOffset = -1; + while (ts.incrementToken()) { + if (lastToken != null) { + tokens.add(lastToken); + } + lastToken = termAtt.toString(); + if (lastToken != null) { + maxEndOffset = Math.max(maxEndOffset, offsetAtt.endOffset()); + } + } + ts.end(); + + if (lastToken != null) { + if (maxEndOffset == offsetAtt.endOffset()) { + // Use PrefixQuery (or the ngram equivalent) when + // there was no trailing discarded chars in the + // string (e.g. whitespace), so that if query does + // not end with a space we show prefix matches for + // that token: + prefixToken = lastToken; + } else { + // Use TermQuery for an exact match if there were + // trailing discarded chars (e.g. whitespace), so + // that if query ends with a space we only show + // exact matches for that term: + tokens.add(lastToken); + } + } + } + return prefixToken; + } + + @Nullable + private Query buildQueryForPhraseMatching(@Nonnull Collection fieldNames, + @Nonnull List matchedTokens, + @Nullable String prefixToken) { + // Construct a query that is essentially: + // - in any field, + // - the phrase must occur (with possibly the last token in the phrase as a prefix) + BooleanQuery.Builder queryBuilder = new BooleanQuery.Builder(); + + for (String field : fieldNames) { + PhraseQuery.Builder phraseQueryBuilder = new PhraseQuery.Builder(); + for (String token : matchedTokens) { + phraseQueryBuilder.add(new Term(field, token)); + } + Query fieldQuery; + if (prefixToken == null) { + fieldQuery = phraseQueryBuilder.build(); + } else { + fieldQuery = getPhrasePrefixQuery(field, phraseQueryBuilder.build(), prefixToken); + } + queryBuilder.add(fieldQuery, BooleanClause.Occur.SHOULD); + } + + queryBuilder.setMinimumNumberShouldMatch(1); + return queryBuilder.build(); + } + + @Nullable + private Query buildQueryForTermsMatching(@Nonnull Collection fieldNames, + @Nonnull Set tokenSet, + @Nullable String prefixToken) { + // Construct a query that is essentially: + // - in any field, + // - all of the tokens must occur (with the last one as a prefix) + BooleanQuery.Builder queryBuilder = new BooleanQuery.Builder(); + + for (String field : fieldNames) { + BooleanQuery.Builder fieldQuery = new BooleanQuery.Builder(); + for (String token : tokenSet) { + fieldQuery.add(new TermQuery(new Term(field, token)), BooleanClause.Occur.MUST); + } + if (prefixToken != null) { + fieldQuery.add(new PrefixQuery(new Term(field, prefixToken)), BooleanClause.Occur.MUST); + } + queryBuilder.add(fieldQuery.build(), BooleanClause.Occur.SHOULD); + } + queryBuilder.setMinimumNumberShouldMatch(1); + return queryBuilder.build(); + } + + private Query getPhrasePrefixQuery(@Nonnull String fieldName, @Nonnull PhraseQuery phraseQuery, @Nonnull String lastToken) { + Term[] terms = phraseQuery.getTerms(); + SpanNearQuery.Builder spanQuery = new SpanNearQuery.Builder(fieldName, true); // field + for (Term term : terms) { + spanQuery.addClause(new SpanTermQuery(term)); + } + SpanQuery lastTokenQuery = new SpanMultiTermQueryWrapper<>(new PrefixQuery(new Term(fieldName, lastToken))); + FieldMaskingSpanQuery fieldMask = new FieldMaskingSpanQuery(lastTokenQuery, fieldName); + spanQuery.addClause(fieldMask); + + return spanQuery.build(); + } + + protected RecordCursor createResults(IndexSearcher searcher, + TopDocs topDocs, + Set queryTokens, + @Nullable String prefixToken) { + return RecordCursor.flatMapPipelined( + outerContinuation -> scoreDocsFromLookup(searcher, topDocs), + (scoreDocAndRecord, innerContinuation) -> findIndexEntriesInRecord(scoreDocAndRecord, queryTokens, prefixToken, innerContinuation), + scoreDocAndRecord -> scoreDocAndRecord.rec.getPrimaryKey().pack(), + null, + 1 // Use a pipeline size of 1 because the inner cursors don't do I/O and the outer cursor has its own pipelining + ); + } + + private static final class ScoreDocAndRecord { + private final ScoreDoc scoreDoc; + private final FDBRecord rec; + + private ScoreDocAndRecord(ScoreDoc scoreDoc, FDBRecord rec) { + this.scoreDoc = scoreDoc; + this.rec = rec; + } + } + + private RecordCursor scoreDocsFromLookup(IndexSearcher searcher, TopDocs topDocs) { + return RecordCursor.fromIterator(executor, Arrays.stream(topDocs.scoreDocs).iterator()) + .mapPipelined(scoreDoc -> loadRecordFromScoreDocAsync(searcher, scoreDoc), state.store.getPipelineSize(PipelineOperation.KEY_TO_RECORD)) + .filter(Objects::nonNull) + .mapResult(result -> { + if (result.hasNext()) { + // TODO: this cursor does not support real continuations (yet) + // However, if we want to use the "searchAfter" to resume this scan, this is the + // continuation we'd need for it + RecordCursorContinuation continuationFromDoc = LuceneCursorContinuation.fromScoreDoc(result.get().scoreDoc); + return RecordCursorResult.withNextValue(result.get(), continuationFromDoc); + } else { + // TODO: This always overrides the NoNextReason to SOURCE_EXHAUSTED + // This means that if we wanted to support paginating auto-complete queries with the "limit" + // field, we'd need to do a better job here and gather the final returned result, or something + // and use it as the continuation if the cursor terminates early + return RecordCursorResult.exhausted(); + } + }); + } + + private CompletableFuture loadRecordFromScoreDocAsync(IndexSearcher searcher, ScoreDoc scoreDoc) { + try { + IndexableField primaryKey = searcher.doc(scoreDoc.doc).getField(LuceneIndexMaintainer.PRIMARY_KEY_FIELD_NAME); + BytesRef pk = primaryKey.binaryValue(); + return state.store.loadRecordAsync(Tuple.fromBytes(pk.bytes)).thenApply(rec -> new ScoreDocAndRecord(scoreDoc, rec)); + } catch (IOException e) { + return CompletableFuture.failedFuture(new RecordCoreException("unable to read document from Lucene", e)); + } + } + + private RecordCursor findIndexEntriesInRecord(ScoreDocAndRecord scoreDocAndRecord, Set queryTokens, @Nullable String prefixToken, @Nullable byte[] continuation) { + // Extract the indexed fields from the document again + final List documentFields = LuceneDocumentFromRecord.getRecordFields(state.index.getRootExpression(), scoreDocAndRecord.rec) + .get(groupingKey == null ? TupleHelpers.EMPTY : groupingKey); + return RecordCursor.fromList(executor, documentFields, continuation).map(documentField -> { + // Search each field to find the first match. + final int maxTextLength = Objects.requireNonNull(state.context.getPropertyStorage() + .getPropertyValue(LuceneRecordContextProperties.LUCENE_AUTO_COMPLETE_TEXT_SIZE_UPPER_LIMIT)); + Object fieldValue = documentField.getValue(); + if (!(fieldValue instanceof String)) { + // Only can search through string fields + return null; + } + String text = (String)fieldValue; + if (text.length() > maxTextLength) { + // Apply the text length filter before searching through the text for the + // matched terms + return null; + } + String match = searchAllMaybeHighlight(queryAnalyzer, text, queryTokens, prefixToken, highlight); + if (match == null) { + // Text not found in this field + return null; + } + + // Found a match with this field! + Tuple key = Tuple.from(documentField.getFieldName(), match); + if (groupingKey != null) { + key = groupingKey.addAll(key); + } + // TODO: Add the primary key to the index entry + // Not having the primary key is fine for auto-complete queries that just want the + // text, but queries wanting to do something with both the auto-completed text and the + // original record need to do something else + IndexEntry indexEntry = new IndexEntry(state.index, key, Tuple.from(scoreDocAndRecord.scoreDoc.score)); + if (LOGGER.isTraceEnabled()) { + LOGGER.trace(logMessage("Suggestion read as an index entry") + .addKeyAndValue(LogMessageKeys.INDEX_KEY, key) + .addKeyAndValue(LogMessageKeys.INDEX_VALUE, indexEntry.getValue()) + .toString()); + } + return indexEntry; + }).filter(Objects::nonNull); // Note: may not return any results if all matches exceed the maxTextLength + } + + private KeyValueLogMessage logMessage(String staticMessage) { + final SubspaceProvider subspaceProvider = state.store.getSubspaceProvider(); + return KeyValueLogMessage.build(staticMessage) + .addKeyAndValue(LogMessageKeys.INDEX_NAME, state.index.getName()) + .addKeyAndValue(subspaceProvider.logKey(), subspaceProvider.toString(state.context)); + } } diff --git a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneCursorContinuation.java b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneCursorContinuation.java new file mode 100644 index 0000000000..5f4e39a7c4 --- /dev/null +++ b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneCursorContinuation.java @@ -0,0 +1,79 @@ +/* + * LuceneCursorContinuation.java + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2015-2022 Apple Inc. and the FoundationDB project authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.apple.foundationdb.record.lucene; + +import com.apple.foundationdb.record.RecordCursorContinuation; +import com.google.protobuf.ByteString; +import org.apache.lucene.search.ScoreDoc; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +/** + * Continuation from scanning a Lucene index. This wraps the LuceneIndexContinuation protobuf message, + * which contains enough information to use the Lucene + * {@link org.apache.lucene.search.IndexSearcher#searchAfter(ScoreDoc, org.apache.lucene.search.Query, int) searchAfter} + * feature to resume a query. + */ +class LuceneCursorContinuation implements RecordCursorContinuation { + @Nonnull + private final LuceneContinuationProto.LuceneIndexContinuation protoContinuation; + + @SuppressWarnings("squid:S3077") // Byte array is immutable once created, so does not need to use atomic array + private volatile byte[] byteContinuation; + + private LuceneCursorContinuation(@Nonnull LuceneContinuationProto.LuceneIndexContinuation protoContinuation) { + this.protoContinuation = protoContinuation; + } + + @Nullable + @Override + public byte[] toBytes() { + if (byteContinuation == null) { + synchronized (this) { + if (byteContinuation == null) { + byteContinuation = toByteString().toByteArray(); + } + } + } + return byteContinuation; + } + + @Nonnull + @Override + public ByteString toByteString() { + return protoContinuation.toByteString(); + } + + @Override + public boolean isEnd() { + return false; + } + + public static LuceneCursorContinuation fromScoreDoc(ScoreDoc scoreDoc) { + return new LuceneCursorContinuation(LuceneContinuationProto.LuceneIndexContinuation.newBuilder() + .setDoc(scoreDoc.doc) + .setShard(scoreDoc.shardIndex) + .setScore(scoreDoc.score) + .build() + ); + } +} 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 d79fd6f687..4840230258 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 @@ -32,7 +32,6 @@ import com.apple.foundationdb.record.RecordCursor; import com.apple.foundationdb.record.ScanProperties; import com.apple.foundationdb.record.TupleRange; -import com.apple.foundationdb.record.logging.KeyValueLogMessage; import com.apple.foundationdb.record.logging.LogMessageKeys; import com.apple.foundationdb.record.lucene.directory.FDBDirectoryManager; import com.apple.foundationdb.record.metadata.IndexAggregateFunction; @@ -47,10 +46,10 @@ import com.apple.foundationdb.record.provider.foundationdb.IndexScanBounds; import com.apple.foundationdb.record.provider.foundationdb.indexes.InvalidIndexEntry; import com.apple.foundationdb.record.provider.foundationdb.indexes.StandardIndexMaintainer; -import com.apple.foundationdb.record.provider.foundationdb.properties.RecordLayerPropertyKey; import com.apple.foundationdb.record.query.QueryToKeyMatcher; import com.apple.foundationdb.tuple.Tuple; import com.google.protobuf.Message; +import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.document.Document; import org.apache.lucene.document.DoublePoint; import org.apache.lucene.document.Field; @@ -64,7 +63,6 @@ import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.search.Query; -import org.apache.lucene.search.suggest.analyzing.AnalyzingInfixSuggester; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.NumericUtils; import org.slf4j.Logger; @@ -73,10 +71,8 @@ import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.io.IOException; -import java.nio.charset.StandardCharsets; import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; +import java.util.EnumMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -96,7 +92,6 @@ public class LuceneIndexMaintainer extends StandardIndexMaintainer { private static final Logger LOG = LoggerFactory.getLogger(LuceneIndexMaintainer.class); private final FDBDirectoryManager directoryManager; private final AnalyzerChooser indexAnalyzerChooser; - private final AnalyzerChooser autoCompleteIndexAnalyzerChooser; private final AnalyzerChooser autoCompleteQueryAnalyzerChooser; protected static final String PRIMARY_KEY_FIELD_NAME = "p"; // TODO: Need to find reserved names.. protected static final String PRIMARY_KEY_SEARCH_NAME = "s"; // TODO: Need to find reserved names.. @@ -109,7 +104,6 @@ public LuceneIndexMaintainer(@Nonnull final IndexMaintainerState state, @Nonnull this.executor = executor; this.directoryManager = FDBDirectoryManager.getManager(state); this.indexAnalyzerChooser = LuceneAnalyzerRegistryImpl.instance().getLuceneAnalyzerChooserPair(state.index, LuceneAnalyzerType.FULL_TEXT).getLeft(); - this.autoCompleteIndexAnalyzerChooser = LuceneAnalyzerRegistryImpl.instance().getLuceneAnalyzerChooserPair(state.index, LuceneAnalyzerType.AUTO_COMPLETE).getLeft(); this.autoCompleteQueryAnalyzerChooser = LuceneAnalyzerRegistryImpl.instance().getLuceneAnalyzerChooserPair(state.index, LuceneAnalyzerType.AUTO_COMPLETE).getRight(); this.autoCompleteEnabled = state.index.getBooleanOption(LuceneIndexOptions.AUTO_COMPLETE_ENABLED, false); this.highlightForAutoCompleteIfEnabled = state.index.getBooleanOption(LuceneIndexOptions.AUTO_COMPLETE_HIGHLIGHT, false); @@ -152,14 +146,9 @@ public RecordCursor scan(@Nonnull final IndexScanBounds scanBounds, .addLogInfo(LogMessageKeys.INDEX_NAME, state.index.getName()); } LuceneScanAutoComplete scanAutoComplete = (LuceneScanAutoComplete)scanBounds; - try { - return new LuceneAutoCompleteResultCursor(getSuggester(scanAutoComplete.getGroupKey(), - Collections.singletonList(scanAutoComplete.getKeyToComplete()), null), scanAutoComplete.getKeyToComplete(), - executor, scanProperties, state, scanAutoComplete.getGroupKey(), highlightForAutoCompleteIfEnabled); - } catch (IOException ex) { - throw new RecordCoreException("Exception to get suggester for auto-complete search", ex) - .addLogInfo(LogMessageKeys.INDEX_NAME, state.index.getName()); - } + Analyzer analyzer = autoCompleteQueryAnalyzerChooser.chooseAnalyzer(scanAutoComplete.getKeyToComplete()).getAnalyzer(); + return new LuceneAutoCompleteResultCursor(scanAutoComplete.getKeyToComplete(), + executor, scanProperties, analyzer, state, scanAutoComplete.getGroupKey(), highlightForAutoCompleteIfEnabled); } if (scanType.equals(LuceneScanTypes.BY_LUCENE_SPELL_CHECK)) { @@ -174,62 +163,21 @@ public RecordCursor scan(@Nonnull final IndexScanBounds scanBounds, throw new RecordCoreException("unsupported scan type for Lucene index: " + scanType); } - private boolean addTermToSuggesterIfNeeded(@Nonnull String value, @Nonnull String fieldName, @Nullable AnalyzingInfixSuggester suggester) { - if (suggester == null) { - return false; - } - - final byte[] valueBytes = value.getBytes(StandardCharsets.UTF_8); - final RecordLayerPropertyKey sizeLimitProp = LuceneRecordContextProperties.LUCENE_AUTO_COMPLETE_TEXT_SIZE_UPPER_LIMIT; - final int sizeLimit = Objects.requireNonNullElse(state.context.getPropertyStorage().getPropertyValue(sizeLimitProp), sizeLimitProp.getDefaultValue()).intValue(); - // Ignore this text if its size exceeds the limitation - if (valueBytes.length > sizeLimit) { - if (LOG.isTraceEnabled()) { - LOG.trace(KeyValueLogMessage.of("Skip auto-complete indexing due to exceeding size limitation", - LuceneLogMessageKeys.DATA_SIZE, valueBytes.length, - LuceneLogMessageKeys.DATA_VALUE, value.substring(0, Math.min(value.length(), 100)), - LogMessageKeys.FIELD_NAME, fieldName)); - } - return false; - } - - try { - suggester.add(new BytesRef(valueBytes), - Set.of(new BytesRef(fieldName.getBytes(StandardCharsets.UTF_8))), - state.context.getPropertyStorage().getPropertyValue(LuceneRecordContextProperties.LUCENE_AUTO_COMPLETE_DEFAULT_WEIGHT), - new BytesRef(Tuple.from(fieldName).pack())); - if (LOG.isTraceEnabled()) { - LOG.trace(KeyValueLogMessage.of("Added auto-complete suggestion to suggester", - LuceneLogMessageKeys.DATA_SIZE, valueBytes.length, - LuceneLogMessageKeys.DATA_VALUE, value.substring(0, Math.min(value.length(), 100)), - LogMessageKeys.FIELD_NAME, fieldName)); - } - return true; - } catch (IOException ex) { - throw new RecordCoreException("Exception to add term into suggester", ex) - .addLogInfo(LogMessageKeys.INDEX_NAME, state.index.getName()); - } - } - /** * Insert a field into the document and add a suggestion into the suggester if needed. - * @return whether a suggestion has been added to the suggester */ @SuppressWarnings("java:S3776") - private boolean insertField(LuceneDocumentFromRecord.DocumentField field, final Document document, - @Nullable AnalyzingInfixSuggester suggester) { + private void insertField(LuceneDocumentFromRecord.DocumentField field, final Document document) { final String fieldName = field.getFieldName(); final Object value = field.getValue(); final Field luceneField; final Field sortedField; final StoredField storedField; - boolean suggestionAdded = false; switch (field.getType()) { case TEXT: luceneField = new Field(fieldName, (String) value, getTextFieldType(field)); sortedField = null; storedField = null; - suggestionAdded = addTermToSuggesterIfNeeded((String) value, fieldName, suggester); break; case STRING: luceneField = new StringField(fieldName, (String)value, field.isStored() ? Field.Store.YES : Field.Store.NO); @@ -266,7 +214,6 @@ private boolean insertField(LuceneDocumentFromRecord.DocumentField field, final if (storedField != null) { document.add(storedField); } - return suggestionAdded; } private void writeDocument(@Nonnull List fields, Tuple groupingKey, @@ -274,22 +221,17 @@ private void writeDocument(@Nonnull List final List texts = fields.stream() .filter(f -> f.getType().equals(LuceneIndexExpressions.DocumentFieldType.TEXT)) .map(f -> (String) f.getValue()).collect(Collectors.toList()); + Document document = new Document(); final IndexWriter newWriter = directoryManager.getIndexWriter(groupingKey, indexAnalyzerChooser.chooseAnalyzer(texts)); BytesRef ref = new BytesRef(primaryKey); - Document document = new Document(); document.add(new StoredField(PRIMARY_KEY_FIELD_NAME, ref)); document.add(new SortedDocValuesField(PRIMARY_KEY_SEARCH_NAME, ref)); Map> indexOptionsToFieldsMap = getIndexOptionsToFieldsMap(fields); for (Map.Entry> entry : indexOptionsToFieldsMap.entrySet()) { - final AnalyzingInfixSuggester suggester = autoCompleteEnabled ? getSuggester(groupingKey, texts, entry.getKey()) : null; - boolean suggestionAdded = false; for (LuceneDocumentFromRecord.DocumentField field : entry.getValue()) { - suggestionAdded = insertField(field, document, suggester) || suggestionAdded; - } - if (suggestionAdded) { - suggester.refresh(); + insertField(field, document); } } newWriter.addDocument(document); @@ -297,7 +239,7 @@ private void writeDocument(@Nonnull List @Nonnull private Map> getIndexOptionsToFieldsMap(@Nonnull List fields) { - final Map> map = new HashMap<>(); + final Map> map = new EnumMap<>(IndexOptions.class); fields.stream().forEach(f -> { final IndexOptions indexOptions = getIndexOptions((String) Objects.requireNonNullElse(f.getConfig(LuceneFunctionNames.LUCENE_AUTO_COMPLETE_FIELD_INDEX_OPTIONS), LuceneFunctionNames.LuceneFieldIndexOptions.DOCS_AND_FREQS_AND_POSITIONS.name())); @@ -364,16 +306,6 @@ public CompletableFuture update(@Nullable FDBIndexable return AsyncUtil.DONE; } - /** - * Get the {@link AnalyzingInfixSuggester} for indexing or query, from the session of the context if there exists a corresponding one, or by creating a new one. - * @param indexOptions the {@link IndexOptions} for suggester's {@link FieldType}. This only matters for when the suggester is for indexing. - * The one for query can just use an arbitrary one, so just pass in a NULL when getting a suggester for query, so the existing one from session of context can be reused. - */ - private AnalyzingInfixSuggester getSuggester(@Nullable Tuple groupingKey, @Nonnull List texts, @Nullable IndexOptions indexOptions) throws IOException { - return directoryManager.getAutocompleteSuggester(groupingKey, autoCompleteIndexAnalyzerChooser.chooseAnalyzer(texts), - autoCompleteQueryAnalyzerChooser.chooseAnalyzer(texts), highlightForAutoCompleteIfEnabled, indexOptions); - } - private FieldType getTextFieldType(LuceneDocumentFromRecord.DocumentField field) { FieldType ft = new FieldType(); diff --git a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneIndexOptions.java b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneIndexOptions.java index 3f5402f6ac..0b9000d09b 100644 --- a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneIndexOptions.java +++ b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneIndexOptions.java @@ -33,11 +33,15 @@ public class LuceneIndexOptions { public static final String AUTO_COMPLETE_ENABLED = "autoCompleteEnabled"; /** * The type of auto complete blender to transform the weight after search to take into account the position of the searched term into the indexed text. + * @deprecated this option is ignored as the blender suggester is no longer used by auto-complete queries */ + @Deprecated public static final String AUTO_COMPLETE_BLENDER_TYPE = "autoCompleteBlenderType"; /** * The number factor to multiply the number of searched elements for auto complete blender. + * @deprecated this option is ignored as the blender suggester is no longer used by auto-complete queries */ + @Deprecated public static final String AUTO_COMPLETE_BLENDER_NUM_FACTOR = "autoCompleteBlenderNumFactor"; /** * The minimum number of leading characters before prefix query is used for auto complete. @@ -45,7 +49,9 @@ public class LuceneIndexOptions { public static final String AUTO_COMPLETE_MIN_PREFIX_SIZE = "autoCompleteMinPrefixSize"; /** * The exponent to use for auto complete when the blender type is POSITION_EXPONENTIAL_RECIPROCAL. + * @deprecated this option is ignored as the blender suggester is no longer used by auto-complete queries */ + @Deprecated public static final String AUTO_COMPLETE_BLENDER_EXPONENT = "autoCompleteBlenderExponent"; /** * Whether highlight suggest query in suggestions. diff --git a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneRecordContextProperties.java b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneRecordContextProperties.java index 62e6c2ca73..79ce43b97f 100644 --- a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneRecordContextProperties.java +++ b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/LuceneRecordContextProperties.java @@ -90,19 +90,23 @@ public final class LuceneRecordContextProperties { /** * Maximum segment size to produce during normal merging for auto-complete search with Lucene. + * @deprecated No longer in use as auto-complete no longer has its own directory */ + @Deprecated public static final RecordLayerPropertyKey LUCENE_AUTO_COMPLETE_MERGE_MAX_SIZE = RecordLayerPropertyKey.doublePropertyKey("com.apple.foundationdb.record.lucene.autoCompleteMergeMaxSize", 5.0); /** * Maximum number of segments to be merged at a time for auto-complete search with Lucene, during forceMerge for forceMergeDeletes. + * @deprecated No longer in use as auto-complete no longer has its own directory */ + @Deprecated public static final RecordLayerPropertyKey LUCENE_AUTO_COMPLETE_MERGE_MAX_NUMBER = RecordLayerPropertyKey.integerPropertyKey("com.apple.foundationdb.record.lucene.autoCompleteMergeMaxNum", 2); /** * This controls the suggester's base class to use for Lucene auto-complete search. - * True to use a {@link org.apache.lucene.search.suggest.analyzing.BlendedInfixSuggester}, that sorts the matches based on positions stored in term vectors. - * False to use a {@link com.apple.foundationdb.record.lucene.codec.LuceneOptimizedBlendedInfixSuggesterWithoutTermVectors} that does not store term vectors, and sort matches based on positions detection in memory. + * @deprecated No longer in use as auto-complete no longer has its own directory */ + @Deprecated public static final RecordLayerPropertyKey LUCENE_AUTO_COMPLETE_WITH_TERM_VECTORS = RecordLayerPropertyKey.booleanPropertyKey("com.apple.foundationdb.record.lucene.autoComplete.withTermVectors", true); /** 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 210e8bba0f..46c0123997 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 @@ -21,10 +21,10 @@ package com.apple.foundationdb.record.lucene; import com.apple.foundationdb.annotation.API; -import com.apple.foundationdb.record.ByteArrayContinuation; import com.apple.foundationdb.record.IndexEntry; import com.apple.foundationdb.record.RecordCoreException; import com.apple.foundationdb.record.RecordCursorContinuation; +import com.apple.foundationdb.record.RecordCursorEndContinuation; import com.apple.foundationdb.record.RecordCursorResult; import com.apple.foundationdb.record.RecordCursorVisitor; import com.apple.foundationdb.record.ScanProperties; @@ -258,14 +258,9 @@ public CompletableFuture> onNext() { @Nonnull private RecordCursorContinuation continuationHelper() { if (currentPosition >= topDocs.scoreDocs.length && limitRemaining > 0) { - return ByteArrayContinuation.fromNullable(null); + return RecordCursorEndContinuation.END; } else { - LuceneContinuationProto.LuceneIndexContinuation continuation = LuceneContinuationProto.LuceneIndexContinuation.newBuilder() - .setDoc(searchAfter.doc) - .setScore(searchAfter.score) - .setShard(searchAfter.shardIndex) - .build(); - return ByteArrayContinuation.fromNullable(continuation.toByteArray()); + return LuceneCursorContinuation.fromScoreDoc(searchAfter); } } diff --git a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/codec/LuceneOptimizedBlendedInfixSuggesterWithoutTermVectors.java b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/codec/LuceneOptimizedBlendedInfixSuggesterWithoutTermVectors.java deleted file mode 100644 index 33ace4e32f..0000000000 --- a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/codec/LuceneOptimizedBlendedInfixSuggesterWithoutTermVectors.java +++ /dev/null @@ -1,571 +0,0 @@ -/* - * LuceneOptimizedBlendedInfixSuggesterWithoutTermVectors.java - * - * This source file is part of the FoundationDB open source project - * - * Copyright 2015-2022 Apple Inc. and the FoundationDB project authors - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.apple.foundationdb.record.lucene.codec; - -import com.apple.foundationdb.record.RecordCoreArgumentException; -import com.apple.foundationdb.record.lucene.LuceneLoggerInfoStream; -import com.apple.foundationdb.record.lucene.LuceneRecordContextProperties; -import com.apple.foundationdb.record.lucene.directory.FDBDirectoryManager; -import com.apple.foundationdb.record.provider.foundationdb.IndexMaintainerState; -import com.apple.foundationdb.tuple.Tuple; -import org.apache.lucene.analysis.Analyzer; -import org.apache.lucene.analysis.TokenStream; -import org.apache.lucene.analysis.tokenattributes.CharTermAttribute; -import org.apache.lucene.analysis.tokenattributes.OffsetAttribute; -import org.apache.lucene.document.FieldType; -import org.apache.lucene.document.TextField; -import org.apache.lucene.index.BinaryDocValues; -import org.apache.lucene.index.ConcurrentMergeScheduler; -import org.apache.lucene.index.IndexOptions; -import org.apache.lucene.index.IndexWriterConfig; -import org.apache.lucene.index.MergePolicy; -import org.apache.lucene.index.MergeTrigger; -import org.apache.lucene.index.MultiDocValues; -import org.apache.lucene.index.Term; -import org.apache.lucene.index.TieredMergePolicy; -import org.apache.lucene.search.BooleanClause; -import org.apache.lucene.search.BooleanQuery; -import org.apache.lucene.search.FieldDoc; -import org.apache.lucene.search.IndexSearcher; -import org.apache.lucene.search.PhraseQuery; -import org.apache.lucene.search.PrefixQuery; -import org.apache.lucene.search.Query; -import org.apache.lucene.search.SearcherManager; -import org.apache.lucene.search.Sort; -import org.apache.lucene.search.SortField; -import org.apache.lucene.search.TermQuery; -import org.apache.lucene.search.TopFieldCollector; -import org.apache.lucene.search.TopFieldDocs; -import org.apache.lucene.search.spans.FieldMaskingSpanQuery; -import org.apache.lucene.search.spans.SpanMultiTermQueryWrapper; -import org.apache.lucene.search.spans.SpanNearQuery; -import org.apache.lucene.search.spans.SpanQuery; -import org.apache.lucene.search.spans.SpanTermQuery; -import org.apache.lucene.search.suggest.Lookup; -import org.apache.lucene.search.suggest.analyzing.AnalyzingInfixSuggester; -import org.apache.lucene.search.suggest.analyzing.BlendedInfixSuggester; -import org.apache.lucene.store.Directory; -import org.apache.lucene.util.BytesRef; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.annotation.Nonnull; -import javax.annotation.Nullable; -import java.io.IOException; -import java.io.Serializable; -import java.io.StringReader; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.Comparator; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.NavigableSet; -import java.util.Set; -import java.util.TreeSet; -import java.util.concurrent.ThreadLocalRandom; - -/** - * Optimized {@link BlendedInfixSuggester} that does not rely on term vectors persisted in DB. - * The implementation of methods {@link #getIndexWriterConfig(Analyzer, IndexWriterConfig.OpenMode)}, {@link #getTextFieldType()} and {@link #createCoefficient(Set, String, String, BytesRef)} - * are the main differences between this and {@link BlendedInfixSuggester}. - * This implementation also overrides the {@link AnalyzingInfixSuggester#lookup(CharSequence, BooleanQuery, int, boolean, boolean)} to also support phrase query. - */ -public class LuceneOptimizedBlendedInfixSuggesterWithoutTermVectors extends AnalyzingInfixSuggester { - private static final Logger LOGGER = LoggerFactory.getLogger(LuceneOptimizedBlendedInfixSuggesterWithoutTermVectors.class); - private static final Comparator LOOKUP_COMP = new LookUpComparator(); - /** - * Coefficient used for linear blending. - */ - private static final double LINEAR_COEF = 0.10; - - /** - * How we sort the postings and search results. - */ - private static final Sort SORT = new Sort(new SortField("weight", SortField.Type.LONG, true)); - - @Nonnull - private final IndexMaintainerState state; - @Nonnull - private final IndexOptions indexOptions; - - /** - * Type of blender used by the suggester. - */ - @Nonnull - private final BlendedInfixSuggester.BlenderType blenderType; - - /** - * Factor to multiply the number of searched elements. - */ - private final int numFactor; - - /** - * A copy of its superclass's minPrefixChars. - */ - private final int minPrefixCharsCopy; - - private final int mergeDirectoryCount; - - private Double exponent = 2.0; - - @SuppressWarnings("squid:S107") - LuceneOptimizedBlendedInfixSuggesterWithoutTermVectors(@Nonnull IndexMaintainerState state, @Nonnull Directory dir, @Nonnull Analyzer indexAnalyzer, - @Nonnull Analyzer queryAnalyzer, int minPrefixChars, BlendedInfixSuggester.BlenderType blenderType, int numFactor, - @Nullable Double exponent, boolean highlight, @Nonnull IndexOptions indexOptions, int mergeDirectoryCount) throws IOException { - super(dir, indexAnalyzer, queryAnalyzer, minPrefixChars, false, true, highlight); - this.state = state; - this.blenderType = blenderType; - this.indexOptions = indexOptions; - this.numFactor = numFactor; - this.minPrefixCharsCopy = minPrefixChars; - this.mergeDirectoryCount = mergeDirectoryCount; - if (exponent != null) { - this.exponent = exponent; - } - } - - @Override - public List lookup(CharSequence key, Set contexts, boolean onlyMorePopular, int num) throws IOException { - // Don't * numFactor here since we do it down below, once, in the call chain: - return super.lookup(key, contexts, onlyMorePopular, num); - } - - @Override - public List lookup(CharSequence key, Set contexts, int num, boolean allTermsRequired, boolean doHighlight) throws IOException { - // Don't * numFactor here since we do it down below, once, in the call chain: - return super.lookup(key, contexts, num, allTermsRequired, doHighlight); - } - - @Override - public List lookup(CharSequence key, Map contextInfo, int num, boolean allTermsRequired, boolean doHighlight) throws IOException { - // Don't * numFactor here since we do it down below, once, in the call chain: - return super.lookup(key, contextInfo, num, allTermsRequired, doHighlight); - } - - /** - * This one is different from the implementation by {@link BlendedInfixSuggester}. - * This method supports phrase query, whereas the {@link BlendedInfixSuggester} does not. - */ - @Override - public List lookup(CharSequence key, BooleanQuery contextQuery, int num, boolean allTermsRequired, boolean doHighlight) throws IOException { - if (searcherMgr == null) { - throw new IllegalStateException("suggester was not built"); - } - - final BooleanClause.Occur occur; - if (allTermsRequired) { - occur = BooleanClause.Occur.MUST; - } else { - occur = BooleanClause.Occur.SHOULD; - } - - BooleanQuery.Builder query = new BooleanQuery.Builder(); - Set matchedTokens = new HashSet<>(); - - final boolean phraseQueryNeeded = key.toString().startsWith("\"") && key.toString().endsWith("\""); - final String searchKey = phraseQueryNeeded ? key.toString().substring(1, key.toString().length() - 1) : key.toString(); - - String prefixToken = phraseQueryNeeded - ? buildQueryForPhraseMatching(query, matchedTokens, searchKey, occur) - : buildQueryForTermsMatching(query, matchedTokens, searchKey, occur); - - if (contextQuery != null) { - boolean allMustNot = true; - for (BooleanClause clause : contextQuery.clauses()) { - if (clause.getOccur() != BooleanClause.Occur.MUST_NOT) { - allMustNot = false; - break; - } - } - - if (allMustNot) { - // All are MUST_NOT: add the contextQuery to the main query instead (not as sub-query) - for (BooleanClause clause : contextQuery.clauses()) { - query.add(clause); - } - } else if (allTermsRequired == false) { - // We must carefully upgrade the query clauses to MUST: - BooleanQuery.Builder newQuery = new BooleanQuery.Builder(); - newQuery.add(query.build(), BooleanClause.Occur.MUST); - newQuery.add(contextQuery, BooleanClause.Occur.MUST); - query = newQuery; - } else { - // Add contextQuery as sub-query - query.add(contextQuery, BooleanClause.Occur.MUST); - } - } - - // TODO: we could allow blended sort here, combining - // weight w/ score. Now we ignore score and sort only - // by weight: - - Query finalQuery = finishQuery(query, allTermsRequired); - - //System.out.println("finalQuery=" + finalQuery); - - // Sort by weight, descending: - TopFieldCollector c = TopFieldCollector.create(SORT, num * numFactor, 1); - List results = null; - SearcherManager mgr; - IndexSearcher searcher; - synchronized (searcherMgrLock) { - mgr = searcherMgr; // acquire & release on same SearcherManager, via local reference - searcher = mgr.acquire(); - } - try { - //System.out.println("got searcher=" + searcher); - searcher.search(finalQuery, c); - - TopFieldDocs hits = c.topDocs(); - - // Slower way if postings are not pre-sorted by weight: - // hits = searcher.search(query, null, num, SORT); - results = createResults(searcher, hits, num * numFactor, key, doHighlight, matchedTokens, prefixToken); - } finally { - mgr.release(searcher); - } - - //System.out.println((System.currentTimeMillis() - t0) + " msec for infix suggest"); - //System.out.println(results); - - return results; - } - - /** - * This one is different from the implementation by {@link BlendedInfixSuggester}. - * This method overrides the {@link IndexWriterConfig}, whereas the {@link BlendedInfixSuggester} does not. - */ - @Override - protected IndexWriterConfig getIndexWriterConfig(Analyzer indexAnalyzer, IndexWriterConfig.OpenMode openMode) { - TieredMergePolicy tieredMergePolicy = new TieredMergePolicy(); - tieredMergePolicy.setMaxMergedSegmentMB(Math.max(0.0, state.context.getPropertyStorage().getPropertyValue(LuceneRecordContextProperties.LUCENE_AUTO_COMPLETE_MERGE_MAX_SIZE))); - tieredMergePolicy.setNoCFSRatio(1.00); - IndexWriterConfig iwc = super.getIndexWriterConfig(indexAnalyzer, openMode); - iwc.setUseCompoundFile(true); - iwc.setMergePolicy(tieredMergePolicy); - iwc.setMergeScheduler(new ConcurrentMergeScheduler() { - - @Override - public synchronized void merge(final MergeSource mergeSource, final MergeTrigger trigger) throws IOException { - if (state.context.getPropertyStorage().getPropertyValue(LuceneRecordContextProperties.LUCENE_MULTIPLE_MERGE_OPTIMIZATION_ENABLED) && trigger == MergeTrigger.FULL_FLUSH) { - if (ThreadLocalRandom.current().nextInt(mergeDirectoryCount) == 0) { - if (LOGGER.isTraceEnabled()) { - LOGGER.trace(FDBDirectoryManager.getMergeLogMessage(mergeSource, trigger, state, "Auto-complete index merge based on probability")); - } - super.merge(mergeSource, trigger); - } else { - if (LOGGER.isTraceEnabled()) { - LOGGER.trace(FDBDirectoryManager.getMergeLogMessage(mergeSource, trigger, state, "Auto-complete index merge aborted based on probability")); - } - synchronized (this) { - MergePolicy.OneMerge nextMerge = mergeSource.getNextMerge(); - while (nextMerge != null) { - nextMerge.setAborted(); - mergeSource.onMergeFinished(nextMerge); - nextMerge = mergeSource.getNextMerge(); - } - } - } - } else { - if (LOGGER.isTraceEnabled()) { - LOGGER.trace(FDBDirectoryManager.getMergeLogMessage(mergeSource, trigger, state, "Auto-complete index merge")); - } - super.merge(mergeSource, trigger); - } - } - }); - iwc.setCodec(new LuceneOptimizedCodec()); - iwc.setInfoStream(new LuceneLoggerInfoStream(LOGGER)); - return iwc; - } - - /** - * This one is different from the implementation by {@link BlendedInfixSuggester}. - * This does not enable term vectors, and has the indexOptions configurable. - */ - @Override - protected FieldType getTextFieldType() { - FieldType ft = new FieldType(TextField.TYPE_NOT_STORED); - ft.setIndexOptions(indexOptions); - ft.setOmitNorms(true); - - return ft; - } - - @Override - protected List createResults(IndexSearcher searcher, TopFieldDocs hits, int num, CharSequence key, - boolean doHighlight, Set matchedTokens, String prefixToken) - throws IOException { - - NavigableSet results = new TreeSet<>(LOOKUP_COMP); - - // we reduce the num to the one initially requested - int actualNum = num / numFactor; - - for (int i = 0; i < hits.scoreDocs.length; i++) { - FieldDoc fd = (FieldDoc) hits.scoreDocs[i]; - - BinaryDocValues textDV = MultiDocValues.getBinaryValues(searcher.getIndexReader(), TEXT_FIELD_NAME); - assert textDV != null; - - textDV.advance(fd.doc); - - final String text = textDV.binaryValue().utf8ToString(); - long weight = (Long) fd.fields[0]; - - // This will just be null if app didn't pass payloads to build(): - // TODO: maybe just stored fields? they compress... - BinaryDocValues payloadsDV = MultiDocValues.getBinaryValues(searcher.getIndexReader(), "payloads"); - - BytesRef payload; - if (payloadsDV != null) { - if (payloadsDV.advance(fd.doc) == fd.doc) { - payload = BytesRef.deepCopyOf(payloadsDV.binaryValue()); - } else { - payload = new BytesRef(BytesRef.EMPTY_BYTES); - } - } else { - payload = null; - } - - double coefficient; - if (text.startsWith(key.toString())) { - // if hit starts with the key, we don't change the score - coefficient = 1; - } else { - coefficient = createCoefficient(matchedTokens, prefixToken, text, payload); - } - if (weight == 0) { - weight = 1; - } - if (weight < 1 / LINEAR_COEF && weight > -1 / LINEAR_COEF) { - weight *= 1 / LINEAR_COEF; - } - long score = (long) (weight * coefficient); - - LookupResult result; - if (doHighlight) { - result = new LookupResult(text, highlight(text, matchedTokens, prefixToken), score, payload); - } else { - result = new LookupResult(text, score, payload); - } - - if (results.size() >= actualNum) { - if (results.first().value < result.value) { - results.pollFirst(); - } else { - continue; - } - } - results.add(result); - } - - return new ArrayList<>(results.descendingSet()); - } - - /** - * This one is different from the implementation by {@link BlendedInfixSuggester}. - * This one figures out the positions for matches by tokenizing the text and finding the matched tokens from it, - * instead of relying on the term vectors. - */ - private double createCoefficient(Set matchedTokens, String prefixToken, - String text, BytesRef payload) throws IOException { - final String fieldName = payload == null ? "" : (String) Tuple.fromBytes(payload.bytes).get(0); - Integer position = Integer.MAX_VALUE; - try (TokenStream tokenStream = indexAnalyzer.tokenStream(fieldName, text)) { - CharTermAttribute charTermAttribute = tokenStream.addAttribute(CharTermAttribute.class); - tokenStream.reset(); - - int p = 0; - while (tokenStream.incrementToken()) { - String term = charTermAttribute.toString(); - if (matchedTokens.contains(term) || (prefixToken != null && term.startsWith(prefixToken))) { - position = p; - break; - } - p++; - } - } - - // create corresponding coefficient based on position - return calculateCoefficient(position); - } - - private double calculateCoefficient(int position) { - double coefficient; - switch (blenderType) { - case POSITION_LINEAR: - coefficient = 1 - LINEAR_COEF * position; - break; - case POSITION_RECIPROCAL: - coefficient = 1. / (position + 1); - break; - case POSITION_EXPONENTIAL_RECIPROCAL: - coefficient = 1. / Math.pow((position + 1.0), exponent); - break; - default: - throw new RecordCoreArgumentException("Invalid blender type for Lucene auto-complete suggestion search: " + blenderType.name()); - } - - return coefficient; - } - - @Nullable - private String buildQueryForPhraseMatching(@Nonnull BooleanQuery.Builder query, @Nonnull Set matchedTokens, - @Nonnull String searchKey, @Nonnull BooleanClause.Occur occur) throws IOException { - String prefixToken = null; - try (TokenStream ts = queryAnalyzer.tokenStream("", new StringReader(searchKey))) { - //long t0 = System.currentTimeMillis(); - ts.reset(); - final CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class); - final OffsetAttribute offsetAtt = ts.addAttribute(OffsetAttribute.class); - String lastToken = null; - PhraseQuery.Builder phraseQueryBuilder = new PhraseQuery.Builder(); - int maxEndOffset = -1; - while (ts.incrementToken()) { - if (lastToken != null) { - matchedTokens.add(lastToken); - phraseQueryBuilder.add(new Term(TEXT_FIELD_NAME, lastToken)); - } - lastToken = termAtt.toString(); - if (lastToken != null) { - maxEndOffset = Math.max(maxEndOffset, offsetAtt.endOffset()); - } - } - ts.end(); - - if (lastToken != null) { - if (maxEndOffset == offsetAtt.endOffset()) { - // Use PrefixQuery (or the ngram equivalent) when - // there was no trailing discarded chars in the - // string (e.g. whitespace), so that if query does - // not end with a space we show prefix matches for - // that token: - query.add(getPhrasePrefixQuery(phraseQueryBuilder.build(), lastToken), occur); - prefixToken = lastToken; - } else { - // Use TermQuery for an exact match if there were - // trailing discarded chars (e.g. whitespace), so - // that if query ends with a space we only show - // exact matches for that term: - matchedTokens.add(lastToken); - phraseQueryBuilder.add(new Term(TEXT_FIELD_NAME, lastToken)); - query.add(phraseQueryBuilder.build(), occur); - } - } - } - return prefixToken; - } - - @Nullable - private String buildQueryForTermsMatching(@Nonnull BooleanQuery.Builder query, @Nonnull Set matchedTokens, - @Nonnull String searchKey, @Nonnull BooleanClause.Occur occur) throws IOException { - String prefixToken = null; - try (TokenStream ts = queryAnalyzer.tokenStream("", new StringReader(searchKey))) { - //long t0 = System.currentTimeMillis(); - ts.reset(); - final CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class); - final OffsetAttribute offsetAtt = ts.addAttribute(OffsetAttribute.class); - String lastToken = null; - int maxEndOffset = -1; - while (ts.incrementToken()) { - if (lastToken != null) { - matchedTokens.add(lastToken); - query.add(new TermQuery(new Term(TEXT_FIELD_NAME, lastToken)), occur); - } - lastToken = termAtt.toString(); - if (lastToken != null) { - maxEndOffset = Math.max(maxEndOffset, offsetAtt.endOffset()); - } - } - ts.end(); - - if (lastToken != null) { - Query lastQuery; - if (maxEndOffset == offsetAtt.endOffset()) { - // Use PrefixQuery (or the ngram equivalent) when - // there was no trailing discarded chars in the - // string (e.g. whitespace), so that if query does - // not end with a space we show prefix matches for - // that token: - lastQuery = getLastTokenQuery(lastToken); - prefixToken = lastToken; - } else { - // Use TermQuery for an exact match if there were - // trailing discarded chars (e.g. whitespace), so - // that if query ends with a space we only show - // exact matches for that term: - matchedTokens.add(lastToken); - lastQuery = new TermQuery(new Term(TEXT_FIELD_NAME, lastToken)); - } - - if (lastQuery != null) { - query.add(lastQuery, occur); - } - } - } - return prefixToken; - } - - private Query getPhrasePrefixQuery(@Nonnull PhraseQuery phraseQuery, @Nonnull String lastToken) { - Term[] terms = phraseQuery.getTerms(); - SpanNearQuery.Builder spanQuery = new SpanNearQuery.Builder(TEXT_FIELD_NAME, true); // field - for (Term term : terms) { - spanQuery.addClause(new SpanTermQuery(term)); - } - - SpanQuery lastTokenQuery = lastToken.length() < minPrefixCharsCopy - ? new SpanTermQuery(new Term(TEXTGRAMS_FIELD_NAME, new BytesRef(lastToken.getBytes(StandardCharsets.UTF_8)))) - : new SpanMultiTermQueryWrapper<>(new PrefixQuery(new Term(TEXT_FIELD_NAME, lastToken))); - FieldMaskingSpanQuery fieldMask = new FieldMaskingSpanQuery(lastTokenQuery, TEXT_FIELD_NAME); - spanQuery.addClause(fieldMask); - - return spanQuery.build(); - } - - @SuppressWarnings("serial") - private static class LookUpComparator implements Comparator, Serializable { - @Override - public int compare(Lookup.LookupResult o1, Lookup.LookupResult o2) { - // order on weight - if (o1.value > o2.value) { - return 1; - } else if (o1.value < o2.value) { - return -1; - } - - // otherwise on alphabetic order - int keyCompare = CHARSEQUENCE_COMPARATOR.compare(o1.key, o2.key); - - if (keyCompare != 0) { - return keyCompare; - } - - // if same weight and title, use the payload if there is one - if (o1.payload != null) { - return o1.payload.compareTo(o2.payload); - } - - return 0; - } - } -} diff --git a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/codec/LuceneOptimizedWrappedBlendedInfixSuggester.java b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/codec/LuceneOptimizedWrappedBlendedInfixSuggester.java deleted file mode 100644 index f80407a1e6..0000000000 --- a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/codec/LuceneOptimizedWrappedBlendedInfixSuggester.java +++ /dev/null @@ -1,142 +0,0 @@ -/* - * LuceneOptimizedWrappedBlendedInfixSuggester.java - * - * This source file is part of the FoundationDB open source project - * - * Copyright 2015-2022 Apple Inc. and the FoundationDB project authors - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.apple.foundationdb.record.lucene.codec; - -import com.apple.foundationdb.record.RecordCoreArgumentException; -import com.apple.foundationdb.record.logging.LogMessageKeys; -import com.apple.foundationdb.record.lucene.LuceneIndexOptions; -import com.apple.foundationdb.record.lucene.LuceneLoggerInfoStream; -import com.apple.foundationdb.record.lucene.LuceneRecordContextProperties; -import com.apple.foundationdb.record.lucene.directory.FDBDirectoryManager; -import com.apple.foundationdb.record.provider.foundationdb.IndexMaintainerState; -import org.apache.lucene.analysis.Analyzer; -import org.apache.lucene.index.ConcurrentMergeScheduler; -import org.apache.lucene.index.IndexOptions; -import org.apache.lucene.index.IndexWriterConfig; -import org.apache.lucene.index.MergePolicy; -import org.apache.lucene.index.MergeTrigger; -import org.apache.lucene.index.TieredMergePolicy; -import org.apache.lucene.search.suggest.analyzing.AnalyzingInfixSuggester; -import org.apache.lucene.search.suggest.analyzing.BlendedInfixSuggester; -import org.apache.lucene.store.Directory; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.annotation.Nonnull; -import javax.annotation.Nullable; -import java.io.IOException; -import java.util.concurrent.ThreadLocalRandom; - -/** - * Optimized suggester based on {@link BlendedInfixSuggester} to override the {@link IndexWriterConfig} for index writer. - */ -public class LuceneOptimizedWrappedBlendedInfixSuggester extends BlendedInfixSuggester { - private static final Logger LOGGER = LoggerFactory.getLogger(LuceneOptimizedWrappedBlendedInfixSuggester.class); - - private static final BlenderType DEFAULT_BLENDER_TYPE = BlenderType.POSITION_LINEAR; - - @Nonnull - private final IndexMaintainerState state; - - private final int mergeDirectoryCount; - - @SuppressWarnings("squid:S107") - private LuceneOptimizedWrappedBlendedInfixSuggester(@Nonnull IndexMaintainerState state, @Nonnull Directory dir, @Nonnull Analyzer indexAnalyzer, - @Nonnull Analyzer queryAnalyzer, int minPrefixChars, BlenderType blenderType, int numFactor, - @Nullable Double exponent, boolean highlight, int mergeDirectoryCount) throws IOException { - super(dir, indexAnalyzer, queryAnalyzer, minPrefixChars, blenderType, numFactor, exponent, false, true, highlight); - this.state = state; - this.mergeDirectoryCount = mergeDirectoryCount; - } - - @Override - protected IndexWriterConfig getIndexWriterConfig(Analyzer indexAnalyzer, IndexWriterConfig.OpenMode openMode) { - TieredMergePolicy tieredMergePolicy = new TieredMergePolicy(); - tieredMergePolicy.setMaxMergedSegmentMB(Math.max(0.0, state.context.getPropertyStorage().getPropertyValue(LuceneRecordContextProperties.LUCENE_AUTO_COMPLETE_MERGE_MAX_SIZE))); - tieredMergePolicy.setNoCFSRatio(1.00); - IndexWriterConfig iwc = super.getIndexWriterConfig(indexAnalyzer, openMode); - iwc.setUseCompoundFile(true); - iwc.setMergePolicy(tieredMergePolicy); - iwc.setMergeScheduler(new ConcurrentMergeScheduler() { - - @Override - public synchronized void merge(final MergeSource mergeSource, final MergeTrigger trigger) throws IOException { - if (state.context.getPropertyStorage().getPropertyValue(LuceneRecordContextProperties.LUCENE_MULTIPLE_MERGE_OPTIMIZATION_ENABLED) && trigger == MergeTrigger.FULL_FLUSH) { - if (ThreadLocalRandom.current().nextInt(mergeDirectoryCount) == 0) { - if (LOGGER.isTraceEnabled()) { - LOGGER.trace(FDBDirectoryManager.getMergeLogMessage(mergeSource, trigger, state, "Auto-complete index merge based on probability")); - } - super.merge(mergeSource, trigger); - } else { - if (LOGGER.isTraceEnabled()) { - LOGGER.trace(FDBDirectoryManager.getMergeLogMessage(mergeSource, trigger, state, "Auto-complete index merge aborted based on probability")); - } - synchronized (this) { - MergePolicy.OneMerge nextMerge = mergeSource.getNextMerge(); - while (nextMerge != null) { - nextMerge.setAborted(); - mergeSource.onMergeFinished(nextMerge); - nextMerge = mergeSource.getNextMerge(); - } - } - } - } else { - if (LOGGER.isTraceEnabled()) { - LOGGER.trace(FDBDirectoryManager.getMergeLogMessage(mergeSource, trigger, state, "Auto-complete index merge")); - } - super.merge(mergeSource, trigger); - } - } - }); - iwc.setCodec(new LuceneOptimizedCodec()); - iwc.setInfoStream(new LuceneLoggerInfoStream(LOGGER)); - return iwc; - } - - @Nonnull - public static AnalyzingInfixSuggester getSuggester(@Nonnull IndexMaintainerState state, @Nonnull Directory dir, - @Nonnull Analyzer indexAnalyzer, @Nonnull Analyzer queryAnalyzer, - boolean highlight, @Nonnull IndexOptions indexOptions, int mergeDirectoryCount) { - final String autoCompleteBlenderType = state.index.getOption(LuceneIndexOptions.AUTO_COMPLETE_BLENDER_TYPE); - final String autoCompleteBlenderNumFactor = state.index.getOption(LuceneIndexOptions.AUTO_COMPLETE_BLENDER_NUM_FACTOR); - final String autoCompleteMinPrefixSize = state.index.getOption(LuceneIndexOptions.AUTO_COMPLETE_MIN_PREFIX_SIZE); - final String autoCompleteBlenderExponent = state.index.getOption(LuceneIndexOptions.AUTO_COMPLETE_BLENDER_EXPONENT); - - final boolean useTermVectors = state.context.getPropertyStorage().getPropertyValue(LuceneRecordContextProperties.LUCENE_AUTO_COMPLETE_WITH_TERM_VECTORS); - final int minPrefixChars = autoCompleteMinPrefixSize == null ? DEFAULT_MIN_PREFIX_CHARS : Integer.parseInt(autoCompleteMinPrefixSize); - final BlenderType blenderType = autoCompleteBlenderType == null ? DEFAULT_BLENDER_TYPE : BlenderType.valueOf(autoCompleteBlenderType); - final int numFactor = autoCompleteBlenderNumFactor == null ? DEFAULT_NUM_FACTOR : Integer.parseInt(autoCompleteBlenderNumFactor); - final Double exponent = autoCompleteBlenderExponent == null ? null : Double.valueOf(autoCompleteBlenderExponent); - - try { - return useTermVectors - ? new LuceneOptimizedWrappedBlendedInfixSuggester(state, dir, indexAnalyzer, queryAnalyzer, minPrefixChars, blenderType, numFactor, exponent, highlight, mergeDirectoryCount) - : new LuceneOptimizedBlendedInfixSuggesterWithoutTermVectors(state, dir, indexAnalyzer, queryAnalyzer, minPrefixChars, blenderType, - numFactor, exponent, highlight, indexOptions, mergeDirectoryCount); - } catch (IllegalArgumentException iae) { - throw new RecordCoreArgumentException("Invalid parameter for auto complete suggester", iae) - .addLogInfo(LogMessageKeys.INDEX_NAME, state.index.getName()); - } catch (IOException ioe) { - throw new RecordCoreArgumentException("Failure with underlying lucene index opening for auto complete suggester", ioe) - .addLogInfo(LogMessageKeys.INDEX_NAME, state.index.getName()); - } - } -} diff --git a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/FDBDirectoryManager.java b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/FDBDirectoryManager.java index a05cebc31e..7946273cfd 100644 --- a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/FDBDirectoryManager.java +++ b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/FDBDirectoryManager.java @@ -35,12 +35,10 @@ import com.apple.foundationdb.tuple.Tuple; import com.apple.foundationdb.tuple.TupleHelpers; import com.google.common.annotations.VisibleForTesting; -import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.MergeScheduler; import org.apache.lucene.index.MergeTrigger; -import org.apache.lucene.search.suggest.analyzing.AnalyzingInfixSuggester; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -124,20 +122,6 @@ public IndexWriter getIndexWriter(@Nullable Tuple groupingKey, @Nonnull LuceneAn return getDirectoryWrapper(groupingKey).getWriter(analyzerWrapper); } - @Nonnull - public AnalyzingInfixSuggester getAutocompleteSuggester(@Nullable Tuple groupingKey, - @Nonnull LuceneAnalyzerWrapper indexAnalyzerWrapper, - @Nonnull LuceneAnalyzerWrapper queryAnalyzerWrapper, - boolean highlight, @Nullable IndexOptions indexOptions) throws IOException { - // The auto complete suggester reads and writes from a separate directory from the main - // directory used for Lucene indexes, so add a suffix to the groupingKey to separate it - // from the other Lucene index data but so that the data are still prefixed by the grouping key, - // which is necessary for range deletes - Tuple autoCompleteKey = groupingKey == null ? AUTO_COMPLETE_SUFFIX : groupingKey.addAll(AUTO_COMPLETE_SUFFIX); - return getDirectoryWrapper(autoCompleteKey) - .getAutocompleteSuggester(indexAnalyzerWrapper, queryAnalyzerWrapper, highlight, indexOptions); - } - @Nonnull public static FDBDirectoryManager getManager(@Nonnull IndexMaintainerState state) { synchronized (state.context) { diff --git a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/FDBDirectoryWrapper.java b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/FDBDirectoryWrapper.java index 3c756a0587..477fe74b6f 100644 --- a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/FDBDirectoryWrapper.java +++ b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/FDBDirectoryWrapper.java @@ -24,11 +24,9 @@ import com.apple.foundationdb.record.lucene.LuceneLoggerInfoStream; import com.apple.foundationdb.record.lucene.LuceneRecordContextProperties; import com.apple.foundationdb.record.lucene.codec.LuceneOptimizedCodec; -import com.apple.foundationdb.record.lucene.codec.LuceneOptimizedWrappedBlendedInfixSuggester; import com.apple.foundationdb.record.provider.foundationdb.IndexMaintainerState; import org.apache.lucene.index.ConcurrentMergeScheduler; import org.apache.lucene.index.DirectoryReader; -import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; @@ -36,12 +34,10 @@ import org.apache.lucene.index.MergeTrigger; import org.apache.lucene.index.StandardDirectoryReaderOptimization; import org.apache.lucene.index.TieredMergePolicy; -import org.apache.lucene.search.suggest.analyzing.AnalyzingInfixSuggester; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import javax.annotation.Nonnull; -import javax.annotation.Nullable; import java.io.IOException; import java.util.concurrent.ThreadLocalRandom; @@ -60,15 +56,7 @@ class FDBDirectoryWrapper implements AutoCloseable { @SuppressWarnings({"squid:S3077"}) // object is thread safe, so use of volatile to control instance creation is correct private volatile IndexWriter writer; @SuppressWarnings({"squid:S3077"}) // object is thread safe, so use of volatile to control instance creation is correct - private volatile AnalyzingInfixSuggester suggester; - @SuppressWarnings({"squid:S3077"}) // object is thread safe, so use of volatile to control instance creation is correct private volatile String writerAnalyzerId; - @SuppressWarnings({"squid:S3077"}) // object is thread safe, so use of volatile to control instance creation is correct - private volatile String suggesterIndexAnalyzerId; - @SuppressWarnings({"squid:S3077"}) // object is thread safe, so use of volatile to control instance creation is correct - private volatile String suggesterQueryAnalyzerId; - @SuppressWarnings({"squid:S3077"}) // object is thread safe, so use of volatile to control instance creation is correct - private volatile IndexOptions suggesterFieldIndexOptions; FDBDirectoryWrapper(IndexMaintainerState state, FDBDirectory directory, int mergeDirectoryCount) { this.state = state; @@ -90,6 +78,50 @@ public IndexReader getReader() throws IOException { } } + private static class FDBDirectoryMergeScheduler extends ConcurrentMergeScheduler { + @Nonnull + private final IndexMaintainerState state; + private final int mergeDirectoryCount; + + private FDBDirectoryMergeScheduler(@Nonnull IndexMaintainerState state, int mergeDirectoryCount) { + this.state = state; + this.mergeDirectoryCount = mergeDirectoryCount; + } + + @SuppressWarnings({ + "squid:S2245", // ThreadLocalRandom not used in case where cryptographic security is needed + "squid:S3776", // Cognitive complexity is too high. Candidate for later refactoring + }) + @Override + public synchronized void merge(final MergeSource mergeSource, final MergeTrigger trigger) throws IOException { + if (state.context.getPropertyStorage().getPropertyValue(LuceneRecordContextProperties.LUCENE_MULTIPLE_MERGE_OPTIMIZATION_ENABLED) && trigger == MergeTrigger.FULL_FLUSH) { + if (ThreadLocalRandom.current().nextInt(mergeDirectoryCount) == 0) { + if (LOGGER.isTraceEnabled()) { + LOGGER.trace(FDBDirectoryManager.getMergeLogMessage(mergeSource, trigger, state, "Basic Lucene index merge based on probability")); + } + super.merge(mergeSource, trigger); + } else { + if (LOGGER.isTraceEnabled()) { + LOGGER.trace(FDBDirectoryManager.getMergeLogMessage(mergeSource, trigger, state, "Basic Lucene index merge aborted based on probability")); + } + synchronized (this) { + MergePolicy.OneMerge nextMerge = mergeSource.getNextMerge(); + while (nextMerge != null) { + nextMerge.setAborted(); + mergeSource.onMergeFinished(nextMerge); + nextMerge = mergeSource.getNextMerge(); + } + } + } + } else { + if (LOGGER.isTraceEnabled()) { + LOGGER.trace(FDBDirectoryManager.getMergeLogMessage(mergeSource, trigger, state, "Basic Lucene index merge")); + } + super.merge(mergeSource, trigger); + } + } + } + @Nonnull public IndexWriter getWriter(LuceneAnalyzerWrapper analyzerWrapper) throws IOException { if (writer == null || !writerAnalyzerId.equals(analyzerWrapper.getUniqueIdentifier())) { @@ -101,36 +133,7 @@ public IndexWriter getWriter(LuceneAnalyzerWrapper analyzerWrapper) throws IOExc IndexWriterConfig indexWriterConfig = new IndexWriterConfig(analyzerWrapper.getAnalyzer()) .setUseCompoundFile(true) .setMergePolicy(tieredMergePolicy) - .setMergeScheduler(new ConcurrentMergeScheduler() { - @Override - public synchronized void merge(final MergeSource mergeSource, final MergeTrigger trigger) throws IOException { - if (state.context.getPropertyStorage().getPropertyValue(LuceneRecordContextProperties.LUCENE_MULTIPLE_MERGE_OPTIMIZATION_ENABLED) && trigger == MergeTrigger.FULL_FLUSH) { - if (ThreadLocalRandom.current().nextInt(mergeDirectoryCount) == 0) { - if (LOGGER.isTraceEnabled()) { - LOGGER.trace(FDBDirectoryManager.getMergeLogMessage(mergeSource, trigger, state, "Basic Lucene index merge based on probability")); - } - super.merge(mergeSource, trigger); - } else { - if (LOGGER.isTraceEnabled()) { - LOGGER.trace(FDBDirectoryManager.getMergeLogMessage(mergeSource, trigger, state, "Basic Lucene index merge aborted based on probability")); - } - synchronized (this) { - MergePolicy.OneMerge nextMerge = mergeSource.getNextMerge(); - while (nextMerge != null) { - nextMerge.setAborted(); - mergeSource.onMergeFinished(nextMerge); - nextMerge = mergeSource.getNextMerge(); - } - } - } - } else { - if (LOGGER.isTraceEnabled()) { - LOGGER.trace(FDBDirectoryManager.getMergeLogMessage(mergeSource, trigger, state, "Basic Lucene index merge")); - } - super.merge(mergeSource, trigger); - } - } - }) + .setMergeScheduler(new FDBDirectoryMergeScheduler(state, mergeDirectoryCount)) .setCodec(new LuceneOptimizedCodec()) .setInfoStream(new LuceneLoggerInfoStream(LOGGER)); @@ -146,42 +149,6 @@ public synchronized void merge(final MergeSource mergeSource, final MergeTrigger return writer; } - @Nonnull - public AnalyzingInfixSuggester getAutocompleteSuggester(@Nonnull LuceneAnalyzerWrapper indexAnalyzerWrapper, - @Nonnull LuceneAnalyzerWrapper queryAnalyzerWrapper, - boolean highlight, @Nullable IndexOptions indexOptions) throws IOException { - if (suggesterRenewNeeded(indexAnalyzerWrapper, queryAnalyzerWrapper, indexOptions)) { - synchronized (this) { - if (suggesterRenewNeeded(indexAnalyzerWrapper, queryAnalyzerWrapper, indexOptions)) { - AnalyzingInfixSuggester oldSuggester = suggester; - if (oldSuggester != null) { - oldSuggester.close(); - } - - suggester = LuceneOptimizedWrappedBlendedInfixSuggester.getSuggester(state, directory, - indexAnalyzerWrapper.getAnalyzer(), queryAnalyzerWrapper.getAnalyzer(), - highlight, indexOptions == null ? suggesterFieldIndexOptions : indexOptions, - mergeDirectoryCount); - suggesterIndexAnalyzerId = indexAnalyzerWrapper.getUniqueIdentifier(); - suggesterQueryAnalyzerId = queryAnalyzerWrapper.getUniqueIdentifier(); - if (indexOptions != null) { - suggesterFieldIndexOptions = indexOptions; - } - } - } - } - return suggester; - } - - private boolean suggesterRenewNeeded(@Nonnull LuceneAnalyzerWrapper indexAnalyzerWrapper, - @Nonnull LuceneAnalyzerWrapper queryAnalyzerWrapper, - @Nullable IndexOptions indexOptions) { - return suggester == null - || !suggesterIndexAnalyzerId.equals(indexAnalyzerWrapper.getUniqueIdentifier()) - || !suggesterQueryAnalyzerId.equals(queryAnalyzerWrapper.getUniqueIdentifier()) - || (indexOptions != null && !indexOptions.equals(suggesterFieldIndexOptions)); - } - @Override public synchronized void close() throws IOException { IndexWriter indexWriter = writer; @@ -190,13 +157,6 @@ public synchronized void close() throws IOException { writer = null; writerAnalyzerId = null; } - if (suggester != null) { - suggester.close(); - suggester = null; - suggesterIndexAnalyzerId = null; - suggesterQueryAnalyzerId = null; - suggesterFieldIndexOptions = null; - } directory.close(); } } 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 new file mode 100644 index 0000000000..41bd16331d --- /dev/null +++ b/fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/LuceneAutoCompleteResultCursorTest.java @@ -0,0 +1,132 @@ +/* + * LuceneAutoCompleteResultCursorTest.java + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2015-2022 Apple Inc. and the FoundationDB project authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.apple.foundationdb.record.lucene; + +import org.apache.lucene.analysis.Analyzer; +import org.apache.lucene.analysis.en.EnglishAnalyzer; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Stream; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +class LuceneAutoCompleteResultCursorTest { + + private static Analyzer getTestAnalyzer() { + return new EnglishAnalyzer(); + } + + private static Stream searchArgs(String[][] matches) { + // Each match is a 3-tuple of: + // 0. The text to search through + // 1. The match results without highlighting + // 2. The match results with highlighting + // A value of "null" indicates that the search text was not found + return Arrays.stream(matches).flatMap(match -> Stream.of( + Arguments.of(false, match[0], match[1]), + Arguments.of(true, match[0], match[2]) + )); + } + + // Auto-complete searches for the prefix "qua" + private static final String[][] QUA_MATCHES = { + {"quality", "quality", "quality"}, + {"The basic qualia of objects", "The basic qualia of objects", "The basic qualia of objects"}, + {"Quality over quantity!", "Quality over quantity!", "Quality over quantity!"}, + {"quorum logic", null, null}, + {"square", null, null}, + {"example qua example", "example qua example", "example qua example"}, + }; + + @SuppressWarnings("unused") // used as argument source for parameterized test + static Stream searchForQua() { + return searchArgs(QUA_MATCHES); + } + + @ParameterizedTest(name = "searchForQua[highlight={0},text={1}]") + @MethodSource + void searchForQua(boolean highlight, String text, String expected) throws IOException { + assertSearchMatches("qua", Collections.emptyList(), "qua", highlight, text, expected); + } + + // Auto-complete searches for the phrase "good mor" (with a prefix search on "mor") + private static final String[][] GOOD_MOR_MATCHES = { + {"Good morning!", "Good morning!", "Good morning!"}, + {"It is all for the good, and I'll see you on the morrow", "It is all for the good, and I'll see you on the morrow", "It is all for the good, and I'll see you on the morrow"}, + {"The more good we do, the more good we see", "The more good we do, the more good we see", "The more good we do, the more good we see"}, + {"Good day!", null, null}, + {"Morning!", null, null}, + }; + + @SuppressWarnings("unused") // used as argument source for parameterized test + static Stream searchForGoodMor() { + return searchArgs(GOOD_MOR_MATCHES); + } + + @ParameterizedTest(name = "searchForGoodMor[highlight={0},text={1}]") + @MethodSource + void searchForGoodMor(boolean highlight, String text, String expected) throws IOException { + assertSearchMatches("Good mor", List.of("good"), "mor", highlight, text, expected); + } + + // Auto-complete searches for the phrase "hello world " (ending space intentional--indicates "world" is not a prefix search) + private static final String[][] HELLO_WORLD_MATCHES = { + {"Hello, world!", "Hello, world!", "Hello, world!"}, + {"Hello, worldlings!", null, null}, + {"World--hello!", "World--hello!", "World--hello!"}, + {"Worldly--hello!", null, null}, + }; + + @SuppressWarnings("unused") // used as argument source for parameterized test + static Stream searchForHelloWorld() { + return searchArgs(HELLO_WORLD_MATCHES); + } + + @ParameterizedTest(name = "searchForHelloWorld[highlight={0},text={1}]") + @MethodSource + void searchForHelloWorld(boolean highlight, String text, String expected) throws IOException { + assertSearchMatches("Hello World ", List.of("hello", "world"), null, highlight, text, expected); + } + + private static void assertSearchMatches(String queryString, List expectedTokens, @Nullable String expectedPrefixToken, + boolean highlight, String text, @Nullable String expectedMatch) throws IOException { + final Analyzer analyzer = getTestAnalyzer(); + + List tokens = new ArrayList<>(); + String prefixToken = LuceneAutoCompleteResultCursor.getQueryTokens(analyzer, queryString, tokens); + assertEquals(expectedTokens, tokens); + assertEquals(expectedPrefixToken, prefixToken); + + Set queryTokenSet = new HashSet<>(tokens); + @Nullable String match = LuceneAutoCompleteResultCursor.searchAllMaybeHighlight(analyzer, text, queryTokenSet, prefixToken, highlight); + assertEquals(expectedMatch, match); + } +} 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 f6a8ceb401..c74caf6236 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 @@ -33,7 +33,6 @@ import com.apple.foundationdb.record.ScanProperties; import com.apple.foundationdb.record.TestRecordsTextProto; import com.apple.foundationdb.record.lucene.directory.FDBDirectory; -import com.apple.foundationdb.record.lucene.directory.FDBDirectoryManager; import com.apple.foundationdb.record.lucene.directory.FDBLuceneFileReference; import com.apple.foundationdb.record.lucene.ngram.NgramAnalyzer; import com.apple.foundationdb.record.lucene.synonym.EnglishSynonymMapConfig; @@ -54,8 +53,6 @@ import com.apple.foundationdb.record.provider.foundationdb.FDBRecordStoreTestBase; 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.record.provider.foundationdb.IndexMaintenanceFilter; import com.apple.foundationdb.record.provider.foundationdb.indexes.TextIndexTestUtils; import com.apple.foundationdb.record.provider.foundationdb.properties.RecordLayerPropertyStorage; import com.apple.foundationdb.record.query.RecordQuery; @@ -68,7 +65,6 @@ import com.apple.foundationdb.subspace.Subspace; import com.apple.foundationdb.tuple.Tuple; import com.apple.foundationdb.tuple.TupleHelpers; -import com.apple.test.BooleanSource; import com.apple.test.Tags; import com.google.auto.service.AutoService; import com.google.common.collect.ImmutableList; @@ -78,19 +74,16 @@ import org.apache.commons.lang3.tuple.Pair; import org.apache.lucene.analysis.en.EnglishAnalyzer; import org.apache.lucene.index.IndexFileNames; -import org.apache.lucene.search.suggest.analyzing.AnalyzingInfixSuggester; -import org.apache.lucene.util.BytesRef; +import org.hamcrest.Matchers; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; -import org.junit.jupiter.params.ParameterizedTest; import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.io.IOException; import java.io.InputStream; import java.io.SequenceInputStream; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -117,6 +110,7 @@ import static com.apple.foundationdb.record.query.plan.match.PlanMatchers.indexScan; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.allOf; +import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; @@ -145,7 +139,8 @@ public class LuceneIndexTest extends FDBRecordStoreTestBase { private static final Index SIMPLE_TEXT_WITH_AUTO_COMPLETE_NO_FREQS_POSITIONS = new Index("Simple_with_auto_complete", function(LuceneFunctionNames.LUCENE_TEXT, concat(field("text"), - function(LuceneFunctionNames.LUCENE_AUTO_COMPLETE_FIELD_INDEX_OPTIONS, value(LuceneFunctionNames.LuceneFieldIndexOptions.DOCS.name())))), + function(LuceneFunctionNames.LUCENE_AUTO_COMPLETE_FIELD_INDEX_OPTIONS, value(LuceneFunctionNames.LuceneFieldIndexOptions.DOCS.name())), + function(LuceneFunctionNames.LUCENE_FULL_TEXT_FIELD_INDEX_OPTIONS, value(LuceneFunctionNames.LuceneFieldIndexOptions.DOCS.name())))), LuceneIndexTypes.LUCENE, ImmutableMap.of(LuceneIndexOptions.AUTO_COMPLETE_ENABLED, "true", LuceneIndexOptions.AUTO_COMPLETE_MIN_PREFIX_SIZE, "3")); @@ -399,12 +394,9 @@ void simpleEmptyIndex() { } } - @ParameterizedTest(name = "simpleEmptyAutoComplete[withTermVectors={0}]") - @BooleanSource - void simpleEmptyAutoComplete(boolean withTermVectors) { - final RecordLayerPropertyStorage.Builder storageBuilder = RecordLayerPropertyStorage.newBuilder() - .addProp(LuceneRecordContextProperties.LUCENE_AUTO_COMPLETE_WITH_TERM_VECTORS, withTermVectors); - try (FDBRecordContext context = openContext(storageBuilder)) { + @Test + void simpleEmptyAutoComplete() { + try (FDBRecordContext context = openContext()) { rebuildIndexMetaData(context, SIMPLE_DOC, SIMPLE_TEXT_WITH_AUTO_COMPLETE); try (RecordCursor cursor = recordStore.scanIndex(SIMPLE_TEXT_WITH_AUTO_COMPLETE, autoComplete(SIMPLE_TEXT_WITH_AUTO_COMPLETE, "something"), null, ScanProperties.FORWARD_SCAN)) { assertEquals(RecordCursorResult.exhausted(), cursor.getNext()); @@ -951,53 +943,45 @@ void scanWithNgramIndex() { } } - @ParameterizedTest(name = "searchForAutoComplete[withTermVectors={0}]") - @BooleanSource - void searchForAutoComplete(boolean withTermVectors) throws Exception { - searchForAutoCompleteAndAssert("good", true, false, DEFAULT_AUTO_COMPLETE_TEXT_SIZE_LIMIT, withTermVectors); + @Test + void searchForAutoComplete() throws Exception { + searchForAutoCompleteAndAssert("good", true, false, DEFAULT_AUTO_COMPLETE_TEXT_SIZE_LIMIT); } - @ParameterizedTest(name = "searchForAutoCompleteWithoutFieldWithoutTerm[withTermVectors={0}]") - @BooleanSource - void searchForAutoCompleteWithoutFieldWithoutTerm(boolean withTermVectors) throws Exception { + @Test + void searchForAutoCompleteWithoutFieldWithoutTerm() throws Exception { assertThrows(RecordCoreArgumentException.class, - () -> searchForAutoCompleteAndAssert("", true, false, DEFAULT_AUTO_COMPLETE_TEXT_SIZE_LIMIT, withTermVectors), + () -> searchForAutoCompleteAndAssert("", true, false, DEFAULT_AUTO_COMPLETE_TEXT_SIZE_LIMIT), "Invalid query for auto-complete search"); } - @ParameterizedTest(name = "searchForAutoCompleteWithPrefix[withTermVectors={0}]") - @BooleanSource - void searchForAutoCompleteWithPrefix(boolean withTermVectors) throws Exception { - searchForAutoCompleteAndAssert("goo", true, false, DEFAULT_AUTO_COMPLETE_TEXT_SIZE_LIMIT, withTermVectors); + @Test + void searchForAutoCompleteWithPrefix() throws Exception { + searchForAutoCompleteAndAssert("goo", true, false, DEFAULT_AUTO_COMPLETE_TEXT_SIZE_LIMIT); } - @ParameterizedTest(name = "searchForAutoCompleteWithHighlight[withTermVectors={0}]") - @BooleanSource - void searchForAutoCompleteWithHighlight(boolean withTermVectors) throws Exception { - searchForAutoCompleteAndAssert("good", true, true, DEFAULT_AUTO_COMPLETE_TEXT_SIZE_LIMIT, withTermVectors); + @Test + void searchForAutoCompleteWithHighlight() throws Exception { + searchForAutoCompleteAndAssert("good", true, true, DEFAULT_AUTO_COMPLETE_TEXT_SIZE_LIMIT); } - @ParameterizedTest(name = "searchForAutoCompleteWithoutHittingSizeLimitation[withTermVectors={0}]") - @BooleanSource - void searchForAutoCompleteWithoutHittingSizeLimitation(boolean withTermVectors) throws Exception { - searchForAutoCompleteWithTextSizeLimit(DEFAULT_AUTO_COMPLETE_TEXT_SIZE_LIMIT, true, withTermVectors); + @Test + void searchForAutoCompleteWithoutHittingSizeLimitation() throws Exception { + searchForAutoCompleteWithTextSizeLimit(DEFAULT_AUTO_COMPLETE_TEXT_SIZE_LIMIT, true); } - @ParameterizedTest(name = "searchForAutoCompleteWithHittingSizeLimitation[withTermVectors={0}]") - @BooleanSource - void searchForAutoCompleteWithHittingSizeLimitation(boolean withTermVectors) throws Exception { - searchForAutoCompleteWithTextSizeLimit(10, false, withTermVectors); + @Test + void searchForAutoCompleteWithHittingSizeLimitation() throws Exception { + searchForAutoCompleteWithTextSizeLimit(10, false); } /** * To verify the suggestion lookup can work correctly if the suggester is never built and no entries exist in the directory. */ - @ParameterizedTest(name = "searchForAutoCompleteWithLoadingNoRecords[withTermVectors={0}]") - @BooleanSource - void searchForAutoCompleteWithLoadingNoRecords(boolean withTermVectors) throws Exception { + @Test + void searchForAutoCompleteWithLoadingNoRecords() throws Exception { final RecordLayerPropertyStorage.Builder storageBuilder = RecordLayerPropertyStorage.newBuilder() - .addProp(LuceneRecordContextProperties.LUCENE_AUTO_COMPLETE_TEXT_SIZE_UPPER_LIMIT, DEFAULT_AUTO_COMPLETE_TEXT_SIZE_LIMIT) - .addProp(LuceneRecordContextProperties.LUCENE_AUTO_COMPLETE_WITH_TERM_VECTORS, withTermVectors); + .addProp(LuceneRecordContextProperties.LUCENE_AUTO_COMPLETE_TEXT_SIZE_UPPER_LIMIT, DEFAULT_AUTO_COMPLETE_TEXT_SIZE_LIMIT); try (FDBRecordContext context = openContext(storageBuilder)) { openRecordStore(context, metaDataBuilder -> { metaDataBuilder.removeIndex(TextIndexTestUtils.SIMPLE_DEFAULT_NAME); @@ -1009,16 +993,13 @@ void searchForAutoCompleteWithLoadingNoRecords(boolean withTermVectors) throws E null, ScanProperties.FORWARD_SCAN).asList().get(); assertTrue(results.isEmpty()); - assertEquals(0, context.getTimer().getCounter(LuceneEvents.Counts.LUCENE_SCAN_MATCHED_AUTO_COMPLETE_SUGGESTIONS).getCount()); + assertEquals(0, context.getTimer().getCount(LuceneEvents.Counts.LUCENE_SCAN_MATCHED_AUTO_COMPLETE_SUGGESTIONS)); } } - @ParameterizedTest(name = "searchForAutoCompleteCrossingMultipleFields[withTermVectors={0}]") - @BooleanSource - void searchForAutoCompleteCrossingMultipleFields(boolean withTermVectors) throws Exception { - final RecordLayerPropertyStorage.Builder storageBuilder = RecordLayerPropertyStorage.newBuilder() - .addProp(LuceneRecordContextProperties.LUCENE_AUTO_COMPLETE_WITH_TERM_VECTORS, withTermVectors); - try (FDBRecordContext context = openContext(storageBuilder)) { + @Test + void searchForAutoCompleteCrossingMultipleFields() throws Exception { + try (FDBRecordContext context = openContext()) { openRecordStore(context, metaDataBuilder -> { metaDataBuilder.removeIndex(TextIndexTestUtils.SIMPLE_DEFAULT_NAME); metaDataBuilder.addIndex(COMPLEX_DOC, COMPLEX_MULTIPLE_TEXT_INDEXES_WITH_AUTO_COMPLETE); @@ -1052,31 +1033,19 @@ void searchForAutoCompleteCrossingMultipleFields(boolean withTermVectors) throws // Assert the suggestions' keys List suggestions = results.stream().map(i -> (String) i.getKey().get(i.getKeySize() - 1)).collect(Collectors.toList()); - assertEquals(ImmutableList.of("good evening", "Good night", "Good morning", "Good afternoon", "I'm good", "That's really good!"), suggestions); + assertThat(suggestions, containsInAnyOrder("good evening", "Good night", "Good morning", "Good afternoon", "I'm good", "That's really good!")); // Assert the corresponding field for the suggestions List fields = results.stream().map(i -> (String) i.getKey().get(i.getKeySize() - 2)).collect(Collectors.toList()); assertEquals(ImmutableList.of("text", "text", "text", "text", "text2", "text2"), fields); - // Assert the suggestions are sorted according to their values, which are determined by the position of the term into the indexed text - List values = results.stream().map(i -> (Long) i.getValue().get(0)).collect(Collectors.toList()); - List valuesSorted = new ArrayList<>(values); - Collections.sort(valuesSorted, Collections.reverseOrder()); - assertEquals(valuesSorted, values); - assertEquals(values.get(0), values.get(1)); - assertEquals(values.get(1), values.get(2)); - assertEquals(values.get(2), values.get(3)); - assertTrue(values.get(3) > values.get(4)); - assertTrue(values.get(4) > values.get(5)); + commit(context); } } - @ParameterizedTest(name = "searchForAutoCompleteWithContinueTyping[withTermVectors={0}]") - @BooleanSource - void searchForAutoCompleteWithContinueTyping(boolean withTermVectors) throws Exception { - final RecordLayerPropertyStorage.Builder storageBuilder = RecordLayerPropertyStorage.newBuilder() - .addProp(LuceneRecordContextProperties.LUCENE_AUTO_COMPLETE_WITH_TERM_VECTORS, withTermVectors); - try (FDBRecordContext context = openContext(storageBuilder)) { + @Test + void searchForAutoCompleteWithContinueTyping() throws Exception { + try (FDBRecordContext context = openContext()) { addIndexAndSaveRecordForAutoComplete(context, false); List results = recordStore.scanIndex(SIMPLE_TEXT_WITH_AUTO_COMPLETE, autoComplete(SIMPLE_TEXT_WITH_AUTO_COMPLETE, "good mor"), @@ -1094,15 +1063,14 @@ void searchForAutoCompleteWithContinueTyping(boolean withTermVectors) throws Exc assertEquals(1, context.getTimer().getCounter(LuceneEvents.Counts.LUCENE_SCAN_MATCHED_AUTO_COMPLETE_SUGGESTIONS).getCount()); assertAutoCompleteEntriesAndSegmentInfoStoredInCompoundFile(recordStore.indexSubspace(SIMPLE_TEXT_WITH_AUTO_COMPLETE), context, "_0.cfs", true); + + commit(context); } } - @ParameterizedTest(name = "searchForAutoCompleteForGroupedRecord[withTermVectors={0}]") - @BooleanSource - void searchForAutoCompleteForGroupedRecord(boolean withTermVectors) throws Exception { - final RecordLayerPropertyStorage.Builder storageBuilder = RecordLayerPropertyStorage.newBuilder() - .addProp(LuceneRecordContextProperties.LUCENE_AUTO_COMPLETE_WITH_TERM_VECTORS, withTermVectors); - try (FDBRecordContext context = openContext(storageBuilder)) { + @Test + void searchForAutoCompleteForGroupedRecord() throws Exception { + try (FDBRecordContext context = openContext()) { openRecordStore(context, metaDataBuilder -> { metaDataBuilder.removeIndex(TextIndexTestUtils.SIMPLE_DEFAULT_NAME); metaDataBuilder.addIndex(MAP_DOC, MAP_ON_VALUE_INDEX_WITH_AUTO_COMPLETE); @@ -1133,14 +1101,14 @@ void searchForAutoCompleteForGroupedRecord(boolean withTermVectors) throws Excep assertEquals(1, context.getTimer().getCounter(LuceneEvents.Counts.LUCENE_SCAN_MATCHED_AUTO_COMPLETE_SUGGESTIONS).getCount()); assertAutoCompleteEntriesAndSegmentInfoStoredInCompoundFile(recordStore.indexSubspace(MAP_ON_VALUE_INDEX_WITH_AUTO_COMPLETE).subspace(Tuple.from("sampleTextPhrase")), context, "_0.cfs", true); + + commit(context); } } @Test void testAutoCompleteSearchForPhrase() throws Exception { - final RecordLayerPropertyStorage.Builder storageBuilder = RecordLayerPropertyStorage.newBuilder() - .addProp(LuceneRecordContextProperties.LUCENE_AUTO_COMPLETE_WITH_TERM_VECTORS, false); - try (FDBRecordContext context = openContext(storageBuilder)) { + try (FDBRecordContext context = openContext()) { final Index index = SIMPLE_TEXT_WITH_AUTO_COMPLETE; addIndexAndSaveRecordsForAutoCompleteOfPhrase(context, index); @@ -1191,14 +1159,14 @@ void testAutoCompleteSearchForPhrase() throws Exception { queryAndAssertAutoCompleteSuggestionsReturned(index, "\"united states o\"", ImmutableList.of("united states of america", "welcome to the united states of america")); + + commit(context); } } @Test void testAutoCompleteSearchWithHighlightForPhrase() throws Exception { - final RecordLayerPropertyStorage.Builder storageBuilder = RecordLayerPropertyStorage.newBuilder() - .addProp(LuceneRecordContextProperties.LUCENE_AUTO_COMPLETE_WITH_TERM_VECTORS, false); - try (FDBRecordContext context = openContext(storageBuilder)) { + try (FDBRecordContext context = openContext()) { final Index index = SIMPLE_TEXT_WITH_AUTO_COMPLETE_WITH_HIGHLIGHT; addIndexAndSaveRecordsForAutoCompleteOfPhrase(context, index); @@ -1249,14 +1217,45 @@ void testAutoCompleteSearchWithHighlightForPhrase() throws Exception { queryAndAssertAutoCompleteSuggestionsReturned(index, "\"united states o\"", ImmutableList.of("united states of america", "welcome to the united states of america")); + + commit(context); + } + } + + @Test + void testAutoCompleteSearchMultipleResultsSingleDocument() { + try (FDBRecordContext context = openContext()) { + final Index index = COMPLEX_MULTIPLE_TEXT_INDEXES_WITH_AUTO_COMPLETE; + openRecordStore(context, metaDataBuilder -> { + metaDataBuilder.removeIndex(TextIndexTestUtils.SIMPLE_DEFAULT_NAME); + metaDataBuilder.addIndex(COMPLEX_DOC, index); + }); + + TestRecordsTextProto.ComplexDocument doc = TestRecordsTextProto.ComplexDocument.newBuilder() + .setDocId(1597L) + // Romeo and Juliet, Act II, Scene II. + .setText("Good night! Good night! Parting is such sweet sorrow") + .setText2("That I shall say good night till it be morrow") + .build(); + recordStore.saveRecord(doc); + + List entries = recordStore.scanIndex(index, autoComplete(index, "good night "), null, ScanProperties.FORWARD_SCAN) + .asList() + .join(); + assertThat(entries, hasSize(2)); + + List fieldAndText = entries.stream() + .map(entry -> TupleHelpers.subTuple(entry.getKey(), 0, 2)) + .collect(Collectors.toList()); + assertThat(fieldAndText, containsInAnyOrder(Tuple.from("text", doc.getText()), Tuple.from("text2", doc.getText2()))); + + commit(context); } } @Test void testAutoCompleteSearchForPhraseWithoutFreqsAndPositions() { - final RecordLayerPropertyStorage.Builder storageBuilder = RecordLayerPropertyStorage.newBuilder() - .addProp(LuceneRecordContextProperties.LUCENE_AUTO_COMPLETE_WITH_TERM_VECTORS, false); - try (FDBRecordContext context = openContext(storageBuilder)) { + try (FDBRecordContext context = openContext()) { final Index index = SIMPLE_TEXT_WITH_AUTO_COMPLETE_NO_FREQS_POSITIONS; addIndexAndSaveRecordsForAutoCompleteOfPhrase(context, index); @@ -1265,6 +1264,8 @@ void testAutoCompleteSearchForPhraseWithoutFreqsAndPositions() { assertThrows(ExecutionException.class, () -> queryAndAssertAutoCompleteSuggestionsReturned(index, "\"united states \"", ImmutableList.of())); + + commit(context); } } @@ -1298,6 +1299,8 @@ void searchForSpellCheck() throws Exception { assertEquals("keyboard", result.getKey().get(1)); assertEquals("text", result.getKey().get(0)); assertEquals(0.85714287F, result.getValue().get(0)); + + commit(context); } } @@ -1335,6 +1338,8 @@ void searchForSpellcheckForGroupedRecord() throws Exception { // assertEquals(1, context.getTimer().getCounter(LuceneEvents.Counts.LUCENE_SCAN_MATCHED_AUTO_COMPLETE_SUGGESTIONS).getCount()); // assertAutoCompleteEntriesAndSegmentInfoStoredInCompoundFile(recordStore.indexSubspace(MAP_ON_VALUE_INDEX_WITH_void).subspace(Tuple.from("sampleTextPhrase")), context, "_0.cfs", true); + + commit(context); } } @@ -1533,17 +1538,14 @@ void testDeleteWhereComplexGrouped() { } } - @ParameterizedTest(name = "testDeleteWhereAutoComplete[withTermVectors={0}]") - @BooleanSource - void testDeleteWhereAutoComplete(boolean withTermVectors) { + @Test + void testDeleteWhereAutoComplete() { final RecordMetaDataHook hook = metaDataBuilder -> { TextIndexTestUtils.addRecordTypePrefix(metaDataBuilder); metaDataBuilder.addIndex(COMPLEX_DOC, COMPLEX_MULTI_GROUPED_WITH_AUTO_COMPLETE); }; final int maxGroup = 10; - RecordLayerPropertyStorage.Builder storageBuilder = RecordLayerPropertyStorage.newBuilder() - .addProp(LuceneRecordContextProperties.LUCENE_AUTO_COMPLETE_WITH_TERM_VECTORS, withTermVectors); - try (FDBRecordContext context = openContext(storageBuilder)) { + try (FDBRecordContext context = openContext()) { openRecordStore(context, hook); for (int group = 0; group < maxGroup; group++) { for (long docId = 0L; docId < 10L; docId++) { @@ -1558,18 +1560,18 @@ void testDeleteWhereAutoComplete(boolean withTermVectors) { } commit(context); } - // Re-initialized the builder so the LUCENE_INDEX_COMPRESSION_ENABLED prop is not added twice - storageBuilder = RecordLayerPropertyStorage.newBuilder() - .addProp(LuceneRecordContextProperties.LUCENE_AUTO_COMPLETE_WITH_TERM_VECTORS, withTermVectors); - try (FDBRecordContext context = openContext(storageBuilder)) { + // Re-initialize the builder so the LUCENE_INDEX_COMPRESSION_ENABLED prop is not added twice + try (FDBRecordContext context = openContext()) { openRecordStore(context, hook); for (int group = 0; group < maxGroup; group++) { List autoCompleted = recordStore.scanIndex(COMPLEX_MULTI_GROUPED_WITH_AUTO_COMPLETE, groupedAutoComplete(COMPLEX_MULTI_GROUPED_WITH_AUTO_COMPLETE, "hello", group), null, ScanProperties.FORWARD_SCAN) .asList() .join(); - assertThat(autoCompleted, hasSize(1)); - Tuple key = autoCompleted.get(0).getKey(); - assertEquals(Tuple.from("text", String.format("hello there %d", group)), TupleHelpers.subTuple(key, key.size() - 2, key.size())); + assertThat(autoCompleted, hasSize(10)); + for (IndexEntry entry : autoCompleted) { + Tuple key = entry.getKey(); + assertEquals(Tuple.from("text", String.format("hello there %d", group)), TupleHelpers.subTuple(key, key.size() - 2, key.size())); + } } final int groupToDelete = maxGroup / 2; @@ -1582,9 +1584,11 @@ void testDeleteWhereAutoComplete(boolean withTermVectors) { if (group == groupToDelete) { assertThat(autoCompleted, empty()); } else { - assertThat(autoCompleted, hasSize(1)); - Tuple key = autoCompleted.get(0).getKey(); - assertEquals(Tuple.from("text", String.format("hello there %d", group)), TupleHelpers.subTuple(key, key.size() - 2, key.size())); + assertThat(autoCompleted, hasSize(10)); + for (IndexEntry entry : autoCompleted) { + Tuple key = entry.getKey(); + assertEquals(Tuple.from("text", String.format("hello there %d", group)), TupleHelpers.subTuple(key, key.size() - 2, key.size())); + } } } } @@ -1611,30 +1615,6 @@ void analyzerChooserTest() { } } - @Test - void suggesterRenewTest() throws Exception { - try (FDBRecordContext context = openContext()) { - rebuildIndexMetaData(context, SIMPLE_DOC, SIMPLE_TEXT_WITH_AUTO_COMPLETE); - FDBDirectoryManager manager = FDBDirectoryManager.getManager(new IndexMaintainerState(recordStore, SIMPLE_TEXT_WITH_AUTO_COMPLETE, IndexMaintenanceFilter.NORMAL)); - - // Get the first suggester - AnalyzingInfixSuggester suggester = manager.getAutocompleteSuggester(null, LuceneAnalyzerWrapper.getStandardAnalyzerWrapper(), LuceneAnalyzerWrapper.getStandardAnalyzerWrapper(), false, org.apache.lucene.index.IndexOptions.DOCS); - suggester.add(new BytesRef("test1"), Collections.emptySet(), 1, new BytesRef("payload")); - - // Get the suggester with same parameters, so FDBDirectoryWrapper uses the existing one - manager.getAutocompleteSuggester(null, LuceneAnalyzerWrapper.getStandardAnalyzerWrapper(), LuceneAnalyzerWrapper.getStandardAnalyzerWrapper(), false, org.apache.lucene.index.IndexOptions.DOCS); - // The suggester is not closed, so refresh() works - suggester.refresh(); - suggester.add(new BytesRef("test2"), Collections.emptySet(), 1, new BytesRef("payload")); - - // Get the suggester with different IndexOptions, so the FDBDirectoryWrapper renews its suggester - manager.getAutocompleteSuggester(null, LuceneAnalyzerWrapper.getStandardAnalyzerWrapper(), LuceneAnalyzerWrapper.getStandardAnalyzerWrapper(), false, org.apache.lucene.index.IndexOptions.DOCS_AND_FREQS_AND_POSITIONS); - // The old suggester is closed by FDBDirectoryWrapper - assertThrows(IllegalStateException.class, - () -> suggester.refresh()); - } - } - public static String[] generateRandomWords(int numberOfWords) { assert numberOfWords > 0 : "Number of words have to be greater than 0"; StringBuilder builder = new StringBuilder(); @@ -1656,7 +1636,7 @@ public static String[] generateRandomWords(int numberOfWords) { } private static void assertAutoCompleteEntriesAndSegmentInfoStoredInCompoundFile(@Nonnull Subspace subspace, @Nonnull FDBRecordContext context, @Nonnull String segment, boolean cleanFiles) { - assertEntriesAndSegmentInfoStoredInCompoundFile(subspace.subspace(FDBDirectoryManager.AUTO_COMPLETE_SUFFIX), context, segment, cleanFiles); + assertEntriesAndSegmentInfoStoredInCompoundFile(subspace, context, segment, cleanFiles); } private static void assertEntriesAndSegmentInfoStoredInCompoundFile(@Nonnull Subspace subspace, @Nonnull FDBRecordContext context, @Nonnull String segment, boolean cleanFiles) { @@ -1684,10 +1664,9 @@ private static void assertOnlyCompoundFileExisting(@Nonnull Subspace subspace, @ } } - private void searchForAutoCompleteAndAssert(String query, boolean matches, boolean highlight, int textSizeLimit, boolean withTermVectors) throws Exception { + private void searchForAutoCompleteAndAssert(String query, boolean matches, boolean highlight, int textSizeLimit) throws Exception { final RecordLayerPropertyStorage.Builder storageBuilder = RecordLayerPropertyStorage.newBuilder() - .addProp(LuceneRecordContextProperties.LUCENE_AUTO_COMPLETE_TEXT_SIZE_UPPER_LIMIT, textSizeLimit) - .addProp(LuceneRecordContextProperties.LUCENE_AUTO_COMPLETE_WITH_TERM_VECTORS, withTermVectors); + .addProp(LuceneRecordContextProperties.LUCENE_AUTO_COMPLETE_TEXT_SIZE_UPPER_LIMIT, textSizeLimit); try (FDBRecordContext context = openContext(storageBuilder)) { final RecordType recordType = addIndexAndSaveRecordForAutoComplete(context, highlight); final Index index = highlight ? SIMPLE_TEXT_WITH_AUTO_COMPLETE_WITH_HIGHLIGHT : SIMPLE_TEXT_WITH_AUTO_COMPLETE; @@ -1699,7 +1678,7 @@ private void searchForAutoCompleteAndAssert(String query, boolean matches, boole if (!matches) { // Assert no suggestions assertTrue(results.isEmpty()); - assertEquals(0, context.getTimer().getCounter(LuceneEvents.Counts.LUCENE_SCAN_MATCHED_AUTO_COMPLETE_SUGGESTIONS).getCount()); + assertEquals(0, context.getTimer().getCount(LuceneEvents.Counts.LUCENE_SCAN_MATCHED_AUTO_COMPLETE_SUGGESTIONS)); return; } @@ -1709,40 +1688,30 @@ private void searchForAutoCompleteAndAssert(String query, boolean matches, boole // Assert the suggestions' keys List suggestions = results.stream().map(i -> i.getKey().getString(i.getKeySize() - 1)).collect(Collectors.toList()); if (highlight) { - assertEquals(ImmutableList.of("good evening", "Good night", "Good morning", "Good afternoon", "I'm good", "That's really good!"), suggestions); + assertEquals(ImmutableList.of("Good morning", "Good afternoon", "good evening", "Good night", "That's really good!", "I'm good"), suggestions); } else { - assertEquals(ImmutableList.of("good evening", "Good night", "Good morning", "Good afternoon", "I'm good", "That's really good!"), suggestions); + assertEquals(ImmutableList.of("Good morning", "Good afternoon", "good evening", "Good night", "That's really good!", "I'm good"), suggestions); } // Assert the corresponding field for the suggestions List fields = results.stream().map(i -> i.getKey().getString(i.getKeySize() - 2)).collect(Collectors.toList()); assertEquals(ImmutableList.of("text", "text", "text", "text", "text", "text"), fields); - // Assert the suggestions are sorted according to their values, which are determined by the position of the term into the indexed text - List values = results.stream().map(i -> (Long) i.getValue().get(0)).collect(Collectors.toList()); - List valuesSorted = new ArrayList<>(values); - Collections.sort(valuesSorted, Collections.reverseOrder()); - assertEquals(valuesSorted, values); - assertEquals(values.get(0), values.get(1)); - assertEquals(values.get(1), values.get(2)); - assertEquals(values.get(2), values.get(3)); - assertTrue(values.get(3) > values.get(4)); - assertTrue(values.get(4) > values.get(5)); - results.stream().forEach(i -> assertDocumentPartialRecordFromIndexEntry(recordType, i, (String) i.getKey().get(i.getKeySize() - 1), (String) i.getKey().get(i.getKeySize() - 2), LuceneScanTypes.BY_LUCENE_AUTO_COMPLETE)); - assertEquals(6, context.getTimer().getCounter(LuceneEvents.Counts.LUCENE_SCAN_MATCHED_AUTO_COMPLETE_SUGGESTIONS).getCount()); + assertEquals(6, context.getTimer().getCount(LuceneEvents.Counts.LUCENE_SCAN_MATCHED_AUTO_COMPLETE_SUGGESTIONS)); assertAutoCompleteEntriesAndSegmentInfoStoredInCompoundFile(recordStore.indexSubspace(SIMPLE_TEXT_WITH_AUTO_COMPLETE), context, "_0.cfs", true); + + commit(context); } } - void searchForAutoCompleteWithTextSizeLimit(int limit, boolean matches, boolean withTermVectors) throws Exception { + void searchForAutoCompleteWithTextSizeLimit(int limit, boolean matches) throws Exception { final RecordLayerPropertyStorage.Builder storageBuilder = RecordLayerPropertyStorage.newBuilder() - .addProp(LuceneRecordContextProperties.LUCENE_AUTO_COMPLETE_TEXT_SIZE_UPPER_LIMIT, limit) - .addProp(LuceneRecordContextProperties.LUCENE_AUTO_COMPLETE_WITH_TERM_VECTORS, withTermVectors); + .addProp(LuceneRecordContextProperties.LUCENE_AUTO_COMPLETE_TEXT_SIZE_UPPER_LIMIT, limit); try (FDBRecordContext context = openContext(storageBuilder)) { final RecordType recordType = addIndexAndSaveRecordForAutoComplete(context, false); List results = recordStore.scanIndex(SIMPLE_TEXT_WITH_AUTO_COMPLETE, @@ -1809,7 +1778,7 @@ private void addIndexAndSaveRecordsForAutoCompleteOfPhrase(@Nonnull FDBRecordCon recordStore.saveRecord(createSimpleDocument(1628L, "all the states united as a country", 1)); recordStore.saveRecord(createSimpleDocument(1629L, "states have been united as a country", 1)); recordStore.saveRecord(createSimpleDocument(1630L, "all the states have been united as a country", 1)); - recordStore.saveRecord(createSimpleDocument(1630L, "united states is a country in the continent of america", 1)); + recordStore.saveRecord(createSimpleDocument(1631L, "united states is a country in the continent of america", 1)); } private void queryAndAssertAutoCompleteSuggestionsReturned(@Nonnull Index index, @Nonnull String searchKey, @Nonnull List expectedSuggestions) throws Exception { @@ -1820,7 +1789,7 @@ private void queryAndAssertAutoCompleteSuggestionsReturned(@Nonnull Index index, assertEquals(expectedSuggestions.size(), results.size()); List suggestions = results.stream().map(i -> i.getKey().getString(i.getKeySize() - 1)).collect(Collectors.toList()); - assertEquals(expectedSuggestions, suggestions); + assertThat(suggestions, containsInAnyOrder(expectedSuggestions.stream().map(Matchers::equalTo).collect(Collectors.toList()))); } private void assertDocumentPartialRecordFromIndexEntry(@Nonnull RecordType recordType, @Nonnull IndexEntry indexEntry,