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();