You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by do...@apache.org on 2023/01/13 11:48:59 UTC

[lucene] 06/09: hunspell: allow for faster dictionary iteration during 'suggest' by using more memory (opt-in) (#11893)

This is an automated email from the ASF dual-hosted git repository.

donnerpeter pushed a commit to branch branch_9x
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit 462f0cf8f2bad6d516698eb99c9d0efe2e2804c8
Author: Peter Gromov <pe...@jetbrains.com>
AuthorDate: Wed Nov 9 08:20:50 2022 +0100

    hunspell: allow for faster dictionary iteration during 'suggest' by using more memory (opt-in) (#11893)
    
    hunspell: allow for faster dictionary iteration during 'suggest' by using more memory (opt-in)
---
 lucene/CHANGES.txt                                 |   2 +-
 .../analysis/hunspell/GeneratingSuggester.java     |  60 +++---
 .../apache/lucene/analysis/hunspell/Hunspell.java  | 150 +------------
 .../apache/lucene/analysis/hunspell/Suggester.java | 237 +++++++++++++++++++++
 .../analysis/hunspell/SuggestibleEntryCache.java   | 101 +++++++++
 .../lucene/analysis/hunspell/WordStorage.java      |  58 +++--
 .../analysis/hunspell/TestAllDictionaries.java     |  13 ++
 .../lucene/analysis/hunspell/TestDictionary.java   |   6 +
 .../lucene/analysis/hunspell/TestHunspell.java     |  28 +++
 .../lucene/analysis/hunspell/TestPerformance.java  |  12 +-
 .../analysis/hunspell/TestSpellChecking.java       |  19 +-
 11 files changed, 490 insertions(+), 196 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 71e572c8c3e..29a4ffcc09d 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -194,7 +194,7 @@ Optimizations
 
 * GITHUB#12078: Enhance XXXField#newRangeQuery. (Lu Xugang)
 
-* GITHUB#11857, GITHUB#11859: Hunspell: improved suggestion performance (Peter Gromov)
+* GITHUB#11857, GITHUB#11859, GITHUB#11893: Hunspell: improved suggestion performance (Peter Gromov)
 
 Other
 ---------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/GeneratingSuggester.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/GeneratingSuggester.java
index 08b58f925f9..dafb1494ea5 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/GeneratingSuggester.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/GeneratingSuggester.java
@@ -28,7 +28,11 @@ import java.util.Objects;
 import java.util.PriorityQueue;
 import java.util.Set;
 import java.util.TreeSet;
+import java.util.function.BiConsumer;
+import java.util.function.IntPredicate;
+import java.util.function.Supplier;
 import java.util.stream.Collectors;
+import org.apache.lucene.util.CharsRef;
 import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.fst.FST;
 
@@ -43,10 +47,12 @@ class GeneratingSuggester {
   private static final int MAX_ROOT_LENGTH_DIFF = 4;
   private final Dictionary dictionary;
   private final Hunspell speller;
+  private final SuggestibleEntryCache entryCache;
 
-  GeneratingSuggester(Hunspell speller) {
+  GeneratingSuggester(Hunspell speller, SuggestibleEntryCache entryCache) {
     this.dictionary = speller.dictionary;
     this.speller = speller;
+    this.entryCache = entryCache;
   }
 
   List<String> suggest(String word, WordCase originalCase, Set<Suggestion> prevSuggestions) {
@@ -60,7 +66,11 @@ class GeneratingSuggester {
       String word, WordCase originalCase) {
     Comparator<Weighted<Root<String>>> natural = Comparator.naturalOrder();
     PriorityQueue<Weighted<Root<String>>> roots = new PriorityQueue<>(natural.reversed());
-    EntryFilter filter = new EntryFilter(dictionary);
+
+    char[] excludeFlags = dictionary.allNonSuggestibleFlags();
+    FlagEnumerator.Lookup flagLookup = dictionary.flagLookup;
+    IntPredicate isSuggestible = formId -> !flagLookup.hasAnyFlag(formId, excludeFlags);
+
     boolean ignoreTitleCaseRoots = originalCase == WordCase.LOWER && !dictionary.hasLanguage("de");
     TrigramAutomaton automaton =
         new TrigramAutomaton(word) {
@@ -70,10 +80,10 @@ class GeneratingSuggester {
           }
         };
 
-    dictionary.words.processSuggestibleWords(
+    processSuggestibleWords(
         Math.max(1, word.length() - MAX_ROOT_LENGTH_DIFF),
         word.length() + MAX_ROOT_LENGTH_DIFF,
-        (rootChars, forms) -> {
+        (rootChars, formSupplier) -> {
           if (ignoreTitleCaseRoots
               && Character.isUpperCase(rootChars.charAt(0))
               && WordCase.caseOf(rootChars) == WordCase.TITLE) {
@@ -87,44 +97,34 @@ class GeneratingSuggester {
 
           sc += commonPrefix(word, rootChars) - longerWorsePenalty(word.length(), rootChars.length);
 
-          if (roots.size() == MAX_ROOTS && sc <= roots.peek().score) {
+          boolean overflow = roots.size() == MAX_ROOTS;
+          if (overflow && sc <= roots.peek().score) {
             return;
           }
 
           speller.checkCanceled.run();
 
           String root = rootChars.toString();
-          int suitable = filter.findSuitableFormIndex(forms, 0);
-          do {
-            roots.add(new Weighted<>(new Root<>(root, forms.ints[forms.offset + suitable]), sc));
-            suitable = filter.findSuitableFormIndex(forms, suitable + filter.formStep);
-          } while (suitable > 0);
-          while (roots.size() > MAX_ROOTS) {
-            roots.poll();
+          IntsRef forms = formSupplier.get();
+          for (int i = 0; i < forms.length; i++) {
+            if (isSuggestible.test(forms.ints[forms.offset + i])) {
+              roots.add(new Weighted<>(new Root<>(root, forms.ints[forms.offset + i]), sc));
+              if (overflow) {
+                roots.poll();
+              }
+            }
           }
         });
 
     return roots.stream().sorted().collect(Collectors.toList());
   }
 
-  private static class EntryFilter {
-    private final int formStep;
-    private final FlagEnumerator.Lookup flagLookup;
-    private final char[] excludeFlags;
-
-    EntryFilter(Dictionary dic) {
-      formStep = dic.formStep();
-      flagLookup = dic.flagLookup;
-      excludeFlags = dic.allNonSuggestibleFlags();
-    }
-
-    int findSuitableFormIndex(IntsRef forms, int start) {
-      for (int i = start; i < forms.length; i += formStep) {
-        if (!flagLookup.hasAnyFlag(forms.ints[forms.offset + i], excludeFlags)) {
-          return i;
-        }
-      }
-      return -1;
+  private void processSuggestibleWords(
+      int minLength, int maxLength, BiConsumer<CharsRef, Supplier<IntsRef>> processor) {
+    if (entryCache != null) {
+      entryCache.processSuggestibleWords(minLength, maxLength, processor);
+    } else {
+      dictionary.words.processSuggestibleWords(minLength, maxLength, processor);
     }
   }
 
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Hunspell.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Hunspell.java
index b850402bfb4..998bfc77429 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Hunspell.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Hunspell.java
@@ -26,16 +26,8 @@ import static org.apache.lucene.analysis.hunspell.WordContext.COMPOUND_RULE_END;
 import static org.apache.lucene.analysis.hunspell.WordContext.SIMPLE_WORD;
 
 import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedHashSet;
 import java.util.List;
-import java.util.Map;
-import java.util.Optional;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 import org.apache.lucene.util.CharsRef;
 import org.apache.lucene.util.IntsRef;
@@ -56,7 +48,6 @@ import org.apache.lucene.util.IntsRef;
  */
 public class Hunspell {
   static final long SUGGEST_TIME_LIMIT = 250;
-
   final Dictionary dictionary;
   final Stemmer stemmer;
   private final TimeoutPolicy policy;
@@ -75,7 +66,7 @@ public class Hunspell {
     this.dictionary = dictionary;
     this.policy = policy;
     this.checkCanceled = checkCanceled;
-    stemmer = new Stemmer(dictionary);
+    this.stemmer = new Stemmer(dictionary);
   }
 
   /**
@@ -568,6 +559,7 @@ public class Hunspell {
    * @return suggestions for the given misspelled word
    * @throws SuggestionTimeoutException if the computation takes too long and {@link
    *     TimeoutPolicy#THROW_EXCEPTION} was specified in the constructor
+   * @see Suggester for finer-grained APIs and performance optimizations
    */
   public List<String> suggest(String word) throws SuggestionTimeoutException {
     return suggest(word, SUGGEST_TIME_LIMIT);
@@ -579,143 +571,19 @@ public class Hunspell {
    *     TimeoutPolicy}'s effects (exception or partial result) may kick in
    * @throws SuggestionTimeoutException if the computation takes too long and {@link
    *     TimeoutPolicy#THROW_EXCEPTION} was specified in the constructor
+   * @see Suggester for finer-grained APIs and performance optimizations
    */
   public List<String> suggest(String word, long timeLimitMs) throws SuggestionTimeoutException {
-    checkCanceled.run();
-    if (word.length() >= 100) return Collections.emptyList();
-
-    if (dictionary.needsInputCleaning(word)) {
-      word = dictionary.cleanInput(word, new StringBuilder()).toString();
-    }
+    Suggester suggester = new Suggester(dictionary);
+    if (policy == NO_TIMEOUT) return suggester.suggestNoTimeout(word, checkCanceled);
 
-    WordCase wordCase = WordCase.caseOf(word);
-    if (dictionary.forceUCase != FLAG_UNSET && wordCase == WordCase.LOWER) {
-      String title = dictionary.toTitleCase(word);
-      if (spell(title)) {
-        return Collections.singletonList(title);
-      }
-    }
-
-    LinkedHashSet<Suggestion> suggestions = new LinkedHashSet<>();
-    Runnable checkCanceled =
-        policy == NO_TIMEOUT ? this.checkCanceled : checkTimeLimit(word, suggestions, timeLimitMs);
     try {
-      doSuggest(word, wordCase, suggestions, checkCanceled);
+      return suggester.suggestWithTimeout(word, timeLimitMs, checkCanceled);
     } catch (SuggestionTimeoutException e) {
-      if (policy != RETURN_PARTIAL_RESULT) {
-        throw e;
-      }
-    }
-
-    return postprocess(suggestions);
-  }
-
-  private void doSuggest(
-      String word,
-      WordCase wordCase,
-      LinkedHashSet<Suggestion> suggestions,
-      Runnable checkCanceled) {
-    Hunspell suggestionSpeller =
-        new Hunspell(dictionary, policy, checkCanceled) {
-          // Cache for expensive "findStem" requests issued when trying to split a compound word.
-          // The suggestion algorithm issues many of them, often with the same text.
-          // The cache can be large, but will be GC-ed after the "suggest" call.
-          final Map<String, Optional<Root<CharsRef>>> compoundCache = new HashMap<>();
-
-          @Override
-          boolean acceptsStem(int formID) {
-            return !dictionary.hasFlag(formID, dictionary.noSuggest)
-                && !dictionary.hasFlag(formID, dictionary.subStandard);
-          }
-
-          @Override
-          Root<CharsRef> findStem(
-              char[] chars, int offset, int length, WordCase originalCase, WordContext context) {
-            if (context == COMPOUND_BEGIN && originalCase == null) {
-              return compoundCache
-                  .computeIfAbsent(
-                      new String(chars, offset, length),
-                      __ ->
-                          Optional.ofNullable(super.findStem(chars, offset, length, null, context)))
-                  .orElse(null);
-            }
-            return super.findStem(chars, offset, length, originalCase, context);
-          }
-        };
-    boolean hasGoodSuggestions =
-        new ModifyingSuggester(suggestionSpeller, suggestions, word, wordCase).suggest();
-
-    if (!hasGoodSuggestions && dictionary.maxNGramSuggestions > 0) {
-      List<String> generated =
-          new GeneratingSuggester(suggestionSpeller)
-              .suggest(dictionary.toLowerCase(word), wordCase, suggestions);
-      for (String raw : generated) {
-        suggestions.add(new Suggestion(raw, word, wordCase, suggestionSpeller));
+      if (policy == RETURN_PARTIAL_RESULT) {
+        return e.getPartialResult();
       }
+      throw e;
     }
-
-    if (word.contains("-") && suggestions.stream().noneMatch(s -> s.raw.contains("-"))) {
-      for (String raw : modifyChunksBetweenDashes(word)) {
-        suggestions.add(new Suggestion(raw, word, wordCase, suggestionSpeller));
-      }
-    }
-  }
-
-  private Runnable checkTimeLimit(String word, Set<Suggestion> suggestions, long timeLimitMs) {
-    return new Runnable() {
-      final long deadline = System.nanoTime() + TimeUnit.MILLISECONDS.toNanos(timeLimitMs);
-      int invocationCounter = 100;
-
-      @Override
-      public void run() {
-        checkCanceled.run();
-        if (--invocationCounter <= 0) {
-          if (System.nanoTime() - deadline > 0) {
-            stop();
-          }
-          invocationCounter = 100;
-        }
-      }
-
-      private void stop() {
-        List<String> partialResult =
-            policy == RETURN_PARTIAL_RESULT ? null : postprocess(suggestions);
-        String message = "Time limit of " + timeLimitMs + "ms exceeded for " + word;
-        throw new SuggestionTimeoutException(message, partialResult);
-      }
-    };
-  }
-
-  private List<String> postprocess(Collection<Suggestion> suggestions) {
-    return suggestions.stream()
-        .flatMap(s -> Arrays.stream(s.result))
-        .distinct()
-        .collect(Collectors.toList());
-  }
-
-  private List<String> modifyChunksBetweenDashes(String word) {
-    List<String> result = new ArrayList<>();
-    int chunkStart = 0;
-    while (chunkStart < word.length()) {
-      int chunkEnd = word.indexOf('-', chunkStart);
-      if (chunkEnd < 0) {
-        chunkEnd = word.length();
-      }
-
-      if (chunkEnd > chunkStart) {
-        String chunk = word.substring(chunkStart, chunkEnd);
-        if (!spell(chunk)) {
-          for (String chunkSug : suggest(chunk)) {
-            String replaced = word.substring(0, chunkStart) + chunkSug + word.substring(chunkEnd);
-            if (spell(replaced)) {
-              result.add(replaced);
-            }
-          }
-        }
-      }
-
-      chunkStart = chunkEnd + 1;
-    }
-    return result;
   }
 }
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Suggester.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Suggester.java
new file mode 100644
index 00000000000..35c21841bcc
--- /dev/null
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Suggester.java
@@ -0,0 +1,237 @@
+/*
+ * 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.analysis.hunspell;
+
+import static org.apache.lucene.analysis.hunspell.Dictionary.FLAG_UNSET;
+import static org.apache.lucene.analysis.hunspell.TimeoutPolicy.NO_TIMEOUT;
+import static org.apache.lucene.analysis.hunspell.WordContext.COMPOUND_BEGIN;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import org.apache.lucene.util.CharsRef;
+
+/**
+ * A generator for misspelled word corrections based on Hunspell flags. The suggestions are searched
+ * for in two main ways:
+ *
+ * <ol>
+ *   <li>Modification: trying to insert/remove/delete/swap parts of the word to get something
+ *       acceptable. The performance of this part depends heavily on the contents of TRY, MAP, REP,
+ *       KEY directives in the .aff file.
+ *   <li>Enumeration: if the modification hasn't produced "good enough" suggestions, the whole
+ *       dictionary is scanned and simple affixes are added onto the entries to check if that
+ *       produces anything similar to the given misspelled word. This depends on the dictionary size
+ *       and the affix count, and it can take noticeable amount of time. To speed this up, {@link
+ *       #withSuggestibleEntryCache()} can be used.
+ * </ol>
+ */
+public class Suggester {
+  private final Dictionary dictionary;
+  private final SuggestibleEntryCache suggestibleCache;
+
+  public Suggester(Dictionary dictionary) {
+    this(dictionary, null);
+  }
+
+  private Suggester(Dictionary dictionary, SuggestibleEntryCache suggestibleCache) {
+    this.dictionary = dictionary;
+    this.suggestibleCache = suggestibleCache;
+  }
+
+  /**
+   * Returns a copy of this suggester instance with better "Enumeration" phase performance (see
+   * {@link Suggester} documentation), but using more memory. With this option, the dictionary
+   * entries are stored as fast-to-iterate plain words instead of highly compressed prefix trees.
+   */
+  public Suggester withSuggestibleEntryCache() {
+    return new Suggester(dictionary, SuggestibleEntryCache.buildCache(dictionary.words));
+  }
+
+  /**
+   * Compute suggestions for the given misspelled word
+   *
+   * @param word the misspelled word to calculate suggestions for
+   * @param checkCanceled an object that's periodically called, allowing to interrupt or suggestion
+   *     generation by throwing an exception
+   */
+  public List<String> suggestNoTimeout(String word, Runnable checkCanceled) {
+    LinkedHashSet<Suggestion> suggestions = new LinkedHashSet<>();
+    return suggest(word, suggestions, handleCustomTimeoutException(checkCanceled, suggestions));
+  }
+
+  private Runnable handleCustomTimeoutException(
+      Runnable checkCanceled, LinkedHashSet<Suggestion> suggestions) {
+    return () -> {
+      try {
+        checkCanceled.run();
+      } catch (SuggestionTimeoutException e) {
+        if (e.getPartialResult() != null) {
+          throw e;
+        }
+
+        throw new SuggestionTimeoutException(e.getMessage(), postprocess(suggestions));
+      }
+    };
+  }
+
+  /**
+   * @param word the misspelled word to calculate suggestions for
+   * @param timeLimitMs the duration limit in milliseconds after which the computation is interruped
+   *     by an exception
+   * @param checkCanceled an object that's periodically called, allowing to interrupt or suggestion
+   *     generation by throwing an exception
+   * @throws SuggestionTimeoutException if the computation takes too long. Use {@link
+   *     SuggestionTimeoutException#getPartialResult()} to get the suggestions computed up to that
+   *     point
+   */
+  public List<String> suggestWithTimeout(String word, long timeLimitMs, Runnable checkCanceled)
+      throws SuggestionTimeoutException {
+    LinkedHashSet<Suggestion> suggestions = new LinkedHashSet<>();
+    Runnable checkTime = checkTimeLimit(word, suggestions, timeLimitMs, checkCanceled);
+    return suggest(word, suggestions, handleCustomTimeoutException(checkTime, suggestions));
+  }
+
+  private List<String> suggest(
+      String word, LinkedHashSet<Suggestion> suggestions, Runnable checkCanceled)
+      throws SuggestionTimeoutException {
+    checkCanceled.run();
+    if (word.length() >= 100) return Collections.emptyList();
+
+    if (dictionary.needsInputCleaning(word)) {
+      word = dictionary.cleanInput(word, new StringBuilder()).toString();
+    }
+
+    Hunspell suggestionSpeller =
+        new Hunspell(dictionary, NO_TIMEOUT, checkCanceled) {
+          // Cache for expensive "findStem" requests issued when trying to split a compound word.
+          // The suggestion algorithm issues many of them, often with the same text.
+          // The cache can be large, but will be GC-ed after the "suggest" call.
+          final Map<String, Optional<Root<CharsRef>>> compoundCache = new HashMap<>();
+
+          @Override
+          boolean acceptsStem(int formID) {
+            return !dictionary.hasFlag(formID, dictionary.noSuggest)
+                && !dictionary.hasFlag(formID, dictionary.subStandard);
+          }
+
+          @Override
+          Root<CharsRef> findStem(
+              char[] chars, int offset, int length, WordCase originalCase, WordContext context) {
+            if (context == COMPOUND_BEGIN && originalCase == null) {
+              return compoundCache
+                  .computeIfAbsent(
+                      new String(chars, offset, length),
+                      __ ->
+                          Optional.ofNullable(super.findStem(chars, offset, length, null, context)))
+                  .orElse(null);
+            }
+            return super.findStem(chars, offset, length, originalCase, context);
+          }
+        };
+
+    WordCase wordCase = WordCase.caseOf(word);
+    if (dictionary.forceUCase != FLAG_UNSET && wordCase == WordCase.LOWER) {
+      String title = dictionary.toTitleCase(word);
+      if (suggestionSpeller.spell(title)) {
+        return Collections.singletonList(title);
+      }
+    }
+
+    boolean hasGoodSuggestions =
+        new ModifyingSuggester(suggestionSpeller, suggestions, word, wordCase).suggest();
+
+    if (!hasGoodSuggestions && dictionary.maxNGramSuggestions > 0) {
+      List<String> generated =
+          new GeneratingSuggester(suggestionSpeller, suggestibleCache)
+              .suggest(dictionary.toLowerCase(word), wordCase, suggestions);
+      for (String raw : generated) {
+        suggestions.add(new Suggestion(raw, word, wordCase, suggestionSpeller));
+      }
+    }
+
+    if (word.contains("-") && suggestions.stream().noneMatch(s -> s.raw.contains("-"))) {
+      for (String raw : modifyChunksBetweenDashes(word, suggestionSpeller, checkCanceled)) {
+        suggestions.add(new Suggestion(raw, word, wordCase, suggestionSpeller));
+      }
+    }
+    return postprocess(suggestions);
+  }
+
+  private Runnable checkTimeLimit(
+      String word, Set<Suggestion> suggestions, long timeLimitMs, Runnable checkCanceled) {
+    return new Runnable() {
+      final long deadline = System.nanoTime() + TimeUnit.MILLISECONDS.toNanos(timeLimitMs);
+      int invocationCounter = 100;
+
+      @Override
+      public void run() {
+        checkCanceled.run();
+        if (--invocationCounter <= 0) {
+          if (System.nanoTime() - deadline > 0) {
+            stop();
+          }
+          invocationCounter = 100;
+        }
+      }
+
+      private void stop() {
+        String message = "Time limit of " + timeLimitMs + "ms exceeded for " + word;
+        throw new SuggestionTimeoutException(message, postprocess(suggestions));
+      }
+    };
+  }
+
+  private List<String> postprocess(Collection<Suggestion> suggestions) {
+    return suggestions.stream().flatMap(s -> Arrays.stream(s.result)).distinct().toList();
+  }
+
+  private List<String> modifyChunksBetweenDashes(
+      String word, Hunspell speller, Runnable checkCanceled) {
+    List<String> result = new ArrayList<>();
+    int chunkStart = 0;
+    while (chunkStart < word.length()) {
+      int chunkEnd = word.indexOf('-', chunkStart);
+      if (chunkEnd < 0) {
+        chunkEnd = word.length();
+      }
+
+      if (chunkEnd > chunkStart) {
+        String chunk = word.substring(chunkStart, chunkEnd);
+        if (!speller.spell(chunk)) {
+          for (String chunkSug : suggestNoTimeout(chunk, checkCanceled)) {
+            String replaced = word.substring(0, chunkStart) + chunkSug + word.substring(chunkEnd);
+            if (speller.spell(replaced)) {
+              result.add(replaced);
+            }
+          }
+        }
+      }
+
+      chunkStart = chunkEnd + 1;
+    }
+    return result;
+  }
+}
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/SuggestibleEntryCache.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/SuggestibleEntryCache.java
new file mode 100644
index 00000000000..0dd8ce91d1d
--- /dev/null
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/SuggestibleEntryCache.java
@@ -0,0 +1,101 @@
+/*
+ * 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.analysis.hunspell;
+
+import java.util.function.BiConsumer;
+import java.util.function.Supplier;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.CharsRef;
+import org.apache.lucene.util.IntsRef;
+
+/**
+ * A cache allowing for CPU-cache-friendlier iteration over {@link WordStorage} entries that can be
+ * used for suggestions. The words and the form data are stored in plain contiguous arrays with no
+ * compression.
+ */
+class SuggestibleEntryCache {
+  private final short[] lengths;
+  private final char[] roots;
+  private final int[] formData;
+
+  private SuggestibleEntryCache(short[] lengths, char[] roots, int[] formData) {
+    this.lengths = lengths;
+    this.roots = roots;
+    this.formData = formData;
+  }
+
+  static SuggestibleEntryCache buildCache(WordStorage storage) {
+    var consumer =
+        new BiConsumer<CharsRef, Supplier<IntsRef>>() {
+          short[] lengths = new short[10];
+          final StringBuilder roots = new StringBuilder();
+          int[] formData = new int[10];
+          int lenOffset = 0;
+          int formDataOffset = 0;
+
+          @Override
+          public void accept(CharsRef root, Supplier<IntsRef> formSupplier) {
+            if (root.length > Short.MAX_VALUE) {
+              throw new UnsupportedOperationException(
+                  "Too long dictionary entry, please report this to dev@lucene.apache.org");
+            }
+
+            IntsRef forms = formSupplier.get();
+
+            lengths = ArrayUtil.grow(lengths, lenOffset + 2);
+            lengths[lenOffset] = (short) root.length;
+            lengths[lenOffset + 1] = (short) forms.length;
+            lenOffset += 2;
+
+            roots.append(root.chars, root.offset, root.length);
+
+            formData = ArrayUtil.grow(formData, formDataOffset + forms.length);
+            System.arraycopy(forms.ints, forms.offset, formData, formDataOffset, forms.length);
+            formDataOffset += forms.length;
+          }
+        };
+
+    storage.processSuggestibleWords(1, Integer.MAX_VALUE, consumer);
+
+    return new SuggestibleEntryCache(
+        ArrayUtil.copyOfSubArray(consumer.lengths, 0, consumer.lenOffset),
+        consumer.roots.toString().toCharArray(),
+        ArrayUtil.copyOfSubArray(consumer.formData, 0, consumer.formDataOffset));
+  }
+
+  void processSuggestibleWords(
+      int minLength, int maxLength, BiConsumer<CharsRef, Supplier<IntsRef>> processor) {
+    CharsRef chars = new CharsRef(roots, 0, 0);
+    IntsRef forms = new IntsRef(formData, 0, 0);
+    Supplier<IntsRef> formSupplier = () -> forms;
+    int rootOffset = 0;
+    int formDataOffset = 0;
+    for (int i = 0; i < lengths.length; i += 2) {
+      int rootLength = lengths[i];
+      short formDataLength = lengths[i + 1];
+      if (rootLength >= minLength && rootLength <= maxLength) {
+        chars.offset = rootOffset;
+        chars.length = rootLength;
+        forms.offset = formDataOffset;
+        forms.length = formDataLength;
+        processor.accept(chars, formSupplier);
+      }
+      rootOffset += rootLength;
+      formDataOffset += formDataLength;
+    }
+  }
+}
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/WordStorage.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/WordStorage.java
index b66428010b5..84d65da8ad8 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/WordStorage.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/WordStorage.java
@@ -20,6 +20,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.function.BiConsumer;
+import java.util.function.Supplier;
 import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.store.ByteArrayDataOutput;
 import org.apache.lucene.store.DataOutput;
@@ -54,7 +55,8 @@ class WordStorage {
   private static final int COLLISION_MASK = 0x40;
   private static final int SUGGESTIBLE_MASK = 0x20;
   private static final int MAX_STORED_LENGTH = SUGGESTIBLE_MASK - 1;
-
+  private final int maxEntryLength;
+  private final boolean hasCustomMorphData;
   /**
    * A map from word's hash (modulo array's length) into an int containing:
    *
@@ -89,7 +91,10 @@ class WordStorage {
    */
   private final byte[] wordData;
 
-  private WordStorage(int[] hashTable, byte[] wordData) {
+  private WordStorage(
+      int maxEntryLength, boolean hasCustomMorphData, int[] hashTable, byte[] wordData) {
+    this.maxEntryLength = maxEntryLength;
+    this.hasCustomMorphData = hasCustomMorphData;
     this.hashTable = hashTable;
     this.wordData = wordData;
   }
@@ -153,11 +158,13 @@ class WordStorage {
    * can be modified in any way, but may not be saved for later by the processor
    */
   void processSuggestibleWords(
-      int minLength, int maxLength, BiConsumer<CharsRef, IntsRef> processor) {
+      int minLength, int maxLength, BiConsumer<CharsRef, Supplier<IntsRef>> processor) {
     assert minLength <= maxLength;
+    maxLength = Math.min(maxEntryLength, maxLength);
+
     CharsRef chars = new CharsRef(maxLength);
-    IntsRef forms = new IntsRef();
     ByteArrayDataInput in = new ByteArrayDataInput(wordData);
+    var formSupplier = new LazyFormReader(in);
     for (int entryCode : hashTable) {
       int pos = entryCode & OFFSET_MASK;
       int mask = entryCode >>> OFFSET_BITS;
@@ -179,11 +186,7 @@ class WordStorage {
         }
 
         if (mightMatch) {
-          int dataLength = in.readVInt();
-          if (forms.ints.length < dataLength) {
-            forms.ints = new int[dataLength];
-          }
-          readForms(forms, in, dataLength);
+          formSupplier.dataPos = in.getPosition();
           while (prevPos != 0 && wordStart > 0) {
             in.setPosition(prevPos);
             chars.chars[--wordStart] = (char) in.readVInt();
@@ -193,7 +196,7 @@ class WordStorage {
           if (prevPos == 0) {
             chars.offset = wordStart;
             chars.length = maxLength - wordStart;
-            processor.accept(chars, forms);
+            processor.accept(chars, formSupplier);
           }
         }
 
@@ -257,6 +260,7 @@ class WordStorage {
     private final ByteArrayDataOutput dataWriter;
     private int commonPrefixLength, commonPrefixPos;
     private int actualWords;
+    private int maxEntryLength;
 
     /**
      * @param wordCount an approximate number of the words in the resulting dictionary, used to
@@ -297,6 +301,8 @@ class WordStorage {
      * {@link String#compareTo} rules.
      */
     void add(String entry, char[] flags, int morphDataID) throws IOException {
+      maxEntryLength = Math.max(maxEntryLength, entry.length());
+
       if (!entry.equals(currentEntry)) {
         if (currentEntry != null) {
           if (entry.compareTo(currentEntry) < 0) {
@@ -411,8 +417,36 @@ class WordStorage {
     WordStorage build() throws IOException {
       assert !group.isEmpty() : "build() should be only called once";
       flushGroup();
-      return new WordStorage(
-          hashTable, ArrayUtil.copyOfSubArray(wordData, 0, dataWriter.getPosition()));
+      byte[] trimmedData = ArrayUtil.copyOfSubArray(wordData, 0, dataWriter.getPosition());
+      return new WordStorage(maxEntryLength, hasCustomMorphData, hashTable, trimmedData);
+    }
+  }
+
+  private class LazyFormReader implements Supplier<IntsRef> {
+    int dataPos;
+    private final ByteArrayDataInput in;
+    private final IntsRef forms;
+
+    LazyFormReader(ByteArrayDataInput in) {
+      this.in = in;
+      forms = new IntsRef();
+    }
+
+    @Override
+    public IntsRef get() {
+      in.setPosition(dataPos);
+      int entryCount = in.readVInt() / (hasCustomMorphData ? 2 : 1);
+      if (forms.ints.length < entryCount) {
+        forms.ints = new int[entryCount];
+      }
+      for (int i = 0; i < entryCount; i++) {
+        forms.ints[i] = in.readVInt();
+        if (hasCustomMorphData) {
+          in.readVInt();
+        }
+      }
+      forms.length = entryCount;
+      return forms;
     }
   }
 }
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestAllDictionaries.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestAllDictionaries.java
index 999ee2b234b..5c9db6fbdfb 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestAllDictionaries.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestAllDictionaries.java
@@ -149,6 +149,7 @@ public class TestAllDictionaries extends LuceneTestCase {
   }
 
   public void testDictionariesLoadSuccessfully() throws Exception {
+    AtomicLong memoryWithCache = new AtomicLong();
     AtomicLong totalMemory = new AtomicLong();
     AtomicLong totalWords = new AtomicLong();
     int threads = Runtime.getRuntime().availableProcessors();
@@ -159,7 +160,16 @@ public class TestAllDictionaries extends LuceneTestCase {
         (Path aff) -> {
           try {
             Dictionary dic = loadDictionary(aff);
+            new Hunspell(dic).spell("aaaa");
+            Suggester suggester = new Suggester(dic).withSuggestibleEntryCache();
+            try {
+              suggester.suggestWithTimeout("aaaaaaaaaa", Hunspell.SUGGEST_TIME_LIMIT, () -> {});
+            } catch (
+                @SuppressWarnings("unused")
+                SuggestionTimeoutException e) {
+            }
             totalMemory.addAndGet(RamUsageTester.ramUsed(dic));
+            memoryWithCache.addAndGet(RamUsageTester.ramUsed(suggester));
             totalWords.addAndGet(RamUsageTester.ramUsed(dic.words));
             System.out.println(aff + "\t" + memoryUsageSummary(dic));
           } catch (Throwable e) {
@@ -195,6 +205,9 @@ public class TestAllDictionaries extends LuceneTestCase {
     System.out.println("Total memory: " + RamUsageEstimator.humanReadableUnits(totalMemory.get()));
     System.out.println(
         "Total memory for word storage: " + RamUsageEstimator.humanReadableUnits(totalWords.get()));
+    System.out.println(
+        "Additional memory if withSuggestibleEntryCache is enabled: "
+            + RamUsageEstimator.humanReadableUnits(memoryWithCache.get() - totalMemory.get()));
   }
 
   private static String memoryUsageSummary(Dictionary dic) {
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestDictionary.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestDictionary.java
index 7b704d2e31c..81d8f55d788 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestDictionary.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestDictionary.java
@@ -110,6 +110,12 @@ public class TestDictionary extends LuceneTestCase {
     Set<String> processed = new HashSet<>();
     dictionary.words.processSuggestibleWords(
         minLength, maxLength, (word, __) -> processed.add(word.toString()));
+
+    Set<String> cached = new HashSet<>();
+    SuggestibleEntryCache.buildCache(dictionary.words)
+        .processSuggestibleWords(minLength, maxLength, (word, __) -> cached.add(word.toString()));
+    assertEquals(processed, cached);
+
     return processed;
   }
 
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestHunspell.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestHunspell.java
index 9524f682546..8081b534fdb 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestHunspell.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestHunspell.java
@@ -29,6 +29,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.CancellationException;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 import org.apache.lucene.tests.util.LuceneTestCase;
@@ -54,6 +55,33 @@ public class TestHunspell extends LuceneTestCase {
     assertThrows(CancellationException.class, () -> hunspell.suggest("apac"));
   }
 
+  public void testCustomCheckCanceledGivesPartialResult() throws Exception {
+    Dictionary dictionary = loadDictionary(false, "simple.aff", "simple.dic");
+
+    List<String> expected = List.of("apach");
+    assertEquals(expected, new Hunspell(dictionary, NO_TIMEOUT, () -> {}).suggest("apac"));
+
+    AtomicInteger counter = new AtomicInteger();
+    String msg = "msg";
+    Runnable checkCanceled =
+        () -> {
+          if (counter.incrementAndGet() > 400) {
+            throw new SuggestionTimeoutException(msg, null);
+          }
+        };
+
+    Hunspell hunspell = new Hunspell(dictionary, RETURN_PARTIAL_RESULT, checkCanceled);
+    assertEquals(expected, hunspell.suggest("apac"));
+
+    counter.set(0);
+    var e =
+        assertThrows(
+            SuggestionTimeoutException.class,
+            () -> new Suggester(dictionary).suggestNoTimeout("apac", checkCanceled));
+    assertEquals(expected, e.getPartialResult());
+    assertEquals("msg", e.getMessage());
+  }
+
   public void testSuggestionTimeLimit() throws IOException, ParseException {
     int timeLimitMs = 10;
 
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestPerformance.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestPerformance.java
index 190f3cab8c4..3add18ccfa7 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestPerformance.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestPerformance.java
@@ -86,7 +86,7 @@ public class TestPerformance extends LuceneTestCase {
 
   @Test
   public void de_suggest() throws Exception {
-    checkSuggestionPerformance("de", 100);
+    checkSuggestionPerformance("de", 150);
   }
 
   @Test
@@ -163,33 +163,33 @@ public class TestPerformance extends LuceneTestCase {
 
   private void checkSuggestionPerformance(String code, int wordCount) throws Exception {
     Dictionary dictionary = loadDictionary(code);
+    Suggester suggester = new Suggester(dictionary).withSuggestibleEntryCache();
     Hunspell speller = new Hunspell(dictionary, TimeoutPolicy.THROW_EXCEPTION, () -> {});
     List<String> words =
         loadWords(code, wordCount, dictionary).stream()
             .distinct()
-            .filter(w -> hasQuickSuggestions(speller, w))
+            .filter(w -> hasQuickSuggestions(speller, suggester, w))
             .collect(Collectors.toList());
     System.out.println("Checking " + words.size() + " misspelled words");
 
-    Hunspell fullSpeller = new Hunspell(dictionary, TimeoutPolicy.NO_TIMEOUT, () -> {});
     measure(
         "Suggestions for " + code,
         words.size(),
         blackHole -> {
           for (String word : words) {
-            blackHole.accept(fullSpeller.suggest(word));
+            blackHole.accept(suggester.suggestNoTimeout(word, () -> {}));
           }
         });
     System.out.println();
   }
 
-  private boolean hasQuickSuggestions(Hunspell speller, String word) {
+  private boolean hasQuickSuggestions(Hunspell speller, Suggester suggester, String word) {
     if (speller.spell(word)) {
       return false;
     }
 
     try {
-      speller.suggest(word);
+      suggester.suggestWithTimeout(word, Hunspell.SUGGEST_TIME_LIMIT, () -> {});
     } catch (
         @SuppressWarnings("unused")
         SuggestionTimeoutException e) {
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestSpellChecking.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestSpellChecking.java
index 21927aa231a..b5f39c4214f 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestSpellChecking.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestSpellChecking.java
@@ -249,10 +249,14 @@ public class TestSpellChecking extends LuceneTestCase {
     InputStream dictStream = Files.newInputStream(dicFile);
 
     Hunspell speller;
+    Suggester defaultSuggester;
+    Suggester cachingSuggester;
     try {
       Dictionary dictionary =
           new Dictionary(new ByteBuffersDirectory(), "dictionary", affixStream, dictStream);
       speller = new Hunspell(dictionary, TimeoutPolicy.NO_TIMEOUT, () -> {});
+      defaultSuggester = new Suggester(dictionary);
+      cachingSuggester = new Suggester(dictionary).withSuggestibleEntryCache();
     } finally {
       IOUtils.closeWhileHandlingException(affixStream);
       IOUtils.closeWhileHandlingException(dictStream);
@@ -273,12 +277,8 @@ public class TestSpellChecking extends LuceneTestCase {
         assertFalse("Unexpectedly considered correct: " + word, speller.spell(word.trim()));
       }
       if (Files.exists(sug)) {
-        String suggestions =
-            wrongWords.stream()
-                .map(s -> String.join(", ", speller.suggest(s)))
-                .filter(s -> !s.isEmpty())
-                .collect(Collectors.joining("\n"));
-        assertEquals(Files.readString(sug).trim(), suggestions);
+        assertEquals(Files.readString(sug).trim(), suggest(defaultSuggester, wrongWords));
+        assertEquals(Files.readString(sug).trim(), suggest(cachingSuggester, wrongWords));
       }
     } else {
       assertFalse(".sug file without .wrong file!", Files.exists(sug));
@@ -290,6 +290,13 @@ public class TestSpellChecking extends LuceneTestCase {
     }
   }
 
+  private static String suggest(Suggester suggester, List<String> wrongWords) {
+    return wrongWords.stream()
+        .map(s -> String.join(", ", suggester.suggestNoTimeout(s, () -> {})))
+        .filter(s -> !s.isEmpty())
+        .collect(Collectors.joining("\n"));
+  }
+
   private static Set<String> expandWholeDictionary(Path dic, Hunspell speller) throws IOException {
     Set<String> everythingGenerated = new HashSet<>();
     boolean generatedEverything = true;