You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ho...@apache.org on 2022/04/08 16:25:51 UTC
[lucene] branch main updated: LUCENE-10292: Suggest: Fix AnalyzingInfixSuggester / BlendedInfixSuggester to correctly return existing lookup() results during concurrent build()
This is an automated email from the ASF dual-hosted git repository.
hossman pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/lucene.git
The following commit(s) were added to refs/heads/main by this push:
new 5015dc6dbb8 LUCENE-10292: Suggest: Fix AnalyzingInfixSuggester / BlendedInfixSuggester to correctly return existing lookup() results during concurrent build()
5015dc6dbb8 is described below
commit 5015dc6dbb89a2d3f9c2cd0eb1674f6f146d09b4
Author: Chris Hostetter <ho...@apache.org>
AuthorDate: Fri Apr 8 09:25:33 2022 -0700
LUCENE-10292: Suggest: Fix AnalyzingInfixSuggester / BlendedInfixSuggester to correctly return existing lookup() results during concurrent build()
Fix other FST based suggesters so that getCount() returned results consistent with lookup() during concurrent build()
---
lucene/CHANGES.txt | 4 +
.../suggest/analyzing/AnalyzingInfixSuggester.java | 63 +++++++----
.../suggest/analyzing/AnalyzingSuggester.java | 7 +-
.../search/suggest/fst/FSTCompletionLookup.java | 7 +-
.../search/suggest/fst/WFSTCompletionLookup.java | 7 +-
.../search/suggest/SuggestRebuildTestUtil.java | 125 +++++++++++++++++++++
.../analyzing/TestAnalyzingInfixSuggester.java | 47 ++++++++
.../suggest/analyzing/TestAnalyzingSuggester.java | 38 +++++++
.../suggest/analyzing/TestFreeTextSuggester.java | 25 +++++
.../search/suggest/fst/TestFSTCompletion.java | 25 +++++
.../search/suggest/fst/TestWFSTCompletion.java | 36 ++++++
11 files changed, 356 insertions(+), 28 deletions(-)
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index a50d34025ad..6232275892e 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -119,6 +119,10 @@ Bug Fixes
* LUCENE-10466: Ensure IndexSortSortedNumericDocValuesRangeQuery handles sort field
types besides LONG (Andriy Redko)
+* LUCENE-10292: Suggest: Fix AnalyzingInfixSuggester / BlendedInfixSuggester to correctly return
+ existing lookup() results during concurrent build(). Fix other FST based suggesters so that
+ getCount() returned results consistent with lookup() during concurrent build(). (hossman)
+
Build
---------------------
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
index 7b312062f1d..43e93990ae4 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
@@ -25,6 +25,9 @@ import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.AnalyzerWrapper;
import org.apache.lucene.analysis.TokenFilter;
@@ -125,14 +128,26 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
private final boolean commitOnBuild;
private final boolean closeIndexWriterOnBuild;
- /** Used for ongoing NRT additions/updates. */
+ /**
+ * Used for ongoing NRT additions/updates. May be null depending on <code>closeIndexWriterOnBuild
+ * </code> constructor arg
+ */
protected IndexWriter writer;
- /** {@link IndexSearcher} used for lookups. */
+ /** Used to manage concurrent access to writer */
+ protected final Object writerLock = new Object();
+
+ /**
+ * {@link IndexSearcher} used for lookups. May be null if {@link Directory} did not exist on
+ * instantiation and neither {@link #build}, {@link #add}, or {@link #update} have been called
+ */
protected SearcherManager searcherMgr;
/** Used to manage concurrent access to searcherMgr */
- protected final Object searcherMgrLock = new Object();
+ protected final ReadWriteLock searcherMgrLock = new ReentrantReadWriteLock();
+
+ private final Lock searcherMgrReadLock = searcherMgrLock.readLock();
+ private final Lock searcherMgrWriteLock = searcherMgrLock.writeLock();
/** Default minimum number of leading characters before PrefixQuery is used (4). */
public static final int DEFAULT_MIN_PREFIX_CHARS = 4;
@@ -274,6 +289,20 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
}
}
+ private void setAndCloseOldSearcherManager(final SearcherManager newSearcherMgr)
+ throws IOException {
+ searcherMgrWriteLock.lock();
+ try {
+ final SearcherManager oldSearcherMgr = searcherMgr;
+ searcherMgr = newSearcherMgr;
+ if (oldSearcherMgr != null) {
+ oldSearcherMgr.close();
+ }
+ } finally {
+ searcherMgrWriteLock.unlock();
+ }
+ }
+
/** Override this to customize index settings, e.g. which codec to use. */
protected IndexWriterConfig getIndexWriterConfig(
Analyzer indexAnalyzer, IndexWriterConfig.OpenMode openMode) {
@@ -296,12 +325,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
@Override
public void build(InputIterator iter) throws IOException {
- synchronized (searcherMgrLock) {
- if (searcherMgr != null) {
- searcherMgr.close();
- searcherMgr = null;
- }
-
+ synchronized (writerLock) {
if (writer != null) {
writer.close();
writer = null;
@@ -334,7 +358,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
if (commitOnBuild || closeIndexWriterOnBuild) {
commit();
}
- searcherMgr = new SearcherManager(writer, null);
+ setAndCloseOldSearcherManager(new SearcherManager(writer, null));
success = true;
} finally {
if (success) {
@@ -394,7 +418,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
}
private void ensureOpen() throws IOException {
- synchronized (searcherMgrLock) {
+ synchronized (writerLock) {
if (writer == null) {
if (DirectoryReader.indexExists(dir)) {
// Already built; open it:
@@ -406,12 +430,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
new IndexWriter(
dir, getIndexWriterConfig(getGramAnalyzer(), IndexWriterConfig.OpenMode.CREATE));
}
-
- SearcherManager oldSearcherMgr = searcherMgr;
- searcherMgr = new SearcherManager(writer, null);
- if (oldSearcherMgr != null) {
- oldSearcherMgr.close();
- }
+ setAndCloseOldSearcherManager(new SearcherManager(writer, null));
}
}
}
@@ -712,9 +731,12 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
List<LookupResult> results = null;
SearcherManager mgr;
IndexSearcher searcher;
- synchronized (searcherMgrLock) {
+ searcherMgrReadLock.lock();
+ try {
mgr = searcherMgr; // acquire & release on same SearcherManager, via local reference
searcher = mgr.acquire();
+ } finally {
+ searcherMgrReadLock.unlock();
}
try {
// System.out.println("got searcher=" + searcher);
@@ -948,9 +970,12 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
}
SearcherManager mgr;
IndexSearcher searcher;
- synchronized (searcherMgrLock) {
+ searcherMgrReadLock.lock();
+ try {
mgr = searcherMgr; // acquire & release on same SearcherManager, via local reference
searcher = mgr.acquire();
+ } finally {
+ searcherMgrReadLock.unlock();
}
try {
return searcher.getIndexReader().numDocs();
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java
index 9139ec3d470..f20dded1c1c 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java
@@ -164,7 +164,7 @@ public class AnalyzingSuggester extends Lookup {
private boolean preservePositionIncrements;
/** Number of entries the lookup was built with */
- private long count = 0;
+ private volatile long count = 0;
/**
* Calls {@link #AnalyzingSuggester(Directory,String,Analyzer,Analyzer,int,int,int,boolean)
@@ -407,7 +407,7 @@ public class AnalyzingSuggester extends Lookup {
String tempSortedFileName = null;
- count = 0;
+ long newCount = 0;
byte[] buffer = new byte[8];
try {
ByteArrayDataOutput output = new ByteArrayDataOutput(buffer);
@@ -416,7 +416,7 @@ public class AnalyzingSuggester extends Lookup {
LimitedFiniteStringsIterator finiteStrings =
new LimitedFiniteStringsIterator(toAutomaton(surfaceForm, ts2a), maxGraphExpansions);
- for (IntsRef string; (string = finiteStrings.next()) != null; count++) {
+ for (IntsRef string; (string = finiteStrings.next()) != null; newCount++) {
Util.toBytesRef(string, scratch);
// length of the analyzed text (FST input)
@@ -588,6 +588,7 @@ public class AnalyzingSuggester extends Lookup {
}
}
fst = fstCompiler.compile();
+ count = newCount;
// Util.dotToFile(fst, "/tmp/suggest.dot");
} finally {
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionLookup.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionLookup.java
index c76824bc92a..9d91f32bf62 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionLookup.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionLookup.java
@@ -98,7 +98,7 @@ public class FSTCompletionLookup extends Lookup {
private FSTCompletion normalCompletion;
/** Number of entries the lookup was built with */
- private long count = 0;
+ private volatile long count = 0;
/** This constructor should only be used to read a previously saved suggester. */
public FSTCompletionLookup() {
@@ -171,7 +171,6 @@ public class FSTCompletionLookup extends Lookup {
// Push floats up front before sequences to sort them. For now, assume they are non-negative.
// If negative floats are allowed some trickery needs to be done to find their byte order.
- count = 0;
try {
byte[] buffer = new byte[0];
ByteArrayDataOutput output = new ByteArrayDataOutput(buffer);
@@ -210,6 +209,7 @@ public class FSTCompletionLookup extends Lookup {
int previousScore = 0;
ByteArrayDataInput input = new ByteArrayDataInput();
BytesRef tmp2 = new BytesRef();
+ long newCount = 0;
while (true) {
BytesRef scratch = reader.next();
if (scratch == null) {
@@ -234,13 +234,14 @@ public class FSTCompletionLookup extends Lookup {
builder.add(tmp2, bucket);
line++;
- count++;
+ newCount++;
}
// The two FSTCompletions share the same automaton.
this.higherWeightsCompletion = builder.build();
this.normalCompletion =
new FSTCompletion(higherWeightsCompletion.getFST(), false, exactMatchFirst);
+ this.count = newCount;
} finally {
IOUtils.closeWhileHandlingException(reader, writer, externalSorter);
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java
index a9c3924e06d..0e7999bdc96 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java
@@ -68,7 +68,7 @@ public class WFSTCompletionLookup extends Lookup {
private final boolean exactFirst;
/** Number of entries the lookup was built with */
- private long count = 0;
+ private volatile long count = 0;
private final Directory tempDir;
private final String tempFileNamePrefix;
@@ -102,7 +102,7 @@ public class WFSTCompletionLookup extends Lookup {
if (iterator.hasContexts()) {
throw new IllegalArgumentException("this suggester doesn't support contexts");
}
- count = 0;
+ long newCount = 0;
BytesRef scratch = new BytesRef();
InputIterator iter = new WFSTInputIterator(tempDir, tempFileNamePrefix, iterator);
IntsRefBuilder scratchInts = new IntsRefBuilder();
@@ -121,9 +121,10 @@ public class WFSTCompletionLookup extends Lookup {
Util.toIntsRef(scratch, scratchInts);
fstCompiler.add(scratchInts.get(), cost);
previous.copyBytes(scratch);
- count++;
+ newCount++;
}
fst = fstCompiler.compile();
+ count = newCount;
}
@Override
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/SuggestRebuildTestUtil.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/SuggestRebuildTestUtil.java
new file mode 100644
index 00000000000..14b74eeeab7
--- /dev/null
+++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/SuggestRebuildTestUtil.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.lucene.search.suggest;
+
+import static org.junit.Assert.assertNull;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicReference;
+
+/** Reusable Logic for confirming that Lookup impls can return suggestions during a 'rebuild' */
+public final class SuggestRebuildTestUtil {
+
+ /**
+ * Given a {@link Lookup} impl and some assertion callbacks, confirms that assertions which pass
+ * after an initial build will continue to pass during a (slow) rebuild w/new data (in a
+ * background thread), and that (optional) new assertions will pass once the rebuild is complete
+ *
+ * @param suggester to be tested
+ * @param initialData initial data to use for initial {@link Lookup#build}
+ * @param initialChecks assertions to test after the initial build, and during the re-{@link
+ * Lookup#build}
+ * @param extraData will be aded to <code>initialData</code> and used to re-<code>build()</code>
+ * the suggester
+ * @param finalChecks assertions to test after the re-<code>build()</code> completes
+ */
+ public static void testLookupsDuringReBuild(
+ final Lookup suggester,
+ final List<Input> initialData,
+ final ExceptionalCallback initialChecks,
+ final List<Input> extraData,
+ final ExceptionalCallback finalChecks)
+ throws Exception {
+ // copy we can mutate
+ final List<Input> data = new ArrayList<>(initialData);
+ suggester.build(new InputArrayIterator(data));
+
+ // sanity check initial results
+ initialChecks.check(suggester);
+
+ // modify source data we're going to build from, and spin up background thread that
+ // will rebuild (slowly)
+ data.addAll(extraData);
+ final Semaphore rebuildGate = new Semaphore(0);
+ final AtomicReference<Throwable> buildError = new AtomicReference<>();
+ final Thread rebuilder =
+ new Thread(
+ () -> {
+ try {
+ suggester.build(
+ new InputArrayIterator(new DelayedIterator<>(rebuildGate, data.iterator())));
+ } catch (Throwable t) {
+ buildError.set(t);
+ }
+ });
+ rebuilder.start();
+ // at every stage of the slow rebuild, we should still be able to get our original suggestions
+ for (int i = 0; i < data.size(); i++) {
+ initialChecks.check(suggester);
+ rebuildGate.release();
+ }
+ // once all the data is releasedfrom the iterator, the background rebuild should finish, and
+ // suggest results
+ // should change
+ rebuilder.join();
+ assertNull(buildError.get());
+ finalChecks.check(suggester);
+ }
+
+ /**
+ * Simple marker interface to allow {@link #testLookupsDuringReBuild} callbacks to throw
+ * Exceptions
+ */
+ public static interface ExceptionalCallback {
+ public void check(final Lookup suggester) throws Exception;
+ }
+
+ /**
+ * An iterator wrapper whose {@link Iterator#next} method will only return when a Semaphore permit
+ * is acquirable
+ */
+ private static final class DelayedIterator<E> implements Iterator<E> {
+ final Iterator<E> inner;
+ final Semaphore gate;
+
+ public DelayedIterator(final Semaphore gate, final Iterator<E> inner) {
+ assert null != gate;
+ assert null != inner;
+ this.gate = gate;
+ this.inner = inner;
+ }
+
+ @Override
+ public boolean hasNext() {
+ return inner.hasNext();
+ }
+
+ @Override
+ public E next() {
+ gate.acquireUninterruptibly();
+ return inner.next();
+ }
+
+ @Override
+ public void remove() {
+ inner.remove();
+ }
+ }
+}
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/TestAnalyzingInfixSuggester.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/TestAnalyzingInfixSuggester.java
index a9cf00e9dcd..8df561256f3 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/TestAnalyzingInfixSuggester.java
+++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/TestAnalyzingInfixSuggester.java
@@ -21,6 +21,7 @@ import java.io.StringReader;
import java.io.UncheckedIOException;
import java.nio.file.Path;
import java.util.ArrayList;
+import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
@@ -40,6 +41,7 @@ import org.apache.lucene.search.SearcherManager;
import org.apache.lucene.search.suggest.Input;
import org.apache.lucene.search.suggest.InputArrayIterator;
import org.apache.lucene.search.suggest.Lookup.LookupResult;
+import org.apache.lucene.search.suggest.SuggestRebuildTestUtil;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.Directory;
import org.apache.lucene.tests.analysis.MockAnalyzer;
@@ -180,6 +182,51 @@ public class TestAnalyzingInfixSuggester extends LuceneTestCase {
a.close();
}
+ public void testLookupsDuringReBuild() throws Exception {
+ Analyzer a = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false);
+ AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(newDirectory(), a, a, 3, false);
+
+ SuggestRebuildTestUtil.testLookupsDuringReBuild(
+ suggester,
+ Arrays.asList(
+ new Input("lend me your ear", 8, new BytesRef("foobar")),
+ new Input("a penny saved is a penny earned", 10, new BytesRef("foobaz"))),
+ s -> {
+ assertEquals(2, s.getCount());
+ List<LookupResult> results =
+ s.lookup(
+ TestUtil.stringToCharSequence("ear", random()),
+ (BooleanQuery) null,
+ 10,
+ true,
+ true);
+ assertEquals(2, results.size());
+ assertEquals("a penny saved is a penny earned", results.get(0).key);
+ assertEquals("a penny saved is a penny <b>ear</b>ned", results.get(0).highlightKey);
+ assertEquals(10, results.get(0).value);
+ assertEquals(new BytesRef("foobaz"), results.get(0).payload);
+ },
+ Arrays.asList(new Input("earned run average", 42, new BytesRef("yakbaz"))),
+ s -> {
+ assertEquals(3, s.getCount());
+ List<LookupResult> results =
+ s.lookup(
+ TestUtil.stringToCharSequence("ear", random()),
+ (BooleanQuery) null,
+ 10,
+ true,
+ true);
+ assertEquals(3, results.size());
+ assertEquals("earned run average", results.get(0).key);
+ assertEquals("<b>ear</b>ned run average", results.get(0).highlightKey);
+ assertEquals(42, results.get(0).value);
+ assertEquals(new BytesRef("yakbaz"), results.get(0).payload);
+ });
+
+ suggester.close();
+ a.close();
+ }
+
/** Used to return highlighted result; see {@link LookupResult#highlightKey} */
private static final class LookupHighlightFragment {
/** Portion of text for this fragment. */
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/TestAnalyzingSuggester.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/TestAnalyzingSuggester.java
index 433fa3717d2..4f6fe5a5ffc 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/TestAnalyzingSuggester.java
+++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/TestAnalyzingSuggester.java
@@ -23,6 +23,7 @@ import java.io.Reader;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.ArrayList;
+import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
@@ -42,6 +43,7 @@ import org.apache.lucene.document.Document;
import org.apache.lucene.search.suggest.Input;
import org.apache.lucene.search.suggest.InputArrayIterator;
import org.apache.lucene.search.suggest.Lookup.LookupResult;
+import org.apache.lucene.search.suggest.SuggestRebuildTestUtil;
import org.apache.lucene.store.Directory;
import org.apache.lucene.tests.analysis.CannedBinaryTokenStream;
import org.apache.lucene.tests.analysis.CannedBinaryTokenStream.BinaryToken;
@@ -169,6 +171,42 @@ public class TestAnalyzingSuggester extends LuceneTestCase {
IOUtils.close(analyzer, tempDir);
}
+ public void testLookupsDuringReBuild() throws Exception {
+ Directory tempDir = getDirectory();
+ Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.KEYWORD, false);
+ AnalyzingSuggester suggester = new AnalyzingSuggester(tempDir, "suggest", analyzer);
+ SuggestRebuildTestUtil.testLookupsDuringReBuild(
+ suggester,
+ Arrays.asList(new Input("foo", 50), new Input("bar", 10), new Input("barbar", 12)),
+ s -> {
+ assertEquals(3, s.getCount());
+ // top 3, but only 2 found
+ List<LookupResult> results =
+ s.lookup(TestUtil.stringToCharSequence("ba", random()), false, 3);
+ assertEquals(2, results.size());
+ assertEquals("barbar", results.get(0).key.toString());
+ assertEquals(12, results.get(0).value);
+ assertEquals("bar", results.get(1).key.toString());
+ assertEquals(10, results.get(1).value);
+ },
+ Arrays.asList(new Input("barbara", 6)),
+ s -> {
+ assertEquals(4, s.getCount());
+ // top 3
+ List<LookupResult> results =
+ s.lookup(TestUtil.stringToCharSequence("ba", random()), false, 3);
+ assertEquals(3, results.size());
+ assertEquals("barbar", results.get(0).key.toString());
+ assertEquals(12, results.get(0).value);
+ assertEquals("bar", results.get(1).key.toString());
+ assertEquals(10, results.get(1).value);
+ assertEquals("barbara", results.get(2).key.toString());
+ assertEquals(6, results.get(2).value);
+ });
+
+ IOUtils.close(analyzer, tempDir);
+ }
+
public void testRandomRealisticKeys() throws IOException {
LineFileDocs lineFile = new LineFileDocs(random());
Map<String, Long> mapping = new HashMap<>();
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/TestFreeTextSuggester.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/TestFreeTextSuggester.java
index 9ba471400a8..6f1df315ba6 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/TestFreeTextSuggester.java
+++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/TestFreeTextSuggester.java
@@ -22,6 +22,7 @@ import java.io.OutputStream;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.ArrayList;
+import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
@@ -39,6 +40,7 @@ import org.apache.lucene.search.suggest.Input;
import org.apache.lucene.search.suggest.InputArrayIterator;
import org.apache.lucene.search.suggest.InputIterator;
import org.apache.lucene.search.suggest.Lookup.LookupResult;
+import org.apache.lucene.search.suggest.SuggestRebuildTestUtil;
import org.apache.lucene.tests.analysis.MockAnalyzer;
import org.apache.lucene.tests.analysis.MockTokenizer;
import org.apache.lucene.tests.util.LineFileDocs;
@@ -93,6 +95,29 @@ public class TestFreeTextSuggester extends LuceneTestCase {
a.close();
}
+ public void testLookupsDuringReBuild() throws Exception {
+ Analyzer a = new MockAnalyzer(random());
+ FreeTextSuggester sug = new FreeTextSuggester(a, a, 2, (byte) 0x20);
+ SuggestRebuildTestUtil.testLookupsDuringReBuild(
+ sug,
+ Arrays.asList(new Input("foo bar baz blah", 50)),
+ s -> {
+ assertEquals(1, s.getCount());
+ List<LookupResult> result = s.lookup("foo ", true, 5);
+ assertEquals(1, result.size());
+ assertEquals("foo bar", result.get(0).key.toString());
+ },
+ Arrays.asList(new Input("boo foo bar foo bee", 20)),
+ s -> {
+ assertEquals(2, s.getCount());
+ List<LookupResult> result = s.lookup("foo ", true, 5);
+ assertEquals(2, result.size());
+ assertEquals("foo bar", result.get(0).key.toString());
+ assertEquals("foo bee", result.get(1).key.toString());
+ });
+ a.close();
+ }
+
public void testIllegalByteDuringBuild() throws Exception {
// Default separator is INFORMATION SEPARATOR TWO
// (0x1e), so no input token is allowed to contain it
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/TestFSTCompletion.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/TestFSTCompletion.java
index adcaaf12bba..76e52afa5ea 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/TestFSTCompletion.java
+++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/TestFSTCompletion.java
@@ -210,6 +210,31 @@ public class TestFSTCompletion extends LuceneTestCase {
assertMatchEquals(completion.lookup(stringToCharSequence(""), 10));
}
+ public void testLookupsDuringReBuild() throws Exception {
+ Directory tempDir = getDirectory();
+ FSTCompletionLookup lookup = new FSTCompletionLookup(tempDir, "fst");
+ SuggestRebuildTestUtil.testLookupsDuringReBuild(
+ lookup,
+ Arrays.asList(tf("wit", 42), tf("ham", 3), tf("with", 7)),
+ s -> {
+ assertEquals(3, s.getCount());
+ List<LookupResult> result = s.lookup(stringToCharSequence("wit"), true, 5);
+ assertEquals(2, result.size());
+ assertEquals("wit", result.get(0).key.toString());
+ assertEquals("with", result.get(1).key.toString());
+ },
+ Arrays.asList(tf("witch", 30)),
+ s -> {
+ assertEquals(4, s.getCount());
+ List<LookupResult> result = s.lookup(stringToCharSequence("wit"), true, 5);
+ assertEquals(3, result.size());
+ assertEquals("wit", result.get(0).key.toString());
+ assertEquals("witch", result.get(1).key.toString());
+ assertEquals("with", result.get(2).key.toString());
+ });
+ tempDir.close();
+ }
+
public void testRandom() throws Exception {
List<Input> freqs = new ArrayList<>();
Random rnd = random();
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/TestWFSTCompletion.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/TestWFSTCompletion.java
index 460bef8bafd..a61e6275b9c 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/TestWFSTCompletion.java
+++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/TestWFSTCompletion.java
@@ -20,6 +20,7 @@ import java.util.*;
import org.apache.lucene.search.suggest.Input;
import org.apache.lucene.search.suggest.InputArrayIterator;
import org.apache.lucene.search.suggest.Lookup.LookupResult;
+import org.apache.lucene.search.suggest.SuggestRebuildTestUtil;
import org.apache.lucene.store.Directory;
import org.apache.lucene.tests.util.LuceneTestCase;
import org.apache.lucene.tests.util.TestUtil;
@@ -80,6 +81,41 @@ public class TestWFSTCompletion extends LuceneTestCase {
tempDir.close();
}
+ public void testLookupsDuringReBuild() throws Exception {
+ Directory tempDir = getDirectory();
+ WFSTCompletionLookup suggester = new WFSTCompletionLookup(tempDir, "wfst");
+ SuggestRebuildTestUtil.testLookupsDuringReBuild(
+ suggester,
+ Arrays.asList(new Input("foo", 50), new Input("bar", 10), new Input("barbar", 12)),
+ s -> {
+ assertEquals(3, s.getCount());
+ // top 3, but only 2 found
+ List<LookupResult> results =
+ s.lookup(TestUtil.stringToCharSequence("ba", random()), false, 3);
+ assertEquals(2, results.size());
+ assertEquals("barbar", results.get(0).key.toString());
+ assertEquals(12, results.get(0).value);
+ assertEquals("bar", results.get(1).key.toString());
+ assertEquals(10, results.get(1).value);
+ },
+ Arrays.asList(new Input("barbara", 6)),
+ s -> {
+ assertEquals(4, s.getCount());
+ // top 3
+ List<LookupResult> results =
+ s.lookup(TestUtil.stringToCharSequence("ba", random()), false, 3);
+ assertEquals(3, results.size());
+ assertEquals("barbar", results.get(0).key.toString());
+ assertEquals(12, results.get(0).value);
+ assertEquals("bar", results.get(1).key.toString());
+ assertEquals(10, results.get(1).value);
+ assertEquals("barbara", results.get(2).key.toString());
+ assertEquals(6, results.get(2).value);
+ });
+
+ tempDir.close();
+ }
+
public void testExactFirst() throws Exception {
Directory tempDir = getDirectory();