Skip to content
New issue

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

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

Already on GitHub? Sign in to your account

Resolves #1682: Support auto-complete via querying main Lucene directory #1683

Merged
merged 14 commits into from
May 25, 2022
Merged
Show file tree
Hide file tree
Changes from 6 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
2 changes: 1 addition & 1 deletion docs/ReleaseNotes.md
Original file line number Diff line number Diff line change
Expand Up @@ -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** Fix 5 [(Issue #NNN)](https://github.com/FoundationDB/fdb-record-layer/issues/NNN)
* **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)
Expand Down

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
@@ -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 {
alecgrieser marked this conversation as resolved.
Show resolved Hide resolved
@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()
);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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;
Expand All @@ -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;
Expand All @@ -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;
Expand All @@ -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..
Expand All @@ -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);
Expand Down Expand Up @@ -152,14 +146,8 @@ public RecordCursor<IndexEntry> 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());
}
return new LuceneAutoCompleteResultCursor(scanAutoComplete.getKeyToComplete(),
executor, scanProperties, getAutocompleteQueryAnalyzer(List.of(scanAutoComplete.getKeyToComplete())), state, scanAutoComplete.getGroupKey(), highlightForAutoCompleteIfEnabled);
}

if (scanType.equals(LuceneScanTypes.BY_LUCENE_SPELL_CHECK)) {
Expand All @@ -174,62 +162,21 @@ public RecordCursor<IndexEntry> 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<Integer> 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);
Expand Down Expand Up @@ -266,38 +213,32 @@ private boolean insertField(LuceneDocumentFromRecord.DocumentField field, final
if (storedField != null) {
document.add(storedField);
}
return suggestionAdded;
}

private void writeDocument(@Nonnull List<LuceneDocumentFromRecord.DocumentField> fields, Tuple groupingKey,
byte[] primaryKey) throws IOException {
final List<String> 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<IndexOptions, List<LuceneDocumentFromRecord.DocumentField>> indexOptionsToFieldsMap = getIndexOptionsToFieldsMap(fields);
for (Map.Entry<IndexOptions, List<LuceneDocumentFromRecord.DocumentField>> 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);
}

@Nonnull
private Map<IndexOptions, List<LuceneDocumentFromRecord.DocumentField>> getIndexOptionsToFieldsMap(@Nonnull List<LuceneDocumentFromRecord.DocumentField> fields) {
final Map<IndexOptions, List<LuceneDocumentFromRecord.DocumentField>> map = new HashMap<>();
final Map<IndexOptions, List<LuceneDocumentFromRecord.DocumentField>> 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()));
Expand Down Expand Up @@ -364,14 +305,8 @@ public <M extends Message> CompletableFuture<Void> 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<String> texts, @Nullable IndexOptions indexOptions) throws IOException {
return directoryManager.getAutocompleteSuggester(groupingKey, autoCompleteIndexAnalyzerChooser.chooseAnalyzer(texts),
autoCompleteQueryAnalyzerChooser.chooseAnalyzer(texts), highlightForAutoCompleteIfEnabled, indexOptions);
private Analyzer getAutocompleteQueryAnalyzer(@Nonnull List<String> texts) {
return autoCompleteQueryAnalyzerChooser.chooseAnalyzer(texts).getAnalyzer();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Considering the auto-complete uses the same index as basic index now, not sure whether it still needs a separate query analyzer. I think this depends on the behavior we want to support and we can further discuss this offline.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, sounds good. I moved this to use the index's main analyzer, though there's some weirdness here regarding how synonyms are (or, in the current implementation, aren't) handled. There's also some stuff around getting the analyzer by type that could be cleaned up, though we could also push that to a follow up PR if this approach goes well. (I'm a little hesitant to do it here because it will affect the API of the analyzer registry, and if we decide to revert this PR, that would mean also reverting any changes to implementor of the analyzer registry, too.)

}

private FieldType getTextFieldType(LuceneDocumentFromRecord.DocumentField field) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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<Double> 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<Integer> 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<Boolean> LUCENE_AUTO_COMPLETE_WITH_TERM_VECTORS = RecordLayerPropertyKey.booleanPropertyKey("com.apple.foundationdb.record.lucene.autoComplete.withTermVectors", true);

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -258,14 +258,9 @@ public CompletableFuture<RecordCursorResult<IndexEntry>> 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);
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I abstracted this kind of continuation into its own class. This change preserves the continuation value, but it means we will no longer need to serialize the proto to a byte array (if the continuation isn't needed)

}
}

Expand Down