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 #2993: AnalyzerChooser no longer takes the text when choosing an analyzer #2994

Draft
wants to merge 8 commits into
base: main
Choose a base branch
from
2 changes: 1 addition & 1 deletion docs/ReleaseNotes.md
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ The Apache Commons library has been removed as a dependency. There were a few lo
* **Breaking change** Change 2 [(Issue #NNN)](https://github.com/FoundationDB/fdb-record-layer/issues/NNN)
* **Breaking change** Change 3 [(Issue #NNN)](https://github.com/FoundationDB/fdb-record-layer/issues/NNN)
* **Breaking change** Change 4 [(Issue #NNN)](https://github.com/FoundationDB/fdb-record-layer/issues/NNN)
* **Breaking change** Change 5 [(Issue #NNN)](https://github.com/FoundationDB/fdb-record-layer/issues/NNN)
* **Breaking change** AnalyzerChooser no longer takes the text when chosing an analyzer [(Issue #2993)](https://github.com/FoundationDB/fdb-record-layer/issues/2993)

// end next release
-->
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,18 +23,11 @@
import org.apache.lucene.analysis.Analyzer;

import javax.annotation.Nonnull;
import java.util.Collections;
import java.util.List;

/**
* Choose an {@link Analyzer} given texts.
* Choose an {@link Analyzer}.
*/
public interface AnalyzerChooser {
@Nonnull
default LuceneAnalyzerWrapper chooseAnalyzer(@Nonnull String text) {
return chooseAnalyzer(Collections.singletonList(text));
}

@Nonnull
LuceneAnalyzerWrapper chooseAnalyzer(@Nonnull List<String> texts);
LuceneAnalyzerWrapper chooseAnalyzer();
}
Original file line number Diff line number Diff line change
Expand Up @@ -65,7 +65,7 @@ public AnalyzerChooser getIndexAnalyzerChooser(@Nonnull Index index) {
final String minLengthString = Optional.ofNullable(index.getOption(IndexOptions.TEXT_TOKEN_MIN_SIZE)).orElse(DEFAULT_MINIMUM_TOKEN_LENGTH);
final String maxLengthString = Optional.ofNullable(index.getOption(IndexOptions.TEXT_TOKEN_MAX_SIZE)).orElse(Integer.toString(UAX29URLEmailAnalyzer.DEFAULT_MAX_TOKEN_LENGTH));

return t -> new LuceneAnalyzerWrapper(UNIQUE_IDENTIFIER,
return () -> new LuceneAnalyzerWrapper(UNIQUE_IDENTIFIER,
new EmailCjkSynonymAnalyzer(MINIMAL_STOP_WORDS, 1, Integer.parseInt(minLengthString), Integer.parseInt(maxLengthString), true,
false, null));
} catch (NumberFormatException ex) {
Expand All @@ -81,7 +81,7 @@ public AnalyzerChooser getQueryAnalyzerChooser(@Nonnull Index index, @Nonnull An
final String minLengthString = Optional.ofNullable(index.getOption(IndexOptions.TEXT_TOKEN_MIN_SIZE)).orElse(DEFAULT_MINIMUM_TOKEN_LENGTH);
final String maxLengthString = Optional.ofNullable(index.getOption(IndexOptions.TEXT_TOKEN_MAX_SIZE)).orElse(DEFAULT_MAXIMUM_TOKEN_LENGTH);
final String synonymConfigName = index.getOption(LuceneIndexOptions.TEXT_SYNONYM_SET_NAME_OPTION);
return t -> new LuceneAnalyzerWrapper(UNIQUE_IDENTIFIER,
return () -> new LuceneAnalyzerWrapper(UNIQUE_IDENTIFIER,
new EmailCjkSynonymAnalyzer(MINIMAL_STOP_WORDS, 1, Integer.parseInt(minLengthString), Integer.parseInt(maxLengthString), true,
synonymConfigName != null, synonymConfigName != null ? SynonymMapRegistryImpl.instance().getSynonymMap(synonymConfigName) : null));
} catch (NumberFormatException ex) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,8 +24,6 @@

import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.SortedMap;
import java.util.TreeMap;
Expand All @@ -37,47 +35,39 @@
* The default analyzer chooser is used for all fields of one Lucene index except the fields which has overrides in the analyzer chooser per field mapping.
*/
public class LuceneAnalyzerCombinationProvider {
public static final String DELINEATOR_BETWEEN_KEY_AND_VALUE = ":";

public static final String DELINEATOR_BETWEEN_KEY_VALUE_PAIRS = ",";
private AnalyzerChooser defaultIndexAnalyzerChooser;
private AnalyzerChooser defaultQueryAnalyzerChooser;
private Map<String, AnalyzerChooser> indexAnalyzerChooserPerFieldOverride;
private Map<String, AnalyzerChooser> queryAnalyzerChooserPerFieldOverride;
@Nonnull
private final LuceneAnalyzerWrapper indexAnalyzerWrapper;
@Nonnull
private final LuceneAnalyzerWrapper queryAnalyzerWrapper;

public LuceneAnalyzerCombinationProvider(@Nonnull AnalyzerChooser defaultIndexAnalyzerChooser, @Nonnull AnalyzerChooser defaultQueryAnalyzerChooser,
@Nullable Map<String, AnalyzerChooser> indexAnalyzerChooserPerFieldOverride, @Nullable Map<String, AnalyzerChooser> queryAnalyzerChooserPerFieldOverride) {
this.defaultIndexAnalyzerChooser = defaultIndexAnalyzerChooser;
this.defaultQueryAnalyzerChooser = defaultQueryAnalyzerChooser;
this.indexAnalyzerChooserPerFieldOverride = indexAnalyzerChooserPerFieldOverride;
this.queryAnalyzerChooserPerFieldOverride = queryAnalyzerChooserPerFieldOverride;
public LuceneAnalyzerCombinationProvider(@Nonnull AnalyzerChooser defaultIndexAnalyzerChooser,
@Nonnull AnalyzerChooser defaultQueryAnalyzerChooser,
@Nullable Map<String, AnalyzerChooser> indexAnalyzerChooserPerFieldOverride,
@Nullable Map<String, AnalyzerChooser> queryAnalyzerChooserPerFieldOverride) {
indexAnalyzerWrapper = buildAnalyzerWrapper(defaultIndexAnalyzerChooser, indexAnalyzerChooserPerFieldOverride);
queryAnalyzerWrapper = buildAnalyzerWrapper(defaultQueryAnalyzerChooser, queryAnalyzerChooserPerFieldOverride);
}

public LuceneAnalyzerWrapper provideIndexAnalyzer(@Nonnull String text) {
return provideIndexAnalyzer(Collections.singletonList(text));
@Nonnull
public LuceneAnalyzerWrapper provideIndexAnalyzer() {
return indexAnalyzerWrapper;
}

public LuceneAnalyzerWrapper provideIndexAnalyzer(@Nonnull List<String> texts) {
return buildAnalyzerWrapper(texts, defaultIndexAnalyzerChooser, indexAnalyzerChooserPerFieldOverride);
}

public LuceneAnalyzerWrapper provideQueryAnalyzer(@Nonnull String text) {
return provideQueryAnalyzer(Collections.singletonList(text));
}

public LuceneAnalyzerWrapper provideQueryAnalyzer(@Nonnull List<String> texts) {
return buildAnalyzerWrapper(texts, defaultQueryAnalyzerChooser, queryAnalyzerChooserPerFieldOverride);
@Nonnull
public LuceneAnalyzerWrapper provideQueryAnalyzer() {
return queryAnalyzerWrapper;
}

@Nonnull
@SuppressWarnings("PMD.CloseResource")
private static LuceneAnalyzerWrapper buildAnalyzerWrapper(@Nonnull List<String> texts,
@Nonnull AnalyzerChooser defaultAnalyzerChooser,
private static LuceneAnalyzerWrapper buildAnalyzerWrapper(@Nonnull AnalyzerChooser defaultAnalyzerChooser,
@Nullable Map<String, AnalyzerChooser> customizedAnalyzerChooserPerField) {
final LuceneAnalyzerWrapper defaultAnalyzerWrapper = defaultAnalyzerChooser.chooseAnalyzer(texts);
final LuceneAnalyzerWrapper defaultAnalyzerWrapper = defaultAnalyzerChooser.chooseAnalyzer();
if (customizedAnalyzerChooserPerField != null) {
// The order of keys matters because the identifier for each map needs to be consistent
SortedMap<String, LuceneAnalyzerWrapper> analyzerWrapperMap = new TreeMap<>(customizedAnalyzerChooserPerField.entrySet().stream()
.collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().chooseAnalyzer(texts))));
.collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().chooseAnalyzer())));

PerFieldAnalyzerWrapper analyzerWrapper = new PerFieldAnalyzerWrapper(defaultAnalyzerWrapper.getAnalyzer(),
analyzerWrapperMap.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().getAnalyzer())));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -75,6 +75,6 @@ public interface LuceneAnalyzerFactory {
*/
@Nonnull
default AnalyzerChooser getQueryAnalyzerChooser(@Nonnull Index index, @Nonnull AnalyzerChooser indexAnalyzerChooser) {
return t -> LuceneAnalyzerWrapper.getStandardAnalyzerWrapper();
return LuceneAnalyzerWrapper::getStandardAnalyzerWrapper;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@
import com.apple.foundationdb.record.metadata.Index;
import com.apple.foundationdb.record.metadata.MetaDataException;
import com.apple.foundationdb.record.util.ServiceLoaderProvider;
import com.apple.foundationdb.record.util.pair.Pair;
import com.apple.foundationdb.record.util.pair.NonnullPair;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -96,14 +96,14 @@ public LuceneAnalyzerCombinationProvider getLuceneAnalyzerCombinationProvider(@N
@Nonnull final Map<String, LuceneIndexExpressions.DocumentFieldDerivation> auxiliaryFieldInfo) {
final String defaultAnalyzerName = index.getOption(type.getAnalyzerOptionKey());
final String analyzerPerFieldName = index.getOption(type.getAnalyzerPerFieldOptionKey());
Pair<AnalyzerChooser, AnalyzerChooser> defaultAnalyzerChooserPair = getAnalyzerChooser(index, defaultAnalyzerName, type);
NonnullPair<AnalyzerChooser, AnalyzerChooser> defaultAnalyzerChooserPair = getAnalyzerChooser(index, defaultAnalyzerName, type);

Map<String, AnalyzerChooser> indexAnalyzerChooserPerFieldOverride = new TreeMap<>();
Map<String, AnalyzerChooser> queryAnalyzerChooserPerFieldOverride = new TreeMap<>();

if (analyzerPerFieldName != null) {
LuceneIndexOptions.parseKeyValuePairOptionValue(analyzerPerFieldName).forEach((fieldName, analyzerName) -> {
Pair<AnalyzerChooser, AnalyzerChooser> perFieldAnalyzerChooserPair = getAnalyzerChooser(index, analyzerName, type);
NonnullPair<AnalyzerChooser, AnalyzerChooser> perFieldAnalyzerChooserPair = getAnalyzerChooser(index, analyzerName, type);
indexAnalyzerChooserPerFieldOverride.put(fieldName, perFieldAnalyzerChooserPair.getLeft());
queryAnalyzerChooserPerFieldOverride.put(fieldName, perFieldAnalyzerChooserPair.getRight());
});
Expand Down Expand Up @@ -145,11 +145,11 @@ private static boolean isEligibleForNoOpAnalyzer(@Nonnull final LuceneIndexExpre
return fieldInfo.getType() != LuceneIndexExpressions.DocumentFieldType.TEXT;
}

private Pair<AnalyzerChooser, AnalyzerChooser> getAnalyzerChooser(@Nonnull Index index, @Nullable String analyzerName, @Nonnull LuceneAnalyzerType type) {
private NonnullPair<AnalyzerChooser, AnalyzerChooser> getAnalyzerChooser(@Nonnull Index index, @Nullable String analyzerName, @Nonnull LuceneAnalyzerType type) {
final Map<String, LuceneAnalyzerFactory> registryForType = Objects.requireNonNullElse(registry.get(type), Collections.emptyMap());
if (analyzerName == null || !registryForType.containsKey(analyzerName)) {
return Pair.of(t -> LuceneAnalyzerWrapper.getStandardAnalyzerWrapper(),
t -> LuceneAnalyzerWrapper.getStandardAnalyzerWrapper());
return NonnullPair.of(LuceneAnalyzerWrapper::getStandardAnalyzerWrapper,
LuceneAnalyzerWrapper::getStandardAnalyzerWrapper);
} else {
LuceneAnalyzerFactory analyzerFactory = registryForType.get(analyzerName);
if (analyzerFactory == null) {
Expand All @@ -158,7 +158,7 @@ private Pair<AnalyzerChooser, AnalyzerChooser> getAnalyzerChooser(@Nonnull Index
LuceneLogMessageKeys.ANALYZER_TYPE, type.name());
}
final AnalyzerChooser indexAnalyzerChooser = analyzerFactory.getIndexAnalyzerChooser(index);
return Pair.of(indexAnalyzerChooser, analyzerFactory.getQueryAnalyzerChooser(index, indexAnalyzerChooser));
return NonnullPair.of(indexAnalyzerChooser, analyzerFactory.getQueryAnalyzerChooser(index, indexAnalyzerChooser));
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,6 @@ public LuceneAnalyzerType getType() {
@Nonnull
@Override
public AnalyzerChooser getIndexAnalyzerChooser(@Nonnull Index index) {
return t -> LuceneAnalyzerWrapper.getStandardAnalyzerWrapper();
return LuceneAnalyzerWrapper::getStandardAnalyzerWrapper;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -122,12 +122,11 @@ public BoundQuery bind(@Nonnull FDBRecordStoreBase<?> store, @Nonnull Index inde
final Map<String, PointsConfig> pointsConfigMap = LuceneIndexExpressions.constructPointConfigMap(store, index);
LuceneQueryParserFactory parserFactory = LuceneQueryParserFactoryProvider.instance().getParserFactory();
final QueryParser parser = parserFactory.createMultiFieldQueryParser(fields.toArray(new String[0]),
analyzerSelector.provideIndexAnalyzer(searchKey).getAnalyzer(), pointsConfigMap);
analyzerSelector.provideIndexAnalyzer().getAnalyzer(), pointsConfigMap);


final var finalQuery = phraseQueryNeeded
? buildQueryForPhraseMatching(parser, fields, searchKey)
: buildQueryForTermsMatching(analyzerSelector.provideIndexAnalyzer(searchKey).getAnalyzer(), fields, searchKey);
final Query finalQuery;
finalQuery = phraseQueryNeeded ? buildQueryForPhraseMatching(parser, fields, searchKey) : buildQueryForTermsMatching(analyzerSelector.provideIndexAnalyzer().getAnalyzer(), fields, searchKey);
if (LOGGER.isDebugEnabled()) {
LOGGER.debug(KeyValueLogMessage.build("query for auto-complete")
.addKeyAndValue(LogMessageKeys.INDEX_NAME, index.getName())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -112,7 +112,7 @@ public class LuceneIndexMaintainer extends StandardIndexMaintainer {
private static final Logger LOG = LoggerFactory.getLogger(LuceneIndexMaintainer.class);

private final FDBDirectoryManager directoryManager;
private final LuceneAnalyzerCombinationProvider indexAnalyzerSelector;
private final LuceneAnalyzerCombinationProvider queryAnalyzerSelector;
private final LuceneAnalyzerCombinationProvider autoCompleteAnalyzerSelector;
public static final String PRIMARY_KEY_FIELD_NAME = "_p";
protected static final String PRIMARY_KEY_SEARCH_NAME = "_s";
Expand All @@ -128,7 +128,7 @@ public LuceneIndexMaintainer(@Nonnull final IndexMaintainerState state, @Nonnull
this.executor = executor;
this.directoryManager = createDirectoryManager(state);
final var fieldInfos = LuceneIndexExpressions.getDocumentFieldDerivations(state.index, state.store.getRecordMetaData());
this.indexAnalyzerSelector = LuceneAnalyzerRegistryImpl.instance().getLuceneAnalyzerCombinationProvider(state.index, LuceneAnalyzerType.FULL_TEXT, fieldInfos);
this.queryAnalyzerSelector = LuceneAnalyzerRegistryImpl.instance().getLuceneAnalyzerCombinationProvider(state.index, LuceneAnalyzerType.FULL_TEXT, fieldInfos);
this.autoCompleteAnalyzerSelector = LuceneAnalyzerRegistryImpl.instance().getLuceneAnalyzerCombinationProvider(state.index, LuceneAnalyzerType.AUTO_COMPLETE, fieldInfos);
String formatString = state.index.getOption(LuceneIndexOptions.PRIMARY_KEY_SERIALIZATION_FORMAT);
keySerializer = LuceneIndexKeySerializer.fromStringFormat(formatString);
Expand Down Expand Up @@ -175,7 +175,7 @@ public RecordCursor<IndexEntry> scan(@Nonnull final IndexScanBounds scanBounds,
state.context.getPropertyStorage().getPropertyValue(LuceneRecordContextProperties.LUCENE_INDEX_CURSOR_PAGE_SIZE),
scanProperties, state, scanQuery.getQuery(), scanQuery.getSort(), continuation,
scanQuery.getGroupKey(), partitionInfo, scanQuery.getLuceneQueryHighlightParameters(), scanQuery.getTermMap(),
scanQuery.getStoredFields(), scanQuery.getStoredFieldTypes(), indexAnalyzerSelector, autoCompleteAnalyzerSelector);
scanQuery.getStoredFields(), scanQuery.getStoredFieldTypes(), queryAnalyzerSelector, autoCompleteAnalyzerSelector);
}

if (scanType.equals(LuceneScanTypes.BY_LUCENE_SPELL_CHECK)) {
Expand All @@ -184,7 +184,8 @@ public RecordCursor<IndexEntry> scan(@Nonnull final IndexScanBounds scanBounds,
}
LuceneScanSpellCheck scanSpellcheck = (LuceneScanSpellCheck)scanBounds;
return new LuceneSpellCheckRecordCursor(scanSpellcheck.getFields(), scanSpellcheck.getWord(),
executor, scanProperties, state, scanSpellcheck.getGroupKey(), partitioner.selectQueryPartitionId(scanSpellcheck.getGroupKey()));
executor, scanProperties, state, scanSpellcheck.getGroupKey(),
partitioner.selectQueryPartitionId(scanSpellcheck.getGroupKey()));
}

throw new RecordCoreException("unsupported scan type for Lucene index: " + scanType);
Expand Down Expand Up @@ -255,7 +256,7 @@ private void writeDocument(@Nonnull List<LuceneDocumentFromRecord.DocumentField>
.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, partitionId, indexAnalyzerSelector.provideIndexAnalyzer(texts));
final IndexWriter newWriter = directoryManager.getIndexWriter(groupingKey, partitionId);

BytesRef ref = new BytesRef(keySerializer.asPackedByteArray(primaryKey));
// use packed Tuple for the Stored and Sorted fields
Expand Down Expand Up @@ -297,7 +298,7 @@ private Map<IndexOptions, List<LuceneDocumentFromRecord.DocumentField>> getIndex
@SuppressWarnings({"PMD.CloseResource", "java:S2095"})
int deleteDocument(Tuple groupingKey, Integer partitionId, Tuple primaryKey) throws IOException {
final long startTime = System.nanoTime();
final IndexWriter indexWriter = directoryManager.getIndexWriter(groupingKey, partitionId, indexAnalyzerSelector.provideIndexAnalyzer(""));
final IndexWriter indexWriter = directoryManager.getIndexWriter(groupingKey, partitionId);
@Nullable final LucenePrimaryKeySegmentIndex segmentIndex = directoryManager.getDirectory(groupingKey, partitionId).getPrimaryKeySegmentIndex();

if (segmentIndex != null) {
Expand Down Expand Up @@ -358,16 +359,15 @@ public CompletableFuture<Void> mergeIndex() {
return rebalancePartitions()
.thenCompose(ignored -> {
state.store.getIndexDeferredMaintenanceControl().setLastStep(IndexDeferredMaintenanceControl.LastStep.MERGE);
return directoryManager.mergeIndex(partitioner, indexAnalyzerSelector.provideIndexAnalyzer(""));
return directoryManager.mergeIndex(partitioner);
});
}

@VisibleForTesting
public void mergeIndexForTesting(@Nonnull final Tuple groupingKey,
@Nullable final Integer partitionId,
@Nonnull final AgilityContext agilityContext) throws IOException {
directoryManager.mergeIndexWithContext(indexAnalyzerSelector.provideIndexAnalyzer(""),
groupingKey, partitionId, agilityContext);
directoryManager.mergeIndexWithContext(groupingKey, partitionId, agilityContext);
}

@Nonnull
Expand Down
Loading