You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by dw...@apache.org on 2021/02/08 09:56:25 UTC

[lucene-solr] branch master updated: LUCENE-9735: Hunspell: speed up flag checks by avoiding allocations (#2315)

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

dweiss pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/master by this push:
     new 061233c  LUCENE-9735: Hunspell: speed up flag checks by avoiding allocations (#2315)
061233c is described below

commit 061233ca4ee5baad0d2c531297c51fbc665c2844
Author: Peter Gromov <pe...@jetbrains.com>
AuthorDate: Mon Feb 8 10:56:10 2021 +0100

    LUCENE-9735: Hunspell: speed up flag checks by avoiding allocations (#2315)
---
 .../analysis/hunspell/CheckCompoundPattern.java    |   4 +-
 .../lucene/analysis/hunspell/CompoundRule.java     |  24 +++--
 .../lucene/analysis/hunspell/Dictionary.java       | 103 +++++++--------------
 .../lucene/analysis/hunspell/FlagEnumerator.java   |  86 +++++++++++++++++
 .../lucene/analysis/hunspell/SpellChecker.java     |  15 ++-
 .../apache/lucene/analysis/hunspell/Stemmer.java   |  54 +++++------
 .../lucene/analysis/hunspell/TestDictionary.java   |  25 +++--
 .../lucene/analysis/hunspell/TestPerformance.java  |   4 +-
 8 files changed, 178 insertions(+), 137 deletions(-)

diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/CheckCompoundPattern.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/CheckCompoundPattern.java
index 3d70591..b1c4b3d 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/CheckCompoundPattern.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/CheckCompoundPattern.java
@@ -16,7 +16,6 @@
  */
 package org.apache.lucene.analysis.hunspell;
 
-import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
 import org.apache.lucene.util.IntsRef;
 
@@ -27,7 +26,6 @@ class CheckCompoundPattern {
   private final char[] endFlags;
   private final char[] beginFlags;
   private final Dictionary dictionary;
-  private final BytesRef scratch = new BytesRef();
 
   CheckCompoundPattern(
       String unparsed, Dictionary.FlagParsingStrategy strategy, Dictionary dictionary) {
@@ -93,7 +91,7 @@ class CheckCompoundPattern {
 
   private boolean hasAllFlags(char[] flags, IntsRef forms) {
     for (char flag : flags) {
-      if (!dictionary.hasFlag(forms, flag, scratch)) {
+      if (!dictionary.hasFlag(forms, flag)) {
         return false;
       }
     }
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/CompoundRule.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/CompoundRule.java
index 0f89de8..726c1dc 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/CompoundRule.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/CompoundRule.java
@@ -17,7 +17,6 @@
 package org.apache.lucene.analysis.hunspell;
 
 import java.util.List;
-import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IntsRef;
 
 class CompoundRule {
@@ -51,16 +50,15 @@ class CompoundRule {
     data = parsedFlags.toString().toCharArray();
   }
 
-  boolean mayMatch(List<IntsRef> words, BytesRef scratch) {
-    return match(words, 0, 0, scratch, false);
+  boolean mayMatch(List<IntsRef> words) {
+    return match(words, 0, 0, false);
   }
 
-  boolean fullyMatches(List<IntsRef> words, BytesRef scratch) {
-    return match(words, 0, 0, scratch, true);
+  boolean fullyMatches(List<IntsRef> words) {
+    return match(words, 0, 0, true);
   }
 
-  private boolean match(
-      List<IntsRef> words, int patternIndex, int wordIndex, BytesRef scratch, boolean fully) {
+  private boolean match(List<IntsRef> words, int patternIndex, int wordIndex, boolean fully) {
     if (patternIndex >= data.length) {
       return wordIndex >= words.size();
     }
@@ -71,12 +69,12 @@ class CompoundRule {
     char flag = data[patternIndex];
     if (patternIndex < data.length - 1 && data[patternIndex + 1] == '*') {
       int startWI = wordIndex;
-      while (wordIndex < words.size() && dictionary.hasFlag(words.get(wordIndex), flag, scratch)) {
+      while (wordIndex < words.size() && dictionary.hasFlag(words.get(wordIndex), flag)) {
         wordIndex++;
       }
 
       while (wordIndex >= startWI) {
-        if (match(words, patternIndex + 2, wordIndex, scratch, fully)) {
+        if (match(words, patternIndex + 2, wordIndex, fully)) {
           return true;
         }
 
@@ -86,16 +84,16 @@ class CompoundRule {
     }
 
     boolean currentWordMatches =
-        wordIndex < words.size() && dictionary.hasFlag(words.get(wordIndex), flag, scratch);
+        wordIndex < words.size() && dictionary.hasFlag(words.get(wordIndex), flag);
 
     if (patternIndex < data.length - 1 && data[patternIndex + 1] == '?') {
-      if (currentWordMatches && match(words, patternIndex + 2, wordIndex + 1, scratch, fully)) {
+      if (currentWordMatches && match(words, patternIndex + 2, wordIndex + 1, fully)) {
         return true;
       }
-      return match(words, patternIndex + 2, wordIndex, scratch, fully);
+      return match(words, patternIndex + 2, wordIndex, fully);
     }
 
-    return currentWordMatches && match(words, patternIndex + 1, wordIndex + 1, scratch, fully);
+    return currentWordMatches && match(words, patternIndex + 1, wordIndex + 1, fully);
   }
 
   @Override
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Dictionary.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Dictionary.java
index ae1a3a1..95a4b83 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Dictionary.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Dictionary.java
@@ -52,8 +52,6 @@ import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefBuilder;
-import org.apache.lucene.util.BytesRefHash;
 import org.apache.lucene.util.CharsRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.IntsRef;
@@ -77,11 +75,11 @@ public class Dictionary {
 
   static final char FLAG_UNSET = (char) 0;
   private static final int DEFAULT_FLAGS = 65510;
-  private static final char HIDDEN_FLAG = (char) 65511; // called 'ONLYUPCASEFLAG' in Hunspell
+  static final char HIDDEN_FLAG = (char) 65511; // called 'ONLYUPCASEFLAG' in Hunspell
 
   // TODO: really for suffixes we should reverse the automaton and run them backwards
-  private static final String PREFIX_CONDITION_REGEX_PATTERN = "%s.*";
-  private static final String SUFFIX_CONDITION_REGEX_PATTERN = ".*%s";
+  private static final String PREFIX_CONDITION_REGEX = "%s.*";
+  private static final String SUFFIX_CONDITION_REGEX = ".*%s";
   private static final Pattern MORPH_KEY_PATTERN = Pattern.compile("\\s+(?=\\p{Alpha}{2}:)");
   static final Charset DEFAULT_CHARSET = StandardCharsets.ISO_8859_1;
   CharsetDecoder decoder = replacingDecoder(DEFAULT_CHARSET);
@@ -106,7 +104,7 @@ public class Dictionary {
    * The list of unique flagsets (wordforms). theoretically huge, but practically small (for Polish
    * this is 756), otherwise humans wouldn't be able to deal with it either.
    */
-  BytesRefHash flagLookup = new BytesRefHash();
+  final FlagEnumerator.Lookup flagLookup;
 
   // the list of unique strip affixes.
   char[] stripData;
@@ -224,7 +222,6 @@ public class Dictionary {
     this.ignoreCase = ignoreCase;
     this.needsInputCleaning = ignoreCase;
     this.needsOutputCleaning = false; // set if we have an OCONV
-    flagLookup.add(new BytesRef()); // no flags -> ord 0
 
     Path tempPath = getDefaultTempDir(); // TODO: make this configurable?
     Path aff = Files.createTempFile(tempPath, "affix", "aff");
@@ -244,12 +241,14 @@ public class Dictionary {
 
       // pass 2: parse affixes
       aff2 = new BufferedInputStream(Files.newInputStream(aff));
-      readAffixFile(aff2, decoder);
+      FlagEnumerator flagEnumerator = new FlagEnumerator();
+      readAffixFile(aff2, decoder, flagEnumerator);
 
       // read dictionary entries
       IndexOutput unsorted = mergeDictionaries(tempDir, tempFileNamePrefix, dictionaries, decoder);
       String sortedFile = sortWordsOffline(tempDir, tempFileNamePrefix, unsorted);
-      words = readSortedDictionaries(tempDir, sortedFile);
+      words = readSortedDictionaries(tempDir, sortedFile, flagEnumerator);
+      flagLookup = flagEnumerator.finish();
       aliases = null; // no longer needed
       morphAliases = null; // no longer needed
       success = true;
@@ -321,7 +320,7 @@ public class Dictionary {
    * @param decoder CharsetDecoder to decode the content of the file
    * @throws IOException Can be thrown while reading from the InputStream
    */
-  private void readAffixFile(InputStream affixStream, CharsetDecoder decoder)
+  private void readAffixFile(InputStream affixStream, CharsetDecoder decoder, FlagEnumerator flags)
       throws IOException, ParseException {
     TreeMap<String, List<Integer>> prefixes = new TreeMap<>();
     TreeMap<String, List<Integer>> suffixes = new TreeMap<>();
@@ -351,11 +350,9 @@ public class Dictionary {
       } else if ("AM".equals(firstWord)) {
         parseMorphAlias(line);
       } else if ("PFX".equals(firstWord)) {
-        parseAffix(
-            prefixes, line, reader, PREFIX_CONDITION_REGEX_PATTERN, seenPatterns, seenStrips);
+        parseAffix(prefixes, line, reader, PREFIX_CONDITION_REGEX, seenPatterns, seenStrips, flags);
       } else if ("SFX".equals(firstWord)) {
-        parseAffix(
-            suffixes, line, reader, SUFFIX_CONDITION_REGEX_PATTERN, seenPatterns, seenStrips);
+        parseAffix(suffixes, line, reader, SUFFIX_CONDITION_REGEX, seenPatterns, seenStrips, flags);
       } else if (line.equals("COMPLEXPREFIXES")) {
         complexPrefixes =
             true; // 2-stage prefix+1-stage suffix instead of 2-stage suffix+1-stage prefix
@@ -583,15 +580,15 @@ public class Dictionary {
       LineNumberReader reader,
       String conditionPattern,
       Map<String, Integer> seenPatterns,
-      Map<String, Integer> seenStrips)
+      Map<String, Integer> seenStrips,
+      FlagEnumerator flags)
       throws IOException, ParseException {
 
-    BytesRefBuilder scratch = new BytesRefBuilder();
     StringBuilder sb = new StringBuilder();
     String[] args = header.split("\\s+");
 
     boolean crossProduct = args[2].equals("Y");
-    boolean isSuffix = conditionPattern.equals(SUFFIX_CONDITION_REGEX_PATTERN);
+    boolean isSuffix = conditionPattern.equals(SUFFIX_CONDITION_REGEX);
 
     int numLines = Integer.parseInt(args[3]);
     affixData = ArrayUtil.grow(affixData, currentAffix * 4 + numLines * 4);
@@ -617,7 +614,6 @@ public class Dictionary {
         }
 
         appendFlags = flagParsingStrategy.parseFlags(flagPart);
-        Arrays.sort(appendFlags);
         twoStageAffix = true;
       }
       // zero affix -> empty string
@@ -676,8 +672,7 @@ public class Dictionary {
         appendFlags = NOFLAGS;
       }
 
-      encodeFlags(scratch, appendFlags);
-      int appendFlagsOrd = flagLookup.add(scratch.get());
+      int appendFlagsOrd = flags.add(appendFlags);
       if (appendFlagsOrd < 0) {
         // already exists in our hash
         appendFlagsOrd = (-appendFlagsOrd) - 1;
@@ -1064,10 +1059,11 @@ public class Dictionary {
     return sorted;
   }
 
-  private FST<IntsRef> readSortedDictionaries(Directory tempDir, String sorted) throws IOException {
+  private FST<IntsRef> readSortedDictionaries(
+      Directory tempDir, String sorted, FlagEnumerator flags) throws IOException {
     boolean success = false;
 
-    EntryGrouper grouper = new EntryGrouper();
+    EntryGrouper grouper = new EntryGrouper(flags);
 
     try (ByteSequencesReader reader =
         new ByteSequencesReader(tempDir.openChecksumInput(sorted, IOContext.READONCE), sorted)) {
@@ -1104,7 +1100,6 @@ public class Dictionary {
             wordForm = ArrayUtil.growExact(wordForm, wordForm.length + 1);
             wordForm[wordForm.length - 1] = HIDDEN_FLAG;
           }
-          Arrays.sort(wordForm);
           entry = line.substring(0, flagSep);
         }
         // we possibly have morphological data
@@ -1191,9 +1186,13 @@ public class Dictionary {
         new FSTCompiler<>(FST.INPUT_TYPE.BYTE4, IntSequenceOutputs.getSingleton());
     private final List<char[]> group = new ArrayList<>();
     private final List<Integer> stemExceptionIDs = new ArrayList<>();
-    private final BytesRefBuilder flagsScratch = new BytesRefBuilder();
     private final IntsRefBuilder scratchInts = new IntsRefBuilder();
     private String currentEntry = null;
+    private final FlagEnumerator flagEnumerator;
+
+    EntryGrouper(FlagEnumerator flagEnumerator) {
+      this.flagEnumerator = flagEnumerator;
+    }
 
     void add(String entry, char[] flags, int stemExceptionID) throws IOException {
       if (!entry.equals(currentEntry)) {
@@ -1229,12 +1228,7 @@ public class Dictionary {
           continue;
         }
 
-        encodeFlags(flagsScratch, flags);
-        int ord = flagLookup.add(flagsScratch.get());
-        if (ord < 0) {
-          ord = -ord - 1; // already exists in our hash
-        }
-        currentOrds.append(ord);
+        currentOrds.append(flagEnumerator.add(flags));
         if (hasStemExceptions) {
           currentOrds.append(stemExceptionIDs.get(i));
         }
@@ -1248,34 +1242,13 @@ public class Dictionary {
     }
   }
 
-  static boolean hasHiddenFlag(char[] flags) {
-    return hasFlag(flags, HIDDEN_FLAG);
-  }
-
-  char[] decodeFlags(int entryId, BytesRef b) {
-    this.flagLookup.get(entryId, b);
-
-    if (b.length == 0) {
-      return CharsRef.EMPTY_CHARS;
-    }
-    int len = b.length >>> 1;
-    char[] flags = new char[len];
-    int upto = 0;
-    int end = b.offset + b.length;
-    for (int i = b.offset; i < end; i += 2) {
-      flags[upto++] = (char) ((b.bytes[i] << 8) | (b.bytes[i + 1] & 0xff));
-    }
-    return flags;
-  }
-
-  private static void encodeFlags(BytesRefBuilder b, char[] flags) {
-    int len = flags.length << 1;
-    b.grow(len);
-    b.clear();
-    for (int flag : flags) {
-      b.append((byte) ((flag >> 8) & 0xff));
-      b.append((byte) (flag & 0xff));
+  private static boolean hasHiddenFlag(char[] flags) {
+    for (char flag : flags) {
+      if (flag == HIDDEN_FLAG) {
+        return true;
+      }
     }
+    return false;
   }
 
   private void parseAlias(String line) {
@@ -1341,18 +1314,18 @@ public class Dictionary {
         .collect(Collectors.toList());
   }
 
-  boolean isForbiddenWord(char[] word, int length, BytesRef scratch) {
+  boolean isForbiddenWord(char[] word, int length) {
     if (forbiddenword != FLAG_UNSET) {
       IntsRef forms = lookupWord(word, 0, length);
-      return forms != null && hasFlag(forms, forbiddenword, scratch);
+      return forms != null && hasFlag(forms, forbiddenword);
     }
     return false;
   }
 
-  boolean hasFlag(IntsRef forms, char flag, BytesRef scratch) {
+  boolean hasFlag(IntsRef forms, char flag) {
     int formStep = formStep();
     for (int i = 0; i < forms.length; i += formStep) {
-      if (hasFlag(forms.ints[forms.offset + i], flag, scratch)) {
+      if (hasFlag(forms.ints[forms.offset + i], flag)) {
         return true;
       }
     }
@@ -1468,12 +1441,8 @@ public class Dictionary {
     }
   }
 
-  boolean hasFlag(int entryId, char flag, BytesRef scratch) {
-    return flag != FLAG_UNSET && hasFlag(decodeFlags(entryId, scratch), flag);
-  }
-
-  static boolean hasFlag(char[] flags, char flag) {
-    return flag != FLAG_UNSET && Arrays.binarySearch(flags, flag) >= 0;
+  boolean hasFlag(int entryId, char flag) {
+    return flagLookup.hasFlag(entryId, flag);
   }
 
   CharSequence cleanInput(CharSequence input, StringBuilder reuse) {
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/FlagEnumerator.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/FlagEnumerator.java
new file mode 100644
index 0000000..57aac40
--- /dev/null
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/FlagEnumerator.java
@@ -0,0 +1,86 @@
+/*
+ * 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.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.lucene.util.ArrayUtil;
+
+/**
+ * A structure similar to {@link org.apache.lucene.util.BytesRefHash}, but specialized for sorted
+ * char sequences used for Hunspell flags. It deduplicates flag sequences, gives them unique ids,
+ * stores the sequences in a contiguous char[] (via {@link #finish()} and allows to query presence
+ * of the flags later via {@link Lookup#hasFlag}.
+ */
+class FlagEnumerator {
+  private final StringBuilder builder = new StringBuilder();
+  private final Map<String, Integer> indices = new HashMap<>();
+
+  FlagEnumerator() {
+    add(new char[0]); // no flags -> ord 0
+  }
+
+  int add(char[] chars) {
+    Arrays.sort(chars);
+    String key = new String(chars);
+    if (key.length() > Character.MAX_VALUE) {
+      throw new IllegalArgumentException("Too many flags: " + key);
+    }
+
+    Integer existing = indices.get(key);
+    if (existing != null) {
+      return existing;
+    }
+
+    int result = builder.length();
+    indices.put(key, result);
+    builder.append((char) key.length());
+    builder.append(key);
+    return result;
+  }
+
+  Lookup finish() {
+    char[] result = new char[builder.length()];
+    builder.getChars(0, builder.length(), result, 0);
+    return new Lookup(result);
+  }
+
+  static class Lookup {
+    private final char[] data;
+
+    private Lookup(char[] data) {
+      this.data = data;
+    }
+
+    boolean hasFlag(int entryId, char flag) {
+      if (entryId < 0 || flag == Dictionary.FLAG_UNSET) return false;
+
+      int length = data[entryId];
+      for (int i = entryId + 1; i < entryId + 1 + length; i++) {
+        char c = data[i];
+        if (c == flag) return true;
+        if (c > flag) return false;
+      }
+      return false;
+    }
+
+    char[] getFlags(int entryId) {
+      return ArrayUtil.copyOfSubArray(data, entryId + 1, entryId + 1 + data[entryId]);
+    }
+  }
+}
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/SpellChecker.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/SpellChecker.java
index 53bf53e..e694f4f 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/SpellChecker.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/SpellChecker.java
@@ -26,7 +26,6 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Set;
-import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
 import org.apache.lucene.util.IntsRef;
 
@@ -38,7 +37,6 @@ import org.apache.lucene.util.IntsRef;
 public class SpellChecker {
   final Dictionary dictionary;
   final Stemmer stemmer;
-  private final BytesRef scratch = new BytesRef();
 
   public SpellChecker(Dictionary dictionary) {
     this.dictionary = dictionary;
@@ -66,7 +64,7 @@ public class SpellChecker {
     }
 
     char[] wordChars = word.toCharArray();
-    if (dictionary.isForbiddenWord(wordChars, wordChars.length, scratch)) {
+    if (dictionary.isForbiddenWord(wordChars, wordChars.length)) {
       return false;
     }
 
@@ -135,7 +133,7 @@ public class SpellChecker {
   }
 
   Boolean checkSimpleWord(char[] wordChars, int length, WordCase originalCase) {
-    if (dictionary.isForbiddenWord(wordChars, length, scratch)) {
+    if (dictionary.isForbiddenWord(wordChars, length)) {
       return false;
     }
 
@@ -253,7 +251,7 @@ public class SpellChecker {
     if (originalCase == WordCase.TITLE || originalCase == WordCase.UPPER) return false;
 
     IntsRef forms = dictionary.lookupWord(chars, offset, length);
-    return forms != null && dictionary.hasFlag(forms, dictionary.forceUCase, scratch);
+    return forms != null && dictionary.hasFlag(forms, dictionary.forceUCase);
   }
 
   private boolean equalsIgnoreCase(CharsRef cr1, CharsRef cr2) {
@@ -341,7 +339,7 @@ public class SpellChecker {
         words.add(forms);
 
         if (dictionary.compoundRules != null
-            && dictionary.compoundRules.stream().anyMatch(r -> r.mayMatch(words, scratch))) {
+            && dictionary.compoundRules.stream().anyMatch(r -> r.mayMatch(words))) {
           if (checkLastCompoundPart(wordChars, offset + breakPos, length - breakPos, words)) {
             return true;
           }
@@ -364,8 +362,7 @@ public class SpellChecker {
     if (forms == null) return false;
 
     words.add(forms);
-    boolean result =
-        dictionary.compoundRules.stream().anyMatch(r -> r.fullyMatches(words, scratch));
+    boolean result = dictionary.compoundRules.stream().anyMatch(r -> r.fullyMatches(words));
     words.remove(words.size() - 1);
     return result;
   }
@@ -474,7 +471,7 @@ public class SpellChecker {
         if (!spell(chunk)) {
           for (String chunkSug : suggest(chunk)) {
             String replaced = word.substring(0, chunkStart) + chunkSug + word.substring(chunkEnd);
-            if (!dictionary.isForbiddenWord(replaced.toCharArray(), replaced.length(), scratch)) {
+            if (!dictionary.isForbiddenWord(replaced.toCharArray(), replaced.length())) {
               result.add(replaced);
             }
           }
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Stemmer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Stemmer.java
index 44e2675..9c5afc7 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Stemmer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Stemmer.java
@@ -24,7 +24,6 @@ import java.util.stream.Collectors;
 import java.util.stream.Stream;
 import org.apache.lucene.analysis.CharArraySet;
 import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
 import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.automaton.CharacterRunAutomaton;
@@ -37,7 +36,6 @@ import org.apache.lucene.util.fst.FST;
  */
 final class Stemmer {
   private final Dictionary dictionary;
-  private final BytesRef scratch = new BytesRef();
   private final StringBuilder segment = new StringBuilder();
 
   // used for normalization
@@ -96,7 +94,7 @@ final class Stemmer {
       word = scratchBuffer;
     }
 
-    if (dictionary.isForbiddenWord(word, length, scratch)) {
+    if (dictionary.isForbiddenWord(word, length)) {
       return Collections.emptyList();
     }
 
@@ -251,25 +249,25 @@ final class Stemmer {
     IntsRef forms = dictionary.lookupWord(word, offset, length);
     if (forms != null) {
       for (int i = 0; i < forms.length; i += formStep) {
-        char[] wordFlags = dictionary.decodeFlags(forms.ints[forms.offset + i], scratch);
-        if (!acceptCase(originalCase, wordFlags, word, offset, length)) {
+        int entryId = forms.ints[forms.offset + i];
+        if (!acceptCase(originalCase, entryId, word, offset, length)) {
           continue;
         }
         // we can't add this form, it's a pseudostem requiring an affix
-        if (Dictionary.hasFlag(wordFlags, dictionary.needaffix)) {
+        if (dictionary.hasFlag(entryId, dictionary.needaffix)) {
           continue;
         }
         // we can't add this form, it only belongs inside a compound word
-        if (!context.isCompound() && Dictionary.hasFlag(wordFlags, dictionary.onlyincompound)) {
+        if (!context.isCompound() && dictionary.hasFlag(entryId, dictionary.onlyincompound)) {
           continue;
         }
         if (context.isCompound()) {
           if (context != WordContext.COMPOUND_END
-              && Dictionary.hasFlag(wordFlags, dictionary.compoundForbid)) {
+              && dictionary.hasFlag(entryId, dictionary.compoundForbid)) {
             return false;
           }
-          if (!Dictionary.hasFlag(wordFlags, dictionary.compoundFlag)
-              && !Dictionary.hasFlag(wordFlags, context.requiredFlag(dictionary))) {
+          if (!dictionary.hasFlag(entryId, dictionary.compoundFlag)
+              && !dictionary.hasFlag(entryId, context.requiredFlag(dictionary))) {
             continue;
           }
         }
@@ -300,8 +298,8 @@ final class Stemmer {
   }
 
   private boolean acceptCase(
-      WordCase originalCase, char[] wordFlags, char[] word, int offset, int length) {
-    boolean keepCase = Dictionary.hasFlag(wordFlags, dictionary.keepcase);
+      WordCase originalCase, int entryId, char[] word, int offset, int length) {
+    boolean keepCase = dictionary.hasFlag(entryId, dictionary.keepcase);
     if (originalCase != null) {
       if (keepCase
           && dictionary.checkSharpS
@@ -311,7 +309,7 @@ final class Stemmer {
       }
       return !keepCase;
     }
-    return !Dictionary.hasHiddenFlag(wordFlags);
+    return !dictionary.hasFlag(entryId, Dictionary.HIDDEN_FLAG);
   }
 
   private boolean containsSharpS(char[] word, int offset, int length) {
@@ -593,32 +591,30 @@ final class Stemmer {
     int append = dictionary.affixData(affix, Dictionary.AFFIX_APPEND);
 
     if (context.isCompound()) {
-      if (!isPrefix && dictionary.hasFlag(append, dictionary.compoundForbid, scratch)) {
+      if (!isPrefix && dictionary.hasFlag(append, dictionary.compoundForbid)) {
         return false;
       }
       WordContext allowed = isPrefix ? WordContext.COMPOUND_BEGIN : WordContext.COMPOUND_END;
-      if (context != allowed && !dictionary.hasFlag(append, dictionary.compoundPermit, scratch)) {
+      if (context != allowed && !dictionary.hasFlag(append, dictionary.compoundPermit)) {
         return false;
       }
       if (context == WordContext.COMPOUND_END
           && !isPrefix
           && !previousWasPrefix
-          && dictionary.hasFlag(append, dictionary.onlyincompound, scratch)) {
+          && dictionary.hasFlag(append, dictionary.onlyincompound)) {
         return false;
       }
     }
 
     if (recursionDepth == 0) {
       // check if affix is allowed in a non-compound word
-      return context.isCompound()
-          || !dictionary.hasFlag(append, dictionary.onlyincompound, scratch);
+      return context.isCompound() || !dictionary.hasFlag(append, dictionary.onlyincompound);
     }
 
     if (isCrossProduct(affix)) {
       // cross check incoming continuation class (flag of previous affix) against list.
-      char[] appendFlags = dictionary.decodeFlags(append, scratch);
-      if (context.isCompound() || !Dictionary.hasFlag(appendFlags, dictionary.onlyincompound)) {
-        return previousWasPrefix || Dictionary.hasFlag(appendFlags, prevFlag);
+      if (context.isCompound() || !dictionary.hasFlag(append, dictionary.onlyincompound)) {
+        return previousWasPrefix || dictionary.hasFlag(append, prevFlag);
       }
     }
 
@@ -686,15 +682,15 @@ final class Stemmer {
     IntsRef forms = skipLookup ? null : dictionary.lookupWord(strippedWord, offset, length);
     if (forms != null) {
       for (int i = 0; i < forms.length; i += formStep) {
-        char[] wordFlags = dictionary.decodeFlags(forms.ints[forms.offset + i], scratch);
-        if (Dictionary.hasFlag(wordFlags, flag) || isFlagAppendedByAffix(prefixId, flag)) {
+        int entryId = forms.ints[forms.offset + i];
+        if (dictionary.hasFlag(entryId, flag) || isFlagAppendedByAffix(prefixId, flag)) {
           // confusing: in this one exception, we already chained the first prefix against the
           // second,
           // so it doesnt need to be checked against the word
           boolean chainedPrefix = dictionary.complexPrefixes && recursionDepth == 1 && prefix;
           if (!chainedPrefix && prefixId >= 0) {
             char prefixFlag = dictionary.affixData(prefixId, Dictionary.AFFIX_FLAG);
-            if (!Dictionary.hasFlag(wordFlags, prefixFlag)
+            if (!dictionary.hasFlag(entryId, prefixFlag)
                 && !isFlagAppendedByAffix(affix, prefixFlag)) {
               continue;
             }
@@ -710,17 +706,17 @@ final class Stemmer {
           }
 
           // we are looking for a case variant, but this word does not allow it
-          if (!acceptCase(originalCase, wordFlags, strippedWord, offset, length)) {
+          if (!acceptCase(originalCase, entryId, strippedWord, offset, length)) {
             continue;
           }
-          if (!context.isCompound() && Dictionary.hasFlag(wordFlags, dictionary.onlyincompound)) {
+          if (!context.isCompound() && dictionary.hasFlag(entryId, dictionary.onlyincompound)) {
             continue;
           }
           if (context.isCompound()) {
             char cFlag = context.requiredFlag(dictionary);
-            if (!Dictionary.hasFlag(wordFlags, cFlag)
+            if (!dictionary.hasFlag(entryId, cFlag)
                 && !isFlagAppendedByAffix(affix, cFlag)
-                && !Dictionary.hasFlag(wordFlags, dictionary.compoundFlag)
+                && !dictionary.hasFlag(entryId, dictionary.compoundFlag)
                 && !isFlagAppendedByAffix(affix, dictionary.compoundFlag)) {
               continue;
             }
@@ -798,7 +794,7 @@ final class Stemmer {
   private boolean isFlagAppendedByAffix(int affixId, char flag) {
     if (affixId < 0 || flag == Dictionary.FLAG_UNSET) return false;
     int appendId = dictionary.affixData(affixId, Dictionary.AFFIX_APPEND);
-    return dictionary.hasFlag(appendId, flag, scratch);
+    return dictionary.hasFlag(appendId, flag);
   }
 
   private boolean isCrossProduct(int affix) {
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 8c4bc30..f2dcbe0 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
@@ -24,7 +24,6 @@ import java.nio.charset.StandardCharsets;
 import java.text.ParseException;
 import org.apache.lucene.store.ByteBuffersDirectory;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
 import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.IntsRefBuilder;
@@ -46,16 +45,20 @@ public class TestDictionary extends LuceneTestCase {
     assertNotNull(ordList);
     assertEquals(1, ordList.length);
 
-    BytesRef ref = new BytesRef();
-    char[] flags = dictionary.decodeFlags(ordList.ints[0], ref);
-    assertEquals(1, flags.length);
+    assertEquals('B', assertSingleFlag(dictionary, ordList));
 
     int offset = random().nextInt(10);
     ordList = dictionary.lookupWord((" ".repeat(offset) + "lucen").toCharArray(), offset, 5);
     assertNotNull(ordList);
     assertEquals(1, ordList.length);
-    flags = dictionary.decodeFlags(ordList.ints[0], ref);
+    assertEquals('A', assertSingleFlag(dictionary, ordList));
+  }
+
+  private static char assertSingleFlag(Dictionary dictionary, IntsRef ordList) {
+    int entryId = ordList.ints[0];
+    char[] flags = dictionary.flagLookup.getFlags(entryId);
     assertEquals(1, flags.length);
+    return flags[0];
   }
 
   public void testCompressedDictionary() throws Exception {
@@ -63,9 +66,7 @@ public class TestDictionary extends LuceneTestCase {
     assertEquals(3, dictionary.lookupSuffix(new char[] {'e'}).length);
     assertEquals(1, dictionary.lookupPrefix(new char[] {'s'}).length);
     IntsRef ordList = dictionary.lookupWord(new char[] {'o', 'l', 'r'}, 0, 3);
-    BytesRef ref = new BytesRef();
-    char[] flags = dictionary.decodeFlags(ordList.ints[0], ref);
-    assertEquals(1, flags.length);
+    assertSingleFlag(dictionary, ordList);
   }
 
   public void testCompressedBeforeSetDictionary() throws Exception {
@@ -73,9 +74,7 @@ public class TestDictionary extends LuceneTestCase {
     assertEquals(3, dictionary.lookupSuffix(new char[] {'e'}).length);
     assertEquals(1, dictionary.lookupPrefix(new char[] {'s'}).length);
     IntsRef ordList = dictionary.lookupWord(new char[] {'o', 'l', 'r'}, 0, 3);
-    BytesRef ref = new BytesRef();
-    char[] flags = dictionary.decodeFlags(ordList.ints[0], ref);
-    assertEquals(1, flags.length);
+    assertSingleFlag(dictionary, ordList);
   }
 
   public void testCompressedEmptyAliasDictionary() throws Exception {
@@ -83,9 +82,7 @@ public class TestDictionary extends LuceneTestCase {
     assertEquals(3, dictionary.lookupSuffix(new char[] {'e'}).length);
     assertEquals(1, dictionary.lookupPrefix(new char[] {'s'}).length);
     IntsRef ordList = dictionary.lookupWord(new char[] {'o', 'l', 'r'}, 0, 3);
-    BytesRef ref = new BytesRef();
-    char[] flags = dictionary.decodeFlags(ordList.ints[0], ref);
-    assertEquals(1, flags.length);
+    assertSingleFlag(dictionary, ordList);
   }
 
   // malformed rule causes ParseException
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 e26cae7..f859262b 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
@@ -59,12 +59,12 @@ public class TestPerformance extends LuceneTestCase {
 
   @Test
   public void de() throws Exception {
-    checkPerformance("de", 100_000);
+    checkPerformance("de", 200_000);
   }
 
   @Test
   public void fr() throws Exception {
-    checkPerformance("fr", 20_000);
+    checkPerformance("fr", 40_000);
   }
 
   private void checkPerformance(String code, int wordCount) throws Exception {