forked from FoundationDB/fdb-record-layer
-
Notifications
You must be signed in to change notification settings - Fork 0
/
LuceneAutoCompleteResultCursor.java
596 lines (551 loc) · 28.5 KB
/
LuceneAutoCompleteResultCursor.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
/*
* LuceneAutoCompleteResultCursor.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.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.lucene.search.LuceneOptimizedIndexSearcher;
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.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.io.StringReader;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Executor;
import java.util.function.Function;
import java.util.stream.Collectors;
/**
* This class is a Record Cursor implementation for Lucene auto complete suggestion lookup.
* Because use cases of auto complete never need to get a big number of suggestions in one call, no scan with continuation support is needed.
* Suggestion result is populated as an {@link IndexEntry}, the key is in {@link IndexEntry#getKey()}, the field where it is indexed from and the value are in {@link IndexEntry#getValue()}.
*/
public class LuceneAutoCompleteResultCursor implements BaseCursor<IndexEntry> {
private static final Logger LOGGER = LoggerFactory.getLogger(LuceneAutoCompleteResultCursor.class);
@Nonnull
private final Executor executor;
@Nonnull
private final IndexMaintainerState state;
@Nonnull
private final String query;
@Nullable
private final FDBStoreTimer timer;
private final int limit;
private final int skip;
@Nullable
private RecordCursor<IndexEntry> lookupResults = null;
@Nullable
private final Tuple groupingKey;
private final boolean highlight;
private final Analyzer queryAnalyzer;
private final Set<String> excludedFieldNames;
public LuceneAutoCompleteResultCursor(@Nonnull String query,
@Nonnull Executor executor, @Nonnull ScanProperties scanProperties,
@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.query = query;
this.executor = executor;
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.highlight = highlight;
this.state = state;
this.groupingKey = groupingKey;
this.queryAnalyzer = queryAnalyzer;
final String excludedFields = this.state.index.getOption(LuceneIndexOptions.AUTO_COMPLETE_EXCLUDED_FIELDS);
this.excludedFieldNames = excludedFields == null ? Collections.emptySet() : LuceneIndexOptions.parseMultipleElementsOptionValue(excludedFields);
}
private synchronized IndexReader getIndexReader() throws IOException {
return FDBDirectoryManager.getManager(state).getIndexReader(groupingKey);
}
@Nonnull
@Override
public CompletableFuture<RecordCursorResult<IndexEntry>> onNext() {
if (lookupResults == null) {
return CompletableFuture.<CompletableFuture<RecordCursorResult<IndexEntry>>>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 lookupResults.onNext();
}).thenCompose(Function.identity());
}
return lookupResults.onNext();
}
@Override
public void close() {
// Nothing to close.
}
@Nonnull
@Override
public Executor getExecutor() {
return executor;
}
@Override
public boolean accept(@Nonnull RecordCursorVisitor visitor) {
visitor.visitEnter(this);
return visitor.visitLeave(this);
}
private void performLookup() throws IOException {
if (lookupResults != null) {
return;
}
long startTime = System.nanoTime();
lookupResults = lookup().skip(skip).limitRowsTo(limit);
if (timer != null) {
timer.recordSinceNanoTime(LuceneEvents.Events.LUCENE_AUTO_COMPLETE_SUGGESTIONS_SCAN, startTime);
}
}
@SuppressWarnings("squid:S3776") // Cognitive complexity is too high. Candidate for later refactoring
@Nullable
@VisibleForTesting
static String searchAllMaybeHighlight(@Nonnull String fieldName, @Nonnull Analyzer queryAnalyzer, @Nonnull String text,
@Nonnull Set<String> matchedTokens, @Nullable String prefixToken,
boolean highlight, boolean allMatchingRequired) {
try (TokenStream ts = queryAnalyzer.tokenStream(fieldName, new StringReader(text))) {
CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
OffsetAttribute offsetAtt = ts.addAttribute(OffsetAttribute.class);
ts.reset();
StringBuilder sb = highlight ? new StringBuilder() : null;
int upto = 0;
Set<String> 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) {
int start = startOffset;
while (start < endOffset) {
int index = text.toLowerCase(Locale.ROOT).indexOf(token, start);
int actualStartOffset = index;
int actualEndOffset = index + token.length();
addNonMatch(sb, text.substring(start, index));
String substring = text.substring(actualStartOffset, actualEndOffset);
if (substring.equalsIgnoreCase(token) && !tokenAlreadyHighlighted(text, actualStartOffset, actualEndOffset)) {
addWholeMatch(sb, substring);
} else {
addNonMatch(sb, substring);
}
start = actualEndOffset;
}
}
upto = endOffset;
matchedInText.add(token);
} else if (prefixToken != null && token.startsWith(prefixToken)) {
if (highlight) {
if (text.substring(startOffset, endOffset).equalsIgnoreCase(token) && !tokenAlreadyHighlighted(text, startOffset, endOffset)) {
addPrefixMatch(sb, text.substring(startOffset, endOffset), prefixToken);
} else {
addNonMatch(sb, text.substring(startOffset, endOffset));
}
}
upto = endOffset;
matchedPrefix = true;
}
}
ts.end();
if (allMatchingRequired && ((prefixToken != null && !matchedPrefix) || (matchedInText.size() < matchedTokens.size()))) {
// Query text not actually found in document text. Return null
return null;
}
// Text was found. Return text (highlighted or not)
if (highlight) {
int endOffset = offsetAtt.endOffset();
if (upto < endOffset) {
addNonMatch(sb, text.substring(upto));
}
return sb.toString();
} else {
return text;
}
} catch (IOException e) {
return null;
}
}
// Check this before highlighting tokens, so the highlighting is idempotent
private static boolean tokenAlreadyHighlighted(String text, int startOffset, int endOffset) {
return startOffset - 3 >= 0
&& endOffset + 4 > text.length()
&& text.startsWith("<b>", startOffset - 3)
&& text.startsWith("</b>", endOffset);
}
/** Called while highlighting a single result, to append a
* non-matching chunk of text from the suggestion to the
* provided fragments list.
* @param sb The {@code StringBuilder} to append to
* @param text The text chunk to add
*/
private static void addNonMatch(StringBuilder sb, String text) {
sb.append(text);
}
/** Called while highlighting a single result, to append
* the whole matched token to the provided fragments list.
* @param sb The {@code StringBuilder} to append to
* @param surface The surface form (original) text
*/
private static void addWholeMatch(StringBuilder sb, String surface) {
sb.append("<b>");
sb.append(surface);
sb.append("</b>");
}
/** 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("<b>");
sb.append(surface.substring(0, prefixToken.length()));
sb.append("</b>");
sb.append(surface.substring(prefixToken.length()));
}
@SuppressWarnings("PMD.CloseResource")
public RecordCursor<IndexEntry> 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<String> tokens = new ArrayList<>();
final String prefixToken = getQueryTokens(queryAnalyzer, searchKey, tokens);
IndexReader indexReader = getIndexReader();
Set<String> fieldNames = getAllIndexedFieldNames(indexReader);
final Set<String> 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 LuceneOptimizedIndexSearcher(indexReader, executor);
TopDocs topDocs = searcher.search(finalQuery, limit + skip);
if (timer != null) {
timer.increment(LuceneEvents.Counts.LUCENE_SCAN_MATCHED_AUTO_COMPLETE_SUGGESTIONS, topDocs.scoreDocs.length);
}
return createResults(searcher, topDocs, tokenSet, prefixToken);
}
private Set<String> getAllIndexedFieldNames(IndexReader indexReader) {
Set<String> fieldNames = new HashSet<>();
indexReader.leaves().forEach(leaf -> leaf.reader().getFieldInfos().forEach(fieldInfo -> {
// Exclude the fields where the field's IndexOptions indicate that the field data is not indexed,
// and the ones which are excluded by the index's options in the schema
if (fieldInfo.getIndexOptions() != IndexOptions.NONE && !excludedFieldNames.contains(fieldInfo.name)) {
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<String> 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<String> fieldNames,
@Nonnull List<String> 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<String> fieldNames,
@Nonnull Set<String> 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<IndexEntry> createResults(IndexSearcher searcher,
TopDocs topDocs,
Set<String> 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<ScoreDocAndRecord> 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<ScoreDocAndRecord> 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<IndexEntry> findIndexEntriesInRecord(ScoreDocAndRecord scoreDocAndRecord, Set<String> queryTokens, @Nullable String prefixToken, @Nullable byte[] continuation) {
// Extract the indexed fields from the document again
final List<LuceneDocumentFromRecord.DocumentField> documentFields = LuceneDocumentFromRecord.getRecordFields(state.index.getRootExpression(), scoreDocAndRecord.rec)
.get(groupingKey == null ? TupleHelpers.EMPTY : groupingKey)
.stream()
.filter(f -> !excludedFieldNames.contains(f.getFieldName()))
.collect(Collectors.toList());
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(documentField.getFieldName(), queryAnalyzer, text, queryTokens, prefixToken, highlight, true);
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),
scoreDocAndRecord.rec.getPrimaryKey());
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));
}
}