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 2022/11/11 11:13:56 UTC

[lucene] branch main updated: hunspell: introduce FragmentChecker to speed up ModifyingSuggester (#11909)

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

donnerpeter 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 6fbc5f73c39 hunspell: introduce FragmentChecker to speed up ModifyingSuggester (#11909)
6fbc5f73c39 is described below

commit 6fbc5f73c39d6c32f7c824f27d8973bcf702db3a
Author: Peter Gromov <pe...@jetbrains.com>
AuthorDate: Fri Nov 11 12:13:47 2022 +0100

    hunspell: introduce FragmentChecker to speed up ModifyingSuggester (#11909)
    
    hunspell: introduce FragmentChecker to speed up ModifyingSuggester
    
    add NGramFragmentChecker to quickly check whether insertions/replacements produce strings that are even possible in the language
    
    Co-authored-by: Dawid Weiss <da...@gmail.com>
---
 lucene/CHANGES.txt                                 |   2 +-
 .../lucene/analysis/hunspell/FragmentChecker.java  |  43 +++++
 .../analysis/hunspell/ModifyingSuggester.java      |  34 +++-
 .../analysis/hunspell/NGramFragmentChecker.java    | 182 +++++++++++++++++++++
 .../apache/lucene/analysis/hunspell/Suggester.java |  24 ++-
 .../analysis/hunspell/WordFormGenerator.java       |  45 ++++-
 .../lucene/analysis/hunspell/WordStorage.java      |  11 +-
 .../lucene/analysis/hunspell/TestPerformance.java  |  49 ++++--
 .../analysis/hunspell/TestSpellChecking.java       |  26 ++-
 9 files changed, 386 insertions(+), 30 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index aee9d66a641..88b405faf32 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -63,7 +63,7 @@ Improvements
 Optimizations
 ---------------------
 
-* GITHUB#11857, GITHUB#11859, GITHUB#11893: Hunspell: improved suggestion performance
+* GITHUB#11857, GITHUB#11859, GITHUB#11893, GITHUB#11909: Hunspell: improved suggestion performance
 
 Bug Fixes
 ---------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/FragmentChecker.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/FragmentChecker.java
new file mode 100644
index 00000000000..aa3ab88871a
--- /dev/null
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/FragmentChecker.java
@@ -0,0 +1,43 @@
+/*
+ * 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;
+
+/**
+ * An oracle for quickly checking that a specific part of a word can never be a valid word. This
+ * allows speeding up the "Modification" part of {@link Suggester} by avoiding expensive checks on
+ * impossible words. Implementations may use character case, n-grams or whatever they like.
+ *
+ * @see NGramFragmentChecker
+ */
+public interface FragmentChecker {
+  FragmentChecker EVERYTHING_POSSIBLE = (word, start, end) -> false;
+
+  /**
+   * Check if the given word range intersects any fragment which is impossible in the current
+   * language. For example, if the word is "aaax", and there are no "aaa" combinations in words
+   * accepted by the spellchecker (but "aax" is valid), then {@code true} can be returned for all
+   * ranges in {@code 0..3}, but not for {@code 3..4}.
+   *
+   * <p>The implementation must be monotonic: if some range is considered impossible, larger ranges
+   * encompassing it should also produce {@code true}.
+   *
+   * @param word the whole word being checked for impossible substrings
+   * @param start the start of the range in question, inclusive
+   * @param end the end of the range in question, inclusive, not smaller than {@code start}
+   */
+  boolean hasImpossibleFragmentAround(CharSequence word, int start, int end);
+}
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/ModifyingSuggester.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/ModifyingSuggester.java
index 300e2f79fe3..dbcc8f45c34 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/ModifyingSuggester.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/ModifyingSuggester.java
@@ -30,16 +30,22 @@ class ModifyingSuggester {
   private final LinkedHashSet<Suggestion> result;
   private final String misspelled;
   private final WordCase wordCase;
+  private final FragmentChecker fragmentChecker;
   private final char[] tryChars;
   private final Hunspell speller;
 
   ModifyingSuggester(
-      Hunspell speller, LinkedHashSet<Suggestion> result, String misspelled, WordCase wordCase) {
+      Hunspell speller,
+      LinkedHashSet<Suggestion> result,
+      String misspelled,
+      WordCase wordCase,
+      FragmentChecker checker) {
     this.speller = speller;
     tryChars = speller.dictionary.tryChars.toCharArray();
     this.result = result;
     this.misspelled = misspelled;
     this.wordCase = wordCase;
+    fragmentChecker = checker;
   }
 
   /**
@@ -173,19 +179,28 @@ class ModifyingSuggester {
       return;
     }
 
+    int length = accumulated.length();
+
     for (List<String> entries : speller.dictionary.mapTable) {
       for (String entry : entries) {
         if (word.regionMatches(offset, entry, 0, entry.length())) {
           for (String replacement : entries) {
             if (!entry.equals(replacement)) {
-              enumerateMapReplacements(word, accumulated + replacement, offset + entry.length());
+              String next = accumulated + replacement;
+              int end = length + replacement.length();
+              if (!fragmentChecker.hasImpossibleFragmentAround(next, length, end)) {
+                enumerateMapReplacements(word, next, offset + entry.length());
+              }
             }
           }
         }
       }
     }
 
-    enumerateMapReplacements(word, accumulated + word.charAt(offset), offset + 1);
+    String next = accumulated + word.charAt(offset);
+    if (!fragmentChecker.hasImpossibleFragmentAround(next, length, length + 1)) {
+      enumerateMapReplacements(word, next, offset + 1);
+    }
   }
 
   private boolean checkSimpleWord(String part) {
@@ -235,7 +250,8 @@ class ModifyingSuggester {
         if (group.indexOf(c) >= 0) {
           for (int j = 0; j < group.length(); j++) {
             if (group.charAt(j) != c) {
-              trySuggestion(word.substring(0, i) + group.charAt(j) + word.substring(i + 1));
+              tryModifiedSuggestions(
+                  i, word.substring(0, i) + group.charAt(j) + word.substring(i + 1));
             }
           }
         }
@@ -243,6 +259,12 @@ class ModifyingSuggester {
     }
   }
 
+  private void tryModifiedSuggestions(int modOffset, String candidate) {
+    if (!fragmentChecker.hasImpossibleFragmentAround(candidate, modOffset, modOffset + 1)) {
+      trySuggestion(candidate);
+    }
+  }
+
   private void tryLongSwap(String word) {
     for (int i = 0; i < word.length(); i++) {
       for (int j = i + 2; j < word.length() && j <= i + MAX_CHAR_DISTANCE; j++) {
@@ -268,7 +290,7 @@ class ModifyingSuggester {
       String prefix = word.substring(0, i);
       String suffix = word.substring(i);
       for (char toInsert : tryChars) {
-        trySuggestion(prefix + toInsert + suffix);
+        tryModifiedSuggestions(prefix.length(), prefix + toInsert + suffix);
       }
     }
   }
@@ -292,7 +314,7 @@ class ModifyingSuggester {
       String suffix = word.substring(i + 1);
       for (char toInsert : tryChars) {
         if (toInsert != word.charAt(i)) {
-          trySuggestion(prefix + toInsert + suffix);
+          tryModifiedSuggestions(prefix.length(), prefix + toInsert + suffix);
         }
       }
     }
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/NGramFragmentChecker.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/NGramFragmentChecker.java
new file mode 100644
index 00000000000..6c8a1054b10
--- /dev/null
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/NGramFragmentChecker.java
@@ -0,0 +1,182 @@
+/*
+ * 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.BitSet;
+import java.util.Collection;
+import java.util.Locale;
+import java.util.function.Consumer;
+
+/**
+ * A {@link FragmentChecker} based on all character n-grams possible in a certain language, keeping
+ * them in a relatively memory-efficient, but probabilistic data structure. The n-gram length should
+ * be 2, 3 or 4.
+ *
+ * @see #fromAllSimpleWords for enumerating the whole dictionary automatically
+ * @see #fromWords for creating an instance from a precomputed set of all word forms or n-grams
+ */
+public class NGramFragmentChecker implements FragmentChecker {
+  private final int n;
+  private final BitSet hashes;
+
+  private NGramFragmentChecker(int n, BitSet hashes) {
+    if (n < 2 || n > 4) throw new IllegalArgumentException("N should be between 2 and 4: " + n);
+
+    this.n = n;
+    this.hashes = hashes;
+
+    if (hashes.cardinality() > hashes.size() * 2 / 3) {
+      throw new IllegalArgumentException(
+          "Too many collisions, please report this to dev@lucene.apache.org");
+    }
+  }
+
+  int hashCount() {
+    return hashes.cardinality();
+  }
+
+  @Override
+  public boolean hasImpossibleFragmentAround(CharSequence word, int start, int end) {
+    if (word.length() < n) {
+      return false;
+    }
+    int firstIntersectingStart = Math.max(0, start - n + 1);
+    int lastIntersectingStart = Math.min(end - 1, word.length() - n);
+    for (int i = firstIntersectingStart; i <= lastIntersectingStart; i++) {
+      if (!hashes.get(Math.abs(lowCollisionHash(word, i, i + n) % hashes.size()))) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  private static int lowCollisionHash(CharSequence chars, int offset, int end) {
+    int result = 0;
+    for (int i = offset; i < end; i++) {
+      result = 239 * result + chars.charAt(i);
+    }
+    return result;
+  }
+
+  /**
+   * Iterate the whole dictionary, derive all word forms (using {@link WordFormGenerator}), vary the
+   * case to get all words acceptable by the spellchecker, and create a fragment checker based on
+   * their {@code n}-grams. Note that this enumerates only words derivable by suffixes and prefixes.
+   * If the language has compounds, some n-grams possible via those compounds can be missed. In the
+   * latter case, consider using {@link #fromWords}.
+   *
+   * @param n the length of n-grams
+   * @param dictionary the dictionary to traverse
+   * @param checkCanceled an object that's periodically called, allowing to interrupt the traversal
+   *     by throwing an exception
+   */
+  public static NGramFragmentChecker fromAllSimpleWords(
+      int n, Dictionary dictionary, Runnable checkCanceled) {
+    BitSet hashes = new BitSet(1 << (7 + n * 3)); // some empirical numbers
+    processNGrams(n, dictionary, checkCanceled, collectHashes(hashes));
+    return new NGramFragmentChecker(n, hashes);
+  }
+
+  private static NGramConsumer collectHashes(BitSet hashes) {
+    return (word, start, end) ->
+        hashes.set(Math.abs(lowCollisionHash(word, start, end) % hashes.size()));
+  }
+
+  /**
+   * Create a fragment checker for n-grams found in the given words. The words can be n-grams
+   * themselves or full words of the language. The words are case-sensitive, so be sure to include
+   * upper-case and title-case variants if they're accepted by the spellchecker.
+   *
+   * @param n the length of the ngrams to consider.
+   * @param words the strings to extract n-grams from
+   */
+  public static NGramFragmentChecker fromWords(int n, Collection<? extends CharSequence> words) {
+    BitSet hashes = new BitSet(Integer.highestOneBit(words.size()) * 4);
+    NGramConsumer consumer = collectHashes(hashes);
+    for (CharSequence word : words) {
+      consumer.processNGrams(n, word);
+    }
+    return new NGramFragmentChecker(n, hashes);
+  }
+
+  /**
+   * Traverse the whole dictionary, generate all word forms of its entries, and process all n-grams
+   * in these word forms. No duplication removal is done, so the {@code consumer} should be prepared
+   * to duplicate n-grams. The traversal order is undefined.
+   *
+   * @param n the length of the n-grams
+   * @param dictionary the dictionary to traverse
+   * @param checkCanceled an object that's periodically called, allowing to interrupt the traversal
+   *     by throwing an exception
+   * @param consumer the n-gram consumer to be called for each n-gram
+   */
+  public static void processNGrams(
+      int n, Dictionary dictionary, Runnable checkCanceled, NGramConsumer consumer) {
+    WordFormGenerator gen =
+        new WordFormGenerator(dictionary) {
+          @Override
+          protected boolean canStemToOriginal(AffixedWord derived) {
+            return true; // overgenerate a bit, but avoid very expensive checks
+          }
+        };
+
+    gen.generateAllSimpleWords(
+        new Consumer<>() {
+          DictEntry lastEntry = null;
+          WordCase lastEntryCase = null;
+
+          @Override
+          public void accept(AffixedWord aw) {
+            String word = aw.getWord();
+            consumer.processNGrams(n, word);
+            if (shouldVaryCase(aw.getDictEntry())) {
+              consumer.processNGrams(n, word.toUpperCase(Locale.ROOT));
+              if (word.length() > 1) {
+                String capitalized =
+                    Character.toUpperCase(word.charAt(0))
+                        + word.substring(1, Math.min(n, word.length()));
+                consumer.processNGrams(n, capitalized);
+              }
+            }
+          }
+
+          private boolean shouldVaryCase(DictEntry entry) {
+            if (entry != lastEntry) {
+              lastEntry = entry;
+              lastEntryCase = WordCase.caseOf(entry.getStem());
+            }
+
+            return lastEntryCase != WordCase.MIXED && lastEntryCase != WordCase.NEUTRAL;
+          }
+        },
+        checkCanceled);
+  }
+
+  /** A callback for n-gram ranges in words */
+  public interface NGramConsumer {
+    void processNGram(CharSequence word, int start, int end);
+
+    /** Call {@link #processNGram} for each fragment of the length {@code n} in the given word */
+    default void processNGrams(int n, CharSequence word) {
+      if (word.length() >= n) {
+        for (int i = 0; i <= word.length() - n; i++) {
+          processNGram(word, i, i + n);
+        }
+      }
+    }
+  }
+}
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
index 35c21841bcc..e6da3c90584 100644
--- 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
@@ -40,7 +40,8 @@ import org.apache.lucene.util.CharsRef;
  * <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.
+ *       KEY directives in the .aff file. To speed up this part, consider using {@link
+ *       #withFragmentChecker}.
  *   <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
@@ -51,14 +52,17 @@ import org.apache.lucene.util.CharsRef;
 public class Suggester {
   private final Dictionary dictionary;
   private final SuggestibleEntryCache suggestibleCache;
+  private final FragmentChecker fragmentChecker;
 
   public Suggester(Dictionary dictionary) {
-    this(dictionary, null);
+    this(dictionary, null, FragmentChecker.EVERYTHING_POSSIBLE);
   }
 
-  private Suggester(Dictionary dictionary, SuggestibleEntryCache suggestibleCache) {
+  private Suggester(
+      Dictionary dictionary, SuggestibleEntryCache suggestibleCache, FragmentChecker checker) {
     this.dictionary = dictionary;
     this.suggestibleCache = suggestibleCache;
+    this.fragmentChecker = checker;
   }
 
   /**
@@ -67,7 +71,16 @@ public class Suggester {
    * 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));
+    return new Suggester(
+        dictionary, SuggestibleEntryCache.buildCache(dictionary.words), fragmentChecker);
+  }
+
+  /**
+   * Returns a copy of this suggester instance with {@link FragmentChecker} hint that can improve
+   * the performance of the "Modification" phase performance.
+   */
+  public Suggester withFragmentChecker(FragmentChecker checker) {
+    return new Suggester(dictionary, suggestibleCache, checker);
   }
 
   /**
@@ -161,7 +174,8 @@ public class Suggester {
     }
 
     boolean hasGoodSuggestions =
-        new ModifyingSuggester(suggestionSpeller, suggestions, word, wordCase).suggest();
+        new ModifyingSuggester(suggestionSpeller, suggestions, word, wordCase, fragmentChecker)
+            .suggest();
 
     if (!hasGoodSuggestions && dictionary.maxNGramSuggestions > 0) {
       List<String> generated =
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/WordFormGenerator.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/WordFormGenerator.java
index bc0531edb89..38ef1a9d310 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/WordFormGenerator.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/WordFormGenerator.java
@@ -34,6 +34,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.PriorityQueue;
 import java.util.Set;
+import java.util.function.Consumer;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
@@ -133,9 +134,14 @@ public class WordFormGenerator {
     var encodedFlags = dictionary.flagParsingStrategy.parseUtfFlags(flags);
     if (!shouldConsiderAtAll(encodedFlags)) return List.of();
 
+    return getAllWordForms(DictEntry.create(stem, flags), encodedFlags, checkCanceled);
+  }
+
+  private List<AffixedWord> getAllWordForms(
+      DictEntry entry, char[] encodedFlags, Runnable checkCanceled) {
     encodedFlags = sortAndDeduplicate(encodedFlags);
     List<AffixedWord> result = new ArrayList<>();
-    AffixedWord bare = new AffixedWord(stem, DictEntry.create(stem, flags), List.of(), List.of());
+    AffixedWord bare = new AffixedWord(entry.getStem(), entry, List.of(), List.of());
     checkCanceled.run();
     if (!FlagEnumerator.hasFlagInSortedArray(
         dictionary.needaffix, encodedFlags, 0, encodedFlags.length)) {
@@ -163,6 +169,12 @@ public class WordFormGenerator {
     return toSortedCharArray(set);
   }
 
+  /**
+   * A sanity-check that the word form generated by affixation in {@link #getAllWordForms(String,
+   * String, Runnable)} is indeed accepted by the spell-checker and analyzed to be the form of the
+   * original dictionary entry. This can be overridden for cases where such check is unnecessary or
+   * can be done more efficiently.
+   */
   protected boolean canStemToOriginal(AffixedWord derived) {
     String word = derived.getWord();
     char[] chars = word.toCharArray();
@@ -273,6 +285,37 @@ public class WordFormGenerator {
     return appendId == 0 ? new char[0] : dictionary.flagLookup.getFlags(appendId);
   }
 
+  /**
+   * Traverse the whole dictionary and derive all word forms via affixation (as in {@link
+   * #getAllWordForms(String, String, Runnable)}) for each of the entries. The iteration order is
+   * undefined. Only "simple" words are returned, no compounding flags are processed. Upper- and
+   * title-case variations are not returned, even if the spellchecker accepts them.
+   *
+   * @param consumer the object that receives each derived word form
+   * @param checkCanceled an object that's periodically called, allowing to interrupt the traversal
+   *     and generation by throwing an exception
+   */
+  public void generateAllSimpleWords(Consumer<AffixedWord> consumer, Runnable checkCanceled) {
+    dictionary.words.processAllWords(
+        1,
+        Integer.MAX_VALUE,
+        false,
+        (root, lazyForms) -> {
+          String rootStr = root.toString();
+          IntsRef forms = lazyForms.get();
+          for (int i = 0; i < forms.length; i += dictionary.formStep()) {
+            char[] encodedFlags = dictionary.flagLookup.getFlags(forms.ints[forms.offset + i]);
+            if (shouldConsiderAtAll(encodedFlags)) {
+              String presentableFlags = dictionary.flagParsingStrategy.printFlags(encodedFlags);
+              DictEntry entry = DictEntry.create(rootStr, presentableFlags);
+              for (AffixedWord aw : getAllWordForms(entry, encodedFlags, checkCanceled)) {
+                consumer.accept(aw);
+              }
+            }
+          }
+        });
+  }
+
   /**
    * Given a list of words, try to produce a smaller set of dictionary entries (with some flags)
    * that would generate these words. This is equivalent to "munch" from the "hunspell-tools"
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 84d65da8ad8..cc39c5f4185 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
@@ -159,6 +159,14 @@ class WordStorage {
    */
   void processSuggestibleWords(
       int minLength, int maxLength, BiConsumer<CharsRef, Supplier<IntsRef>> processor) {
+    processAllWords(minLength, maxLength, true, processor);
+  }
+
+  void processAllWords(
+      int minLength,
+      int maxLength,
+      boolean suggestibleOnly,
+      BiConsumer<CharsRef, Supplier<IntsRef>> processor) {
     assert minLength <= maxLength;
     maxLength = Math.min(maxEntryLength, maxLength);
 
@@ -178,7 +186,8 @@ class WordStorage {
 
         boolean last = !hasCollision(mask);
         boolean mightMatch =
-            hasSuggestibleEntries(mask) && hasLengthInRange(mask, minLength, maxLength);
+            (!suggestibleOnly || hasSuggestibleEntries(mask))
+                && hasLengthInRange(mask, minLength, maxLength);
 
         if (!last) {
           mask = in.readByte();
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 3add18ccfa7..8344f8dd968 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
@@ -66,7 +66,7 @@ public class TestPerformance extends LuceneTestCase {
 
   @Test
   public void en_suggest() throws Exception {
-    checkSuggestionPerformance("en", 3_000);
+    checkSuggestionPerformance("en", 3_500);
   }
 
   @Test
@@ -86,7 +86,7 @@ public class TestPerformance extends LuceneTestCase {
 
   @Test
   public void de_suggest() throws Exception {
-    checkSuggestionPerformance("de", 150);
+    checkSuggestionPerformance("de", 250);
   }
 
   @Test
@@ -96,13 +96,15 @@ public class TestPerformance extends LuceneTestCase {
 
   @Test
   public void fr_suggest() throws Exception {
-    checkSuggestionPerformance("fr", 120);
+    checkSuggestionPerformance("fr", 1_000);
   }
 
   private Dictionary loadDictionary(String code) throws IOException, ParseException {
+    long start = System.nanoTime();
     Path aff = findAffFile(code);
     Dictionary dictionary = TestAllDictionaries.loadDictionary(aff);
-    System.out.println("Loaded " + aff);
+    System.out.println(
+        "Loaded " + aff + " in " + TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start) + "ms");
     return dictionary;
   }
 
@@ -163,12 +165,15 @@ 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, () -> {});
+    Runnable checkCanceled = () -> {};
+    Suggester base = new Suggester(dictionary);
+    Suggester optimized =
+        base.withSuggestibleEntryCache().withFragmentChecker(fragmentChecker(dictionary, code));
+    Hunspell speller = new Hunspell(dictionary, TimeoutPolicy.THROW_EXCEPTION, checkCanceled);
     List<String> words =
         loadWords(code, wordCount, dictionary).stream()
             .distinct()
-            .filter(w -> hasQuickSuggestions(speller, suggester, w))
+            .filter(w -> hasQuickSuggestions(speller, base, optimized, w))
             .collect(Collectors.toList());
     System.out.println("Checking " + words.size() + " misspelled words");
 
@@ -177,25 +182,49 @@ public class TestPerformance extends LuceneTestCase {
         words.size(),
         blackHole -> {
           for (String word : words) {
-            blackHole.accept(suggester.suggestNoTimeout(word, () -> {}));
+            blackHole.accept(optimized.suggestNoTimeout(word, checkCanceled));
           }
         });
     System.out.println();
   }
 
-  private boolean hasQuickSuggestions(Hunspell speller, Suggester suggester, String word) {
+  private static FragmentChecker fragmentChecker(Dictionary dictionary, String langCode) {
+    long started = System.nanoTime();
+    var trigram = NGramFragmentChecker.fromAllSimpleWords(3, dictionary, () -> {});
+    long elapsed = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - started);
+    System.out.println("Populated " + trigram.hashCount() + " trigram hashes in " + elapsed + "ms");
+    if ("de".equals(langCode)) {
+      return (word, start, end) ->
+          word.charAt(0) != '-'
+              && word.charAt(word.length() - 1) != '-'
+              && trigram.hasImpossibleFragmentAround(word, start, end);
+    }
+    return trigram;
+  }
+
+  private boolean hasQuickSuggestions(
+      Hunspell speller, Suggester base, Suggester optimized, String word) {
     if (speller.spell(word)) {
       return false;
     }
 
+    List<String> fromOptimized;
     try {
-      suggester.suggestWithTimeout(word, Hunspell.SUGGEST_TIME_LIMIT, () -> {});
+      fromOptimized = optimized.suggestWithTimeout(word, Hunspell.SUGGEST_TIME_LIMIT, () -> {});
     } catch (
         @SuppressWarnings("unused")
         SuggestionTimeoutException e) {
       System.out.println("Timeout happened for " + word + ", skipping");
       return false;
     }
+
+    List<String> fromBase = base.suggestNoTimeout(word, () -> {});
+    if (!fromBase.equals(fromOptimized)) {
+      fail(
+          "Optimization breaks suggestions: "
+              + ("for '" + word + "', base=" + fromBase + ", optimized=" + fromOptimized));
+    }
+
     return true;
   }
 
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 b5f39c4214f..ba8420b5806 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
@@ -16,14 +16,18 @@
  */
 package org.apache.lucene.analysis.hunspell;
 
+import static org.apache.lucene.analysis.hunspell.Dictionary.FLAG_UNSET;
+
 import java.io.IOException;
 import java.io.InputStream;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.text.ParseException;
 import java.util.HashSet;
+import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Locale;
+import java.util.Map;
 import java.util.Set;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
@@ -249,14 +253,22 @@ public class TestSpellChecking extends LuceneTestCase {
     InputStream dictStream = Files.newInputStream(dicFile);
 
     Hunspell speller;
-    Suggester defaultSuggester;
-    Suggester cachingSuggester;
+    Map<String, Suggester> suggesters = new LinkedHashMap<>();
     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();
+      Suggester suggester = new Suggester(dictionary);
+      suggesters.put("default", suggester);
+      suggesters.put("caching", suggester.withSuggestibleEntryCache());
+      if (dictionary.compoundRules == null
+          && dictionary.compoundBegin == FLAG_UNSET
+          && dictionary.compoundFlag == FLAG_UNSET) {
+        for (int n = 2; n <= 4; n++) {
+          var checker = NGramFragmentChecker.fromAllSimpleWords(n, dictionary, () -> {});
+          suggesters.put("ngram" + n, suggester.withFragmentChecker(checker));
+        }
+      }
     } finally {
       IOUtils.closeWhileHandlingException(affixStream);
       IOUtils.closeWhileHandlingException(dictStream);
@@ -277,8 +289,10 @@ public class TestSpellChecking extends LuceneTestCase {
         assertFalse("Unexpectedly considered correct: " + word, speller.spell(word.trim()));
       }
       if (Files.exists(sug)) {
-        assertEquals(Files.readString(sug).trim(), suggest(defaultSuggester, wrongWords));
-        assertEquals(Files.readString(sug).trim(), suggest(cachingSuggester, wrongWords));
+        String sugLines = Files.readString(sug).trim();
+        for (Map.Entry<String, Suggester> e : suggesters.entrySet()) {
+          assertEquals("Suggester=" + e.getKey(), sugLines, suggest(e.getValue(), wrongWords));
+        }
       }
     } else {
       assertFalse(".sug file without .wrong file!", Files.exists(sug));